From ad717ef3c5cda26b88defcdb1bcb5fef2932c6d1 Mon Sep 17 00:00:00 2001 From: lgbo Date: Sun, 3 Nov 2024 19:30:07 +0800 Subject: [PATCH 001/211] [GLUTEN-7753][CORE] Do not replace literals of expand's projects in PullOutPreProject (#7756) --- .../extension/columnar/rewrite/PullOutPreProject.scala | 8 +++++++- .../org/apache/gluten/utils/PullOutProjectHelper.scala | 4 +++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala index 51cdb76a1559..ee81c4124493 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala @@ -222,7 +222,13 @@ object PullOutPreProject extends RewriteSingleNode with PullOutProjectHelper { case expand: ExpandExec if needsPreProject(expand) => val expressionMap = new mutable.HashMap[Expression, NamedExpression]() val newProjections = - expand.projections.map(_.map(replaceExpressionWithAttribute(_, expressionMap))) + expand.projections.map( + _.map( + replaceExpressionWithAttribute( + _, + expressionMap, + replaceBoundReference = false, + replaceLiteral = false))) expand.copy( projections = newProjections, child = ProjectExec( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala b/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala index 85be57493f02..e4fc11441031 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala @@ -62,7 +62,8 @@ trait PullOutProjectHelper { protected def replaceExpressionWithAttribute( expr: Expression, projectExprsMap: mutable.HashMap[Expression, NamedExpression], - replaceBoundReference: Boolean = false): Expression = + replaceBoundReference: Boolean = false, + replaceLiteral: Boolean = true): Expression = expr match { case alias: Alias => alias.child match { @@ -73,6 +74,7 @@ trait PullOutProjectHelper { } case attr: Attribute => attr case e: BoundReference if !replaceBoundReference => e + case literal: Literal if !replaceLiteral => literal case other => projectExprsMap .getOrElseUpdate(other.canonicalized, Alias(other, generatePreAliasName)()) From 8944a18b653b4472fe84a0825ebf5e3d4f053120 Mon Sep 17 00:00:00 2001 From: Zand100 Date: Sun, 3 Nov 2024 08:23:28 -0500 Subject: [PATCH 002/211] [GLUTEN-7446][BUILD] Build third party libs using jar from JAVA_HOME (#7736) --- dev/build-thirdparty.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/build-thirdparty.sh b/dev/build-thirdparty.sh index 90fc5e58fbd5..ee827ef197f7 100755 --- a/dev/build-thirdparty.sh +++ b/dev/build-thirdparty.sh @@ -82,4 +82,4 @@ elif [ "$LINUX_OS" == "debian" ]; then fi fi cd $THIRDPARTY_LIB/ -jar cvf gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar ./ +$JAVA_HOME/bin/jar cvf gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar ./ From 0bb41c742ff3adbe16263db9752dca8e64649d20 Mon Sep 17 00:00:00 2001 From: Yang Zhang Date: Sun, 3 Nov 2024 21:24:06 +0800 Subject: [PATCH 003/211] [GLUTEN-7174][VL] Force fallback scan operator when spark.sql.parquet.mergeSchema enabled (#7634) --- .../backendsapi/clickhouse/CHBackend.scala | 5 +- .../backendsapi/velox/VeloxBackend.scala | 142 ++++++++++-------- .../backendsapi/BackendSettingsApi.scala | 5 +- .../execution/BasicScanExecTransformer.scala | 2 +- .../utils/velox/VeloxTestSettings.scala | 3 - .../parquet/GlutenParquetFilterSuite.scala | 49 ------ .../utils/velox/VeloxTestSettings.scala | 6 - .../parquet/GlutenParquetFilterSuite.scala | 49 ------ .../utils/velox/VeloxTestSettings.scala | 6 - .../parquet/GlutenParquetFilterSuite.scala | 49 ------ .../utils/velox/VeloxTestSettings.scala | 6 - .../parquet/GlutenParquetFilterSuite.scala | 49 ------ 12 files changed, 86 insertions(+), 285 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 59d912d8e75d..9a1b00f71431 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -142,10 +142,11 @@ object CHBackendSettings extends BackendSettingsApi with Logging { .toLowerCase(Locale.getDefault) } - override def validateScan( + override def validateScanExec( format: ReadFileFormat, fields: Array[StructField], - rootPaths: Seq[String]): ValidationResult = { + rootPaths: Seq[String], + properties: Map[String, String]): ValidationResult = { // Validate if all types are supported. def hasComplexType: Boolean = { diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 251b93cc7cec..539059cdb6f8 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -35,12 +35,12 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Alias, CumeDist, DenseRank, Descending, Expression, Lag, Lead, NamedExpression, NthValue, NTile, PercentRank, RangeFrame, Rank, RowNumber, SortOrder, SpecialFrameBoundary, SpecifiedWindowFrame} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, ApproximatePercentile, Percentile} import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} -import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.execution.{ColumnarCachedBatchSerializer, SparkPlan} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.execution.HiveFileFormat import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -88,78 +88,94 @@ object VeloxBackendSettings extends BackendSettingsApi { val GLUTEN_VELOX_INTERNAL_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".internal.udfLibraryPaths" val GLUTEN_VELOX_UDF_ALLOW_TYPE_CONVERSION = VeloxBackend.CONF_PREFIX + ".udfAllowTypeConversion" - val MAXIMUM_BATCH_SIZE: Int = 32768 - - override def validateScan( + override def validateScanExec( format: ReadFileFormat, fields: Array[StructField], - rootPaths: Seq[String]): ValidationResult = { - val filteredRootPaths = distinctRootPaths(rootPaths) - if ( - filteredRootPaths.nonEmpty && !VeloxFileSystemValidationJniWrapper - .allSupportedByRegisteredFileSystems(filteredRootPaths.toArray) - ) { - return ValidationResult.failed( - s"Scheme of [$filteredRootPaths] is not supported by registered file systems.") - } - // Validate if all types are supported. - def validateTypes(validatorFunc: PartialFunction[StructField, String]): ValidationResult = { - // Collect unsupported types. - val unsupportedDataTypeReason = fields.collect(validatorFunc) - if (unsupportedDataTypeReason.isEmpty) { - ValidationResult.succeeded + rootPaths: Seq[String], + properties: Map[String, String]): ValidationResult = { + + def validateScheme(): Option[String] = { + val filteredRootPaths = distinctRootPaths(rootPaths) + if ( + filteredRootPaths.nonEmpty && !VeloxFileSystemValidationJniWrapper + .allSupportedByRegisteredFileSystems(filteredRootPaths.toArray) + ) { + Some(s"Scheme of [$filteredRootPaths] is not supported by registered file systems.") } else { - ValidationResult.failed( - s"Found unsupported data type in $format: ${unsupportedDataTypeReason.mkString(", ")}.") + None } } - format match { - case ParquetReadFormat => - val typeValidator: PartialFunction[StructField, String] = { - // Parquet timestamp is not fully supported yet - case StructField(_, TimestampType, _, _) - if GlutenConfig.getConf.forceParquetTimestampTypeScanFallbackEnabled => - "TimestampType" - } - validateTypes(typeValidator) - case DwrfReadFormat => ValidationResult.succeeded - case OrcReadFormat => - if (!GlutenConfig.getConf.veloxOrcScanEnabled) { - ValidationResult.failed(s"Velox ORC scan is turned off.") + def validateFormat(): Option[String] = { + def validateTypes(validatorFunc: PartialFunction[StructField, String]): Option[String] = { + // Collect unsupported types. + val unsupportedDataTypeReason = fields.collect(validatorFunc) + if (unsupportedDataTypeReason.nonEmpty) { + Some( + s"Found unsupported data type in $format: ${unsupportedDataTypeReason.mkString(", ")}.") } else { + None + } + } + + def isCharType(stringType: StringType, metadata: Metadata): Boolean = { + val charTypePattern = "char\\((\\d+)\\)".r + GlutenConfig.getConf.forceOrcCharTypeScanFallbackEnabled && charTypePattern + .findFirstIn( + CharVarcharUtils + .getRawTypeString(metadata) + .getOrElse(stringType.catalogString)) + .isDefined + } + + format match { + case ParquetReadFormat => val typeValidator: PartialFunction[StructField, String] = { - case StructField(_, arrayType: ArrayType, _, _) - if arrayType.elementType.isInstanceOf[StructType] => - "StructType as element in ArrayType" - case StructField(_, arrayType: ArrayType, _, _) - if arrayType.elementType.isInstanceOf[ArrayType] => - "ArrayType as element in ArrayType" - case StructField(_, mapType: MapType, _, _) - if mapType.keyType.isInstanceOf[StructType] => - "StructType as Key in MapType" - case StructField(_, mapType: MapType, _, _) - if mapType.valueType.isInstanceOf[ArrayType] => - "ArrayType as Value in MapType" - case StructField(_, stringType: StringType, _, metadata) - if isCharType(stringType, metadata) => - CharVarcharUtils.getRawTypeString(metadata) + " not support" - case StructField(_, TimestampType, _, _) => "TimestampType not support" + // Parquet timestamp is not fully supported yet + case StructField(_, TimestampType, _, _) + if GlutenConfig.getConf.forceParquetTimestampTypeScanFallbackEnabled => + "TimestampType(force fallback)" } - validateTypes(typeValidator) - } - case _ => ValidationResult.failed(s"Unsupported file format for $format.") + val parquetOptions = new ParquetOptions(CaseInsensitiveMap(properties), SQLConf.get) + if (parquetOptions.mergeSchema) { + // https://github.com/apache/incubator-gluten/issues/7174 + Some(s"not support when merge schema is true") + } else { + validateTypes(typeValidator) + } + case DwrfReadFormat => None + case OrcReadFormat => + if (!GlutenConfig.getConf.veloxOrcScanEnabled) { + Some(s"Velox ORC scan is turned off, ${GlutenConfig.VELOX_ORC_SCAN_ENABLED.key}") + } else { + val typeValidator: PartialFunction[StructField, String] = { + case StructField(_, arrayType: ArrayType, _, _) + if arrayType.elementType.isInstanceOf[StructType] => + "StructType as element in ArrayType" + case StructField(_, arrayType: ArrayType, _, _) + if arrayType.elementType.isInstanceOf[ArrayType] => + "ArrayType as element in ArrayType" + case StructField(_, mapType: MapType, _, _) + if mapType.keyType.isInstanceOf[StructType] => + "StructType as Key in MapType" + case StructField(_, mapType: MapType, _, _) + if mapType.valueType.isInstanceOf[ArrayType] => + "ArrayType as Value in MapType" + case StructField(_, stringType: StringType, _, metadata) + if isCharType(stringType, metadata) => + CharVarcharUtils.getRawTypeString(metadata) + "(force fallback)" + case StructField(_, TimestampType, _, _) => "TimestampType" + } + validateTypes(typeValidator) + } + case _ => Some(s"Unsupported file format for $format.") + } } - } - def isCharType(stringType: StringType, metadata: Metadata): Boolean = { - val charTypePattern = "char\\((\\d+)\\)".r - GlutenConfig.getConf.forceOrcCharTypeScanFallbackEnabled && charTypePattern - .findFirstIn( - CharVarcharUtils - .getRawTypeString(metadata) - .getOrElse(stringType.catalogString)) - .isDefined + validateScheme().orElse(validateFormat()) match { + case Some(reason) => ValidationResult.failed(reason) + case _ => ValidationResult.succeeded + } } def distinctRootPaths(paths: Seq[String]): Seq[String] = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index f1f46dd87e17..177d19c0c709 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -30,10 +30,11 @@ import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopF import org.apache.spark.sql.types.StructField trait BackendSettingsApi { - def validateScan( + def validateScanExec( format: ReadFileFormat, fields: Array[StructField], - rootPaths: Seq[String]): ValidationResult = ValidationResult.succeeded + rootPaths: Seq[String], + properties: Map[String, String]): ValidationResult = ValidationResult.succeeded def supportWriteFilesExec( format: FileFormat, diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala index d7b824b397e5..f272dc3eca72 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala @@ -95,7 +95,7 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource } val validationResult = BackendsApiManager.getSettings - .validateScan(fileFormat, fields, getRootFilePaths) + .validateScanExec(fileFormat, fields, getRootFilePaths, getProperties) if (!validationResult.ok()) { return validationResult } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 55fb4ae16d1e..363f9c85ed1d 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -861,7 +861,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") // Rewrite for supported INT96 - timestamp. @@ -875,8 +874,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. .exclude("Filter applied on merged Parquet schema with new column should work") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index 51204b0777d6..2f690c615556 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -106,54 +105,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query - // should work without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 8b56f63f65df..4e8f13ee4414 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -663,7 +663,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -679,11 +678,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -699,8 +695,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetIOSuite] diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index a1163f9525b4..02b30a46a63c 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -106,54 +105,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query should work - // without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 22a9e62c09ae..0f3c43dfdf99 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -644,7 +644,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -660,11 +659,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -680,8 +676,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetIOSuite] diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index b4a4b6017b67..a4f830e18716 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -105,54 +104,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query should work - // without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 3f6bea5dd1ce..a9525b1b0e4a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -655,7 +655,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -671,11 +670,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -691,8 +687,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetIOSuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index 063b424e0d13..3c52ec82e9fc 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -105,54 +104,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query should work - // without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ From 95170df224b5b24f638b0533245a8fd60288e354 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sun, 3 Nov 2024 21:50:46 +0800 Subject: [PATCH 004/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_03) (#7786) e5234d1a6 by Zac Wen, Use fallocate for file size extension when supported (11403) e25e7a133 by Jialiang Tan, QueryConfig documentation clean up (11355) 6c7bcd7cc by Jialiang Tan, Avoid hard coded icu4c path (11410) 8bb3bb90f by Kevin Wilfong, Fix formatting of Etc/... time zones in Presto's datetime_format and cast TSwTZ to varchar (11409) --- cpp/velox/compute/WholeStageResultIterator.cc | 4 ++-- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index 5ece7179b97f..29f467c5e761 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -494,8 +494,8 @@ std::unordered_map WholeStageResultIterator::getQueryC std::to_string(veloxCfg_->get(kSpillWriteBufferSize, 4L * 1024 * 1024)); configs[velox::core::QueryConfig::kSpillStartPartitionBit] = std::to_string(veloxCfg_->get(kSpillStartPartitionBit, 29)); - configs[velox::core::QueryConfig::kJoinSpillPartitionBits] = - std::to_string(veloxCfg_->get(kSpillPartitionBits, 2)); + configs[velox::core::QueryConfig::kSpillNumPartitionBits] = + std::to_string(veloxCfg_->get(kSpillPartitionBits, 3)); configs[velox::core::QueryConfig::kSpillableReservationGrowthPct] = std::to_string(veloxCfg_->get(kSpillableReservationGrowthPct, 25)); configs[velox::core::QueryConfig::kSpillCompressionKind] = diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 6d53466b07e7..9f430ba999bc 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_02 +VELOX_BRANCH=2024_11_03 VELOX_HOME="" OS=`uname -s` From 78d3604b6acf7c137456de4086a4484514dca925 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Mon, 4 Nov 2024 10:13:22 +0800 Subject: [PATCH 005/211] [GLUTEN-7782] Fix profile Darwin-x86 os.arch errorr (#7783) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 69a0a17872cb..d3c67f231e9e 100644 --- a/pom.xml +++ b/pom.xml @@ -534,7 +534,7 @@ mac - x86 + x86_64 From c40735bcb34b7f7fbd9b8c5930eb947011cb611a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Mon, 4 Nov 2024 10:16:41 +0800 Subject: [PATCH 006/211] [GLUTEN-7780][CH] Fix split diff (#7781) * fix split diff * fix code style * fix code style --- .../execution/GlutenClickHouseTPCHSuite.scala | 7 + .../Functions/SparkFunctionSplitByRegexp.cpp | 239 ++++++++++++++++++ .../Parser/scalar_function_parser/split.cpp | 12 +- .../clickhouse/ClickHouseTestSettings.scala | 4 - .../clickhouse/ClickHouseTestSettings.scala | 4 - .../clickhouse/ClickHouseTestSettings.scala | 4 - .../clickhouse/ClickHouseTestSettings.scala | 4 - 7 files changed, 252 insertions(+), 22 deletions(-) create mode 100644 cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala index a56f45d1ba3d..8dc178e46ce5 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala @@ -563,5 +563,12 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { compareResultsAgainstVanillaSpark(sql, true, { _ => }) spark.sql("drop table t1") } + + test("GLUTEN-7780 fix split diff") { + val sql = "select split(concat('a|b|c', cast(id as string)), '\\|')" + + ", split(concat('a|b|c', cast(id as string)), '\\\\|')" + + ", split(concat('a|b|c', cast(id as string)), '|') from range(10)" + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } } // scalastyle:off line.size.limit diff --git a/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp b/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp new file mode 100644 index 000000000000..66f37c62033f --- /dev/null +++ b/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByRegexp(regexp, s[, max_substrings]) + */ +namespace +{ + +using Pos = const char *; + +class SparkSplitByRegexpImpl +{ +private: + Regexps::RegexpPtr re; + OptimizedRegularExpression::MatchVec matches; + + Pos pos; + Pos end; + + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = "splitByRegexpSpark"; + + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } + + static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {0, 2}; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 1uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " + "Must be constant string.", arguments[0].column->getName(), name); + + if (!col->getValue().empty()) + re = std::make_shared(Regexps::createRegexp(col->getValue())); + + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 2); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + if (!re) + { + if (pos == end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = end; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + ++pos; + token_end = pos; + ++splits; + } + else + { + if (!pos || pos > end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = nullptr; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + auto res = re->match(pos, end - pos, matches); + if (!res) + { + token_end = end; + pos = end + 1; + } + else if (!matches[0].length) + { + /// If match part is empty, increment position to avoid infinite loop. + token_end = (pos == end ? end : pos + 1); + ++pos; + ++splits; + } + else + { + token_end = pos + matches[0].offset; + pos = token_end + matches[0].length; + ++splits; + } + } + + return true; + } +}; + +using SparkFunctionSplitByRegexp = FunctionTokens; + +/// Fallback splitByRegexp to splitByChar when its 1st argument is a trivial char for better performance +class SparkSplitByRegexpOverloadResolver : public IFunctionOverloadResolver +{ +public: + static constexpr auto name = "splitByRegexpSpark"; + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } + + explicit SparkSplitByRegexpOverloadResolver(ContextPtr context_) + : context(context_) + , split_by_regexp(SparkFunctionSplitByRegexp::create(context)) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return SparkSplitByRegexpImpl::getNumberOfArguments(); } + bool isVariadic() const override { return SparkSplitByRegexpImpl::isVariadic(); } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + if (patternIsTrivialChar(arguments)) + return FunctionFactory::instance().getImpl("splitByChar", context)->build(arguments); + return std::make_unique( + split_by_regexp, collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + return split_by_regexp->getReturnTypeImpl(arguments); + } + +private: + bool patternIsTrivialChar(const ColumnsWithTypeAndName & arguments) const + { + if (!arguments[0].column.get()) + return false; + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + if (!col) + return false; + + String pattern = col->getValue(); + if (pattern.size() == 1) + { + OptimizedRegularExpression re = Regexps::createRegexp(pattern); + + std::string required_substring; + bool is_trivial; + bool required_substring_is_prefix; + re.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + return is_trivial && required_substring == pattern; + } + return false; + } + + ContextPtr context; + FunctionPtr split_by_regexp; +}; +} + +REGISTER_FUNCTION(SparkSplitByRegexp) +{ + factory.registerFunction(); +} + +} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp index ed17c27eade9..3ffd64decb92 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp @@ -19,14 +19,14 @@ namespace local_engine { -class SparkFunctionSplitParser : public FunctionParser +class FunctionSplitParser : public FunctionParser { public: - SparkFunctionSplitParser(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} - ~SparkFunctionSplitParser() override = default; + FunctionSplitParser(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + ~FunctionSplitParser() override = default; static constexpr auto name = "split"; String getName() const override { return name; } - String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "splitByRegexp"; } + String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "splitByRegexpSpark"; } const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { @@ -35,7 +35,7 @@ class SparkFunctionSplitParser : public FunctionParser for (const auto & arg : args) parsed_args.emplace_back(parseExpression(actions_dag, arg.value())); /// In Spark: split(str, regex [, limit] ) - /// In CH: splitByRegexp(regexp, str [, limit]) + /// In CH: splitByRegexpSpark(regexp, str [, limit]) if (parsed_args.size() >= 2) std::swap(parsed_args[0], parsed_args[1]); auto ch_function_name = getCHFunctionName(substrait_func); @@ -43,6 +43,6 @@ class SparkFunctionSplitParser : public FunctionParser return convertNodeTypeIfNeeded(substrait_func, func_node, actions_dag); } }; -static FunctionParserRegister register_split; +static FunctionParserRegister register_split; } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 27e26606f653..50110f15d457 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -844,8 +844,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -854,8 +852,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK-34814: LikeSimplification should handle NULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index da950e2fc1ee..9b3b090e326d 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -817,8 +817,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -827,8 +825,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK - 34814: LikeSimplification should handleNULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index ac08fc5a80cc..e91f1495fbe9 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -740,8 +740,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -750,8 +748,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK - 34814: LikeSimplification should handleNULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 9e4c81081de1..f0637839a762 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -740,8 +740,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -750,8 +748,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK - 34814: LikeSimplification should handleNULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") From eca5103fe2427380686385698c20dde1f1aad6b7 Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Mon, 4 Nov 2024 11:02:18 +0800 Subject: [PATCH 007/211] [GLUTEN-7792][CH] Set default minio ak/sk to minioadmin (#7793) --- .../GlutenClickHouseWholeStageTransformerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index 0bd19dd97172..272323c48732 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -53,8 +53,8 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu val HDFS_URL_ENDPOINT = "hdfs://127.0.0.1:8020" val HDFS_URL = s"$HDFS_URL_ENDPOINT/$SPARK_DIR_NAME" - val S3_ACCESS_KEY = "BypTYzcXOlfr03FFIvt4" - val S3_SECRET_KEY = "K9MDaGItPSaphorZM8t4hXf30gHF9dBWi6L2dK5E" + val S3_ACCESS_KEY = "minioadmin" + val S3_SECRET_KEY = "minioadmin" val CH_DEFAULT_STORAGE_DIR = "/data" From 2bf912bd6a995c8f7979bfafbd6556bff6729f19 Mon Sep 17 00:00:00 2001 From: Yuan Date: Sun, 3 Nov 2024 20:24:32 -0800 Subject: [PATCH 008/211] [DOC] Update release plan for Velox backend (#7744) * Update release plan for Velox backend Signed-off-by: Yuan Zhou --------- Signed-off-by: Yuan Zhou --- docs/release.md | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/docs/release.md b/docs/release.md index a3f20bde857e..99661c07c9a3 100644 --- a/docs/release.md +++ b/docs/release.md @@ -6,10 +6,18 @@ nav_order: 11 [Gluten](https://github.com/apache/incubator-gluten) is a plugin for Apache Spark to double SparkSQL's performance. -## Latest release for velox backend -* [Gluten-1.1.1](https://github.com/apache/incubator-gluten/releases/tag/v1.1.1) (Mar. 2 2024) +## Latest release for Velox backend +* [Gluten-1.2.0](https://github.com/apache/incubator-gluten/releases/tag/v1.2.0) (Sep. 3 2024) + +## Planned release for Velox backend +* [Gluten-1.2.1] (Nov. 30 2024) +* [Gluten-1.3.0] (Dec. 30 2024) +* [Gluten-1.3.1] (Jan. 30 2025) +* [Gluten-1.4.0] (Mar. 30 2025) +* To be updated ## Archived releases +* [Gluten-1.1.1](https://github.com/apache/incubator-gluten/releases/tag/v1.1.1) (Mar. 2 2024) * [Gluten-1.1.0](https://github.com/apache/incubator-gluten/releases/tag/v1.1.0) (Nov. 30 2023) * [Gluten-1.0.0](https://github.com/apache/incubator-gluten/releases/tag/v1.0.0) (Jul. 13 2023) * [Gluten-0.5.0](https://github.com/apache/incubator-gluten/releases/tag/0.5.0) (Apr. 7 2023) From a053338bf352d05e664dce014c80e006268327a0 Mon Sep 17 00:00:00 2001 From: Jiaan Geng Date: Mon, 4 Nov 2024 14:02:10 +0800 Subject: [PATCH 009/211] [GLUTEN-7727][CORE] Unify the the variable name of GlutenConfig with glutenConf (#7728) --- .../backendsapi/clickhouse/CHRuleApi.scala | 10 ++-- .../FallbackBroadcastHashJoinRules.scala | 6 +- .../MergeTwoPhasesHashBaseAggregate.scala | 6 +- .../backendsapi/velox/VeloxRuleApi.scala | 8 +-- .../columnar/ColumnarRuleApplier.scala | 2 +- .../extension/columnar/FallbackRules.scala | 6 +- .../columnar/validator/Validators.scala | 57 ++++++++++--------- .../ColumnarCollapseTransformStages.scala | 2 +- .../execution/GlutenFallbackReporter.scala | 6 +- 9 files changed, 54 insertions(+), 49 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 91698d4cde85..dea0d50c9da6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -86,7 +86,8 @@ private object CHRuleApi { injector.injectTransform( c => intercept( - SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarTransformRules)(c.session))) + SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarTransformRules)( + c.session))) injector.injectTransform(c => InsertTransitions(c.outputsColumnar)) // Gluten columnar: Fallback policies. @@ -98,14 +99,15 @@ private object CHRuleApi { SparkShimLoader.getSparkShims .getExtendedColumnarPostRules() .foreach(each => injector.injectPost(c => intercept(each(c.session)))) - injector.injectPost(c => ColumnarCollapseTransformStages(c.conf)) + injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf)) injector.injectTransform( c => - intercept(SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarPostRules)(c.session))) + intercept( + SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarPostRules)(c.session))) // Gluten columnar: Final rules. injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session)) - injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session)) + injector.injectFinal(c => GlutenFallbackReporter(c.glutenConf, c.session)) injector.injectFinal(_ => RemoveFallbackTagRule()) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala index ec465a3c1506..207bb0e3a4d7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala @@ -38,7 +38,7 @@ import scala.util.control.Breaks.{break, breakable} // queryStagePrepRules. case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { - val columnarConf: GlutenConfig = GlutenConfig.getConf + val glutenConf: GlutenConfig = GlutenConfig.getConf plan.foreach { case bhj: BroadcastHashJoinExec => val buildSidePlan = bhj.buildSide match { @@ -53,8 +53,8 @@ case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extend case Some(exchange @ BroadcastExchangeExec(mode, child)) => val isTransformable = if ( - !columnarConf.enableColumnarBroadcastExchange || - !columnarConf.enableColumnarBroadcastJoin + !glutenConf.enableColumnarBroadcastExchange || + !glutenConf.enableColumnarBroadcastJoin ) { ValidationResult.failed( "columnar broadcast exchange is disabled or " + diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala index 63c5fe017f5e..a10659b6d5e7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala @@ -38,9 +38,9 @@ case class MergeTwoPhasesHashBaseAggregate(session: SparkSession) extends Rule[SparkPlan] with Logging { - val columnarConf: GlutenConfig = GlutenConfig.getConf - val scanOnly: Boolean = columnarConf.enableScanOnly - val enableColumnarHashAgg: Boolean = !scanOnly && columnarConf.enableColumnarHashAgg + val glutenConf: GlutenConfig = GlutenConfig.getConf + val scanOnly: Boolean = glutenConf.enableScanOnly + val enableColumnarHashAgg: Boolean = !scanOnly && glutenConf.enableColumnarHashAgg val replaceSortAggWithHashAgg: Boolean = GlutenConfig.getConf.forceToUseHashAgg private def isPartialAgg(partialAgg: BaseAggregateExec, finalAgg: BaseAggregateExec): Boolean = { diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index a838c463c390..3554bc5c9c01 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -80,11 +80,11 @@ private object VeloxRuleApi { SparkShimLoader.getSparkShims .getExtendedColumnarPostRules() .foreach(each => injector.injectPost(c => each(c.session))) - injector.injectPost(c => ColumnarCollapseTransformStages(c.conf)) + injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf)) // Gluten columnar: Final rules. injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session)) - injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session)) + injector.injectFinal(c => GlutenFallbackReporter(c.glutenConf, c.session)) injector.injectFinal(_ => RemoveFallbackTagRule()) } @@ -116,9 +116,9 @@ private object VeloxRuleApi { SparkShimLoader.getSparkShims .getExtendedColumnarPostRules() .foreach(each => injector.inject(c => each(c.session))) - injector.inject(c => ColumnarCollapseTransformStages(c.conf)) + injector.inject(c => ColumnarCollapseTransformStages(c.glutenConf)) injector.inject(c => RemoveGlutenTableCacheColumnarToRow(c.session)) - injector.inject(c => GlutenFallbackReporter(c.conf, c.session)) + injector.inject(c => GlutenFallbackReporter(c.glutenConf, c.session)) injector.inject(_ => RemoveFallbackTagRule()) } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala index ecf13967e3ef..bf7b84c9b316 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala @@ -31,7 +31,7 @@ object ColumnarRuleApplier { val session: SparkSession, val ac: AdaptiveContext, val outputsColumnar: Boolean) { - val conf: GlutenConfig = { + val glutenConf: GlutenConfig = { new GlutenConfig(session.sessionState.conf) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala index a5bba46dc605..794186bfa957 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala @@ -169,9 +169,9 @@ case class FallbackOnANSIMode(session: SparkSession) extends Rule[SparkPlan] { } case class FallbackMultiCodegens(session: SparkSession) extends Rule[SparkPlan] { - lazy val columnarConf: GlutenConfig = GlutenConfig.getConf - lazy val physicalJoinOptimize = columnarConf.enablePhysicalJoinOptimize - lazy val optimizeLevel: Integer = columnarConf.physicalJoinOptimizationThrottle + lazy val glutenConf: GlutenConfig = GlutenConfig.getConf + lazy val physicalJoinOptimize = glutenConf.enablePhysicalJoinOptimize + lazy val optimizeLevel: Integer = glutenConf.physicalJoinOptimizationThrottle def existsMultiCodegens(plan: SparkPlan, count: Int = 0): Boolean = plan match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala index 50201efc07d2..898a2dbae824 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala @@ -149,47 +149,50 @@ object Validators { } } - private class FallbackByUserOptions(conf: GlutenConfig) extends Validator { + private class FallbackByUserOptions(glutenConf: GlutenConfig) extends Validator { override def validate(plan: SparkPlan): Validator.OutCome = plan match { - case p: SortExec if !conf.enableColumnarSort => fail(p) - case p: WindowExec if !conf.enableColumnarWindow => fail(p) - case p: SortMergeJoinExec if !conf.enableColumnarSortMergeJoin => fail(p) - case p: BatchScanExec if !conf.enableColumnarBatchScan => fail(p) - case p: FileSourceScanExec if !conf.enableColumnarFileScan => fail(p) - case p: ProjectExec if !conf.enableColumnarProject => fail(p) - case p: FilterExec if !conf.enableColumnarFilter => fail(p) - case p: UnionExec if !conf.enableColumnarUnion => fail(p) - case p: ExpandExec if !conf.enableColumnarExpand => fail(p) - case p: SortAggregateExec if !conf.forceToUseHashAgg => fail(p) - case p: ShuffledHashJoinExec if !conf.enableColumnarShuffledHashJoin => fail(p) - case p: ShuffleExchangeExec if !conf.enableColumnarShuffle => fail(p) - case p: BroadcastExchangeExec if !conf.enableColumnarBroadcastExchange => fail(p) - case p @ (_: LocalLimitExec | _: GlobalLimitExec) if !conf.enableColumnarLimit => fail(p) - case p: GenerateExec if !conf.enableColumnarGenerate => fail(p) - case p: CoalesceExec if !conf.enableColumnarCoalesce => fail(p) - case p: CartesianProductExec if !conf.cartesianProductTransformerEnabled => fail(p) + case p: SortExec if !glutenConf.enableColumnarSort => fail(p) + case p: WindowExec if !glutenConf.enableColumnarWindow => fail(p) + case p: SortMergeJoinExec if !glutenConf.enableColumnarSortMergeJoin => fail(p) + case p: BatchScanExec if !glutenConf.enableColumnarBatchScan => fail(p) + case p: FileSourceScanExec if !glutenConf.enableColumnarFileScan => fail(p) + case p: ProjectExec if !glutenConf.enableColumnarProject => fail(p) + case p: FilterExec if !glutenConf.enableColumnarFilter => fail(p) + case p: UnionExec if !glutenConf.enableColumnarUnion => fail(p) + case p: ExpandExec if !glutenConf.enableColumnarExpand => fail(p) + case p: SortAggregateExec if !glutenConf.forceToUseHashAgg => fail(p) + case p: ShuffledHashJoinExec if !glutenConf.enableColumnarShuffledHashJoin => fail(p) + case p: ShuffleExchangeExec if !glutenConf.enableColumnarShuffle => fail(p) + case p: BroadcastExchangeExec if !glutenConf.enableColumnarBroadcastExchange => fail(p) + case p @ (_: LocalLimitExec | _: GlobalLimitExec) if !glutenConf.enableColumnarLimit => + fail(p) + case p: GenerateExec if !glutenConf.enableColumnarGenerate => fail(p) + case p: CoalesceExec if !glutenConf.enableColumnarCoalesce => fail(p) + case p: CartesianProductExec if !glutenConf.cartesianProductTransformerEnabled => fail(p) case p: TakeOrderedAndProjectExec - if !(conf.enableTakeOrderedAndProject && conf.enableColumnarSort && - conf.enableColumnarShuffle && conf.enableColumnarProject) => + if !(glutenConf.enableTakeOrderedAndProject && glutenConf.enableColumnarSort && + glutenConf.enableColumnarShuffle && glutenConf.enableColumnarProject) => fail(p) - case p: BroadcastHashJoinExec if !conf.enableColumnarBroadcastJoin => + case p: BroadcastHashJoinExec if !glutenConf.enableColumnarBroadcastJoin => fail(p) case p: BroadcastNestedLoopJoinExec - if !(conf.enableColumnarBroadcastJoin && - conf.broadcastNestedLoopJoinTransformerTransformerEnabled) => + if !(glutenConf.enableColumnarBroadcastJoin && + glutenConf.broadcastNestedLoopJoinTransformerTransformerEnabled) => fail(p) case p @ (_: HashAggregateExec | _: SortAggregateExec | _: ObjectHashAggregateExec) - if !conf.enableColumnarHashAgg => + if !glutenConf.enableColumnarHashAgg => fail(p) case p if SparkShimLoader.getSparkShims.isWindowGroupLimitExec( - plan) && !conf.enableColumnarWindowGroupLimit => + plan) && !glutenConf.enableColumnarWindowGroupLimit => fail(p) case p - if HiveTableScanExecTransformer.isHiveTableScan(p) && !conf.enableColumnarHiveTableScan => + if HiveTableScanExecTransformer.isHiveTableScan( + p) && !glutenConf.enableColumnarHiveTableScan => fail(p) case p: SampleExec - if !(conf.enableColumnarSample && BackendsApiManager.getSettings.supportSampleExec()) => + if !(glutenConf.enableColumnarSample && BackendsApiManager.getSettings + .supportSampleExec()) => fail(p) case _ => pass() } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index ada7283da8d5..32575e4f13d5 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -114,7 +114,7 @@ case class InputIteratorTransformer(child: SparkPlan) extends UnaryTransformSupp * generate/compile code. */ case class ColumnarCollapseTransformStages( - glutenConfig: GlutenConfig, + glutenConf: GlutenConfig, transformStageCounter: AtomicInteger = ColumnarCollapseTransformStages.transformStageCounter) extends Rule[SparkPlan] { diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala index f6e23e7cff67..481e16b0a5be 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala @@ -31,12 +31,12 @@ import org.apache.spark.sql.execution.ui.GlutenEventUtils * This rule is used to collect all fallback reason. * 1. print fallback reason for each plan node 2. post all fallback reason using one event */ -case class GlutenFallbackReporter(glutenConfig: GlutenConfig, spark: SparkSession) +case class GlutenFallbackReporter(glutenConf: GlutenConfig, spark: SparkSession) extends Rule[SparkPlan] with LogLevelUtil { override def apply(plan: SparkPlan): SparkPlan = { - if (!glutenConfig.enableFallbackReport) { + if (!glutenConf.enableFallbackReport) { return plan } printFallbackReason(plan) @@ -52,7 +52,7 @@ case class GlutenFallbackReporter(glutenConfig: GlutenConfig, spark: SparkSessio } private def printFallbackReason(plan: SparkPlan): Unit = { - val validationLogLevel = glutenConfig.validationLogLevel + val validationLogLevel = glutenConf.validationLogLevel plan.foreachUp { case _: GlutenPlan => // ignore case p: SparkPlan if FallbackTags.nonEmpty(p) => From a98193f9e1a7dc0a3ea2443dcd6c9685f27c868b Mon Sep 17 00:00:00 2001 From: exmy Date: Mon, 4 Nov 2024 14:26:41 +0800 Subject: [PATCH 010/211] [GLUTEN-7700][CH] Fix issue when partition values contain space (#7719) --- .../hive/GlutenClickHouseHiveTableSuite.scala | 31 +++++++++++++++++++ .../Storages/Output/WriteBufferBuilder.cpp | 15 +++++---- 2 files changed, 40 insertions(+), 6 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index 4e2b5ad63e0a..8d311614c7de 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -542,6 +542,37 @@ class GlutenClickHouseHiveTableSuite ) } + test("GLUTEN-7700: test hive table with partition values contain space") { + val tbl = "test_7700" + val create_table_sql = + s""" + |create table if not exists $tbl ( + | id int + |) partitioned by (itime string) + |stored as orc; + |""".stripMargin + val insert_sql = + s""" + |insert overwrite table $tbl partition (itime = '2024-10-24 10:02:04') + |select id from range(3) + |""".stripMargin + val select_sql = + s""" + |select * from $tbl + |""".stripMargin + val drop_sql = s"drop table if exists $tbl" + + spark.sql(create_table_sql) + spark.sql(insert_sql) + + compareResultsAgainstVanillaSpark( + select_sql, + compareResult = true, + df => assert(df.count() == 3) + ) + spark.sql(drop_sql) + } + test("test hive compressed txt table") { withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> "11") { Seq("DefaultCodec", "BZip2Codec").foreach { diff --git a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp index 6926b86a34e0..c03b1918d271 100644 --- a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp @@ -47,7 +47,7 @@ class LocalFileWriteBufferBuilder : public WriteBufferBuilder Poco::URI file_uri(file_uri_); const String & file_path = file_uri.getPath(); - //mkdir + // mkdir std::filesystem::path p(file_path); if (!std::filesystem::exists(p.parent_path())) std::filesystem::create_directories(p.parent_path()); @@ -78,16 +78,19 @@ class HDFSFileWriteBufferBuilder : public WriteBufferBuilder auto builder = DB::createHDFSBuilder(new_file_uri, context->getConfigRef()); auto fs = DB::createHDFSFS(builder.get()); + auto begin_of_path = new_file_uri.find('/', new_file_uri.find("//") + 2); - auto last = new_file_uri.find_last_of('/'); - auto dir = new_file_uri.substr(begin_of_path, last - begin_of_path); + auto url_without_path = new_file_uri.substr(0, begin_of_path); + + // use uri.getPath() instead of new_file_uri.substr(begin_of_path) to avoid space character uri-encoded + std::filesystem::path file_path(uri.getPath()); + auto dir = file_path.parent_path().string(); + if (hdfsCreateDirectory(fs.get(), dir.c_str())) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create dir for {} because {}", dir, std::string(hdfsGetLastError())); - const std::string hdfs_file_path = new_file_uri.substr(begin_of_path); - const std::string hdfs_uri_without_path = new_file_uri.substr(0, begin_of_path); DB::WriteSettings write_settings; - return std::make_unique(hdfs_uri_without_path, hdfs_file_path, context->getConfigRef(), 0, write_settings); + return std::make_unique(url_without_path, file_path.string(), context->getConfigRef(), 0, write_settings); } }; #endif From 48d312a58e8554b5a0af0b998051d6217870415f Mon Sep 17 00:00:00 2001 From: Yuan Date: Sun, 3 Nov 2024 23:00:27 -0800 Subject: [PATCH 011/211] [GLUTEN-7741][VL] refine build package tool (#7742) Refine the package script fixes #7741 Signed-off-by: Yuan Zhou --- .github/workflows/build_bundle_package.yml | 103 +++------------------ 1 file changed, 15 insertions(+), 88 deletions(-) diff --git a/.github/workflows/build_bundle_package.yml b/.github/workflows/build_bundle_package.yml index 3afe5b5e02a8..d4f7046f2b0c 100644 --- a/.github/workflows/build_bundle_package.yml +++ b/.github/workflows/build_bundle_package.yml @@ -17,6 +17,7 @@ name: Build bundle package env: ACTIONS_ALLOW_USE_UNSECURE_NODE_VERSION: true + CCACHE_DIR: "${{ github.workspace }}/.ccache" concurrency: group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} @@ -25,10 +26,6 @@ concurrency: on: workflow_dispatch: inputs: - os: - description: 'OS version: ubuntu:20.04, ubuntu:22.04, centos:7 or centos:8' - required: true - default: 'ubuntu:20.04' spark: description: 'Spark version: spark-3.2, spark-3.3, spark-3.4 or spark-3.5' required: true @@ -41,19 +38,22 @@ on: jobs: build-native-lib: runs-on: ubuntu-20.04 - container: apache/gluten:gluten-vcpkg-builder_2024_05_29 + container: apache/gluten:vcpkg-centos-7 steps: - uses: actions/checkout@v2 + - name: Get Ccache + uses: actions/cache/restore@v3 + with: + path: '${{ env.CCACHE_DIR }}' + key: ccache-centos7-release-default-${{github.sha}} + restore-keys: | + ccache-centos7-release-default - name: Build Gluten velox third party run: | - yum install sudo patch java-1.8.0-openjdk-devel -y && \ - cd $GITHUB_WORKSPACE/ep/build-velox/src && \ - ./get_velox.sh && \ - source /opt/rh/devtoolset-11/enable && \ - cd $GITHUB_WORKSPACE/ && \ - export NUM_THREADS=4 - ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --build_tests=OFF --build_benchmarks=OFF --enable_s3=OFF \ - --enable_gcs=OFF --enable_hdfs=ON --enable_abfs=OFF + df -a + yum install ccache -y + cd $GITHUB_WORKSPACE/ + bash dev/ci-velox-buildstatic-centos-7.sh - name: Upload native libs uses: actions/upload-artifact@v2 with: @@ -66,44 +66,10 @@ jobs: path: /root/.m2/repository/org/apache/arrow/ name: velox-arrow-jar-centos-7-${{github.sha}} - build-bundle-package-ubuntu: - if: startsWith(github.event.inputs.os, 'ubuntu') - needs: build-native-lib - runs-on: ubuntu-20.04 - container: ${{ github.event.inputs.os }} - steps: - - uses: actions/checkout@v2 - - name: Download All Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-native-lib-${{github.sha}} - path: ./cpp/build/releases - - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-arrow-jar-centos-7-${{github.sha}} - path: /root/.m2/repository/org/apache/arrow/ - - name: Setup java and maven - run: | - apt-get update && \ - apt-get install -y openjdk-8-jdk maven && \ - apt remove openjdk-11* -y - - name: Build for Spark ${{ github.event.inputs.spark }} - run: | - cd $GITHUB_WORKSPACE/ && \ - mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -DskipTests -Dmaven.source.skip - - name: Upload bundle package - uses: actions/upload-artifact@v2 - with: - name: gluten-velox-bundle-package - path: package/target/gluten-velox-bundle-*.jar - retention-days: 7 - build-bundle-package-centos7: - if: ${{ github.event.inputs.os == 'centos:7' }} needs: build-native-lib runs-on: ubuntu-20.04 - container: ${{ github.event.inputs.os }} + container: centos:7 steps: - uses: actions/checkout@v2 - name: Download All Artifacts @@ -127,7 +93,7 @@ jobs: cd $GITHUB_WORKSPACE/ && \ export MAVEN_HOME=/usr/lib/maven && \ export PATH=${PATH}:${MAVEN_HOME}/bin && \ - mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -DskipTests -Dmaven.source.skip + mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -Puniffle -DskipTests -Dmaven.source.skip - name: Upload bundle package uses: actions/upload-artifact@v2 with: @@ -135,42 +101,3 @@ jobs: path: package/target/gluten-velox-bundle-*.jar retention-days: 7 - build-bundle-package-centos8: - if: ${{ github.event.inputs.os == 'centos:8' }} - needs: build-native-lib - runs-on: ubuntu-20.04 - container: ${{ github.event.inputs.os }} - steps: - - uses: actions/checkout@v2 - - name: Download All Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-native-lib-${{github.sha}} - path: ./cpp/build/releases - - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-arrow-jar-centos-7-${{github.sha}} - path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true && \ - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup java and maven - run: | - yum update -y && yum install -y java-1.8.0-openjdk-devel wget && \ - wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz && \ - tar -xvf apache-maven-3.8.8-bin.tar.gz && \ - mv apache-maven-3.8.8 /usr/lib/maven - - name: Build for Spark ${{ github.event.inputs.spark }} - run: | - cd $GITHUB_WORKSPACE/ && \ - export MAVEN_HOME=/usr/lib/maven && \ - export PATH=${PATH}:${MAVEN_HOME}/bin && \ - mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -DskipTests -Dmaven.source.skip - - name: Upload bundle package - uses: actions/upload-artifact@v2 - with: - name: gluten-velox-bundle-package - path: package/target/gluten-velox-bundle-*.jar - retention-days: 7 From 5dfbe5258b2b0ff17097ed49b1b63346416e76ba Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Mon, 4 Nov 2024 21:56:02 +0800 Subject: [PATCH 012/211] [GLUTEN-7797][VL] Fix missing icu lib on centos-7 (#7798) --- ep/build-velox/src/setup-centos7.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/setup-centos7.sh b/ep/build-velox/src/setup-centos7.sh index 87150634134b..45880161a4a5 100755 --- a/ep/build-velox/src/setup-centos7.sh +++ b/ep/build-velox/src/setup-centos7.sh @@ -242,7 +242,7 @@ dnf_install epel-release dnf-plugins-core # For ccache, ninja dnf_install ccache wget which libevent-devel \ yasm \ openssl-devel libzstd-devel lz4-devel double-conversion-devel \ - curl-devel libxml2-devel libgsasl-devel libuuid-devel patch + curl-devel libxml2-devel libgsasl-devel libuuid-devel patch libicu-devel $SUDO dnf remove -y gflags From 284cf714c5676bf3cf8b07ea6c361b263ee12afc Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 5 Nov 2024 07:46:41 +0800 Subject: [PATCH 013/211] [VL] Remove a duplicated Maven dependency, and some follow-ups for #7764 (#7773) --- backends-velox/pom.xml | 6 ------ .../org/apache/spark/sql/gluten/GlutenFallbackSuite.scala | 5 +++++ .../org/apache/spark/sql/gluten/GlutenFallbackSuite.scala | 5 +++++ 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/backends-velox/pom.xml b/backends-velox/pom.xml index 7cab49b25550..e6900e50dd70 100755 --- a/backends-velox/pom.xml +++ b/backends-velox/pom.xml @@ -72,12 +72,6 @@ spark-hive_${scala.binary.version} provided - - org.apache.gluten - gluten-substrait - ${project.version} - compile - org.apache.gluten gluten-arrow diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala index c2446e38d75f..4a73a8dc72de 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala @@ -21,6 +21,7 @@ import org.apache.gluten.events.GlutenPlanFallbackEvent import org.apache.gluten.execution.FileSourceScanExecTransformer import org.apache.gluten.utils.BackendTestUtils +import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} import org.apache.spark.sql.execution.ProjectExec @@ -32,6 +33,10 @@ import org.apache.spark.status.ElementTrackingStore import scala.collection.mutable.ArrayBuffer class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelper { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.RAS_ENABLED.key, "false") + } testGluten("test fallback logging") { val testAppender = new LogAppender("fallback reason") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala index d7ec1f0fa57e..7855f289a707 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala @@ -21,6 +21,7 @@ import org.apache.gluten.events.GlutenPlanFallbackEvent import org.apache.gluten.execution.FileSourceScanExecTransformer import org.apache.gluten.utils.BackendTestUtils +import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} import org.apache.spark.sql.execution.ProjectExec @@ -32,6 +33,10 @@ import org.apache.spark.status.ElementTrackingStore import scala.collection.mutable.ArrayBuffer class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelper { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.RAS_ENABLED.key, "false") + } testGluten("test fallback logging") { val testAppender = new LogAppender("fallback reason") From 71b764500144962dddbef6939d38fc1e4881185f Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 5 Nov 2024 09:28:45 +0800 Subject: [PATCH 014/211] [GLUTEN-7143][VL] RAS: Enable the RAS UT jobs in GHA CI (#7770) --- .github/workflows/velox_backend.yml | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index ba5d4f96ff14..6e9d0eab7c70 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -993,16 +993,14 @@ jobs: pip3 install setuptools && \ pip3 install pyspark==3.5.3 cython && \ pip3 install pandas pyarrow - - name: (To be fixed) Build and Run unit test for Spark 3.5.3 (other tests) - continue-on-error: true + - name: Build and Run unit test for Spark 3.5.3 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.12 $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/ -Dspark.gluten.ras.enabled=true" \ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags - - name: (To be enabled) Upload test report - if: false + - name: Upload test report uses: actions/upload-artifact@v4 with: name: test-report-spark35-ras @@ -1035,15 +1033,13 @@ jobs: - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 - - name: (To be fixed) Build and Run unit test for Spark 3.5.3 (slow tests) - continue-on-error: true + - name: Build and Run unit test for Spark 3.5.3 (slow tests) run: | cd $GITHUB_WORKSPACE/ $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/ -Dspark.gluten.ras.enabled=true" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest - - name: (To be enabled) Upload test report - if: false + - name: Upload test report uses: actions/upload-artifact@v4 with: name: test-report-spark35-slow-ras From 8b22ceddb39a562980c8fc3544643f159f93be34 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 5 Nov 2024 11:43:30 +0800 Subject: [PATCH 015/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_05) (#7808) Upstream Velox's New Commits: c95f1e023 by Richard Barnes, Remove unused-variable in velox/buffer/tests/BufferTest.cpp +8 (11421) 7fde3c6b6 by Kevin Wilfong, Add support for (UT/UTC/GMT)(+/-)H[H] time zone IDs in from_unixtime and at_timezone Presto UDFs (11411) f02524b97 by Pedro Eugenio Rocha Pedreira, refactor: Cosmetic changes to PlanBuilder and IExpr (11413) 03a317398 by zuyu, Cleanup velox/tool/trace/CMakeLists.txt (11406) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 9f430ba999bc..16467f938da0 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_03 +VELOX_BRANCH=2024_11_05 VELOX_HOME="" OS=`uname -s` From 3108d91f5fec877176d3c5e4a206d43341138879 Mon Sep 17 00:00:00 2001 From: Jaime Pan <33685703+NEUpanning@users.noreply.github.com> Date: Tue, 5 Nov 2024 12:55:44 +0800 Subject: [PATCH 016/211] [VL] In `ColumnarBatchSerializerJniWrapper_serialize`, check if the byte array is constructed successfully (#7733) --- cpp/core/jni/JniWrapper.cc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 45f19c25c749..6a0a5b0057f7 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -1079,6 +1079,10 @@ JNIEXPORT jobject JNICALL Java_org_apache_gluten_vectorized_ColumnarBatchSeriali auto serializer = ctx->createColumnarBatchSerializer(nullptr); auto buffer = serializer->serializeColumnarBatches(batches); auto bufferArr = env->NewByteArray(buffer->size()); + GLUTEN_CHECK( + bufferArr != nullptr, + "Cannot construct a byte array of size " + std::to_string(buffer->size()) + + " byte(s) to serialize columnar batches"); env->SetByteArrayRegion(bufferArr, 0, buffer->size(), reinterpret_cast(buffer->data())); jobject columnarBatchSerializeResult = From 5c3545aab4b9e33bc5752326152ac38d0b1954ea Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Tue, 5 Nov 2024 15:24:25 +0800 Subject: [PATCH 017/211] [GLUTEN-7814][CH] Support trigger Gluten ClickHouse CI on ARM (#7815) --- .github/workflows/clickhouse_be_trigger.yml | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/.github/workflows/clickhouse_be_trigger.yml b/.github/workflows/clickhouse_be_trigger.yml index 153fa391b8de..a45dfc8a948d 100644 --- a/.github/workflows/clickhouse_be_trigger.yml +++ b/.github/workflows/clickhouse_be_trigger.yml @@ -47,9 +47,16 @@ jobs: with: github-token: ${{ secrets.GITHUB_TOKEN }} script: | + const issueNumber = context.payload.number; + let body; + if (issueNumber % 10 === 0) { + body = "Run Gluten ClickHouse CI on ARM"; + } else { + body = "Run Gluten Clickhouse CI on x86"; + } await github.rest.issues.createComment({ owner: context.repo.owner, repo: context.repo.repo, - issue_number: context.payload.number, - body: "Run Gluten Clickhouse CI" + issue_number: issueNumber, + body: body }); From c928c895ad866281a68ac3c366ebb2c774555d31 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 5 Nov 2024 15:54:59 +0800 Subject: [PATCH 018/211] [GLUTEN-7243][VL] Suspend the Velox task while reading an input Java iterator to make the task spillable (#7748) --- cpp/velox/compute/WholeStageResultIterator.cc | 12 ---- .../operators/plannodes/RowVectorStream.h | 70 +++++++++++++------ cpp/velox/substrait/SubstraitToVeloxPlan.cc | 3 +- .../apache/gluten/extension/GlutenPlan.scala | 6 +- 4 files changed, 52 insertions(+), 39 deletions(-) diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index 29f467c5e761..adc9e9bbe988 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -226,18 +226,6 @@ int64_t WholeStageResultIterator::spillFixedSize(int64_t size) { std::string logPrefix{"Spill[" + poolName + "]: "}; int64_t shrunken = memoryManager_->shrink(size); if (spillStrategy_ == "auto") { - if (task_->numThreads() != 0) { - // Task should have zero running threads, otherwise there's - // possibility that this spill call hangs. See https://github.com/apache/incubator-gluten/issues/7243. - // As of now, non-zero running threads usually happens when: - // 1. Task A spills task B; - // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again; - LOG(INFO) << fmt::format( - "{} spill is requested on a task {} that has non-zero running threads, which is not currently supported. Skipping.", - logPrefix, - task_->taskId()); - return shrunken; - } int64_t remaining = size - shrunken; LOG(INFO) << fmt::format("{} trying to request spill for {}.", logPrefix, velox::succinctBytes(remaining)); auto mm = memoryManager_->getMemoryManager(); diff --git a/cpp/velox/operators/plannodes/RowVectorStream.h b/cpp/velox/operators/plannodes/RowVectorStream.h index c72e9137f4a4..ce26305fa054 100644 --- a/cpp/velox/operators/plannodes/RowVectorStream.h +++ b/cpp/velox/operators/plannodes/RowVectorStream.h @@ -26,16 +26,33 @@ namespace gluten { class RowVectorStream { public: explicit RowVectorStream( + facebook::velox::exec::DriverCtx* driverCtx, facebook::velox::memory::MemoryPool* pool, - std::shared_ptr iterator, + ResultIterator* iterator, const facebook::velox::RowTypePtr& outputType) - : iterator_(std::move(iterator)), outputType_(outputType), pool_(pool) {} + : driverCtx_(driverCtx), pool_(pool), outputType_(outputType), iterator_(iterator) {} bool hasNext() { - if (!finished_) { - finished_ = !iterator_->hasNext(); + if (finished_) { + return false; } - return !finished_; + bool hasNext; + { + // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current + // driver to make the current task open to spilling. + // + // When a task is getting spilled, it should have been suspended so has zero running threads, otherwise there's + // possibility that this spill call hangs. See https://github.com/apache/incubator-gluten/issues/7243. + // As of now, non-zero running threads usually happens when: + // 1. Task A spills task B; + // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again. + facebook::velox::exec::SuspendedSection(driverCtx_->driver); + hasNext = iterator_->hasNext(); + } + if (!hasNext) { + finished_ = true; + } + return hasNext; } // Convert arrow batch to rowvector and use new output columns @@ -43,7 +60,14 @@ class RowVectorStream { if (finished_) { return nullptr; } - const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, iterator_->next()); + std::shared_ptr cb; + { + // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current + // driver to make the current task open to spilling. + facebook::velox::exec::SuspendedSection(driverCtx_->driver); + cb = iterator_->next(); + } + const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, cb); auto vp = vb->getRowVector(); VELOX_DCHECK(vp != nullptr); return std::make_shared( @@ -51,10 +75,12 @@ class RowVectorStream { } private: - bool finished_{false}; - std::shared_ptr iterator_; - const facebook::velox::RowTypePtr outputType_; + facebook::velox::exec::DriverCtx* driverCtx_; facebook::velox::memory::MemoryPool* pool_; + const facebook::velox::RowTypePtr outputType_; + ResultIterator* iterator_; + + bool finished_{false}; }; class ValueStreamNode final : public facebook::velox::core::PlanNode { @@ -62,21 +88,19 @@ class ValueStreamNode final : public facebook::velox::core::PlanNode { ValueStreamNode( const facebook::velox::core::PlanNodeId& id, const facebook::velox::RowTypePtr& outputType, - std::unique_ptr valueStream) - : facebook::velox::core::PlanNode(id), outputType_(outputType), valueStream_(std::move(valueStream)) { - VELOX_CHECK_NOT_NULL(valueStream_); - } + std::shared_ptr iterator) + : facebook::velox::core::PlanNode(id), outputType_(outputType), iterator_(std::move(iterator)) {} const facebook::velox::RowTypePtr& outputType() const override { return outputType_; } const std::vector& sources() const override { - return kEmptySources; + return kEmptySources_; }; - RowVectorStream* rowVectorStream() const { - return valueStream_.get(); + ResultIterator* iterator() const { + return iterator_.get(); } std::string_view name() const override { @@ -91,8 +115,8 @@ class ValueStreamNode final : public facebook::velox::core::PlanNode { void addDetails(std::stringstream& stream) const override{}; const facebook::velox::RowTypePtr outputType_; - std::unique_ptr valueStream_; - const std::vector kEmptySources; + std::shared_ptr iterator_; + const std::vector kEmptySources_; }; class ValueStream : public facebook::velox::exec::SourceOperator { @@ -107,15 +131,17 @@ class ValueStream : public facebook::velox::exec::SourceOperator { operatorId, valueStreamNode->id(), valueStreamNode->name().data()) { - valueStream_ = valueStreamNode->rowVectorStream(); + ResultIterator* itr = valueStreamNode->iterator(); + VELOX_CHECK_NOT_NULL(itr); + rvStream_ = std::make_unique(driverCtx, pool(), itr, outputType_); } facebook::velox::RowVectorPtr getOutput() override { if (finished_) { return nullptr; } - if (valueStream_->hasNext()) { - return valueStream_->next(); + if (rvStream_->hasNext()) { + return rvStream_->next(); } else { finished_ = true; return nullptr; @@ -132,7 +158,7 @@ class ValueStream : public facebook::velox::exec::SourceOperator { private: bool finished_ = false; - RowVectorStream* valueStream_; + std::unique_ptr rvStream_; }; class RowVectorStreamOperatorTranslator : public facebook::velox::exec::Operator::PlanNodeTranslator { diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 01386115b308..9e2959043334 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -1129,8 +1129,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::constructValueStreamNode( VELOX_CHECK_LT(streamIdx, inputIters_.size(), "Could not find stream index {} in input iterator list.", streamIdx); iterator = inputIters_[streamIdx]; } - auto valueStream = std::make_unique(pool_, iterator, outputType); - auto node = std::make_shared(nextPlanNodeId(), outputType, std::move(valueStream)); + auto node = std::make_shared(nextPlanNodeId(), outputType, std::move(iterator)); auto splitInfo = std::make_shared(); splitInfo->isStream = true; diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala index 856d208eada2..06d798e50fc0 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala @@ -62,16 +62,16 @@ trait GlutenPlan extends SparkPlan with Convention.KnownBatchType with LogLevelU * Validate whether this SparkPlan supports to be transformed into substrait node in Native Code. */ final def doValidate(): ValidationResult = { - val schemaVaidationResult = BackendsApiManager.getValidatorApiInstance + val schemaValidationResult = BackendsApiManager.getValidatorApiInstance .doSchemaValidate(schema) .map { reason => ValidationResult.failed(s"Found schema check failure for $schema, due to: $reason") } .getOrElse(ValidationResult.succeeded) - if (!schemaVaidationResult.ok()) { + if (!schemaValidationResult.ok()) { TestStats.addFallBackClassName(this.getClass.toString) - return schemaVaidationResult + return schemaValidationResult } try { TransformerState.enterValidation From 40ec2cc88f94a68c9f0f97f147ee6b8431bc2d8e Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Tue, 5 Nov 2024 16:13:58 +0800 Subject: [PATCH 019/211] [GLUTEN-7654][CH] Fix round for arm (#7794) --- cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h index 432595e09140..3e219b51a695 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h @@ -111,7 +111,14 @@ class BaseFloatRoundingHalfUpComputation template static VectorType apply(VectorType val) { - return roundWithMode(val, mode); + if constexpr (std::is_same_v) + { + return std::roundf(val); + } + else + { + return std::round(val); + } } static VectorType prepare(size_t scale) From 34a88dddd98d248ee18141cba7b170ec092c82e8 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Tue, 5 Nov 2024 04:56:58 -0600 Subject: [PATCH 020/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241105) (#7809) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241105) * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/71261 * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/68682 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 +- .../Parser/RelParsers/CrossRelParser.cpp | 21 ++++++++-- .../Parser/RelParsers/JoinRelParser.cpp | 40 +++++++++++++++---- .../SubstraitSource/ReadBufferBuilder.cpp | 14 +++++-- cpp-ch/local-engine/tests/gtest_ch_join.cpp | 8 +++- 5 files changed, 69 insertions(+), 18 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 445cd99068a1..a4dd7eb5f2b0 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241101 -CH_COMMIT=7cd7bb8ece2 \ No newline at end of file +CH_BRANCH=rebase_ch/20241105 +CH_COMMIT=500e1e35c0b \ No newline at end of file diff --git a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp index 4fef282fe49c..2e7f3531568f 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp @@ -17,6 +17,7 @@ #include "CrossRelParser.h" #include +#include #include #include #include @@ -37,6 +38,10 @@ namespace DB { +namespace Setting +{ +extern const SettingsUInt64 max_block_size; +} namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -194,8 +199,15 @@ DB::QueryPlanPtr CrossRelParser::parseJoin(const substrait::CrossRel & join, DB: else { JoinPtr hash_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty()); - QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left->getCurrentHeader(), + right->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("CROSS_JOIN"); steps.emplace_back(join_step.get()); std::vector plans; @@ -243,7 +255,10 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); table_join.setColumnsFromJoinedTable( - right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); + right.getCurrentHeader().getNamesAndTypesList(), + left_columns_set, + getUniqueName("right") + ".", + left.getCurrentHeader().getNamesAndTypesList()); // fix right table key duplicate NamesWithAliases right_table_alias; diff --git a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp index 0781614bf06c..99ec54306676 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp @@ -45,6 +45,7 @@ namespace DB namespace Setting { extern const SettingsJoinAlgorithm join_algorithm; +extern const SettingsUInt64 max_block_size; } namespace ErrorCodes { @@ -313,8 +314,15 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q JoinPtr smj_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty(), -1); MultiEnum join_algorithm = context->getSettingsRef()[Setting::join_algorithm]; - QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), smj_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left->getCurrentHeader(), + right->getCurrentHeader(), + smj_join, + context->getSettingsRef()[Setting::max_block_size], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("SORT_MERGE_JOIN"); steps.emplace_back(join_step.get()); @@ -382,7 +390,11 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, NameSet left_columns_set; for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); - table_join.setColumnsFromJoinedTable(right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); + table_join.setColumnsFromJoinedTable( + right.getCurrentHeader().getNamesAndTypesList(), + left_columns_set, + getUniqueName("right") + ".", + left.getCurrentHeader().getNamesAndTypesList()); // fix right table key duplicate NamesWithAliases right_table_alias; @@ -772,8 +784,15 @@ DB::QueryPlanPtr JoinRelParser::buildMultiOnClauseHashJoin( LOG_INFO(getLogger("JoinRelParser"), "multi join on clauses:\n{}", DB::TableJoin::formatClauses(table_join->getClauses())); JoinPtr hash_join = std::make_shared(table_join, right_plan->getCurrentHeader()); - QueryPlanStepPtr join_step - = std::make_unique(left_plan->getCurrentHeader(), right_plan->getCurrentHeader(), hash_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left_plan->getCurrentHeader(), + right_plan->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("Multi join on clause hash join"); steps.emplace_back(join_step.get()); std::vector plans; @@ -806,8 +825,15 @@ DB::QueryPlanPtr JoinRelParser::buildSingleOnClauseHashJoin( { hash_join = std::make_shared(table_join, right_plan->getCurrentHeader().cloneEmpty()); } - QueryPlanStepPtr join_step - = std::make_unique(left_plan->getCurrentHeader(), right_plan->getCurrentHeader(), hash_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left_plan->getCurrentHeader(), + right_plan->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("HASH_JOIN"); steps.emplace_back(join_step.get()); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index 2681ed7c1ce7..f5f7f95e6202 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -28,10 +28,10 @@ #include #include #include -#include #include #include #include +#include #include #include #include @@ -321,8 +321,11 @@ class HDFSFileReadBufferBuilder : public ReadBufferBuilder DB::StoredObjects stored_objects{DB::StoredObject{remote_path, "", *file_size}}; auto cache_creator = wrapWithCache( read_buffer_creator, read_settings, remote_path, *modified_time, *file_size); + size_t buffer_size = std::max(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); + if (*file_size > 0) + buffer_size = std::min(*file_size, buffer_size); auto cache_hdfs_read = std::make_unique( - std::move(cache_creator), stored_objects, read_settings, nullptr, /* use_external_buffer */ false); + std::move(cache_creator), stored_objects, read_settings, nullptr, /* use_external_buffer */ false, buffer_size); read_buffer = std::move(cache_hdfs_read); } @@ -406,11 +409,14 @@ class S3FileReadBufferBuilder : public ReadBufferBuilder DB::StoredObjects stored_objects{DB::StoredObject{pathKey, "", object_size}}; auto s3_impl = std::make_unique( - std::move(cache_creator), stored_objects, read_settings, /* cache_log */ nullptr, /* use_external_buffer */ true); + std::move(cache_creator), stored_objects, read_settings, /* cache_log */ nullptr, /* use_external_buffer */ true, 0); auto & pool_reader = context->getThreadPoolReader(DB::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + size_t buffer_size = std::max(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); + if (object_size > 0) + buffer_size = std::min(object_size, buffer_size); auto async_reader - = std::make_unique(std::move(s3_impl), pool_reader, read_settings); + = std::make_unique(std::move(s3_impl), pool_reader, read_settings, buffer_size); if (read_settings.remote_fs_prefetch) async_reader->prefetch(Priority{}); diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index 52120cede088..2d853b2eba01 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -97,6 +97,10 @@ TEST(TestJoin, simple) for (const auto & column : join->columnsFromJoinedTable()) join->addJoinedColumn(column); + auto columns_from_left_table = left_plan.getCurrentHeader().getNamesAndTypesList(); + for (auto & column_from_joined_table : columns_from_left_table) + join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); + auto left_keys = left.getNamesAndTypesList(); join->addJoinedColumnsAndCorrectTypes(left_keys, true); std::cerr << "after join:\n"; @@ -122,8 +126,8 @@ TEST(TestJoin, simple) } auto hash_join = std::make_shared(join, right_plan.getCurrentHeader()); - QueryPlanStepPtr join_step - = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 1, NameSet{}, false, false); std::cerr << "join step:" << join_step->getOutputHeader().dumpStructure() << std::endl; From f498fe74850348e4ac3120da41138fe4dda122b1 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Tue, 5 Nov 2024 21:11:53 +0800 Subject: [PATCH 021/211] [CH] Ignore unstabe uts and add more message when failed. (#7821) * read data from orc file format - ignore reading except date32 * dumpPlan and dumpMessage * fix due to comments --- .../GlutenClickHouseFileFormatSuite.scala | 5 +- cpp-ch/local-engine/Common/DebugUtils.cpp | 50 ++++++++++++++++ cpp-ch/local-engine/Common/DebugUtils.h | 12 ++++ cpp-ch/local-engine/Common/GlutenConfig.cpp | 3 +- .../Parser/RelParsers/ReadRelParser.cpp | 6 +- .../Parser/SerializedPlanParser.cpp | 40 +++++-------- .../Parser/SubstraitParserUtils.cpp | 57 ------------------- .../Parser/SubstraitParserUtils.h | 12 ++-- .../Storages/MergeTree/SparkMergeTreeMeta.cpp | 6 +- 9 files changed, 93 insertions(+), 98 deletions(-) delete mode 100644 cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala index 88a34a786a8c..2337316257aa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala @@ -1037,12 +1037,13 @@ class GlutenClickHouseFileFormatSuite ) } - test("read data from orc file format") { + test("read data from orc file format - except date32") { val filePath = s"$orcDataPath/all_data_types_with_non_primitive_type.snappy.orc" val orcFileFormat = "orc" val sql = s""" - | select * + | select string_field, int_field, long_field, float_field, double_field, short_field, + | byte_field, boolean_field, decimal_field | from $orcFileFormat.`$filePath` | where long_field > 30 |""".stripMargin diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index c144a1384ee2..7b67f7d81227 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -25,9 +25,59 @@ #include #include #include +#include +#include +#include +#include +#include +#include + +namespace pb_util = google::protobuf::util; namespace debug { + +void dumpPlan(DB::QueryPlan & plan, bool force, LoggerPtr logger) +{ + if (!logger) + { + logger = getLogger("SerializedPlanParser"); + if (!logger) + return; + } + + if (!force && !logger->debug()) + return; + + auto out = local_engine::PlanUtil::explainPlan(plan); + if (force) // force + LOG_ERROR(logger, "clickhouse plan:\n{}", out); + else + LOG_DEBUG(logger, "clickhouse plan:\n{}", out); +} + +void dumpMessage(const google::protobuf::Message & message, const char * type, bool force, LoggerPtr logger) +{ + if (!logger) + { + logger = getLogger("SubstraitPlan"); + if (!logger) + return; + } + + if (!force && !logger->debug()) + return; + pb_util::JsonOptions options; + std::string json; + if (auto s = google::protobuf::json::MessageToJsonString(message, &json, options); !s.ok()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); + + if (force) // force + LOG_ERROR(logger, "{}:\n{}", type, json); + else + LOG_DEBUG(logger, "{}:\n{}", type, json); +} + void headBlock(const DB::Block & block, size_t count) { std::cout << "============Block============" << std::endl; diff --git a/cpp-ch/local-engine/Common/DebugUtils.h b/cpp-ch/local-engine/Common/DebugUtils.h index cc0ecdc59dfe..55a0be5140c5 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.h +++ b/cpp-ch/local-engine/Common/DebugUtils.h @@ -18,8 +18,20 @@ #include +namespace google::protobuf +{ +class Message; +} +namespace DB +{ +class QueryPlan; +} namespace debug { + +void dumpPlan(DB::QueryPlan & plan, bool force = false, LoggerPtr = nullptr); +void dumpMessage(const google::protobuf::Message & message, const char * type, bool force = false, LoggerPtr = nullptr); + void headBlock(const DB::Block & block, size_t count = 10); String printBlock(const DB::Block & block, size_t count = 10); diff --git a/cpp-ch/local-engine/Common/GlutenConfig.cpp b/cpp-ch/local-engine/Common/GlutenConfig.cpp index 44d77cf372cb..93d074ecc21c 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.cpp +++ b/cpp-ch/local-engine/Common/GlutenConfig.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include namespace local_engine @@ -45,7 +46,7 @@ std::map SparkConfigs::load(std::string_view plan, boo auto configMaps = local_engine::BinaryToMessage(plan); if (!processStart) - logDebugMessage(configMaps, "Update Config Map Plan"); + debug::dumpMessage(configMaps, "Update Config Map Plan"); for (const auto & pair : configMaps.configs()) configs.emplace(pair.first, pair.second); diff --git a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp index 75e6e14c4a27..2a98db344f3c 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp @@ -30,7 +30,7 @@ #include #include #include - +#include namespace DB { @@ -77,7 +77,7 @@ DB::QueryPlanPtr ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substra else { extension_table = BinaryToMessage(split_info); - logDebugMessage(extension_table, "extension_table"); + debug::dumpMessage(extension_table, "extension_table"); } MergeTreeRelParser mergeTreeParser(parser_context, getContext()); query_plan = mergeTreeParser.parseReadRel(std::make_unique(), read, extension_table); @@ -131,7 +131,7 @@ QueryPlanStepPtr ReadRelParser::parseReadRelWithLocalFile(const substrait::ReadR else { local_files = BinaryToMessage(split_info); - logDebugMessage(local_files, "local_files"); + debug::dumpMessage(local_files, "local_files"); } auto source = std::make_shared(getContext(), header, local_files); auto source_pipe = Pipe(source); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 9799933b3385..74c1d350014d 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -20,35 +20,20 @@ #include #include #include -#include #include #include -#include #include #include #include #include -#include -#include -#include -#include -#include #include -#include -#include -#include #include -#include #include -#include -#include #include #include #include -#include #include #include -#include #include #include #include @@ -73,6 +58,7 @@ #include #include #include +#include #include #include #include @@ -121,13 +107,17 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel { ActionsDAG actions_dag{blockToNameAndTypeList(query_plan->getCurrentHeader())}; NamesWithAliases aliases; - auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); + const auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); if (cols.getNames().size() != static_cast(root_rel.root().names_size())) + { + debug::dumpPlan(*query_plan, true); + debug::dumpMessage(root_rel, "substrait::PlanRel", true); throw Exception( ErrorCodes::LOGICAL_ERROR, - "Missmatch result columns size. plan column size {}, subtrait plan size {}.", + "Missmatch result columns size. plan column size {}, subtrait plan name size {}.", cols.getNames().size(), root_rel.root().names_size()); + } for (int i = 0; i < static_cast(cols.getNames().size()); i++) aliases.emplace_back(NameWithAlias(cols.getNames()[i], root_rel.root().names(i))); actions_dag.project(aliases); @@ -144,13 +134,14 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel const auto & original_cols = original_header.getColumnsWithTypeAndName(); if (static_cast(output_schema.types_size()) != original_cols.size()) { + debug::dumpPlan(*query_plan, true); + debug::dumpMessage(root_rel, "substrait::PlanRel", true); throw Exception( ErrorCodes::LOGICAL_ERROR, - "Mismatch output schema. plan column size {} [header: '{}'], subtrait plan size {}[schema: {}].", + "Missmatch result columns size. plan column size {}, subtrait plan output schema size {}, subtrait plan name size {}.", original_cols.size(), - original_header.dumpStructure(), output_schema.types_size(), - dumpMessage(output_schema)); + root_rel.root().names_size()); } bool need_final_project = false; ColumnsWithTypeAndName final_cols; @@ -192,7 +183,7 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) { - logDebugMessage(plan, "substrait plan"); + debug::dumpMessage(plan, "substrait::Plan"); //parseExtensions(plan.extensions()); if (plan.relations_size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "too many relations found"); @@ -213,12 +204,7 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) PlanUtil::checkOuputType(*query_plan); #endif - if (auto * logger = &Poco::Logger::get("SerializedPlanParser"); logger->debug()) - { - auto out = PlanUtil::explainPlan(*query_plan); - LOG_DEBUG(logger, "clickhouse plan:\n{}", out); - } - + debug::dumpPlan(*query_plan); return query_plan; } diff --git a/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp b/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp deleted file mode 100644 index c16405eff3c8..000000000000 --- a/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -#include "SubstraitParserUtils.h" -#include -#include - -using namespace DB; - -namespace local_engine -{ -namespace pb_util = google::protobuf::util; -void logDebugMessage(const google::protobuf::Message & message, const char * type) -{ - if (auto * logger = &Poco::Logger::get("SubstraitPlan"); logger->debug()) - { - pb_util::JsonOptions options; - std::string json; - if (auto s = MessageToJsonString(message, &json, options); !s.ok()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); - LOG_DEBUG(logger, "{}:\n{}", type, json); - } -} -std::string dumpMessage(const google::protobuf::Message & message) -{ - pb_util::JsonOptions options; - std::string json; - if (auto s = MessageToJsonString(message, &json, options); !s.ok()) - { - if (auto * logger = &Poco::Logger::get("SubstraitPlan")) - LOG_ERROR(logger, "Can not convert message to Json"); - return ""; - } - return json; -} -std::string toString(const google::protobuf::Any & any) -{ - google::protobuf::StringValue sv; - sv.ParseFromString(any.value()); - return sv.value(); -} -} \ No newline at end of file diff --git a/cpp-ch/local-engine/Parser/SubstraitParserUtils.h b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h index a6c252034c56..c020d96d8e76 100644 --- a/cpp-ch/local-engine/Parser/SubstraitParserUtils.h +++ b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h @@ -18,6 +18,7 @@ #include #include +#include #include namespace DB::ErrorCodes @@ -67,9 +68,10 @@ Message BinaryToMessage(const std::string_view binary) return message; } -void logDebugMessage(const google::protobuf::Message & message, const char * type); - -std::string dumpMessage(const google::protobuf::Message & message); - -std::string toString(const google::protobuf::Any & any); +inline std::string toString(const google::protobuf::Any & any) +{ + google::protobuf::StringValue sv; + sv.ParseFromString(any.value()); + return sv.value(); +} } // namespace local_engine diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp index b7c60552468d..63c6225a1f94 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp @@ -27,9 +27,9 @@ #include #include #include -#include - #include +#include +#include using namespace DB; using namespace local_engine; @@ -228,7 +228,7 @@ MergeTreeTableInstance::MergeTreeTableInstance(const google::protobuf::Any & any MergeTreeTableInstance::MergeTreeTableInstance(const substrait::ReadRel::ExtensionTable & extension_table) : MergeTreeTableInstance(extension_table.detail()) { - logDebugMessage(extension_table, "merge_tree_table"); + debug::dumpMessage(extension_table, "merge_tree_table"); } SparkStorageMergeTreePtr MergeTreeTableInstance::restoreStorage(const ContextMutablePtr & context) const From ddf28c097b0dfe2fbba7c2a4fdbfee5323bd833d Mon Sep 17 00:00:00 2001 From: Zhichao Zhang Date: Wed, 6 Nov 2024 07:13:51 +0800 Subject: [PATCH 022/211] [GLUTEN-7812][CH] Fix the query failed for the mergetree format when the 'spark.databricks.delta.stats.skipping' is off (#7813) For the Spark 3.3 + Delta 2.3, when the 'spark.databricks.delta.stats.skipping' is off, there are some queries failed with the subquery, the error message is below: ``` java.lang.IllegalArgumentException: requirement failed at scala.Predef$.require(Predef.scala:268) at org.apache.spark.sql.delta.SubqueryTransformerHelper.transformWithSubqueries(SubqueryTransformerHelper.scala:42) at org.apache.spark.sql.delta.SubqueryTransformerHelper.transformWithSubqueries$(SubqueryTransformerHelper.scala:40) at org.apache.spark.sql.delta.stats.PrepareDeltaScan.transformWithSubqueries(PrepareDeltaScan.scala:291) at org.apache.spark.sql.delta.PreprocessTableWithDVs.preprocessTablesWithDVs(PreprocessTableWithDVs.scala:67) at org.apache.spark.sql.delta.PreprocessTableWithDVs.preprocessTablesWithDVs$(PreprocessTableWithDVs.scala:66) at org.apache.spark.sql.delta.stats.PrepareDeltaScan.preprocessTablesWithDVs(PrepareDeltaScan.scala:291) at org.apache.spark.sql.delta.stats.PrepareDeltaScanBase.apply(PrepareDeltaScan.scala:227) at org.apache.spark.sql.delta.stats.PrepareDeltaScanBase.apply$(PrepareDeltaScan.scala:191) at org.apache.spark.sql.delta.stats.PrepareDeltaScan.apply(PrepareDeltaScan.scala:291) at org.apache.spark.sql.delta.stats.PrepareDeltaScan.apply(PrepareDeltaScan.scala:291) ``` Close #7812. --- backends-clickhouse/pom.xml | 1 + .../CHOptimizeMetadataOnlyDeltaQuery.scala | 30 ++ .../CHOptimizeMetadataOnlyDeltaQuery.scala | 77 ++++ .../sql/delta/stats/PrepareDeltaScan.scala | 406 ++++++++++++++++++ .../io/delta/tables/ClickhouseTable.scala | 7 +- .../ClickhouseOptimisticTransaction.scala | 14 +- .../org/apache/spark/sql/delta/DeltaLog.scala | 2 +- .../org/apache/spark/sql/delta/Snapshot.scala | 2 +- .../sql/delta/catalog/ClickHouseTableV2.scala | 8 - .../sql/delta/commands/DeleteCommand.scala | 2 +- .../delta/commands/OptimizeTableCommand.scala | 2 +- .../sql/delta/commands/UpdateCommand.scala | 2 +- .../sql/delta/commands/VacuumCommand.scala | 5 +- .../commands/merge/ClassicMergeExecutor.scala | 2 +- .../CHOptimizeMetadataOnlyDeltaQuery.scala | 77 ++++ .../backendsapi/clickhouse/CHBackend.scala | 4 + .../backendsapi/clickhouse/CHRuleApi.scala | 2 + .../GlutenClickHouseMergeTreeWriteSuite.scala | 119 ++++- .../extension/injector/SparkInjector.scala | 4 + 19 files changed, 722 insertions(+), 44 deletions(-) create mode 100644 backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala create mode 100644 backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala create mode 100644 backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala create mode 100644 backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index b00470f9ea4a..0f593a861bd8 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -368,6 +368,7 @@ src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/commands/*.scala src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/commands/merge/*.scala + src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/stats/*.scala src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/DeltaLog.scala src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/Snapshot.scala diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala new file mode 100644 index 000000000000..a360fa8d7291 --- /dev/null +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.delta.rules + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.delta.metering.DeltaLogging + +class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession) + extends Rule[LogicalPlan] + with DeltaLogging { + + // For Delta 2.0, it can not support to optimize query with the metadata + override def apply(plan: LogicalPlan): LogicalPlan = plan +} diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala new file mode 100644 index 000000000000..dbb5c4050a2c --- /dev/null +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.delta.rules + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.delta.{OptimisticTransaction, Snapshot, SubqueryTransformerHelper} +import org.apache.spark.sql.delta.files.TahoeLogFileIndex +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.stats.DeltaScanGenerator + +import org.apache.hadoop.fs.Path + +class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession) + extends Rule[LogicalPlan] + with DeltaLogging + with SubqueryTransformerHelper + with OptimizeMetadataOnlyDeltaQuery { + + private val scannedSnapshots = + new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot] + + protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = { + // The first case means that we've fixed the table snapshot for time travel + if (index.isTimeTravelQuery) return index.getSnapshot + OptimisticTransaction + .getActive() + .map(_.getDeltaScanGenerator(index)) + .getOrElse { + // Will be called only when the log is accessed the first time + scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot) + } + } + + override def apply(plan: LogicalPlan): LogicalPlan = { + // Should not be applied to subqueries to avoid duplicate delta jobs. + val isSubquery = isSubqueryRoot(plan) + // Should not be applied to DataSourceV2 write plans, because they'll be planned later + // through a V1 fallback and only that later planning takes place within the transaction. + val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand] + if (isSubquery || isDataSourceV2) { + return plan + } + // when 'stats.skipping' is off, it still use the metadata to optimize query for count/min/max + if ( + spark.sessionState.conf + .getConfString( + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE, + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE) + .toBoolean && + !spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING, true) + ) { + optimizeQueryWithMetadata(plan) + } else { + plan + } + } +} diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala new file mode 100644 index 000000000000..21e31d35411e --- /dev/null +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.stats + +import java.util.Objects + +import scala.collection.mutable + +import org.apache.spark.sql.delta._ +import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.files.{TahoeFileIndexWithSnapshot, TahoeLogFileIndex} +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.PROJECT +import org.apache.spark.sql.execution.datasources.LogicalRelation + +/** + * Gluten overwrite Delta: + * + * This file is copied from Delta 2.3.0, it is modified to overcome the following issues: + * 1. Returns the plan directly even if stats.skipping is turned off + */ + +/** + * Before query planning, we prepare any scans over delta tables by pushing + * any projections or filters in allowing us to gather more accurate statistics + * for CBO and metering. + * + * Note the following + * - This rule also ensures that all reads from the same delta log use the same snapshot of log + * thus providing snapshot isolation. + * - If this rule is invoked within an active [[OptimisticTransaction]], then the scans are + * generated using the transaction. + */ +trait PrepareDeltaScanBase extends Rule[LogicalPlan] + with PredicateHelper + with DeltaLogging + with OptimizeMetadataOnlyDeltaQuery + with PreprocessTableWithDVs { self: PrepareDeltaScan => + + /** + * Tracks the first-access snapshots of other logs planned by this rule. The snapshots are + * the keyed by the log's unique id. Note that the lifetime of this rule is a single + * query, therefore, the map tracks the snapshots only within a query. + */ + private val scannedSnapshots = + new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot] + + /** + * Gets the [[DeltaScanGenerator]] for the given log, which will be used to generate + * [[DeltaScan]]s. Every time this method is called on a log within the lifetime of this + * rule (i.e., the lifetime of the query for which this rule was instantiated), the returned + * generator will read a snapshot that is pinned on the first access for that log. + * + * Internally, it will use the snapshot of the file index, the snapshot of the active transaction + * (if any), or the latest snapshot of the given log. + */ + protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = { + // The first case means that we've fixed the table snapshot for time travel + if (index.isTimeTravelQuery) return index.getSnapshot + val scanGenerator = OptimisticTransaction.getActive() + .map(_.getDeltaScanGenerator(index)) + .getOrElse { + // Will be called only when the log is accessed the first time + scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot) + } + import PrepareDeltaScanBase._ + if (onGetDeltaScanGeneratorCallback != null) onGetDeltaScanGeneratorCallback(scanGenerator) + scanGenerator + } + + /** + * Helper method to generate a [[PreparedDeltaFileIndex]] + */ + protected def getPreparedIndex( + preparedScan: DeltaScan, + fileIndex: TahoeLogFileIndex): PreparedDeltaFileIndex = { + assert(fileIndex.partitionFilters.isEmpty, + "Partition filters should have been extracted by DeltaAnalysis.") + PreparedDeltaFileIndex( + spark, + fileIndex.deltaLog, + fileIndex.path, + preparedScan, + fileIndex.versionToUse) + } + + /** + * Scan files using the given `filters` and return `DeltaScan`. + * + * Note: when `limitOpt` is non empty, `filters` must contain only partition filters. Otherwise, + * it can contain arbitrary filters. See `DeltaTableScan` for more details. + */ + protected def filesForScan( + scanGenerator: DeltaScanGenerator, + limitOpt: Option[Int], + filters: Seq[Expression], + delta: LogicalRelation): DeltaScan = { + withStatusCode("DELTA", "Filtering files for query") { + if (limitOpt.nonEmpty) { + // If we trigger limit push down, the filters must be partition filters. Since + // there are no data filters, we don't need to apply Generated Columns + // optimization. See `DeltaTableScan` for more details. + return scanGenerator.filesForScan(limitOpt.get, filters) + } + val filtersForScan = + if (!GeneratedColumn.partitionFilterOptimizationEnabled(spark)) { + filters + } else { + val generatedPartitionFilters = GeneratedColumn.generatePartitionFilters( + spark, scanGenerator.snapshotToScan, filters, delta) + filters ++ generatedPartitionFilters + } + scanGenerator.filesForScan(filtersForScan) + } + } + + /** + * Prepares delta scans sequentially. + */ + protected def prepareDeltaScan(plan: LogicalPlan): LogicalPlan = { + // A map from the canonicalized form of a DeltaTableScan operator to its corresponding delta + // scan. This map is used to avoid fetching duplicate delta indexes for structurally-equal + // delta scans. + val deltaScans = new mutable.HashMap[LogicalPlan, DeltaScan]() + + transformWithSubqueries(plan) { + case scan @ DeltaTableScan(planWithRemovedProjections, filters, fileIndex, + limit, delta) => + val scanGenerator = getDeltaScanGenerator(fileIndex) + val preparedScan = deltaScans.getOrElseUpdate(planWithRemovedProjections.canonicalized, + filesForScan(scanGenerator, limit, filters, delta)) + val preparedIndex = getPreparedIndex(preparedScan, fileIndex) + optimizeGeneratedColumns(scan, preparedIndex, filters, limit, delta) + } + } + + protected def optimizeGeneratedColumns( + scan: LogicalPlan, + preparedIndex: PreparedDeltaFileIndex, + filters: Seq[Expression], + limit: Option[Int], + delta: LogicalRelation): LogicalPlan = { + if (limit.nonEmpty) { + // If we trigger limit push down, the filters must be partition filters. Since + // there are no data filters, we don't need to apply Generated Columns + // optimization. See `DeltaTableScan` for more details. + return DeltaTableUtils.replaceFileIndex(scan, preparedIndex) + } + if (!GeneratedColumn.partitionFilterOptimizationEnabled(spark)) { + DeltaTableUtils.replaceFileIndex(scan, preparedIndex) + } else { + val generatedPartitionFilters = + GeneratedColumn.generatePartitionFilters(spark, preparedIndex, filters, delta) + val scanWithFilters = + if (generatedPartitionFilters.nonEmpty) { + scan transformUp { + case delta @ DeltaTable(_: TahoeLogFileIndex) => + Filter(generatedPartitionFilters.reduceLeft(And), delta) + } + } else { + scan + } + DeltaTableUtils.replaceFileIndex(scanWithFilters, preparedIndex) + } + } + + override def apply(_plan: LogicalPlan): LogicalPlan = { + var plan = _plan + + // --- modified start + // Should not be applied to subqueries to avoid duplicate delta jobs. + val isSubquery = isSubqueryRoot(plan) + // Should not be applied to DataSourceV2 write plans, because they'll be planned later + // through a V1 fallback and only that later planning takes place within the transaction. + val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand] + if (isSubquery || isDataSourceV2) { + return plan + } + + val shouldPrepareDeltaScan = ( + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING) + ) + val updatedPlan = if (shouldPrepareDeltaScan) { + if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_METADATA_QUERY_ENABLED)) { + plan = optimizeQueryWithMetadata(plan) + } + prepareDeltaScan(plan) + } else { + // If this query is running inside an active transaction and is touching the same table + // as the transaction, then mark that the entire table as tainted to be safe. + OptimisticTransaction.getActive.foreach { txn => + val logsInPlan = plan.collect { case DeltaTable(fileIndex) => fileIndex.deltaLog } + if (logsInPlan.exists(_.isSameLogAs(txn.deltaLog))) { + txn.readWholeTable() + } + } + + // Just return the plan if statistics based skipping is off. + // It will fall back to just partition pruning at planning time. + plan + } + // --- modified end + preprocessTablesWithDVs(updatedPlan) + } + + /** + * This is an extractor object. See https://docs.scala-lang.org/tour/extractor-objects.html. + */ + object DeltaTableScan { + + /** + * The components of DeltaTableScanType are: + * - the plan with removed projections. We remove projections as a plan differentiator + * because it does not affect file listing results. + * - filter expressions collected by `PhysicalOperation` + * - the `TahoeLogFileIndex` of the matched DeltaTable` + * - integer value of limit expression, if any + * - matched `DeltaTable` + */ + private type DeltaTableScanType = + (LogicalPlan, Seq[Expression], TahoeLogFileIndex, Option[Int], LogicalRelation) + + /** + * This is an extractor method (basically, the opposite of a constructor) which takes in an + * object `plan` and tries to give back the arguments as a [[DeltaTableScanType]]. + */ + def unapply(plan: LogicalPlan): Option[DeltaTableScanType] = { + val limitPushdownEnabled = spark.conf.get(DeltaSQLConf.DELTA_LIMIT_PUSHDOWN_ENABLED) + + // Remove projections as a plan differentiator because it does not affect file listing + // results. Plans with the same filters but different projections therefore will not have + // duplicate delta indexes. + def canonicalizePlanForDeltaFileListing(plan: LogicalPlan): LogicalPlan = { + val planWithRemovedProjections = plan.transformWithPruning(_.containsPattern(PROJECT)) { + case p: Project if p.projectList.forall(_.isInstanceOf[AttributeReference]) => p.child + } + planWithRemovedProjections + } + + plan match { + case LocalLimit(IntegerLiteral(limit), + PhysicalOperation(_, filters, delta @ DeltaTable(fileIndex: TahoeLogFileIndex))) + if limitPushdownEnabled && containsPartitionFiltersOnly(filters, fileIndex) => + Some((canonicalizePlanForDeltaFileListing(plan), filters, fileIndex, Some(limit), delta)) + case PhysicalOperation( + _, + filters, + delta @ DeltaTable(fileIndex: TahoeLogFileIndex)) => + val allFilters = fileIndex.partitionFilters ++ filters + Some((canonicalizePlanForDeltaFileListing(plan), allFilters, fileIndex, None, delta)) + + case _ => None + } + } + + private def containsPartitionFiltersOnly( + filters: Seq[Expression], + fileIndex: TahoeLogFileIndex): Boolean = { + val partitionColumns = fileIndex.snapshotAtAnalysis.metadata.partitionColumns + import DeltaTableUtils._ + filters.forall(expr => !containsSubquery(expr) && + isPredicatePartitionColumnsOnly(expr, partitionColumns, spark)) + } + } +} + +class PrepareDeltaScan(protected val spark: SparkSession) + extends PrepareDeltaScanBase + +object PrepareDeltaScanBase { + + /** + * Optional callback function that is called after `getDeltaScanGenerator` is called + * by the PrepareDeltaScan rule. This is primarily used for testing purposes. + */ + @volatile private var onGetDeltaScanGeneratorCallback: DeltaScanGenerator => Unit = _ + + /** + * Run a thunk of code with the given callback function injected into the PrepareDeltaScan rule. + * The callback function is called after `getDeltaScanGenerator` is called + * by the PrepareDeltaScan rule. This is primarily used for testing purposes. + */ + private[delta] def withCallbackOnGetDeltaScanGenerator[T]( + callback: DeltaScanGenerator => Unit)(thunk: => T): T = { + try { + onGetDeltaScanGeneratorCallback = callback + thunk + } finally { + onGetDeltaScanGeneratorCallback = null + } + } +} + +/** + * A [[TahoeFileIndex]] that uses a prepared scan to return the list of relevant files. + * This is injected into a query right before query planning by [[PrepareDeltaScan]] so that + * CBO and metering can accurately understand how much data will be read. + * + * @param versionScanned The version of the table that is being scanned, if a specific version + * has specifically been requested, e.g. by time travel. + */ +case class PreparedDeltaFileIndex( + override val spark: SparkSession, + override val deltaLog: DeltaLog, + override val path: Path, + preparedScan: DeltaScan, + versionScanned: Option[Long]) + extends TahoeFileIndexWithSnapshot(spark, deltaLog, path, preparedScan.scannedSnapshot) + with DeltaLogging { + + /** + * Returns all matching/valid files by the given `partitionFilters` and `dataFilters` + */ + override def matchingFiles( + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): Seq[AddFile] = { + val currentFilters = ExpressionSet(partitionFilters ++ dataFilters) + val (addFiles, eventData) = if (currentFilters == preparedScan.allFilters || + currentFilters == preparedScan.filtersUsedForSkipping) { + // [[DeltaScan]] was created using `allFilters` out of which only `filtersUsedForSkipping` + // filters were used for skipping while creating the DeltaScan. + // If currentFilters is same as allFilters, then no need to recalculate files and we can use + // previous results. + // If currentFilters is same as filtersUsedForSkipping, then also we don't need to recalculate + // files as [[DeltaScan.files]] were calculates using filtersUsedForSkipping only. So if we + // recalculate, we will get same result. So we should use previous result in this case also. + val eventData = Map( + "reused" -> true, + "currentFiltersSameAsPreparedAllFilters" -> (currentFilters == preparedScan.allFilters), + "currentFiltersSameAsPreparedFiltersUsedForSkipping" -> + (currentFilters == preparedScan.filtersUsedForSkipping) + ) + (preparedScan.files.distinct, eventData) + } else { + logInfo( + s""" + |Prepared scan does not match actual filters. Reselecting files to query. + |Prepared: ${preparedScan.allFilters} + |Actual: ${currentFilters} + """.stripMargin) + val eventData = Map( + "reused" -> false, + "preparedAllFilters" -> preparedScan.allFilters.mkString(","), + "preparedFiltersUsedForSkipping" -> preparedScan.filtersUsedForSkipping.mkString(","), + "currentFilters" -> currentFilters.mkString(",") + ) + val files = preparedScan.scannedSnapshot.filesForScan(partitionFilters ++ dataFilters).files + (files, eventData) + } + recordDeltaEvent(deltaLog, + opType = "delta.preparedDeltaFileIndex.reuseSkippingResult", + data = eventData) + addFiles + } + + /** + * Returns the list of files that will be read when scanning this relation. This call may be + * very expensive for large tables. + */ + override def inputFiles: Array[String] = + preparedScan.files.map(f => absolutePath(f.path).toString).toArray + + /** Refresh any cached file listings */ + override def refresh(): Unit = { } + + /** Sum of table file sizes, in bytes */ + override def sizeInBytes: Long = + preparedScan.scanned.bytesCompressed + .getOrElse(spark.sessionState.conf.defaultSizeInBytes) + + override def equals(other: Any): Boolean = other match { + case p: PreparedDeltaFileIndex => + p.deltaLog == deltaLog && p.path == path && p.preparedScan == preparedScan && + p.partitionSchema == partitionSchema && p.versionScanned == versionScanned + case _ => false + } + + override def hashCode(): Int = { + Objects.hash(deltaLog, path, preparedScan, partitionSchema, versionScanned) + } + +} diff --git a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala index e747c87c6a67..ba4c21df3a34 100644 --- a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala +++ b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala @@ -27,12 +27,7 @@ import scala.collection.JavaConverters._ class ClickhouseTable( @transient private val _df: Dataset[Row], @transient private val table: ClickHouseTableV2) - extends DeltaTable(_df, table) { - - override def optimize(): DeltaOptimizeBuilder = { - DeltaOptimizeBuilder(table) - } -} + extends DeltaTable(_df, table) {} object ClickhouseTable { diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 2f5824b58092..9097a02b9337 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -196,8 +196,9 @@ class ClickhouseOptimisticTransaction( isOptimize: Boolean, additionalConstraints: Seq[Constraint]): Seq[FileAction] = { - if (isOptimize) + if (isOptimize) { throw new UnsupportedOperationException("Optimize is not supported for ClickHouse") + } hasWritten = true @@ -258,7 +259,7 @@ class ClickhouseOptimisticTransaction( DeltaOptimizedWriterExec(checkInvariants, metadata.partitionColumns, deltaLog) } else { checkInvariants - }*/ + } */ val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { @@ -304,10 +305,11 @@ class ClickhouseOptimisticTransaction( committer.addedStatuses }) .filter { - // In some cases, we can write out an empty `inputData`. Some examples of this (though, they - // may be fixed in the future) are the MERGE command when you delete with empty source, or - // empty target, or on disjoint tables. This is hard to catch before the write without - // collecting the DF ahead of time. Instead, we can return only the AddFiles that + // In some cases, we can write out an empty `inputData`. Some examples of this (though, + // they may be fixed in the future) are the MERGE command when you delete with empty + // source, or empty target, or on disjoint tables. This is hard to catch before + // the write without collecting the DF ahead of time. Instead, + // we can return only the AddFiles that // a) actually add rows, or // b) don't have any stats so we don't know the number of rows at all case a: AddFile => a.numLogicalRecords.forall(_ > 0) diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala index bac5231309b8..f64de28f4214 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala @@ -60,7 +60,7 @@ import org.apache.spark.util._ /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0, it is modified to overcome the following issues: + * This file is copied from Delta 3.2.1, it is modified to overcome the following issues: * 1. return ClickhouseOptimisticTransaction * 2. return DeltaMergeTreeFileFormat * 3. create HadoopFsRelation with the bucket options diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala index 8836f7c88d23..5bfda914db67 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. filesForScan() will cache the DeltaScan by the FilterExprsAsKey * 2. filesForScan() should return DeltaScan of AddMergeTreeParts instead of AddFile */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala index 5f6a2dc3d712..d887e7a21b34 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.V1Table import org.apache.spark.sql.connector.read.InputPartition -import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.delta.{ClickhouseSnapshot, DeltaErrors, DeltaLog, DeltaTableUtils, DeltaTimeTravelSpec, Snapshot, UnresolvedPathBasedDeltaTable} import org.apache.spark.sql.delta.actions.{Metadata, Protocol} import org.apache.spark.sql.delta.sources.DeltaDataSource @@ -89,13 +88,6 @@ class ClickHouseTableV2( ret } - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - new WriteIntoDeltaBuilder( - this, - info.options, - spark.sessionState.conf.useNullsForMissingDefaultColumnValues) - } - def getFileFormat(protocol: Protocol, meta: Metadata): DeltaMergeTreeFileFormat = { new DeltaMergeTreeFileFormat( protocol, diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala index dec1f4b9c3f5..0a25346fc6c3 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.types.LongType /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala index 5b2170220228..439111df1b1c 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala @@ -49,7 +49,7 @@ import org.apache.spark.util.{SystemClock, ThreadUtils} /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified in: + * This file is copied from Delta 3.2.1. It is modified in: * 1. getDeltaTable supports to get ClickHouseTableV2 * 2. runOptimizeBinJobClickhouse * 3. groupFilesIntoBinsClickhouse diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala index 9a7fb96775f0..4e75b8461970 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.types.LongType /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala index 5d05bdb86896..7a350ae4d594 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock} /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Gluten, part is a directory, but VacuumCommand assumes part is a file. So we need some * modifications to make it work. * 2. Set the 'gluten.enabledForCurrentThread' to false, now gluten can not support vacuum cmd. @@ -255,7 +255,8 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { val originalEnabledGluten = spark.sparkContext.getLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY) // gluten can not support vacuum command - spark.sparkContext.setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false") + spark.sparkContext + .setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false") // --- modified end val validFiles = diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala index 42a89d427197..aa1f94c5c9f5 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.functions.{coalesce, col, count, input_file_name, li /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). In findTouchedFiles func. */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala new file mode 100644 index 000000000000..dbb5c4050a2c --- /dev/null +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.delta.rules + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.delta.{OptimisticTransaction, Snapshot, SubqueryTransformerHelper} +import org.apache.spark.sql.delta.files.TahoeLogFileIndex +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.stats.DeltaScanGenerator + +import org.apache.hadoop.fs.Path + +class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession) + extends Rule[LogicalPlan] + with DeltaLogging + with SubqueryTransformerHelper + with OptimizeMetadataOnlyDeltaQuery { + + private val scannedSnapshots = + new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot] + + protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = { + // The first case means that we've fixed the table snapshot for time travel + if (index.isTimeTravelQuery) return index.getSnapshot + OptimisticTransaction + .getActive() + .map(_.getDeltaScanGenerator(index)) + .getOrElse { + // Will be called only when the log is accessed the first time + scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot) + } + } + + override def apply(plan: LogicalPlan): LogicalPlan = { + // Should not be applied to subqueries to avoid duplicate delta jobs. + val isSubquery = isSubqueryRoot(plan) + // Should not be applied to DataSourceV2 write plans, because they'll be planned later + // through a V1 fallback and only that later planning takes place within the transaction. + val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand] + if (isSubquery || isDataSourceV2) { + return plan + } + // when 'stats.skipping' is off, it still use the metadata to optimize query for count/min/max + if ( + spark.sessionState.conf + .getConfString( + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE, + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE) + .toBoolean && + !spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING, true) + ) { + optimizeQueryWithMetadata(plan) + } else { + plan + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 9a1b00f71431..ba17d12ffa94 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -133,6 +133,10 @@ object CHBackendSettings extends BackendSettingsApi with Logging { val GLUTEN_CLICKHOUSE_TABLE_PATH_TO_MTPS_CACHE_SIZE: String = CHConf.prefixOf("table.path.to.mtps.cache.size") + val GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE: String = + CHConf.prefixOf("delta.metadata.optimize") + val GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE: String = "true" + def affinityMode: String = { SparkEnv.get.conf .get( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index dea0d50c9da6..470ece4037a5 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -30,6 +30,7 @@ import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRewrite} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.delta.DeltaLogFileIndex +import org.apache.spark.sql.delta.rules.CHOptimizeMetadataOnlyDeltaQuery import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, CommandResultExec, FileSourceScanExec, GlutenFallbackReporter, RDDScanExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.V2CommandExec @@ -59,6 +60,7 @@ private object CHRuleApi { injector.injectOptimizerRule(spark => CHAggregateFunctionRewriteRule(spark)) injector.injectOptimizerRule(_ => CountDistinctWithoutExpand) injector.injectOptimizerRule(_ => EqualToRewrite) + injector.injectPreCBORule(spark => new CHOptimizeMetadataOnlyDeltaQuery(spark)) } def injectLegacy(injector: LegacyInjector): Unit = { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index c1210c5fbaca..033d51467177 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -1977,24 +1977,29 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | count(*) AS count_order - |FROM - | lineitem_mergetree_count_opti - |""".stripMargin - runSql(sqlStr)( - df => { - val result = df.collect() - assertResult(1)(result.length) - assertResult("600572")(result(0).getLong(0).toString) + Seq("true", "false").foreach { + skip => + withSQLConf("spark.databricks.delta.stats.skipping" -> skip.toString) { + val sqlStr = + s""" + |SELECT + | count(*) AS count_order + |FROM + | lineitem_mergetree_count_opti + |""".stripMargin + runSql(sqlStr)( + df => { + val result = df.collect() + assertResult(1)(result.length) + assertResult("600572")(result(0).getLong(0).toString) - // Spark 3.2 + Delta 2.0 does not support this feature - if (!spark32) { - assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) + // Spark 3.2 + Delta 2.0 does not support this feature + if (!spark32) { + assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) + } + }) } - }) + } } test("test mergetree with column case sensitive") { @@ -2128,4 +2133,86 @@ class GlutenClickHouseMergeTreeWriteSuite } }) } + + test( + "GLUTEN-7812: Fix the query failed for the mergetree format " + + "when the 'spark.databricks.delta.stats.skipping' is off") { + // Spark 3.2 + Delta 2.0 doesn't not support this feature + if (!spark32) { + withSQLConf(("spark.databricks.delta.stats.skipping", "false")) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_stats_skipping; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_stats_skipping + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_returnflag) + |TBLPROPERTIES (orderByKey='l_orderkey', + | primaryKey='l_orderkey') + |LOCATION '$basePath/lineitem_mergetree_stats_skipping' + |""".stripMargin) + + // dynamic partitions + spark.sql(s""" + | insert into table lineitem_mergetree_stats_skipping + | select * from lineitem + |""".stripMargin) + + val sqlStr = + s""" + |SELECT + | o_orderpriority, + | count(*) AS order_count + |FROM + | orders + |WHERE + | o_orderdate >= date'1993-07-01' + | AND o_orderdate < date'1993-07-01' + interval 3 month + | AND EXISTS ( + | SELECT + | * + | FROM + | lineitem + | WHERE + | l_orderkey = o_orderkey + | AND l_commitdate < l_receiptdate) + |GROUP BY + | o_orderpriority + |ORDER BY + | o_orderpriority; + | + |""".stripMargin + runSql(sqlStr)( + df => { + val result = df.collect() + assertResult(5)(result.length) + assertResult("1-URGENT")(result(0).getString(0)) + assertResult(999)(result(0).getLong(1)) + assertResult("2-HIGH")(result(1).getString(0)) + assertResult(997)(result(1).getLong(1)) + assertResult("5-LOW")(result(4).getString(0)) + assertResult(1077)(result(4).getLong(1)) + }) + } + } + } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala index 87942c4155e7..fe6db65c3a1d 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala @@ -49,4 +49,8 @@ class SparkInjector private[injector] ( def injectFunction(functionDescription: FunctionDescription): Unit = { extensions.injectFunction(control.disabler().wrapFunction(functionDescription)) } + + def injectPreCBORule(builder: RuleBuilder): Unit = { + extensions.injectPreCBORule(control.disabler().wrapRule(builder)) + } } From 6387359f6ce944e89fd11013b03dc47c02edd8d1 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Wed, 6 Nov 2024 07:52:02 +0800 Subject: [PATCH 023/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_06) (#7822) Upstream Velox's New Commits: 3034dd37e by Chengcheng Jin, Respect kPreferredOutputBatchRows strictly in Unnest (10711) a5ae22832 by duanmeng, Fix flakey FilterProjectReplayerTest (11436) f9b24d55d by Yang Zhang, Fix GCC build error with std::to_char (11369) 30fc13e01 by Mike Lui, prepare for Arrow 18.0.0 by adding new arrow internal header (11385) c0768d7f5 by Chengcheng Jin, Add SimpleFunctionMetadata toDebugString function (10821) 62b0a1287 by Jialiang Tan, Let arbitrator system use nano second (11415) 1e22de825 by duanmeng, Add HashJoinReplayer (11271) 084879f9a by Jialiang Tan, Add icu4c brew search for icu4c (11416) 2ade7f7de by Kevin Wilfong, Presto's from_unixtime with TSwTZ can be 1 millisecond off in Velox compared to Presto Java (11426) f4a1223c0 by Kevin Wilfong, Handle strings with partial date/time specifications and time zones in Presto's from_iso8601_t --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 16467f938da0..3c1bc083ed6d 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_05 +VELOX_BRANCH=2024_11_06 VELOX_HOME="" OS=`uname -s` From fc737220adfa8a21c1edea0dd54b742ce71f43b7 Mon Sep 17 00:00:00 2001 From: Yang Zhang Date: Wed, 6 Nov 2024 09:40:28 +0800 Subject: [PATCH 024/211] [VL] Remove load shared libhdfs (#7818) --- .../org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala | 1 - .../org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala | 1 - .../org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala | 1 - .../org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala | 1 - .../org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala | 1 - .../org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala | 1 - .../org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala | 1 - 7 files changed, 7 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala index 16cd18e41a05..b9d90d589c02 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala @@ -37,7 +37,6 @@ class SharedLibraryLoaderCentos7 extends SharedLibraryLoader { loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false) loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.10", "libre2.so", false) loader.loadAndCreateLink("libzstd.so.1", "libzstd.so", false) loader.loadAndCreateLink("liblz4.so.1", "liblz4.so", false) diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala index 0a75c30c22ff..dbb7d59f889d 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala @@ -42,7 +42,6 @@ class SharedLibraryLoaderCentos8 extends SharedLibraryLoader { loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false) loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.0", "libre2.so", false) loader.loadAndCreateLink("libsodium.so.23", "libsodium.so", false) } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala index 50f9fe4aaadc..f633a79f8fcd 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala @@ -42,7 +42,6 @@ class SharedLibraryLoaderCentos9 extends SharedLibraryLoader { loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false) loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.9", "libre2.so", false) loader.loadAndCreateLink("libsodium.so.23", "libsodium.so", false) } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala index 06c065ba2883..4f9b6fdd7250 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala @@ -44,6 +44,5 @@ class SharedLibraryLoaderDebian11 extends SharedLibraryLoader { loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false) loader.loadAndCreateLink("libcurl.so.4", "libcurl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala index 8018995328fe..e3967eea3267 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala @@ -50,6 +50,5 @@ class SharedLibraryLoaderDebian12 extends SharedLibraryLoader { loader.loadAndCreateLink("libevent-2.1.so.7", "libevent-2.1.so", false) loader.loadAndCreateLink("libcurl.so.4", "libcurl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala index d1f21a0013fb..6d9271e9e3d9 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala @@ -57,7 +57,6 @@ class SharedLibraryLoaderUbuntu2004 extends SharedLibraryLoader { loader.loadAndCreateLink("libicudata.so.66", "libicudata.so", false) loader.loadAndCreateLink("libicuuc.so.66", "libicuuc.so", false) loader.loadAndCreateLink("libxml2.so.2", "libxml2.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.5", "libre2.so", false) loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false) loader.loadAndCreateLink("libthrift-0.13.0.so", "libthrift.so", false) diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala index 3cf4d30237ac..95f7db5655b2 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala @@ -42,7 +42,6 @@ class SharedLibraryLoaderUbuntu2204 extends SharedLibraryLoader { loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) loader.loadAndCreateLink("libxml2.so.2", "libxml2.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.9", "libre2.so", false) loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false) loader.loadAndCreateLink("libthrift-0.16.0.so", "libthrift.so", false) From 3099799e023bdd85d53fcf1e95e9a0e661adb24f Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Wed, 6 Nov 2024 11:44:15 +0800 Subject: [PATCH 025/211] [GLUTEN-7749][VL] Trim ISOControl characters when casting string to integral type (#7806) --- .../velox/VeloxSparkPlanExecApi.scala | 43 +++++----- .../spark/sql/GlutenDataFrameSuite.scala | 80 +++++++++++-------- 2 files changed, 67 insertions(+), 56 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index 81564a440117..7901374f6bf6 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -723,30 +723,29 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { val trimParaSepStr = "\u2029" // Needs to be trimmed for casting to float/double/decimal val trimSpaceStr = ('\u0000' to '\u0020').toList.mkString + // ISOControl characters, refer java.lang.Character.isISOControl(int) + val isoControlStr = (('\u0000' to '\u001F') ++ ('\u007F' to '\u009F')).toList.mkString // scalastyle:on nonascii - c.dataType match { - case BinaryType | _: ArrayType | _: MapType | _: StructType | _: UserDefinedType[_] => - c - case FloatType | DoubleType | _: DecimalType => - c.child.dataType match { - case StringType if GlutenConfig.getConf.castFromVarcharAddTrimNode => - val trimNode = StringTrim(c.child, Some(Literal(trimSpaceStr))) - c.withNewChildren(Seq(trimNode)).asInstanceOf[Cast] - case _ => - c - } - case _ => - c.child.dataType match { - case StringType if GlutenConfig.getConf.castFromVarcharAddTrimNode => - val trimNode = StringTrim( - c.child, - Some( - Literal(trimWhitespaceStr + - trimSpaceSepStr + trimLineSepStr + trimParaSepStr))) - c.withNewChildren(Seq(trimNode)).asInstanceOf[Cast] - case _ => - c + if (GlutenConfig.getConf.castFromVarcharAddTrimNode && c.child.dataType == StringType) { + val trimStr = c.dataType match { + case BinaryType | _: ArrayType | _: MapType | _: StructType | _: UserDefinedType[_] => + None + case FloatType | DoubleType | _: DecimalType => + Some(trimSpaceStr) + case _ => + Some( + (trimWhitespaceStr + trimSpaceSepStr + trimLineSepStr + + trimParaSepStr + isoControlStr).toSet.mkString + ) + } + trimStr + .map { + trim => + c.withNewChildren(Seq(StringTrim(c.child, Some(Literal(trim))))).asInstanceOf[Cast] } + .getOrElse(c) + } else { + c } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala index 4008f862e17d..3b2db7117f4a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql +import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.{ProjectExecTransformer, WholeStageTransformer} import org.apache.spark.SparkException @@ -323,41 +324,52 @@ class GlutenDataFrameSuite extends DataFrameSuite with GlutenSQLTestsTrait { } testGluten("Allow leading/trailing whitespace in string before casting") { - def checkResult(df: DataFrame, expectedResult: Seq[Row]): Unit = { - checkAnswer(df, expectedResult) - assert(find(df.queryExecution.executedPlan)(_.isInstanceOf[ProjectExecTransformer]).isDefined) - } + withSQLConf(GlutenConfig.CAST_FROM_VARCHAR_ADD_TRIM_NODE.key -> "true") { + def checkResult(df: DataFrame, expectedResult: Seq[Row]): Unit = { + checkAnswer(df, expectedResult) + assert( + find(df.queryExecution.executedPlan)(_.isInstanceOf[ProjectExecTransformer]).isDefined) + } - // scalastyle:off nonascii - Seq(" 123", "123 ", " 123 ", "\u2000123\n\n\n", "123\r\r\r", "123\f\f\f", "123\u000C") - .toDF("col1") - .createOrReplaceTempView("t1") - // scalastyle:on nonascii - val expectedIntResult = Row(123) :: Row(123) :: - Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Nil - var df = spark.sql("select cast(col1 as int) from t1") - checkResult(df, expectedIntResult) - df = spark.sql("select cast(col1 as long) from t1") - checkResult(df, expectedIntResult) - - Seq(" 123.5", "123.5 ", " 123.5 ", "123.5\n\n\n", "123.5\r\r\r", "123.5\f\f\f", "123.5\u000C") - .toDF("col1") - .createOrReplaceTempView("t1") - val expectedFloatResult = Row(123.5) :: Row(123.5) :: - Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Nil - df = spark.sql("select cast(col1 as float) from t1") - checkResult(df, expectedFloatResult) - df = spark.sql("select cast(col1 as double) from t1") - checkResult(df, expectedFloatResult) - - // scalastyle:off nonascii - val rawData = - Seq(" abc", "abc ", " abc ", "\u2000abc\n\n\n", "abc\r\r\r", "abc\f\f\f", "abc\u000C") - // scalastyle:on nonascii - rawData.toDF("col1").createOrReplaceTempView("t1") - val expectedBinaryResult = rawData.map(d => Row(d.getBytes(StandardCharsets.UTF_8))).seq - df = spark.sql("select cast(col1 as binary) from t1") - checkResult(df, expectedBinaryResult) + // scalastyle:off nonascii + Seq( + " 123", + "123 ", + " 123 ", + "\u2000123\n\n\n", + "123\r\r\r", + "123\f\f\f", + "123\u000C", + "123\u0000") + .toDF("col1") + .createOrReplaceTempView("t1") + // scalastyle:on nonascii + val expectedIntResult = Row(123) :: Row(123) :: + Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Nil + var df = spark.sql("select cast(col1 as int) from t1") + checkResult(df, expectedIntResult) + df = spark.sql("select cast(col1 as long) from t1") + checkResult(df, expectedIntResult) + + Seq(" 123.5", "123.5 ", " 123.5 ", "123.5\n\n\n", "123.5\r\r\r", "123.5\f\f\f", "123.5\u000C") + .toDF("col1") + .createOrReplaceTempView("t1") + val expectedFloatResult = Row(123.5) :: Row(123.5) :: + Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Nil + df = spark.sql("select cast(col1 as float) from t1") + checkResult(df, expectedFloatResult) + df = spark.sql("select cast(col1 as double) from t1") + checkResult(df, expectedFloatResult) + + // scalastyle:off nonascii + val rawData = + Seq(" abc", "abc ", " abc ", "\u2000abc\n\n\n", "abc\r\r\r", "abc\f\f\f", "abc\u000C") + // scalastyle:on nonascii + rawData.toDF("col1").createOrReplaceTempView("t1") + val expectedBinaryResult = rawData.map(d => Row(d.getBytes(StandardCharsets.UTF_8))).seq + df = spark.sql("select cast(col1 as binary) from t1") + checkResult(df, expectedBinaryResult) + } } private def withExpr(newExpr: Expression): Column = new Column(newExpr) From 17080e73cd15d6bc5889e3837ac2c5a28627edac Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Wed, 6 Nov 2024 13:59:34 +0800 Subject: [PATCH 026/211] [CH] Rename Mergetree part file name to avoid duplicated file name (#7769) What changes were proposed in this pull request? Rename Mergetree part file name to avoid duplicated file name and support prefetch mergetree data file How was this patch tested? unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- ...tenClickHouseMergeTreeWriteOnS3Suite.scala | 4 +-- cpp-ch/local-engine/Common/GlutenConfig.cpp | 6 +++++ cpp-ch/local-engine/Common/GlutenConfig.h | 9 +++++++ .../CompactObjectStorageDiskTransaction.cpp | 4 +-- .../CompactObjectStorageDiskTransaction.h | 3 +++ .../Storages/Cache/CacheManager.cpp | 7 ++++- .../Storages/Cache/CacheManager.h | 2 +- .../MergeTree/SparkStorageMergeTree.cpp | 26 +++++++++++++------ .../MergeTree/SparkStorageMergeTree.h | 4 ++- 9 files changed, 50 insertions(+), 15 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index af67b01f49f0..331009d12c96 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -196,9 +196,9 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite val objectName = obj.get().objectName() if (objectName.contains("metadata.gluten")) { metadataGlutenExist = true - } else if (objectName.contains("meta.bin")) { + } else if (objectName.contains("part_meta.gluten")) { metadataBinExist = true - } else if (objectName.contains("data.bin")) { + } else if (objectName.contains("part_data.gluten")) { dataBinExist = true } else if (objectName.contains("_commits")) { // Spark 35 has _commits directory diff --git a/cpp-ch/local-engine/Common/GlutenConfig.cpp b/cpp-ch/local-engine/Common/GlutenConfig.cpp index 93d074ecc21c..eb6c2dcab622 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.cpp +++ b/cpp-ch/local-engine/Common/GlutenConfig.cpp @@ -137,4 +137,10 @@ GlutenJobSchedulerConfig GlutenJobSchedulerConfig::loadFromContext(const DB::Con config.job_scheduler_max_threads = context->getConfigRef().getUInt64(JOB_SCHEDULER_MAX_THREADS, 10); return config; } +MergeTreeCacheConfig MergeTreeCacheConfig::loadFromContext(const DB::ContextPtr & context) +{ + MergeTreeCacheConfig config; + config.enable_data_prefetch = context->getConfigRef().getBool(ENABLE_DATA_PREFETCH, config.enable_data_prefetch); + return config; +} } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h index 82402eaafa47..11220afb4878 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.h +++ b/cpp-ch/local-engine/Common/GlutenConfig.h @@ -142,4 +142,13 @@ struct GlutenJobSchedulerConfig static GlutenJobSchedulerConfig loadFromContext(const DB::ContextPtr & context); }; + +struct MergeTreeCacheConfig +{ + inline static const String ENABLE_DATA_PREFETCH = "enable_data_prefetch"; + + bool enable_data_prefetch = true; + + static MergeTreeCacheConfig loadFromContext(const DB::ContextPtr & context); +}; } diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp index 82afeb85e2b8..5b1fe63a09c5 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp @@ -31,8 +31,8 @@ bool isMetaDataFile(const std::string & path) void CompactObjectStorageDiskTransaction::commit() { auto metadata_tx = disk.getMetadataStorage()->createTransaction(); - std::filesystem::path data_path = std::filesystem::path(prefix_path) / "data.bin"; - std::filesystem::path meta_path = std::filesystem::path(prefix_path) / "meta.bin"; + std::filesystem::path data_path = std::filesystem::path(prefix_path) / PART_DATA_FILE_NAME; + std::filesystem::path meta_path = std::filesystem::path(prefix_path) / PART_META_FILE_NAME; auto object_storage = disk.getObjectStorage(); auto data_key = object_storage->generateObjectKeyForPath(data_path, std::nullopt); diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h index e15c362f304a..becb5371aad2 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h @@ -34,6 +34,9 @@ namespace local_engine class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { public: + static inline const String PART_DATA_FILE_NAME = "part_data.gluten"; + static inline const String PART_META_FILE_NAME = "part_meta.gluten"; + explicit CompactObjectStorageDiskTransaction(DB::IDisk & disk_, const DB::DiskPtr tmp_) : disk(disk_), tmp_data(tmp_) { diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp index e2ba48e9d272..3218db2741d4 100644 --- a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp @@ -88,7 +88,9 @@ Task CacheManager::cachePart( job_context.table.parts.clear(); job_context.table.parts.push_back(part); job_context.table.snapshot_id = ""; - Task task = [job_detail = job_context, context = this->context, read_columns = columns, only_meta_cache]() + MergeTreeCacheConfig config = MergeTreeCacheConfig::loadFromContext(context); + Task task = [job_detail = job_context, context = this->context, read_columns = columns, only_meta_cache, + prefetch_data = config.enable_data_prefetch]() { try { @@ -106,6 +108,9 @@ Task CacheManager::cachePart( job_detail.table.parts.front().name); return; } + // prefetch part data + if (prefetch_data) + storage->prefetchPartDataFile({job_detail.table.parts.front().name}); auto storage_snapshot = std::make_shared(*storage, storage->getInMemoryMetadataPtr()); NamesAndTypesList names_and_types_list; diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.h b/cpp-ch/local-engine/Storages/Cache/CacheManager.h index 8fd26d249abc..c44026ce0bbc 100644 --- a/cpp-ch/local-engine/Storages/Cache/CacheManager.h +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.h @@ -29,7 +29,7 @@ struct MergeTreePart; struct MergeTreeTableInstance; /*** - * Manage the cache of the MergeTree, mainly including meta.bin, data.bin, metadata.gluten + * Manage the cache of the MergeTree, mainly including part_data.gluten, part_meta.gluten, metadata.gluten */ class CacheManager { diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 45be9dcf7442..15da00fbee74 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -16,6 +16,7 @@ */ #include "SparkStorageMergeTree.h" +#include #include #include #include @@ -159,27 +160,36 @@ SparkStorageMergeTree::SparkStorageMergeTree( std::atomic SparkStorageMergeTree::part_num; -void SparkStorageMergeTree::prefetchMetaDataFile(std::unordered_set parts) const +void SparkStorageMergeTree::prefetchPartDataFile(const std::unordered_set& parts) const +{ + prefetchPartFiles(parts, CompactObjectStorageDiskTransaction::PART_DATA_FILE_NAME); +} + +void SparkStorageMergeTree::prefetchPartFiles(const std::unordered_set& parts, String file_name) const { auto disk = getDisks().front(); if (!disk->isRemote()) return; - std::vector meta_paths; - std::ranges::for_each(parts, [&](const String & name) { meta_paths.emplace_back(fs::path(relative_data_path) / name / "meta.bin"); }); + std::vector data_paths; + std::ranges::for_each(parts, [&](const String & name) { data_paths.emplace_back(fs::path(relative_data_path) / name / file_name); }); auto read_settings = ReadSettings{}; - // read_settings.enable_filesystem_cache = false; read_settings.remote_fs_method = RemoteFSReadMethod::read; - for (const auto & meta_path : meta_paths) + for (const auto & data_path : data_paths) { - if (!disk->existsDirectory(meta_path)) + if (!disk->existsFile(data_path)) continue; - - auto in = disk->readFile(meta_path, read_settings); + LOG_DEBUG(log, "Prefetching part file {}", data_path); + auto in = disk->readFile(data_path, read_settings); String ignore_data; readStringUntilEOF(ignore_data, *in); } } +void SparkStorageMergeTree::prefetchMetaDataFile(const std::unordered_set& parts) const +{ + prefetchPartFiles(parts, CompactObjectStorageDiskTransaction::PART_META_FILE_NAME); +} + std::vector SparkStorageMergeTree::loadDataPartsWithNames(const std::unordered_set & parts) { Stopwatch watch; diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h index cec1597eab08..237cf6919208 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h @@ -71,6 +71,7 @@ class SparkStorageMergeTree : public MergeTreeData std::map getUnfinishedMutationCommands() const override; std::vector loadDataPartsWithNames(const std::unordered_set & parts); void removePartFromMemory(const MergeTreeData::DataPart & part_to_detach); + void prefetchPartDataFile(const std::unordered_set& parts) const; MergeTreeDataSelectExecutor reader; MergeTreeDataMergerMutator merger_mutator; @@ -91,7 +92,8 @@ class SparkStorageMergeTree : public MergeTreeData static std::atomic part_num; SimpleIncrement increment; - void prefetchMetaDataFile(std::unordered_set parts) const; + void prefetchPartFiles(const std::unordered_set& parts, String file_name) const; + void prefetchMetaDataFile(const std::unordered_set& parts) const; void startBackgroundMovesIfNeeded() override; std::unique_ptr getDefaultSettings() const override; LoadPartResult loadDataPart( From 49e1ca5a8ce556f2e661038e487b5823823facb8 Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Wed, 6 Nov 2024 15:04:50 +0800 Subject: [PATCH 027/211] [GLUTEN-7807] Bind attr with name if its exprId is not found in transforming relation (#7819) --- .../sql/execution/ColumnarBuildSideRelation.scala | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala index feaf72f64fb2..d5c848dd9700 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala @@ -25,7 +25,7 @@ import org.apache.gluten.utils.ArrowAbiUtil import org.apache.gluten.vectorized.{ColumnarBatchSerializerJniWrapper, NativeColumnarToRowJniWrapper} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.joins.BuildSideRelation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.utils.SparkArrowUtil @@ -103,6 +103,18 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra var closed = false + val exprIds = output.map(_.exprId) + val projExpr = key.transformDown { + case attr: AttributeReference if !exprIds.contains(attr.exprId) => + val i = output.count(_.name == attr.name) + if (i != 1) { + throw new IllegalArgumentException(s"Only one attr with the same name is supported: $key") + } else { + output.find(_.name == attr.name).get + } + } + val proj = UnsafeProjection.create(Seq(projExpr), output) + // Convert columnar to Row. val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime) val c2rId = jniWrapper.nativeColumnarToRowInit() @@ -141,7 +153,6 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra ColumnarBatches.getNativeHandle(batch), 0) batch.close() - val proj = UnsafeProjection.create(Seq(key), output) new Iterator[InternalRow] { var rowId = 0 From b19cbd1311e7fe8c96d85ccfb8d47b82ed0479a1 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Wed, 6 Nov 2024 01:20:11 -0600 Subject: [PATCH 028/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241106) (#7824) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241106) * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/70806 * set 'tmp_path' related to spark version for avoiding conflict --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- .../GlutenClickHouseWholeStageTransformerSuite.scala | 1 + cpp-ch/clickhouse.version | 4 ++-- .../Storages/MergeTree/SparkStorageMergeTree.cpp | 7 +++++-- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index 272323c48732..5c6f669198c2 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -83,6 +83,7 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu .set("spark.sql.warehouse.dir", warehouse) .setCHConfig("user_defined_path", "/tmp/user_defined") .setCHConfig("path", UTSystemParameters.diskOutputDataPath) + .setCHConfig("tmp_path", s"/tmp/libch/$SPARK_DIR_NAME") if (UTSystemParameters.testMergeTreeOnObjectStorage) { conf .set("spark.hadoop.fs.s3a.access.key", S3_ACCESS_KEY) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index a4dd7eb5f2b0..1a089bdae0d0 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241105 -CH_COMMIT=500e1e35c0b \ No newline at end of file +CH_BRANCH=rebase_ch/20241106 +CH_COMMIT=32ca18ed214 \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 15da00fbee74..d44873196b93 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -224,7 +224,10 @@ MergeTreeData::LoadPartResult SparkStorageMergeTree::loadDataPart( try { - res.part = getDataPartBuilder(part_name, single_disk_volume, part_name).withPartInfo(part_info).withPartFormatFromDisk().build(); + res.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getContext()->getReadSettings()) + .withPartInfo(part_info) + .withPartFormatFromDisk() + .build(); } catch (...) { @@ -439,7 +442,7 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( VolumePtr volume = data.getStoragePolicy()->getVolume(0); VolumePtr data_part_volume = std::make_shared(volume->getName(), volume->getDisk(), volume->max_data_part_size); - auto new_data_part = data.getDataPartBuilder(part_dir, data_part_volume, part_dir) + auto new_data_part = data.getDataPartBuilder(part_dir, data_part_volume, part_dir, context->getReadSettings()) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) .build(); From 52dd83c541caa51c603ca439a72f0865bdb53ec7 Mon Sep 17 00:00:00 2001 From: Shuai li Date: Wed, 6 Nov 2024 16:17:20 +0800 Subject: [PATCH 029/211] [GLUTEN-7795][CH] Add backend task id log (#7801) * fix2 * Add task id * add more task id log * fix ci error * fix ci build error * fix review * fix rebase --- .../apache/gluten/memory/CHThreadGroup.java | 14 +++++++---- cpp-ch/local-engine/Common/QueryContext.cpp | 22 ++++++++++++++---- cpp-ch/local-engine/Common/QueryContext.h | 5 ++-- .../Parser/SerializedPlanParser.cpp | 23 ++++++++++--------- .../Parser/SerializedPlanParser.h | 2 ++ cpp-ch/local-engine/local_engine_jni.cpp | 5 ++-- 6 files changed, 47 insertions(+), 24 deletions(-) diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java index 119dc61893d9..36f7c388baee 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java @@ -28,7 +28,13 @@ public class CHThreadGroup implements TaskResource { */ public static void registerNewThreadGroup() { if (TaskResources.isResourceRegistered(CHThreadGroup.class.getName())) return; - CHThreadGroup group = new CHThreadGroup(); + + String taskId = ""; + if (TaskResources.getLocalTaskContext() != null) { + taskId = String.valueOf(TaskResources.getLocalTaskContext().taskAttemptId()); + } + + CHThreadGroup group = new CHThreadGroup(taskId); TaskResources.addResource(CHThreadGroup.class.getName(), group); TaskContext.get() .addTaskCompletionListener( @@ -40,8 +46,8 @@ public static void registerNewThreadGroup() { private long thread_group_id = 0; private long peak_memory = -1; - private CHThreadGroup() { - thread_group_id = createThreadGroup(); + private CHThreadGroup(String taskId) { + thread_group_id = createThreadGroup(taskId); } public long getPeakMemory() { @@ -69,7 +75,7 @@ public String resourceName() { return "CHThreadGroup"; } - private static native long createThreadGroup(); + private static native long createThreadGroup(String taskId); private static native long threadGroupPeakMemory(long id); diff --git a/cpp-ch/local-engine/Common/QueryContext.cpp b/cpp-ch/local-engine/Common/QueryContext.cpp index 4992ba0a455c..eca9ad5b18de 100644 --- a/cpp-ch/local-engine/Common/QueryContext.cpp +++ b/cpp-ch/local-engine/Common/QueryContext.cpp @@ -44,6 +44,7 @@ struct QueryContext::Data std::shared_ptr thread_status; std::shared_ptr thread_group; ContextMutablePtr query_context; + String task_id; static DB::ContextMutablePtr global_context; static SharedContextHolder shared_context; @@ -83,11 +84,12 @@ DB::ContextPtr QueryContext::globalContext() return Data::global_context; } -int64_t QueryContext::initializeQuery() +int64_t QueryContext::initializeQuery(const String & task_id) { std::shared_ptr query_context = std::make_shared(); query_context->query_context = Context::createCopy(globalContext()); query_context->query_context->makeQueryContext(); + query_context->task_id = task_id; // empty input will trigger random query id to be set // FileCache will check if query id is set to decide whether to skip cache or not @@ -95,7 +97,7 @@ int64_t QueryContext::initializeQuery() // // Notice: // this generated random query id a qualified global queryid for the spark query - query_context->query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4())); + query_context->query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4()) + "_" + task_id); auto config = MemoryConfig::loadFromContext(query_context->query_context); query_context->thread_status = std::make_shared(false); query_context->thread_group = std::make_shared(query_context->query_context); @@ -124,14 +126,24 @@ std::shared_ptr QueryContext::currentThreadGroup() throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found."); } -void QueryContext::logCurrentPerformanceCounters(ProfileEvents::Counters & counters) const +String QueryContext::currentTaskIdOrEmpty() +{ + if (auto thread_group = CurrentThread::getGroup()) + { + const int64_t id = reinterpret_cast(thread_group.get()); + return query_map_.get(id)->task_id; + } + return ""; +} + +void QueryContext::logCurrentPerformanceCounters(ProfileEvents::Counters & counters, const String & task_id) const { if (!CurrentThread::getGroup()) return; if (logger_->information()) { std::ostringstream msg; - msg << "\n---------------------Task Performance Counters-----------------------------\n"; + msg << "\n---------------------Task Performance Counters(" << task_id << ")-----------------------------\n"; for (ProfileEvents::Event event = ProfileEvents::Event(0); event < counters.num_counters; event++) { const auto * name = ProfileEvents::getName(event); @@ -167,7 +179,7 @@ void QueryContext::finalizeQuery(int64_t id) if (currentThreadGroupMemoryUsage() > 2_MiB) LOG_WARNING(logger_, "{} bytes memory didn't release, There may be a memory leak!", currentThreadGroupMemoryUsage()); - logCurrentPerformanceCounters(context->thread_group->performance_counters); + logCurrentPerformanceCounters(context->thread_group->performance_counters, context->task_id); context->thread_status->detachFromGroup(); context->thread_group.reset(); context->thread_status.reset(); diff --git a/cpp-ch/local-engine/Common/QueryContext.h b/cpp-ch/local-engine/Common/QueryContext.h index 821144f5fcb6..6ced1f632095 100644 --- a/cpp-ch/local-engine/Common/QueryContext.h +++ b/cpp-ch/local-engine/Common/QueryContext.h @@ -40,10 +40,11 @@ class QueryContext static QueryContext instance; return instance; } - int64_t initializeQuery(); + int64_t initializeQuery(const String & task_id); DB::ContextMutablePtr currentQueryContext(); + String currentTaskIdOrEmpty(); static std::shared_ptr currentThreadGroup(); - void logCurrentPerformanceCounters(ProfileEvents::Counters & counters) const; + void logCurrentPerformanceCounters(ProfileEvents::Counters & counters, const String & task_id) const; size_t currentPeakMemory(int64_t id); void finalizeQuery(int64_t id); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 74c1d350014d..4c1d5b902da5 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -62,6 +62,7 @@ #include #include #include +#include #include #include @@ -101,7 +102,7 @@ std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) +void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) const { if (root_rel.root().names_size()) { @@ -110,8 +111,8 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel const auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); if (cols.getNames().size() != static_cast(root_rel.root().names_size())) { - debug::dumpPlan(*query_plan, true); - debug::dumpMessage(root_rel, "substrait::PlanRel", true); + debug::dumpPlan(*query_plan, true, log); + debug::dumpMessage(root_rel, "substrait::PlanRel", true, log); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan name size {}.", @@ -134,8 +135,8 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel const auto & original_cols = original_header.getColumnsWithTypeAndName(); if (static_cast(output_schema.types_size()) != original_cols.size()) { - debug::dumpPlan(*query_plan, true); - debug::dumpMessage(root_rel, "substrait::PlanRel", true); + debug::dumpPlan(*query_plan, true, log); + debug::dumpMessage(root_rel, "substrait::PlanRel", true, log); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan output schema size {}, subtrait plan name size {}.", @@ -183,7 +184,7 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) { - debug::dumpMessage(plan, "substrait::Plan"); + debug::dumpMessage(plan, "substrait::Plan", false, log); //parseExtensions(plan.extensions()); if (plan.relations_size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "too many relations found"); @@ -204,7 +205,7 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) PlanUtil::checkOuputType(*query_plan); #endif - debug::dumpPlan(*query_plan); + debug::dumpPlan(*query_plan, false, log); return query_plan; } @@ -323,7 +324,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla } catch (...) { - LOG_ERROR(getLogger("SerializedPlanParser"), "Invalid plan:\n{}", PlanUtil::explainPlan(*query_plan)); + LOG_ERROR(log, "Invalid plan:\n{}", PlanUtil::explainPlan(*query_plan)); throw; } @@ -332,10 +333,9 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla assert(root_rel.has_root()); if (root_rel.root().input().has_write()) addSinkTransform(parser_context->queryContext(), root_rel.root().input().write(), builder); - auto * logger = &Poco::Logger::get("SerializedPlanParser"); - LOG_INFO(logger, "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); + LOG_INFO(log, "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); LOG_DEBUG( - logger, + log, "clickhouse plan [optimization={}]:\n{}", settings[Setting::query_plan_enable_optimizations], PlanUtil::explainPlan(*query_plan)); @@ -347,6 +347,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla SerializedPlanParser::SerializedPlanParser(ParserContextPtr parser_context_) : parser_context(parser_context_) { context = parser_context->queryContext(); + log = getLogger("SerializedPlanParser(" + QueryContext::instance().currentTaskIdOrEmpty() + ")"); } NonNullableColumnsResolver::NonNullableColumnsResolver( diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index 201fc46b2e9c..5fd84433132e 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -118,7 +118,9 @@ class SerializedPlanParser private: DB::QueryPlanPtr parseOp(const substrait::Rel & rel, std::list & rel_stack); + void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) const; + LoggerPtr log; std::vector input_iters; std::vector split_infos; int split_info_index = 0; diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 8ff8a866b7ca..952f6e50e94d 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -1310,10 +1310,11 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_SimpleExpressionEval_nativeNex LOCAL_ENGINE_JNI_METHOD_END(env, -1) } -JNIEXPORT jlong Java_org_apache_gluten_memory_CHThreadGroup_createThreadGroup(JNIEnv * env, jclass) +JNIEXPORT jlong Java_org_apache_gluten_memory_CHThreadGroup_createThreadGroup(JNIEnv * env, jclass, jstring task_id_) { LOCAL_ENGINE_JNI_METHOD_START - return local_engine::QueryContext::instance().initializeQuery(); + auto task_id = jstring2string(env, task_id_); + return local_engine::QueryContext::instance().initializeQuery(task_id); LOCAL_ENGINE_JNI_METHOD_END(env, 0l) } From 3e160bf0080c5ca66c5ff16792feba5e6366bbc9 Mon Sep 17 00:00:00 2001 From: lgbo Date: Wed, 6 Nov 2024 16:18:52 +0800 Subject: [PATCH 030/211] [GLUTEN-7647][CH] Lazy expand for aggregation (#7649) * add lazy expand rule * stage * stage * apply rule before query stage * stage * update * 1028 * update * refactor * refactor 1101 * fixed some bugs --- .../backendsapi/clickhouse/CHBackend.scala | 12 + .../backendsapi/clickhouse/CHRuleApi.scala | 1 + .../extension/LazyAggregateExpandRule.scala | 370 ++++++++++++++++++ ...enClickHouseTPCHSaltNullParquetSuite.scala | 45 +++ .../Operator/AdvancedExpandStep.cpp | 354 +++++++++++++++++ .../Operator/AdvancedExpandStep.h | 97 +++++ .../Parser/RelParsers/AggregateRelParser.cpp | 3 +- .../Parser/RelParsers/ExpandRelParser.cpp | 97 ++++- .../Parser/RelParsers/ExpandRelParser.h | 12 +- 9 files changed, 984 insertions(+), 7 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala create mode 100644 cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp create mode 100644 cpp-ch/local-engine/Operator/AdvancedExpandStep.h diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index ba17d12ffa94..54ab38569bb8 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -353,6 +353,18 @@ object CHBackendSettings extends BackendSettingsApi with Logging { ) } + // It try to move the expand node after the pre-aggregate node. That is to make the plan from + // expand -> pre-aggregate -> shuffle -> final-aggregate + // to + // pre-aggregate -> expand -> shuffle -> final-aggregate + // It could reduce the overhead of pre-aggregate node. + def enableLazyAggregateExpand(): Boolean = { + SparkEnv.get.conf.getBoolean( + CHConf.runtimeConfig("enable_lazy_aggregate_expand"), + defaultValue = true + ) + } + override def enableNativeWriteFiles(): Boolean = { GlutenConfig.getConf.enableNativeWriter.getOrElse(false) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 470ece4037a5..4107844f322d 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -85,6 +85,7 @@ private object CHRuleApi { injector.injectTransform(_ => CollapseProjectExecTransformer) injector.injectTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session)) injector.injectTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session)) + injector.injectTransform(c => LazyAggregateExpandRule.apply(c.session)) injector.injectTransform( c => intercept( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala new file mode 100644 index 000000000000..e06503a5e1e6 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -0,0 +1,370 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings +import org.apache.gluten.execution._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types._ + +/* + * For aggregation with grouping sets, we need to expand the grouping sets + * to individual group by. + * 1. It need to make copies of the original data. + * 2. run the aggregation on the multi copied data. + * Both of these two are expensive. + * + * We could do this as following + * 1. Run the aggregation on full grouping keys. + * 2. Expand the aggregation result to the full grouping sets. + * 3. Run the aggregation on the expanded data. + * + * So the plan is transformed from + * expand -> partial aggregating -> shuffle -> final merge aggregating + * to + * partial aggregating -> expand -> shuffle -> final merge aggregating + * + * Notice: + * If the aggregation involves distinct, we can't do this optimization. + */ + +case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan] with Logging { + override def apply(plan: SparkPlan): SparkPlan = { + logDebug(s"xxx enable lazy aggregate expand: ${CHBackendSettings.enableLazyAggregateExpand}") + if (!CHBackendSettings.enableLazyAggregateExpand) { + return plan + } + plan.transformUp { + case shuffle @ ColumnarShuffleExchangeExec( + HashPartitioning(hashExpressions, _), + CHHashAggregateExecTransformer( + _, + groupingExpressions, + aggregateExpressions, + _, + _, + resultExpressions, + ExpandExecTransformer(projections, output, child)), + _, + _, + _ + ) => + logDebug(s"xxx match plan:$shuffle") + val partialAggregate = shuffle.child.asInstanceOf[CHHashAggregateExecTransformer] + val expand = partialAggregate.child.asInstanceOf[ExpandExecTransformer] + logDebug( + s"xxx partialAggregate: groupingExpressions:" + + s"${partialAggregate.groupingExpressions}\n" + + s"aggregateAttributes:${partialAggregate.aggregateAttributes}\n" + + s"aggregateExpressions:${partialAggregate.aggregateExpressions}\n" + + s"resultExpressions:${partialAggregate.resultExpressions}") + if (doValidation(partialAggregate, expand, shuffle)) { + + val attributesToReplace = buildReplaceAttributeMap(expand) + logDebug(s"xxx attributesToReplace: $attributesToReplace") + + val newPartialAggregate = buildAheadAggregateExec( + partialAggregate, + expand, + attributesToReplace + ) + + val newExpand = buildPostExpandExec( + expand, + partialAggregate, + newPartialAggregate, + attributesToReplace + ) + + val newShuffle = shuffle.copy(child = newExpand) + logDebug(s"xxx new plan: $newShuffle") + newShuffle + } else { + shuffle + } + case shuffle @ ColumnarShuffleExchangeExec( + HashPartitioning(hashExpressions, _), + CHHashAggregateExecTransformer( + _, + groupingExpressions, + aggregateExpressions, + _, + _, + resultExpressions, + FilterExecTransformer(_, ExpandExecTransformer(projections, output, child))), + _, + _, + _ + ) => + val partialAggregate = shuffle.child.asInstanceOf[CHHashAggregateExecTransformer] + val filter = partialAggregate.child.asInstanceOf[FilterExecTransformer] + val expand = filter.child.asInstanceOf[ExpandExecTransformer] + logDebug( + s"xxx partialAggregate: groupingExpressions:" + + s"${partialAggregate.groupingExpressions}\n" + + s"aggregateAttributes:${partialAggregate.aggregateAttributes}\n" + + s"aggregateExpressions:${partialAggregate.aggregateExpressions}\n" + + s"resultExpressions:${partialAggregate.resultExpressions}") + if (doValidation(partialAggregate, expand, shuffle)) { + val attributesToReplace = buildReplaceAttributeMap(expand) + logDebug(s"xxx attributesToReplace: $attributesToReplace") + + val newPartialAggregate = buildAheadAggregateExec( + partialAggregate, + expand, + attributesToReplace + ) + + val newExpand = buildPostExpandExec( + expand, + partialAggregate, + newPartialAggregate, + attributesToReplace + ) + + val newFilter = filter.copy(child = newExpand) + + val newShuffle = shuffle.copy(child = newFilter) + logDebug(s"xxx new plan: $newShuffle") + newShuffle + + } else { + shuffle + } + } + } + + // Just enable for simple cases. Some of cases that are not supported: + // 1. select count(a),count(b), count(1), count(distinct(a)), count(distinct(b)) from values + // (1, null), (2,2) as data(a,b); + // 2. select n_name, count(distinct n_regionkey) as col1, + // count(distinct concat(n_regionkey, n_nationkey)) as col2 from + // nation group by n_name; + def doValidation( + aggregate: CHHashAggregateExecTransformer, + expand: ExpandExecTransformer, + shuffle: ColumnarShuffleExchangeExec): Boolean = { + // all grouping keys must be attribute references + val expandOutputAttributes = expand.child.output.toSet + if ( + !aggregate.groupingExpressions.forall( + e => e.isInstanceOf[Attribute] || e.isInstanceOf[Literal]) + ) { + logDebug(s"xxx Not all grouping expression are attribute references") + return false + } + // all shuffle keys must be attribute references + if ( + !shuffle.outputPartitioning + .asInstanceOf[HashPartitioning] + .expressions + .forall(e => e.isInstanceOf[Attribute] || e.isInstanceOf[Literal]) + ) { + logDebug(s"xxx Not all shuffle hash expression are attribute references") + return false + } + + // 1. for safty, we don't enbale this optimization for all aggregate functions. + // 2. if any aggregate function uses attributes which is not from expand's child, we don't + // enable this + if ( + !aggregate.aggregateExpressions.forall( + e => + isValidAggregateFunction(e) && + e.aggregateFunction.references.forall(expandOutputAttributes.contains(_))) + ) { + logDebug(s"xxx Some aggregate functions are not supported") + return false + } + + // get the group id's position in the expand's output + val gidIndex = findGroupingIdIndex(expand) + gidIndex != -1 + } + + // group id column doesn't have a fixed position, so we need to find it. + def findGroupingIdIndex(expand: ExpandExecTransformer): Int = { + def isValidGroupIdColumn(e: Expression, gids: Set[Long]): Long = { + if (!e.isInstanceOf[Literal]) { + return -1 + } + val literalValue = e.asInstanceOf[Literal].value + e.dataType match { + case _: LongType => + if (gids.contains(literalValue.asInstanceOf[Long])) { + -1 + } else { + literalValue.asInstanceOf[Long] + } + case _: IntegerType => + if (gids.contains(literalValue.asInstanceOf[Int].toLong)) { + -1 + } else { + literalValue.asInstanceOf[Int].toLong + } + case _ => -1 + } + } + + var groupIdIndexes = Seq[Int]() + for (col <- 0 until expand.output.length) { + val expandCol = expand.projections(0)(col) + // gids should be unique + var gids = Set[Long]() + if (isValidGroupIdColumn(expandCol, gids) != -1) { + if ( + expand.projections.forall { + projection => + val res = isValidGroupIdColumn(projection(col), gids) + gids += res + res != -1 + } + ) { + groupIdIndexes +:= col + } + } + } + if (groupIdIndexes.length == 1) { + logDebug(s"xxx gid is at pos ${groupIdIndexes(0)}") + groupIdIndexes(0) + } else { + -1 + } + } + + // Some of aggregate functions' output columns are not consistent with the output of gluten. + // - average: in partial aggregation, the outputs are sum and count, but gluten only generates one + // column, avg. + // - sum: if the input's type is decimal, the output are sum and isEmpty, but gluten doesn't use + // the isEmpty column. + def isValidAggregateFunction(aggregateExpression: AggregateExpression): Boolean = { + if (aggregateExpression.filter.isDefined) { + return false + } + aggregateExpression.aggregateFunction match { + case _: Count => true + case _: Max => true + case _: Min => true + case sum: Sum => !sum.dataType.isInstanceOf[DecimalType] + case _ => false + } + } + + def getReplaceAttribute( + toReplace: Attribute, + attributesToReplace: Map[Attribute, Attribute]): Attribute = { + attributesToReplace.getOrElse(toReplace, toReplace) + } + + def buildReplaceAttributeMap(expand: ExpandExecTransformer): Map[Attribute, Attribute] = { + var fullExpandProjection = Seq[Expression]() + for (i <- 0 until expand.projections(0).length) { + val attr = expand.projections.find(x => x(i).isInstanceOf[Attribute]) match { + case Some(projection) => projection(i).asInstanceOf[Attribute] + case None => null + } + fullExpandProjection = fullExpandProjection :+ attr + } + + var attributeMap = Map[Attribute, Attribute]() + for (i <- 0 until expand.output.length) { + if (fullExpandProjection(i).isInstanceOf[Attribute]) { + attributeMap += (expand.output(i) -> fullExpandProjection(i).asInstanceOf[Attribute]) + } + } + attributeMap + } + + def buildPostExpandProjections( + originalExpandProjections: Seq[Seq[Expression]], + originalExpandOutput: Seq[Attribute], + newExpandOutput: Seq[Attribute]): Seq[Seq[Expression]] = { + val newExpandProjections = originalExpandProjections.map { + projection => + newExpandOutput.map { + attr => + val index = originalExpandOutput.indexWhere(_.semanticEquals(attr)) + if (index != -1) { + projection(index) + } else { + attr + } + } + } + newExpandProjections + } + + // 1. make expand's child be aggregate's child + // 2. replace the attributes in groupingExpressions and resultExpressions as needed + def buildAheadAggregateExec( + partialAggregate: CHHashAggregateExecTransformer, + expand: ExpandExecTransformer, + attributesToReplace: Map[Attribute, Attribute]): SparkPlan = { + val groupIdAttribute = expand.output(findGroupingIdIndex(expand)) + + // New grouping expressions should include the group id column + val groupingExpressions = + partialAggregate.groupingExpressions + .filter( + e => e.toAttribute != groupIdAttribute && attributesToReplace.contains(e.toAttribute)) + .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) + .distinct + logDebug( + s"xxx newGroupingExpresion: $groupingExpressions,\n" + + s"groupingExpressions: ${partialAggregate.groupingExpressions}") + + // Remove group id column from result expressions + val resultExpressions = partialAggregate.resultExpressions + .filter(_.toAttribute != groupIdAttribute) + .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) + logDebug( + s"xxx newResultExpressions: $resultExpressions\n" + + s"resultExpressions:${partialAggregate.resultExpressions}") + partialAggregate.copy( + groupingExpressions = groupingExpressions, + resultExpressions = resultExpressions, + child = expand.child) + } + + def buildPostExpandExec( + expand: ExpandExecTransformer, + partialAggregate: CHHashAggregateExecTransformer, + child: SparkPlan, + attributesToReplace: Map[Attribute, Attribute]): SparkPlan = { + // The output of the native plan is not completely consistent with Spark. + val aggregateOutput = partialAggregate.output + logDebug(s"xxx aggregateResultAttributes: ${partialAggregate.aggregateResultAttributes}") + logDebug(s"xxx aggregateOutput: $aggregateOutput") + + val expandProjections = buildPostExpandProjections( + expand.projections, + expand.output, + aggregateOutput + ) + logDebug(s"xxx expandProjections: $expandProjections\nprojections:${expand.projections}") + ExpandExecTransformer(expandProjections, aggregateOutput, child) + } + +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 739b040dba1d..40b704d2e8d7 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -3022,5 +3022,50 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr compareResultsAgainstVanillaSpark(query_sql, true, { _ => }) spark.sql("drop table test_tbl_7220") } + + test("GLLUTEN-7647 lazy expand") { + def checkLazyExpand(df: DataFrame): Unit = { + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: ExpandExecTransformer if (e.child.isInstanceOf[HashAggregateExecBaseTransformer]) => + e + } + assert(expands.size == 1) + } + var sql = + """ + |select n_regionkey, n_nationkey, + |sum(n_regionkey), count(n_name), max(n_regionkey), min(n_regionkey) + |from nation group by n_regionkey, n_nationkey with cube + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select n_regionkey, n_nationkey, sum(n_regionkey), count(distinct n_name) + |from nation group by n_regionkey, n_nationkey with cube + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select * from( + |select n_regionkey, n_nationkey, + |sum(n_regionkey), count(n_name), max(n_regionkey), min(n_regionkey) + |from nation group by n_regionkey, n_nationkey with cube + |) where n_regionkey != 0 + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select * from( + |select n_regionkey, n_nationkey, + |sum(n_regionkey), count(distinct n_name), max(n_regionkey), min(n_regionkey) + |from nation group by n_regionkey, n_nationkey with cube + |) where n_regionkey != 0 + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + } } // scalastyle:on line.size.limit diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp new file mode 100644 index 000000000000..6ac5f5fc8f8b --- /dev/null +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp @@ -0,0 +1,354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "AdvancedExpandStep.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +namespace Setting +{ +extern const SettingsUInt64 max_bytes_before_external_group_by; +extern const SettingsBool optimize_group_by_constant_keys; +extern const SettingsUInt64 min_free_disk_space_for_temporary_data; +extern const SettingsMaxThreads max_threads; +extern const SettingsBool empty_result_for_aggregation_by_empty_set; +extern const SettingsUInt64 group_by_two_level_threshold_bytes; +extern const SettingsOverflowModeGroupBy group_by_overflow_mode; +extern const SettingsUInt64 max_rows_to_group_by; +extern const SettingsBool enable_memory_bound_merging_of_aggregation_results; +extern const SettingsUInt64 aggregation_in_order_max_block_bytes; +extern const SettingsUInt64 group_by_two_level_threshold; +extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; +extern const SettingsMaxThreads max_threads; +extern const SettingsUInt64 max_block_size; +} +} + +namespace local_engine +{ + +static DB::ITransformingStep::Traits getTraits() +{ + return DB::ITransformingStep::Traits{ + { + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = false, + }}; +} + +AdvancedExpandStep::AdvancedExpandStep( + DB::ContextPtr context_, + const DB::Block & input_header_, + size_t grouping_keys_, + const DB::AggregateDescriptions & aggregate_descriptions_, + const ExpandField & project_set_exprs_) + : DB::ITransformingStep(input_header_, buildOutputHeader(input_header_, project_set_exprs_), getTraits()) + , context(context_) + , grouping_keys(grouping_keys_) + , aggregate_descriptions(aggregate_descriptions_) + , project_set_exprs(project_set_exprs_) +{ +} + +DB::Block AdvancedExpandStep::buildOutputHeader(const DB::Block &, const ExpandField & project_set_exprs_) +{ + DB::ColumnsWithTypeAndName cols; + const auto & types = project_set_exprs_.getTypes(); + const auto & names = project_set_exprs_.getNames(); + + chassert(names.size() == types.size()); + + for (size_t i = 0; i < project_set_exprs_.getExpandCols(); ++i) + cols.emplace_back(DB::ColumnWithTypeAndName(types[i], names[i])); + + return DB::Block(std::move(cols)); +} + +void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & pipeline_settings) +{ + const auto & settings = context->getSettingsRef(); + DB::Names aggregate_grouping_keys; + for (size_t i = 0; i < output_header->columns(); ++i) + { + const auto & col = output_header->getByPosition(i); + if (typeid_cast(col.column.get())) + break; + aggregate_grouping_keys.push_back(col.name); + } + DB::Aggregator::Params params( + aggregate_grouping_keys, + aggregate_descriptions, + false, + settings[DB::Setting::max_rows_to_group_by], + settings[DB::Setting::group_by_overflow_mode], + settings[DB::Setting::group_by_two_level_threshold], + settings[DB::Setting::group_by_two_level_threshold_bytes], + settings[DB::Setting::max_bytes_before_external_group_by], + settings[DB::Setting::empty_result_for_aggregation_by_empty_set], + context->getTempDataOnDisk(), + settings[DB::Setting::max_threads], + settings[DB::Setting::min_free_disk_space_for_temporary_data], + true, + 3, + PODArrayUtil::adjustMemoryEfficientSize(settings[DB::Setting::max_block_size]), + /*enable_prefetch*/ true, + /*only_merge*/ false, + settings[DB::Setting::optimize_group_by_constant_keys], + settings[DB::Setting::min_hit_rate_to_use_consecutive_keys_optimization], + /*StatsCollectingParams*/ {}); + + auto input_header = input_headers.front(); + auto build_transform = [&](DB::OutputPortRawPtrs outputs) + { + DB::Processors new_processors; + for (auto & output : outputs) + { + auto expand_processor + = std::make_shared(input_header, *output_header, grouping_keys, project_set_exprs); + DB::connect(*output, expand_processor->getInputs().front()); + new_processors.push_back(expand_processor); + + auto expand_output_header = expand_processor->getOutputs().front().getHeader(); + + auto transform_params = std::make_shared(expand_output_header, params, false); + auto aggregate_processor + = std::make_shared(expand_output_header, transform_params, context, false, false); + DB::connect(expand_processor->getOutputs().back(), aggregate_processor->getInputs().front()); + new_processors.push_back(aggregate_processor); + auto aggregate_output_header = aggregate_processor->getOutputs().front().getHeader(); + + auto resize_processor = std::make_shared(expand_output_header, 2, 1); + DB::connect(aggregate_processor->getOutputs().front(), resize_processor->getInputs().front()); + DB::connect(expand_processor->getOutputs().front(), resize_processor->getInputs().back()); + new_processors.push_back(resize_processor); + } + return new_processors; + }; + pipeline.transform(build_transform); +} + +void AdvancedExpandStep::describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const +{ + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); +} + +void AdvancedExpandStep::updateOutputHeader() +{ + output_header = buildOutputHeader(input_headers.front(), project_set_exprs); +} + +/// It has two output ports. The 1st output port is for high cardinality data, the 2nd output port is for +/// low cardinality data. +AdvancedExpandTransform::AdvancedExpandTransform( + const DB::Block & input_header_, const DB::Block & output_header_, size_t grouping_keys_, const ExpandField & project_set_exprs_) + : DB::IProcessor({input_header_}, {output_header_, output_header_}) + , grouping_keys(grouping_keys_) + , project_set_exprs(project_set_exprs_) + , input_header(input_header_) +{ + for (size_t i = 0; i < project_set_exprs.getKinds().size(); ++i) + { + is_low_cardinality_expand.push_back(true); + } + + for (auto & port : outputs) + { + output_ports.push_back(&port); + } +} + +DB::IProcessor::Status AdvancedExpandTransform::prepare() +{ + auto & input = inputs.front(); + + if (isCancelled() || output_ports[0]->isFinished() || output_ports[1]->isFinished()) + { + input.close(); + output_ports[0]->finish(); + output_ports[1]->finish(); + return Status::Finished; + } + + if (has_output) + { + auto & output_port = *output_ports[is_low_cardinality_expand[expand_expr_iterator - 1]]; + if (output_port.canPush()) + { + output_port.push(std::move(output_chunk)); + has_output = false; + auto status = expand_expr_iterator >= project_set_exprs.getExpandRows() ? Status::NeedData : Status::Ready; + return status; + } + else + { + return Status::PortFull; + } + } + + if (!has_input) + { + if (input.isFinished()) + { + if (!cardinality_detect_blocks.empty()) + { + input_finished = true; + return Status::Ready; + } + else + { + output_ports[0]->finish(); + output_ports[1]->finish(); + return Status::Finished; + } + } + + input.setNeeded(); + if (!input.hasData()) + { + return Status::NeedData; + } + input_chunk = input.pull(true); + has_input = true; + expand_expr_iterator = 0; + } + + return Status::Ready; +} + +void AdvancedExpandTransform::work() +{ + if (!input_finished && cardinality_detect_rows < rows_for_detect_cardinality) + { + cardinality_detect_blocks.push_back(input_header.cloneWithColumns(input_chunk.detachColumns())); + cardinality_detect_rows += cardinality_detect_blocks.back().rows(); + has_input = false; + } + if ((input_finished || cardinality_detect_rows >= rows_for_detect_cardinality) && !cardinality_detect_blocks.empty()) + { + detectCardinality(); + } + else if (!input_finished && cardinality_detect_rows < rows_for_detect_cardinality) + return; + + /// The phase of detecting grouping keys' cardinality is finished here. + expandInputChunk(); +} + +void AdvancedExpandTransform::detectCardinality() +{ + DB::Block block = BlockUtil::concatenateBlocksMemoryEfficiently(std::move(cardinality_detect_blocks)); + std::vector is_col_low_cardinality; + for (size_t i = 0; i < grouping_keys; ++i) + { + DB::WeakHash32 hash(cardinality_detect_rows); + std::unordered_set distinct_ids; + const auto & data = hash.getData(); + for (size_t j = 0; j < cardinality_detect_rows; ++j) + distinct_ids.insert(data[j]); + size_t distinct_ids_cnt = distinct_ids.size(); + is_col_low_cardinality.push_back(distinct_ids.size() < 1000); + } + + for (size_t i = 0; i < project_set_exprs.getExpandRows(); ++i) + { + const auto & kinds = project_set_exprs.getKinds()[i]; + for (size_t k = 0; k < grouping_keys; ++k) + { + const auto & kind = kinds[k]; + if (kind == EXPAND_FIELD_KIND_SELECTION && !is_col_low_cardinality[k]) + { + is_low_cardinality_expand[i] = false; + break; + } + } + } + LOG_DEBUG(getLogger("AdvancedExpandTransform"), "Low cardinality expand: {}", fmt::join(is_low_cardinality_expand, ",")); + + input_chunk = DB::Chunk(block.getColumns(), block.rows()); + cardinality_detect_blocks.clear(); +} + +void AdvancedExpandTransform::expandInputChunk() +{ + const auto & input_columns = input_chunk.getColumns(); + const auto & types = project_set_exprs.getTypes(); + const auto & kinds = project_set_exprs.getKinds()[expand_expr_iterator]; + const auto & fields = project_set_exprs.getFields()[expand_expr_iterator]; + size_t rows = input_chunk.getNumRows(); + + DB::Columns columns(types.size()); + for (size_t col_i = 0; col_i < types.size(); ++col_i) + { + const auto & type = types[col_i]; + const auto & kind = kinds[col_i]; + const auto & field = fields[col_i]; + + if (kind == EXPAND_FIELD_KIND_SELECTION) + { + auto index = field.safeGet(); + const auto & input_column = input_columns[index]; + + DB::ColumnWithTypeAndName input_arg; + input_arg.column = input_column; + input_arg.type = input_header.getByPosition(index).type; + /// input_column maybe non-Nullable + columns[col_i] = DB::castColumn(input_arg, type); + } + else if (kind == EXPAND_FIELD_KIND_LITERAL) + { + /// Add const column with field value + auto column = type->createColumnConst(rows, field)->convertToFullColumnIfConst(); + columns[col_i] = std::move(column); + } + else + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown ExpandFieldKind {}", magic_enum::enum_name(kind)); + } + + output_chunk = DB::Chunk(std::move(columns), rows); + has_output = true; + + ++expand_expr_iterator; + has_input = expand_expr_iterator < project_set_exprs.getExpandRows(); +} +} diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.h b/cpp-ch/local-engine/Operator/AdvancedExpandStep.h new file mode 100644 index 000000000000..295084658531 --- /dev/null +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.h @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace local_engine +{ +// This step is used when we move the expand operator after the partial aggregator. +// To avoid increasing the overhead of shuffle when some of the grouping keys are high cardinality, we add an extra aggregate operator after +// this expand operator and aggregate the low cardinality grouping keys. +class AdvancedExpandStep : public DB::ITransformingStep +{ +public: + explicit AdvancedExpandStep( + DB::ContextPtr context_, + const DB::Block & input_header_, + size_t grouping_keys_, + const DB::AggregateDescriptions & aggregate_descriptions_, + const ExpandField & project_set_exprs_); + ~AdvancedExpandStep() override = default; + + String getName() const override { return "AdvancedExpandStep"; } + + void transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & settings) override; + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override; + + static DB::Block buildOutputHeader(const DB::Block & header, const ExpandField & project_set_exprs_); + +protected: + DB::ContextPtr context; + size_t grouping_keys; + DB::AggregateDescriptions aggregate_descriptions; + ExpandField project_set_exprs; + + void updateOutputHeader() override; +}; + +class AdvancedExpandTransform : public DB::IProcessor +{ +public: + using Status = DB::IProcessor::Status; + /// Need to ensure that the input header is [grouping keys] ++ [aggregation columns] + explicit AdvancedExpandTransform( + const DB::Block & inpput_header_, const DB::Block & output_header_, size_t goruping_keys_, const ExpandField & project_set_exprs_); + ~AdvancedExpandTransform() override = default; + + Status prepare() override; + void work() override; + String getName() const override { return "AdvancedExpandTransform"; } + +private: + size_t grouping_keys = 0; + ExpandField project_set_exprs; + DB::Block input_header; + bool has_input = false; + bool has_output = false; + size_t expand_expr_iterator = 0; + std::vector is_low_cardinality_expand; + std::vector approximate_grouping_keys; + size_t cardinality_detect_rows = 0; + std::vector cardinality_detect_blocks; + static constexpr size_t rows_for_detect_cardinality = 10000; + bool input_finished = false; + + std::vector output_ports; + + DB::Chunk input_chunk; + DB::Chunk output_chunk; + + void detectCardinality(); + void expandInputChunk(); +}; + +} diff --git a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp index 269d55e64560..6bc8c7e6e14a 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp @@ -525,8 +525,7 @@ void AggregateRelParser::addAggregatingStep() /// We cannot use streaming aggregating strategy in step3. Otherwise it will generate multiple blocks with same n_name in them. This /// will make the result for count(distinct(n_name)) wrong. step3 must finish all inputs before it puts any block into step4. /// So we introduce GraceAggregatingStep here, it can handle mass data with high cardinality. - auto aggregating_step - = std::make_unique(getContext(), plan->getCurrentHeader(), params, has_first_stage); + auto aggregating_step = std::make_unique(getContext(), plan->getCurrentHeader(), params, has_first_stage); steps.emplace_back(aggregating_step.get()); plan->addStep(std::move(aggregating_step)); } diff --git a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp index 8a64c445e7ab..f3d8b4ab11ec 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp @@ -15,11 +15,12 @@ * limitations under the License. */ #include "ExpandRelParser.h" +#include #include #include #include +#include #include -#include #include #include #include @@ -45,11 +46,17 @@ void updateType(DB::DataTypePtr & type, const DB::DataTypePtr & new_type) } } -DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list &) +DB::QueryPlanPtr +ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack) { - const auto & expand_rel = rel.expand(); - const auto & header = query_plan->getCurrentHeader(); + if (!isLazyAggregateExpand(rel.expand())) + return normalParse(std::move(query_plan), rel, rel_stack); + else + return lazyAggregateExpandParse(std::move(query_plan), rel, rel_stack); +} +ExpandField ExpandRelParser::buildExpandField(const DB::Block & header, const substrait::ExpandRel & expand_rel) +{ std::vector> expand_kinds; std::vector> expand_fields; std::vector types; @@ -82,6 +89,10 @@ DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const subst auto field = project_expr.selection().direct_reference().struct_field().field(); kinds.push_back(ExpandFieldKind::EXPAND_FIELD_KIND_SELECTION); fields.push_back(field); + if (field >= header.columns()) + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Field index out of range: {}, header: {}", field, header.dumpStructure()); + } updateType(types[i], header.getByPosition(field).type); const auto & name = header.getByPosition(field).name; if (names[i].empty()) @@ -123,6 +134,28 @@ DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const subst } ExpandField expand_field(names, types, expand_kinds, expand_fields); + return expand_field; +} + +bool ExpandRelParser::isLazyAggregateExpand(const substrait::ExpandRel & expand_rel) +{ + const auto & input_rel = expand_rel.input(); + if (input_rel.rel_type_case() != substrait::Rel::RelTypeCase::kAggregate) + return false; + const auto & aggregate_rel = input_rel.aggregate(); + for (const auto & measure : aggregate_rel.measures()) + { + if (measure.measure().phase() != substrait::AggregationPhase::AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE) + return false; + } + return true; +} + +DB::QueryPlanPtr ExpandRelParser::normalParse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list &) +{ + const auto & expand_rel = rel.expand(); + const auto & header = query_plan->getCurrentHeader(); + auto expand_field = buildExpandField(header, expand_rel); auto expand_step = std::make_unique(query_plan->getCurrentHeader(), std::move(expand_field)); expand_step->setStepDescription("Expand Step"); steps.emplace_back(expand_step.get()); @@ -130,6 +163,62 @@ DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const subst return query_plan; } +DB::QueryPlanPtr ExpandRelParser::lazyAggregateExpandParse( + DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack) +{ + DB::Block input_header = query_plan->getCurrentHeader(); + const auto & expand_rel = rel.expand(); + auto expand_field = buildExpandField(input_header, expand_rel); + auto aggregate_rel = rel.expand().input().aggregate(); + auto aggregate_descriptions = buildAggregations(input_header, expand_field, aggregate_rel); + + size_t grouping_keys = aggregate_rel.groupings(0).grouping_expressions_size(); + + auto expand_step + = std::make_unique(getContext(), input_header, grouping_keys, aggregate_descriptions, expand_field); + expand_step->setStepDescription("Advanced Expand Step"); + steps.emplace_back(expand_step.get()); + query_plan->addStep(std::move(expand_step)); + return query_plan; +} + +DB::AggregateDescriptions ExpandRelParser::buildAggregations( + const DB::Block & input_header, const ExpandField & expand_field, const substrait::AggregateRel & aggregate_rel) +{ + auto header = AdvancedExpandStep::buildOutputHeader(input_header, expand_field); + DB::AggregateDescriptions descriptions; + DB::ColumnsWithTypeAndName aggregate_columns; + for (const auto & col : header.getColumnsWithTypeAndName()) + { + if (typeid_cast(col.column.get())) + aggregate_columns.push_back(col); + } + + for (size_t i = 0; i < aggregate_rel.measures_size(); ++i) + { + /// The output header of the aggregate is [grouping keys] ++ [aggregation columns] + const auto & measure = aggregate_rel.measures(i); + const auto & col = aggregate_columns[i]; + DB::AggregateDescription description; + auto aggregate_col = typeid_cast(col.column.get()); + + description.column_name = col.name; + description.argument_names = {col.name}; + + auto aggregate_function = aggregate_col->getAggregateFunction(); + description.parameters = aggregate_function->getParameters(); + + // Need apply "PartialMerge" combinator for the aggregate function. + auto function_name_with_combinator = aggregate_function->getName() + "PartialMerge"; + DB::AggregateFunctionProperties aggregate_function_properties; + description.function + = getAggregateFunction(function_name_with_combinator, {col.type}, aggregate_function_properties, description.parameters); + + descriptions.emplace_back(description); + } + return descriptions; +} + void registerExpandRelParser(RelParserFactory & factory) { auto builder = [](ParserContextPtr parser_context) { return std::make_shared(parser_context); }; diff --git a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h index 7ba985a3f6d9..4a178ab08de7 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h @@ -16,9 +16,9 @@ */ #pragma once #include +#include #include - namespace local_engine { class SerializedPlanParser; @@ -29,7 +29,17 @@ class ExpandRelParser : public RelParser ~ExpandRelParser() override = default; DB::QueryPlanPtr parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_) override; + DB::QueryPlanPtr normalParse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); + DB::QueryPlanPtr + lazyAggregateExpandParse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.expand().input(); } + +private: + bool isLazyAggregateExpand(const substrait::ExpandRel & expand_rel); + ExpandField buildExpandField(const DB::Block & header, const substrait::ExpandRel & expand_rel); + + DB::AggregateDescriptions + buildAggregations(const DB::Block & input_header, const ExpandField & expand_field, const substrait::AggregateRel & aggregate_rel); }; } From 4fecd1dda81c4ac2f4a33e2f4a6f5e6351898873 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Wed, 6 Nov 2024 17:05:10 +0800 Subject: [PATCH 031/211] [VL] Remove one legacy Velox config used for Spark collect_list function (#7826) --- cpp/velox/compute/WholeStageResultIterator.cc | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index adc9e9bbe988..b6ecbd959f09 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -430,8 +430,6 @@ std::unordered_map WholeStageResultIterator::getQueryC std::to_string(veloxCfg_->get(kSparkBatchSize, 4096)); configs[velox::core::QueryConfig::kMaxOutputBatchRows] = std::to_string(veloxCfg_->get(kSparkBatchSize, 4096)); - // Find offheap size from Spark confs. If found, set the max memory usage of partial aggregation. - // FIXME this uses process-wise off-heap memory which is not for task try { if (veloxCfg_->valueExists(kDefaultSessionTimezone)) { configs[velox::core::QueryConfig::kSessionTimezone] = veloxCfg_->get(kDefaultSessionTimezone, ""); @@ -443,6 +441,8 @@ std::unordered_map WholeStageResultIterator::getQueryC configs[velox::core::QueryConfig::kAdjustTimestampToTimezone] = "true"; { + // Find offheap size from Spark confs. If found, set the max memory usage of partial aggregation. + // FIXME this uses process-wise off-heap memory which is not for task // partial aggregation memory config auto offHeapMemory = veloxCfg_->get(kSparkTaskOffHeapMemory, facebook::velox::memory::kMaxMemory); auto maxPartialAggregationMemory = @@ -456,8 +456,6 @@ std::unordered_map WholeStageResultIterator::getQueryC std::to_string(veloxCfg_->get(kAbandonPartialAggregationMinPct, 90)); configs[velox::core::QueryConfig::kAbandonPartialAggregationMinRows] = std::to_string(veloxCfg_->get(kAbandonPartialAggregationMinRows, 100000)); - // Spark's collect_set ignore nulls. - configs[velox::core::QueryConfig::kPrestoArrayAggIgnoreNulls] = "true"; } // Spill configs if (spillStrategy_ == "none") { From 9b6f24844ca2b7e0785a810d407adb1eb9eef20f Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Thu, 7 Nov 2024 08:19:46 +0800 Subject: [PATCH 032/211] [CORE] Remove unused dependencies of gluten-substrait (#7833) --- gluten-substrait/pom.xml | 54 ---------------------------------------- 1 file changed, 54 deletions(-) diff --git a/gluten-substrait/pom.xml b/gluten-substrait/pom.xml index 14334c2443fb..e8eb6a57e6c4 100644 --- a/gluten-substrait/pom.xml +++ b/gluten-substrait/pom.xml @@ -117,60 +117,6 @@ 3.1.0.0-RC2 test - - org.seleniumhq.selenium - selenium-htmlunit-driver - 2.52.0 - test - - - io.trino.tpch - tpch - 1.1 - test - - - io.trino.tpcds - tpcds - 1.4 - test - - - org.kohsuke - github-api - 1.117 - test - - - io.jsonwebtoken - jjwt-api - 0.10.5 - test - - - io.jsonwebtoken - jjwt-impl - 0.10.5 - test - - - io.jsonwebtoken - jjwt-jackson - 0.10.5 - test - - - org.knowm.xchart - xchart - 3.6.5 - test - - - de.erichseifert.vectorgraphics2d - VectorGraphics2D - - - com.google.protobuf protobuf-java From bb9b7ef034f87d2a532f688b31e606398f66578a Mon Sep 17 00:00:00 2001 From: Tengfei Huang Date: Thu, 7 Nov 2024 08:22:02 +0800 Subject: [PATCH 033/211] [GLUTEN-7079][VL] Fix metrics for InputIteratorTransformer of broadcast exchange (#7167) Closes #7079 Closes #4672 --- .../backendsapi/clickhouse/CHMetricsApi.scala | 30 +++++++++++--- .../GlutenClickHouseTPCHMetricsSuite.scala | 39 +++++++++++++++++- .../backendsapi/velox/VeloxMetricsApi.scala | 36 +++++++++++++---- .../metrics/InputIteratorMetricsUpdater.scala | 21 +++++----- .../gluten/execution/VeloxMetricsSuite.scala | 40 ++++++++++++++++++- .../gluten/backendsapi/MetricsApi.scala | 9 ++++- .../ColumnarCollapseTransformStages.scala | 18 ++++++++- 7 files changed, 164 insertions(+), 29 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala index 6a4f0c9a6f77..73b2d0f21101 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala @@ -23,7 +23,8 @@ import org.apache.gluten.substrait.{AggregationParams, JoinParams} import org.apache.spark.SparkContext import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ColumnarInputAdapter, SparkPlan} +import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import java.lang.{Long => JLong} @@ -39,21 +40,40 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { } override def genInputIteratorTransformerMetrics( - sparkContext: SparkContext): Map[String, SQLMetric] = { + child: SparkPlan, + sparkContext: SparkContext, + forBroadcast: Boolean): Map[String, SQLMetric] = { + def metricsPlan(plan: SparkPlan): SparkPlan = { + plan match { + case ColumnarInputAdapter(child) => metricsPlan(child) + case q: QueryStageExec => metricsPlan(q.plan) + case _ => plan + } + } + + val outputMetrics = if (forBroadcast) { + metricsPlan(child).metrics + .filterKeys(key => key.equals("numOutputRows")) + } else { + Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows") + ) + } + Map( "iterReadTime" -> SQLMetrics.createTimingMetric( sparkContext, "time of reading from iterator"), "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "fillingRightJoinSideTime" -> SQLMetrics.createTimingMetric( sparkContext, "filling right join side time") - ) + ) ++ outputMetrics } override def genInputIteratorTransformerMetricsUpdater( - metrics: Map[String, SQLMetric]): MetricsUpdater = { + metrics: Map[String, SQLMetric], + forBroadcast: Boolean): MetricsUpdater = { InputIteratorMetricsUpdater(metrics) } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala index 932686433e5b..3cfb8cc4fc9c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala @@ -21,7 +21,9 @@ import org.apache.gluten.extension.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.InputIteratorTransformer +import org.apache.spark.sql.execution.{ColumnarInputAdapter, InputIteratorTransformer} +import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.task.TaskResources import scala.collection.JavaConverters._ @@ -422,4 +424,39 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite assert(nativeMetricsDataFinal.metricsDataList.get(2).getName.equals("kProject")) } } + + test("Metrics for input iterator of broadcast exchange") { + createTPCHNotNullTables() + val partTableRecords = spark.sql("select * from part").count() + + // Repartition to make sure we have multiple tasks executing the join. + spark + .sql("select * from lineitem") + .repartition(2) + .createOrReplaceTempView("lineitem") + + Seq("true", "false").foreach { + adaptiveEnabled => + withSQLConf("spark.sql.adaptive.enabled" -> adaptiveEnabled) { + val sqlStr = + """ + |select /*+ BROADCAST(part) */ * from part join lineitem + |on l_partkey = p_partkey + |""".stripMargin + + runQueryAndCompare(sqlStr) { + df => + val inputIterator = find(df.queryExecution.executedPlan) { + case InputIteratorTransformer(ColumnarInputAdapter(child)) => + child.isInstanceOf[BroadcastQueryStageExec] || child + .isInstanceOf[BroadcastExchangeLike] + case _ => false + } + assert(inputIterator.isDefined) + val metrics = inputIterator.get.metrics + assert(metrics("numOutputRows").value == partTableRecords) + } + } + } + } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala index 10b0c493c10a..e70e1d13bdfe 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala @@ -22,7 +22,8 @@ import org.apache.gluten.substrait.{AggregationParams, JoinParams} import org.apache.spark.SparkContext import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ColumnarInputAdapter, SparkPlan} +import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import java.lang.{Long => JLong} @@ -38,18 +39,37 @@ class VeloxMetricsApi extends MetricsApi with Logging { } override def genInputIteratorTransformerMetrics( - sparkContext: SparkContext): Map[String, SQLMetric] = { + child: SparkPlan, + sparkContext: SparkContext, + forBroadcast: Boolean): Map[String, SQLMetric] = { + def metricsPlan(plan: SparkPlan): SparkPlan = { + plan match { + case ColumnarInputAdapter(child) => metricsPlan(child) + case q: QueryStageExec => metricsPlan(q.plan) + case _ => plan + } + } + + val outputMetrics = if (forBroadcast) { + metricsPlan(child).metrics + .filterKeys(key => key.equals("numOutputRows") || key.equals("outputVectors")) + } else { + Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "outputVectors" -> SQLMetrics.createMetric(sparkContext, "number of output vectors") + ) + } + Map( "cpuCount" -> SQLMetrics.createMetric(sparkContext, "cpu wall time count"), - "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of input iterator"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "outputVectors" -> SQLMetrics.createMetric(sparkContext, "number of output vectors") - ) + "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of input iterator") + ) ++ outputMetrics } override def genInputIteratorTransformerMetricsUpdater( - metrics: Map[String, SQLMetric]): MetricsUpdater = { - InputIteratorMetricsUpdater(metrics) + metrics: Map[String, SQLMetric], + forBroadcast: Boolean): MetricsUpdater = { + InputIteratorMetricsUpdater(metrics, forBroadcast) } override def genBatchScanTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala index a9067d069e03..8002a44ae92b 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala @@ -17,20 +17,23 @@ package org.apache.gluten.metrics import org.apache.spark.sql.execution.metric.SQLMetric -case class InputIteratorMetricsUpdater(metrics: Map[String, SQLMetric]) extends MetricsUpdater { +case class InputIteratorMetricsUpdater(metrics: Map[String, SQLMetric], forBroadcast: Boolean) + extends MetricsUpdater { override def updateNativeMetrics(opMetrics: IOperatorMetrics): Unit = { if (opMetrics != null) { val operatorMetrics = opMetrics.asInstanceOf[OperatorMetrics] metrics("cpuCount") += operatorMetrics.cpuCount metrics("wallNanos") += operatorMetrics.wallNanos - if (operatorMetrics.outputRows == 0 && operatorMetrics.outputVectors == 0) { - // Sometimes, velox does not update metrics for intermediate operator, - // here we try to use the input metrics - metrics("numOutputRows") += operatorMetrics.inputRows - metrics("outputVectors") += operatorMetrics.inputVectors - } else { - metrics("numOutputRows") += operatorMetrics.outputRows - metrics("outputVectors") += operatorMetrics.outputVectors + if (!forBroadcast) { + if (operatorMetrics.outputRows == 0 && operatorMetrics.outputVectors == 0) { + // Sometimes, velox does not update metrics for intermediate operator, + // here we try to use the input metrics + metrics("numOutputRows") += operatorMetrics.inputRows + metrics("outputVectors") += operatorMetrics.inputVectors + } else { + metrics("numOutputRows") += operatorMetrics.outputRows + metrics("outputVectors") += operatorMetrics.outputVectors + } } } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala index 4a251449845f..0b74824832f2 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala @@ -22,8 +22,9 @@ import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} import org.apache.spark.sql.TestUtils -import org.apache.spark.sql.execution.CommandResultExec -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.{ColumnarInputAdapter, CommandResultExec, InputIteratorTransformer} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, BroadcastQueryStageExec} +import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.sql.internal.SQLConf class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPlanHelper { @@ -227,4 +228,39 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa assert(inputRecords == (partTableRecords + itemTableRecords)) } + + test("Metrics for input iterator of broadcast exchange") { + createTPCHNotNullTables() + val partTableRecords = spark.sql("select * from part").count() + + // Repartition to make sure we have multiple tasks executing the join. + spark + .sql("select * from lineitem") + .repartition(2) + .createOrReplaceTempView("lineitem") + + Seq("true", "false").foreach { + adaptiveEnabled => + withSQLConf("spark.sql.adaptive.enabled" -> adaptiveEnabled) { + val sqlStr = + """ + |select /*+ BROADCAST(part) */ * from part join lineitem + |on l_partkey = p_partkey + |""".stripMargin + + runQueryAndCompare(sqlStr) { + df => + val inputIterator = find(df.queryExecution.executedPlan) { + case InputIteratorTransformer(ColumnarInputAdapter(child)) => + child.isInstanceOf[BroadcastQueryStageExec] || child + .isInstanceOf[BroadcastExchangeLike] + case _ => false + } + assert(inputIterator.isDefined) + val metrics = inputIterator.get.metrics + assert(metrics("numOutputRows").value == partTableRecords) + } + } + } + } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala index 62008767f51b..c67d4b5f8876 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala @@ -33,9 +33,14 @@ trait MetricsApi extends Serializable { "pipelineTime" -> SQLMetrics .createTimingMetric(sparkContext, WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) - def genInputIteratorTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] + def genInputIteratorTransformerMetrics( + child: SparkPlan, + sparkContext: SparkContext, + forBroadcast: Boolean): Map[String, SQLMetric] - def genInputIteratorTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater + def genInputIteratorTransformerMetricsUpdater( + metrics: Map[String, SQLMetric], + forBroadcast: Boolean): MetricsUpdater def metricsUpdatingFunction( child: SparkPlan, diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index 32575e4f13d5..d222dcfef84f 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -32,6 +32,8 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.ColumnarBatch @@ -49,14 +51,18 @@ case class InputIteratorTransformer(child: SparkPlan) extends UnaryTransformSupp @transient override lazy val metrics: Map[String, SQLMetric] = - BackendsApiManager.getMetricsApiInstance.genInputIteratorTransformerMetrics(sparkContext) + BackendsApiManager.getMetricsApiInstance.genInputIteratorTransformerMetrics( + child, + sparkContext, + forBroadcast()) override def simpleString(maxFields: Int): String = { s"$nodeName${truncatedString(output, "[", ", ", "]", maxFields)}" } override def metricsUpdater(): MetricsUpdater = - BackendsApiManager.getMetricsApiInstance.genInputIteratorTransformerMetricsUpdater(metrics) + BackendsApiManager.getMetricsApiInstance + .genInputIteratorTransformerMetricsUpdater(metrics, forBroadcast()) override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning @@ -75,6 +81,14 @@ case class InputIteratorTransformer(child: SparkPlan) extends UnaryTransformSupp override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { copy(child = newChild) } + + private def forBroadcast(): Boolean = { + child match { + case ColumnarInputAdapter(c) if c.isInstanceOf[BroadcastQueryStageExec] => true + case ColumnarInputAdapter(c) if c.isInstanceOf[BroadcastExchangeLike] => true + case _ => false + } + } } /** From 761f1c58a3adb6d768680523431ccb67b459f6c5 Mon Sep 17 00:00:00 2001 From: Yang Zhang Date: Thu, 7 Nov 2024 08:32:05 +0800 Subject: [PATCH 034/211] [GLUTEN-7800][VL] Add config for max reclaim wait time to avoid dead lock when memory arbitration --- cpp/velox/config/VeloxConfig.h | 3 ++ cpp/velox/jni/VeloxJniWrapper.cc | 1 - cpp/velox/memory/VeloxMemoryManager.cc | 42 ++++++++++--------- .../shuffle/VeloxRssSortShuffleWriter.cc | 1 - .../org/apache/gluten/GlutenConfig.scala | 7 ++++ 5 files changed, 33 insertions(+), 21 deletions(-) diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index 792beda96f7d..cb70dc62780b 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -76,6 +76,9 @@ const bool kMemoryUseHugePagesDefault = false; const std::string kVeloxMemInitCapacity = "spark.gluten.sql.columnar.backend.velox.memInitCapacity"; const uint64_t kVeloxMemInitCapacityDefault = 8 << 20; +const std::string kVeloxMemReclaimMaxWaitMs = "spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs"; +const uint64_t kVeloxMemReclaimMaxWaitMsDefault = 3600000; // 60min + const std::string kHiveConnectorId = "test-hive"; const std::string kVeloxCacheEnabled = "spark.gluten.sql.columnar.backend.velox.cacheEnabled"; diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index b8d2b0c3c2ff..6ea60d651a73 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -385,7 +385,6 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_VeloxDataSourceJniWrapp jlong dsHandle, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getRuntime(env, wrapper); auto datasource = ObjectStore::retrieve(dsHandle); auto batch = ObjectStore::retrieve(batchHandle); datasource->write(batch); diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 19a5d458049b..55420d938035 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -22,6 +22,7 @@ #include "velox/common/memory/MallocAllocator.h" #include "velox/common/memory/MemoryPool.h" +#include "velox/common/memory/SharedArbitrator.h" #include "velox/exec/MemoryReclaimer.h" #include "compute/VeloxBackend.h" @@ -36,12 +37,6 @@ namespace gluten { using namespace facebook; namespace { - -static constexpr std::string_view kMemoryPoolInitialCapacity{"memory-pool-initial-capacity"}; -static constexpr uint64_t kDefaultMemoryPoolInitialCapacity{256 << 20}; -static constexpr std::string_view kMemoryPoolTransferCapacity{"memory-pool-transfer-capacity"}; -static constexpr uint64_t kDefaultMemoryPoolTransferCapacity{128 << 20}; - template T getConfig( const std::unordered_map& configs, @@ -57,24 +52,28 @@ T getConfig( return defaultValue; } } // namespace + /// We assume in a single Spark task. No thread-safety should be guaranteed. class ListenableArbitrator : public velox::memory::MemoryArbitrator { public: ListenableArbitrator(const Config& config, AllocationListener* listener) : MemoryArbitrator(config), listener_(listener), - memoryPoolInitialCapacity_(velox::config::toCapacity( - getConfig( - config.extraConfigs, - kMemoryPoolInitialCapacity, - std::to_string(kDefaultMemoryPoolInitialCapacity)), - velox::config::CapacityUnit::BYTE)), + reclaimMaxWaitMs_( + velox::memory::SharedArbitrator::ExtraConfig::memoryReclaimMaxWaitTimeMs(config.extraConfigs)), + memoryPoolInitialCapacity_( + velox::memory::SharedArbitrator::ExtraConfig::memoryPoolInitialCapacity(config.extraConfigs)), memoryPoolTransferCapacity_(velox::config::toCapacity( getConfig( config.extraConfigs, - kMemoryPoolTransferCapacity, - std::to_string(kDefaultMemoryPoolTransferCapacity)), - velox::config::CapacityUnit::BYTE)) {} + kMemoryReservationBlockSize, + std::to_string(kMemoryReservationBlockSizeDefault)), + velox::config::CapacityUnit::BYTE)) { + if (reclaimMaxWaitMs_ == 0) { + LOG(WARNING) << kVeloxMemReclaimMaxWaitMs + << " was set to 0, it may cause dead lock when memory arbitration has bug."; + } + } std::string kind() const override { return kind_; } @@ -121,7 +120,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { VELOX_CHECK_EQ(candidates_.size(), 1, "ListenableArbitrator should only be used within a single root pool"); pool = candidates_.begin()->first; } - pool->reclaim(targetBytes, 0, status); // ignore the output + pool->reclaim(targetBytes, reclaimMaxWaitMs_, status); // ignore the output return shrinkCapacity0(pool, 0); } @@ -168,6 +167,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { } gluten::AllocationListener* listener_; + const uint64_t reclaimMaxWaitMs_; const uint64_t memoryPoolInitialCapacity_; // FIXME: Unused. const uint64_t memoryPoolTransferCapacity_; @@ -208,14 +208,18 @@ VeloxMemoryManager::VeloxMemoryManager(const std::string& kind, std::unique_ptr< kMemoryReservationBlockSize, kMemoryReservationBlockSizeDefault); auto memInitCapacity = VeloxBackend::get()->getBackendConf()->get(kVeloxMemInitCapacity, kVeloxMemInitCapacityDefault); + auto memReclaimMaxWaitMs = + VeloxBackend::get()->getBackendConf()->get(kVeloxMemReclaimMaxWaitMs, kVeloxMemReclaimMaxWaitMsDefault); blockListener_ = std::make_unique(listener_.get(), reservationBlockSize); listenableAlloc_ = std::make_unique(defaultMemoryAllocator().get(), blockListener_.get()); arrowPool_ = std::make_unique(listenableAlloc_.get()); std::unordered_map extraArbitratorConfigs; - extraArbitratorConfigs["memory-pool-initial-capacity"] = folly::to(memInitCapacity) + "B"; - extraArbitratorConfigs["memory-pool-transfer-capacity"] = folly::to(reservationBlockSize) + "B"; - extraArbitratorConfigs["memory-reclaim-max-wait-time"] = folly::to(0) + "ms"; + extraArbitratorConfigs[std::string(velox::memory::SharedArbitrator::ExtraConfig::kMemoryPoolInitialCapacity)] = + folly::to(memInitCapacity) + "B"; + extraArbitratorConfigs[kMemoryReservationBlockSize] = folly::to(reservationBlockSize) + "B"; + extraArbitratorConfigs[std::string(velox::memory::SharedArbitrator::ExtraConfig::kMemoryReclaimMaxWaitTime)] = + folly::to(memReclaimMaxWaitMs) + "ms"; ArbitratorFactoryRegister afr(listener_.get()); velox::memory::MemoryManagerOptions mmOptions{ diff --git a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc index 34796e378e1d..e17ad5e2f713 100644 --- a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc @@ -117,7 +117,6 @@ arrow::Status VeloxRssSortShuffleWriter::write(std::shared_ptr cb } arrow::Status VeloxRssSortShuffleWriter::evictBatch(uint32_t partitionId) { - int64_t rawSize = batch_->size(); bufferOutputStream_->seekp(0); batch_->flush(bufferOutputStream_.get()); auto buffer = bufferOutputStream_->getBuffer(); diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index a28a7d26b386..13bf72d47ec2 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -1414,6 +1414,13 @@ object GlutenConfig { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("8MB") + val COLUMNAR_VELOX_MEM_RECLAIM_MAX_WAIT_MS = + buildConf("spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs") + .internal() + .doc("The max time in ms to wait for memory reclaim.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(TimeUnit.MINUTES.toMillis(60)) + val COLUMNAR_VELOX_SSD_CACHE_PATH = buildStaticConf("spark.gluten.sql.columnar.backend.velox.ssdCachePath") .internal() From 1db66598913411c9936ab0ebde84bb14845d1199 Mon Sep 17 00:00:00 2001 From: Shuai li Date: Thu, 7 Nov 2024 08:52:43 +0800 Subject: [PATCH 035/211] [GLUTEN-7829][CH] Fix read csv file with datetime field not equals spark (#7832) --- ...=> GlutenClickHouseExcelFormatSuite.scala} | 5 +- .../Serializations/ExcelReadHelpers.cpp | 51 ++++++++++++------- .../Serializations/ExcelReadHelpers.h | 2 +- 3 files changed, 38 insertions(+), 20 deletions(-) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{GlutenClickHouseFileFormatSuite.scala => GlutenClickHouseExcelFormatSuite.scala} (99%) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala similarity index 99% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala index 2337316257aa..a5b866cb448d 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala @@ -46,7 +46,7 @@ case class AllDataTypesWithNonPrimitiveType( // data: (Seq[Int], (Int, String)) ) -class GlutenClickHouseFileFormatSuite +class GlutenClickHouseExcelFormatSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { import testImplicits._ @@ -273,7 +273,8 @@ class GlutenClickHouseFileFormatSuite StructField.apply("boolean_field", BooleanType, nullable = true), StructField.apply("decimal_field", DecimalType.apply(10, 2), nullable = true), StructField.apply("date_field", DateType, nullable = true), - StructField.apply("timestamp_field", TimestampType, nullable = true) + StructField.apply("timestamp_field", TimestampType, nullable = true), + StructField.apply("boolean_field2", BooleanType, nullable = true) )) val options = new util.HashMap[String, String]() diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp index 6a7b7b2e29cc..8c5bce2641e8 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp @@ -34,7 +34,7 @@ namespace local_engine bool readDateText(LocalDate & date, DB::ReadBuffer & buf, const DB::FormatSettings & settings) { bool is_us_style = settings.date_time_input_format == DB::FormatSettings::DateTimeInputFormat::BestEffortUS; - return readDateTextWithExcel(date, buf, is_us_style); + return readDateTextWithExcel(date, buf, is_us_style, settings); } bool readDateTime64Text( @@ -84,6 +84,20 @@ bool readDatetime64TextWithExcel( /// yyyy-MM-dd'T'HH:mm:ss.SSS'Z' /// Other will fallback to ch read. /// The whole value is in buffer. + /// + auto quick_return + = [&time_zone, &scale, &datetime64]( + UInt16 year, UInt8 month, UInt8 day, UInt8 hour, UInt8 minute, UInt8 second, DB::DateTime64::NativeType fractional) -> bool + { + if (!day) + day = 1; + + if (!checkDate(year, month, day)) + return false; + + time_t datetime = time_zone.makeDateTime(year, month, day, hour, minute, second); + return DB::DecimalUtils::tryGetDecimalFromComponents(datetime, fractional, scale, datetime64); + }; UInt16 year = 0; UInt8 month = 0; @@ -91,6 +105,7 @@ bool readDatetime64TextWithExcel( UInt8 hour = 0; UInt8 minute = 0; UInt8 second = 0; + DB::DateTime64::NativeType fractional = 0; char year_digits[std::numeric_limits::digits10]; size_t num_year_digits = readDigits(year_digits, sizeof(year_digits), buf); @@ -106,11 +121,13 @@ bool readDatetime64TextWithExcel( char month_digits[std::numeric_limits::digits10]; size_t num_month_digits = readDigits(month_digits, sizeof(month_digits), buf); - if (num_month_digits != 2) + if (num_month_digits == 1) + readDecimalNumber<1>(month, month_digits); + else if (num_month_digits == 2) + readDecimalNumber<2>(month, month_digits); + else return false; - readDecimalNumber<2>(month, month_digits); - if (*buf.position() != delimiter_after_year) // delimiter must same char return false; @@ -118,13 +135,18 @@ bool readDatetime64TextWithExcel( char day_digits[std::numeric_limits::digits10]; size_t num_day_digits = readDigits(day_digits, sizeof(day_digits), buf); - if (num_day_digits != 2) + if (num_day_digits == 1) + readDecimalNumber<1>(day, day_digits); + else if (num_day_digits == 2) + readDecimalNumber<2>(day, day_digits); + else return false; - readDecimalNumber<2>(day, day_digits); - char delimiter_after_day = *buf.position(); + if (delimiter_after_day == settings.delimiter) + return quick_return(year, month, day, hour, minute, second, fractional); + if (delimiter_after_day != ' ' && delimiter_after_day != '\'') return false; @@ -159,7 +181,6 @@ bool readDatetime64TextWithExcel( /// .SSS'Z' /// if not has quote, not allow ',' after 'ss' - DB::DateTime64::NativeType fractional = 0; bool allow_comma = (settings.delimiter == ',' && quote) || (!quote && settings.delimiter != ','); if (!buf.eof() && (*buf.position() == '.' || (allow_comma && *buf.position() == ','))) { @@ -186,17 +207,10 @@ bool readDatetime64TextWithExcel( buf.position() = buf.position() + 3; } - if (!day) - day = 1; - - if (!checkDate(year, month, day)) - return false; - - time_t datetime = time_zone.makeDateTime(year, month, day, hour, minute, second); - return DB::DecimalUtils::tryGetDecimalFromComponents(datetime, fractional, scale, datetime64); + return quick_return(year, month, day, hour, minute, second, fractional); } -inline bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style) +inline bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style, const DB::FormatSettings & settings) { if (buf.eof()) return false; @@ -268,6 +282,9 @@ inline bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool i readDecimalNumber<2>(month, first_digits); char delimiter_after_year = *buf.position(); + if (delimiter_after_year == settings.csv.delimiter) + return false; + ++buf.position(); diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h index f96b31f7049d..a5f272334e54 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h @@ -98,7 +98,7 @@ bool readDateTime64Text( const DateLUTImpl & utc_time_zone, bool quote); -bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style); +bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style, const DB::FormatSettings & settings); bool readDateText(LocalDate & date, DB::ReadBuffer & buf, const DB::FormatSettings & settings); From 983e269df1b768266b4af346a1d219f3acd08cf7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 7 Nov 2024 09:07:49 +0800 Subject: [PATCH 036/211] [GLUTEN-7796][CH] Fix diff while casting bool to string (#7804) * fix cast bool to string * fix failed uts * fix cast bool to string aghain * remove std::couts --- .../GlutenFunctionValidateSuite.scala | 5 ++ .../local-engine/Parser/ExpressionParser.cpp | 86 +++++++++---------- cpp-ch/local-engine/Parser/FunctionParser.cpp | 69 ++++++++++----- cpp-ch/local-engine/Parser/FunctionParser.h | 1 + 4 files changed, 97 insertions(+), 64 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala index 08342e588730..dbe8852290aa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala @@ -855,4 +855,9 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS compareResultsAgainstVanillaSpark(sql, true, { _ => }) } } + + test("GLUTEN-7796 cast bool to string") { + val sql = "select cast(id % 2 = 1 as string) from range(10)" + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } } diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.cpp b/cpp-ch/local-engine/Parser/ExpressionParser.cpp index ee64aff078de..30ef92f17607 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.cpp +++ b/cpp-ch/local-engine/Parser/ExpressionParser.cpp @@ -93,7 +93,7 @@ std::pair LiteralParser::parse(const substrait::Expr break; } case substrait::Expression_Literal::kBoolean: { - type = std::make_shared(); + type = DB::DataTypeFactory::instance().get("Bool"); field = literal.boolean() ? UInt8(1) : UInt8(0); break; } @@ -288,77 +288,77 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ case substrait::Expression::RexTypeCase::kCast: { if (!rel.cast().has_type() || !rel.cast().has_input()) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Doesn't have type or input in cast node."); - DB::ActionsDAG::NodeRawConstPtrs args; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Doesn't have type or input in cast node."); + ActionsDAG::NodeRawConstPtrs args; - String cast_function = "CAST"; const auto & input = rel.cast().input(); args.emplace_back(parseExpression(actions_dag, input)); const auto & substrait_type = rel.cast().type(); const auto & input_type = args[0]->result_type; - DB::DataTypePtr non_nullable_input_type = DB::removeNullable(input_type); - DB::DataTypePtr output_type = TypeParser::parseType(substrait_type); - DB::DataTypePtr non_nullable_output_type = DB::removeNullable(output_type); + DataTypePtr denull_input_type = removeNullable(input_type); + DataTypePtr output_type = TypeParser::parseType(substrait_type); + DataTypePtr denull_output_type = removeNullable(output_type); - const DB::ActionsDAG::Node * function_node = nullptr; + const ActionsDAG::Node * result_node = nullptr; if (substrait_type.has_binary()) { /// Spark cast(x as BINARY) -> CH reinterpretAsStringSpark(x) - function_node = toFunctionNode(actions_dag, "reinterpretAsStringSpark", args); + result_node = toFunctionNode(actions_dag, "reinterpretAsStringSpark", args); } - else if (DB::isString(non_nullable_input_type) && DB::isDate32(non_nullable_output_type)) - function_node = toFunctionNode(actions_dag, "sparkToDate", args); - else if (DB::isString(non_nullable_input_type) && DB::isDateTime64(non_nullable_output_type)) - function_node = toFunctionNode(actions_dag, "sparkToDateTime", args); - else if (DB::isDecimal(non_nullable_input_type) && DB::isString(non_nullable_output_type)) + else if (isString(denull_input_type) && isDate32(denull_output_type)) + result_node = toFunctionNode(actions_dag, "sparkToDate", args); + else if (isString(denull_input_type) && isDateTime64(denull_output_type)) + result_node = toFunctionNode(actions_dag, "sparkToDateTime", args); + else if (isDecimal(denull_input_type) && isString(denull_output_type)) { /// Spark cast(x as STRING) if x is Decimal -> CH toDecimalString(x, scale) - UInt8 scale = DB::getDecimalScale(*non_nullable_input_type); - args.emplace_back(addConstColumn(actions_dag, std::make_shared(), DB::Field(scale))); - function_node = toFunctionNode(actions_dag, "toDecimalString", args); + UInt8 scale = getDecimalScale(*denull_input_type); + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), Field(scale))); + result_node = toFunctionNode(actions_dag, "toDecimalString", args); } - else if (DB::isFloat(non_nullable_input_type) && DB::isInt(non_nullable_output_type)) + else if (isFloat(denull_input_type) && isInt(denull_output_type)) { - String function_name = "sparkCastFloatTo" + non_nullable_output_type->getName(); - function_node = toFunctionNode(actions_dag, function_name, args); + String function_name = "sparkCastFloatTo" + denull_output_type->getName(); + result_node = toFunctionNode(actions_dag, function_name, args); } - else if ((isDecimal(non_nullable_input_type) && substrait_type.has_decimal())) + else if ((isDecimal(denull_input_type) && substrait_type.has_decimal())) { args.emplace_back(addConstColumn(actions_dag, std::make_shared(), substrait_type.decimal().precision())); args.emplace_back(addConstColumn(actions_dag, std::make_shared(), substrait_type.decimal().scale())); - - function_node = toFunctionNode(actions_dag, "checkDecimalOverflowSparkOrNull", args); + result_node = toFunctionNode(actions_dag, "checkDecimalOverflowSparkOrNull", args); } - else if (isMap(non_nullable_input_type) && isString(non_nullable_output_type)) + else if (isMap(denull_input_type) && isString(denull_output_type)) { // ISSUE-7389: spark cast(map to string) has different behavior with CH cast(map to string) - auto map_input_type = std::static_pointer_cast(non_nullable_input_type); + auto map_input_type = std::static_pointer_cast(denull_input_type); args.emplace_back(addConstColumn(actions_dag, map_input_type->getKeyType(), map_input_type->getKeyType()->getDefault())); args.emplace_back(addConstColumn(actions_dag, map_input_type->getValueType(), map_input_type->getValueType()->getDefault())); - function_node = toFunctionNode(actions_dag, "sparkCastMapToString", args); + result_node = toFunctionNode(actions_dag, "sparkCastMapToString", args); + } + else if (isString(denull_input_type) && substrait_type.has_bool_()) + { + /// cast(string to boolean) + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); + result_node = toFunctionNode(actions_dag, "accurateCastOrNull", args); + } + else if (isString(denull_input_type) && isInt(denull_output_type)) + { + /// Spark cast(x as INT) if x is String -> CH cast(trim(x) as INT) + /// Refer to https://github.com/apache/incubator-gluten/issues/4956 + args[0] = toFunctionNode(actions_dag, "trim", {args[0]}); + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); + result_node = toFunctionNode(actions_dag, "CAST", args); } else { - if (DB::isString(non_nullable_input_type) && DB::isInt(non_nullable_output_type)) - { - /// Spark cast(x as INT) if x is String -> CH cast(trim(x) as INT) - /// Refer to https://github.com/apache/incubator-gluten/issues/4956 - args[0] = toFunctionNode(actions_dag, "trim", {args[0]}); - } - else if (DB::isString(non_nullable_input_type) && substrait_type.has_bool_()) - { - /// cast(string to boolean) - cast_function = "accurateCastOrNull"; - } - - /// Common process - args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); - function_node = toFunctionNode(actions_dag, cast_function, args); + /// Common process: CAST(input, type) + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); + result_node = toFunctionNode(actions_dag, "CAST", args); } - actions_dag.addOrReplaceInOutputs(*function_node); - return function_node; + actions_dag.addOrReplaceInOutputs(*result_node); + return result_node; } case substrait::Expression::RexTypeCase::kIfThen: { diff --git a/cpp-ch/local-engine/Parser/FunctionParser.cpp b/cpp-ch/local-engine/Parser/FunctionParser.cpp index c8158d3e3e46..7e794dabec64 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/FunctionParser.cpp @@ -115,33 +115,60 @@ const ActionsDAG::Node * FunctionParser::convertNodeTypeIfNeeded( const substrait::Expression_ScalarFunction & substrait_func, const ActionsDAG::Node * func_node, ActionsDAG & actions_dag) const { const auto & output_type = substrait_func.output_type(); - if (!TypeParser::isTypeMatched(output_type, func_node->result_type)) + const ActionsDAG::Node * result_node = nullptr; + + auto convert_type_if_needed = [&]() { - auto result_type = TypeParser::parseType(substrait_func.output_type()); - if (DB::isDecimalOrNullableDecimal(result_type)) + if (!TypeParser::isTypeMatched(output_type, func_node->result_type)) { - return ActionsDAGUtil::convertNodeType( - actions_dag, - func_node, - // as stated in isTypeMatched, currently we don't change nullability of the result type - func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, result_type) - : local_engine::removeNullable(result_type), - func_node->result_name, - CastType::accurateOrNull); + auto result_type = TypeParser::parseType(substrait_func.output_type()); + if (DB::isDecimalOrNullableDecimal(result_type)) + { + return ActionsDAGUtil::convertNodeType( + actions_dag, + func_node, + // as stated in isTypeMatched, currently we don't change nullability of the result type + func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, result_type) + : local_engine::removeNullable(result_type), + func_node->result_name, + CastType::accurateOrNull); + } + else + { + return ActionsDAGUtil::convertNodeType( + actions_dag, + func_node, + // as stated in isTypeMatched, currently we don't change nullability of the result type + func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, TypeParser::parseType(output_type)) + : DB::removeNullable(TypeParser::parseType(output_type)), + func_node->result_name); + } } else + return func_node; + }; + result_node = convert_type_if_needed(); + + /// Notice that in CH Bool and UInt8 have different serialization and deserialization methods, which will cause issue when executing cast(bool as string) in spark in spark. + auto convert_uint8_to_bool_if_needed = [&]() -> const auto * + { + auto * mutable_result_node = const_cast(result_node); + auto denull_result_type = DB::removeNullable(result_node->result_type); + if (isUInt8(denull_result_type) && output_type.has_bool_()) { - return ActionsDAGUtil::convertNodeType( - actions_dag, - func_node, - // as stated in isTypeMatched, currently we don't change nullability of the result type - func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, TypeParser::parseType(output_type)) - : DB::removeNullable(TypeParser::parseType(output_type)), - func_node->result_name); + auto bool_type = DB::DataTypeFactory::instance().get("Bool"); + if (result_node->result_type->isNullable()) + bool_type = DB::makeNullable(bool_type); + + mutable_result_node->result_type = std::move(bool_type); + return mutable_result_node; } - } - else - return func_node; + else + return result_node; + }; + result_node = convert_uint8_to_bool_if_needed(); + + return result_node; } void FunctionParserFactory::registerFunctionParser(const String & name, Value value) diff --git a/cpp-ch/local-engine/Parser/FunctionParser.h b/cpp-ch/local-engine/Parser/FunctionParser.h index d9ca7b51285b..23216f22034b 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.h +++ b/cpp-ch/local-engine/Parser/FunctionParser.h @@ -60,6 +60,7 @@ class FunctionParser { return parseFunctionArguments(substrait_func, actions_dag); } + virtual DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const; From 96045b117531a2a0db93bf4f9ea7bac5d7fba117 Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 7 Nov 2024 09:51:19 +0800 Subject: [PATCH 037/211] [GLUTEN-7778][CH] Make aggregation output schema same as CH native (#7811) * unity agg output * update * update * update * fixed --- .../clickhouse/CHSparkPlanExecApi.scala | 9 ++++- .../clickhouse/CHTransformerApi.scala | 11 ++++-- .../CHHashAggregateExecTransformer.scala | 39 +++++++++++++++++++ 3 files changed, 53 insertions(+), 6 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index ba165d936eed..190fcb13eaeb 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -158,16 +158,21 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { aggregateAttributes: Seq[Attribute], initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], - child: SparkPlan): HashAggregateExecBaseTransformer = + child: SparkPlan): HashAggregateExecBaseTransformer = { + val replacedResultExpressions = CHHashAggregateExecTransformer.getCHAggregateResultExpressions( + groupingExpressions, + aggregateExpressions, + resultExpressions) CHHashAggregateExecTransformer( requiredChildDistributionExpressions, groupingExpressions.distinct, aggregateExpressions, aggregateAttributes, initialInputBufferOffset, - resultExpressions.distinct, + replacedResultExpressions.distinct, child ) + } /** Generate HashAggregateExecPullOutHelper */ override def genHashAggregateExecPullOutHelper( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala index e5b71825853b..bf909c52ac20 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala @@ -177,10 +177,13 @@ class CHTransformerApi extends TransformerApi with Logging { // output name will be different from grouping expressions, // so using output attribute instead of grouping expression val groupingExpressions = hash.output.splitAt(hash.groupingExpressions.size)._1 - val aggResultAttributes = CHHashAggregateExecTransformer.getAggregateResultAttributes( - groupingExpressions, - hash.aggregateExpressions - ) + val aggResultAttributes = CHHashAggregateExecTransformer + .getCHAggregateResultExpressions( + groupingExpressions, + hash.aggregateExpressions, + hash.resultExpressions + ) + .map(_.toAttribute) if (aggResultAttributes.size == hash.output.size) { aggResultAttributes } else { diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala index f5e64330cd15..fcf6320f8ec4 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala @@ -43,6 +43,45 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer object CHHashAggregateExecTransformer { + // The result attributes of aggregate expressions from vanilla may be different from CH native. + // For example, the result attributes of `avg(x)` are `sum(x)` and `count(x)`. This could bring + // some unexpected issues. So we need to make the result attributes consistent with CH native. + def getCHAggregateResultExpressions( + groupingExpressions: Seq[NamedExpression], + aggregateExpressions: Seq[AggregateExpression], + resultExpressions: Seq[NamedExpression]): Seq[NamedExpression] = { + var adjustedResultExpressions = resultExpressions.slice(0, groupingExpressions.length) + var resultExpressionIndex = groupingExpressions.length + adjustedResultExpressions ++ aggregateExpressions.flatMap { + aggExpr => + aggExpr.mode match { + case Partial | PartialMerge => + // For partial aggregate, the size of the result expressions of an aggregate expression + // is the same as aggBufferAttributes' length + val aggBufferAttributesCount = aggExpr.aggregateFunction.aggBufferAttributes.length + aggExpr.aggregateFunction match { + case avg: Average => + val res = Seq(aggExpr.resultAttribute) + resultExpressionIndex += aggBufferAttributesCount + res + case sum: Sum if (sum.dataType.isInstanceOf[DecimalType]) => + val res = Seq(resultExpressions(resultExpressionIndex)) + resultExpressionIndex += aggBufferAttributesCount + res + case _ => + val res = resultExpressions + .slice(resultExpressionIndex, resultExpressionIndex + aggBufferAttributesCount) + resultExpressionIndex += aggBufferAttributesCount + res + } + case _ => + val res = Seq(resultExpressions(resultExpressionIndex)) + resultExpressionIndex += 1 + res + } + } + } + def getAggregateResultAttributes( groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression]): Seq[Attribute] = { From a94f51e01f90b8e26d954c271caabca51ff921a5 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Thu, 7 Nov 2024 09:53:02 +0800 Subject: [PATCH 038/211] =?UTF-8?q?Revert=20"[GLUTEN-7800][VL]=20Add=20con?= =?UTF-8?q?fig=20for=20max=20reclaim=20wait=20time=20to=20avoid=20dead=20?= =?UTF-8?q?=E2=80=A6"=20(#7836)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 761f1c58a3adb6d768680523431ccb67b459f6c5. --- cpp/velox/config/VeloxConfig.h | 3 -- cpp/velox/jni/VeloxJniWrapper.cc | 1 + cpp/velox/memory/VeloxMemoryManager.cc | 42 +++++++++---------- .../shuffle/VeloxRssSortShuffleWriter.cc | 1 + .../org/apache/gluten/GlutenConfig.scala | 7 ---- 5 files changed, 21 insertions(+), 33 deletions(-) diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index cb70dc62780b..792beda96f7d 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -76,9 +76,6 @@ const bool kMemoryUseHugePagesDefault = false; const std::string kVeloxMemInitCapacity = "spark.gluten.sql.columnar.backend.velox.memInitCapacity"; const uint64_t kVeloxMemInitCapacityDefault = 8 << 20; -const std::string kVeloxMemReclaimMaxWaitMs = "spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs"; -const uint64_t kVeloxMemReclaimMaxWaitMsDefault = 3600000; // 60min - const std::string kHiveConnectorId = "test-hive"; const std::string kVeloxCacheEnabled = "spark.gluten.sql.columnar.backend.velox.cacheEnabled"; diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index 6ea60d651a73..b8d2b0c3c2ff 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -385,6 +385,7 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_VeloxDataSourceJniWrapp jlong dsHandle, jlong batchHandle) { JNI_METHOD_START + auto ctx = gluten::getRuntime(env, wrapper); auto datasource = ObjectStore::retrieve(dsHandle); auto batch = ObjectStore::retrieve(batchHandle); datasource->write(batch); diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 55420d938035..19a5d458049b 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -22,7 +22,6 @@ #include "velox/common/memory/MallocAllocator.h" #include "velox/common/memory/MemoryPool.h" -#include "velox/common/memory/SharedArbitrator.h" #include "velox/exec/MemoryReclaimer.h" #include "compute/VeloxBackend.h" @@ -37,6 +36,12 @@ namespace gluten { using namespace facebook; namespace { + +static constexpr std::string_view kMemoryPoolInitialCapacity{"memory-pool-initial-capacity"}; +static constexpr uint64_t kDefaultMemoryPoolInitialCapacity{256 << 20}; +static constexpr std::string_view kMemoryPoolTransferCapacity{"memory-pool-transfer-capacity"}; +static constexpr uint64_t kDefaultMemoryPoolTransferCapacity{128 << 20}; + template T getConfig( const std::unordered_map& configs, @@ -52,28 +57,24 @@ T getConfig( return defaultValue; } } // namespace - /// We assume in a single Spark task. No thread-safety should be guaranteed. class ListenableArbitrator : public velox::memory::MemoryArbitrator { public: ListenableArbitrator(const Config& config, AllocationListener* listener) : MemoryArbitrator(config), listener_(listener), - reclaimMaxWaitMs_( - velox::memory::SharedArbitrator::ExtraConfig::memoryReclaimMaxWaitTimeMs(config.extraConfigs)), - memoryPoolInitialCapacity_( - velox::memory::SharedArbitrator::ExtraConfig::memoryPoolInitialCapacity(config.extraConfigs)), + memoryPoolInitialCapacity_(velox::config::toCapacity( + getConfig( + config.extraConfigs, + kMemoryPoolInitialCapacity, + std::to_string(kDefaultMemoryPoolInitialCapacity)), + velox::config::CapacityUnit::BYTE)), memoryPoolTransferCapacity_(velox::config::toCapacity( getConfig( config.extraConfigs, - kMemoryReservationBlockSize, - std::to_string(kMemoryReservationBlockSizeDefault)), - velox::config::CapacityUnit::BYTE)) { - if (reclaimMaxWaitMs_ == 0) { - LOG(WARNING) << kVeloxMemReclaimMaxWaitMs - << " was set to 0, it may cause dead lock when memory arbitration has bug."; - } - } + kMemoryPoolTransferCapacity, + std::to_string(kDefaultMemoryPoolTransferCapacity)), + velox::config::CapacityUnit::BYTE)) {} std::string kind() const override { return kind_; } @@ -120,7 +121,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { VELOX_CHECK_EQ(candidates_.size(), 1, "ListenableArbitrator should only be used within a single root pool"); pool = candidates_.begin()->first; } - pool->reclaim(targetBytes, reclaimMaxWaitMs_, status); // ignore the output + pool->reclaim(targetBytes, 0, status); // ignore the output return shrinkCapacity0(pool, 0); } @@ -167,7 +168,6 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { } gluten::AllocationListener* listener_; - const uint64_t reclaimMaxWaitMs_; const uint64_t memoryPoolInitialCapacity_; // FIXME: Unused. const uint64_t memoryPoolTransferCapacity_; @@ -208,18 +208,14 @@ VeloxMemoryManager::VeloxMemoryManager(const std::string& kind, std::unique_ptr< kMemoryReservationBlockSize, kMemoryReservationBlockSizeDefault); auto memInitCapacity = VeloxBackend::get()->getBackendConf()->get(kVeloxMemInitCapacity, kVeloxMemInitCapacityDefault); - auto memReclaimMaxWaitMs = - VeloxBackend::get()->getBackendConf()->get(kVeloxMemReclaimMaxWaitMs, kVeloxMemReclaimMaxWaitMsDefault); blockListener_ = std::make_unique(listener_.get(), reservationBlockSize); listenableAlloc_ = std::make_unique(defaultMemoryAllocator().get(), blockListener_.get()); arrowPool_ = std::make_unique(listenableAlloc_.get()); std::unordered_map extraArbitratorConfigs; - extraArbitratorConfigs[std::string(velox::memory::SharedArbitrator::ExtraConfig::kMemoryPoolInitialCapacity)] = - folly::to(memInitCapacity) + "B"; - extraArbitratorConfigs[kMemoryReservationBlockSize] = folly::to(reservationBlockSize) + "B"; - extraArbitratorConfigs[std::string(velox::memory::SharedArbitrator::ExtraConfig::kMemoryReclaimMaxWaitTime)] = - folly::to(memReclaimMaxWaitMs) + "ms"; + extraArbitratorConfigs["memory-pool-initial-capacity"] = folly::to(memInitCapacity) + "B"; + extraArbitratorConfigs["memory-pool-transfer-capacity"] = folly::to(reservationBlockSize) + "B"; + extraArbitratorConfigs["memory-reclaim-max-wait-time"] = folly::to(0) + "ms"; ArbitratorFactoryRegister afr(listener_.get()); velox::memory::MemoryManagerOptions mmOptions{ diff --git a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc index e17ad5e2f713..34796e378e1d 100644 --- a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc @@ -117,6 +117,7 @@ arrow::Status VeloxRssSortShuffleWriter::write(std::shared_ptr cb } arrow::Status VeloxRssSortShuffleWriter::evictBatch(uint32_t partitionId) { + int64_t rawSize = batch_->size(); bufferOutputStream_->seekp(0); batch_->flush(bufferOutputStream_.get()); auto buffer = bufferOutputStream_->getBuffer(); diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 13bf72d47ec2..a28a7d26b386 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -1414,13 +1414,6 @@ object GlutenConfig { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("8MB") - val COLUMNAR_VELOX_MEM_RECLAIM_MAX_WAIT_MS = - buildConf("spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs") - .internal() - .doc("The max time in ms to wait for memory reclaim.") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefault(TimeUnit.MINUTES.toMillis(60)) - val COLUMNAR_VELOX_SSD_CACHE_PATH = buildStaticConf("spark.gluten.sql.columnar.backend.velox.ssdCachePath") .internal() From f7971f6c16d8ae51bed984632afa735fe4c7c585 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Thu, 7 Nov 2024 09:53:54 +0800 Subject: [PATCH 039/211] [VL] Sort shuffle writer use vectorized c2r (#6782) --- cpp/velox/shuffle/VeloxSortShuffleWriter.cc | 29 +++++++++++++-------- cpp/velox/shuffle/VeloxSortShuffleWriter.h | 9 +++++-- 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc index f87eaabb56ae..ab37c0be74ad 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc @@ -180,7 +180,7 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr rowSize_.resize(inputRows, *fixedRowSize_); } - uint32_t rowOffset = 0; + facebook::velox::vector_size_t rowOffset = 0; while (rowOffset < inputRows) { auto remainingRows = inputRows - rowOffset; auto rows = maxRowsToInsert(rowOffset, remainingRows); @@ -201,18 +201,23 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr return arrow::Status::OK(); } -void VeloxSortShuffleWriter::insertRows(facebook::velox::row::CompactRow& row, uint32_t offset, uint32_t rows) { +void VeloxSortShuffleWriter::insertRows( + facebook::velox::row::CompactRow& compact, + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t size) { VELOX_CHECK(!pages_.empty()); - for (auto i = offset; i < offset + rows; ++i) { - auto pid = row2Partition_[i]; + std::vector offsets(size); + for (auto i = 0; i < size; ++i) { + auto row = offset + i; + auto pid = row2Partition_[row]; arrayPtr_[offset_++] = toCompactRowId(pid, pageNumber_, pageCursor_); // size(RowSize) | bytes - memcpy(currentPage_ + pageCursor_, &rowSize_[i], sizeof(RowSizeType)); - pageCursor_ += sizeof(RowSizeType); - auto size = row.serialize(i, currentPage_ + pageCursor_); - pageCursor_ += size; + memcpy(currentPage_ + pageCursor_, &rowSize_[row], sizeof(RowSizeType)); + offsets[i] = pageCursor_ + sizeof(RowSizeType); + pageCursor_ += rowSize_[row]; VELOX_DCHECK_LE(pageCursor_, currenPageSize_); } + compact.serialize(offset, size, offsets.data(), currentPage_); } arrow::Status VeloxSortShuffleWriter::maybeSpill(uint32_t nextRows) { @@ -337,19 +342,21 @@ VeloxSortShuffleWriter::evictPartition0(uint32_t partitionId, int32_t numRows, u return arrow::Status::OK(); } -uint32_t VeloxSortShuffleWriter::maxRowsToInsert(uint32_t offset, uint32_t remainingRows) { +facebook::velox::vector_size_t VeloxSortShuffleWriter::maxRowsToInsert( + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t remainingRows) { // Check how many rows can be handled. if (pages_.empty()) { return 0; } auto remainingBytes = pages_.back()->size() - pageCursor_; if (fixedRowSize_) { - return std::min((uint32_t)(remainingBytes / (fixedRowSize_.value())), remainingRows); + return std::min((facebook::velox::vector_size_t)(remainingBytes / (fixedRowSize_.value())), remainingRows); } auto beginIter = rowSizePrefixSum_.begin() + 1 + offset; auto bytesWritten = rowSizePrefixSum_[offset]; auto iter = std::upper_bound(beginIter, rowSizePrefixSum_.end(), remainingBytes + bytesWritten); - return iter - beginIter; + return (facebook::velox::vector_size_t)(iter - beginIter); } void VeloxSortShuffleWriter::acquireNewBuffer(uint64_t memLimit, uint64_t minSizeRequired) { diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.h b/cpp/velox/shuffle/VeloxSortShuffleWriter.h index 531ed1fe3e76..5b8cff452d56 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.h @@ -69,7 +69,10 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { arrow::Status insert(const facebook::velox::RowVectorPtr& vector, int64_t memLimit); - void insertRows(facebook::velox::row::CompactRow& row, uint32_t offset, uint32_t rows); + void insertRows( + facebook::velox::row::CompactRow& compact, + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t size); arrow::Status maybeSpill(uint32_t nextRows); @@ -79,7 +82,9 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { arrow::Status evictPartition0(uint32_t partitionId, int32_t numRows, uint8_t* buffer, int64_t rawLength); - uint32_t maxRowsToInsert(uint32_t offset, uint32_t remainingRows); + facebook::velox::vector_size_t maxRowsToInsert( + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t remainingRows); void acquireNewBuffer(uint64_t memLimit, uint64_t minSizeRequired); From 427aeb2f574ac661cb666284d39f63cbefad67fb Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 7 Nov 2024 11:21:14 +0800 Subject: [PATCH 040/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_07) (#7834) Upstream Velox's New Commits: c3023b695 by Jimmy Lu, Merge nest loop join build vectors (11428) 6b11a568c by Jia Ke, Change the arrow dependency from private to public in velox_external_hdfs (11371) a50d9400a by Huameng (Michael) Jiang, selective reader deduplicated array type (11427) 7f3588ef3 by Jialiang Tan, Fix test MockSharedArbitratorTest.arbitrationFailure (11450) 3ca03d82e by Jialiang Tan, Minor change to oom message to reduce confusion (11443) ef4614d05 by Christian Zentgraf, refactor: Fix deprecated errors when using shared_ptr::unique() (11318) bfc199fc6 by Christian Zentgraf, Add gcc11 to Ubuntu20.04 setup and add PkgConfig install (11326) 415f14e16 by Jimmy Lu, Handle UNKNOWN type input in approx_distinct and count(distinct ...) (11367) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 3c1bc083ed6d..a3220cbb3455 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_06 +VELOX_BRANCH=2024_11_07 VELOX_HOME="" OS=`uname -s` From f6a9665ca7b215fd03643fec783898c6f6c162f4 Mon Sep 17 00:00:00 2001 From: Xiuli Wei Date: Thu, 7 Nov 2024 11:31:02 +0800 Subject: [PATCH 041/211] [GLUTEN-7675][VL] Support parquet write with complex data type(eg. MAP, ARRYY) (#7676) --- .../backendsapi/velox/VeloxBackend.scala | 25 +++-- .../execution/WriteFilesExecTransformer.scala | 39 ++++++- .../parquet/GlutenParquetFieldIdIOSuite.scala | 25 ++++- .../utils/velox/VeloxTestSettings.scala | 7 ++ .../sql/GlutenCharVarcharTestSuite.scala | 101 ++++++++++++++++- .../parquet/GlutenParquetFieldIdIOSuite.scala | 25 ++++- .../utils/velox/VeloxTestSettings.scala | 7 ++ .../sql/GlutenCharVarcharTestSuite.scala | 102 +++++++++++++++++- .../parquet/GlutenParquetFieldIdIOSuite.scala | 25 ++++- 9 files changed, 339 insertions(+), 17 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 539059cdb6f8..03d5aa2549e5 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -232,15 +232,26 @@ object VeloxBackendSettings extends BackendSettingsApi { // Validate if all types are supported. def validateDataTypes(): Option[String] = { - val unsupportedTypes = fields.flatMap { - field => - field.dataType match { - case _: StructType => Some("StructType") - case _: ArrayType => Some("ArrayType") - case _: MapType => Some("MapType") - case _: YearMonthIntervalType => Some("YearMonthIntervalType") + val unsupportedTypes = format match { + case _: ParquetFileFormat => + fields.flatMap { + case StructField(_, _: YearMonthIntervalType, _, _) => + Some("YearMonthIntervalType") + case StructField(_, _: StructType, _, _) => + Some("StructType") case _ => None } + case _ => + fields.flatMap { + field => + field.dataType match { + case _: StructType => Some("StructType") + case _: ArrayType => Some("ArrayType") + case _: MapType => Some("MapType") + case _: YearMonthIntervalType => Some("YearMonthIntervalType") + case _ => None + } + } } if (unsupportedTypes.nonEmpty) { Some(unsupportedTypes.mkString("Found unsupported type:", ",", "")) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala index 7034a5a8f0e9..8162e538cf28 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala @@ -21,6 +21,7 @@ import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.extension.ValidationResult import org.apache.gluten.metrics.MetricsUpdater +import org.apache.gluten.planner.plan.GlutenPlanModel.GroupLeafExec import org.apache.gluten.substrait.`type`.ColumnTypeNode import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.extensions.ExtensionBuilder @@ -29,12 +30,15 @@ import org.apache.gluten.utils.SubstraitUtil import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, MapType} import org.apache.spark.sql.types.MetadataBuilder import io.substrait.proto.NamedStruct @@ -127,6 +131,37 @@ case class WriteFilesExecTransformer( override protected def doValidateInternal(): ValidationResult = { val finalChildOutput = getFinalChildOutput + + def isConstantComplexType(e: Expression): Boolean = { + e match { + case Literal(_, _: ArrayType | _: MapType) => true + case _ => e.children.exists(isConstantComplexType) + } + } + + lazy val hasConstantComplexType = child match { + case t: ProjectExecTransformer => + t.projectList.exists(isConstantComplexType) + case p: ProjectExec => + p.projectList.exists(isConstantComplexType) + case g: GroupLeafExec => // support the ras + g.metadata + .logicalLink() + .plan + .collectFirst { + case p: Project if p.projectList.exists(isConstantComplexType) => true + } + .isDefined + case _ => false + } + // TODO: currently the velox don't support parquet write with complex data type + // with constant. + if (fileFormat.isInstanceOf[ParquetFileFormat] && hasConstantComplexType) { + return ValidationResult.failed( + "Unsupported native parquet write: " + + "complex data type with constant") + } + val validationResult = BackendsApiManager.getSettings.supportWriteFilesExec( fileFormat, diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala index 9e4d94e1c298..bd1c269843fb 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -16,6 +16,27 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} -class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 0f3c43dfdf99..ec13265f9e85 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -963,6 +963,13 @@ class VeloxTestSettings extends BackendTestSettings { // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") enableSuite[GlutenFileSourceCharVarcharTestSuite] + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in map key") + .exclude("length check for input string values: nested in map value") + .exclude("length check for input string values: nested in both map key and value") + .exclude("length check for input string values: nested in array of struct") + .exclude("length check for input string values: nested in array of array") enableSuite[GlutenDSV2CharVarcharTestSuite] enableSuite[GlutenColumnExpressionSuite] // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala index 84502ace5110..89d9114870b6 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala @@ -16,8 +16,107 @@ */ package org.apache.spark.sql +import org.apache.spark.SparkException class GlutenFileSourceCharVarcharTestSuite extends FileSourceCharVarcharTestSuite - with GlutenSQLTestsTrait {} + with GlutenSQLTestsTrait { + private def testTableWrite(f: String => Unit): Unit = { + withTable("t")(f("char")) + withTable("t")(f("varchar")) + } + + private val ERROR_MESSAGE = + "Exceeds char/varchar type length limitation: 5" + + testGluten("length check for input string values: nested in struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT) USING $format") + sql("INSERT INTO t SELECT struct(null)") + checkAnswer(spark.table("t"), Row(Row(null))) + val e = intercept[RuntimeException] { + sql("INSERT INTO t SELECT struct('123456')") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY<$typeName(5)>) USING $format") + sql("INSERT INTO t VALUES (array(null))") + checkAnswer(spark.table("t"), Row(Seq(null))) + val e = intercept[SparkException] { + sql("INSERT INTO t VALUES (array('a', '123456'))") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map key") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), STRING>) USING $format") + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP) USING $format") + sql("INSERT INTO t VALUES (map('a', null))") + checkAnswer(spark.table("t"), Row(Map("a" -> null))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in both map key and value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), $typeName(5)>) USING $format") + val e1 = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e1.getMessage.contains(ERROR_MESSAGE)) + val e2 = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e2.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in struct of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT>) USING $format") + sql("INSERT INTO t SELECT struct(array(null))") + checkAnswer(spark.table("t"), Row(Row(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t SELECT struct(array('123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(struct(null)))") + checkAnswer(spark.table("t"), Row(Seq(Row(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(struct('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(array(null)))") + checkAnswer(spark.table("t"), Row(Seq(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(array('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } +} class GlutenDSV2CharVarcharTestSuite extends DSV2CharVarcharTestSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala index 9e4d94e1c298..bd1c269843fb 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -16,6 +16,27 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} -class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index a9525b1b0e4a..876d213b0583 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -983,6 +983,13 @@ class VeloxTestSettings extends BackendTestSettings { // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") enableSuite[GlutenFileSourceCharVarcharTestSuite] + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in map key") + .exclude("length check for input string values: nested in map value") + .exclude("length check for input string values: nested in both map key and value") + .exclude("length check for input string values: nested in array of struct") + .exclude("length check for input string values: nested in array of array") enableSuite[GlutenDSV2CharVarcharTestSuite] enableSuite[GlutenColumnExpressionSuite] // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala index 84502ace5110..ce2f1b465e7f 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala @@ -16,8 +16,108 @@ */ package org.apache.spark.sql +import org.apache.spark.SparkException + class GlutenFileSourceCharVarcharTestSuite extends FileSourceCharVarcharTestSuite - with GlutenSQLTestsTrait {} + with GlutenSQLTestsTrait { + private def testTableWrite(f: String => Unit): Unit = { + withTable("t")(f("char")) + withTable("t")(f("varchar")) + } + + private val ERROR_MESSAGE = + "Exceeds char/varchar type length limitation: 5" + + testGluten("length check for input string values: nested in struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT) USING $format") + sql("INSERT INTO t SELECT struct(null)") + checkAnswer(spark.table("t"), Row(Row(null))) + val e = intercept[RuntimeException] { + sql("INSERT INTO t SELECT struct('123456')") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY<$typeName(5)>) USING $format") + sql("INSERT INTO t VALUES (array(null))") + checkAnswer(spark.table("t"), Row(Seq(null))) + val e = intercept[SparkException] { + sql("INSERT INTO t VALUES (array('a', '123456'))") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map key") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), STRING>) USING $format") + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP) USING $format") + sql("INSERT INTO t VALUES (map('a', null))") + checkAnswer(spark.table("t"), Row(Map("a" -> null))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in both map key and value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), $typeName(5)>) USING $format") + val e1 = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e1.getMessage.contains(ERROR_MESSAGE)) + val e2 = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e2.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in struct of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT>) USING $format") + sql("INSERT INTO t SELECT struct(array(null))") + checkAnswer(spark.table("t"), Row(Row(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t SELECT struct(array('123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(struct(null)))") + checkAnswer(spark.table("t"), Row(Seq(Row(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(struct('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(array(null)))") + checkAnswer(spark.table("t"), Row(Seq(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(array('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } +} class GlutenDSV2CharVarcharTestSuite extends DSV2CharVarcharTestSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala index 9e4d94e1c298..bd1c269843fb 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -16,6 +16,27 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} -class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} From 27ab77c97b72bac0614ae4f1ec749047089dca2b Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Thu, 7 Nov 2024 14:10:25 +0800 Subject: [PATCH 042/211] [GLUTEN-7759][CH]Fix pre project push down in aggregate (#7779) * fix pre project * add test * another fix * fix ci * fix ci * fix review --------- Co-authored-by: zouyunhe <811-zouyunhe@users.noreply.git.sysop.bigo.sg> --- ...ownAggregatePreProjectionAheadExpand.scala | 26 ++++++++++++++++--- ...enClickHouseTPCHSaltNullParquetSuite.scala | 25 ++++++++++++++++++ 2 files changed, 48 insertions(+), 3 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala index a3fab3c954ee..21f1be2f2f2e 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan // If there is an expression (not a attribute) in an aggregation function's -// parameters. It will introduce a pr-projection to calculate the expression +// parameters. It will introduce a pre-projection to calculate the expression // at first, and make all the parameters be attributes. // If it's a aggregation with grouping set, this pre-projection is placed after // expand operator. This is not efficient, we cannot move this pre-projection @@ -83,7 +83,7 @@ case class PushdownAggregatePreProjectionAheadExpand(session: SparkSession) val originInputAttributes = aheadProjectExprs.filter(e => isAttributeOrLiteral(e)) val preProjectExprs = aheadProjectExprs.filter(e => !isAttributeOrLiteral(e)) - if (preProjectExprs.length == 0) { + if (preProjectExprs.isEmpty) { return hashAggregate } @@ -93,11 +93,31 @@ case class PushdownAggregatePreProjectionAheadExpand(session: SparkSession) return hashAggregate } + def projectInputExists(expr: Expression, inputs: Seq[Attribute]): Boolean = { + expr.children.foreach { + case a: Attribute => + return inputs.indexOf(a) != -1 + case p: Expression => + return projectInputExists(p, inputs) + case _ => + return true + } + true + } + + val couldPushDown = preProjectExprs.forall { + case p: Expression => projectInputExists(p, rootChild.output) + case _ => true + } + + if (!couldPushDown) { + return hashAggregate; + } + // The new ahead project node will take rootChild's output and preProjectExprs as the // the projection expressions. val aheadProject = ProjectExecTransformer(rootChild.output ++ preProjectExprs, rootChild) val aheadProjectOuput = aheadProject.output - val preProjectOutputAttrs = aheadProjectOuput.filter( e => !originInputAttributes.exists(_.exprId.equals(e.asInstanceOf[NamedExpression].exprId))) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 40b704d2e8d7..12047b300c9c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -3067,5 +3067,30 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |""".stripMargin compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) } + + test("GLUTEN-7759: Fix bug of agg pre-project push down") { + val table_create_sql = + "create table test_tbl_7759(id bigint, name string, day string) using parquet" + val insert_data_sql = + "insert into test_tbl_7759 values(1, 'a123', '2024-11-01'),(2, 'a124', '2024-11-01')" + val query_sql = + """ + |select distinct day, name from( + |select '2024-11-01' as day + |,coalesce(name,'all') name + |,cnt + |from + |( + |select count(distinct id) as cnt, name + |from test_tbl_7759 + |group by name + |with cube + |)) limit 10 + |""".stripMargin + spark.sql(table_create_sql) + spark.sql(insert_data_sql) + compareResultsAgainstVanillaSpark(query_sql, true, { _ => }) + spark.sql("drop table test_tbl_7759") + } } // scalastyle:on line.size.limit From c653337cdf54067cd4a01d14b908a521fdd11b3a Mon Sep 17 00:00:00 2001 From: Shuai li Date: Thu, 7 Nov 2024 14:17:39 +0800 Subject: [PATCH 043/211] [GLUTEN-7795][CH] Remove duplicate log object (#7839) --- cpp-ch/local-engine/Common/DebugUtils.cpp | 9 +++++---- .../Parser/SerializedPlanParser.cpp | 20 +++++++++---------- .../Parser/SerializedPlanParser.h | 1 - 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index 7b67f7d81227..2fcab59bf856 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include namespace pb_util = google::protobuf::util; @@ -51,9 +52,9 @@ void dumpPlan(DB::QueryPlan & plan, bool force, LoggerPtr logger) auto out = local_engine::PlanUtil::explainPlan(plan); if (force) // force - LOG_ERROR(logger, "clickhouse plan:\n{}", out); + LOG_ERROR(logger, "clickhouse plan({}):\n{}", local_engine::QueryContext::instance().currentTaskIdOrEmpty(), out); else - LOG_DEBUG(logger, "clickhouse plan:\n{}", out); + LOG_DEBUG(logger, "clickhouse plan({}):\n{}", local_engine::QueryContext::instance().currentTaskIdOrEmpty(), out); } void dumpMessage(const google::protobuf::Message & message, const char * type, bool force, LoggerPtr logger) @@ -73,9 +74,9 @@ void dumpMessage(const google::protobuf::Message & message, const char * type, b throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); if (force) // force - LOG_ERROR(logger, "{}:\n{}", type, json); + LOG_ERROR(logger, "{}({}):\n{}", type, local_engine::QueryContext::instance().currentTaskIdOrEmpty(), json); else - LOG_DEBUG(logger, "{}:\n{}", type, json); + LOG_DEBUG(logger, "{}({}):\n{}", type, local_engine::QueryContext::instance().currentTaskIdOrEmpty(), json); } void headBlock(const DB::Block & block, size_t count) diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 4c1d5b902da5..3b70069ca410 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -62,7 +62,6 @@ #include #include #include -#include #include #include @@ -111,8 +110,8 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con const auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); if (cols.getNames().size() != static_cast(root_rel.root().names_size())) { - debug::dumpPlan(*query_plan, true, log); - debug::dumpMessage(root_rel, "substrait::PlanRel", true, log); + debug::dumpPlan(*query_plan, true); + debug::dumpMessage(root_rel, "substrait::PlanRel", true); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan name size {}.", @@ -135,8 +134,8 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con const auto & original_cols = original_header.getColumnsWithTypeAndName(); if (static_cast(output_schema.types_size()) != original_cols.size()) { - debug::dumpPlan(*query_plan, true, log); - debug::dumpMessage(root_rel, "substrait::PlanRel", true, log); + debug::dumpPlan(*query_plan, true); + debug::dumpMessage(root_rel, "substrait::PlanRel", true); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan output schema size {}, subtrait plan name size {}.", @@ -184,7 +183,7 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) { - debug::dumpMessage(plan, "substrait::Plan", false, log); + debug::dumpMessage(plan, "substrait::Plan"); //parseExtensions(plan.extensions()); if (plan.relations_size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "too many relations found"); @@ -205,7 +204,7 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) PlanUtil::checkOuputType(*query_plan); #endif - debug::dumpPlan(*query_plan, false, log); + debug::dumpPlan(*query_plan); return query_plan; } @@ -324,7 +323,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla } catch (...) { - LOG_ERROR(log, "Invalid plan:\n{}", PlanUtil::explainPlan(*query_plan)); + LOG_ERROR(getLogger("SerializedPlanParser"), "Invalid plan:\n{}", PlanUtil::explainPlan(*query_plan)); throw; } @@ -333,9 +332,9 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla assert(root_rel.has_root()); if (root_rel.root().input().has_write()) addSinkTransform(parser_context->queryContext(), root_rel.root().input().write(), builder); - LOG_INFO(log, "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); + LOG_INFO(getLogger("SerializedPlanParser"), "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); LOG_DEBUG( - log, + getLogger("SerializedPlanParser"), "clickhouse plan [optimization={}]:\n{}", settings[Setting::query_plan_enable_optimizations], PlanUtil::explainPlan(*query_plan)); @@ -347,7 +346,6 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla SerializedPlanParser::SerializedPlanParser(ParserContextPtr parser_context_) : parser_context(parser_context_) { context = parser_context->queryContext(); - log = getLogger("SerializedPlanParser(" + QueryContext::instance().currentTaskIdOrEmpty() + ")"); } NonNullableColumnsResolver::NonNullableColumnsResolver( diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index 5fd84433132e..2bcc09a8ed9e 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -120,7 +120,6 @@ class SerializedPlanParser DB::QueryPlanPtr parseOp(const substrait::Rel & rel, std::list & rel_stack); void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) const; - LoggerPtr log; std::vector input_iters; std::vector split_infos; int split_info_index = 0; From f16915b1f91317cef5bfa34ac1feb8c5ac65aa25 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Thu, 7 Nov 2024 04:33:52 -0600 Subject: [PATCH 044/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241107) (#7835) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241107) * Revert "Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/68682", see https://github.com/ClickHouse/ClickHouse/pull/71527 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- .../Parser/RelParsers/CrossRelParser.cpp | 15 ++---------- .../Parser/RelParsers/JoinRelParser.cpp | 23 ++++--------------- cpp-ch/local-engine/tests/gtest_ch_join.cpp | 8 ++----- 4 files changed, 10 insertions(+), 40 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 1a089bdae0d0..20e3f8828cd6 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241106 -CH_COMMIT=32ca18ed214 \ No newline at end of file +CH_BRANCH=rebase_ch/20241107 +CH_COMMIT=6528c9d384d \ No newline at end of file diff --git a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp index 2e7f3531568f..59d4c39f8737 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp @@ -200,14 +200,7 @@ DB::QueryPlanPtr CrossRelParser::parseJoin(const substrait::CrossRel & join, DB: { JoinPtr hash_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty()); QueryPlanStepPtr join_step = std::make_unique( - left->getCurrentHeader(), - right->getCurrentHeader(), - hash_join, - context->getSettingsRef()[Setting::max_block_size], - 1, - /* required_output_ = */ NameSet{}, - false, - /* use_new_analyzer_ = */ false); + left->getCurrentHeader(), right->getCurrentHeader(), hash_join, context->getSettingsRef()[Setting::max_block_size], 1, false); join_step->setStepDescription("CROSS_JOIN"); steps.emplace_back(join_step.get()); std::vector plans; @@ -254,11 +247,7 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left NameSet left_columns_set; for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); - table_join.setColumnsFromJoinedTable( - right.getCurrentHeader().getNamesAndTypesList(), - left_columns_set, - getUniqueName("right") + ".", - left.getCurrentHeader().getNamesAndTypesList()); + table_join.setColumnsFromJoinedTable(right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); // fix right table key duplicate NamesWithAliases right_table_alias; diff --git a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp index 99ec54306676..46f7926cf729 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp @@ -315,14 +315,7 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q JoinPtr smj_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty(), -1); MultiEnum join_algorithm = context->getSettingsRef()[Setting::join_algorithm]; QueryPlanStepPtr join_step = std::make_unique( - left->getCurrentHeader(), - right->getCurrentHeader(), - smj_join, - context->getSettingsRef()[Setting::max_block_size], - 1, - /* required_output_ = */ NameSet{}, - false, - /* use_new_analyzer_ = */ false); + left->getCurrentHeader(), right->getCurrentHeader(), smj_join, context->getSettingsRef()[Setting::max_block_size], 1, false); join_step->setStepDescription("SORT_MERGE_JOIN"); steps.emplace_back(join_step.get()); @@ -390,11 +383,7 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, NameSet left_columns_set; for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); - table_join.setColumnsFromJoinedTable( - right.getCurrentHeader().getNamesAndTypesList(), - left_columns_set, - getUniqueName("right") + ".", - left.getCurrentHeader().getNamesAndTypesList()); + table_join.setColumnsFromJoinedTable(right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); // fix right table key duplicate NamesWithAliases right_table_alias; @@ -790,9 +779,7 @@ DB::QueryPlanPtr JoinRelParser::buildMultiOnClauseHashJoin( hash_join, context->getSettingsRef()[Setting::max_block_size], 1, - /* required_output_ = */ NameSet{}, - false, - /* use_new_analyzer_ = */ false); + false); join_step->setStepDescription("Multi join on clause hash join"); steps.emplace_back(join_step.get()); std::vector plans; @@ -831,9 +818,7 @@ DB::QueryPlanPtr JoinRelParser::buildSingleOnClauseHashJoin( hash_join, context->getSettingsRef()[Setting::max_block_size], 1, - /* required_output_ = */ NameSet{}, - false, - /* use_new_analyzer_ = */ false); + false); join_step->setStepDescription("HASH_JOIN"); steps.emplace_back(join_step.get()); diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index 2d853b2eba01..52120cede088 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -97,10 +97,6 @@ TEST(TestJoin, simple) for (const auto & column : join->columnsFromJoinedTable()) join->addJoinedColumn(column); - auto columns_from_left_table = left_plan.getCurrentHeader().getNamesAndTypesList(); - for (auto & column_from_joined_table : columns_from_left_table) - join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); - auto left_keys = left.getNamesAndTypesList(); join->addJoinedColumnsAndCorrectTypes(left_keys, true); std::cerr << "after join:\n"; @@ -126,8 +122,8 @@ TEST(TestJoin, simple) } auto hash_join = std::make_shared(join, right_plan.getCurrentHeader()); - QueryPlanStepPtr join_step = std::make_unique( - left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 1, NameSet{}, false, false); + QueryPlanStepPtr join_step + = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 1, false); std::cerr << "join step:" << join_step->getOutputHeader().dumpStructure() << std::endl; From e204dc5f06eef147d88b5ae6e8532b3902b19153 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Thu, 7 Nov 2024 20:38:58 +0800 Subject: [PATCH 045/211] [VL] Fix ccache installation in docker (#7848) --- dev/docker/Dockerfile.centos7-static-build | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/docker/Dockerfile.centos7-static-build b/dev/docker/Dockerfile.centos7-static-build index a0da8bda5b47..5fe3e15684e8 100644 --- a/dev/docker/Dockerfile.centos7-static-build +++ b/dev/docker/Dockerfile.centos7-static-build @@ -11,7 +11,7 @@ RUN sed -i \ -e 's/mirror\.centos\.org/vault.centos.org/' \ /etc/yum.repos.d/CentOS-SCLo-scl-rh.repo -RUN yum install -y git patch wget sudo java-1.8.0-openjdk-devel ccache +RUN yum install -y git patch wget sudo java-1.8.0-openjdk-devel epel-release ccache RUN git clone --depth=1 https://github.com/apache/incubator-gluten /opt/gluten From 6abde8f8d61e9fc61130a42735a8a377d382d129 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 8 Nov 2024 10:33:43 +0800 Subject: [PATCH 046/211] [CORE] Minor: Rename LimitTransformer to LimitExecTransformer (#7843) --- .../backendsapi/velox/VeloxSparkPlanExecApi.scala | 2 +- .../apache/gluten/execution/MiscOperatorSuite.scala | 2 +- .../execution/VeloxOrcDataTypeValidationSuite.scala | 2 +- .../VeloxParquetDataTypeValidationSuite.scala | 2 +- ...mitTransformer.scala => LimitExecTransformer.scala} | 4 ++-- .../TakeOrderedAndProjectExecTransformer.scala | 10 +++++----- .../gluten/extension/columnar/FallbackRules.scala | 4 ++-- .../gluten/extension/columnar/OffloadSingleNode.scala | 4 ++-- 8 files changed, 15 insertions(+), 15 deletions(-) rename gluten-substrait/src/main/scala/org/apache/gluten/execution/{LimitTransformer.scala => LimitExecTransformer.scala} (97%) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index 7901374f6bf6..5925666d2c00 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -806,7 +806,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { override def maybeCollapseTakeOrderedAndProject(plan: SparkPlan): SparkPlan = { // This to-top-n optimization assumes exchange operators were already placed in input plan. plan.transformUp { - case p @ LimitTransformer(SortExecTransformer(sortOrder, _, child, _), 0, count) => + case p @ LimitExecTransformer(SortExecTransformer(sortOrder, _, child, _), 0, count) => val global = child.outputPartitioning.satisfies(AllTuples) val topN = TopNTransformer(count, sortOrder, global, child) if (topN.doValidate().ok()) { diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index 7e9b48e96ca9..da5184f752b3 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -566,7 +566,7 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa | select * from lineitem limit 10 |) where l_suppkey != 0 limit 100; |""".stripMargin) { - checkGlutenOperatorMatch[LimitTransformer] + checkGlutenOperatorMatch[LimitExecTransformer] } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala index 3bf49d33c511..24e04f2dfce3 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala @@ -265,7 +265,7 @@ class VeloxOrcDataTypeValidationSuite extends VeloxWholeStageTransformerSuite { | select date, int from type1 limit 100 |) where int != 0 limit 10; |""".stripMargin) { - checkGlutenOperatorMatch[LimitTransformer] + checkGlutenOperatorMatch[LimitExecTransformer] } // Validation: Window. diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala index 8b6cc63c954d..57ca448fec79 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala @@ -264,7 +264,7 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit | select date, int from type1 limit 100 |) where int != 0 limit 10; |""".stripMargin) { - checkGlutenOperatorMatch[LimitTransformer] + checkGlutenOperatorMatch[LimitExecTransformer] } // Validation: Window. diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala similarity index 97% rename from gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitTransformer.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala index 8859844be48e..58ef90fd148a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.SparkPlan import scala.collection.JavaConverters._ -case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) +case class LimitExecTransformer(child: SparkPlan, offset: Long, count: Long) extends UnaryTransformSupport { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @@ -39,7 +39,7 @@ case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) override def output: Seq[Attribute] = child.output - override protected def withNewChildInternal(newChild: SparkPlan): LimitTransformer = + override protected def withNewChildInternal(newChild: SparkPlan): LimitExecTransformer = copy(child = newChild) override def metricsUpdater(): MetricsUpdater = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala index b31471e21397..c4e192d1837b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala @@ -73,7 +73,7 @@ case class TakeOrderedAndProjectExecTransformer( var tagged: ValidationResult = null val orderingSatisfies = SortOrder.orderingSatisfies(child.outputOrdering, sortOrder) if (orderingSatisfies) { - val limitPlan = LimitTransformer(child, offset, limit) + val limitPlan = LimitExecTransformer(child, offset, limit) tagged = limitPlan.doValidate() } else { // Here we are validating sort + limit which is a kind of whole stage transformer, @@ -86,7 +86,7 @@ case class TakeOrderedAndProjectExecTransformer( if (!sortValidation.ok()) { return sortValidation } - val limitPlan = LimitTransformer(sortPlan, offset, limit) + val limitPlan = LimitExecTransformer(sortPlan, offset, limit) tagged = limitPlan.doValidate() } @@ -127,13 +127,13 @@ case class TakeOrderedAndProjectExecTransformer( // remove this WholeStageTransformer, put the new sort, limit and project // into a new whole stage. val localSortPlan = withLocalSort(wholeStage.child) - LimitTransformer(localSortPlan, limitBeforeShuffleOffset, limit) + LimitExecTransformer(localSortPlan, limitBeforeShuffleOffset, limit) case other => // if the child it is not WholeStageTransformer, add the adapter first // so that, later we can wrap WholeStageTransformer. val localSortPlan = withLocalSort( ColumnarCollapseTransformStages.wrapInputIteratorTransformer(other)) - LimitTransformer(localSortPlan, limitBeforeShuffleOffset, limit) + LimitExecTransformer(localSortPlan, limitBeforeShuffleOffset, limit) } val transformStageCounter: AtomicInteger = ColumnarCollapseTransformStages.transformStageCounter @@ -150,7 +150,7 @@ case class TakeOrderedAndProjectExecTransformer( sortOrder, false, ColumnarCollapseTransformStages.wrapInputIteratorTransformer(transformedShuffleExec)) - LimitTransformer(localSortPlan, offset, limit) + LimitExecTransformer(localSortPlan, offset, limit) } val projectPlan = if (projectList != child.output) { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala index 794186bfa957..b557abe001a1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala @@ -405,10 +405,10 @@ case class AddFallbackTagRule() extends Rule[SparkPlan] { case plan: GlobalLimitExec => val (limit, offset) = SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) - val transformer = LimitTransformer(plan.child, offset, limit) + val transformer = LimitExecTransformer(plan.child, offset, limit) transformer.doValidate().tagOnFallback(plan) case plan: LocalLimitExec => - val transformer = LimitTransformer(plan.child, 0L, plan.limit) + val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) transformer.doValidate().tagOnFallback(plan) case plan: GenerateExec => val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala index ce5b21206986..220a45f7b35c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala @@ -298,11 +298,11 @@ object OffloadOthers { val child = plan.child val (limit, offset) = SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) - LimitTransformer(child, offset, limit) + LimitExecTransformer(child, offset, limit) case plan: LocalLimitExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") val child = plan.child - LimitTransformer(child, 0L, plan.limit) + LimitExecTransformer(child, 0L, plan.limit) case plan: GenerateExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") val child = plan.child From d8a6cde241b02eeb83aa8dfa6bc06fef70127c71 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Fri, 8 Nov 2024 10:51:05 +0800 Subject: [PATCH 047/211] [VL] Follow-up fix for PR #7848 to install ccache (#7858) --- dev/docker/Dockerfile.centos7-static-build | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/docker/Dockerfile.centos7-static-build b/dev/docker/Dockerfile.centos7-static-build index 5fe3e15684e8..6de2c73cd7b7 100644 --- a/dev/docker/Dockerfile.centos7-static-build +++ b/dev/docker/Dockerfile.centos7-static-build @@ -3,7 +3,7 @@ FROM centos:7 RUN sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true RUN sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true -RUN yum install -y centos-release-scl +RUN yum install -y epel-release centos-release-scl RUN rm /etc/yum.repos.d/CentOS-SCLo-scl.repo -f RUN sed -i \ -e 's/^mirrorlist/#mirrorlist/' \ @@ -11,7 +11,7 @@ RUN sed -i \ -e 's/mirror\.centos\.org/vault.centos.org/' \ /etc/yum.repos.d/CentOS-SCLo-scl-rh.repo -RUN yum install -y git patch wget sudo java-1.8.0-openjdk-devel epel-release ccache +RUN yum install -y git patch wget sudo java-1.8.0-openjdk-devel ccache RUN git clone --depth=1 https://github.com/apache/incubator-gluten /opt/gluten From 7435d69e0397906c90190f266971ce3b50d6ceef Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 8 Nov 2024 12:43:30 +0800 Subject: [PATCH 048/211] [GLUTEN-7458][VL] Upgrade GCC to version 11 in gluten-te's ubuntu dockerfile (#7859) --- tools/gluten-te/ubuntu/dockerfile-buildenv | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tools/gluten-te/ubuntu/dockerfile-buildenv b/tools/gluten-te/ubuntu/dockerfile-buildenv index 47c7b80d8619..df13e8726190 100644 --- a/tools/gluten-te/ubuntu/dockerfile-buildenv +++ b/tools/gluten-te/ubuntu/dockerfile-buildenv @@ -108,9 +108,18 @@ RUN cd /opt && wget https://github.com/Kitware/CMake/releases/download/v3.28.3/c && mkdir cmake \ && bash cmake-3.28.3-linux-x86_64.sh --skip-license --prefix=/opt/cmake \ && ln -s /opt/cmake/bin/cmake /usr/bin/cmake - RUN cmake --version +# Install GCC 11 +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y software-properties-common +RUN add-apt-repository ppa:ubuntu-toolchain-r/test +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y gcc-11 g++-11 +RUN rm -f /usr/bin/gcc /usr/bin/g++ +RUN ln -s /usr/bin/gcc-11 /usr/bin/gcc +RUN ln -s /usr/bin/g++-11 /usr/bin/g++ +RUN cc --version +RUN c++ --version + # Spark binaries WORKDIR /opt ARG BUILD_SPARK_BINARIES From 6cf278c3440cef2d448ec57bb40705ec4456c76d Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 8 Nov 2024 12:43:57 +0800 Subject: [PATCH 049/211] [CORE] Remove member TransformContext#inputAttributes as unused (#7844) --- .../gluten/execution/CHHashAggregateExecTransformer.scala | 2 +- .../execution/CHWindowGroupLimitExecTransformer.scala | 2 +- .../gluten/execution/HashAggregateExecTransformer.scala | 2 +- .../scala/org/apache/gluten/execution/TopNTransformer.scala | 2 +- .../gluten/execution/BasicPhysicalOperatorTransformer.scala | 6 +++--- .../apache/gluten/execution/BasicScanExecTransformer.scala | 2 +- .../gluten/execution/CartesianProductExecTransformer.scala | 2 +- .../org/apache/gluten/execution/ExpandExecTransformer.scala | 2 +- .../gluten/execution/GenerateExecTransformerBase.scala | 2 +- .../main/scala/org/apache/gluten/execution/JoinUtils.scala | 2 +- .../org/apache/gluten/execution/LimitExecTransformer.scala | 2 +- .../org/apache/gluten/execution/SampleExecTransformer.scala | 2 +- .../org/apache/gluten/execution/SortExecTransformer.scala | 2 +- .../org/apache/gluten/execution/WholeStageTransformer.scala | 5 +---- .../org/apache/gluten/execution/WindowExecTransformer.scala | 2 +- .../gluten/execution/WindowGroupLimitExecTransformer.scala | 2 +- .../apache/gluten/execution/WriteFilesExecTransformer.scala | 2 +- .../sql/execution/ColumnarCollapseTransformStages.scala | 2 +- .../sql/execution/python/EvalPythonExecTransformer.scala | 2 +- 19 files changed, 21 insertions(+), 24 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala index fcf6320f8ec4..48b0d7336103 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala @@ -209,7 +209,7 @@ case class CHHashAggregateExecTransformer( RelBuilder.makeReadRelForInputIteratorWithoutRegister(typeList, nameList, context) (getAggRel(context, operatorId, aggParams, readRel), inputAttrs, outputAttrs) } - TransformContext(inputAttributes, outputAttributes, relNode) + TransformContext(outputAttributes, relNode) } override def getAggRel( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala index c2648f29ec4c..793d733abf96 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala @@ -182,6 +182,6 @@ case class CHWindowGroupLimitExecTransformer( val currRel = getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Group Limit Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala index fe5e0d92d6d5..61af545e411d 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala @@ -72,7 +72,7 @@ abstract class HashAggregateExecTransformer( val aggParams = new AggregationParams val operatorId = context.nextOperatorId(this.nodeName) val relNode = getAggRel(context, operatorId, aggParams, childCtx.root) - TransformContext(childCtx.outputAttributes, output, relNode) + TransformContext(output, relNode) } // Return whether the outputs partial aggregation should be combined for Velox computing. diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala index 01c89bee217b..f3adbe351aa4 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala @@ -79,7 +79,7 @@ case class TopNTransformer( child.output, childCtx.root, validation = false) - TransformContext(child.output, child.output, relNode) + TransformContext(child.output, relNode) } private def getRelNode( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index 09b56a5ac717..76505da3e0cc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -156,7 +156,7 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP context.registerEmptyRelToOperator(operatorId) // Since some columns' nullability will be removed after this filter, we need to update the // outputAttributes of child context. - return TransformContext(childCtx.inputAttributes, output, childCtx.root) + return TransformContext(output, childCtx.root) } val currRel = getRelNode( context, @@ -166,7 +166,7 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP childCtx.root, validation = false) assert(currRel != null, "Filter rel should be valid.") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } } @@ -205,7 +205,7 @@ abstract class ProjectExecTransformerBase(val list: Seq[NamedExpression], val in val currRel = getRelNode(context, list, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Project Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override def output: Seq[Attribute] = list.map(_.toAttribute) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala index f272dc3eca72..d768ac2c5936 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala @@ -162,6 +162,6 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource extensionNode, context, context.nextOperatorId(this.nodeName)) - TransformContext(output, output, readNode) + TransformContext(output, readNode) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala index 28bf1eeabd23..7f7e54e9c72c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala @@ -105,7 +105,7 @@ case class CartesianProductExecTransformer( context, operatorId ) - TransformContext(inputLeftOutput ++ inputRightOutput, output, currRel) + TransformContext(output, currRel) } override protected def doValidateInternal(): ValidationResult = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala index 63f76a25a231..b600175b2826 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala @@ -122,7 +122,7 @@ case class ExpandExecTransformer( val currRel = getRelNode(context, projections, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Expand Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): ExpandExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala index af4a92f194c1..698d1f14c5b9 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala @@ -79,7 +79,7 @@ abstract class GenerateExecTransformerBase( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) val relNode = getRelNode(context, childCtx.root, getGeneratorNode(context), validation = false) - TransformContext(child.output, output, relNode) + TransformContext(output, relNode) } protected def getExtensionNodeForValidation: AdvancedExtensionNode = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala index 12d08518509a..dae5d51af81a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala @@ -312,7 +312,7 @@ object JoinUtils { } else { inputStreamedOutput ++ inputBuildOutput } - TransformContext(inputAttributes, output, rel) + TransformContext(output, rel) } def createCrossRel( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala index 58ef90fd148a..0d49acd30f0d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala @@ -57,7 +57,7 @@ case class LimitExecTransformer(child: SparkPlan, offset: Long, count: Long) val childCtx = child.asInstanceOf[TransformSupport].transform(context) val operatorId = context.nextOperatorId(this.nodeName) val relNode = getRelNode(context, operatorId, offset, count, child.output, childCtx.root, false) - TransformContext(child.output, child.output, relNode) + TransformContext(child.output, relNode) } def getRelNode( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala index bed59b913a1e..8e664a3b6ebc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala @@ -118,7 +118,7 @@ case class SampleExecTransformer( val currRel = getRelNode(context, condition, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Filter rel should be valid.") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): SampleExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala index b69925d60fd2..c62a30b84632 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala @@ -113,7 +113,7 @@ case class SortExecTransformer( val currRel = getRelNode(context, sortOrder, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Sort Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): SortExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index e1dfd3f5704a..70839ffc2eba 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -47,10 +47,7 @@ import com.google.common.collect.Lists import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -case class TransformContext( - inputAttributes: Seq[Attribute], - outputAttributes: Seq[Attribute], - root: RelNode) +case class TransformContext(outputAttributes: Seq[Attribute], root: RelNode) case class WholeStageTransformContext(root: PlanNode, substraitContext: SubstraitContext = null) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala index 4902b6c6cf1b..7b9e2865f883 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala @@ -187,7 +187,7 @@ case class WindowExecTransformer( val currRel = getWindowRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): WindowExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala index 6068412fbad3..5d8a18b11164 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala @@ -163,6 +163,6 @@ case class WindowGroupLimitExecTransformer( val currRel = getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Group Limit Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala index 8162e538cf28..c93d2dda5121 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala @@ -185,7 +185,7 @@ case class WriteFilesExecTransformer( val currRel = getRelNode(context, getFinalChildOutput, operatorId, childCtx.root, validation = false) assert(currRel != null, "Write Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): WriteFilesExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index d222dcfef84f..5ea5e4159dda 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -75,7 +75,7 @@ case class InputIteratorTransformer(child: SparkPlan) extends UnaryTransformSupp override protected def doTransform(context: SubstraitContext): TransformContext = { val operatorId = context.nextOperatorId(nodeName) val readRel = RelBuilder.makeReadRelForInputIterator(child.output.asJava, context, operatorId) - TransformContext(output, output, readRel) + TransformContext(output, readRel) } override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala index 6a9da0a9cf92..7b4c09d4f9e5 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala @@ -99,7 +99,7 @@ case class EvalPythonExecTransformer( val relNode = getRelNode(childCtx.root, expressionNodes, context, operatorId, child.output, false) - TransformContext(child.output, output, relNode) + TransformContext(output, relNode) } def getRelNode( From 3e6263e204aee9b5f1d1afe06102cb97251c08ed Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 8 Nov 2024 12:44:17 +0800 Subject: [PATCH 050/211] [VL] Re-enable background IO threads by default (#7845) --- .github/workflows/velox_backend.yml | 42 ------------------- .../apache/gluten/test/MockVeloxBackend.java | 1 + .../org/apache/gluten/GlutenConfig.scala | 18 ++++---- 3 files changed, 9 insertions(+), 52 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 6e9d0eab7c70..737895be57bd 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -249,48 +249,6 @@ jobs: --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ --extra-conf=spark.gluten.ras.enabled=true - run-tpc-test-ubuntu-iothreads: - needs: build-native-lib-centos-7 - strategy: - fail-fast: false - matrix: - spark: [ "spark-3.5" ] - runs-on: ubuntu-20.04 - steps: - - uses: actions/checkout@v2 - - name: Download All Native Artifacts - uses: actions/download-artifact@v3 - with: - name: velox-native-lib-centos-7-${{github.sha}} - path: ./cpp/build/releases/ - - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v3 - with: - name: arrow-jars-centos-7-${{github.sha}} - path: /home/runner/.m2/repository/org/apache/arrow/ - - name: Setup java and maven - run: | - sudo apt-get update - sudo apt-get install -y openjdk-8-jdk maven - - name: Set environment variables - run: | - echo "JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64" >> $GITHUB_ENV - - name: Build for Spark ${{ matrix.spark }} - run: | - cd $GITHUB_WORKSPACE/ - $MVN_CMD clean install -P${{ matrix.spark }} -Pbackends-velox -DskipTests - cd $GITHUB_WORKSPACE/tools/gluten-it - $MVN_CMD clean install -P${{ matrix.spark }} - - name: Build and run TPC-H / TPC-DS - run: | - cd $GITHUB_WORKSPACE/tools/gluten-it - GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.IOThreads=16 - GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.IOThreads=16 - run-tpc-test-ubuntu-oom: needs: build-native-lib-centos-7 strategy: diff --git a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java index b5ce30e39b25..254e8e2830f4 100644 --- a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java +++ b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java @@ -71,6 +71,7 @@ public Object ask(Object message) throws Exception { private static SparkConf newSparkConf() { final SparkConf conf = new SparkConf(); conf.set(GlutenConfig.SPARK_OFFHEAP_SIZE_KEY(), "1g"); + conf.set(GlutenConfig.COLUMNAR_VELOX_CONNECTOR_IO_THREADS(), "0"); return conf; } } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index a28a7d26b386..3e491eb2753f 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -291,7 +291,7 @@ class GlutenConfig(conf: SQLConf) extends Logging { def veloxSsdODirectEnabled: Boolean = conf.getConf(COLUMNAR_VELOX_SSD_ODIRECT_ENABLED) def veloxConnectorIOThreads: Int = { - conf.getConf(COLUMNAR_VELOX_CONNECTOR_IO_THREADS) + conf.getConf(COLUMNAR_VELOX_CONNECTOR_IO_THREADS).getOrElse(numTaskSlotsPerExecutor) } def veloxSplitPreloadPerDriver: Integer = conf.getConf(COLUMNAR_VELOX_SPLIT_PRELOAD_PER_DRIVER) @@ -759,7 +759,9 @@ object GlutenConfig { (AWS_S3_RETRY_MODE.key, AWS_S3_RETRY_MODE.defaultValueString), ( COLUMNAR_VELOX_CONNECTOR_IO_THREADS.key, - COLUMNAR_VELOX_CONNECTOR_IO_THREADS.defaultValueString), + conf.getOrElse( + NUM_TASK_SLOTS_PER_EXECUTOR.key, + NUM_TASK_SLOTS_PER_EXECUTOR.defaultValueString)), (COLUMNAR_SHUFFLE_CODEC.key, ""), (COLUMNAR_SHUFFLE_CODEC_BACKEND.key, ""), ("spark.hadoop.input.connect.timeout", "180000"), @@ -1449,19 +1451,15 @@ object GlutenConfig { .booleanConf .createWithDefault(false) - // FIXME: May cause issues when toggled on. Examples: - // https://github.com/apache/incubator-gluten/issues/7161 - // https://github.com/facebookincubator/velox/issues/10173 val COLUMNAR_VELOX_CONNECTOR_IO_THREADS = buildStaticConf("spark.gluten.sql.columnar.backend.velox.IOThreads") .internal() .doc( - "Experimental: The Size of the IO thread pool in the Connector." + - " This thread pool is used for split preloading and DirectBufferedInput." + - " The option is experimental. Toggling on it (setting a non-zero value) may cause some" + - " unexpected issues when application reaches some certain conditions.") + "The Size of the IO thread pool in the Connector. " + + "This thread pool is used for split preloading and DirectBufferedInput. " + + "By default, the value is the same as the maximum task slots per Spark executor.") .intConf - .createWithDefault(0) + .createOptional val COLUMNAR_VELOX_ASYNC_TIMEOUT = buildStaticConf("spark.gluten.sql.columnar.backend.velox.asyncTimeoutOnTaskStopping") From cfb55ef933d2209edb443dada88c267691f0eb38 Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Fri, 8 Nov 2024 14:05:26 +0800 Subject: [PATCH 051/211] native writer support CreateHiveTableAsSelectCommand (#7851) --- .../sql/execution/VeloxParquetWriteForHiveSuite.scala | 9 +++++++++ .../datasources/GlutenWriterColumnarRules.scala | 6 ++++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala index 412548de9c44..8c2b98988ea2 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala @@ -176,4 +176,13 @@ class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { checkAnswer(sql("SELECT * FROM t"), Row(1)) } } + + test("native writer support CreateHiveTableAsSelectCommand") { + withTable("t") { + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { + checkNativeWrite("CREATE TABLE t STORED AS PARQUET AS SELECT 1 as c", checkNative = true) + } + checkAnswer(spark.table("t"), Row(1)) + } + } } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala index d33e779eb332..917f6c03dfe0 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala @@ -121,8 +121,10 @@ object GlutenWriterColumnarRules { command.table.storage.outputFormat .flatMap(formatMapping.get) .filter(GlutenFormatFactory.isRegistered) - case _: CreateHiveTableAsSelectCommand => - None + case command: CreateHiveTableAsSelectCommand => + command.tableDesc.storage.outputFormat + .flatMap(formatMapping.get) + .filter(GlutenFormatFactory.isRegistered) case _ => None } From aac1b87cde55163c8c54c725bee1e112d2b25444 Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Fri, 8 Nov 2024 14:09:18 +0800 Subject: [PATCH 052/211] fix pre-projection not take effect (#7863) --- .../extension/PushdownAggregatePreProjectionAheadExpand.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala index 21f1be2f2f2e..9335fd8829a9 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala @@ -96,7 +96,7 @@ case class PushdownAggregatePreProjectionAheadExpand(session: SparkSession) def projectInputExists(expr: Expression, inputs: Seq[Attribute]): Boolean = { expr.children.foreach { case a: Attribute => - return inputs.indexOf(a) != -1 + return inputs.exists(i => i.name.equals(a.name) && i.exprId.equals(a.exprId)) case p: Expression => return projectInputExists(p, inputs) case _ => From a0378b759f86176c7bd2b3c9a68ed7daf9c148e3 Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Fri, 8 Nov 2024 15:32:45 +0800 Subject: [PATCH 053/211] [CH][Doc] Add Gluten CH Debug docs. (#7846) --- docs/developers/clickhouse-backend-debug.md | 98 ++++++++++++++++++ .../ClickHouse/gluten-debug-build-libch.png | Bin 0 -> 224526 bytes .../ClickHouse/gluten-debug-clion-debug.png | Bin 0 -> 290162 bytes .../gluten-debug-clion-toolchains.png | Bin 0 -> 164977 bytes .../ClickHouse/gluten-debug-cmake-debug.png | Bin 0 -> 254090 bytes .../ClickHouse/gluten-debug-cmake-reload.png | Bin 0 -> 412031 bytes .../ClickHouse/gluten-debug-idea-config.png | Bin 0 -> 226892 bytes .../gluten-debug-profile-settings.png | Bin 0 -> 28429 bytes .../ClickHouse/gluten-debug-program-args.png | Bin 0 -> 175558 bytes 9 files changed, 98 insertions(+) create mode 100644 docs/developers/clickhouse-backend-debug.md create mode 100644 docs/image/ClickHouse/gluten-debug-build-libch.png create mode 100644 docs/image/ClickHouse/gluten-debug-clion-debug.png create mode 100644 docs/image/ClickHouse/gluten-debug-clion-toolchains.png create mode 100644 docs/image/ClickHouse/gluten-debug-cmake-debug.png create mode 100644 docs/image/ClickHouse/gluten-debug-cmake-reload.png create mode 100644 docs/image/ClickHouse/gluten-debug-idea-config.png create mode 100644 docs/image/ClickHouse/gluten-debug-profile-settings.png create mode 100644 docs/image/ClickHouse/gluten-debug-program-args.png diff --git a/docs/developers/clickhouse-backend-debug.md b/docs/developers/clickhouse-backend-debug.md new file mode 100644 index 000000000000..13799ac2cba2 --- /dev/null +++ b/docs/developers/clickhouse-backend-debug.md @@ -0,0 +1,98 @@ +--- +layout: page +title: Debug CH Backend +nav_order: 13 +has_children: true +parent: /developer-overview/ +--- + +# Debug CH Backend + +## Debug Java/Scala Code with IntelliJ IDEA + +1. Build Gluten ClickHouse Native Lib. + ``` + export CMAKE_BUILD_TYPE=Release && bash ep/build-clickhouse/src/build_clickhouse.sh + ``` + libch.so will be generated in `cpp-ch/build/utils/extern-local-engine/libch.so`. + +2. Maven Build Gluten ClickHouse with Profile + ``` + mvn clean install -DskipTests -Pbackends-clickhouse -Pspark-3.3 -Pspark-ut + ``` + +3. Set Maven Profiles in IntelliJ IDEA + + ![gluten-debug-profile-settings.png](../image/ClickHouse/gluten-debug-profile-settings.png) + +4. Set Debug Configuration in IntelliJ IDEA + + For example, debug GlutenMathExpressionsSuite. + ![gluten-debug-idea-config.png](../image/ClickHouse/gluten-debug-idea-config.png) + + VM Options: + `-Dtpcds.data.path=/data/tpcds-data-sf1 -Dclickhouse.lib.path=/path/to/gluten/cpp-ch/build/utils/extern-local-engine/libch.so -Dspark.test.home=/path/to/spark33` + > Download tpcds-data in https://gluten-nginx.kyligence.com/dataset/ + > Download spark33 using `git clone --depth 1 --branch v3.3.1 https://github.com/apache/spark.git /tmp/spark33` + + Environment Variables: + `LD_PRELOAD=/path/to/gluten/cpp-ch/build/utils/extern-local-engine/libch.so:/usr/lib/jvm/java-1.8.0-openjdk-amd64/jre/lib/amd64/libjsig.so` + +## Debug Native Code with CLion + +1. Toolchains Settings + + ![gluten-debug-clion-toolchains](../image/ClickHouse/gluten-debug-clion-toolchains.png) + > Some Clion versions have not supported lldb-18, you can manually set refer to [CLion Custom LLDB](https://youtrack.jetbrains.com/issue/CPP-3589/Support-using-custom-LLDB). Or you can use gdb as default. + +2. CMake Debug Configuration + + ![gluten-debug-clion-debug](../image/ClickHouse/gluten-debug-clion-debug.png) + + CMake Options: + `-DENABLE_PROTOBUF=ON -DENABLE_TESTS=OFF -DENABLE_BENCHMARKS=ON -DENABLE_JEMALLOC=ON -DENABLE_MULTITARGET_CODE=ON -DENABLE_EXTERN_LOCAL_ENGINE=ON -DENABLE_ODBC=OFF -DENABLE_CAPNP=OFF -DENABLE_ROCKSDB=OFF -DENABLE_GRPC=OFF -DENABLE_RUST=OFF -DENABLE_H3=OFF -DENABLE_AMQPCPP=OFF -DENABLE_CASSANDRA=OFF -DENABLE_KAFKA=OFF -DENABLE_NATS=OFF -DENABLE_LIBPQXX=OFF -DENABLE_NURAFT=OFF -DENABLE_DATASKETCHES=OFF -DENABLE_SQLITE=OFF -DENABLE_S2_GEOMETRY=OFF -DENABLE_ANNOY=OFF -DENABLE_ULID=OFF -DENABLE_MYSQL=OFF -DENABLE_BCRYPT=OFF -DENABLE_LDAP=OFF -DENABLE_MSGPACK=OFF -DUSE_REPLXX=OFF -DENABLE_CLICKHOUSE_ALL=OFF -DCOMPILER_FLAGS="-fvisibility=hidden -fvisibility-inlines-hidden" -DENABLE_BUILD_PATH_MAPPING=OFF -DDEBUG_0_LEVEL="0"` + +3. Reload CMake Project + + ![gluten-debug-cmake-reload](../image/ClickHouse/gluten-debug-cmake-reload.png) + + After reload cmake projects, you can find target `libch` in run configurations. + +4. Build `libchd.so` with Debug Mode + + ![gluten-debug-build-libch](../image/ClickHouse/gluten-debug-build-libch.png) + + `libchd.so` will be generated in `cmake-build-debug/utils/extern-local-engine/libchd.so`. + +5. Create File `.gdbinit` and `.lldbinit` to Avoid Unused Signal + + vi ~/.gdbinit + ``` + handle SIGSEGV nostop noprint + + set print pretty on + set print object on + python + import sys + sys.path.insert(0, '/path/to/libcxx-pretty-printers/src') + from libcxx.v1.printers import register_libcxx_printers + register_libcxx_printers(None) + end + ``` + > Download libcxx-pretty-printers in https://github.com/koutheir/libcxx-pretty-printers + + vi ~/.lldbinit + ``` + process handle -n true -p true -s false SIGBUS SIGSEGV + ``` + +6. Debug Application + + ![gluten-debug-cmake-debug.png](../image/ClickHouse/gluten-debug-cmake-debug.png) + + Executable: `/path/to/java` + + Program Arguments: Copy from IntelliJ IDEA debug command line and remove `-javaagent` + ![gluten-debug-program-args.png](../image/ClickHouse/gluten-debug-program-args.png) + + diff --git a/docs/image/ClickHouse/gluten-debug-build-libch.png b/docs/image/ClickHouse/gluten-debug-build-libch.png new file mode 100644 index 0000000000000000000000000000000000000000..4cf914dc8b2ea297cd2de067c118e8804f71a9aa GIT binary patch literal 224526 zcmY)UV|bk1_dX6!(y+13#x@(Q z2$aEYshD*=s`tI100KLz&!kpcFc%IWj}KrKW+DqTJE2~GEhlQb8}t*48is9X1i#8m z0ZvgR->THY$x?x8GmO91lafUMEqcRK}8&1U)G@ zAKKPm&)kY$yB^0J-qxd*EynZSgN6L_8_#y?%5v?a%g5h`@>X0|M`Om{N4j3G*2Cnx z?#tI74a1jTS0K8_iOTLB-k$E?S{0+;jyyVrEMHja*EU0p!m4n&j!;hm`!TruNvctJ>ZOj5?Km zHNEZT`P7%P-aR__`#+snm;78@NAugO?rho)_Pu|2Fn1OOV4iyX735T{f5QqsI5)a1WY=`6E3Nx zEZrsli&uZEsDaMMCV$o%|E}|w@#?0=HvgyKE@F?C!MomaEf zpthi{r`u|hGu(>@5}(6Of_}TN_hbGqU0oO@1rF~!)lc7K-{#-7HP=z!rkXnSG&nbf z-p&Y)*Vlc<6yn|w^Za-{FDe~;uL&5PAGqJ122867;j!-3h6Aq*wE{XR<{SRgNQTrZ z7IriV@&t7{jhgPIMqnDV5%Nau0I^=P;}>@UG}d~ZyGI|8gjM;!U(1OHj(;VG?55CN zGC`Wu?Q;LnStup={!P!mBT)H+a19IqTM~g~_u7%r@?@cQL3Mp;X?PUMs*>AwAzxdm z{3jhFQ^F5y2TzDJZ6H->FGa-nswyr*>nN8C=C0RO@>M!*nxEmJ1c{fi0lgMATRh15 z)JnPfs;bUZ#MR~H1a13=hL9{m4wTXGc&Vh2#+ckOy0!Gl{Un zTZl~Tu>i+s!1q#q#B$jiMH3xO=VZS*wp1Ee0@~P6Mceh`7 z0+m`>Cc!of<_ZmFrY0xlwC3dMBHI|~*b#!r5wi?^{PnE_$H=Fa5gUR~wU7KDZeaTm ziC2xQObJ)mz`($p0{w{F>+FPAv|%ipmmO1Aha@J4&+X^>AE(6}o>>3UbXKr%R+g3u zIqwPBOm@Ccy1^23KF3h+k7m9eDy<8;pIG6D){&PZ=`eud!uc_QNaCwR9)l#A*G_^I z;ebx3Ps?%Zyn#o~4l-_sqa;
Y_lTDHPXquc(8Yh8 zw%xa#prmLer=*k|;RawZv|R2j(?iPEX~KmJN6Eda(s3IwS_5Qhi_WEKBN|YHG7RR; z?<}(@0F_v5pD2(lhJXF49~;#-G&I!5oT+{2>F~CQCMl8TI|z`I zaH-~zD1asDsy5O&JWmx`Us|&VRRH8ngmG^fr%4Pb%2tj8n$>hosA?Put<@zIf@GhB z9&y(Fc4z0;eSoub8r7!@4QFv(mmSvArO>Wzy-W0hUez3bv6&$ao7S8yL;TEnJ4n(4Y>Cc3$jGbig+K4{T4HJLe=z$e_Zs9*`^&Uc+hkb?U$;^kBA(hnGCwj=rD6SJaAZ%BXo0X)vEOoNBh*@ro02Jq zioX75=LKUxCLc6THDBs<<8N*2zzn<~+6n)S_$hKcFad9Wg`b}+bW0*QLNFNU7?C5z z^==CUn2O~BZGjyyvZD;roSfMzv>p!^33;vCv5yGYDaqe(gEk*tMY>3q?5c&_=<$2qJ&24=A`)q~9eY&V`;*N>w#CYx(-VXhhQulkrhS2X83wiJMhbX z^=G5yTo~Nk>gZqei=)RIK|ZJM03gJZN?%jg=aFGEWY<5fa?hPQ_-?lBmnuDadDgVs z?Cq#=GQYlQy*FIVu2V1TZ5PTXk^e`5X0QkM09-K{0IAZI zFR#^F?Kj!Z7^4+GT!$O1QT+E5Ne1X(kaX(IhRzMW_rLAFpU+zQ|H$5QoO;|AdMwhz zQ6QblPj)nYzZqSpp+*HFv=wfFO1Ke^NSs#=ZVol13=MCeci-EOO5Wi-YOu{S#SaEMKk_bN zMa7t%_!jK{Fu$(X>e_=aE^uG&@3q(EcUv!B@hM_LAqvNcwI*>%uU%3j#|908xSZsi zR+(ZO^Mu=M;kT#Sy6?5dG@XzK1?@p%FDV+$72!yAFOjYaZ9c8mqDdiU_GXb}D(?@B z=U;xuz;znN@(ODkg@}C7?sNKF9{Zi8*4YvNr%{JI?>Q~?%ck|`C&vWh4Qt8A01>$~ zrq|nZqxV~-JkKwlEa+ONaH~$MF=DWqs#4-s-?Du(Tz($Kp1$`qb?qS_U8N!NxV84A zNCd+mxk?Aen5W~Z*-V%CHZw8)jktW$d1AN0>R=X!R|!w{b~64Z9}c?bL}S!%c6@tz z>@Vcy4@gkeO&I*E3`+PxWCmCC=wKhh+f%5t?lwNxD^x95=fAvXCxQF^%7W!riIky{-Pa`Or96?X)(nSix8SbWq6!OkO=rwsN>q06Ltg)?VDnefOs|5 zi;w_Vsx6s8v#nzc(!IBTTR{ zt;FOXS~Y_u?^fPSX^OCD!up#zy)-5RI^MpWuKU2SExH=7)zH#8cO88}uZ4+-?J5hN z_%~+X&4`5Fu&QMz;RM(1+ErLZIOh%4$H{o6$I$5JBUqt_weiApVzP19#`RRWezbVrc{#3sJO`o_7_~XDaqwj68Uk9Rz)zIiXihUkj;!0H`klK)&IA>A1yuzoF z$-71b3yZ}g%`etqiJ9m~9)=^ifdorWNcdj&j2>%F1C@f}mkwwi2YS)hIhhDh~z>|0ihdk&cme7J1&2tt8;=B{L?W4dvZX{~Hx-(8VkF?9 zB{&=%jxU=*gR#2m>-Aw?m)GpW&-1t0`KT91*WoW(YycV_d1E+0Bov?^g_Nop5|K(F zhzwUW0g5yM)m9e{DNKeXgbILK0iC#Nn{q{lS1F{Xs~co2Go|tVu-)bF&>Q>c@%iCV z=%H7Y=+fmkp^(UjC7!4ZSwKLs30XJt0K9E%&ZgP@@pgzsb6rTmhj4ZmSxjK4n9P){ z4a5~C6f5cGM|TC3wjESgsp%h6h2$#wsq(Gw`_%qjHm`Tkd44-l{ukd=XVHb(yctLj zMP2U#up5H3~xRE}jsKA~~fQ9CcDGM1*43#C_LZPq>Rj;ljfNWnpGMEbS3~fgs_MgECZC?or&&&d#u^i&(y$8 z8dp9)^Iub>m7H&<`&}#To-R%j5>lVBB3z;5=u;W8af*|<^U=isqPsE4+PoD28YM-P zpsdwuIS5n=)zOpR5TvfM0AmxLNC1t)Cj-cQ=){R5OQGk*y|JvQk>@_X#M9G=!o({| zg#Cw&Ai_TR-`c^z8uzKQz#uYx2n$M`UMhX3c)Odh3`_4O-~oegkSwF!1N zc@$Mt&}`k=|M4!vY6sf&Y*bN$1;N`Mh8_tQ#U0*?%IV7Wy64j@Wnb_fDE)}E9GDLr zwY@U)$Kh!^MpyHi|FgO`P7VVHrAjwjZPYb9Jj`B+<{sIvnCmxgd6vg^CR<>VsPXG{ zW=yedE|2#}@khCG=JW4mi^-)^E#CaXI)7g|sLrk0r0S06wdYbpcG>U9COFA4XX@R^ z+2PU67V@oEtx>%16xg=@frGob95$=3$vPlI#5u2ja2tmxuCA#GzFgg%xPY;3IA2ra znZ-ZYo;uKBI+WE)fD3>(W>|FNDHx21iYFha#z^KeZeck+$&oIGVC+uh)GiL0xhNZi zwIV6E^#m}uITPv9vO21&TRgn| zVwx0nf0YyCH*XZtstFW+k1htsOJ)cM$jH}L1_zCul?w!yPxRr$&(<1@V_a}Nm@p4D z(Go5-s14G<6!v|6Zwd4tZZ=8qv zaXH4K8{%LbB{uq0?La%MW4?WtqC&}`w)x9k?JFiab3;=u z@x|K`KU%RPa|RhcLEU3dx{NexFdnKr72ntS)vv$(yiX#oKsc^3>Pruh zz1WPvny*`q>9K9reCn|FJ+AZpa*JS*jd`1{N2AX?C`(zU`%?`bP zmRtNK3=+JNlrFl7k|N>1WaF=zYYUoX=d=kn&)C&yGw`yo!-M)GY82uAefRgjEpQWs zee(uKT;N0@{6FsYc2AlKi7=*)ppF5w0it>i|1ja-)qEbbzYUtS zd`6?fOwjlzp??>2c6y+R5|_u}$s2wo(?&QGqr-?Qv9y&m`+tt7FE2Puj&?~Y69}yJ z`EJ%omNoM?X#U^%a4^c0N-|oMNUFqVayZ3Ngs^sgP`|ft>}6iuOe-N^1GH}{K?rhQ z^SNi2_}GgJN~wk>jb#7t{^(Cs=(o4mjj%(d)i#Q`-_;#mEY8ZKzje|Zt+JZ@fdkBP zHkK-q{fX}pBOFkX{noo_CH3;+0T#u-+x=ax$R=sgS=mcEnxFtZUW;|+0bidE%VLD5 zp8>7Di3VOe3*SZk?{C<}p{xM?UeZambFA-?hG+NmMvsxah8D5Y$5T|-yu214 z&R7Jnk;4p97?=?>)YZlLk3jW`|IdlRQb37okWQY|bRKmHA6z)b4J^G!Ht}_nIrjYm z7)fP3rbCB^2_O-?-md4GvQjOLer#%Q_W@TKvbgMy)Yx0e!#V0VT>0p_*5SX2iU#4T zlD4h=g^2$nvdc!nh;B~U=v|LsL55A7wzJyXRE z)A2e|odrM36;_~LgXM*RP})E_siwqqY%Wl$oy%r!8`Y{yc02VoGCUm%jaA*J&xq1(~H@f8gH!%M{vXh&&=dNG;B18?$nN;1LUob4AcKX=Kqhh70-b;*Hise*%HWZUg&8u4gg`LkeShS zmr`whd{`o*;_>VY)`)32*Ho4TNn*ARYTN)SnEXnTjK4VfKW$EpBSd!ILBG72&I$Y) zt)Wt86(l<8>qTL?47n3dyVlBnC%bwiMJ0d-z^2Mo-AwGTu3pqya*xP|R}yGJ?2hw$ zNy>$;s0c~6WTe#oK#q%uj5~o+90C0Stizr$wOrzJI@(Et5Smkd8fmUg`I8}V{QWkUmpK(EW1Ynd?4 zAl+A;l>^_Bra9Hs0Y<)i{Et{iYggaXMBmI$a{hU5%CWGQ&NIK4Ict|3cD6$*x~*?@ z`hwL?rqxeJE2rzw``o4bMS?)C)7)RO0}Ye`3KvP&nUF8vqo4Q&YJGXv*5dq_fmhX| zh6V=vE+aX54o0M2$M)_+ScMg`x1;RiEMTiAM{|V8^`VmUU4pG-MpfYr2LafT@yTY?Sc{m_W{RB0sCs!E9J<1@?r;y2=XwO_yTNRN3 z;?+p%OWgHgh~~VR4zD8s(M8U8!}9@V=y_7qJ#it)N*7iKg)+pJsYEi`y9I zE>B`&-xeh`zJJ+!$$sD_l;^>GD+sbC<6B;|tEx3@sJZuDXf$&;Ntv?i$lpKsgwgyY zM)|CAr18zm;q&fmKBtDxx52B)O}2#3905sXil`KTujt(PQZ?C{rHrs0<^d;2$PflJ3<$9Dc46_%=^_uMT%+!3QxY53joedBXMe24b%dq8 zuU*LD;he&I&L-rCEmR130~IcDWs*vE`(_~`wt^wfd^N`YURoCRP{Ysm!^`pE%|Lw? zKmf}#R>+yT?76Sq-9jUUTcf&(=-tz0a#xL}MA}w{WvzDWz_YCYa8tDUBKzl+{7x(O zC%B82&+cul+IJWGEj_3Ks1t1W9Uv-YQsKi3Gk`dX5K+ar5z!@ev=9AwUUzI6pFjVw~I3k&L zl4i@da5wjPUIc@|$Hem^iOFY*$&6njHQ4%J#f!(JSpVn%&+I?1WKr$fInNbTXd%)q z1O$x;+-rEmd>(glTT}Et_&BV=*NUwuXRxZS?8q!WL+h7wdgG>*zmGJoYv<6Z)(Wof z`f%@7@rISxwz28%8@()g&mD;P))5~7*;}_yTFy#wZqJ+7$OM=KK~D+sBwk&g;ca3uPW>WZibQoFwZ}ETI75dD#{hO0|;?(O@?FG%a8{4wfF)PUl7R>W>6_)`{TO$&n1!1 z*0+CqxJOB&WwX55bj6;nw=I7g2l8hgLSzd0zg1*RzpgkPnVq^YR}*sESB%@87rf~Q zGpluvEfJ4pu+t|xk{^zK6MZ~n;}D^BJDU0N>lYV$z)*Ql^LsERpt4?!jhvdtf6eyD3S80d z`oCF#l{)jNs}xWkx6+@iUv^9JS4%wJ?W0~c<^}Q^t4KKBT%WnO1qQo=JnMfHIW4D- z!NbAG&?MycezJ=m)sc&(J}fx9A}3lfak1Gubg2F$Z_<3yCBD;Yub`kH zr}t<1vqfpNOxmRl$5J+AFd<-#eRg$i{Aat_{lbVC(?M8I3v$-lQ`_;TyOGA&$(yYRLuR z`Ss0s0)>=VsZ{kxkd_t-PoH_gT5`RHA-l&YwXL7%Ge z398r61JvHOyw5thXPvKNQs^-#sO7V_xJU$l`IiMjb)$-l?gQ5OhWXY+tU)|>cl`>BxL(?{>=Q#T^A(Yt`T%R{7bY2R zuObfft~CwCW&Om?9)tifbdAI z0DXZQ69r;=OUOHf2vA8Zq1JepAWEzFZP|CM;4mnS<&4%$%ol^cfiqI_ZQI$(ykxZJ z)*)rh3C(h~I?Tg(!_`L{T>8&No}S#}iy;&*lqf=Gk;q_O&xijSCb-m7q`1?&* zvwK|wLygfMMg=^OE91xC_JnkcoXCG@gWIJ-Prb~iYmKiRjRbD}G>P3ojYEjm#M+3W zRG|%8*h9n1cn)mWpMv=rz1iWZQj8A}$O70z2$Wb$!W-wTkfdCjH30ZiU*kpS(E~iC zEDeb=;YnFAY6C1%?fbW*aiOp?ZYVR^ipXK4+lJKTnmJ2KuD)>;8>0dtLw*>cYO7Mc zTV~PNj#f5d8`RNWur{+`2heMFPg6uq2{B|}1pPt-|**=&^Or6LD z^<=WM*4O)r>Y8&7oOrR%U`ga8aNhHrqoOUWf>hY z7#*Th15&5FdK%E?pk-%(&ALuMt!=Q`Zs~6|)LLWzE>K8KKn-hku6g1241kh`B+x-s zwfxg&n+L3!vXV6lV;US_SuPpJDJdJK3`~!n=qIHL-F|TBdc6}1L!Oy+Gi1*UfoZr< z$m8{XX$ER`zBxBE(pG&_{n^-vwW(=RlFpF;CFd@Ky6~{~vx8&8l#p<(hv_>(6b?85F1h*Pv?4_y|XWLwsdgrwRZ9!Eo1gGSxpBosfIOvLwi z_&)stk5%<)lb3AF%R}`^2R4NEI);-akTxfBlIlOih%f_&ba5j>p)nvB;xX|f343B; zSm7lgcldMz)u7-WR51Y!c^FVgcb0iNhfi+%aQ_DmzQzp6 z84vP=1|1(g0F!}-b^$tsf=W7#FccF|MpDt=#a*;o!=+~O5fQG&aS?(SfB_5&#STa! zMy#=b3oVjH?~TK3IUD9)SElHZ5Rcu5!Rg12NRQB|ac+>?>p)@8+^niP{O7)z$Yyw* zd6!Z5AQw>GU|cTG22-?WT8e`q(M`Fs>a zlIl*e1trMv9yd?uj35ItBZdY>A*;+N$$II7HAIcyAHI z$T*<}QxFfD1C)|bPH3D}MXiK|<=nC0kw{2=vWd0WLxM=8m!qX=*hS!k6G#|1dFA7? z6aT1|b?^MH9z0Qec3t8M)1d>y8ayL!iZ>=s^5}{6+Yw|bw{9;EP36O=@BQ<(LizgF zl9Jxa?-e&PinN+jDtI__BxU**;Q{172Fd0G72%tWQ4y#D%njdZoH=~tMeFDbp!Iu{ zcW@X0l``o-25&`Zv4juc9%MLRwj_;1%!^#x=ABIt8YTM3A;zF~x@p=mzqF@7I}=P&MmVf^e1# z;RypMBs?{iI1SFU`4Wrt%pKal(Ig;6gP_^1;pYB`(x5{%xx}I3Nrl{){%mikXm9_V z63riLWCpTi*&ty1ntyDQw@!i_2ChA6??7SF4^m=Bu*upeVanIsqZM-J`Qhzq-!JX{ zxpFF1`iSPOTF4ei#+kiWI`{VaY}E91@|a#L%Ju7~oLYpvp(N_{uiG)ZeQ)C4$DIcJ znw4HkEvQpE?O!h&lc>~_lX-&_HS$(GjL$hSsE~>H#Hlfso9uMj+}(!VG9Y)Z$MZBe zO_v(3l)u+@-gPt?3OUuy8R=hL^rD1~6MCMeq$TNh^F5t>6$KYB5CTT&BMCj%1@239 z6+0j2mI0(z4}%GD0&n6nyBl+kX4C!m_h+jE;-jQe@)>OAnzJ*Jb%kGxcgH;DPx$dH zejn6o#SyqrLW{-3j2+K+dT;%KPy6_AU%DP16dH&B*fNku^Mi0q)=>eT9d>L`{rR6M!U+uAbZp$966{v2>Fov6qTVQEaqYe$Wl15@WT|kTE+iDm zR3`8*47Ltg3HOdq30dwn@UmOpA*685%vb8Y-o1$p;9ZpxIHa>#OfaXtldq8Zyls&9 zUA_slc~l!T!OLqYweNwJ&$ZvG-v5?(>z94jXU{7nZbnAt zyE2na0sf|H!w#y{kmOE@C>(`$cOP?|42oH0t$Lf;pQchuZ0XRY}O=aZpv$v?679X<4VG7+cwP{MYk-x5ZnG)DRe!@P@17X@O4fA^{s^++(ge+!J`A( z@^rY*wuK=RIWMBAj%INC56{Sapc8de1Frd6_NG><4(qt=fik&%v{!tu-Mry<=rAzw zjcf#Y1u0~*J?;^tCZ}Qc@h^Ftu5?<=wmHquDjIJ!COK&Hcq^M*OL;FZaUOHoFIP@K z(TY4^GwHMV{yy--D#`p&RptJ=bRSD7mb=PUZrR_UcCh< zi?@SKeNNe^&Eob5>~pQ*Qo%z_FOGEuUVXr7x`ztnf@78xLe77xab(O%}?6^1c^QSe|?yiER&AKm8!p+U?V-EW? zaztGnThQI}tAzqxNqWQjB|O=SAH+djUESTCdlODvqa3(HbpWWbzxFN5L~FK0ov8Tt$3y8A%Af zv-^ub`S)1aS$leHdwZ<;uK(!2h&>G}8E92yQJmN8dfAe@{QB}UL2ISKN>WX3d+!uB z!fGeH$F35oy6Z}o6t(&6=@`ezf3=?z$W>t|U^Y7Ivd?T?QMu-^p)8s@me=JtIWt+V z)uFMR3^s}WRgcw9pU113@^2T#79%U4T3saakY)D+KQvm9DQ+J-t{f#EyRc+-1h8I7hRdrL{P5)y%-r7Rd|u6n@byDeQtRB}7N5``5q zmlZ@4GM(w5mikf+g{5C#rVm^D{d%kLSzL52VdA5Y^ZxbwM^X{qelnpR*L1_JH&+lW zlW!m7i1)B&T)E2>{J@moy-*Mm2`)-^aKcf)i3J`J4?wgB~k%!6;Z)AI3JLGzw zKDm`EWyi{=vuswi3Fdgc^$AF2{Zz`i-cU?iMMw9X5+*B&NVE~4${>kEw6x+% zg&u(`hXfH#$jY;#d;e6P1sT}AMNnnrvx<~8-`?bRnn|W*!0zyNt5BwN#_vc0&r)or zRfhnFttu?1m@o)$4OQuGzx>O0dWiyyvw>dg!Oeo6@xN)hz9sxM@9n-~H9_;?3OkN8a!Lwd=L7>sMm~g$&zc!v!tan4JeN zHfvL-&)Zu;w@B$==^B93V9Tx-3G#*Wy(|a8X7k%&-?p5>MCq{dbZFAW)kgE_Jq7-5 za(s9ok6UAzq37XlY%DT+mA$?FO2alOhGZ-*Av|}re(i6k?8AD?Rrh^Mzw7I~u-;8x zmpumU2IG;?+cHH$W;!~)Ra}|jEz0YC=1jh)4h@HS4rT&H`3ytmz}-3Z~XEc=$n zi@jO;`_t5(EvwHZjgrix!cUU1NGg11P&~|hvU7{d4NXmDsy~4JG1Y=zq$%=fym(23 zSGjHd?%cFdIG`Mt>Gq-H`3gZNs?T4x9@@@2ukX9O-@0r0$UScl(1;RJEsczNMmHVB z6WINGvJcrytGvB6)YQ~wS6UY9_Y#dD0t5|Qj#AQ$BHjLk#o~MS7J4R>PU|(DGn@)- zmFn65#~*@c_vawWgxKOad z4ow0%Fg!d0qQhgnN&&d07%nUaTe46+mJ$KAQ96!7L;K-+f82=0?I!bnt;$*+l*@j7 z)@J`adu22i7Bij?*kCK1)b{*7ldjCk%{^e^gsqRxa5P_?$C?l&Sw!a@>AjN}A)c+F z5wAke{+?(&_eZ&qz0+0^N}pWjkA_APKOf&65zlXY@qXVTMdxX1o-Q+CcOT!}=n2z4BSHG*d6=Fa1F-jNQWTP~~}IB#u8l}4 z%c;;~)S*B3koEFXVJA~o=W|=SA<|o4Znsx}P0`Bxd1&cunM1SvWG;&Pw`CL9@sR3V zmM?#53c`>Hqs4HWxY<9W7|&diW0MuiK3QOMM6^jigbyW7X9dQ#x$UOX6ll0xKPr22MX8;p&daI)ME z+^aj}o7=U&{=FXs*Fx!5T;jA#K!4V5{YLV3nde(`-#ur?pNW6W3B+Lt(B_!ZUY`(= zAS(Q0#N}u+8*Pv^Z|~`^AfK%9pIp$uVai>PX*GJVc4GqvA-|uvb=W2|6S9UvPP^0U z(Q2E!b?IKy`r{N1F$t&10hl$1;(ipFnYy}JM|kQ6>um0$-#4`o*IF1gFyTIu@q5Eh zUDM{=A&eNZ4=Ab&Jxvg??~MvLZ)ph2&5e!S4#bi;r9eSPMu||$+3gskn$l@}60*wLZtNo}IC72+M z{U?~=2%{uNIg_k4_6LRV;=ls<^PPrCHhFsw6=LyRDs+ zxn8l`@BMQ;UY~`Gxn8QPRQRq{d}{<4pTGd-4EbM89yO#a68t75LsJ3k5!4uVb}DC! zRg}FYXrC6EJFUFl9{2e$5KDS__g}z>>w$shj3QQl^JwKYyR$tIcfRbuvhO1JJh(bG z7E|5vlsflVAix3jEC<#-K?n60rdvW@$BlRxV>U`p`LI^5TzU0M`C?xTETdg~a?i@7 zkNq9oC8JCnGL$b`1_GTO7S+A*^Vfknrn_c#;S2@`SAUekzT73il*cUJA>#^ers_F( z9o~1L>EHmbVu*crU!YyW5DCTUGVy8FjI+w&+Wh9{ zLo(DabHIX)Fafj#DY4|l%U)B2dTTb+mS9+9@*KL<2(d{x4lz+-ggMBiZy0t;RQiYXbhOnp^eEx2N4=w?jvU<~$~5Y# zs_wmGO-u%@91;*g$f{D-G7_&ny#vT3W4{XaW~`C;!ZP}Nn6Z`E8kty}t2;)WpOj1} z-F)72a|V$8Slmw*erVL^K5VU^aCJQ2{p75T-iBlcLI8Gm%_aH*kB)3isA%RDh$}1e z*qmflSkj|C!a>$r1q)ibtoaSiJE>Xr?YI5QxS~mS2~3QPoSO~lA?Rotk@IWMZYIUT&YP*HvLR-Gu>N8bAw)330-Y-(+QF?VBIxO5_~mRnVky$W(N7RSaVjjW2eF zl8SqR5)`XV1$SaP$?8=4ruLE~Yrz-hGMx~AZQ&zN+yv@K4DmE7wM9M7*Bs=A8t zdvZbHS9M*rgX+UM)B@f?b!E+MD=ulHCcA6iXL6QL!?#5yN}uxl-yU5Rf!9qDhYG)? z>nAc7KF7GaxVY%)WwB-SvN=7}-&)0_reZ5feL~rp@3rTP#FE+Wq?w|wrsn48Gb2B( zcNckU|Ng*WM%C{(mq^2GBOuJAZG|o$M79{>FBcY7ul*#fi?Vl96R3j*>^UjD(K~N{ z?h8B5<8-9H68wD<9*HFQ3=XU3>$n8Jxa#_45z`hAFGI_P5T)|Dlw zq}YfhGs7O5xqMzM-vI3V${5v7%<2XW4IKxNYx}HpFjJ$-|{!`d?m-x9q6xXo(lAY8koh z#-iMz(-+^Sy?9MIrSVwWXXPiWKF8){a(XCsU2PQvKUe#hs_9twg_l=Fa^M`A)fH4} z#t(89T0o#Cl&6Qop|hAur;_sv^ozQdIayg@P|@aO{%i|SP99<#sTi{~(fv$rl6dIV zVE70c2A4I8sg(=1C%H<0$LfyCS3rx!&*^5dRyuDpcRc)#-PB^p7I<}u6R36C-fLQ0 zWLQ`bHb<5keER^i;r6mJEylXQ)X|NoRK?2^`naR(B_N2=BR7cV73W-fi^&y zE``_tL5N*0wrL>IUbXY#_jwaam&s3VXN7BlSiJlO}(G3ibJbR3-a?1Y}irv_HKjkOA!!B zDX0_fhV4GTPs7Fd#z3Dc-A;YaMz20*(NMvG_N~Su} zzAK_PAl`doQJFA+msW+;6{amQ^xEa5qJawEa1^a^-#nL(NRzGj`dE@53yCb|Xeh7| zo$$>#k!r!;kmBXWGH(D$q3u%3(D%om7_(2O7|Ffzju|1EztdsinlsA6*P(fc`V27i ze>6teot@&ymMM(hHZfPCR>rEz zD@vGmnMm2`G#$ySwI$b83KU3NoDbChxik4FRTLp^A}n!O)z9w4WI_g=h*KEkYuS|X~_zB<7 zPXCU^t+C>xSe+jRZ1xzLa=GpFl_dxWbWsQez=?gaS;N-D<#vQN zWFbwxhf@FLSy^ZDNUe57oxjj^C8M(tY>X2cJAFw|j|n@1bOkic_-jN)MlqF?tmRBR z#A)Sigm_S~F&sKW!cjPv8^uX0!?KiwNm!OX~Z@H z&R7Po@6*$58%9J(`GS^uqCLD>SL87Hz-pbnh5=Y|Z0K?p1GBlvGI~$^Cz4CL%SpJO zZB9BraZW@=h>%ZNNX{4J#FQnekBTNLvO@KYmmX~RzkfA zsxwo_g!aJh7-ellHi&`av`os{+M1ofy_5QOP6q?~X0hwKlIJo< zj?N2QIU9M7M5Pr0sN*v1P` zgz@??0f@Y4T}FsAG0Ea8>^9NGI!$Yt8q6bGZ(RV!yk6*sl%T25wdIu(OJ z;PFr?E<3HV0-hy55mk;w?C5|pL)<{3PP)A-FMi@5@J^UI$-p++(T`AiFkVN(wwvqt zcg-mzB!cVSS1v;ojmcxq`?IFhxdBGrVq*P0L1t1?hf`TX{s+~)29t4d5T@Vpc|zN= zU8;-!^63j9fz;fJD=zrF{iOD($gnh&fO7PZlf$}}U_{uSVjj=aKx)d#`Ib*%J9gYxM!{xn z|D4Eay8L|<$;-!wO2pq7AXa2iQu0)d7*bhTDJB)Ss+^+#C!}u>7EKLHk%Wy#8LqXq z)?wRT4#ez^aTJTs*0@rgG9$Y@yW$BZdN8m$AJM@dHLCB{72jy({rRi1vfVLXC{S51 z859hSCW$aIGjbU$HcY_ffH7Xfc$he9wk7(Dkrr2?6~E_+&c z$nwN+Jh&?yH8_HqoNA4N~$g=w%)Y$&6ws92X<7Ty^vmw-E z)ZF=&*1iytlan)^wmCoN!bERDGp>Nk1gf%SPvTkI3|!jqoXYb*fk?FsXgxH`lYG*> z^C?USq3O z?`6GI^2uZ(U65YQBRQKK-R=CT=4ooF1rY{;I`Bp506Bc~9Zy|r1O%lh1_V@v637R7=N-#@{5G^kVZ}U% z1e}iEM)NU)zMlS{9V6q7;<7J=o_f419Ur&V0`I}^S6gu4=f(-xB{GdLQaO=X>%56m z?;AMZwJmQ`Pc4quUU(8ex046~KF_n|HqY(8CrE#a?gi&?aUl$vc;;D+z^tZo8K}ApcH3VLu66>_s4X*wog4d#y3N8_* z^gS|Ji{*3hz0NpzX_2fdYvvS;EV1+qxZBg_||%Deox1e==gs@eib0<$YNl61ANT3DVzW? zr0L7keE;in77ZGR8zJhmJkln4LwROlS%Ioi{Z8!~7{tqbcYDZPU=)G(QFl6L(oVna zvDP=sljzxQ`kZI0JMd}s&Jv<3Ftpf#|27GLXg*u(fvo3~Gi#^fmJED%7cw*?LZFr> zuKFX-bDH+O=4nHnyz;?2;mBW_ftG6^mr`|w{DhW|%%;xysbF~;+EPsYg zjSVWuR?`7F_)jiq3v_m_wRF0lKFX9NV}oojf=S-~_aS z1-tsQ*Wu>zbA^!4a|R1rhv#$|pJ$HO>)DU+Q_m@ly$#EELkvCsA#Y-RF;sAK83;z%7n}Cy&u;ve@B*BG`XeRJa|9{yQmqci##En)&F6lgVRtyry#IDn zKX3U~csFxsF{{)}D&pbvv2}MYT*)IqLyc>9v@8#CJ`^mtvADV_Yqqmg5I^guj#YI) zQ*UF`KYf4tHu}&~ZL>aeeGpl4QVk3YVtx!9mS49kh zTzV?~Xy(k2w`f|qb`|J)T`8?W^)InKS;b>JQ~7gb8Tg#F^lNhM`q4?5$7283>9%j! z@pWhPqWke$Zd&kWfu6a`iZ-D<=j|fk>D$5U#l_2Gs7j`6^q%3{&hX@<TI-|A7=1_NPjAN?n6S?ffXPG+$ ztRYolgrr0=(r?9y#t0(#WEJIfG{&^(FfZ$DLRJot=0+EXAv;8H1{Bx-pvOat{+Kq` z!(49n=OH`f9(_;7v%i1wM$IcFR|#Yl6zuSmUnK+=`a}@K=wyi8zX(GGGsKu;<02Ny1H#A>NyHk- zm61v1Xi05EDp+BVd(etf(HmpXBBg1`X3FzYp@mWGROdvq3uFN@M9o;mN%IC?=a$Ax zqYqntlwyeC!F0U{*!SdayPcnbNa1sKg zF}+t#z6V2H8Sr#-_?(Fv@OZw~uXm=%3rMvSPUTRi9H*623DakvRz=@I0V2}`GkkHg zMS|Y$A<8I@W35;e-_Fu%RRq9fl0}g+CqlzIRESx$cn$u|;WA1YlH?Puwj**xDX`1_R%`HczLFLi+eBA_M`8yHAPlpdEF>%2?eskPjvm^T_WM!;;g)@GktNml1nNi(!e6kE=3fT-V!a~upZ=PN~A&8wq@39 zRqiOKA$WN!d%p|>7?s<`u6NPD9=;zs7R}dGCLhP_wr1zhD!cA^qJV1Xqw>zQ-)u*xr z{m)Nz9ddNs?L2*>`u3srruop?P+Ys*uzO-iV+&KrlXm(t3Ba4Q&{gqsn(OX6Oc8w# z$;i_+UpMdKxtukDWJHq|X2O7CwiP`l_?{ziNGam~A#@N`!HC>rzA_7nG`ztWW92Vg z8K5Voc#oX^x-WQyUSuLWkFi~w1b~Jw{&s0h)0+*7t2VCCDj2LkDhWCuPx;RRW#*Hr8UNpEUQ z?q)EUcCEcbx^$H_=8Kd;4XQ7_u=Irl<*Uc;wF+l+i*M+&1b)QreBXVbD*2nYLF4tEmB&sF1vxq_i|_eM_j6UZ4d2!?IMAOK!>#aO z=+S1F)fl;`D3P;-#%S;@BQj+f1_EB*t{~i#8mrI!M1>3yDFl#1_z134!c&~pt$+T6 zrU(gAEJZ~=&|xB9OC%zpOGDz;LzLmzZlAe+FlHFwoFb3eL}M+vd7JH)TbRSmTzxf@ zLGIalyW5gypO5Aw%5cOI0cuRljZ?q}@t!F%DO*4p$&BETd^r zp@ZM7oBw?E zD;&FR=xG^y}0!FSYg3C|m^|sA(aal_3d5ZPUb}sm*KwSi(_5_oDhw7vzW`?V&IRyze5|w` zQ@bc8vO002@|E(-ox7nMe1a^qY@g|p4Lf!GpBr9j&DU+rp@CUV@7GG*Uk*)ZkT|o` zN~j)_|D7ATgp%x-WV6vWH346AbBwN20}(VR2n+{HTd&}K&&95OBzuj`ENfCNwKS$l ze-p?UXrtRrzJ=Whk0Hk@0DH0e*T><<%eP7+wq~# zi>*Z;&QM(cB5hDc!PeVGASDgRiC?yzyE1XwU=675G0>HIBwkGv^K100ugm!b309vO z7Wuk-LexMx@vEDqmh^#311ClBvietlIDj-IAM49g*Xs{^16-ucDSdeg3=SanXGEUa z0aXCGY+2Lr50XH){+J1)hl=^h0s=9nAt>2wBK#)^h<@_m#fZ*tZ;Zi)+x|QACoMA0 zL9hEa&vw&SAP^Y&gSe?6E@hF!We~R<12YU@UNlg9+X@`>{SNACBmGKq^$&5T(_C5R zXf-?M>yYJo;#=2|Hyt=Xp`ENmE)>OK#<~rTmnzGstYWiru z9Bxnu#26x}m;m?r2dC(E&FHP=3%o6+Sn*&a3PR7Hx@Ng_A9TE5x(;0y_enQ1rg zfD9WwKunD`=TDHfd>1urn}Yibe3~Y40>(ls0;!_BN!S4`Yjjzz=@K);iDou~pPU6w zTpx!T3vL%qpU^i5oLs2=S^bKu^^RToU?X( zjOE)dCC|vB(39Ua+6~6YeG|ftmYSNL)~y6dHl%GyQm8`p$M>a4WuVA;GU|LzmrwA7 zS<%fmfW8SMrQ!W4rV!xFl)kUj{7MLBfZBYt>98U%|&-4fL&^%M0{Se}gfC9xc;rSEVE;vno)RS||rdY`3=R zHOi@8n{4N4KHC|`O9>nM!?Je?_O{kj71G$hkG&Sq$lJR3#|7AyyS+Ykfy7(~cmJfl zHX7cK2dprI_Z>a75dhJZ;Wy>2+f9Z*#@HMrHNgify1YMZD>6n;6FXzB-|3}7_>fRX zhxES#l`|EwS4LDB&i=+UQ13dRo6{0i%05-s%KXCgM>FLM+~s?qqb&GJ7H`D~1F=FGHVBA0lzwRVD5KfbLqF1Vdma+mGMWa#0d%$ovMZ@&^-ac{3Ld(DkPj3eZzSegSbWNN@UHe^Mlhy zPDgEFSK4M>mw?g@q(Fj9dp-Da?pki=YdgPImm*&me@s=XZf!skT6FD9lCyZBGpDNp zc>-A3uiQhyWb7k$G}XN5Q)X-GYkPp6br^|rj#)Kg7X0O6UB5GSl5=;*KKbl`{u5H> zX=^9J51G%~CzBKxrdwVVv)n(1wuw6qF~_2?3dv1^Z%w4i!wyz*Al@pG`JuIp(TaTPCTSCCRToS35klG#BWxq1H{8*Qi=!c z!Y=u9SGk5*kz(=umDK*Gg6b+MrmDg^B_VVHBu5xLS^*z@ulK|$EvUUeq%5C3($W9$-Obh0ve>@F z;kp9OPnvihK905*bff*ZU{ICr9p?6=FVi1pt@FyFh+k{tYCi5QCB*Nb!q%OB-Sc5d zI7A=Y{Jz6F2CuUZ#sJEHqg)x3Ovj=0fHGZsiV2aZMnnENutdB1p~Z6de71G+yM0?V z_5RtR=4aoZi_H*}6sT(mM2{9%$?NAb^gn`x$06!B zmkY^mk0DM4Sy*C|iRn89H?D(_+^rwS5F$hNI;wDV3-4?TS(ERMCG38K+-@J{rW3WN zDdRik7jkrbHM&}Un-EtZ6N z_xIbyE7bpb0kXM8Pi4P~U*RBSfnyS4R2~?}-|wBTiK+kgYqD6o(z=8X;1FP@xb4#v zXw}+j-)b#JUc3y zJ{7?1J;?J{=@KQzL{UaZm79NVNj-%M!JN(+^!=g!C(NTbQI^rgb;PkZVNtYB+-jb9 zaZ2FZ{f4Dq^U^TdKab+@N3`>hZve_a5Sn<}eV@t~<2&P^xP<8;bgj z98`E&@@P4%E|aU({M1s?!tTFcw8*6;;A zc5e4V-LNRkWY=DFpaikh?bc^Tjy6Al1wV2CCSV9l>o^uQV+cm%OaaQ=q`4P@)d&M{ zLW5p++-# zrWzc=lfPJX&gXf)euZfnO6s-!R0i?ywAlMfF@(jK#2|&?S7hP_+}qWH?rOLUh$N5@ zqNK57NVld|N=wx=ZajpHfZun*b{1zABp#yTo$W1{TRL73Y>T7^VM?uxukTJ_?Rc84 zu88{2hM_05SXA5$WEZkr0!t7H7|c#Vstzhj$uPw%@}l)&@uV>TK-E(8n=}G@^^`?f zSetB?oDRN}iW~Ew>9?0_Pd6TiS&hfTmSV?$P=I;2UuRlMJO{rGhFJ8TGHU`Ql`wD{ z&id-^->H5j-(yzUxm7Jq?sK1!NAsNx1AvLP0_eP*lY66~L7$6_-rnB$L=}e(idwwsmZaj8N*3Lu z@pYh~=UOr?8=b2AsQPUVITXOW`}BX%&=X&Kby%?8{hZbFX-`k|TX3)XAPse6#P+|c zxaFL@MOWYlaq=OrMVjPzR0298nB?D|=xb4-8^mk0HCRL=SQb{-2_5Y)NsFcxGhe1R`!XqHn7*l9oAJ0+;iljoT z;+a7{3H@W(2LPg()}SgK3}Qkhfl0YJHYGP1Z7@oZdZo|!2VKybj?fwiWN};#(fIfy zPEYDO7>ueAab4Z~r`itTiqx?!()L(&52p-tw6&AQGu$%SiozeLNtOU@@o{jj+B+s)AC{m-i>L@SPuA&*z3jSL-_&I=UW@IJ3;*_z;FckVlCaUea=h^}N8tCP(wHp>n;BT6k3h={m7E_2<^@pL2YOw!k6zbTLdwP<=7-&6_`0h)AoKK;!TyZ(5Bs8elxWfkrcGw^{ryxtrAjG@0ym?+#iC~UDDgpJ zYHjZSt8DMC4+cYGUK4tOTu|avt073jla{)i8(iH5#$$g zW|c!@zUvY@o&EeNbp{B6o3NOV7y?#l`d#-*ZES4NFa4))Pj|n8f;TrATqjzkl4DDR zji9xP3MkopH#>Sa?>#q{i8K@f{}jM?x48G0<(;lJxL)*yBeytc1y=0#C6s#5FsF0* zdevWqdTyE|wOT%nQ3qbXWWB$fKostp-}>&2o)tBB-hX@7B7qeTincl8z_%x~MWO9e>f_B;(CYSzBJ)cDU)=y$tTP1PKxpM%>`8UUZuN z3Q3rrQq>CaaQlgB*?5DyvAOO|ky5S2e({?bm1I&I8HeWmH-cdd;z zj})C|)SVN))y9TCWb{M+xJa$(f3uu|um_u=mVCUcQjKj--y9sz`62Qi?j-2$&vw5t z=gsJp{x=+c@GEryzpF+6`38wWm!Eb|waKU!UVYm|?O@AfPTQ;D=9m^XxuNJi9}+r# z$}tP5+(Scw3Adc+YenhX+eTUzpQsA7Fh`{P05|7>y_ zY51}++E9)i=BqjGYL$KK$$A#f`Ww+*Z*Iz;zou?j{*MiyM63Te<|Mh>i%pz-?(w3v zuhDptkRN1FNk?lEepcck>t0E|^80tGsA6pcOx0c(Gdb&w@V}bH@2NJq^ZsC0Hp{7M zIm*neIF)K2&5h(rb-I;+f=M0;kcX&~bhNemd?h`7*TdibE#EzrAdv_e+fwyl#Qfz@ zTBa6x7tkOn4eBQpc#(K{Dle_-z2``&P<0Lfa2!p8Ocl0~$KKjrHxRrKV`c)_xz~5M zJ4g#j#sJ_j=5~Ac4`s|-#_v_iLp(Sv1sS(d0*gM*=ZRpq_Z96Xi$x6z9i`8R6awt> zj)5;LN_lU_?@vX;sHaT`VTqy)Mz)FL7i}lek(>}csA>{-bx~aeQC){<^k#z$tzyF5 z5(tF*(yz_giCv*I?Lo3@l>du-jTK8L^H%(GWn;)VQpqr0(SPKn+TA`@Fi zbbRh%>t^V5nB*0wJ>MExBv|cO9{bcNbw(O=$E}dEE!MLm5+UdFg;63BKT^({{^4H$ z(lBzr;-hucTrU?$Gcq~H?i$dIs-YvZvUzL<*P%v~e5d))Ek2tRvj?(>agW85fTzIQ z$On?no8yLlcv$RIdF!wC)1HXrCK95}b9f;LS61m@PB8`bOmWcP?nH09oZ*0!AQ=`C9x;7na6q)*|9D%0pZoZsJ>qKlL zWwJ3@*okU1C|wHMGQ}FRm6n_SxN~FEi5{n+6?UZU)2FF|?OVvrx7@+m(+_@$A9_%|Kkp*;R z|2nUQ|6XyH^zuk?#S!sO*(5HKI-<&+Ob&}qCe--vo;vJm-VS~)Q_4HX0_>UUb?N7d zOWs{>=8)ktE&4z=U{@xj$g-2AoQ_=Lf>HT@dF#_y@h%xBiYS<4({nT1(?mt(OaFgZ zThIB71GFIp)v)}5m4pN@zG!h}HDE7Vs*_vm`?#1CSFK^yN_|ON#(&N{0Sht4SRtf& zilSN;f#l^;plI~~ig6SM2!ST%dYcpmI@xR6{+niv&f*4N>*JI!#M}HjMNhJyT)>$` zoFnA;_A{eNlIcf2+Oq@zhT=n~zs5{!zy@xlqPA^SWhEHS1sYcN{$b(h08ON_{W>9% zSP*iAL@Ha^8hF0@aj+Y*fK1j3r0x7aI=EKI}PY2)ZwF430j^XNi`;hy1u#YcJ?gb+->ju-uh zRBW<30w4ZdM+~T!sKI|~)?CMALJEvQYV2w!4`CtyV_Lav&Cr-QS`-TPg0{Zl{~>YL zZk0`UkFg1fh12m*K?{qE`g*V^=5WxXIs87C!HX>%Is6*}*;CFTq{Rg#!aYM4L-taO zbY6$Gokd4Z8)N$?oOUQ|U#OKhiTFGwhCBuYn#iMOSx85fjiR+vfUX055&6mu1ERsI zaR4CQDjw}Oqr3JywG%=(pZ3j-`>}*lHnI}52(afxF8HxD<%IKB*AtYj+XU7`e3)4~ zv!Tyx4v{XW~ZnRUp3mh zS>T^Xv^=-$wKouLr(-eEYVIMn|9)L1fdtpX2^I=`2bVTcURmT`U=`9_-NZ0U3(|d5 zDK)(vGuz=CTN-UKAQ!+IKzK+40_1O%AI2_xx-i1~iG|+6i~GKYphuf@uo-ws3`YrS zuDd;fAl}tDmj-t#OtHP6PS**s|HHcwPqzS8f8G@k)@L-3qe~--8kv&?vyQ#77Mrc+ z7sJRwkp}B%P7@1DV?�nE=U3gcpJ+W>n&V3fioYYDDIk2Ew{N$aWBswJyOz^~A=< zBeNq5$1~*3(@AIh72(4+sw0wC6h$#&iYt>bVIe3h;G5K zt(ixOM5tp0qYan+Wg=#v5F3LEXJ`}66RTGZ*}m6QPfFE_(&A87af-tTDt5yXj-X3f zwz67)c2dO+icbvpVk3dSJTo?P$}cTFut|(ruw=V|hkKu_OhJx#2K-dCew+-R)-7AO zFQHo}*iX8&#f+v3#f^5rtMpkSYRTjGW_{ZSLZPGY(o48QLPl<9MB^=-qkmYLF(9~G zMm0}nn&dOh%VzV!!u$OV^G>(te-E#ABBzC!c7l{@2TlTGu#9A<6b30G5lIYK^UWp5 zX8U59=L0R2;sPN54_dVJEGagNy6|W7lwk3X=*VA1Hp;M|qF};a=#^Drc+i5^Gh@*b zj5+zdX^LTjgk7r}D2NkBdmhM!h9dw9t$hMp6mla0RpR>dCP*VjE#A_Gw}Y z7zy$9yr_=Gup%ffg#c9~%ribps8(Lbn0Txhifj;CPi3v+2+7xyuu4{j1y$$NDTkas z${p;&3sSv$qOuV>{o&<#mI5>_Ruw;~3L2g_-ni(}TgBqm>$9P8v89BsTfaxJy3_Ua zDCT}3*$xR7CO8L9)Y0Yb@V#m55qUU6 zmUao6B^kL`=qr<(H1MMq0;be9i!}ak9q`)LtD5nTU4xw|UO0sLFaF){Wn1XMNNj$9 zOhP=oAeVxcD~0`C!J&*M^n_L!Q?f-Rj+ibw89@v$>X8eoaAxW49ZLi;#4)r7Ml!%) z-Nd6M&zpTa1TMeHVv;-A+)o@;)OX*g`hIeQmFOwM+F(I?XiuMJpG@ZfZ<;@#C(z{Z zxz8OPgoB{|^wEvtZT@stpkiwe+#&^rN^tA?5-i$%ycBvMdNJ^rbH7PG&pXR7Jp)i56O9Wtl(Ao-aOa% z+mx536lU>}@8T&g>4DKguJ`>Z@O2njjSfB$k)thnUYI`oXR+QnES5n=y*Zs7$9Bz zu;J;bG`$ZAA3#N=;Ex1y2Gprh{oWrxKRRmjyS$tog~9X(p3wgb^!}+IZ$3J!i5C{W zB;*l_AL(eO1XE=@(zIdR0HU;k)OJzoiz(N}q)g|r3UN^>uTZj<9aAoRZ_{5}Euz(B zWa?O@2-HaE0?~GcE&Fhyq)oT+3n^z+c84re95w&ONKM?-L_+P_#A900-aTz&a(GVr z?uS0Wxgt=ut_yuu4N=!({DRC4Pyb~~;@g0g**x)(-Nr|kVh9gqVl{`MYa(j3zvgbUWcXBvSDW8Tv)-O1-Wt=?z^<7`iJ8AxcNz`)H zSIWNyjB%fd7L|2-c+iZPU(=bsHItXd#<5sw+~4xGI!QjRoW1?C9cl=%o9?<<4XRh` zb{j8twoF@?Uk!sENJ0^RGju!p?;clt{Fq2|cGhvZJFE-`UhlDAnL=-4 zYYSp-4!nKSwVXCYArrSaIRKMvtXuWgb>8*jDoir-KfLq|Hmz;?^;fo{3}^X!y^j zFEkGKH?{%2mlak11Z6xFY&b-TV={W-T4mvAg*Utx;LiUp$ac9|NscP19da91C*kvr~&1bGy6@I`F6Chx&OT6Xf&T1n|23_7dS=HXzK15xGCNZm( z=*dR6fKA}3yMI?X^m(qz;>NqHncEhhmZwa)!madHd;Y zl8|eo+wW?aq;u(al5UH`hMLgZwqpPo6c;NDIRjDXeWxi$yYiHHlvXnVF*|fwyk1EX ze4w&8Pp!1!{UAV!X7;|>X@95-n8;{>AUJpBh`$+XWy9+>J7p9IArS6yJ~ubleoY9@ zi1w>z@7eqL7Gvgb<)y8)CfmcfU8%SO_IjP(U~Q1iO8d~_T{B02sMv*r@8Lq51!=R{ z(e?XT-2P%kKB(!eA9YK}|Gpzcn`~frJ;9+zosuih@7b%adYsCC)}To98) zFBVMs6B69ql4S^ykl@l+zfspOw9lAT7{lvDMe{aT>+|zVTk|FPH~Ri^ILzt3C$Y>z zs44kj!JAXRU8(2GiT97i?{0xr8bq%z^Eo7)Lb28y&5JxBRT*an$Z`v-bCY-LIX(R2YtpOGzBprF(4sdh<4Y|LOLgQQK$u zA@Gu7pH(U@k3wyu)#0_DQ1fgJR8`X4+}POO)PxySd4GF*x^_cr*Xl7DL-O`_`2A8S zU}H(hh}Qed{no$Q7Gybm!VHS~^qCvd@vzxywy>4*3epJ&a?P?yU0X}BE}kHb^|EUj zNS^Qa^}v}lMbq%*gSxCb3Z-RpRiTlZ9#6Njl>#gvf&K$A+bgYOq&!P0>T%`0{EY9I zl#^cXB=y@*zicGICod9ML_CUMB+}|$ickim>1|pnos_!p!jqZD-+z`Q4zJh6V{kjz zf}Aa}C4r36+-+u8eeg?lnO}EPQ7DrbaKXCBd6d|y2p@#?{T0lCjcsk(izz3jRWSL# ze*J3q8SDS?dO0gaGHwmqpfO|Rt{^az<_ObAT0V=W4DpHkKXi!RGKOH^uPP!$&+47D zmA-<7nv%JH&jIy?Oc!JJ6%>Y>D@_(FfsywYo8T*7d|9MKWm9PKFadocYNcGX{n~={ z?x&l)sW$fmWy2Oyfc%@HR;ym8XY>DVx2}xk$F=$(IyyRw>D)5|Yg^kjh>@E^>C!P% zy4Z*eI(@_&s6`V>@unFbr3K`vHKD0cHJXUo_biz|+EuU=r=aSNg&Z$T%7TK(j%&YE zqc%4+(tueuY{j6N-o&jPIhp~EF==d^5?5Aka2EUW-Qdk+t4pt|@-`)1?^4|sX85wlSMpg?*lqj`{!h7f^+!)Y6?(aZtX6rP_N)8jX^@M4 zmvEXq5~sV~_m3FSLj!Fzq~gl|NQmWh2_nU@v9I6>Hl^?LpD9OXDpSTGNB9095=ufy zWafW&b98+8Wq%Qc{Y@){T{T#w@*q4@6ZgT;`yVs4rj3`9Ki8l6k{#WEw0x0GdKqC9$Y&+1*S zz7^9=*j$>goFV<88Rjbi5){E=Uf-f@fp^zr>p(-F%XV11=5gt7%ES)GC2Ghq;=~FZ zD6wXAA{cG@X?k=YLi<HuOn3eWRRwW31SE3 z&beK$lNEP73$yzcv8O#4^h=hjqgr=9gukpm3XJ`yb^kko4Uy+&4mRKabWUb*U97jU z{p3l0`lJ-FX7$utev#v>3ua=W7&@(u_2JK&ZedoI!-$SI8fv0W2fNnbZ6| zBb<3NC=cQ;Mf=_WN`3h`c~_>4SpL|KK|dlYGib#F+iKWP4gablOb5WSi{5bOUS2{*T&h`f`a4qq zRastHnaEC0i6DB8Bo1Jyi2d|2mpi~uZ1;I~R$k81V%GkKP7NDH2K6J6z~g?ut)@)6 z;EGGMLg9Y_%j!vY4-Y0oA0=B`R~;|z(EHq8++47BM|;LYdpoTbVJ^3a-^w+(9RxZ# zK9-NKuE|3JAQ4)C{{duKzujKKntze<(Zd+Y1(W`v5&OL{DNgIOI0-{49p2C~+1*?P zV$4D84-S^6;`4nAQ7g;*BUnPl*Zs>xB^bJ}p7g{2VOLjunX>tdE^w{x8}Aj#R;x67 zf+j@z-xdYrSBeha4Xse_h?b^{SOFwQ5b%G;bX*N_LW#Tt;29k*ag0( zAmKx!jEIk#jHz1A>$-I)KZNy=lOf=dRGy^gZKGJ--Jtx<(V44SX~rU*5} zsXyjBs(q!+-(xFEY#c<2PRqkMr=Nk7$BXn_3x|zIP1HX~_onIzTFGn180WT7g3Kmv-e|kevIUJ@cI9sxnsOTfW@Q_6H&~GfK$>eo| z!j9F}^0WT~LABo>o1W@0-<{d2vR+Td>-Kv3EMi$r)G-plO=Wp>r^Pg3Nzq{|?FG(@ zW7xpaoHD5AT0CA=IqXQ%Q(v=@^wgPdfg=C& zHN$G4ciPa4%=PK?{%nmy+P*FDX|UT4Rr^NG#-hnBzH>Eg;puo(A?jTWzK% z3gKDq@P?cOxz1BlkUHq`v*m2H?Pg-7Nm5MC88f5MX=tJl0L`-Hc`?VB#ks*+Lngrf zcr^7xXd_Tq!RbO^{Qu$Vt)kj$qb<me8zo)eqTBqe23LO8cIzqXIG?BX7I-O;cLbg~2cv#SvBvC(|WF@|d=GH`H#_cQilEu2N9=IE`DZ?qX7I&4gHxM>6<$%k^5f60t-NPfrc& zOkeO+fOjxXo72zFkD8v1d$)!zNr#^rax&8FI@nYA&f3IE~d9zAnH49~sCDNK~rYU`ntA6@Xw)s3;tLlpIf3F)F zg7NPkY_+6(s$BfGv2wg`bHFd2FrU6(TiZ<%iC#-l!rvF5Y`+&0Y41yS{Gr z8Q31c$jzcqZ?Wi-#I74CUJ&HUvaDHc;L$KVtGQdOFQPL3g8qVen(?&N_Q!KC{ovGP z_F7SMnL}o<-epWc1D9Tnu2+#e8*p5AiUA0x8mE?8tfpUly1S1T&;0fc$%K;Z7^w9( zR^X!PJN=^Ft)49@AZV-~741`;Bh$J7AQ?NJ+i$OZG1ULd_WZWSHd}0eY_s15MV~jB z^imWA34o9MD~O8RzKk&$VBJ0^j~&QFvRzRimWRlmIu4zO-nGz6bf7@5zub_C0Z(M%4k zeZz|HG`T+|cC-%eG~pt4AZVCQluab5-ra~zXJerPbze_jx|8)-f3dEkRXHG`WrDVMTB0#!tn&KO4AykCWdu=Y$h@kGr)gGW=n+U+7w zq`5nPf%iUq9eNu0bSf~pOrEI|8mc1f5A|~eKHN)=UqS5NOFcP@f#Ant<-*8!P{Xj} zLo_a^#-f4-zw#ar!>@j{QXbtxvM4rJb{$Uc!va3~I`Lyf(k84At%pFU5^^?eu_087 zpa*0I(n&0=Nu60K5Hbo1GAVKpa(>OsJ~{CVf$0JXJUJuw6{0_8&krv8AQMxhpb(r; z^zD%eHf!u_`1Dq={*Tr}aWi z>OCgD4jRG;cG-cvhQcbX1|D}2*zUYnmd6IEo_=|9&HHexPJ7+=EIw>H9P8NL*5Nj! zd3hN&v~<6ue`X}?b3D)~qQq_pUsCETZ}WR6naJ4dCRF?RAXCynT}yswt`bsOlEn55Ni;E~IEW>2v-K|v4hOty4nruw%g;eX0i?o|bWmH<2a3(E0$(t}C#BE7) zF4!&ZF59YaUAob*>{%GDA~mc)qK&en(Cw#zq?d`~>PQ{AdL2 zO=8xd0Ph5z#@xPU8~w8JAAU~iJCgy4`7{4AK`#(lY_gHuE@bU2Maz75d?%(FNFZN) zUiWo|Md-Vkq-|Ledd*#=Wyj@BYKNgL9Quw#OACS;v9`8Xo~+n}Q4H|q`l{RY%ur4L zC1bq}Y<@PTq+f4Q)13pp9K3kiEB$SF3;TP!)p}<|JsEZ2ocGZlW-4g~KpvGYll<5- zbfG$1=YFSzvuv$pvqYm`C6b97X#kU&4sh_7gh8ydQfF&mBQu)Ugk{%nJzCbC_tHx3 z#Q{FGyek%6pD+dId|6TWjkJZ+dr^!3_v|5e8+P4s;u4tC4r!-rm*NQ?$>_nM=99Vw ziO^@SZ%oIEUIGt~NC(WTul7cpy-vy0HX0>O;^F|HHirVa4@M_{DaY7!uLriV)_uIj zV=@^b-@L@GZ!q_s)aZ7?BO`S3P*BQ?To)sYr@h}&5XQ>}Y04;AyYbKFLbRnepBY_L z^G%r4QYfppvBz8Yv)kInN{=E=`W;pOkZSSZHgC#&jT`%5*yeI@SZAfp2DZGwd%8Q7 zFESiYEYU81-W}T2EJ@|mueTdtn_g_ow&T+mEvzmQ;zy^BW6FgEmer@JsBwqI``LqS zM&1E!NK*7AHn(eIGp430E|yfJDM3j9KT9wIfisLo9I(pK`Yf7N`Fjhb*&6Nbra-I~ zdifTV-x_`iUW<-eBqomcA&*?Ai8Maq7PX*}=p@$oEXbV=yOX9oOwJ5S_&uIK`5Aumco^E1 zw$Ls9bl6ioy@c0ZaQ#cSiVGBiAB?I9zza!&XRVAgV~Qght7b!nB*l4ebcUy}m7Ja& zf&Wf2?vH1+UL=l|Ga-!?PP?Ave6*iC-YgdfC4_Nxh)GHH(q8S?dAfZ`P7`qTm>abz zV`cd{ngP|(mI>W8{E*kFKSxfOhe1|H)ao-Xvy-g+L(N z{JJXnI@h|BN(=|9@7Id(j+CSdy9cb9IrP#^&A5meWHw&NpGjL*xenw0rMEW?X_Q)Y zxbRfD<6T0w>*h@B2-eZ6=IF-%SIi=1Vf~wl#xRUk$#?bwh*~t+F2wbPHKst#cz;1O5(y!X69uc)?-o(1^YdR9 zb79KnIih}Al24q)(tFxnDPDxmDRxTrLG=*B>JQ98*dfW?JE+3&0D6glASQ^J^yp<% zaWcV+$SUc)pYPU+)VDvkD=$7C>vv1j3SYGN%GN=OcGPazs_@Bd%w@%fqJB%Ca1N|( zUtL~4z6hlAl;zcz`|r)5Z`t+N?O`5?s_cge{~~X`eQUA0M6ZzY?<;MFg<%w_7=~Dk zW(v42T@S58H*@f9!nm%&p|5AKe(>_wIXdy_Tb+Uh1ebe&Qc}(dD_W4S49P>e1mO7( z2%G5f<=bK9Jry(l+axL|zT-2f4R8BX_%*BJ+;jKgPJ3^)J4zqpQXF5=ddQqEbD4ul%2&!eBC##4MTa=KuOvg&?YB*YIKF z3Ye*e6A$qd0tGHWwIkrua7r&qlBF65)W6#W*gL;0T`8K-5f}UrA5n#mGG(n~w}B8w z+E(-S%C@ALqm5L4&@}}vouLH$ObctLK&KF2PC(dDdt=57?Bu)T<~Pwnw{t}X3CP1k zX3vfl4xxf@ zho_l~K_o^e?z{x{kABEQD>c*Io}iPIEM2jjjagiM4;J?}S5*C$?Bkd2tnPC$u#L?@ zPpWi4q=Ds{81yWKBkQ9VxtObNFC`XvtGdx7X_Sn&P{87*AEJ7V9rfzbR>QgMj1V1AUHp7 zY+T1LD*v5x&N z24!I2^ew91v)i<|HX^mu*Z%z$lvFW{*n+pv8`~pAjJJ>ydoimU6GWGJJ z5JL+rFToO7QkDNH?<2X>wFji4a9%1mEs<6}F%4lFC#zhFsDs=|7JVjbXsMWd_O1^I z#l_Qz1CY%20$-0{qEuYGb_PL}IPE?63yj}4veI-N0a|lc12+ z6bq9lxd=K29ZjIu&KbS1lSc1xxe3pvB|zQC!CUZU3E>W?^lP9nyzD+bu+8u!i~J3> znHVTZvvkAg6Bp00=cP-M{xUep(Nv`tFF|O_Al;(IE(KgLoHD>BQBHN2}Kv;Uj| z4%h{tIp}?CF~X~jxYc!&T?r9kn%YZ=O+>r3SjxEl`J(c(m2x=ea)o)(&A>LcRCHKy zc~W$zN4PLr;(v z^l*MX1=x7Aj?|M=f~x%-CHAf2`)BJ5Sq?6%lcRDmw7P3JA2K zuf{LX4KXD zE?XFqgeq3)dscJi9dB{1bLB|QBR-B#d`o_3o5qH>?&^S|?gi%g@5>VD^wbfksTN=>Jztr6Kn2K&m*HfWH{kW>RR*5;0o5`a-mD;$<%~MLS>~ zVQP+Lp%BPR?X-Bd6?Qzw$Q^ez1C+raB3jW7geW1NkZLA~f%b5fAOC}o#u7OV0x7LR zIXR;p;Yb3Sqn%vGnt>>A!Ff9y*nA1%XRVWnVW%}A6@=`(xfqNX2FtPB_oeb_;Jfp$CpjXie63*zO8 zbRb4{&yZ}2Tm&yKuUqTW;4|?O8dZ@f{T%qInI`n%WMo$~17?p#&cA;hTRE z-~dC~WEna$e>RIi`PXRiG`*=^X68+@k!O(KxQ1=H2Dp4bo?UW_M~*%7l32W{Xr$Ig zh}3!S_&PnMaRMOHQ(}G??ZJ5-YpRYUg}&*Yehnnh@80r;v5aBlK3-gY%WB#kSiU-L@`nYf7Mo zwCj7mWfWaw>XdqO_8=8Lk>&cTY{?_Z9N* zET7X_C^F3=viIL`UOYX#1a6`>hOW^U#KuLuD=ZL5u1kSS zfX-W-%`r8IG;#8j*OtD{C(C46jsoYVUX{9z(N*_j&(u2P;7#T370LhpkIxK1&Gyaf zufoj1G?)x{SDzM+Js==DY+QKI%Sk>KoH2_TE`44E8<^yPv;Af+)NSLvTp5c*h2o}rSF_8D)L)rrP!baJn7aQPDHe^{5-c# zeak&63I#g7`y-jz6EEZ^_H2ZSmkkBaoh+nP33sorsScjg`^LWM8Wt*Sx_j;h{NT$t z28AdSvRrv)`Zv@Ttl!`Ju+(^#wUQF=W~o^h0l!gh)AQ2oqfM7Ng}jdr2Yah=1rWqT zDp7GJJ-rHEIRvj4sOmZrt=;NN<3$MeqA72(NQ@~YOdEh-q{}Tw)L}bGa&)*2&riEK z50n1-lke?(!7#YN-sJKCAM@Y)`bS|BwlFb0HwP=({&1YPY0E3+h>9f3GrMmKcD<7M zelju{?e6YAftj+e@S_*uHuNXTqB^$S)VC{w+Wu$|b6&zoTRfeR_<3guM|iW7kJpL3 z@1^D2yFHa~(Jiy+9hLhu4aiR9Iav$6S(ty&S%V*wV;%*Pr1~Q1o4SpC;PVLSt1jw zb9Okgr7c+iHV9!^I!rO}7gSIjvAzRdtSg}2gt~SV2nIZx_QS`~sHIr6zdjzpS|c0B z^+O0Jox;THuD3W<_)IQaACtUui7nX>_kj z;M?QH5t#Mc?tscbStkSpJ7I6Fw&wpQ;H?yLnZjBL*qrwl2Zxw)qIj65YJqxjZZ7rs zRi7OJK}4T^jIM``4Kp6ht(T#G69qQ{{RJlkz*HIIA9oi4QI_q>9W>Xr@l362ahwh^k*|_NI^tGNNwEP0oN6#~@`AE>lNT4X}@}Rfs_&kyEOt&!*R- zMiDb%6_d27_I2*T(MF&!`_4uPAKy)Kj!&g3PxA zvAp2lR+X?1b3#n1To(i3SM`n*@Cv-yJ+ny4GJhiC@m?dYr_su+*@QpGelIwQCAa3z z`fMhp{Q7IA7f;UXYv6_@-d=uMf*}Ztq{eb*9A~py7C3o5E54nxG@*`~m`0yCsF2FB z%Nsm%hZDs8S_m!9at-cuVx*xl&*u2=mCny^XmH%CJ;?7t41gpI?o6mIT}6%r><$mWybPq^cpO^uD3^?bJFmmfUx|e--b;v;?DcaF8;-o z$cc7?-Ln6wJkjy}(0BUb6pmsx)-!9z#rlV%VI_8f<*(j1?}rJ6VK^tEu3^^6@<+c_ zuY+m~6c1m`(Q59K8ZEo8hJLeoGFG1$iAwAASEZgFw4$$eTIW^~VS%=I%m&5q`Gg|42y zhM{gY2?_D@V~P7d3CUpnmy(i0+uuKvl3^;MrweT_u!R+DNTNZjx4TJuUU|%|ea1(F zKQqUHxuvru+M?~h0c)=R zodtmI_Ffgq4Ia*xRm%8DS zb%U!%O5nZgQKkF%N@MVRg}p(;u#Yy~CGxvqqyn=eY*p z`Tiq5pP|!JN`}d@QSJ8o=bBBoy)_hyua@pFCUChR{_== z#h1uTmeq1|yH0dDOx3Q?(QR;? zAG0kx!QwP{UdlN0z1hqm;V>G*`L;lrp{Jp7wMSY3jRpEoWPir-J3il}ee&F6TKmwW z*GrKGm38>>1-^Io&WHap*KtkS#BNMKK>QMqO7q%1oTma&%O4 z1z{)maaka7GyAYeMQ+rg$U=ESPJQpgfi**YzPD4gQP^6dD=Q`CKUR3lnJ>X4^Vyk zf+gi=c%sc|Jfcb?;~X|U?b=#Vq#>p)Q~~6LzCP`Z2zuYg!6y!!Z-`l0Ss@SBvP7hg zlFaURbl|lc3pt*?v>sS4%VXD~?Doi~uE)h%GrNzlXkY;Cn! zn15f-XPC8!t)jTi?I2GK?UOY$daW%NdU=#Hn*P4zWe9*478cZsr6J#+0C;Q?`p)se zTs1+9o~*=#8n-?dPHTfTm4iam%_)tA)2LCfjOKQx+JbSsjwuj6R5vroR~eDP8X_dj zJF&P}F7T5%MLt=ZleyM)u~301lq%$PE#{{J5s&Q8vYpb#_sP{TDSAD9eVBEUyAS!C z|21~}idau$gnVS^&Wi@aEfQ%Wdz;fd-U&Qn5Pg7?Q4S4|5?Bj1ad_Gdfs`^+62PdZz10HOqijaCe zH~>w8sfN7&1k4iwXoH92`uRIA3;G=#nr_n z+4?&A+K!)@GMs_WcAs^H=T>GU-soVfQ)E_07F(b-mmLH_tgBrVl}6{afPA#?Vhyd{ zs!*}@op+e@O)K+s@&W=7=}eHASO?T_f6iHAKUl1EOt$Qzm-gJGmDjqn`)&<_*anvf za00vTZLCA_yO}B1$Ba&d2q2EJxNfy|AG6!f`6>$QSNGaja@g3?y^V3xK`-s za8pu98&J)d+Si;&D7_&dVIf&2#miF#wcZxlNS`2(6r5??(7r9@tdZCo`UyTAkUmI} z)+SaK;%HF<*d5lLlof0$bH(8FY%v3?5la_eU?hCWH_Y#3%+{PV zRK2Rn(^f4PQ6Ke5V;okmN4a{9eI7zvxW#J~?ui0gFm)F|=1~-6Bq^VU7sIaJV%A8% zEzSVSDYi-CaKC@Ll>ut;xI}h&e+i`Fl74$We_sPElvd8~pczKqP`QV|_tQYYOI;l~ z7s;YDXdix31rj!=kBh$S{PlCyVmH40$8Leo+P>*~ezDBQ^8+J_8@)ELBw-|T1%?9Z zL(wfS@w`1iz)OODZo~u)?0Ec;3(B(`X65CpYh80M^uEgkc+<%qyVo} z-!X2FMU9lJ=6+4H4X65WMDS*OgW+@ffX4qYval*NX16{9L}P}==$6tE86qW{gpEoz zUyh3v01v*L?56|3xKrx+o72|2wLXk}`d_1n@K`rk7B!0p$-Di{%hj?x%=E`<;l;aW4muVG28(5>cS_u5*6 z(yU+IC|;HMTC2_No&v8+gRZ#6>_xLg(1S-0V4Dm%7Bjvx86zK8MzeZ;otX*hCxMMH zJt|DY=3-%_3ddse+4;ss<^Ful`;{;7QdGZQP%z9Qki|^&4dC2<-F;)++8HeInmp&- z!K0K~hKNDL`HFq^6s>N{>+WJc-Vucbu;mpI0w<@Z-M40#1ANuwKxu4cTUV6cZoh77 z*>z)QWPjVOqkJK=4L-C3eu>R_Fbq#{~x(m7bOACd&FQ3dEkMSC_7 z57HOqF36Srs;Cba!lD%BdJ~Qo<0bE9H!DF6YXS8df2XvrWUr#6QOl>HnX`r#5sIMQ zDrfVvN=S3^!N`ml=QE1F@PYqcJpG;%MZ(V!nM=*ntOLTfHOh$JL!l)qbOUeRWC^(b zY^!(R%CbEi5We4NF{hDcOYeMz4MDW&=tP{4U%W7w$VktmRE9bSyZ`aq{YIsRn`p)@Q+qw?2!6ry zOUrM`McZ6>IvL9|_*8kYSQ3ar75^4F$P>rAbwbOt8n+SWNW0fQW z#aHvlV#nIeL`=BYJwOj?DHGfzr7Y+5;8-t@UJV7p_4lrQGL%r1s3P60X0L;X{cPPv zmz&wWxt*kRZj69GH1mSq6y9!|L9As`dJszm<2H`-x)BK z{7M6Y{XKn1nMX1|Fs5GB^2Iu=(+1+bOVX-pm@6;#e-=kl`2kqiI$f$e$|!3_IKvW4 zMCw3}LQNd9`V|Nvc#L{@559f*8iY+B%H{6XW>|yQFMLhyRU?asByCMP*I|4Lx`(WK8RNA&8NP zO=qJ5pGN$3hlN4MfKJFApSpJ_9eVybeJ|5&K0*i#Eio=?E-yE}D@=($%m{3bV?w6* z18vLB<_KA=hRA|+3kW&AtBZ<`b|Pxl&*Dk{OS7;r?kUg~;M_cGi?16}dA(twMc088QMJh^{iS4oFZx-f311PzbDO@U*-C(p|%9OUPd0EHd~OGgElI zXi!VA!E=91cbBuR>DDky;cU%kLw%lg!JaYkDHBA+QP|Hm5%L2c8JcBAc-N@ zyVDf`n0o%b&wAb6#O@piJ-kaAnOJ-!eKzi3LfK$_C#-}|VNZjO`Lxdl)z8^dzMLfX z^;rhe=Eaf;XsN4fY3M_*-$%e8li5jMKNG6%FGuCUz>>_DtB>=C$LE0}LKQ}huy4}# zE9*$~<$R@=b*Vw4hdu+=(ksfQB+au`WC+wr$dzyFUCkf05f53W&D_`3zFDV$1A%ZJ z62C$Y?HX&{yPMw7`&Kj^`5_2gBGIm&F}w{hAHcoEc1_+DnilA4!+STNjynx1wZpC7 z&`TgP15v#}WmDoqGwY)3Q59qg{b_%}%N&NlgrN(u!XSIoL*Lje7X93KPMl0B=Q5(J1~< z6>cfzx>M!t`Xt7-RWd%(R8+($#5I7r5=S=>d;&FoxF{+Bm_Z1Zf0f*3(Xi4Cew%>x?)&leOJzq0fDDrY2WjhXdv3U78i1gu@+t;Fp37G6 z0QlOy#5wQh3|mB#dE1Sw6JaH(zxB1!AQ^z2$O}i2pjtqOz|~EtgIC)SA`)qHxePEt zvPYAwrD0bkaaQ8%AgpD7M-QpB!dSu`$BmWaK8{ycqQVS3>!&VI#l_=c+19B2Bj5Qv5b6vuQ>-`RDsGGwuJSlp!pwO+yRsT@ ztg3EU^Jb+_j|ILusV?u(eX&i^;?z~w)mocf`wuJ_)t*iWh@AbzZfK;hsjDT(y8yj; z{08%5cwBvI|MJJl|7cEx;?tMIQXovX=9W{`wq@bG1o+%NqFW=IME^Yc$azaCeECuY{_tnj}nX}gU&YP;v{cMc2XI>rYY z#ao-fW|m;%0AUx>vQ8NU`lxhlA$9G(8Mz@HRDqAn{gi@RET+39vYlEf6?;sk5RbAn zqKwPsyEW$w75}T7CeaU?vh=O~Qpf7*iMvoi=JgU;HjB+x^kydhJaHAYZO#HN_C03`hADf}jl(~|f1Ar-mN*J{3`$Tn=a?fKi5oPfUpI#rL4WB^`~XNb3iWjp!%M$%5oKj!?lIc!DDq8A8A<4(J_lr0*$* z_w7XLLC0nqBjxlzd90EdDxiXK=oNutIEWPDX+J=ru7_%TPd->*n&I-2+N1tf?VOma z$8{*v!w(|+F!rj-Bbw&CL5dc^3ez7>b3{iVNnseK>?Rb3_!U_ohW?F0JiyH2;r zP&RqqCPDA-py{^*b?Pv9L! zY}pidw;OaeF_5r&S1Dp75@FP+K$I)Y=9j+`%Jl>J35D20-}Z_q$1hviiZWB^sTU@M z;nVz1ISE`Y*g2UrZI{r_+rMKbJf_7ehc|86DxSzVm0 zcE8%|8`ch$6B>kpZOI>^ijoQih4dO6*Zr$DTUg)rv%XG0wKsZrtjE|dhsCp1ls*^T zfg4|sDq^zQZ+GybjC9mKxXRmjdztK?iuhoWxkn^r{Vpyp*1g=l4MSh;jsGnjb{MFib@$XOc*Oj(I z%Xo|ST93aU@gc+j8m#L#8*4s$pE!+FwQ$G3uE10+Dk#B%Nn?Rq@%A#;s9>f0qUclm zmD@J0E?UO|8RGo=dxmc*z{@4A3(t8|C+P)!FSOzx*Zgo=s%0awKXLH6Kj^%Fl*Nek z6SkGnBP0~%6b4R4FQ^I8))#a4Td!@d6H@q|cYNhnt3G-hhTow^>uD1KU6mQFfsaBb8@AdoNUBUM1JPc2b)$5M$ zJF{Q{kwd?e!ujb^Ak5TpxWM!HTw?SdbTc$8tha6+9_T`ffYQ@{KSp7zvmWl9k~OET zR=0t)`)H*hoW@OFI}_9@$2CfZy9633?7D8JE9qi?@o^;BlJ)+WAG8@BDU%BchcF`q z`>DY2pO&Z7Rq*eF_Ll}ICxeagH1Bk%Gh{`8Ubb)Cbj*tehIAPpuRLwwsfcJpnF94W z_%easT*)U5h;O1yX4zqeUoO3ytwQ=0ygw5>tgKXCt%4xDy6|ya;vuDnO-)3HspJa{ zNAemPJi1e}vjl{OwdBLEi>#dV+ZWLLVdH8Wb`Jwo5fKrT+vCt%SQ=<+vmDKN-5qa5 z2! z)tas{b|;zlgtX?mNyn3rP#CoPLD$RtTF(w6YQnK1btEFHBQFyYs`p{OyuC!mZp_LX zh!zU+tgI}m39|ptaJtXmqo=U1=tsJJFoTIz^{C9c)vxw}m0zS*OW)N*)G)L9I9LM$ zAoU>xK_k-~8dr6;R}|?f5P=sHbXRumPoI*KlKvM?l5qy0$)~2I=wrj}i|nDd!S$*1 z6#ac?0%VPoa=9N_d>8e_l3j1g`<1NYeaZ}$PQ+MihTYhg$fAU*NBY<-!CJrl7;-ky zMyAU`agQv+F#wKU9Y!*+3;8>Zx7m}{;D7F+N^vpcpdW-8P5Zte+$p{c8^6xq1V;b- zLsP%$?pgbRH}mGlQuxaH>m#M*X!_AEN2$)`L@~nh?eS`Q;=rSS z%Pg_(9=U%|+dssu8|xe7c%_auK6*UeFBbUii@p77ollW(l)CU{re7&|qKCkS;Y86* z1_DY9{Lw8Xg&4qF%zc&896@Ts4ct{^4LbugFDl9`*TK=%MV6?psL7AEdcKYN=-xMS z@ZR#R@sXDQbNv4g3*z8A@dqXS;0h$hK;7wxb)rMA0_WqjtG^QiDbCd%OwYe=-SlBt zH8`xGIve|Ztus!Gg@7*C&9F@fK*JAjCG>39S%{ni>o+km3#B0V;u{$1lcS&=pXn8_ zFg6aA;fejQR?EbS*XB&QJ;h>d&+xwJGibSu9f0UzVPRo2-eeySys3MKxf1s@EwNX-MfZ{T=rr zp?KF~SFY&h5UG-|u-eYp#sU0TWi$9kM7Hyc{y~!S6J7I?9?5u)@ztI_5hd6cy4kHX zYKYyOS6a=p`+VG6WbAi=^o_VXQrAFT@9le@|82?|Vt<);j=&s;{cnZ^RD4f0$3#G1+B$ooM;$9!2rIRY57NsaXR{ zWRx#YIdvr{Pbjc%IhvUOh8npfO;OPw$Cn{b9T=RxKAknYZSn%tsm5&VZ?Kw^r!Da! zdbg~E7`#TZOPEtwZbq?nG+2x1?9VpJ%j{PrQ!Fsw?DOgZ0Q5tb?V{+A#p{liCm-+c zJOV^962uaG{QNM-$|5CZ;M!_RdMNfNs85ov;FpYMny7Kl zMYYfC3%JrE8D7$Z=V0N}`Du`}Ph@;FhcDqn76uWW0Tz9!YK|Qu=XIb7{w5|J$PF8YtQ9h(>{)9aR+?f3j&d|Z1+tLv5BU~|<67d?ee}CS6~;yGR)&&GD)<^0iWEJQP@Q86 z%WCf8Wr+BUZAQNmkD2|b$k6?AbQntpEE16`lz)7Dgr_x4Rmr;OXn8R2%{I{cU?si# zSeDb2C+o(M)dr&~-dm1}uFrzE?k9=9q}Xt}ZeD({5TYOK8JxCKkEp&|dwDvt|8~^= zJYvC6?a1=9q!I?t43fV5h%=et*ixPbeH^svR<8ctWH~od40HRd>QUi4!+i6PH`;%d zQ8u>L=%Ta3;je&0q7LumfPs^+6PWq+io?JfJ3wk#6}__O)@*WP<4CV-(Rill$-!}(p^)28@K;Uy(V^<&^NG_FnE$U{ zESYEuOYyJrVxPx!bB4r&(=V-}6FJmNZATld$?w5!mSw5K$ICUI`{gw`;8ypW!y<62 zIg%TP(27svMILJKgj?OqU?|4xYpZc!D-1Tt0bjS1aDKGy(e-({J8rwPDoLH4Gd60l zSdTdoUePv$xv6L$?1gb=3;8@=+sJM#sK?<9a&;lDd(xPxp|gcVtJh?ges4CJtgmRT9%oYh>sf|?t0kB+K0peU_}~7) z@j}h8cB5iyMtNUW&&18^Sh)uVM(b*dcJ ziGa7m!*8o;T?1Qtf8Wl;tNkSK!-Y17P+sH3bEtNXu2Ie7;_zse2CMuwAr7skz)Rpm ztY=+A-^S{Dzu#kTBPb|F4qy67kB)K%8K>Yt0GW5zg=>%h1%9-98EGO?P*C8s zdo1PsXe~*lsQNw0B|%ej_^^afi8a=0I#}QHbj|&~tL5noyq`>lWgNgEp2@t;Lz z!sPYMcSh%KrL}|Z2I^a2_Go8_r3EH&0h5i5(7)oISU^24YpezoCf?MOzW@o4HscToS?u{CegmYh6QjC{6hPusR5%tuC%;|Etq|ZP-FuckEBDv4wPkk@ z`d*d>50hS{oUG#5bMfZt@C4-{ zKcA~_{pR|-f_RFnIvwb60)6OmaKzar0nh%@=s;Cz}Pq63Sd+%7V{vRuN zY66;z5Vj|pgoD3J{k!vP;L?kv7jaApsOny$X2caJ0qVMjP3;ld{m6TlweEeyP9rhN zM%Ve|j1%AGyE^ij_Vkhuv_tWuquo-GgC?>Rg^H?v>u%&b&fTNnlD=Wja}7}X@m&Lt zUIFwOw*Mp)xceEOhUK)vJg{f{6}M`$uUxU=1J#(5E%S){`)tNg#d{mKPl}neBaG{# zvfrb#HexgW&cf8`0)eTNLCA2}^QW8cl<-dcAH^N>#}7la%Le>J zx61e&#wVRWs6lfRr|3>UP4cb(MlSq+({ZQMG8!u!3Rc79yBFebW|uq3;Ss@cu(dXv z*1Fph-=&@QS2KH;zD#+IkY z{XMue_2^SK@xO6_%NM-DHLEsB%z&5&x9hrx?{ME51rp~{=p;3vM*ZeOa*2kCX|^vN zBQHCEr2=CwgRIcrGfOLqiC-FepizME$&}toGNW&G1;t@70f}%W0~gH7B5OVE2e~(j zlL!?$HA)Fw0EqSQyh$j#K1cjpUpgce6Y0&-YP-L0=(15M&_u@ONJ-(~1+7t4aoKa> z3oF!0#6Uvqwdxr%LdeY{AiX%~>MAxabhXpDnGi20-b^m+?z$e@0MJ+byB};^5e(Mw zfQ%Pgp4$Ra$~;OeHRFT`Kc+6gu*BIL;Mt8=OjPf?i0IKfk3>=NkKOzyV0lqzsk8fs z-RG0+mjat@J5AYm?)T|6dcW>e4!;xDzH{W2pvEQn`dJ~5mBBRp|DHD^&#NRqH)>B} z`V+oSHl;o~JCD+ztOsN0e=qk`MrLy}mtIm-o=Ln_8oVD;vD#!1#vUEWU z^!=$;@=^VId!T1waAwcYAq*3YVwE5M2i7?}9;$0Z)oZ9!_*))qV`(N8iD_BMEZ?&K zKU7_1SX@iCZ8Sgx50>CA!QG*8cMI+o2oT)eT>=F6#@*fBf?IHR_jm5h+&lC7AI(Q| zs!rFb+G{Ua``Y5L;3q6=V0=9VF_yYKrf08pR+Z5HoAjX~KYKnTu818>#{aAD#|nnvv5D9Tuj@5{$P!GydZJS?M05 zLDB9GQe-%P6FDBCGEQE90WuYi`D{PQ8x;X`IzZ5`H1yYHI$5oGzC5|mWp51L0#+BL zrpU+hO`~Kg8VdJ8uUEt6Z7&aC;lqojgsikinWef@K4-KIxs)q1WsRH31>*JVd>lhY`rvQF50Qh*Iwe@X*N~# z5LTK}vv;$y-Ix*B@o&8sa7zHf>>C0TVDy(Q?!_V8I4+A6?L8l@mP~dkGd{M6zW~g< ziSn#J$7KREZZ8n5DQeLQ@el?db+DwvNmrT7Bv#+MZH|>yTg2QaT#4ymu8C9b7yQb4f6E?euK@{4wd**o(Ak-a*n_n_~OfpSH9Jnd1;6 z5iv3Tfg)XnW~+sb&suekPS;;bi@mImxxJp}URT$aQk5bRxgJrOxn!(hV#{F|M>PQh7il z$8Hg?TEbyf+2I43?@!~T+*rT;bv#_#W}XoLV3wkA$q!3|dfM`lftY)*rl+kLTU=%C zX% z!NWC&hzK_t7SY$=61$7L>qidWacl9(7pF_W$X-=NjT;3+f`xL20wZ{-{$gizq)A2D z^YGUx>RFHP>8H?--xiy;5ANfBWPJX#&e=;wX6{uS3l5c>wOMq}KJlZ$8jG-Vu`=2V zSv%2aGy48mssk7J=$2x`{_axmb>fV_?K~$4i*C}=;LxH?o1nlf z^et>wTboa3+c!8?C*%^HTKB;&xZUx590T!w(Lh(XW^~F>dxcyh!HM0OuwAhRQb${} zW1CxAt*RQYk{M<=SQ^QI$zc6&mX?-)t%RTuoaCf$SJ#lt;kbv6)6*RzK^ERIjIlj6 zgr=U2jg8drzy>M3q8bg#&%CgpxUrJY$Lv>F7ujW<)q>srNF{m@G8tvcEW>v}ef@>} z_j2hFZoB&pIECrk{Z6mD=2zqUSTZ&E2kfm8@lU#eaj5Mu4nKbI0$yu0swD&FulK~0 zNhVZY?@g3Q?KnPt2um5T*;^4lbzFSw?rh69(*%)&yE%&e}f($3u6yryt# zVTFIb_3@rXL=a7+s-odnEx%I4I|JDBn9AG)`I?G`l1$2JPp1@OSgK5;@878u_(_>b zi73Oj&kn>?d9<31U!H={5JbWq-?7SICl+E;emB>UP*51Or+=}%J=vZ9L?i-5_Ew78 zk*~SUM5}bNQkmy`?RD*S!TEAWs{G~0^C&A8AOGs|Y^wYhIvRSavnDN@VhC9rn#H%{ z`RltDGty5a$$G<#Csf1`Nu^A}fdJ zy~}CHMUhs8GxC+eqS+iKX8p8A_tN5IzoDWLVAj;}>oK`_FMXsr?&>&gwr&AG;IEne z*9717Wj&Q3stQc5uBM;jo^SscDd@;LOR7tjkt0_0w;F=MQ_Nbn`GYpaVu}Ys7-0`* z1XbZUuw-@B5BSJ^s9{|dSAg{{*w3nrpD`d-M4)V$qgeWEj}pYlSDo-Yb{S%A#bJvzo-7feK{k=Ct~M1`UjT4JSA_xq4kBnc1nGl%Iuy z($tZj{ozI=J6T)k-4k>wDS{vq*$PxN*eYe_O-FS7nk!9ka%ivUw7J*$MsLLX>y#fH z#NxCcrO5hr)Z%scrqjU6!5X~<3__gyV5q@l;$DMFFfn6$_R~f3*wYQ}&N=QR1c|Pg z@@>HV<))aMn;ZH%XSd;?5Cf<tVrf@+J1^;h$wP=1^*&@Z%ZJYwE|USY z);K-2Y-hS#b}%%WKq2=P=kw8LnL*#Y;_gXi}NJwG&$@aYt8wxB1&-a{mu|48{Xk#vE6Bb95@e&A`BssrF;^0xRq4@ z_k#P+x;x<)N>)dCxVb&5#*r{mE>Gv#k$Y!F6hOh2ol>-uD4`nHBYw14Yq?}u7eN|3 zs(qdiMH}_eUn3^;y>6Ss`aPu7oL^(@x5yDDmdF>KRviB&%JYWTNG9(d@d-~r^|o72 zfeGzT_z5YcAN~#a39^_Xg;gG&nORXGznR-(F}<>~f=8e`x4ax*!xOuKSK6vZuj5?j z8fJrjEu(EE)GHD*oJ3!YNu;0nJa`#oH zlzwVr83zfgFEEh1iHlyRp~c!QB7aME+L|rDqT=jghQ6d+XgUaOGHEe~`#&@G@AWji z{SpdBe-80HKfs5wWaA;Rm^k4)d5lZ$Dl7z(de;Ayf_p}b6DatEr)><$5avF;&YNwSFc%11FGm`(=q!`bKtxcF2=h0rCUU^yP zwpndiv7!>%(B<+nktMQ$tZHZ}|B8&dxv6=F9DzGC(lS7a7wdrFUUg!|#rRcQL%TMI z6qTTq+r6;5_s)y4KU~XlE2p*ZsWLXxM@b^zIp{x8i^2&i08+h&$c$ ze&i4d2s& zdmZtR*wT?fS@Ip)9h{&S4GaE572c8B*ox4V=*tGWFBvjOj{ie5Pi#PC-&laCdWZ4x zrX=NwFF^ed+?RGFBxtYjhxGi22M}!lowRNGI$mu!21rmn7!$Tag;+6=^-RN~_xE=u zDn#*l&_)sK0lIrYlBoJ1gMai53!K0@m9y`;ttzQ6Got4icG!C?EUY!T9J7CIVE-N= zkC}UFLKua;7fFaXRD}=%IkO0n01>C|H${ec<%^MUScaihh)Qv27k0?|psw{r$mk## z(pm}wDq=8h`Gt9;ggt;QMAI^&%~Ckj1456l)|Z^F!!l50xc3$k^sZ9CoJ2uIC8>}W z5}oit5f11lE0xY@Jmy8uvO>YW^H*X0xqruj`jeBY z2!7unah}j13zqCI7gR~Osh^HnS|upVIWzkAHu2YP(eKx<93YNr2DjE?$HJWs_L0T# zM=LVxGvVI+Rf1D$Mhuv;O=d&hTS|o@p|x4tm_Kc(62B;=x7ynrW~R!2Vxwina=10 zAyBXalwOEnaz(D&KvFEvX%6D%V9V1F#WHfPFCH$JEt~t_xCe{OaZebxXy<4rc0R3P zer-zrpY7s=qFQ5x);rjax=))lipPc0+nUWD={67BNO!A2iWXkLXWa8aB@Knu{`8zfP-jss+T`M>($9<>*2ReV`fR(B=XRDn zs%2p^_fpUm8*zWdZ8#%9AdAIV$3=UW2FyGtyr#pJ^FP=0De@GlI%6=_W+HCfNk1fITHxY$bkQUiG^hupE(ms%#E|ZT za|`@#WN2~nC21r#b34j3A7|8dM*4-RaX$N;A$wm7q6tFkb8!5<;39**7C#+$arww( z`t@DR|6a_Ro5@0)IK?2xG+7kt-4|3YH08p%b8uY=G}><|^jVvhy1A7;dwvBJnO~1A z))(%bmvWwme)j+GD$@+uM2O*9FRC*RN02_C3LbIWv`WNvTU5n}5fYKt!b2kzQ}jR} zgi!W%(2HtwAttuCihyM9Jf0c-Fkfb~BTE%mbX>l=|8>vtUVNRQZR3Bg)KOIOh z7x)Jqc1`&%e06S74=IN*(B`)(CfvfG@Z1PZ*43s6pew3-Sd@p@Df8>S6B;P6HLbZ$ zZB7|Y0VNUBOcbOZpmL(7&GtV}1CCF~U)EenCu>#(kkq~7Z%;hX5f1F(R9Z=sMliP| zp05h{d*iuI8kv#mK50}wR2rq){4Nk*w`_k7C}oGRWS( z>>({F`YwErFX8fmFZU;gpDp^iO-#ior(@CY6htL=(K?6Z;Et$U8M$6R{#$y=++OPQKND|g z0#*r;J~>^GWY5WXzS?ahEqF?^su)yIAv;=Wb`nB~R?H`h??Dq_5F&--3Q-eJ@iX|Z z3l^$h(BgmsD+q!`OIbHcW~7lA4HC;>>GS5q{{?6S8%YVLZ%qg6&n6=IS}XapMX z2vv!TvGK7#;p2Zq-8nA=Vwkb9gL@5BsW6b9?c@JqaD1HcIq06$-_`#*uB=YQT}A80U15}}@} zt26P^@W9im>%qbj=9(*9Os`Yfzk^dm?W8a!mMq4kdAtESD8VMH2=GP0HfwID9=-|@ z)I4wHI#ffzA<;Mq)E$ZH2qg#(T&nnUAz$5xV20v3ux{}^45f?8?)1QzocaU*>=I8n zMoRhQrFxM9(=5*Zex;G5s3wu<>1n=ypHS5eP zkQuEmyzYmDU6;4MxPFTS*8t;Kd*1Y-`&CWI-A|7#ZMEwBob>$seClOOSse%=Wc}+6 zn}gWl>KigryQ?NTl1|gOHLHb5pDEK~xGFQA>qjmFM0o_F2mONOVs;Wq< zpAQiuI`Fd{Gvsg~w zxI8xYjuXp#%_N*|E4qHqsva-bK5xa40Nvxij^(*c45ab34p z{+GteO4L7NH?tn6t=u2wXf@j&77FU4u73RD^;+mstHl~m<9@xb#1L^4wN4R&HvG+= zViMdHIFQcygPL9Juz+62GFxwf>6JNIL~V+YC=#$&Wn}X>KiXHy$WZftBdR!zMPIUf zC+_01Z`=PbK#7LiC-I2*){D6 z>J(q$fQ;{m2gpz;cf@f4v9_`j z%1FSZp}nBCC~FOmn(~W>aX-2mea109Vr{Jx-~C+|;HCDpbpa7B?)dZ+unOAlZuTrV zoNIs5keNwL%zq%Am}X~~U$PF-bcI(d*KBj1^nJTK#^mo)8KW+z=H&Z!^DF5(G@Q6T z%K)_S^znne>3Fz;i(_PJ`Kpiw{!Ssf`c}T8B_O7pddpdoN$&FxIm_= zqK@h(x_9iXtVL+SOzm4VySl81#45o=WVs$;`D3dTvt`Jb@$pHRNn#DvjXJEG6SP+8 zo-e(N2at-sxWOxewdY)pPZrkSh=s+PGPEiBRj zgur+KN`o51Li3HUzKF8u?HzDf~I5v1v#VcctlHr{U*H$#G*eb4xZacuwDdpDsd znoV1@40^0W!MO-pcqSPhl;|PwvV-~(_H|Go1A)#F5ZF#+$Zk?N8NJcVzLN<(o}8Fi zB>k!9K1LE!z|Ze?`4EACMjR`k{}RyXb;+*p^c|*h77Y~@3Sv>}JrC0{lDx_?@R;4f zh$p>`JnFW;&3nvF|LMWu6+7GOdc(eP)hN`6P5*re3ne@jhIQ&UbZ#$rZESRf=w>|ZEO zY(PEUH%d6TT}0A&IKjq;lHAx<#JBUrJvz?Uu>mB!rJi<19Dy+`A8B{<)c6o zg1N5ypPoe2REm>RZV3@9zi2erniWzcNROBP1qL~+0MOg&q%WuNxiS$WsMx49LNsTA zkMw8C&UF#2KKjV_nb?4CUh5ELv|r`cMc&E0@Ys?#bP}*BA0sg^j1wbE0_NtY7+HSJ zSDaMlA!D+#vfN2zNMld;zHh|Ne&cgwN*ygWdUwgD@Z}dq`ECQ&MSHojFa59Y`VUYp zFIrk!+fX>6|D5EV&j#vmHR_xZEghqnyKijSNJ7F7X5#vv&buygu6xk6GLd3Sx?mM? z@Irv@khRv2V{zhz<%F-51{yVl(nzZOs8RG9bylmsL1OSBn0$gjB%^n1sEd>;UkY0u z9-H-77Ozz;n^rg=6kKfl>x04WP1!_M} z!}PyU7*Vb!uJQEd8pGW>aF>nE;n&)4-BTg}pmVud1G~LTL`b&N@e!AH0VTJk=ZoqF zCXJf)VpX0OVV&#fgIFS;`&rt!qL8q#FiLw<-Of}VO*bWF=*I(Ws>($NC#T!K{yA25 zb}~BTg2pUE1RQh=Y)s#H1|}il>0EBp}ziyuJL5H?_}i z&85|q_J_)A%hMVy{sif8);h<$*q52%q>Qy2Bk+b7aGsIb=N(TR_7Sm6 z8+qAO%Rdu-Hx|<#)7URFT2GI*Ey5!G(EA7+lTaD#6r<6^;q=7CF)g4d5viDas?T#W zK3_c@r`Ge&sY{Z}W)y1OteZFZkgCcf6r8LzEV4Q!#<8s|n7c*s#4^CKj3sw3dW^K@r4>386 z-1t4ecTN)2%%vrl&Xa)lTUC|+Sy{nGuK%Tet>s5E=O`Sk{tSx?@UNi z=XBkNE@a*31k%lZ*)(g5Q&!T>)_1u%dB&HEVxBh-YoVFd)t9CGtsc#l8dO4{7$^=D zzsL-p$2fx5LWte4PtuYdQ+rD2`(_`_j>NQ8nKuYyX7Z^9=H!vATObV|vra%E3*^dC^%mIGg%e5P?c zt+BrWYVrKaUU&5%Rn&)YA~3;+@I+Ky!;D8e%&+GInxblS$Fv?l?NBJ3ec0>ETzT8= zOPkdDm5<~l5@flkP$;t;-WVbYJb%@aXMUeEeCaC5a?U`o<*q4;D#NHggZXjMMkB6& z%ohu7lwoyJNT5ybMCnN21W@XXXagk3N^&|q{1{Qd@`QweaOx-qhB|JW1<%i|1fI6u zy-bSW3fwv)=vOOZvTMIP!o(WIfr4liV=UAfYpf>0Ehd^H2=`d-n2H_vbJv3OQXgR-qEm=r6XtyTpG3S#pQC zFL@k@n@fcJ@|S{oqdCRe|AUP*h$YhL*!^XDe9CA_u@df``31SiTe+9XPZ?V&zEpbt zEsa~TYAH36At~D@q@a40Z_?y(pRJEYv%pA!T>>E3t{@^z2{dua=xlX~OdpAAXaKN6 zs*GYl5PE0kXjxh^fcvhp>yjBM1hyWI?f2v@@J7I5AGmV3I15e6nhJoMmTq0MiK zyCK>GI9>**0~l{#UuLA?m=3#N>u+d?5YQDoU*Ew!>3!rWTZOfkCnlK~arpCw*A&tS zCBR@u;9ck~G?dkNxfArR?wd1@h3)Z0!dLC;4r}hY`lWb2juBF=iTDE(DAg>xixLVktxij! zU3W@sm)X#RHJ#H-!);njB`GCZAt}Tn6R>Z(%0t_7BARfxIFUqutmFJlYD1E?Cb|%e zfC`5j9)Lor>z32fJdw-2vy+`jKARkOv~k$8XqY@)o%$3`gRT&r!d~g6Pdsz3MzPBH zx;}9P)3wP?hU5wp5Hf@h~e#4XQr9B z_f4jmSf_}Z8PUC25lnHgoO;Yql)9ZF5{AGq+biT0nZ`i2Mz^jC*;$gArOPCalM{40 zIT4M0>HB=Y4&I0K)|`_SXO5o=It!2csvvX&X%?`c{`jIz#~AarG)f~!M3$o0J+#uMVkI4k3=R>cJfWH;OnMAgo&(T9m9gQ z@-{A%c$zVi(HZos$V0v^)pHhjCAJyBry%U?KpdAU=EH zaijV#Zq zNre;spgHZ5pgU1bVkXnn>!@+1XjV zxprswQFLT_O^fLe8q-K1xjJGYx(h{FFtJOH?|ut8-3vmBD?!@-MVABbD(;53wzgK~ zkRVkJUq!;}VIjuQv|av#^-4`yev^f)OF(XaU{mS`d_Kow=JV9Gr~J)_qMRV)C?ZWw zotNgdCx_R50_~=mR>QfnQaK()ollq_YU}D&n{IkHEz{L2oQrbF!(p?!AP~rbjNAwn z)!zOIhx6Gqw^I`@dYOyG?&6QX-n|((J+*i|nijr}Sb-f+nW!sBUurw#1Q3(6NkGpw zTjFKcKbB@sGZdFYk}>z0&-1}5*J!m|_jOzS@cQ|*otqjd)LrSc8QtdZDl77orLpvv zFU(s*xZP%imA9O%0*G1S4lhslX=!PbW;G=xJago!f^aIMDST@Y3sIy)$oI*0P^fT; z0&BAfRr_l95c$5U-SoH?e2$ZqL<@Mkw=bV{{7<3oGs2da;v>e_9TwkoY@tp<#8kt> zdp>wx5ruum`6PcWgn?f8q&HMcs)`WRiV-pI(P{LDd9SlYU|;!F{@?FSY@OGw7S>xdjo&gqgK&k+SI3F)s&PTyCNae85u1e1s<~IS9w?# zJMl@2Wi?EAn5z82bA|{2n=&?!Cg{=a!x49=FXuR4D$ZbxA%$w`*E&lr8B^28ywqdK ztmCCR>y@galL-nK-eZ7+ZjqiHii#4Xr;dZ!q-6hV!$%WBljrxw+YqwqQzpxRzq%6@8 z|GnuR7^?3lKn-?saw8%jKyLkKC`%DQq8pG$ZA$gm{? z=WV^*_gyHN)k)(gr~G7*x+or9E!gd-mvG(3f-C;%GA2t1%XPvpk=t-6C0%_E$TV_C zFAA^tmX`qRkv(>n+0UFy0?o6lvXsI|;A{R?Nc6e$8BV z{48HTJcgO%yex*)0elp3!A{^Nl%?_i*r=B4E6lCWC0!3AtLu4(b8y^Z-^>>fEC-h< z@1=VxU_&9k*z~AYs#yzQ6ZTkRFoX&7dTLVBr9|m?Uy?85(hkdXxu%&7G?$jhX>fgY@snEsDt<7;y9PVB-pDqWmfQUaI*jQS$B#R0{D||_AHa|=p-j)6IdbP=s zaGUs%tQ5)gs}Fg(bYF*-D7>p?DsGT7gCfr^1^1kGvqQeq2*9$@y}X-(lOm{m4>M_Y zz7t`|SI+Hx-{tc5^WE0`+Utsq+83R>UMw}?6m0BIR#sLh+b8M=^_ z8eBHZ#N7Ey^J&eGYbG~XB6ScF2?A!q7~L0yde${x^faT8E3|%5Wmc2f zSeZ`xp3R4cbYVQ}jis{xLaYK?F}}w6?>dOfB~~trvR)1ItORwZefJp@>d&ls^#NPK zd4T2-B)X|17$2<6gnve;c|8!SU7K);#q(=Q-ly_&w{`L)1Xtaco={&&M-w&TW6f&g z=z$;QnJU&yv&l^8n*jCo5%Fw&Kh$#(Ly~jb8u8yWbaeD|;ZL@=%m6<~>yO4yl&r#r z_+W95Kx6TgmQF_glN<&`z2CUkh-mH^G=3q|xv*gr7wUSor zD`eNd|AEChpv`z(O&QQZW@La z-%hJv^_PwN*5{(OEaj1bvt-0v%_4s-&T|c=|Cv2y)I`TP1xks>Ra_!oJYm#X;_+!9 zGdYrC%<*EvmqGwB{ySFs#H)5=*am{0K~JJlIiZhO1-uZ-o>B1%{oyp@Fhg!}>Gq!& zr+yYD45`S-US$MaD07tlpR8uPe>CvqsFmf}coUMLg54$Jb3oh3yjO8aw>2v~UR!Zd zK?+x!1ScQ(2I9(UX5q73ybB?iM4Ls_H2&a$c>b#iIUSeAdFu~Wc(5vD>Y|tCCab zyQJJ;kaR8~YV47k4=yQL!pk)GVxF179!^3ahyn*t6EA;jw?cUy=c)>JI zC6u5`+4)Mp+Dh-ZW?`_2=pZppx`5NJrKR2uq^TWWeT5Sb&_;Z*?zZp4G zrri+eR4H@jQj5Z{PIt*gMfMEuj8S~>^H=$KVZ*vDL5?_d%jmBsX3!mytEJQL(pQXAifCbInfN)96!gkYk2ddU-4#dudly0*}?r(;2zF+ z{JlzE_HH+Akn1!0BZf}Z25S4IsCuj?N)qFV58W52V<@AIIKo^ZBqB5e6*75pz%EPx z{7xY5+}~@XD-t70?J zCU2ELX(7MwE6J>K?BAB!?aR%~?GXNTqTo7q7#ajG=8-xl$6~Z8zae1!xjoO!odvL4 zsf8S*k$i7&wY0qsH5Q49Rv@ypnVWk(+lLILwMNjbU~ZG?!&|a_mS&ZsNV%$|fghYy z;B?z|lo>5^SY%;ed541`D)mLjd?2GVSbl;?L)S`6?kJ|8jnA_{d%DPoiI;(+!C>Z~ z>;yk=V#CEgu1HxCmAdF|X>JwUyuHqAjr%*;`>eGj{J)C`AI)kx{~B&g?Vaz%vGKOO zy0}b3zndFo!P8f4h zQIf%1ieo9uwvFR*U}jIY_iYApS=m169=AD12^Ixm*jj@+BRu{Q^;&RkMaCksK8$rHXIT z76VKraw)2Onod=P?k_auEx`Rf-=qL;^^YIh%Z-=d$?3^Do7Xovh{Z(d(F{j?{~Qd0 zNSZ@Z>%!e$`Z9t|klvv|Gm3RhI`i;Z@5Vi}I3*GP1%2I4omiq^H=FdNd3Qu0=pnf~ zB0zAaV+2T%kVH>c9(Trdp7)pB)=14VtJN<+=fKt&@$fMhL$X@IITv7!b{0sR6V(}& z;1U}E$X53o!ZUw$K_PkRz0m?+aPEf9+MP(%@lfaUW1oC%c0~oz9ns)d{HtG+-}~PC z5sx?@UuWDRJ9yo#i=kMO%7A@sD+p8#q<`aP~w=^c(+_pD>Y5mb3soiJ?mz{_-e_5wu zy;@^#=%S&atM<4q2e=@8Q&GD5tt4mtwb@Qqf{60>^oq+?brhZrJDFftmUTj^1*CqA ziV7bG=W)*u=DHpa#M88M&Kd>8h$GxO)O$Ie$r~ya>z8>v6#Y1y9i^gToS02aJe(`% z2V7AA+wCWZr{#s}A3wyDX0d)=jSEu}q@?+mly*44bu=M++%0d==n?3lu~PhcjzIU$N7MQb8raKDqgvSpq`+4}k)sQ!cVM@jI;ZUQ>a+P1d>CCRotSzUAf zF(~n-dqKS~^{+SWaOJS&nm_rU){z5H=|6@g`%VG<0JI_pEjO&H+NG*z5H7Tya=~Yn z#Pi(Y5c~C(!i?{?51A20JK2plJlkO$ZvE%YUUKP!SPtgJ>{3ZM~? zvZ>l|*moZZ(PV;&h6F=dQpKEj_;*CeTb1`H4i7&-Z^o!e_OZipTTtsRSDudyq7&3! zB_w`quW8PAkTq}mVYXCvOh3!#axov1KbBQ1SK?wkKJ0dT(mFXI$WU9G(Cqj-oy)bR zHOF$M={>*abrQkLZu_!@nWX)973*W&)MkovZVp{Up6wU<0g1~}H~8FKWfv-|z=H}D2P#4^CtP>aER+;|v`r&_H8 z3{+$QSjnNHlwCzj>H2_%A21SnzOCb+*R0~+0Qk5#IvRK1z}GByje+EH$}7B$!0ZquBJ8_n(yOw$+Hrslri zJQ-)SU`FwzgkC@Ich^s09N^2oRR$5D8k|zl^QquYvcH~dVdHoH9h!emZz@z>p#5f@etvlln4w_U}kRg-mY(Xj1y)q+#kAIdmVoz zNs2EfWL;0=cKWu`&{k8?p}FKvDrTz%b9s|{)Cgmv~OnM zP+zg>kX`7-3+8{gg6G_KisV|Qa4Xean(QNm8#6J zG>?cfN~12=;xfVu?m__XHAK~r;c?DKP_nThG0GsTJY8V!HC z8ZSEyS%1&1EURY-u#Ab|eo-=Di`mtgZ*^byeZB90#m2=!MH)9C;_3Gg1xMoZ**#4V z1(F%LUnI2HY~k(FpbiM%pED8LZ4JaUb!Xt#E-Wkl(Rw@Ma)$WsqJXvjzm!ccD%i(! z#CzUptm=ownRQcZd)9OIv+vCo$!*E;2?^jFH~?H1Y|YPXNrEIMBz**TmpUzq$Mc@| zwy&(Vdam?|=jLYDRIqfVjb1}626O3H>m83=-E{<6FsSHhLj&dBwEA3PMlW|rN=dza z`!?gXGYa62i)3VjkSKMOzX};S@C->XnMjg~lA+UrWZwd5jOET&vNeQX)is;0$?yK= z0;no0UoD66FSokY%b|i4dcs17;~$_3HKD@^`Iu5_Hbh8_x3jX%a`WkX6Im%%n~O>3yB~tk?STo7bg`03#}&<<9pRPP6I}L=>O3I zmE4jF(GJAtyIQR*68EelXiSQfK8EL&|Au7x zB7NgKleumuX34@Gi2`nW4hqK^4{;&N8(rOd$J0*je5vqXpNevwqKJm1kJ62kT9CZs z=?i4{)>Gp(>~){P;FO}o{lA+U9BrG@B=B&i`6=phn0{?EKw~Bc0x&*OHykP`MvmX{ zO1#;p5Nxd`RP_cb{4HvGsC%hsMChskW!K@KI5NbhpP4G_-7T{K$3W2RB231=1UV51 zAipi3R?ejs>BOfqlY58A7LNWoEdPd?B~g)Xlnx#S@Q&%BO<}hiy%mj-o_a3`s~WmpuH~j9VqlEqznnxSDgtB+G)KS8a`(K@`>wi5L@Kspm*247|YCBkQCbZIjd=L zX$QQYjHR>pFwQ>R{T)Vky%E}U4V}Z8l4H}5*Yl-~ACw$rbY=_RJ&5I+KQH3t_48)w z_3+i~&p+cgNx*6KwqJ7G0v^N?THZFKt$J+JEb7y6Y!V36<$m!AnHFz; z(m{jp@o{Aei~vPX{h^Kd>gnaPpYS=@V6ZCh{q-DX`mRe9Fs#|Rbm^W4eM2e_=Y<2j zYioEmBZvLi0Kfw$#F+~dPwL?X2-?Q`wgQWR0?(m^&xc}SP^r$8>|WS%rD&su79cbt z4ka(GmPPdw9zjphpI5WZO5aA;jW}7hPqg4>Q%RjC8vY{#DN>f+W&&heuvL{l9Dy$FG> z%^!Yy)a_pDzUqzBuf9aRCQFlh7$kkaNh4pQF@==N*C}N7Q07nLwpD$KBM^2#J_m2E z`!A4;9w>q_TJU+EG1RMMT<%Qm#8MviJe3qBMHk6O0|iI#a8MsEA=o3%x<;yCzf+Vk ze^Es5@Q}}qhE7M3E0x@i{U`IEoB#TOAb0Z*A>pm5XR3R?PU6dAbeZIIJP!zctV=%^&)Iyf zGkX!U@%W|6{1LAasIcn}7AKvn8fh9YV)A0d{h$sz#1`1OcRS_pufL(yTx%U#s~s`P4{&znY|qP7tBp>b%~VQYy%A zB_tBC9;Rl4OCF6-Nh#2UqyXn^#o~CsaldO};c<<3;tKufWLbEWc?Snq_m=? zzB&LLAroZ1aTUT9hhJReY2l`zuWPg0e0tKXv4-HUeBCq1+%~f~1e`*?oE-1pou#qc ztbVIQh`DQAt;0&Zb*zo122-UyJ@uUT@-NmlLT<+11j){Rt_@iqD%Gl=O?=#UevY93 z=Kg(wB3aK6EJ+R4v6q@_y0+AHJ7x`Do0+^{)$(ds`Av<%Hp~D0D42i%vfNZ_xAaSy zY1MW!%yRA*PWQOo_vYo&mu4p>Y>SJoi(8wWB0eQ^Zfxqf*}AJ-sJZq1%%mz#oBsUr z35vMA-e%8ZWPfX#8hoqEGE}<7yZn&)Tix$br(Nas{mp-$)GrtJRldVDoBO*b!U9-I%?vXY6h_$4gFmZKDOjWKqgODPtdb4B)hw z(A1Z@*e(2w7zH^v2(I!W@HKkh0Ga8-jK|}R_P4z{BOV#(>@!*%vK$EfY#{*$86uC= z<_+yHL{|g-aT6`vYe4*j$QTccp$@@TPpo-kIVfcZD*)brMFqo+#KVG7RI#O`sgob! z2L(TsAZsxpMPca6;9k>JqQREv5gE)x-%0+;Mj+IWK>tHRZKX`tjf?4v0_dW$gCU~P zgyI!Uosxm99|%PhK>F`Rv0p4^n$yIuYH|K0kMgP(nf|+{_*WhAGvpDFP)lGSgvLRE zU?I6(uv3U*4Y~K!(ErOH5wj>}BSSB`xp4%+zPG?fWGqgiW9or5zVq2GMG@{Ez&@AV zqx%$URlFQ8NzT2vgh)shtzhhph*aB2N$XNVif|P4yU=K(mgP7GbVinK1frZTSZ)O= z3S-yjkZ*Q_7^p!QI(+fzJ2R6+ zg^!@flK#*vzg`kx|04B2Qmd(Kx`^x7#@n=cbuoY3*5C zLZ3*mYPFJ2;v&sywb}L>-gt;k23xpqicG00(w}1h?Oh+w>Q5FWgs-gXNGfoXzgL41 zk}Vmjf&wnE1YxE>epr+nVcW2JT*mel_~L_RY?Hz-L^1#Wq&rEYN4crnjAWihNE$9( zg!E8!Ngl=tlu9=*Itozppv;bQJUz@I@Mr}bj?r?A9Sk8zQcoCsxuaTEow-kwRl9dFpxu)BfV`Pr8H#N)2*4*#3i?P_FX+ln>d zOpAkqh&ULF9VUSQi$F>lNS#^el5WT4R{CfwYhr9-j2S@dLwBr8X*)Y;f8!u);{0k~ z`l|Rj#qb=>wZ5G@18#k&yBEB*IW_t>egb@dzq2cNz?hpvBNpQE|_SJdDd+z&uzxr4AsOqZPwf0)qoRe?xJjxV3 ztDJ_Me$IY|q7DN;a-#Fs)Ng|=>XFV>t*HOik!_BRtPwzE_~^R)lE6XM_#ldcVfun{ zX$6G9&kb?89k&@B@zz`!_BlE`N|ORoK`BTwL9!}1>6-<`&S3{rKj1M(s*rJuGNTt2 zO?(DMe{Fa2n`7lvet97usEGpRT>2Xp1;bCk2%krisdvDL=pZ`RnRF)-6kc#Po~A`wiHS&3#*@sHxAE7^HS zoWGIdy&y3woYS76e~)8=43%QBOMJCX`UZ<(G|dZj#BVgr550B95@+am*k87lUG1ME z;YNup$7@~Pl%vO$oNp&~9~X%o?#@R(`P&(;0OQX;y^lWbx6(KG z9&?FuEKxM*u|YX^JI!^q0)3dSWvpdK&rfVf*ufqbe!dsBPSJ%xSIwlwix(|j>J$zP zFMx4ahW*T{rwfCm0<`Bx_{P?}_tGIHn#6eY>aSQiRD@40{i-7x#m!@tf4h>;_2N~o zM5>bG7SMNn0A3GDQW?x>e5|x_JKE}O%a+bkWIvM39z`KKgUoj;F?P?`zSdI)tf zZuv)Ee42s8-GR&XZhBn)d2kC<<3cq*qq^(fIpA~Cs}uTIk1=C?$$IOxO0Uj01N)}e0h3zds@7{&GMUPL5fa3( z^y~%~Qpib5hv$fjzerj39?XqNKf3L5S~44&%PiGycj^D7jTFU0n3wx}m~voUcX(S{ zU;_z8^CD4{md^5>el5OWA8JgGjGiVh0)aIeW3P7cQlx8qrHWBgodc}cXm{T~*#y)4 zvycv@qSDRI&&T*r?!F-ii|1*@2Psf2?t!}RA9Mh`5brxsA%M_rZtw4Qb0-Xew_>wq^V66+5;qhjge7*z0rCCvSxJg7xU^bmMr$@H$Jf6)0gQaXS^0*5S-V+h(?y~Yn~G9um3+6KHafgCKoBL(lJd9zU4LTK(s->t zoa9Pc#@ZT6&5C|y^nPx2LS|Zk%6%I)=h^n2288-wX~|Kk2}Zjr0i#A@%TRna3DtS$ zZX&8FWsn(hoSY1RvK_TcGN&>X(9&`Bmxlfpqw4>Kv}FDs-RQ0+*fop}N3$f0mlz3z zk8Kgn!V<(H6ad1I0R7F!wqVELsb%zV$NC5&9gimlo9Cr75{(F;F_~{=p0&X6A{GeX z$cTeLXrqNvc<`7O3)p>Fl^;P#QNhFls^;2aKi^rcwuc4p_eJo0C&_ABUYZM8cPvF7 z;L|;n@DC2-QOmgmtQ$b1R@)x`(0f8%i(BDicxueP=D_CAjT8#vykXT8k7Ict)^?tX z-ZPQb^|P0opp!@D1*}mI zK1QOE9T0}n2FBG9=q?d_G{w{XDFd%e$)6$wX*AKDuX#G6!bTva142yDd*y2Bv8IAv zr@I>~AgX$*G7we8svX*31m;Gs(rwmTnM5&1!=`fJk_Zu*XicY3;m`W5bSYe+qu12P zN|xC_vFPLp<9p5U&ZCROeL8XFfVSQfH&3U6}Os9h9yLk)D*1&zPO+I;ZS&tHz=I!UZ z3k&sIB6y#rc}CI2x#hEy9)?^#Y!p!p-C;9Mpj8I=L@X$giFXwgxxBRdIIXtio!P5Z z(-*g5l04rIiRTFUO9B1)_>7Fp-6>vBGz99YFHm97?l^brLikoc^SZme`FW6IBgtu_ z%kQc#jBqG9h~A-cBI3GJL&rl@5-CQz+NjfJ^Bz5!7}4*^>D2G$pwSL)qxo-8(G4dX z8?>oWfO^s6Om9uOuh{$HdV4HYrGnim8E_E#O3DcsSAPbq$LCeD?tg_a0Er(remrbC zRsMgbUlCi>dYb+!sJF7kIa0aRZu$Ik=irL%H1};@QpM9|P(j4hT#!7JLsgyNX{Wi1 zFtMMl`(;g{cZaDHgilWIG}O0}!f7N>+p%iawxsF-v>A$UMTG3;*~PO({0_{Om9Z-t zS=qEJegDKq3~7N0q5ZAAG$@^HnWTokbW7UZ9;b#^rYp^&9ga&0qb^0X0AAwiys_f$ z?r!6G(}A*)n!)vv2jo8FH6)KYz9}Bkg|YIew1P|d*emwoO~Svj%&1=H?u_5hZ*sfk zG&@Mh=m;}5kcRD}OXv04gZ{;0e$2xp;Kj=Sic52Yk))u*1EFOg> z?}z71WHyUvcV2i-6Hp9`d?Ju;~&fXpvORj~S-&P=h^{`)=*_i%(&=;%Tk4eHI59AWJc?4w8S9aPM z4Y1xpYM~-ex7RM*Pixg4xXd$el`8O=5XW`u=sE zh#J7vbXuF9_B>QSex1^?uiba(V@Me_WHr(4G;}@j^opZIOe8Z91K6CFIzg=_vsX6e z9#1l5pN0#uDHp^aW_1Ftf4~#bQHzKS4sSALB#{NAa<(t>lfKk?#4}e~G8tKWa5*rf6^@F9BE5 zir-^hpB*0Pr;k!tg~WU>-A?JDa_nY5zG8b84VMv-6b^m-Cs(=2q_hFzu; z%Y?G)A^|(IaLm_NH|!6;>o0f49gnNt*NSqL&n_5d6NRqLQHOp;{epHxpJ#~DhxIaH5;U4Lm>Yx&ZeTJq(~%}d`+P8zqJ_fX~S zGVIDq1z>&8BofAED3C?WzDvm3tt2ZpinUaV--VoZ3fb-awQGEZvJco;9WES#h2K`0 z?#4KfPUar&zp8N9Zw=V5Ypu8XRQV+I0cgeUomgw1MbDoweI=Pp7k(Pw#|@cqbkaGD zZd*ng-TV}@=gAZfJ$W28yR`c7a^)28^XEo2gk&X{rIr$6=5 zllL1S6WKqqw}ME^_#kvxZ~1IIIqK*DMA4%fPo!g2X`SryrykEWnba?i3W+ZHfLBQqRHs$~V|3BqmL;)zw#>gB) zfT(Y*aW{uc#;erYsKAXAoNrPq9c(7&h& zr5_?Rp>}?zaj=PaOlLr0P#IQ1Mq0E(sRQm$NNsBIuXkCQ87Zo@9to|Rsk3&pLwG}` z4H)=)dEHG8<@z4g4*5pHgYrB-dTnztae6Yzgqgsj331H+nB{xv8ey82+iw{K#GBc% z0mrpN)R$#^)+PDAB5GE6Y?6Wa{ST-K+VmDon?tgriVA>pv8G#dc7-jX2ZLI5iFo>T z89NEntxOnkAex%W3tXl(BLTdz-@!3WR5DmgYf-=DHb@q>pp+@V5hOQmt)U{yGcb){ z1)?)I$WMI}P(yWY(ETi}a?czJ8|=qb=29u>>%f$-X0gW~J4Vkd zFDol^*?&Ozh$(akX!JI3I-P#6(ug%*E^xja>1n*in{YDSo2*0l);mRqY$X34w;KW& zzg=Bl>u>4iMuQ#wzK^Fxl(AMI^-;Cde3b*Z<3;}+0?Fd=g+(HA1okxgZ55F#rLh6R zh|B|yc-qLb{a2B67zI_X5m`2LX;m;Da=7&6;4rDVy^Rir0nk4<0v4aB`Y6 zmczf_u9oxg!o}MUR4Fk?{?IHvGZ$>~`i%wCtJ$4!;D{>`;v83y>%rK#*;sn{(AT(} zBm*6$T@n|fHCd~*H}JxUpjc0!&HkYL!KdrAyZ$sg+^*54AC8RC2Sr1h4ceP;h6RBj z2Y~`|MjSDd5qShmN~_!$W}#H!FZ;uvw5;^TWyGCR^Tu(N4c9aX{Sl#Vu=+N66v-g*31Jgp@)H>e94C(NFQssF>plNNuwDcn(*+b61+I^Wyq z0oi(`XHii041R~20`By^!_Z^J{U|R0z;ooP!DqPK`EYu00C}yk-_`SZIakzYr;#mH z3wgdIiVFoI65GWH7fd!I;Fx8XBc&CXweUX(l6G>ON z!DdVx3I5_6Nl#GqzK8zm>Uv0H1Q2W(wm-@j(`R$lp?jJ{ED6zW*_wof=Ff!6Z!!QP zx4qxj*Vo|Brl-br?5~BQiPH71;X*aNm-W|$>t2>fyI*eIHlF*O{IivceIrd<20Ud7 z-`6T-87Ed2N}j%srE+v~iUn?vi8!GQ%JiaN+!+%>7{&)j2a3hVvn+%M)p{FXX5}3J zji;|a{Sw7RygS}GqNfQs=mC!_eS95AOQX)^2^LKW8;qsdn4a-`{Pm91t{UVu4;IvR z((Sa|o;^=K`x$XVBjk6Fs7`FE7_ZBB-{~YS59@wd%VZIM>ixQLa&+W-S?NdXGvDexuO^xfzFY7*6f{v2H$H^6dIr?GvTq2uUNatF0_ti<5{*$prr{BfNs5}3#siq11F&oDq3{m1;@K~7hw zwd2)UtwreesC>_SK_Q^g=?$|?qg6FBqB)WjqEzB-ioWi=WKZ!SSxiDf3gqD)_#97DE_0K0qckoF7fI!!yfJG1o zsx|-n6Y0M!025AAH4;B7#y$``akWg-ec9{L?6A`G(&_PNVvz?)O6o;=wLQMB8`}Gs zDHtRv*|{c)7hrZ(?6y03lUG<#p|pt=6UzKnNkIv3G#ekz0=`}OicGC2=oze{atc&J z$w>W}q?|U6`)Qh^ZdS^t-L4D=`?EHQ`Uhf4`Vgr7YMsSyF+TvkIojL$I4>cph+iss zFH|Y&=wn%r_$w>YG>h(+QvEN~bvAzOL?6;DcOCzQGAvXZaaLzNZkfGfj;*%-!RN+^ zt-UGQ;P9~1I2Q9t^mBQw+2j#9uONr)bs{btLDGzVhqYutz=n_0DL<>9K{iI!dY#u= z8!Y4L^mB8w{c_96zEzIjofW6Y4UV?Mr*8L^_cYYytzJ(TEi<}4KJQc!e*GtRRb>|B zVilkuT<3DR^D1+Jx>|Vg$7<7`o}wv{%W;5P5o=vGmH0~fhGOq<)?nyOQ6;D zCoGKcwep?=xjB!UrHYf2ozfuTcvg90?;PpBz=@ez+A(VqIZr+!u4_IYQA*vHS{NLA z9HoCxAHCe5q1o;m2>5+>uHrMw!OD~<8z?~qOu<-)1l9Z|aeuKHfqP`u?sdGAojJz* zSzyDwoSk6$cEl`Vgr=fuxyk*Lw7%V@UG;IRZ=>Oy2LDh@fU#fYdd>OoSHupL(LNsC z>ORNeMIih8A6*{G-eb!X@0yd8t({uSOzB4bfgN*!Tkc|073}_g9dXDO=k7cQc6A77 zbmU41bU6Kd_j|p>(Lyk+3Tc^$wYpsDEe1wWjXZxNLpD|y|K6VlXc;a1(wNm1U->Y% z11&RuJSYNkH*ZbBKPxaI(T4E02EM~l)dF#JI4j`OaA1vX{s6(nnWuWYIK7_qQ3l0W z6HOZ$CK5!)16NP*Ud|+)cpNWS7tBGQ&GE|V^l3Aul2W^(vQCU3#E-PZLZ5fn`{Ov}w)h<8~DvIf2O{V-yqTFrq@@ZvAUKi6k`dQ}B2Hy;ImC=)Kbu*b=G@IGORXieeua&qLwUWNQ!#K}^iL{ZTB&$%>vZq$D}zj> zBxHUwX{*d+8ysIQ%KQX=%H3Tj4}V%t{w~SAT)QJ+s}rEW zbgdV<`eUC@oOUPDovIC&+rEs1C8FBlER+(^ib<=g%K)qHdeeL{Un+CiIHk^iVpU^- zvpU!dVjOi`@8I`;Y|+Z3Zqsbe)TEbd@@QI^GtlBE;m%b0%j|xgUv%Y=3#N}>(kRIi zevs^_K7M()3E-C_OfJo+T`ixw-}%RIG^SgQsKfXdYo^JnZBI4`l8LSx!=rLL#-pyrKt#z;S3 z>pVW)Ldgz8rA{Lne~g|VOVpwX$s|O?&NyEVjo?ip%am#S zUS?|vUw`z&6@cJ+-|cp5nuZNYyyZR%co};ko-0)A_46Sdg;be6C55+f5rFQf@k(HGmNQr-f4p9?>!$jU|>T$dH zZuh$|8D37X!L;#-O=(bQ!}wXl`J{80a`^lK2W|Py=)=K#uCvNnegVMk2{=&3WJn@L zAJ4Dygyo=~{^ZPw%Tpsmw~Nt2c|ul2`sk|#-Gr2Ufm;ms*wTNLCM{-zwoN?d{D>xV zL13x=H@Y>n)s^S4BGFr&}@U)#5j%Y(S6l-xOG*&NhCbU8E{N2f!p8 zdYaVI0Y*-I1m){B%;`!3s~urAEHox4luKs{%%DD0301mqnV|2&kjirsPD7D5y-$8? zCU#KGpOrENHF?}Pn2>KvkD`h>S_N2iCR%hgk?`;YY0LBs;w+0LHJ_94@RT?bRa5)~ zH-(Sk@i6>`;lo*HGx8!8myFpax028md448EsCsPCYG4%xML2SKpgYwft>GI>Vrg$h zOTyDJhoNf|7#1TpL0FI(W|f%u<06ApXpN$aXjQOLf;!U&}#0#&7r# z!Ic$_%LSux&{@`8B2s66zf6b74M=1<@)y>aELSWg7EhhnR-K;icjZtisegV>C`u+Q zxpr^(a(#VAr1i%hUYfTf?jv9s$S0YRkuj>k)+Bxp)>)Z)@Imn<_~v!1H^{5cCDpb^aSVCIV=w3c*r zQIqrj%y<2u8=qdXd1^EpoAO7S7MJ7Z_Aj{o8sBtJv=YkPQlMVswhnTRwA|m8crEQeb^3ntf;M_@ z-p}cLIX*2!A?)Tfpt7A5B2e|Z~=Fnmhzy*a=5b5W8f{2ExgpNc$<*H=GYdz5X(rF+XbVxYg!(k*s%t+#OU z?!M{ptPImc>!L%aYQ!U(y7`O6oajcvUeW3Lk*-3jM%X1e;Z6`xRrv=vsnGB1e8^g# zcA=AFBaB8%{z7@8Xfn9UL}O%wQ=S>*OMGZo*uXAl&&T?-_NvU&b-v?Hr)oCRb~f#< z-s>fps3wk_V8Y{X@miFL)821 zRj7%k8!otVqA&VIEtU9WglZ+U1^5!@R zqs9SFTz1U#wNl2&vpeH&ch|Ao7nVLm%tov>yIE48$u#@uQMDpX>taDMplS|>OJ|Xt=6}%?+I~v zcA3XxdM2`TgkMb&=Wp&?_XB8UriN{mVLN{GkJ|+6w9^!oNd^{_{h*%p#NBGg^#+b? z0l@=11{4bmVk_~sw?aQ|o@I5KG{yfYZtYx+S#Y21$FcimK)Kh(gtz>|6U(Qz zC9nIx`?5`r$1256>$V@yf6YkqmV~lNm_Sr5?`bW>Ev9nc)w_+2LYO@kj#Tm()w!>n z3?EM z-R2x5OXEtal>)vY!DD^ zZfeM^`&(Ho0~ZKaMmQ83x6$p=H|P+nv}bLzTJJM8SMIJ*Bsm}oqVsH$4~_^*CI^xt zmIKayn8luSc(*$gW%V}Qz8 zdw3pkV`vsUPm7}u1s#`J1(p`(7U!4ol~|n3X)@@dkX*bDzFx#Eo5047)G#IJC+MBwMa+Cy1oJ^`zYmb%+B>N zT@$&1pL}veIZWC>5vNQ59wzk6Nc7RL<%;A`I3C0S0&_Y_xNq>~W;qfbE?X&4>5_ksdvdaYA z^d_dh$$IC1ku{fAWUO^LGa;h}xyZj3V@O$VF3ij8@iY7fs^G-^X*JHu!b1L(RXjMT z36ndn_p0Ch4{6dh^!OFZc;7+ZRZA&NPmcL%7R@8dSL{yPqfL1t+yCT5@Lh|z3zRCC z?7V8xPUq|zL9JXqezV;xNBL5*xN~UUhV@9xS zK8z$BlEU5!NVBePuEA#U*p!0B1S8>tBql0|KJHRgFjGme-LEFB zu$bQkk1)_-F@qXsOjWrm}9?MP$n3g;LKKLoZbvp^lkT<%$Cc?9|Gf7j?>#D3g<39?vUzCa z0z&)Rf_9MX544X4!iXALQs>;UAQIenjt=)a-M1D*EY^DbSf5AdzRGs5{##W&gD!QhDRNMu`Q_!ASRxFf#WLi>6vZ;^cFjWYDv{EK z>)@HAnt&XiC45L|tDzccrouXGWfa69%^h`T={K(?z#wgm{Z7nQbM}lZKk&#}maMV9 z{n;L5cvbTYsI^t&6C>8_DAQiB9`so5IT9UM^*LkVa9+iH1JY@Hz3VTz4YcAI(lh1s zXvwyzWT!Xb&_)wbfFhXFb2Y5@v?70S&WwC3_k(|6PI=}(^p0;ldJ{E|`8i$7`~BAv z)@e0d<7{`p^o50%s4WN%4n0Yd{udA@YT`9~8j{4orc(h~>{1cC2T8BX@h?D2jWbgy zP59@7df<9tC<=IhQAt?4y?~N_vu2Sxl<>8<=2H|CRmA33No|=&Opef-UGZn}k>~M$ z{!QPHZ{ef}e?v1$t&mZaQ6jBsZD-GEyb>8GY&|*_7 zFyz_4iL!*TW7EO-GCD|}5vpM*l8wx0@hoMYTk)?BQ#e>+XOh3toZk19+f3+-5eJfu z17wN4jRR6J&+mX@AGwIWb7Y2xYyqoDD)U}OgSyOrwn1cJ!A7ZC?w+F~{}0@A@R8DgbTVixRlHiG;-z#sc{Uv&CCfi>QkUA_!h1 zCLF(q!HN`9yRPPQJas7Tg9i`_M_~%)5?IV2Ib=Kuq0+4XG?hK#mzS4)M3Me|DUiMw zSgSg{`&6D9j4B}GhlswaLPY%?K?S|g z6v)0{LYh{`G#e&BFT<_D2*K_l2>nH<%QrZfDQ@Mxmi(V4R!4lpG;O~;I*GZ&Q6C9j86X*29cj;G)6}L=UitANYP|unCSPN#BbO! zqC;M9s>$Zq$J(lZbr2)iJ=S~^Zu zdRO1K(3z61H{N;P?(HwXT4}EIcb*Q7RIjmVS@JwLJ)S92v>MN`_<3GWLnZFI1yN>W zF{S1-Xt(_d(?X)E6js-8+dK8YU1&_@q!IFDM1CdcauMr)782sr^`-I}L6{d0z<*1&_m=d+z> zKI!F$8I}vn$#n4_OOGdq{7s+l8Dd2)78&MbaTcgVT;y;cOBY;7Ec0?N<>)n5MG z)yv|(LiKc}Wg&tOHI_uUVy_L>9VoRRl>_dH6MS;Pr@Vvc|IMywW@`5!>k4U6j0Xp$ zC1esxf>S*I58Ci5VI{=Y{nKl@$_TftPszDpcr=i7vfe|}eY-XOkvD^r-{bRVcaH|v zJsG{RG@#nDRg*pFAZv)($ry zv$3`}bbmQoa@+Y->VOhTH_{W-)Fe<|0U6A%D8%^sZ6>`g5YecRHhof>5Fp-sf?<~3 zbJz!bXRM2vEtLm-xXCX$7tnaZGL z@4u7oAUSBnU=HPbQd`T-eiUQ+8P%oVK|w9MF6^R*y1xAY*OM+e;jIqua&paLEwcT9#Z!nC4CCxI zT(>p%!Qz>i^@J0#{-$d|VIC+cWh&O}4!px-R(=uvi*TQpxN%C;BX*CgWYolo@SI2! zBsP?Y!G%BmzUjs-Usf!|E5o@nW;5w6JdX!;8@ZKsc8aR;lzDJzrFgwlAsa=}AYPI$ zgK0^?({a)EFF7-ZH{RybV z!Fm^=^nogc&WW)ujJn%FWofEM3Bgz{Jz@5gzMn-0&u*I+Im0 zoip=)#h^imd zSia-$Td7Mz1Ib7Nzn8-3;GyQGUx^t1L%tRWfNW&~zbk5B#o2QrY=Y^BtOksYoTR10 z#!NnMC!8%s(r2c#45P;goK^P}LkIl~M&WQ)-Hf8oTSrWGPESH3 zAmY>*y`DUSqQN9ddrp)fA)Bxq1c9{wxKjK2n|GOOUOhjYj$s@k71X5PNB#3J^{GMd zZ0+nWaab&1-y>G*KiAt+#YX}$NEYD2)d06CT#IkN4CvJ@Lc5FXzDOl3Y^^ug(TP-u!zQ=Y6X0PeU1&=7fu@ge;@AGR~4;p zadhpms&k)&H{XPS7wX-NRGYJN&fwNM6tfu ze+!Y@%W%5k(C$^1OUtRU%F0)Mv0kSh!|7b~wDm@vsUdf>dQ-da^K8H9+|d_KBEKCb z&8$_V^l~%@FU9JgSaIECt)fM~w``j*6Y%lbE4=zfDF3LcKu z!#3-&#OdRuwyQN``mb+U@fy$ieTHU*zmk4iJ%u#bbpEo^6ClSahfGj$dN=)38r-TA*n@bu!N0p+ zFWQ<5vZo&>PdpI^L@g1+-;5`M#z4OjJ-RC{OB9n$cZ@?G$p_bw97w1-Na>^l$`lj? zG;kR}jz)}943)-R!Z-%h;6Bdy=@YY@NBGd>XGk8yn`Mxq-Wf7cmhh;RZIW*@KFzIa zF+N;{i25cy!khet7$G9(rft>X(UI^m(P;E??JB%0@wPLlt>HYGtb(A9(>KqDecTWC z6gzS;JniGJpT0i&{#&C>rQBhXME%vSVJPqR=kVEzNeXRegs;JQ>Yzy~CPe+L zJ%m__CiT ziO8`3>+>1V^BUBMrJM;);?@dF2!@Ws=5Qx71=(5zR=x%PBr<5YeYQzt_RQEar76ln z0?f$BNG0p9sJa(p%m2Lrh=?ZmSiSS*V(hN6-C`M_4aOznNjrQZMSD|;w5k2B+?Xrk zVIuF+qJ~|Y8K8sqRL@q()2VE47(7`4OSJn(w<%2ygWuVSc&w^vTo)iO%=UgE@gdGXQVlgqvM3z>E9)rrjA7(j~_uV$6 zK{zyCHA_AK%dL^erT~c!K1g~A+*vJlbyeP&s;3tt0i^|3P=vE%mr2pWp&)1}qJRSv z7L_8YSbnO$rDQ3SD6OyWtnL;h`(8t=3NH~pY`*Zu&a|>W9}nSuRHRgp@x8**HsF&^ zkQN1-5d4la?*Q`MU-xeKm(dePjdaA;Z0zdtUzBA3)KWwHVKc_n?j#ss)lu-^kVP*i zoc4#8F@QWUyXubKANcQ$Wr%sK3wa}qib9osNt}dN%GD4fcR!-8=n;n6(^klTMtuut zM&wRB>lhd(=7mQ;m?L%Pm|nhua?6V&!10lkom_uVFDSa%BMZ(P99G3p|CM}cF%8vB zmP?Lu2KQ7(yQ4neS|%ujrfIeL-1~a|$CN$!SH!)XwOPLmGMCK-95|kFW24?GHaJg> zsg4MuoI)$(7i1+&KZ?a~OD+Z%gvt+8Fj%?9ERY%oNJ0^W$+A{S;S4^UB8UgQZ@0!e z+o}fv7o$L-Mn*#_P=+Jj7)yrZy~tH1g)`E@#|!paIINBDQX?gxSx_0wTlIS(3kD>D z5l#}xID}6!I3CRlVRAD5H=%E>=jN4QGVK|WIg0`-ihQsW5!?f$6jnW$@R}A8eIA~I zX%ZKzu5GEPbS8Sp`wMiX*HqZ{@Fxjq);?%$MQVXi`*O7saZ0^N{G<}|=18Xn|X;i?qlfi82U z3%|(au%@1@#JnNym69&#RHjh;=k>;mbD>{G@GN+tLa*vP2G|!D|7Nl7$KQ!DFN`)? z7Y-lUy3bh^sE~mF4re+`0PabN@@;dcCB(e^Au^)3pKdis|e5}VSi3PYKIjqolFP1*KoaWq!-@*kFd=Y!z zN*KTlxTAT=k<&BLYc-v7Gb(!?33%E~^13;_9o2cAB%K%{3e2nDdY&1JOc*+p6s zJ^{|3FP<}@P(OoXSzFu5TyZBR9#u_XN-=HPR z6GJgfzlw#W4s_x}iE7|jPM5DR-~!*P#N`FNA2Crz0H}D`A17p_%QTY zdI$yJKRYs>@y?^Bd3#L|cV%%4n^HjJ@~_Uzp_O>2UVz!$GIru?%?7^aGH$mW&I5Y~ z@}onPyPntak~bV#G^dC66Z$^#rOoZK!{t@$4Oe*+uW@JqUhQLt@7zMp)1-cNho(>C z`mPJ_?s%+6^GU!ZgeJSeq%Yhm_x>Q@>2Hy~%XpTkM@>Djy_2=yp9M^`eJzG|3ixGl z=CX8ajMtkcGW9Yjr;Kdr(<(E|=PQjg>Q5zPeHPjdqKyJh-gV!Ns60G)lZ zWcr9s={LLdB~FRll)pTF6cO9AIXK!+e7zx#?Hu5gtICLm+4gCp#{0O9g>Z;pu#q)+ zDzp1rK_koh%DQX$=PA4P&GW6^R^F~UKLc6Wl@VM{%KQhCNt&bM$BA`8c_7DJ|921) zp4Qm6Vtv8uj%#lbY9k?!d7Y-}!z25EmvIYI6lhRaz{S&KRTq<@j>z@ZF6b3tFH<>i zJ6@UBz_Qc^$da@1XdNQx1Lo|3YP8uGK_nc}+# zBYpEz2Exp@+1Zr~)pH*$zO%8Fm+G}&9H9=y(7a2K*-|%K0J4$=v?~E4TpFa1vgXR; z!N##p5uh3V((U&GC9f~uXq#6t_iop7nm5=NSYz7W%<OZB_s z52a0=g&lyfNb7YB)mx99_hAdqh8uvZckIqYw%_?C4)gxrL5u$}47a+-<5$GS%i%{} z%TL{3;+gmtHx}mSi%sJae|nb}`EkYT0d5{XZf3w47;v2~qoP*EPR2AfRfz>Htoynx z{V8*^oD#YEO{`^E_v-)7s*IBONg;x~Up{`dFxcg51G zlY=si5?{|$$Nv*F0Y?D{n(`_R2{UziwLgS|-%_Y4pa14rs~6|-Dk zqx;DqQ71}a%Y;KT;qoPlO$sG@Gt3Lo6fn*cosPUJmAujMze1j0jymrWwH6w8?+om6 zeVjhTjRSTLbU}C_WQp98Ar>P|=JK9NHQQDmT4C?*Irn0!XxcV#*YIDRx`VZ(H3xhQ zsz(j3FYQt}^brxF<)j!j5E3K^-hYVO5=r>^YRa%YU~zZ7Ls*i2MXK*Pl-~VvOW5b_ z%_p0HQ@su8AbLK$c|2paFqHiG9t_DS1vCNyj4%EmENciNXE%lb3_Z-v038Wq*C!vN zhRKYDHW`i_0g0Q5Y>TL7+6Y}jBF~x`YtwNJD~f*+Gl31lQeh9R7}NSZ$%!Tn z-yfDn3?~f#CX{Z5ER@eYG}yD$d}Ql5$PJEN0U4f-v=k|W$vN$7_x-3M@t4KQC|cuC z8ID;37DA$)NCnx0Djm;$lZ5!M`SO;icDhMP!iIP$x-9*B5T@XgI2M^53kLr9stA!z#_srx&jt z9`kl)9M|Pw!A27f4*ZlvL+aP$?8@kNr7+fcQzhiH)Boq4Wh(P{$<>4~tC2sD_~>zL zC=>uMUp#Jp5H9;^vv7U_f0X3J=1W468U_#_Y1nBu)O&|1=W4;MAt zwsGpV+g?ooQX>8$Dp_g+b8JnZkKWvMH{leX1g9FQDScOwqj(kVex-8qpw>6=ASUG3 z(O6_4sU02m|55c8Ky9__+Hjyi(H3oSg1cLBcXxMpFRlfOySqzpcXu!DQruIZxczy* zbKbqrpJ9fXz$7zEp66bd5+fpV-kzEDUS4_d5<)}=S047+Clcs#iR)UM=lA|z%p>uZ zyN*KSbR5;!1HLnuNaCR;0;})3CC}seB0t%}z?(OJN*3=`r|C#pHnRyNwmb{pq*+^C zFWPTEgR8ZfI!Lndy?PD@$5g)R`CjmO-|==@Fp6$}bKO}u`Md<_=@>2CcI79{_3?P> zB4s!Fc7i<;HFfeyh_~7MD5ffNiClVIyQ6w0M{DUw+;TGZULg0xir4ix&bjvcX6SA_ zzJMDVpzdS z7}*loSnwE%&WF6c)^tBpUJ0CXYi?P@=UZD@k&HlPMwGpUOe`FBm04U3Ee#i4EuZx5 z?59Z326w&PFIVcVL_9kyD=N$mCszhLJY@7Q4Zio=ws$o6!k7^cK?P&Fl&8y|8ht@S zPz#e3W6vtl!X`cBwM@?IY^hB$Qn%Pe#gY)ZGL;`lp zIdm9$URFbjP$;Hin6^?{FFT&}<>r8cfmJ7Bt0d19aw)wSLR5TMVL*K#i)den!a&dg zFSY9VP~=xmU&tX`b-CWf8VW!D$r-vz<=u~=ifaW~u zSk%IbO8cYEZ;7hF)Ztp)=wfb)sD)Y49=-Lr!Hr&0H=6aq&8WCO(hUK9* zn@nf_+;($*CSVb&HPcya$$5M0fi!1)`h*;esSy!^sl((j?M}4*Je$7uAvHr9-RxvP zuL3A;J(I)bQr%235T0XHhi}krZ06N^uTay4O0vY2HWmP5%f?32-_?c#FuE~djh{3VCKpiN@mVmpUX$T`-;$#Ls>=rt`4tz zNJQ?IZ$%Sjhp)sUs{-F+#rBJf^S70z`+Pn;FqhWN)?#xX54h8%Z<-zBHm~M~u&5{t zu3}=UJaFH(BP(Cck02}w7`VSyt$!wW3PRzz)TujPC|BXaOEAjD?kSK(T{_?Cz3q$Y zGFk0PJ=H<6ojk55hxlH+RnqC&EISLX;*f}o3PvJ|s6g=2iWGvH7o&y-6p0mCtQ|FX zl^{gsr2M+xLFOft&89pMmrUK9GazXO-7F-5jBRP<%X!WQZze(N16Y<>7FF@SKAoj|6^+g$>O&B@d(Y7-(0skYB{F@D%td z>brisSaGQYbY@qkz@kW^kWQ0>BvISgRMgf<#-%{_7mdZ|UtY>rFfXPdq^UkVbwGr~ zs@OYe;g#{K^k@y^$t9tmr?U{)9~#?Aoe$4Ja-SCrI^)Dufl)vJBr7Sc62uaRPk4HA zB2~yXbb!vNRIAhCp|x5Ptuvj?udH*UA3>9;qkW{xUuX6l6IxM26O0D2-LtW?2k%S= z?lb0W_gKyrOpZ#TM*hUah)x~ZoAjxwYU*m5YFiB0mH#10`4iGa&!U>L>m05D;Qh6} zxSjoT=_{zAc^MKN$vu2(8R7&;R;`YVG)2dgNy2%}`?T z@Q_xH+zjiC!=78qRSVqG%ja#K_P^6_hrc1V9e>B`3l)jjqO$t7ri(YV zNYN7;&2PGDD0H;O#aJ?z+VT15#BgNEVloa9Sj2T|!k94ob06#|vbp{1-um=92J)Lj zpC-_)FmxLOLNWsSYsdB-FFqdKpKqnh|Ad58-dFnWREr@QJilggF$V+;aw%g;F~}Nk zI;HAcOD1cg0)8J$vdKmx`+8jN>G3q%e3{b73~Ot&Jsde=)QXr^%9r#ZhKj?wIKDJ! zv(}qP{)A`VkRE?{_@x36%$_M_W27(VmWtVT+XqFv((U{F@#s|8W@3E2=JO{Y`t*jW zmKv5i5<99C99*@|qTRZ?pJVm3wvMuTPPgCYK`V8TR>KjaL3fs%&*m_F z*vgr5#gGOT4*%Z-C|!_&qj{2TSPXZrYkpr73Brv@UH( z0}_KdmVd3a8Erm6JUUZF)b-bBmP|>LW^_Xn=DZK1q&nZv;l5?czaG6`D||iE>Fxqk zRxtW{9WGqOqd3a~m1w3Ky+3H5Y>f43Nh9o?S`wq#wWppi8T%PG)c=${1fQL9nZSNca$vTsf-T z-{GfmK^k$9ph*MQ`&hp_hb-)ojI;BT=hE%YQES-Yj8;HESUrg$R{o!P1%b_OHF47ftfC-FBnZ^E5$%yneUOV?SkTp76M$ zh$u34u=pa2v$TY@JRiSJpL_1A1?xZ)rFBMTW;nDOagg<8q5^;TAv+s7WF*%Zp>@Dt z)#Z8%KJd^C%ut*0uFGeVFCx>2gkPeCqs2(ck^kLIBBjkXx9$XuD01{55^)ZX!&Y}`j%8)f@rG26k53Jz zbw%`c0yD7G#s-fe$080saY~rWQCYI?^F8*hG+CT}wXN#6Foce0W29^X)9N!G3m7JFDrKgYTs^o)5Jf0xYAWS# z+Yl5rZTRuw)!*l&m?w3!sl)p0M&SAJBV|nY?ewg!(w-P!K^(p2L%A3nXNcE#P=PcF z%OOKWJ7*JpLRV#z1EF6-@s;x@1OQxmpD8zodmE@8_j^BS_~C^!6D0T{1ipH=YpU~a zKiB}WC!wM(pbNwPOnSI*@WV>PHxx{ej(vH4|C1{T3omKspm4PP{<=NFesG;xIrVzD z+|#5ej2ime|9Q%R^qwe{6j6z`RDL0c-)W=F%oGZKS2~#KdFK71=gXQ@Y2y3k_PYd> zVEFdilZMk2AXGI2_Y{(2e?}Sfz?f{ifk5!@WQsbih+#UJJU$P9AS0BC z;$? zgN^YRwV!E3GJyKsp0vnm4CU4h`Ibwez(Y%GOi=ynX4k3k0DIvV8s4P~*dk&aYbdt|{dOdVF_SLFNeMe|ln z)X|H3K&TKA-)z3+kJ+#+y`m7K>8Su`i$J5z^AEhGEM8A%vai}U7{%nXX{0!1c|KBo zLU1Oqq)!To%0fAab z@sxW%kdC?DRbx%yo-zgr_r`$|B~_6IuzhbZF$-#RH>rsKyx%#zTTmF&2N(7YnIZ)B zysr05ebhvt4o)MbrDey3R)kS9V<`*BUVsA_LIGl6#HmPK8H2HLe>OxCZpQ$@ReRC_ zTy_h3c-bFP=}%?D+{Q%TkZ!py^$c{3A(lOJ`AG)HriyZ8i@s>GjfT zI+pZ~*HY^EV!9#_tf{mMcK48}g7YHs94U^7e)gx&j!LOf;*BljI|>%n(nKSc8PFPh z|9>FGKUa>;kSen%rf+KQJIW|{2NQrH*XcN?iRiYTQ1CivhZU|mlgo>n@Vj6Z3AKz~ z1y;&QAGb}EVV5dooGTNvS$rHYlf`v&iB;jv(-CIY_rg=Z`fDw#skyMi;Q-{eoU!Io zFoq0Qb02oxOt6l`PnsSGr5183$*tRI`q&Ta;_c;?aN;UIhvUr)O%loFRdx(U~j;q<_+q?Hg?_?;D}@! z4FjuW-L$cN?X_w#A@*@VQjy^Uo*Yu&ITsQneQlbe*?#Yk=j$}z!&YD=0DWoE;OX*5 zW{ZZv_3P)tf3*M`D;#Vt<{`{*0V}!+8p|IlmYZ}tGMag_4%%a&zWp@ZwPktyo!$_8 zxsf7Unz{B7Y|!Afx;y0W_4~7W$XHkc;>4ajd&E7 zC2h%-#mzdS#Wfp^nxc@O^&7u9V9-fO4FlBZ`?%%$lmWfMsoTCYB~vA_&A{$W z>$)#}ZKR??xUaY*Mw70LDsmr$CyMMVb|yade%12s{X5n8Ruf(|&IEBzVKxzV2ad~Y zum22Eei2cFyB_m}fsUVF%s_Vo+R|qNV}{z^Zo{<%dnPSjfn$|AR@QAXI(EBtyIkU0 z%Qy^K^5&}ezst~BK!tIM$~iCst3a4#P1S&YUbEtCMv5%L=Q}|3JD%Z=#0;-V_>Hw9 z8y6IISh=uE;SY6YT-*eFSmLNab{ZT+=P(2=Bs}Fo=?QZ+EH2+pzdy;Q+-62wJm1@W zmO4xCd>JPx6bJ>BTweMEIkzgRs|npGt!#lQSt}Wd7CAn{E?&rGSI?yvt ztQ|Zr7E@KyP2v5(E9KW(H{H*Z{FzcqgaWf`UW<=aVI9*-v z%YqIE(}(W}L#LF0hNLGdBQ{jB>S`COTnz;feCZ%nzucxBq`JFHffzFcM=V6=B*}c= ztg<_I9RadiKbp!e1nCjbfBe+VoX{L`9$qN%)#q&1;bPsPfpkPZr)zG>M3og&69&;7 zmCaN#2Kskv!~Hb1-FjVv&s~4_+ndNogAT{KPukFcagvpK-w{Y%5uBIz7Wc!*O)4%O zxnw_vM66`}==#Am^0Yma1#!`Ry&==;q1@ElgPJje_d~(T@fVw1B8L!1s+B%2xjP+^vR&x zbZOMTpi`OjD}-WpJ7~le2dM65K*tp;#|@H;2!<(>*Or7+{R~F`%H=DF+G45;km>6X zMKh(&AB1&BH><~OFby_AVL+D^QGp0)$jrqdN0vQpkiK`v7H%L*i@q zldKZr;ZcVkfQao**RWcl@X%j-KJ?0sl(pg&mPQd+_dk`+;15lC*Pf7H4 z4``xixDkaNH_8sj1t1S|Qi4AiaZq71|t?w{lF0gAF} zDCJ>P(29zhq7=YuDy81^6LI}r1}k|q)OZ^MV#MaGfC*Y9X86r$Oh+T7>m0`5QRMY8 z2`Xn+BQey&dB+9M;$t#>%E6t^kci^eFx;a9S!Ord8sn@RBpf7?v7WkN?j9KQAWTdQ zbZT4#B|n_-n>w0)SQ+Yw=$Hs}=&_l#gGyKkYy)YlM4>}%!K{TIWcbYhv|%HIpus~0 zALg>)%R6$|k!U`0X`bCMOipXh3I+*+Py$p9q2+st+){whO(3Uq4PN|J(=7RGJ>E+GVo zo(70hq9R0pdHjGynN&v~K}gfN+%qj4t|0UT+PyqaAeRB$+X_;@=feCu1&rJz?@`Q( z43Q`Wd((4Tx)}e+RcT*!P5buSi2-lVk6zv=Q~n435P{q`N)S~MMn_sur#)v%-Lkyj zwA@FnZ=(@PKd*We6I8Dv%B{(?u?Tvs2GLme zE#YQh4-Z@D8nK+zduZSe!PNvzXG-P64uSrZWc%5)Jl#ET?|nV0_!&2$6Pa3c&biVhM*l-Phl{D}2 z89kCa8_0?2Hx-)>WggO(WD%pHLvhFDS&)vNgc-)0p0+XGiYCuq!|c+Tk1Z%&lxU^g zJ9s`wt)1XhXU)yb`X>}~``*$l$@296x2b!TCGzzB^B1wWIxt`C(ZqOCuGPOwY3$M2 zRFStSF=11Ex-rHp?6_PSjr73{0y_04_zZLSIkI*vG(_faiDS%r(^;x3nN)s8Yb2v` zL)8B{<^L=hIZ|!(C+1OCV`zo^gCToL2#u0W%&o8#rn_UXx7EACHnn4s2&Nhj!(1OU)N_5i=vs&W63tbMMk9aqko-=tCj6^kJEQz8t% zEHjAN-E@+G?$idCHDZn0x(||e^B+hwc7!#49bc+Qi2^G;rkn77rd8%`O|47KPAHLP z0Xp+~-_oO>dyDBMr64y#!CZ_fMB~bp!tcW~8IjKZe~327r&-Y07dkajJ)D5w1qDM` zbTEjMQ}rKyRNP+fS{95OOs}M0JF%IhIhwMOSXqCt(h$W>h#fR4eWuFnUm4aA{{Z2l z;mBZW#VLL({V;jXU{khgye~*3r@>JGn!y6U*zaSUmY=Bh0nrM9-Z-NDE<&QO=T`vfw|8SURo`YI(Kx z;vyNt?VkVVfbTteU2}<{M@JVrEq1_<`@ifFU8jgJJej=AfWh8y0Ri!4EAM;YBd<&Y zoSO79PDr>FBl}~(aSA!1a znl>$UkVlkX{riZoFW*Of-=ATGE43vxHAgR$xw7Ww(VfFzZ%IUV1`Bu&Q}^N8x!0Y6 z#6LV%--zCSRq04$q4`2KwfU>K|j@f)ME`E#u+^%bK+9+CS42zXqN zLH8w`YI<@*;qt!h%bNtXh7l@pTNpLajOjtkadzMdr zLNiW*t}um%Ay1Wt6Z{~<489;^S~i>~1aC41TNNxmh>m0o_?SeFRY(FKOeSqN zF)-vd2>9kt6bDpP1Ck{OHdGP|Nf8?z?o`wytw9M-l9*E)*4L)Qk=FuCgi}_cVXULY zNtJ@)f5n-h7+sCi)=)t&D#z8q)4@tlTH_D{EH@JBFykU-*)YN>k_>?$+dkr`UO{+zhZu`SHPrZ~8&zQ;& zwBBllwsce;9be6;SJ%q+vS*oF6tmN?pVXXlgjyw68+)z;#qdYP#j8~e9{ z)B8V3sI3nAU2nLgU(){kL@)>JZh6X&wz2q*5bE2*g6K=XcNO|2V!L}?e(-46lh+`(5tFW}==D5RQd2#Uo z6&qu{!(s*k(pt=<8I9h#Hh0}j^T^3*NQt36WLy~_mQ=uFo_zDa3UcRc{@FuLvG^@Q z0K@)QGzk4wtD&RMSf0CdDN$j=;B@)3}InQYRyhtty8ZNo{u*DE1e)aojc!qp7*~M z?-|tTH!CKwdi=iokkj{L?mA!9QVoX**UuGF@428A@G~?N6GA^v1g08DQ6$o)Y(5@u zRdu~~HCdcV3XO+2TZxgHm!acTgr@%w89hIiQAls6cau_?)nkGrh=?A6fJXfPBcnMP zvtpBzJ{Josr2@bBxt83iUi?%f#FY*T^48c$#=U&sJgZ6gwQ~!lR7mz%$*H~);CQ_- zcDDQztkrdysc>GXd%72y$|8%nCtEK>x4&CTM;g2mJDu~ovoK4S{5^Js4myw-H$mG5 zo#*?dm+R5f9=k+!qh62yMTW&4-|Gk@{#04yBj!X313vDt=gqGQA=K?oyY-gbr#JB1 z3DHNmYKwHJJDo{J>OI(?SxTo)ng}O_vtY`WZ(f$@n@kZa1-m!dzdR#JW1g_$%lVt_ z*LG15Da5l|zw@j7O$2>WcQd9_bDGqS$|2+|7_@W`srbptzO&Q9(!uPXwAV`TY9&`sVs)n67L)8+E!)msx@)R|M?b z^a7YjVAv2e2SC*1%A?k36U17&C7`5|IPdX4d?}h*KoxqDR!AN>Ir(HsQ`2JfAc>GR zBK!h{L}+^lPt~w$5YSNh4*3Mwce5M3C6O!o6ASm7G)Cu*^cZCe2lSJxXSC7LRXH^WiI6hEr3M z-Bu+fhr_!5vMT_!aBhBkePIDJi07k9&%>BYB)HXN-%CN>ByHhh{U*fX6g6R{d^(#I z^Vv3@PDPI;d+E)~O8s&G1Qd>*S$6sU*pg`+GWrwkKl@?Mi3^V?33-%x5EY;zg1VcN zsHKY-)@t^X;v$vu0IAljrj^)H^3u~&C}sD(?Le`Mg#v+%&6%onURg=98l3TXqw&!7 z<{|`K%belRFlQ-`kAL|kYUTP)2b8J%N#yBs-%Kw~o?Q1)|79Rbu%nv29K?!^PIBa} z%)4-TcQ@SrM0x)Yv*-K2%$}s;mByPi%m-A-6)pm5B<4WOb73q^BUO?fYe)IA;x6Au zkSF=o3KN33dq1}nRdDN^k;xagyUFxovv8ZKDpdAi?MxfnGcUu7_Jo84TM9WP185XD ziF&Tcgt>2hEWx*5UIKVqLUb0ODR}KL3JfW_%M0#%8w(xCKvW|E#!qes6n7QUVlK~- zl7uQ#b!o;wOZsi@4VAZM{B)C+FhxvcFU#0Zl?SzbKAB`Rz1Sx?rwYu{{uk!Z2;0pT zzRC%S~@Q&=mjU!&X6U zgj1#k+`g-w-N4Wd6cr{#CKM{BPY%VpNkYW_9AnSl-JddF)FA!pFEf7n*x`8fnK(c| zbPD!|ym&2++2fKGIfF4fp&Me2ahgG1?zJFC3NabBM|dd@S+thfJ)oH&0tqUt*pI(H zV^qAS^Jwt)Tyy)w*NUnJj%-HK#Qx@vC>@SP#s<&3N2=8CmMc9O#Epux64frNjH(}C zOzTza84>vtM*nj^%kJOdXeewbD>rVM)RA+D6drgK)Ekha*N0vPr|Nqc{cSIA;T{|> z3a-@d`hI%~&d3Da+pf1;on7r6v8Kyfva4%PPa5YBx*SeYv;SVraOd%oh+_UR7Dc|7 zL=Y{S;b;eBB} z{}3NWAw90g1dYmqUBt2!)51heu2~iYBO4Liw`Qv8FvElDboxwl%jxo^y@{6&}T=OC)?{fZQGxsACkwt}b_)A*Ug*y<31^-DK+FRRP#6 znv7I7vj(LTi9};T`~lqs;^d4i&L2F$Ve4)3jQLVk~Tuf?X zMiv?PMu@iBZ1TJs%MdAP<>`6&6ktP{{0Xe9zr1XCo?s+#Z-5Dt5@l` zG);vXab(+Ab)0ep!+a2wLX0<7YK0l1IbzW4_TkEWXph$U!kFfL-1Q+Jm8{aC^<{r6 zdcA_k){+1csE8aSm^^ss)!x3e-O}9lC_S9fN5|=^`!NrOe3~MWD2MC$cp4P13M%*g z^TrP864ZYw{rYCO2rJE$R7tZS7F33=!1nXnCqGT~Rj<>-NkT$`C6PkOjOI=2TWL|z z{xlC59~(QZXlfQ42VfY@rU;94C1;9+BAdnG;IT8FOba=uLqQL3M>%I;ub>`xqf4c2G^Y6xqk6c!X%XlaVLE25AyJYr{ z38zHg_ISv@wYj8?%xsT5GI*Jn(K4{#`p^CKay#ooH(KRK%vJ8+Z`NvA+33#^FmPE(N=wKj--wu9^EuX(_)%AH6mbiT;_sYZ3&h+hv zyKW0R3b()e=_Sn!oHS5)nOtt2q5b>KAifR@Radv>?w2`QH#v zWuR%}@gb%KuU9ubHY&w8aaf_aNmDH3?{(~HmQTs@7C(SYc}~j-2;Emz4t86eb-{T? zmruWTlAqewOu+g*ew)tW0MU{^2I6NW}#_uEsDv@M;SQ`YcWWdER8qu%+!9Sl$1K z^=S^}S2-#qiEl-q;+Zjz6J4r$biqkz*+UCdf+<2@#tfiKh7Pld2tY+3=NJWy66X&F z2E%a`=A*(m$qYk?F3V}4>#(4fG?h~xB^Skn^&|3Mq); zdwHs;EX(5Q@fv9c;=e$|3!8~#G9||*GSFop?ZG5LMXK;&bf>qWo9Aj>JD=x2)D?!~ zP`hl|#~*-1*+qsj6x%JBy8atH_yASpBp~sS?bwF@EvLXmyQ0d?C7u-x0yP{83=B*F zlNlgz7}mr~Nt49c;!BJKgRp1IgW_r&ggw|fG>&Ts3n_MaU%U}G>Gr5R+~l02LZSXt z+bXI$u>C%JuR{8xrt6v}-0eaeF)KyE7)2zB4hITP6P@R28(siEcI1s#4zGom@K%(EqrEGz+{= zN*?coK;y(iH!9;TpQ&)Deco#v*$;2bTj?ILb~-If!q(zn(v`l(m!yG1lF-`gLS`vc z0H`BkYM!n!!jrgpaSg0%4!`lE4=;7wM@oR6-M|ep@IvzG{8gP{B5Ha+YdK8D1lhAr zW=)82n@!hd*N68fiCFqB8?BKL^_b*~m8afAIXv^}ti{b$?VF{1+9`Tsaw_$D_vIys zI6R4L+?#v-aEh&pq=;$w#q|C2^V5?)-(QYzWovc5d!e^C=2NDbe9jM57^O2L>dkG| zBh4r_7Bh4g7q;Hde{`labX>;U@=PaE=oi;HrmMq>L*YJkzTJ(`i%55xfJMA3+-a^L z&L3XSCtgEL_^8LRw^s;iRzv9Xm7?1Me93O3F;ks}_T%Ehn$PFK(L{wu#rApYl>H2x z%h`~?wmj=sb@+N;!^U>N)>q8xf7OuNY{2{Y<_#uS;&xf8G7)X5et9JYclx`YRd1eg@-p=}Q-q}7Z+kv_ zT%_wNwijEET*{ptsiwHOHxY0-7yWkDlQ%J{?DU%Xp#$#es4q}mZES4RZGSj{_)x#l z>*F5I9Vrl%mvix(wpIEJ-#q7@pB{hvbBc$GbE8Cv6YTx(YL!z%5fTvKbYcidDqt}H zNXy7XnialifVDL=Buv)RDO#%9wfus)pAwUFx$c>_sAnT;;nA?j&;bl65fS*x@Y4z( z^MfF?mj)GGPe~R8|8q7IM!N#fADZraXVF( zG*wxgKxDklRXFP3b5X6?|EJB3sFtIm^+fS`K)FS{P+Rxd=rnddL6#qetwJ9e**V(Z7fj0 z)8Q0OGv-2>!w;|w-ah;;FGRc~lET_o1&BWUO}rV2#b=M0ltkabqP>Jv`EtKxWa`xf zx!Z3yd*4qdO2oQ0HC1=Nzm?`}x$lH*>)F~iW727TUlugDKl>O1RC@fwN$B~If4V1O zvnAm5a(N(gf4QlI5yfUZEbhOvd4_G0f_JQft zS+0*O$Ipb8pVc)_L!U2ozgvKle?zc)UytieR*CHiMKKqdGmx5$^W>01J$f2@oUVbt zf{Mxy>AbL1^~vor`S(nN7vqz;j{Z=Y4vwvxlM_-fk*`6BZaa?j zICcgB;rWGi?A%nZXpz_qE?QkbaDIE)rKzqj;RJv5bxOg%em^Dz<@PIdQTsDoHQU0` zg3sE<<2;eCzxU!2t;Sp2ys&C+KOf7bk|KHX-%&#ccZmg)B>W?17MHdKM+lk%GGeA+ zf?Mt0%BhH!Q`bUcq|NOz*#pHY^tfL?%qs4sYClgL(rq-m$*8EPz=5oPR=YRrRuS#A zg{KqwwRl|&xs-Z~2Q@c0v)|L}G`g-nw%_fK_6of0`KR-4P71jHi{;IvM(l>=2uEmb z{ZI>coB<_=*;*c7A55^^y+0!9c(>SDbu1-F7hg*hbVA%=Kgxa5})s~1zlT1XXlGftlwV8w#PR)a%rmK_&J;g8gtky zkIKRWTp9Uv!Ahp-(ComVnKY1Dn9_uxBt8kV$>DkWL7juI3wUq)B@}3H1bsLcvC2wg zF++u*op}aSr}|V|EGSfz0%=y`LWRK);8zXTiVmF~AF+P%Vv*IMp$>02+;9rP8f4+J zx3${#1!vWZGC`-S8dH$O#UXyBc||)iA==bUf~qj34nPcN2LY8})G=6Tj#QHd*E7$E zmsl*wE$uHr-460rB-pjmh~n^Jy~xPqCZRIBaHOMJ1lUrrDMP0!+z{?In3c45Tx#@pmkBS$Z6?s{B1`>FpqXy=NuCvhxPkFTo$ zU}$%F!TFippe-xkIX@tdK-PTMB^A)>q>gFzcP6bH6W9DRkzYf1-GDNHi3*!=e6pO* zDRk-aFGFise*cHv7zK4^IX1+ch$QNYgjC4JUYe};5=Y`}Z*1NonxSx*=Zzi7XbmMp~@;MXp1g39CH=Ax_ldsrZ8GQQhS_HwV@!PK@CQ1brZ(P*T=RJx6EG{ zEA!+CB!+X#Bvq-aa#ufiFt;>1`kNuG1(pNwDW&{334QYvUplbTOduT6{V$+AI9WyWIZBDa~7GDZ96$xDTx2us!{Pb z6>hO3G%5uT8(Z9TaB{kJV#wzP!{?wYA}tMetaA~NQp3Tc376+}&% zP6C9K&n1U%isdqiL4TnZ{N3BzE4jv?@5Q)MyN`~))nsb4eH-}OH6~(#h;ThgaPudm z(8yTbC^)O7EcY(^zpR!4^zcyM#^3h`1WAfS5Vy%Na#S)q+52nzGkdyhi8A%Z6aTJD zG#L%y8krI+hOtjcjRzOG8QU+zMn(pHD1gWfp~9@jqe-EMM-+kTn_C^jwS^S7Jw8ZBPJtqLck||_=))mWyr56DLh7VM5pl~vu_qDW6P$XvN9)}G+2#t zOT+5<-F*3P%9a+5@dyqHPQ-YEjT8)W$0ca`AZbGq5-j((q8vqom5cTUJokK;5Yx)e z&+%Wn7`jK*u1uowqk>fhA;g}xHct#;g|F-K(!?!_Th#!Ozw2e6-8!p!Mm5JrSymYn zB41}KGA^IP$@CF-@FWNliWoCeR!?TH+De5(2Oi|Nb~1~@kqT`h;ymAI$S)z(DswS^D_TAOz-F7}2HPyKO9 zLTP80=UVRDPc|Bvl}+Wc)%k57m;wMGHg(u};|@0F(IVyFSDzU#wq6ZcsxIv4E0#BU zjAtt~-X6q1If><|bbi(2jRzpA=&Rq&zbd%7xpq45xRt?Q1(#lj=E`)F`5}Z%sWxD9R0{noh-YnAVS+8Nn9f>t+|KqU^f^BD7jS$zo@HXMb(Mh}1z|l5grKeMhA(m_7p(suEW}DgT^o36U`RKvK z?U6utrL|2=wxoqKPfKOZRug+BzTf?E-QgvK#Q`&>J&wBPJ{qOTL&DZUSK2(1QXx?J zqibBC@lqHwL2QVM5LOt@5N#FW2g=f7t+{va`F^~pz__FZV?FR$;G2$?d9&?i-E%$; zWQ*5unAlO29wOv~?nGidsU)bL$MzQ$AI3U2jsa{mrR)gceX-j@_Z!&GCZiP@z# zPp}N~p_1hXm08-tx>J7ph#8jvC8g`!YZ<$DnjL*3pP`CQ621#$>PjQ{fa`BQ`)w$l zKRP=*C@czu;yP)Gh0LDjgqWA`2DQdK4He;@>>+jibU zBD6})26H)YZ*T5w^+hLX#Xtxd0WtFVKAn`BbhW_$+GTHH3QZ!10Wt;6N(WU6qr&pM z=LMZl(J0D@rUXHrCK3aI3{-@Dlw6z0D&fB0NL7*J1qPYmdAVK1;FV0 z(|g0czL2w?M*CsU5`VhZCpB0ZY-k`<0uVt508)gNh#(PCE}~N2tXFL*QY`cKA_VVE zcFs&hhqrQ4*l(P8>(DD#sc)xP!a_GX(xGmlHv^KyJRRn1iNM_d_F|0q0aAhXlUC+F4>^qk{; z*7ppyv9wIvWZi?L6#q}oV>buCOV~ht2bh6@VR}DIHaB{HX)QsPqE!F+*XzBWgC*eM z)5pac#soJR;G&v<53(czUhhfHqp2>xnc+rtQ<>461t}h^Y?cjOgwqo;H=CsPB=ocPy34 z%(hIp6cpr#DA$!Bk9t_Gwf1P-{BBF0TzPnh5-~%_ER{Wc*t__VVkgb~5F1) z8UQS&>2S=K4O9FbjZBu)si50!sO`~^7l3ldbxiMp<(ZIJuVgAlg%^|gel_F&(2zid z%$#R!m~T2pN`h_%Bju+0kX=UBhRsQm819BxY9WM)s2><|x34!oP5e%5L81oH%mcn@ zW(UHehFKy9l8}}iTqqkBe5PRWDrJ)XZ$s>;BV<;Kx zl^Z0VQc)Q?$-w*I^Jo}PQKfabEH-Rj%BatM$IOS?!La1lhmZcrr#7DM+eCZl*>t|~ zG*M|Qa~)sA=SBE73HXQVr5F-1qowA@Jo@Ex+syDMp-}oSUuRMc9!u$D5JwEG|N)%Dc}{h;kb}KRcqf6{5DA&`_cY z@sa5D_+9iGf{SBoP{Wb~PwGD{!WE3~hoBx*|k z>%vC`#WUs~>Y@8&ZV(UCtXCT5Q8UWlgil1V-|nRg{B>o`wGvC`Z%~Q={n70jahX1GPK!B2^bm67&*TQ>? z0qOx0m!ioRAHH}|X^*!Ke}We|QS{bN2L~LmolmC~FiL8~Sbe=aOyZ%cK8~_Me_+S= zod7AwVi7rhcWu))lpovI>3lw2tHT%~4ydw3i##X%LB%SHEF*FK|EPM)u(-CZ3lsMQ zCj^4K1&1WKyHmKkJHg%E-5m;t;1)c1aCdii>wWs`d(Ult?}uHrt7@&e<{0xGtO|mS zJdj8qJrn%5ll_7_sS@;yHY$m$1EpOtDDzhQ7`Whl83c*U^X+Vs$ur1$5i-M+MMA|{ zlj+nmFhCQ+k3axni}SX-sS6_=KW`#KTBJvnVA7w3QG51LtdZggUjeItLR#zsePtQS zQ|inl;(7)sqheY&2gUBOF|PDITi4AvP-rL!x|UICKU3m2l0U@*oAyH>2G#(akPrkS zl5AMuwLUd*MaxXK-d>`Ih8hGcvY)?~3Yw>{`ZZ;NVBr_?P7*6@a$9tBN(Ebsqjl9W zWo*MGm)@PR3+-`oLa1pGahWHMM`h_K-nQNpg|`*TF@431w5lvbldoSiDw2F2$GZH5 zO)rZE!-8W^1-88HHj7S&uOVe{Qu?Se|vLCJ;Y_l%(;8Gzz!2u^78D7S2DO> z|C?EBFA$9#&_}tx^lTDFKp{*|nv5cZfMY=u%Q&M}qew0;%PbD_$KpvW zp01DYUK|Q9b%cKzgp1eE_jszVKU9yUU{KJhV95siTX9sD=b0ve6iM#4K#~nLy-41G zLn0Oc6NHHm>DADau94kKr~9R z#Nzpv^$ggAzZd5No(>8Hhb4XSNIsG?#RQ6@upL)hU1Ox=CJwP0M1)!dN?`NSpomTB zK?-?M>fh#pLu=}Ss2~z>W$-1-!ZLUZev58tdFGuqNLNMph#5iqS?dI=Jk@r;ws0O0FTn`HB zQy>s(OvcikQD3OORvjE3K41K9laY}zckM>!WK$S6ibOv?)8-;vZHd4-KYQLDVl@RK zOWzl^<&sl`r!dJ<7_+~7mg-DF;rh3D?1lr-sV-e6m$koo?~_ktDPOf&j-JyF2q_S_ zU26$uf%jw$&Qw~PUiE(X%{5keDzROu-S&pR49w1FC4{09RlXlA93np=Pkdte=PrGm zL^O;v;Zy^BopzmXf4c)2@xo4)Iy_#}ldgxQ1mf1Rt9psrmlvjv$birkuEu$*#b0G? z8cWlw3JNyg*x)$2;fYyAZ3dC=_Wyct;%7kEt_I(F5F!r{ygqC_J^|096BSdl$>z0y z3;bz}K}tH(+w}$jjHO`t0#j#IZf0Y>6e2+K*PRgjM-Gk#fh005ZS9UCi{6#1ctCZ( zU-#)MDvC&*ul6$AA1Aop2T=|VM$Z%R+CO3WnJ}(<_mA$iZH#|F%8xD9!ZjiFe%f8% zjGwr8WlvC;Nepb*U=+-bWpkFC4wUzbpMIf|Y5x1u>nCz%tH@UyfI9wvwbcWw$=!R% z`!r!UZ?D1@un(GC^%IMvk9l!*e&6h;>;P?1*%H1Icj+ZyAdA5al-kZT^+lX_dKlDP z|JD?H{o{X>&M?9Ee8bmqyLI48+^sg((UC*=cUp^Ba+t|Ks z<5v%E4|$|8dVAjQb^7^j?`P~Wq`r@Ca{>8P9rqea3KitM%%n)LxVUHMJkBQt3bgC3 z&c`mDuQwx%uV9aF*k{2wf%r9rjG5n^uV&n+V`5O~+N+|MJ=Q%94M=M(`r`*(Yzc93 zzd7HD#sae^j!%7MZ+V-us8t?YX`iSGx||wFim(N2p=2}i36=rRW%JQR9_kXj;w-yn z_WL+1W5#1*&jDf%qC=u0_ z@9hRQ#3{cs8|Q@kzq;*}>ApXyY8M=i-;FlkSGYUPN7r_q0nq&<2DP`oY@T@l-Lya4 zZ)U%!)}sMKLcy}K?!xU@lI?!x@%|+DZs_FHTx=sYn+!%p{c?1EUJb7Fc;e!FTdmfO zcE35L`kBq;vKKRt7ZD+eiAe@7{M(+`{H}Ie$lF!HB!ilmpxxt|m$s9g>-G>s?4bmS zMot~c;q_crE6d8%;FCTuC{~wG3&HQt#5UP(=@^ClhA?zX^T-Z)oknRvuSEkPebw5! zEi?4oik$Ok7h&3Is&b8LQ8rQA31|}f4 zSnh4&Q8>Sw)PLAm*hm!4T@&9n^5XCBi$8qr4>$1?F3wGn0Y(N^zq*=C{lmAFap-h^ z*(l9kJ~e&S{l?|j0xudu^z?WQ3{@Rl|;!3j?6{_wPxj*gCk0_b5FU35r@Km58A zQNfl4zwFG6wtSv1vjNc4ElQJYQ-WY@*yZHfnhiE|&F`E0`};doC>@{s(-0L#twy6u zb1nGavBkxjKzrhHQ`_>Ct~#~NZTjhWzS?2W7o4jHA5F08^4QbD48|qE_dY0DSFiO{ zUk>#H0_7W{qnCL-1PS9NX>j@_$v4*Cy7M{-rDo>1oYs$$To_x9Ultv^ansrw?exq} zh;%$1hU)r-vW5U{Ba6#^!{{{*F@1T-`+1XHDuA^G?0=QBYL{I&H9w7gGO=;-0DJ~J zQ^r!QRnB+!s2^|dsY@$e4mv^iYaKgP$fni`Y;cB4%gbC`TtQLQgz{f-S#R&|R@>KN zRgMUx-fsGd7_}Ylt@%3KwuJ^`2p=D|+!T&{&VMs{Uyfjpq(q_IFc(x7bd(oh>LFd> z5cr_~q2f$C9G)nj@v;ai&CA<#{CN1^qvo4dTCv(RR;N>(S#l??n+|>&+2`H&U%ZPr z4A;kODiiC zc!_0Ozv)`r+be*pYI)>%F8eTf`ysF#4P@0RCixdEH(tMd!wLD$+42AtM1dd_21*PW zoRU}sX@D%0&~+XTcsnZK^vgH%a4-%2F(F@ln@9P#cKR zjddS?n(hxjf5H(l>~$kGaC37rKg`sFpTZ&&N2eH(r1QYD;aIhLnRDs#IbIS1?$)-t zRz|FXgB0EyWIPUQwb*7m%d=j(+HN-K4@0VuoV{r>ob`*sah@_>g_J_XaZfm$bn@=* zr%|E+VpLVY*81Z;&nNAkzrT^VnxD0l4ZgddgP-fS2O@udub0mjN;1o7YKTfKO^sNV0>FNZX$Ayx z`Afx;5FMwbky~@oRh`(bEl(M?0Vb_iNEpz35HgB zjavdekzR=e7Jj#hJ`EOmkMDhjRfe;g;B5nXVWug2i6IS>vOM*L5Gla7?)?`B!5M@Y zXja_9lk~9UA!=TitLdfclpPQl4<}>^_%7f4P(^~+(MlW=vYYfa>nC{DC+*(;!k^2^ zpJ;jjv>kUh&=W2YV*?fB4wX;>-jIiFWBr_x+3uZ6+iO2T_@d{NDd}(jd8-7|w)67o zu`T6frSZ`A?aq`+Hq-t6H=rQ&;CY%#orM%|slQy@PW?}v{c$ko`y-!}Pk2u0dEY!G zMj2RvrcT12sjL*Pp$Y+`{*d27zyG+pJ)XxvhW)Py&^FMo3BxX|B%=E|+(<+~AVZme zOgqBpqk6*iA1}bo&!ST0(9bkE`C+w}PD{FtRubD|_`Y>WLgI1OKDE_cVewWTQLz|G zIz!i1v43VXWFPOEBZ1qkckPKpfxMaY5lH1uz3q>~1?POln8uP@p=s-epq(5lZ z9|lXDPJ!&cLTzq+Jx;s`9Tvg_u8vFoXI?d{2*q$Pd8MA9vNgD}<#b(BIiK2C;sd3l8$yo@#*%rtS#&Q0BPX=MC!3yp@`h1!^e%Wncm zNQMI3GIEh9{Rv@LZrAX#St=k{I_AWFT=CY*mP6g;R>HD!H>cj-l9>5HDtbzWch>%X z&syn9@`sjGXDn&`(;oE*vK)tMq@$6Eg(KOd$>OsK^8H-(o1FF49mXyGFPTrf7c|rd z(;{&H1_*qnt~q?%sh=e$$~U(T>l{Ba5xu>-bkCv6Ddesd5vU~@BdA9}0#T@3L3!~D zrtEBJL;F^&FR#{L`ZZWHj7hMQNm!Vjc6e^Sa9VIYbJ*9nWXKf$B6r`sYIs;Um@QX5 zvS^GMoMzw|0^NUQjPA#c$Hi|0t~Ok5eBiJzC>wE|oQgjP8u#Yhg5YW(JN<@CV3lo1+6C&#+Cb{B|8*V{XUGa$7 zJXiuVjNptf`kFt%G>R=(9r?N9JDT1VI_ab|EWMkL~V(r$GsbCTWV zDL*?tKO;#7y3!?zoegOr638g+Vj*z))-tDe0H8m*ptP`1c)PHVSYI*6pPpGEX~SN+ zk7_)J=XGb?Lf5XNj1?m+F$rroKA>2|-5)m0=xGRnhnt%X;A7e zcG*2*?^xCDQ~PCSPn%RX2L&t<(+aUjz3@h!1~*%Ss^!OxmczE@Hb$5$D~zw?H14T1h$hA%-^VMyqmI&lXcjNDWgiAWz61)yT6 z{JCxQ6q%3eE${|ZbT1bN&CbeZ~M5BKMF&+|WkFzeH9&b6rQV9;D!tAHp%VLZ#e zj5C^30_M?fT3>oU6Ytp%t)rzjph3d+r;hAF-YKbI5B`AsYwi}nkZ94+(3F>zX|q-P#~vNFzpiO| z3??9w;P`{(MSBhY@XJkb*>=4i+@(8baB32Tg-uFv22W$@gMvY>$?baD#HbsgGTA@g z;$$LcW|VEg9$4NEMm6P4O`m)}2q_8@NJ9F{rJI0)Qo8u$Sr?}7&z8@5S_|A?_bm7- zH=DoE(N!+gSwCw#?!&P9m;$}tp>LlH1j1OEAfR({CAR`%zVLjHFSG{XH_ckhQ4QFv zG`l?xkHyE=9i_LPWIb)axrix(|A-_T=gges_#D{?=k#Um#K?8r_%EbsO*|@mXT9T= zk>9-;7fU7SH0N-!7CtPAm~s(_9Oh;pl?B?c7a&J!uc*^jL*C4++8l`Yf_wCUg3z}x;iKrahbsq{s{TW zf~@%`cqrn+p$I$BXf-`*SK<_u5)kIm>IQPti{A^`X|4Q?>k7mC<;%pw4GS@5zd`2r z3+fdLkov(YiC!3D17P+D1c(5(UejA_op1xm3$`{L;NM14p!$}WPyO=ya5orVYZ1x@ zmf*)ykixPt{0TqvP%Qpi)T{UxRcDaKSCzlYh7Q5I5-X!7V2kH zF&xq25?xCCdE}I8K|mSz5Qk?nu-LL^Gpcx)aU zv3iUM{;@Hvf=6l+k)T)*0-Bf@+?Ss@&gYO4_|>Rii0H5sAcC>}j<#aa#v|?^?iBia zur<7I_>rtnE9Xvz*7aCW9y57tBm5ys*%T5$nYH6gMEomVWng3K-x3#2LIN8aP=ZjH z*rJpVRxNmLiFtRgxZ!zCnYHnJyfp)+E>jZ)IVeg1lAXoO^(rH(8i5GnU9O6yr& z#Bhu_F0cih*M5I!l&CJkB$M?AWivg&z1gw89l7}adwN;af6vx-FrMi-O>Xf~!}F@E z^yju83LJVy^VO`xozbm{6j9Gnu;xkftj9w4B#)UT5SF)Yp$U2t8#a) ztLtfCK^OyTB=P*?GQ8=wwE0gYI9f)=_kKoi9gdQPQkZ>C|5);M*X^k5lxpb6njaa0 zG+c7S8Mr+Gx}y5S3HX%N%3*rpBa%LkE_{r1bYnzu?6*(H)w<12S*$GT17A69JCyF2 zaN92+DpGAe1~FQo@f7KiP?uJ;Z>4m-h=|zJ>yj%Y*w*`gEc&MMLgi6k{Yc7s&c zxSmspX3UuLvw|_B+Ul>r)424njr|kTB^SQyhX2{1imq-8GrVYvpRsuu1vix_wE_ij zg0|3!36(+E*}jx+{#Y+p)i6w-b!zkoS($0Vko>UaxmZ|O(B({=1A(Dcif_zd$m5BsV{xrjT5V%ipQzTjtVn4mJjme%ucLo2>$cF22)0KOU1EhMO@gLvugKqb3JgI zNGoa(Fx}*F&98!5dUu9HBnD!sEw8;o7Lpki$OZKiRZLDFosapZ6@EVPe3~P^p`jCh z6^lH0?0K}x7EYBlPQ@Z)#qTCvFx{22d3hAcmqX})ObzWb7N-t3uRsC`!`Hn!1DXg8@lmz`%d-A`6Gd;2jm~46PpKFQtpWqdHQ|p)uW`--*XpqyU&F?=h;X_B zt)pY}HHzs&TkhB>RB|b;oBSmUhQ7_x_e&>54P!!9Zs(vn6UrM#+k*)-wZz&g)K2{? zcX@x=doO?f2cw`{V&59~1Y61O7gGk*v6~NYdhYks|J7ADAv#6n^4}jujYi^MJvd~) z`goVpvaWA_d;Pv$Quk_#TD$m`C2_7?{6V3tv{VhK^Y+~!0`=rCD&Q95XpUVPTs}mj z$W5Dm6{TnU4FypnBJX*DspZVekT1`Ia>@1~NDduB)Otg?$Y`T~uG)*qK&;yP#U5`h z1zWPf?+tsHIp;w?M^Y98ia!|vJT2nv2P194qeeo>*jQkLPxUmB`QxyBhfc-kPM7jO zUzU(vZo+x~qSP4-Vq%;K2F+GiSJ%ILr*;?AnpY66=_cax z^Qgm;OdCVSJ-j48tbV?!7@^TU161goBkPqW+v*9RtHv1h*b7L`=EAV(Ii#r874g}@ z_@CsS*-kMjSfg2bS11QNX=rLmbXtw7LY-W0%T2a#i$C=j$Mh*Ey4IH77ZCAsep~1| zue=wboL9lD5tZ_}>OC)cN?%**vdtV(sr1%s-wrR4$RNVU6|Z%hAD8s**e0NH`q||u z_V43kjH~`0Z)Tw(ZRa#Z_*)HmD!tPe-IM@^*?Qz8qF%1)(TX}WS6`peAS>?&Hg8kh zFnyX+;ix3WqK|j20;*pVGD!u~Zayt<+{?w+iR-P)ZxEwsentE&X()nGl-Bh1P5Wgf zz<9o>g#PPfTpW;o6XWA#WM}uy+5m~g0uliQA{Z%X3WJ6}T{m170fD!h-xYhe{TGy< zC?thi_wU~jqQT;`5jcFWCrw;cIY;Lfw_$#%v6FV+^v4cVUrg{uzqkvnsX zmru%Y%x?N-TQ$))}A_Dp>3#>Py@kOML?K*^EhALKuGe)FdlWp=@_afF~5QV z-C+vb6#f-TTb#0tBVi_=QMhS}`@H0(w%DfslL^{eiS*CjdPIYqBe+spzkrgWia)@) z%k#a%CAZkKdy8PX@y+$ofBo&QsbibG51kwmfmD)MhDDEtQjZeRq0e7Tx*LcEp%;

NBsQAR~rAm(i3d%7elYrbiJ+^e%qS^fwHH zbhXHv3uxFuB$brDh|&l~4FykXdBiA376z<|^OU39(!24BLj>dA20=t=un0IM33TJd zsTcS{;;IE3Hyn+uajs)<;4q_c%hZJzQO_ViLlh-E_uS$bO~j#v27Tyrk2g^CA3GsO z=Juza!WxgtQ;@O>L-ZE5s=+BWrHOALGyg~E#XAF?B^>|n7Gl&{C7%D+FZGX+PZX*G zr3$v9TJg|yhH|0#RGZ63j43NEiQXZ;vul&Ng~&>&YuRi$B9OAaZz zE_Gm9A4JpD5e6I$#-s1ont$?0Wczc=K&KYpv|&)om&TR>&*Hn$hhRr$O1~QC5bYkS zA?24jsD`_4U#=Jc zWAGGfsJmZ!>lNd*f%-=_Z2)9bC5^?!+3U@`dzM$$nVL;&=LUR^Dotheyu+C|Xrjz- zj>#VY)1F3~v!ii;=G$4-6Dq21m$&xpJq_I5QKJpGZ0$#rr>w2q{A4eD6+0Uo5iK;J zao(S0#o#8n+@JG!9YvzSR8ZD=435j7R%h!f6oa#}vIdV(Io2#Bd&yXqkB>NPG`e!@ zw5H2?;Zd`I{1JG*C;Rh_KN+?}OV?wAbCnto$GWf6u1FAbm7320s;2Yic%A2Y1{2e( z(r2ZM?^yw(CJ3IfuE4|*G+P{gbp^c2-t%)y8*>>4;V9J(b3^N~IkzN7iY(j{a~{_R zaUn-kkIB{}u=I1Z5K;T;%F)36Q{79TvZ$XY zjA$9;88NW0u!1)|Y9x$FOw{~HhBLP4m-OurQD$;#dnh@6rtxQjz zYs|_x12V9jh-r+k_f!*8zUsFA4iI$fpK1f#dLg1zo_DyN0_n=c?wcoNj#S4=$}3zp zc_Jt%A8)dM+)Yv^*{-&-FZ1yundAdKm;o_(pC%t7I#g6t%_|Gbrqd+_m-e^b=)0=t zIcv`+E*}}6hp{M4841>ss;aZg)`r33s@jn zvETvW54BKGsl~;~tx@-*S)8Wrtg==rl|cJ*o`)M?J2HF5ya=a8!oe1`=Vx)b*si;t z#|fRg+>|uUs&oOOv(D3)t!+$ty7h7nfWH|H+dlvGF)<~D=ULeKamAs_+I9af3mCr| zZuEKMnrOed8W~z>AIw>qwtM=+rOFqLl@cjS-b;c6^{Ryu_uoTN>gYVSgz&o8W_3Mm z#r4EPx{I3E?W`&dwI6|2!cgIm3*pc`4~;8oPms|}4L!Lr{-|us{8;pjudB~ceKjmdtow+XQw%?p-dRa%{ zFv6wk#S}Ry*05z4i@z(Cs6sNa57_vRA?85;en@ zFNE(QpK?Av!9y@5;=XP0>sl(+gH&88ii|bO=d$^{PWr^erbF4BK7vNGRO_9k9Jedg zBrXTWK%eC1b6yVYGHEqzn6764i)tzQwkW$wpb2wcJgn^re# ze_H$u&9^=H2^HJ&j@c_5;(yEo<`$hr2Dt;Sg+@Uv!HwQ#{CXe1@O z!--}$A_y`NgpfyG5~yZ=xD=Hk`{f^rbkTm4z{}?$FNfFX{;G&4svv{Yw$t;h;7JI6 zfVru{Q+BCZF<$L3(>Vj|e}~C-{(2um^s;zOr&?h~09S8rYkPihP}7$z&K@oek&Are z_#dC^7uL(72w{ODrZkM>Jb+FhT(I-!Z8*#MXh!ffyn> zGMSZlWeJj6`pM81jQ+P!G0`w~ zI}!zEaeR@Q7(ezfAklV4mB{10Sq>3OPC(@Yy0ME8=rg)h3pyf>x|H56eSaeVg*pRS zKN7OsZ2bvj1yFU!dBxI-!4Bv9O4N$cqQiWoUrPaFVlP~_;^wQzo%baf)=_+~6)}43 zxPQ-iAAIkZ7pZI8zN4#dpPS|j-r72B{a|aK%a1Ey;7Mvup{NB#Y)2L@SGd&rlw7G{ z_|;{l&0#m~T9ru4Yr-EkVn7uBQ_?4*?+)4^*b3N5>BYyucd`%t8xISvQYP4B}f@A4|T^7vZ1B26?%dILE zYrY9lQs)|4=+_$6+9X+T5l#Q67XwQ0>0&)q5H+L-79#^NM?SL^Z|3M&kK~8w8w5qo z$VkV)@Vb{E#|Oy8=E7m6p<6{1j_y_KR)9d7w%;}H>NVCfJ-B4+QrtVb*p7cXy7BMX z9P1zH|sD$6_k5E`SY*!UkJB)4IBn*rWPMg*{-RmiQoM0gdQ#mU%>pcdeAFpW>whe(A zfsoI!+rQfcz?;RKR^YJMc6x0rwBS2CH)PBMIW9J7#na~mV7{Mun<0Ep-gQlG?{BY1 z{l%9?=2o)uFkdRmmeeqRgh`{^t?)V>{2fak=jJ=q#9LHQZELU)swi#lFa@IE z0Pc}L^Bz27XX=Tyx5v6B4&2GCO&NH#u_3?0$KDh>m}7|0JO(_2czEI=v-ySF}P`zE@ygr^>H&!h|^}pm!%!>&9rgw{<1jBB$Y1& zTz0+C=WA9rjiD9G@(ug7+So>x7Ba>?tO@7Cm^EON8Y`z(Z*=YUIL}O1;;;pN711BDq-ba7 zXAzZy^J`=i^2LQ8@uC{$1li>_5M&l51fcXcU+^r`WQ~Hs7mMC3o7{ix{8(pUks~{P z@lVH}*if@bWEF~5qINiS9BNMR0^`H&@Mm2vxg{P(daR(1`7lDnG3|*{b6Q)Tw!yD;tS-cgqWB6pSaFlkcwa}&Ppo$s?KjH!JWuBk z@p-?zLKB7K90<#j{M)uwdYlr;A5Am5LX5&n?f`Lhecq?;hB28XR+v@HXmz^@cp)b9 z+jq1wwZ7eUK8~3u8u}5|el>iKkJlz)|L#RcIRNMLe&-F*8$c$w)@aSn?4bQO>#k~-x0~lmd1d8tz2Wx% z&z^QoJ5tJ?5JM>lB*nYxt#y<)x=YOtJv}u>ufdL-Ys$BYgz}#_e!~%}2(vD)^e}i1 zo*q2LUv4!NG3M!bg;1ONH2#mBtU}Vr%1VnG4ud}xg@4(i>1;{C11PA*qt9k7IX#ab z&(Ggqr8x05r>HgE@53MT?z_Eiy`6NNop1l~r&asBmRC(WnR*dI_yTEG>N>xMgmp(@RxiwU}MxXGN1~O@G@)fzP?pWilI&Och=1CF10OMiBrJ z85Kp5Fa(>l`{}tYo*KtMW!=HbfIeIDSt^;qvfT|@UWF+TtAKrIo%cUpfPw=3=gu>7 z$-o79jYNLthO)-w;`iC5UyiC{b6^>&=o|w4j0?oCh%^6jSlX@lRnx_bhyB5zYIf}z z*{OjE`Gi4=VW+Qwo8NDYU*4$2?X@#AiAv=vdwhBad!4Lbv3iQYKnM>GiUG_CO7dhT zV-CfNz)ipG4>C}fhV&3!C#kV1swiqT+;!Je@xC0Gb;K-0Mf<@Msa5(SV2O4Y z5+|8S3>7nEFlJW)J`{fcaIe1HNwq+P^h?DsByK}NB2ooRI-0dQ=KTqW4#S!{2c6To zN(PxRNx0vC^yDnJqtJD}F|pmDGV-<<9d1<}Yrnn*2)*(kSsW9z+TISgrK~u=oqUH3 zHFM2vYbYZTps@cM_D~xZ*mab>+p$Vei@>1esXCH2LJmfGjbM~|XTJ*b|2v~81muy0 zY~Uy5ns_dsRy`R~5q}8826f;c*26Q_^laSb8cXFzj!k{?SQ! znIBRD3|!6H>+9R9ijR8I>cfQdTUs7#2Q5Z$k~{2QN;Sm!G|lOr)qX#DVL8+4~}6>~huh_32p&`Usq_vPCby7HGObU$;m9 z`=yuOqXXM;%(55fD&xSO&6pJPhL+~nRWy^&%`M4-VJxj%YYR+Ydo3fv!xe{wb90I9 zl=Jex?;`JE1x1tleI)HmJfszuMGp-z0uq3d2C7Q3-!ET3sE%iH&#kUiA(VfJgUTN+ z?1M<~2Yu6Qx0LK=LGoFfe|S*a7*bR?OiIM!WM?w)C8h|M zHdP|}QmC*9^a=kA{e4}OpI_13Tv}Nf1;a$l0#2;mi<383f(!kl#?G73M!P@8fauZ- zU&7F^dbV=8(UYG~-IGuNW5eRN96|H7;Y)|hY`X1?4S<~#fQn0<9N*j{z7qgR1Yrpl zZiKnE78Tw4!JlzC9zDD~78z5-6(=uuib58zjKe(XR^cr<-MdOwEK17-82+dWdLbWO zhlw4@9JZvA&$h7@N$SSi(Uc(egtp| ze^oa*I=Lo_;Le4ZvXi3lvl+T1W#L<04HrdNrtmr3K7E+md9d>l)6_H;F*g*emQx63 zCpW)>Aza=O6N{lQb$u!O?ejKkoyXkABh;R^cm^T zrJ-r1T*EAT47bMI{Bp~A6yJ)OMLm}Br5qub^zGa<C7qsf(jWfsmC8z*sy}QmH#RnwpVwR5PvOv@`HdhLvHJZf z$e>(@`$)43ZMB{rY8g=c!-PRzjklSqF-=R&JE9*dK7BE$ zwu|iz;40Fr^yXQ++oN4N8b0rvi|)|MWwm=VZF^%~7kp*4zJxwt@$~j|ItGGNj)tep z`YMlAt8`j0=)SIM)mvY_Ak~$fRW-WYoo0nnC8>pCW2t?IwK1!YWc>GZ=0oC{l`UKt zx{wIZf%Wo*5sHbN)_j}|0x>te3?oa|w&5{v?X9=Ghq$;4wz+oDPoEpPeNVcSvx;eg40dTl6LYPu~3hdh)(I zo565WTUYD0XxxsD4+H0NjYcMB;2_U;P;al!pvx?ft!k&~wa!TMH9K@L5AVl;5ifEf z!b_JyZR)wHa@S~YE-l#gc6MG!9Y#;$8Sycv>+J(^MYb>RH`j-NfRN8~zX7`eA@yA~ zW7n7IZwZj(*|_hF@3&oX;U=9rxCS#i0P*m>)6SomKW<^C>-}vViqV1uC*$pO(boFw-frl?XL|CXzCNGF>h}+5L4$sudLa2-<*&76bh2#pol{?3>0;P_>8dO`e1AQkM-U9v;IlvJOEaXB{k$N}LB}e=lIFP|Aavg86N>%+T07)) zQgQL``{bU>fNKc2>`Be!F0gfgNqfGx`}YZAf*WEO5jOeIPVoNZe;3FXiL2cRVOG^_ zmWv3hcw!g@-^3*T*i@PNCkWiDC%S4hh>MF0GtxM9>Zrj;k zOccL76buy~D<>mviU`%JPVf4tSt{J}x!55}E%7(T1CH6Lsi~Q{$AdfT7%R_*zA((~ z!^8#-6)kd%M41FDG#R_fao3qCZkmvg!GURWpT#-~nI;>HJqJ;F;p}<>|3G$jZWgxq zUs7jY8^dGE^c=d2L$@Czqh-kx$^fu3`&x6P{J@ojj0T&ABAl=C=S~t*1R+w|-rj(O zSM@U;7N);B;)Q5wqQeTg4mxh+!&H1vHPzL&q#Q2fakS&t5L2Tcnfj(@PLq@0CR^Dp zIoBJ^yqCwyFP$X+WB{&@y2%vr&ZGIsNnkfLn7CFWfq}03yKyc}{l!z)#^omhvjD_c zj6WibEQmtUEOx9P;kBQQ+E2{a*N@s?3=ho}gE|(8#KdlnGdx$+3j#@w^%dsQ*ssQX zh6-sNmZle#H`|{YN|O`0j0sia4b@;0Ky8$(WLsrhXF%)&VR zw*{p1@PjNY@@RMa_APPclblr?t$3Xno}z?Iq-mCL`VTQe_lIt5sg!o$gbzhhYDEIi2oYIVMBHOlICW%Z!9Uf)iL~>L*JIL-En+E6I327Y(#{WRy;adqb7Uv)YR7 zsl}M*w64?&8SKa4HcCy_%Qpos1(sLQQFORn_2YO-(^UzvcEG>sM^f+ih0|lB94tu7}%b zD1w=UdzQA;KSYTI!;^SBpEmwuMl_d@P^guT#RxE(;>DFLnRYr;>^&oB5c%P7bda~f z{UX9k9i5hj_2m;2H#heWw#70pTGrL#7v2Dm>?Ce^Vz$7dI)7D`4)Zrn0b5cG?nBmS zgdkE~<*8AWupXaB;y?j~8u0gLiz(?O{_%79Sv=Cbmlor76{D>Zn8=#_6#xWrjqF!e zqfx*C+sNeC{TV5!Wks7BOV|Gq6H+fDE(O^Ig&3RSH%xf}za?e--`}GMldDq6Ggj`+ z70uMfMIA+#sMVd!9C6CwepeKhAG_EJkL)Mk&lqt=0}@8_(cg_bcnEAeTblA}(@LqQJqE&qJQaDLCd-7U0G7lx zFdwdT8HdXEQd3`|iagDb*xdZ9qS$2lph9|HV{+7fGQw?=P2+M z_Ho*=b1@wZxuqJR#qoT8{AA^@HB-Btb;^)8h*<_E6B4*}i6=?r^} z3XlHvcWmyrhx4vSU7En$A0qP3qU=J>MR4C0ML-@9Nl&B>gt}O#5~0&_-FiX4aX5Ug zCX?lOo#5+Q#n=nNF6#xe-jIK%&f^WeDpc_v{)^y%;2%-t^S*VLf(fJV2gQnmPn{)l zM`P8_Fe&1xfl#_y^FV1FFj6Cyup2WPu{;{4us#iho-#r|zYxEmBpt1hltw&sDeebE zC}WC7adiWJ6sufV2sbc%>|CZoZQ{qF<~SoJs9ZE3N&ZlG3fvvY+R`Qp`@pT_G8q6~ ze!u2e?*uwa31m(&ZnuJY%u!a!~3X=M)Fc~7Yz^pj^XAdiC-!868}|vn{OZyLrN|q%;jX8ZHlEC@rS2uu>#T$kbqA0OVI+8P?;FD!LTrU(@oOpMUnqe&(y7$(h`sr9GD zI~IzGBvNrF&;Y9fib){XjTFN*qpt`{tWa*CoEH55DKj=98H;y*V(j>7?KWZ zJ#LAB68hdkrtEEdEUQ+y1jGTc#CJ5!2X#FP3>Gyo$*8 z9?gPvBfZkO1@M4eXwK>IX1_;_x`_P@Ia3_#9IXr3qMRFCAWQT3RQBCtSAOZBlkH3% zvr8LeUlhmEZ$Lgg3h5nD02GCRzKD3UnW{nwST<36r@R*PJ@Ut_T-%F8fEC^J*9h%v7qJhdF|J8?Jfp zo3VLa{Lv_cq7ITtkVyr+K52fcL+doIgarF?*AB@g?@%Ndk3soEfrZBnG1Z6o0bGgM zWTU#}1Gxm#56$-hiNQZ)N7z66xNZ}HSmEUnzUHg2_kw6#*`G;C%|O&+{{=t0ehnzk zuz}M{Y&E*6vwP{^rwF_KSVj059eu`5&!$-)iM8dlqQV04AN-bG)kDx0UW)910`5Jm zk2x}3=)pin2|DWxb|!{OY2c#O&u=N8w!ok>>@O2ZKWNJ;|BB~e8;>SMNIxz~=%$ZI zaz59QC$VgUP1qFRPccqCXCn#g4PVx%oQnAczdN%x_qBV~mMs6zYm2c|Ume{9UnDxg zsJ7cf_=;2WtnRN&g!&7Q51EaIOX(68;3Sz7K>6n%ue2y~B*)^TN|Ln{iW;JPQ^-C6 z^M5V-5Gp!u59`cmc=2BVB#>Sz7gPQPC};y%YRhGN^!MwQ@J7xl*J91AXYPTxu1PiZ z&zO!=EjMNq(dmheW~4P3E^*S1&w`1@Tq$1^Z@F~^v031B?P^q&47XaeD4nz|))}xT zMrP;HHd&stE)<4J?&(lHbqroqDPvE0dj10;C7gKBdVUG2jpVM%1s{yeQ#~0DM!0a# zV;Ax+9sIkps%Dl~kZ)c3e0{{Mh049GFZufD^$ooY!k6+Fx3H*i=7) z-#c&rnlJwCpQNxKA6R>f6j}5N30Yl{rHV@={?HdkUwT~Pss55ua-|{Fvg~xVN#GhI z?yp6|#0jXs5hr*E`fy9K5K`Q7TQ)KtduBG}E^~g~PWkuvTI5TMC(nRn(fM9+|5KXi z$%njTHMWJnzWTS`Q(>$B|Jz>^5XWnoR*(TOqf;B*aRDm7@5uEN%80|_o@s2VrMMXw zG~p}Td?ymK`J6t9r$%FckSL;I8st{k<$sjQn^>x$KKOs12<-oJ0soyyk>crP)9QmS zembt~Ma~}zP=DYXj*oAEAY~C5`cMXd#4<|L{QsB~w4Oeenje0-XZ6LY0iANqrVK(w!5+>rWP)2< zp1l>2&IQXwdqZob87}mJsa@WE5sH)kijrv7?Nc6+Zl&%x2xlWiTHYc3mlppAtS>Dj zF&0(wtoq>!zN!*X$xJ??yWg@ban}fO7lZPt%W-ic+}Lzz6_$jrHa!He?9NZDiRywvw8dwZ!okVq z`STtHR0u?<5>m&u@p6jC^fY4pq`FNx5insO%qa`?I$rIY}{+Oj1MYhk8LGpNr_q zije)=n^9`T05%#nJR9$jh#2LhQ8v1(c1DWW(x_rp!I7oqKDJgP))3~XuwvnYeJZG46^`L0l?61~<0F=h^>QA~t{i%AWWG(~ zf>Pn(lGgF_tt0dpiFNDx+8m&mk(%I&e#i&Ym~}DqVCr{-Tb?H|BkvXELP$}N5VPR* zSV+VJ`alYON+ye(XaMvvov!1n?rD;dP zh5Ny8zw@hiN@BPi92@w8j+TPAh^s;%qPu$n2QR{pxORCf7!m;lpVDfq{r;*Uny8#r zlyb>l#z0^Al`}Ecs)UkNhZQqCoLf^iw&zUFl1XtePy0t(v2b+Nrlpd4Vu=cHIuwm*3v@nVWLUes%7Hslu{;f9J7w+&L>XYieu{yLXv3B=Ll< z=LX~0bcO6Dil4^%LT|`g&xDN9{jV<#vqiTbZ#$`yI59Z1-#e`Bgs5kdIX@F^`MrKa zU=pCp6h5rqkFw7d0P=FzS?wOM3EtAr^6JHG>gH%=N)wDR}P1yOF|n00b;qDnpz#n2@04Y_tzhVvWUqQDqmxEMI^jQVx$1{ zlSQH1%VgNtZD%h`B5e$c@MqX|Q6a_;E@GOf6SwHfVRU@#IkBxXVUyeoTJrCDPWl0P z6QK}=OoYqQ&~9W=`6I4<7L#7))*5NM-c5IyL4z`R@wo`M7(a2PWx~9q@9loN{(Lk7Z{`&?;LsGg|@`&4H^=ng7{`E zur(>S!}5C}?9XoVU{lM4sbr=+pUbh>8!F08qD5YFmjbYf35_`ZSRsQVVHIrhF}loF zwxP_kxZ-hh+q%hj8c~)j)b*Tnhjpg7!iyP;uXxKYkEn-noW}O~&vk>WyZy$4pSOdH z%R=L}4yUJr%Z1V_q=Z#d#Ce73L`1~9r|#?c08!85@7||dL&4BbH}@}V3%w6%{%7N% z5lEk#%OVcFj9h&J!JD*gne+BKAPai^%8B)O(4HDA0S`}gEUqNUM#LL>n8S-j%0wJ3 zHmke_|KmlzRJnu{fxNEa@0s50d={LTZvPO$!~huQ{V1>QO_7^_vy{LtSXe!X`9t^; z?qaiNMcWb)YEaujVy<|zm($&Bux}9VRjO?U&qKlD!!I^9g!20O#RF{(eyWtSkz};t zmEq3+M`wz)DWL}eQor(zVIbd(?}xdeelBW$GlUi&qyZbgAsl+ItB_Oti1H1SY*$Dd4+mznu zkG%}W^$PnwOjEQA5Q})_8rVKWxvt6~AL(VOuXS5`PO@BUZ_w%F>uMzFx40NoZ& z7t%k$FenxlWekMrnG2Oe6&(+rSBF%q-EIr(ac_=7N8~YoenH0{<5tcT_Sj1ObWXn7 z=pD}F|K#+FAnNURFr_<7d8Y2@Z|LzDQS^DizsLW_x&QTIv~JlA<*^ulQj)MpXpdr?PrBU@o24M;cbUZvhvSP*P)dR2#%NlG~+}3`EGW$>?a>87k z><0-wIUAzjaEhg5TmVA2cmn>6DlEkSHCnsO1?)W9$t*^)?)Pn2_oDio+`Iqg&CH#av|`Hn7_^87Bxnb76% zieoR67(dYnsGz;U>C<%iCW-W2LqAC-DC826g+Gp|_n`fZ`4Jn(zTHjcusv6g{jkZl zXrCC-lrw{b%YNLT%SfC+d7fI)N(>&Aq-HFQ7!xj74*gKK;T);jH%ned|kq#7fcdKGnFp#iT?Mm<4?DIZ_^eFl9l5L zQq$)@iTWKO-{<0$bVTmjgez#9{G5W`ZSsEoh{5g9uL?pu3NY|H3PC4&*z!C?s87V` z1`Axq2dJsl;{<7pWrQ6}vE5SeH#o?a!|mM7uCDAmf3w8R69z9w6+6@ITGviE2r)32 z|5zsBjR?*;eBkygW7>`^`X>G*B+S~b=XwpiGytuIc|8Ht0C_2K4ZPxaW#>;LzJnG6 zVplZ8hJ6y7|5iQsna*tAq>ANqvku}Vneie7VkC7U8GpKE={?dDN=Tq-zK;mkW?t21 zZp!u%C-wh^37BL46db|4X1VwlyH(Qo!yU}qcG~UsFtO|@dVe(AVz=?~J<~3NK+Q_L zkXj9l(0~Yx?Yeld>2&BGrnos;wxgem0nxmX zf2Y1926CJAIlQ0wmUruxoBFH3G=gQg^Or>XTWXw&H;mJ>LXW>MijW06IzE%&OMEs; zwqC&;D)$OHi-vF}Y9rAD(4(0jX7b8xMY$2p6u6AA*JvV=bW_xu=(o~A@gWUS4Z|_I z(uK-}z!(-aqYrOua9ki~8HYU!ZePPHSI6kl8ioLJA#&Qo-MYc6`$g^T5at!_p0g4X z;zw8%#KoK{%k$~^)7qLO?Dm?&_9V8*_vcbs**LS!zJ8`5Xq8u*5Rc6By>%HF{L<5z_H@EcksL>^@S`z%crEx1N zx7sDmqLM~vstZ}LvwQvS=R7Y0oS3jzZr z3LJO$4t%@y=`?DMf`XHmxtUh(&3H{9A|$mm9ri%!aD}@GDI2F@-fpz;^zi6VD8;Fb z$k*OQ^V^*eT9C8?kVbWqMFV$TEF7&kpp=c|C;rfzF+3w6Qb5yK9X^&UD$N)!8A?1IHc)P*FKODW5n8s(TMqSUa^4)eDMF~^&#a*yHR|4>B;F)pVyb#@tOxz%$5n(Rn$ zUGPDt5L2T|0yLsy-zQkgpuRCdg@22qpO8!#g&5_f7${EM-MYk?5EO`?oQ|kAJkd80 zPg`P=jO3Vv#YWl;-&cai0_w{8o+b7>{Uj zq9OhrzV9bIw;C=uJDBodWyxoiR zfV!#7wzJ18cOIom&@^dgcxsT#I)7V}M8;y*-^`?#$vYe0_B^b5=-{r5#(4W1+V~hT zj80F^zr2iz$IVj=>#PMgJ@^z3K`FVNBRW>_QvPO^#7P>0tNL{_Vcarp7P`{w8tD&B`q$o84r!Cuit zU&UiIGfu?b;lyNTxnz`zhOZ|&F1MPTkiBm*PCrra&74dq_LNN zm^3=?PmHE=;1{lJ8TJ~hkl{c*wtXAVpB?nJ=b_L}zf)wG^lZu{6m&#@ZZi$u3mKZ~ za{T$UEPYkRt#W*HwqEB4{az#s4nH~Ze`(Ci=5PA)+@T_Yn54`2dNXU_b2k`wo6{zC zL4dWg(kZtM}nGu+!es3RhK2_AoWZCN9Ua5r%0vt9`jFlZ~Tz`3f~h&*9z= z?R`0#74bZbNg9nKo(HpPiL_j*fjnPxcs+iv($J&P;lFn|36f2P!4ccrt-d<`>h{s4Epo;6aXJ@4Jk0b+!?N3C~c_=hU%Zfa6azrd31x30pVhr zU|zc|kwh%r!&$=~A5R+IKjWb>Oakoe+Vfsl$%K|YWhcKM&%1b%sc_=C&e%R7Ny2As zwAo{ZO&)Uin|zkMy02NFpCEn1*#L{UB{?%}Gy48y#uoB4cR`1Y<@4gH5M`fDgA5NR z6`?UwMR#@)EnShTu|x=tiT(_W!Yvd)$c)F9$Vis^aMF_1NaO_*Kb*Rj+TZNP)%ZMp zS6OjbZ7|-w?mV;6f3b8pJdzBHi?NIrv4YBB3%yiw4oU|uAMSs2% z>aadC+HTM3ybES2cE799&$HW5-Rsz*0j76=VSy=m-YXAoRBqPoNSGJ!N z?}meHVhG+1&$`>bx<1+yaPjfcCmYn;9z7wstoV1`!X}X}Y47fDulz3zUoR_dTVBaL zrl-svU$447_+pOB`YeVlmmVsWv;!Gn2b@kBVzByPfsqhPsduXn<;GWxx!ZmWVKj_GNs#o zcV8;g-rl;MZFblG(0DJum?0qTh6^{_M^ zcr!#uEK)2Ym0iEh+4H0YlT1Kh7WzDw?I8O4?auG6R#rKC?3?tOJ`aqgp;YjlmhLl0 zLyrYB2EXvt?dSHuEzz!58RDj?46Z}(ZSS8Ry^qlPq(eBS4Gmgw9fPJx5vgUrT${z= ztxBu^8zV<*n}nbSY$E=~Lbva&snc8acDoU`E^aq$P$)va(dU!Geb!X_wYr|lMnTfU zbAOk;zZLLP(HrKNhJ39Dr{$FU9xxZ#KXtkfbdf37>;Km2o_bTI!cYjfYWDY46@mXDMmK{8#(< z__moV6l%Tx@di|3p!7-1bjs#q;4*V%$Djh+)dOa(c1Sb+b<6B%*3#q4ZjYz$JpwPXhISsB%<;cSRO=4{cwOp+DBYa4z8L> zNdZLaJt?8g-eBB{&)*ADkW$|Ibre-^JQ?Xn($L`qwhedtwH9b!2TvA*=0vBHgggoC zvi=X}uipw(cduxK3`rxRJ8)TdI?p;zs9rCqP-&RSr{2EyzFzUR>&4>a_O5O$FEd6? z6_1Q+RzLW>Zbcysj&{5`>wNA1+&aVOzNac5tDzjCktG*p3Py4=+ztucvWKNB`u@!A zw19r}UAO3z0MSU>GjFUcW4)LuXHNH=_rqR< zU#3@q|F3@v_iK_?SBkUalTOF_T3zwe?QO*S0EiLmxc?}sW{ zs9yiHheYYd<$0wMVAG(;{WxBLBTq3$4-0F6LiJVeIASLQc9p&?Y$9F3$YgDv&~+i+ z;4$WTOw6Q%S?|A%6uKhNhIXNu5Oi3DebkiXq(7PUz-|ib%&4g3`MGG-#I9uvd$$x7 z4dUt+2a}Q=lWpGnb(p!}^#6?QY0KFy#!U&PD<&hQ^fTQ`o&Kk3ViR6kHGV6qn#ucL zH~wLF9WesMNGn<%shA~|qna|Gh8T&*!u$91OWtecnO7X z3f`pzwE0f!PPc>#Bh9G1@Au>9!xss4CT?lVY0v}kB``kGi9`~gF{3-@zLSwWM5%-* z(W@5z*y=ooOcpn;*|Ow(?r^jh$vfxUIMO-L_RSCX;)uaR1i|IoN^!TG-xzv$GX`;K zD{rxtxDHDZIKr4sCrXd7L(*oQQ=aFst_ooSW1$x% z41~kT8LoGlv_GSrU~ioV&U+-}wZ+VSGQ`7lum{2j*- zGh78Q_iW{!*M4?%bj%yE2!KL#ipDVC?pLQj$V?syo&C*Ap=b<9#DR2_N%egu_&MleR+SkBv70xaMg_QJe$u_pu zG_;lU*;e)$n;@cOYfeJ89FdLYoh`Bxt|QxFeQ{OiH?w9#k-!)|fH6ve_PhCrg&eG{ zS5^Ivtx!5CjT=sulg04xu@0`--+9GmAaxjb=V+k}4g@xPn*fvJxH@Xy?G<*=T2 z!cLNu*lf)@5SxsO+7l5HaH^2g-xvK2-qyTR&GgIrflxUDwzQ21Xb&6=x5&-SBl5Dh z0E-~4hPROWdB8UF{mMb>f2(TtW|&7Eci0O7NsQcg;fxTU`88v6n*etQlJ_|U4qZ!* za!=u{v;pLMuBN!@SPC&&QBK?F-t0a+Z-lmzRme!m0(0|0M{HbSY3YIC(@rfD?_x@7 zDL`uPhIczd$n)~49gFwlN5ZOXE71$X+8z~Cw4fx- zGspy55dIY%6bfarSDOjKm*uHS)-I@nkB^XGnxHk&BbWLd0y(~8zz!(ga@~c!6bCZs z@~R*Ielwo%c^r2bp<3$GAzONsd=ntn;3(88nQL~v&GK_wE3QD_l^a2Nvk*(rV)tW53u2cW!}*j1#%Lr{HF)qM7en9#+c z>t%gwqm)CJ^X{L(%TznTE#1xy%VAX78sjv}=?=v&)g7LVV<<-PROIB>u@3(m+vc7m z*;wi23{`_zAWmAr<`6%(n;#^~P8tpB5gwB5T8C-~qH zP*a9NL7{W~A*4p11AZ^;&C9N#zm2w{A{~t6^#1*O?!=8cr|$4cFYF@tiR9&u5{@mK zkpLf~F)lT|DsfhcJk(+qZgnnSH5Ak84@))&4Xw(BRyL<|b90Loyg?Y9;W)wEek>6H zyNr3;`7Ymu(5#_t_2rRwFSSc`QE)sL_2ZnnqA*+lp40prA-nIB){F4Dcy9W>j&tt zyq3Tma~&N50pb(Xpx)YQ@|R{)MJZ0 z4UO+Wic$~&5(2`SUJoA(V~;0`1y3!9_S4AU;Z^SIN!={k}htte;cDJMfoUhOZGvZYE>8--u z`WK^Sl4cBqK^+tO_{>PAgS(?~qOD#ACy`5|vj(iLf9F3$>NGxqN<2li=apvzqq1X?8G^kXMJtt3@Z3A!jY|L2Qn|DC5#<{8q1j zuQu;YA;H=2qt3&vHLNDL)%R0z_#4Wt=g`jH=4LRurJ!3-OofB(dhcC1g9X0+3?zz{ z)2TQeU)8OEjWJ}>knRntTrf=BuW1Y$O7Ao2c;>&t)9Y<^xSnryf39}0w45Ucevm?! zY~QfYa~quI3iU?D=|(Or>&f0PGX)nMFS3)a=+YCUn0TmYRhaCG1**83k`$Wao1J*k zPMAGn{l5QQKYS%A2&1O+;}w$~tJ~MW_*bzQSa#dp7Q+qubaX%0xL|l?Khx!FtMlGV zaWy%+uTQ@c4mD?W8!o@^T0H9NFd~TqXzGWz&d$Qlz013wHMTCr`pHix;M3g>p;reS zzewl2&$`cNGPq!AX*}QO(5OWGE-ywkYqjek&(~`Yzy(gYz+&e0C6|{oZ;L$ple%Gv z4|#%a9Z(j)-W%k*)3vL=iOZ^cU%j7>fs9U(!Z z0WnEm8G1>=PBQngg6F6FD`qG)lnlk`WaAFo%^vrWCGC3kjk}w|STh+sU)H^AeN^j| zWYF6aHq5vDcAK)|@~Ad@9cFjY55tUqYU^4o3I8X>&~wlmq*jBC!tqc-G^kYe80zfJYC#gmX3#~G|STN+B9J88#^`?P|l$wFOk48iG=O>$nED+`A8-6&|>Qq}| zYY)foMnzdSFR=F6UA-Tiqp<49e$go2;l=`sK8)ghJ!~^h1 z1Pqkd4M5;gusN&xP*&xP&(A2-Nx!}%f~zqxhc}HJX4m?oB=HGb9M-t6j#6BR*!=gq zRYbm?KaXlQ&E&S$S$^QmhWcLzm52=vEXR;JWa5LFHoM$(bsvwavpxp5zeuAKz0(t5 z5asmQ8S5WJz6}G&$x9=W!929~?LP%)d#OV<%d&WL>_zLXc9#oqI=`N6GAQcW8UOwB zlMeZo%A6O(6G5Gp+Qe67gaWKe#3rVxBm^>pjA&a=#ri{s;_-2`6(mEb(GU=Q^`TT< zcP~YgX+lE&;xvm1KYzazJ*UFghuG=zLT|4p+M`lix*OjB8_`Nq7t!?V z5JyaZSK@>97)GpM9`kE-j*Wr2(rUpCK`&I&0d2H=cues?mXeY>v)zaHE4_~WLHCoT z_*dI(}Kn2iiB)_*WW|8&RouWUJPZGn{*Y@YSz1kS#`)VBk3d$ zy=O6Quw1k^yR4pLrnh|#La5e#>~~wBvYibqrR7bQ@;1ARx-)sbf6qG4ef|D~gLIb8 zwz|veo#qRrt68;{Vxpt3-SXPwDaO7n+i$*X?j8BxLWu?je%!KUZEUEZV$fljnx2$m zx|ry_h-a6?HCzf^uL@z*gIi=M7e( zcUE@Zhe>Q?BK49)MC{sij-M52@&x_v~1?tmFWMeQqwBN-|r(} z2|2>@pMUDTC1vnxw=LVt%TO|ZxX!}z_nV`|!3BeY{;wj0h(2h&&(m9t-dU{<34>V` zJzoq6AepYGEpB37zCVtBVxDdTaq=SBPqeEyd3?=za2mD5*5?n#785 z6~#QV*ac$P%!T8otI34;;UoF&L&VMjD94F{WWiN5eG%>O+>SKwztP(KGS@ar{=qrM zQbXz#9!(2C?JqT>FLq`3-Bsjxe*c^Q`nzB)0gx_%Co@aTqofbzlg9i^gntnHywnIa zmE0H#+*_cdmUw@vIGUD1A~jmSxn2u>5?H6MS_Gv7s6Sk_KLD*Hx2)5i1SzvOP zOW_5g4nX0~qAy;|1rZfUF-xX!__hy}uSW3kzv4?^#~K)69jrx5r^hUYMjLJ1jEAg; z`;$yz&p0rLG(0?e%{s;?GLh@5{1%s1y0A;l(_4vljt+qR*^KF~_7l+fbTTY5oN>?i zq5j4jiyASr8s;2kz@l(n24A_!4cELI5*PKbSb$hI&MlRcTL|ikX@o zFmx1QbIDRpRXU%Z-BeP+o>RYxA=geLmL}XD0G#a# z2y`9RgA*Go93sdQ808t#iV0T$Q-~{&c1#k2I^=*z#KsDC)H+7aA#_kT*A$Wn6o@_U z<|HT_^PSP>0^2GXGANijJid2{CHMh}1A-8{Zi?cUFJh$|bJ^hU0wsZod7ASj!STaap=}`cDJ20Y8yVjcwUxS|cGmob}G}N?NmqoR_kpdoXi*w1#u)hO%9t<@aAiY`JLG=G3=$ zcNQE5ZF}`!psV#rX;K%NI-MU9An0AGkn{HG;)o_Cn&t8bFuoWZ07)vMsLrv%cZFh@ z0D)klNT*W72J)ACtA^p+3msmqunNrIT4Sx4tTu6$g3E9QUXpwl5(p?4@T5zVF|aYo zL04P1Qu`9Wq?ov#o`@*}_GwCKPCjNbj}*m2 z1Mp|i?y-k_)a74mnT?dD1C%1D&)eGn$%78P_93NNU@J%bYv}oNdw?~+imVqE$n!a6 zyAlE;W$Hu*lq==~@ac^}WR?ly5<(NtW$;K{g{>6WhX)*Hnxa?UmWLpiZZ)$;#h!{7$c=I`SIC9)=5v zB^5>}$x!ms;7NoOCFYHYrdc?<%7S$v0?scoX%w@>baw7l&XN1OV(f7obCGwB6`yuU z-5+qQqCmycGf%9d0pG*7x+t6p)&Bc+~6tS|q zYeE{XkezO9;umT!;{MeFj4`S+VjgQYv6t$d*HCO+86lAXnYGn1UCc!j+28v38TsRg zJ6D4%bqlGvEN&5iS;B1&hE}f=!v7=_E9w%NH_dIi?})x!flmmX6aZYtFPmX5?FFzF z{f15@7wup$Kb9ypDdnpF+)*UEi+$k&meK>VsMirpVhbW(|2y7+ndM>l;ymZl9k2H@ z?I}W`KaGX%mIGTm{jVK)7Ih5ofD>sP4>0zFq#8InvMZXa<%9n)aevTvm-N;4gKGn? zY@WNSUt=NqT2*dR%t#7s6*mvU^IbWnp#$Cho z?7^dufc0PPkdEmQ(p)WE&;Mz6YbaK%4Igy2AMp$JuSMP- z!330&GZNpDxu@q{YIdP3h8*}ZQsb#h!hs*S{rj_%m{iCGobjzyn+sKGIi=FC#=a5F z+)?s5<`*8jf5P2PO5=P7h>uL!A*=2FoL4~f`WqDubXMHu+guhozuwklrl(kS-1(v6 zJnQc?!4iY3x#-zi#oSo3n|sOaC>o*0u*VLm-ELTzNo`*G{MckS{SNgP*5Z*%YSy?i zb95;wP|cLKlYo;KMWG85$*#^27uVKffrS1qc2|?hzbiIP55TuPDdsYXHw>&YGJYuTf3Ma*=cFK9J)<6NpL@nS5ZQ*Dk^Mvs5s>J&rrkBF2AZ~qm%_Ymr_GT?Oo z9mq=~|GN45lcYE$gR-{gfkk>LV-*)@q#?(Ezv2Bt`-WXYUm#t4?~vsUX)U?7*o@vc z4ZX~C1}$BI^*k^!?j)s&Ar`irZN1~m4pRNh;5V^iSTem73Imfq3)_2bB%_F7B(uq+ z;MW9~IUegGHm8@&E2v|h45Ji43i%OP281G68L$Yl_$tJs)d(KvA*NIQ`DB^6o~Otj zH=1}HU6PRT*Q--Aa`Wq#bh*i0dVdB6CLU?wFBOr6HI-pHlE1%YUZnkvk3if{wD9`8 zV&;mw&Ab=2uWeC*cp@#dX4PUv~pjm(#H0uwlNKKd`lfDl3wRT?v967hUU z=ukPLj_MNaA`37&1P(|;kJ48ah&_rPJJcW{A0e$DxHkugoL&8oKN*q z(-$S#;Pz@VyY5(qrCy7e5g33pcT#tBQptw%Vt?}Bg$V|Zp?Ni-m0I_e-0r@I!;7i- z;8YpuX$S`eSr~h^!Z8VEo0MZrFJ6$3FPjM7S+jQfpxCeYi}2B7iRqK zM||%@AAfVTXs>#g=VZW67`Cf}F?IU0wE@gNdo?Tk z2+P&1Jqt83NmgqbORN`vE`mkKWqaSXe9FuIO`h(zalNEyj!-t!BIE%G;)ir&*R2P8% zN&h6#OohY$>tRdvdF$OrUS8q5Y6kUE?i~#*fR-$3S_8+$S4OepFl76=?e}{h`fh$+u@ytpnNGyqUxxc|tp|*o|;YzZeMC$F=jP*D8 zcu3m+h)>u0_iM-l#9Dku2zpu1BlyYMhf!z)@45tj)h7I3^T!I;ih_JG;N68|JPF!y ziX0f~zXZ*k#~rNz@?bFFkC+i7)&aD?yI=mT$q@{-y1ZyvP-pV@o_Rv@j&ogJ;c1%5 zYqS_m{d6L#21X)cE;=F*N@XG6(%SggbQ0G>MQSILn?$$*!*p8S3GBZaZ|hL3INRM z;ibRySz2_A|30>)2#6=HFyK#9R1XXW6&eMrrLGNCh2y_5BCDR*S8+EJF>thSm7DqZ zAae2bg|FTG$eiGS8ubuu`QjKe20;=hMFfbLGis-bD z@XN)+>Ov#AfI>Cz{j)z8Cq&0M7C=dAApJMK`Utz8-S(!V^a z$7uZLancH~A*2U^LB-WQ(1fN#4kL+ z2PA0oN~PgYTz##fiXt%4y5681l?$MGcT7WwXP_f@*F%}MR(H+M~AD&slLk9l?SyfT8kWGbF_ z?|&MDy*+=b#h}+Lh1QNe#xQDB7%-92bPT04b2Y0iuHW2p=gd*gn~lvr zrZA_ak!uRThm-s~`X`tj1M76F{;GmNQKG*9FXzY=Qc(~K_w7Az`m^o%i&?c|Q$UM4 z(`x#|Na?e89GVd@U1&d4@T_eRF(ZsJL-fl!=DFXcCNm;x9EIrRCd5-wQ^@`4x0vE& zF*OQc1`Cry=&R$=&T@fXPqm=`oqp?z#c`K>^C6@Vj=4~widtJn&Bz(a$FGSko|>z|IGAa_4LFD-hhtP;j}I7NCS=A-2YM>as7KMz z49gXK{qyTX00yZzNTG?Fw1Fi#k`bF2AX6=#s}al@1VEGtY_>G2R6z>g1@KI4{AcK) zj4NYY;gq|6Cdwruu8#l13`& z_3HoT;s3gyry~uMm9f_L`0ebGP?yw^@%m>@62aUwtuSWhljCa#rmmn@9H(tU5G@G} z9BMiPKURc8SIl}}tnGHkS}X?4F=li&FN<@4%KDQB2CjBN%@}K-*w6&80BV!LfIU1g4 z^uLI=(2*_skq1v6uzUM_e%{-1DU;V=$i+b(Z(bq#S|d;ekgL&PStwbjL6UEZV+J0d zWR|h}*xrwx@DdG_4p+05HgQ|`9eQz?H%d0zceCNGE-<;D3f761=Ux1|lx?l`l;s}A zAy&HqY__MbXny}hjz=IT3sW)OeE{OOQ27R#I zhXIAUxC$0e$I1S%E5ehg_0bc;K|w}ag!oMeh#!b8iK; z2p6mV9$!+f6y#dyOM{QV1_*EEl8e?UGU3KI5vNxE3P<>!`pvTCFA+A7Z$&;Ct9|bJ zQ&e8ppMwpDQ{m+=pnk)bU!Ab(Z==q*v;1BsLX8K2nDy0$ zzZGSzwz^EvvU$zwk7bz-+Sh+Cu|C>@H`~0HcJu6C$}I=Oh^ko#PY*jixOwMxq_J$* zTfSHpzkDZ{^zgqnhiq;3?nLm|h{_#y&f`iPFML`+9&nvpOf6A7+iG%Bq~v8YSnSx} zUE%gGP#z2|eUOf;ezJTf#3$3NeK6lx+bvs%O-7u~t4Ll&z3-msSKOdcFP+OGW!|)J z0>hJH#i7yL$uDH&;0jfsS`|235}mZP`2-<-={y zooKR8)6d+H>6+&sk6{+w#R0^3-9Ma+ z6U`ETiT*G`{AH^8lMl^rmXom51ztPLW<-A5dR&3u7Z++S&T(VLY2lGX~F^= zoSq*|EOx2uNZMSB1cVC;fD4#}PHXd*;5TtE534!l|3I3t!T=~l_Hlia3b=eyVxcB1 zhSX$rp=(l?&R>lyWez zHOIW7_@yLXc2*kmy44NDpm*AJKVF}DUu!4R*k2C)g%NOAU~zSb3TQiM6XJP$@1<|D zQ^?B`6eYmWAA7B6@UAo2Yq~4j3!AEj*YnF>M?|x)oVwAycSYyZIf6{p2#2#dbk2sM zyPW1e(N}BCg`k}4c`X&x09bNlHrkEFSZY1*3hL4K|Q`YKl6U>_N`6HRl}AF%zM5z z6o+MtwbVbaw@KsgUQN&_$nlRw?kN_qNJ&(oBx|73g1U3}kWI?Hs3YPvv9QC@vYm4Z zMVA+UOP>UdSUL1QOqzV?-G6Zq&xyniInHK+FJC;03+=qkZJ)-F#K*wxfL4ZsL%7=sWk_lzh9o;xFwp6N-)uCqqTlUnqu|&a z>0@{=S)Kr6v}E|zCBdvG_;K?j&1$s(AuP(_O%=L(CZ zhy(&HMezD&)0^vR8!<)jhY(djQ?4zcO^!#GOGF%o8J%Bqi}X|uZ!{FLy^uM=QKdCi z!m~(E0y}!TOy3}9l7-C}ak{Xi9NYFf?W}$%$9BBffuRZ>6Y*<4!nPgQ-jmKzpKpBR z(TkzGBZz?;^TgHEI2zT}@`1 z4v=4dbATo6R&20;n z;I*#dFLz@}g-GD|D1QBcIw<72DIrT{ zy75Bk|L>U)bG^}dy$fUX!faJ<7G@WAV%iX51a}YHdr9dh764|55v+;83MiyKEyr;1 z(x*;M#mGsT(jkdyv3g}GpYAhlhMHsJ`~)gL?204Sv@i4oP$yvL)03j{USAcb&jZLK z3-;jIAtM0%0(E&sLV6@?{K6(vI4O;DU~q%vhsCOaeXEADJ{s9@d3AUqHPSuxWH%7K zSzKM;4dA$Bo92?98dJ#g|B&^TQEf%t)^KolXesVafda)nrMLub(c+Zi?$Xi*C{BvI zCRlNIcXxMpEB@xWw|(w6#+RQNBj=o)z4wy2=9(+#VzEdm^BU!&OFKX`EC^|Y=8S_1 zN~$c)8Ys9)Rw*zlP57-ij#;r;1olzoxxzm7KYj+j;ET)ONJ^ShVNuFg-MRyM%{f!L ztil~txuvmh8f2oA$EVBO{OB})eEaNg0RU!e@kj&6` zTmCIwQ$y(`Yi$a2Wc8jB`M(Udykvjw&&%fg1iSqHaGr(;z^3&$k12W&mBf_Beg~!? zmcMCLLql>H-;N|e0_8VhGI>bKT`0&Ee#LBy#cYkh%)~}Cz6#(*#zZCZ`TAlt`e#vG z7+R|_iyT9MYGy%h%&GWjEr^5T)=%rnb+^HKh3S9I*v~p7qW>hS+k%r6>d5Ncj6CV* z^kHnVnC;+wxK0r*ITdD(17XowE;^x^F3utR?b9oidyA^qO^*JKR+EjO$&p?KBU$Zf zkc}}y^O8!n6`p^k>u8w>h6e+1J{!I6yQA#~hll}5@dYC%d$C^eQpsVcD`1U|kzGJl zQ4t)#Y^owaHBH|upA1l_4Wv1-qdCE_ipfO}7Xth0o}_6)MrKqZ!~mcmK4mc&`{%KC ztlNWtK;qBGfkzcGX1a>XyTPlH&H5odftY-ILtky*ef}=1w2ut%&Jk6QQ9$lQ#sF*f zYGjZyKE6jFprwoY{00y3Dc9Ic@Vh!TOYFEl3n4u+Kh{)PLBONchGKjH%megDg`pmh z!X~;e)_!geq*bC~!*yU0q;z0GZRgo{7G;&K=R4YQuJ82;US#tx?tba!{F9KXKAd3V zM=6Yt<~Aa}BK|urSZgGs?}C2CU}_}$%3ZM>_to#x@V|TMA&`kl4yY!}Bl%YHH4qmW zEshES#F({d`GZd?AfdMQ^X)pS3S7OMPIZ|;2q9xV>-J(>#c8i%=OfnU`dB|wCQPEn zd91yzLa)7^*OF_yO}x%YE1EvbYN%XDPk;j}r=&sRWUbNzHVH*g=Z z3=ZM`oD|DrBKJ}XdJJX?P8TB2mT!DgUHUd#x0o^) zj^o)(^C3M(%=@G;a!KU=W~<=0M;lCD-DAt;bso3u{)-t#;zYDx9AB&>+6VHTqOnAU zGZ%`T z2b|m690fPJpUM@n79g<~?5d&jq37%p&$Uqj5pG3;f@EGx(iSIXMS;HNOR5RK%***E zIGPYWDa^)lu1_YRris|8q0pL@%YrABDmz+G(~|HtTasc6iMX&E&Cmb@3{gj3 zMO`J1U=$cdk>qzH%a6L*$DjiA5_MW{a+6y5Vx{r;-7lFD&1N0%dj~@+L(F}KB628X z&{(H#v%FNZN5U)$k5&m~rL=AZPY44tBx8YzLOYYOd91%=-m7?iHzp$x)nwW0=KPkG z?UQOIjg9(7ZO8eYG7_@Gt$#N3_|T&HTKvboBnDv0Z1HE~4a^1=A=?~vX?`bu>+&{I z;GbU^htG~7d$HwlGunnI(9P$A6Zqo5$8P7^HdhCN_mvgr3U{HxTeS)K~%h8{QTd|n3i=0rKXLcV^VV5cQ_q& zcG*a7IeAy@ossEaYWbU~zyI+<=+z93x;Zto@jF=Sm4S+(lI*sTlF=57n3CdhuJ{c9poqY*Ja6uXCAxZ`|>|d|icOm#Ihp^qV|=yu6IN%IByJ#Oc4` zu*WafdTe(@mJnBUX{fM7MuGJ$GG@K13I&P|_Mgq>zAo$xS~b2=tL^$MBErk8YH*Cj zWY9F{s3TlqvUjNZIyJ1xawuO7nU#eJ2t>&*(>P(kW0r@|NrE-y5#Qf%q&Z(L_C-q6 z8Jf*Z&+*1}pLPMU*QjUv8Pk0)-t0ydsl`{@e7X_C2zG-G#zilhqB*ATJr;$y94TA>I zgD$#H&qu62GrdIlR$ww%3O#q+7=wpg1yG#1cP$dI!L10{8#TSdiP zR<+PdVN#%Uc9$zm)}gK^)?Mr3<(oR`O)*D*@dZ&REl?eT#QMg^+>0*WMOKll9)&9S zONm5$qL8J+tk*jRWpHY!Re4slyo}aCCI~(3^Rz{?ThWjB|GKbYAZiGy*U{n}IOx2s ze3P2Ys!Tmon#wv;Sdv7=mo0vx;Ur1;s>IqCsVSrKJ-$tbF^$#=>0lDCd5q$S5x0By zVFPaCvpO+a``zAL;lkU}Ikl8Ux2AN#)SKV~u~$-Td84r!-1r_jV!z=Y!{DBPFIs}f|U2}ut~&F6tpZCH94`3|836%$G6c^Sxb5XJrtON z@+qI-IcWQ+WG9kBR8B=aD%BA4uzrL0LFJasw4ukdLy;>AE|$L0?NcnW*)SiU*Rd0dHaf4HEw|1KypEw;E5X_Q-K1$=i9>Q9WSkv=C@JeZni_^w+Gss;4Ep?yjfng!G=3g zRdr0DjOw3VrJgh>oDli(=KAR-!l>W;>fz4&7l%<}5^h&m5_w>1A8?ueop~;M0X{A=~rLpDc;g0ao>ru_lw=n8^La!M%A^T%+j5 zrB^7ns11OxaG}sxd=YG6kBAR~9Ob5=`wT=@fe2$R>dC<}fvV2iAxnyPXjrBGi?I`Z z8_(}~jM&0!r_aVzE%`0J6@eiCcIX*5Pa-f#^P9Y*>lP)<3ppSs$t56Xwj!uCYmY)i zQk2x#Ty~sMxr1U{3CN+mqM_T#|RfD%HnYXl>{vF08?Hms(h$+HtQK#%RK>2KN&KzKXK8 z9XqqvSmmZZUu4q%odP&8xMM?SL#U`23%^U z^oly=P{M0Jh4R^6xyatPx#aIjOQdL%$26eDw^&RXP|N_eZ(jb{Qkn zUovKYZ?cW`f^Q!uO475Dn!U1qv!8UMaLLbaQpqnY`pqS;4XY>g5(X-^!ZqDJZq$VW|%uSlD zN%~ajuHH1f;DfleQw4h1EO#Rjh_j_$CE>%EAJ7_fK8Ch zQPp=-q)?cYftZDd$8JSY{yB&aq=kZxE%|{ssi?XUwy96HViN3LotF7C^L$5*oazl~ zKrX>wFzM&3mqhxtAG=vpQorfMjABS3u%N5u(;>IA1_HDR*3}K4-^rlb2_vX;(%rk(4F=1; z)jW#VTxr^ zaHOq0cfi@!Q6UU5J_V&gi=SEc>Ktu`AQy0&3R`c?%5)>)kIALxL-$x=m-=iU)Y^10dlGfME2_0qWg zGvQ;GhCftO_)5zcw+NI4CLWP^7M0UP+5DCg^h$vZQ~S4u<44#h1Kb;it58vZDmFfm zCL~ju1|Oh}fWe1P_n%CZgz@FH|An5azKCbRLAN;qZ%B|R;}kTQrbvFVJA^Le6C?CJ z+bJe~A&aAvl2TOX*SI)IH7#YsG~|#NS3+WZ9bs*4BGG!~!T%l~p6!dbxIIJ2_>}6g zo|9fvq{0PV#`!W!%{lr)1#u9KP)vfD#1Fv`f2a5%bt9=|M1w+SN~!3<4?L z0Ss{%aG1z@3zUEnJps-DzMhukXCRRs1b^^75~ih5l^SIIjRd$#!PZtck%~$&F(rQW zPFp?dw_>|$$^Pw___ul0!Tpc_d=t+$7OzSoFkAMU-`N9Jnns$3Zfh^}+j_d+zC5-> z4P!vT=Q??@jEC0j$k+Py=PiEcBey0Da6YfyQY_x_!%h0*!8y)?PKg0;ZW$|$e%-v+ z{v;H_u052<%N*fE?dR~PHSlJg1yvV3;2Mp7czwKd-^L^@p`!XzblnKXfnKi#qJyoHX7KKYdXAbu{}ZIdfIbo3FC1 zp0BATFZVEpCvkhaD)LUn9otJyEFkK#cTHx->+u|NBl@(n^3(TnFND2so|wS_xyH6< z`~G<8ajXH&$fgM-;5na-C&5=`eomr4Tkkw@HAh1I zfbNmsEPJo-GABMQ&Lt<3u+I!Ocn`w|D|Wc_qqwU0Vn+b--;AzIjCt}#!kfiE>%M0z zemUa*r!AU8x!K`%f3gpXll(Saa7q^QwU!hn=n6+qphWC$t!?GRn_il<+U(>C-@(+k zr#I#`r z0fB!GzZ)K9sz}kjibjpy#L=4ffWpWdG7gD#y6h zj=P1^e(S|Ds2{6?&yU;F^rhI+)B>BjxhvZxYWB=fhiHXMqnUSY<&+1r!V*X%Gc@6{P9^W5)qR(@XHEJS9?t z`%Ne#)Gv3 zVzupV1KVny?0zEmBc21I35yPz{sw%crc8tIoP=Wid+loFcwNz?z4#IRG$}yF?`jdQ z^sji!KV{nj?v2jL@whsN98+kQER>n_J-}61-ib5m>}$MLf*z{hnmjh8!slwJG&!tx zu)edZFQH9Lqy@%p5@F~Pph|t-TTD!hunCrq5IpS!ZcbYAiq+{_EH>?4*4e0ZLGc+z zS(5jLUWRgN*Hn4L>qY7 zk05~UM^`1G4?%!>KPQOs1$QT}P=Hpiw|~>)_8mzfmr--v%VVzdTbsuKrKC5ePg&+& z8Rn07SI!nn{!eo|z((Qtfxg9CEQ^tZ@v$+-!^zw51-C8d<|Emelw5Z5iOdN>ky-!C z-Lhhl8!Lk1_}9Ok;nE1=kC&Cqk_Mi(h-bKJCc)$<5`q6+1*JKl1)`Wi0L0#{#RvNE?Va@fpVX78b}autuBqJ8r( zS{fA0TDlSAe$vsBgbir2GoyGZz#9ip>DoNl~Pym65^15OShZa~& zNcBJcRcKp?Gwfv(bW-pl^|iaiUE4v6*Y@Gu#Rt1{Vg}bzBu+%NVhuVvbq@sg{tLN4 zS{@W>{@_ARxl7kyMiwy z(?z1F{a^$us9p3ge+ybBtEfuUY&8HazEm6nTnSni(qC%ciAjRIDp?RGpTm}89AoAg z6bB26BOua*>*DqF4w*f`9zi3GzlT9oBwejFfu^ju|$U*9HEF1VkZgY8Fk zA29Rh5Nwqg)SW(5aG8~R=F)s1<$;<2#pUXV-H-a!^->o88!!H+g6>$6`PzbHixfvK zBJ$_hX*1tf7@^jb2bCj77)immZ{5wul%*Qnacz8Lx#NTxMY$T*TO9dgmZifbFzu%E zd3Q(VgHp0=7>frV+MXdd7^g_(AIfl~J);d>uV~xkvXM{mQMFQ7Sc`laJP0eOt`6 z1l~wUgTUpvNS|XN74LQHNRp}5-<9+4XYzCB)5aYr`0(!6smY*@2Iyv~zwB=`HyA%7 zXilj0+;V67{QO)5gRtNeXjuX@BXV$_U$qv)9yF<+$QUE{aKjU?9Nv={{&}&{)6Z5P z&f^^1{ibnWP3TAO+apxc`{jm#RP0E_!2c|YwsY@vc_q-5J$qF&@a zLe(d3@$Ygri!#+RMdL1Lj``bYe(jPQq37KdpzV{;jP}BE}VDxqWtHlZcGIoq)FoJ3(U=&OP?>5lE zBt|gRp1Li9zv{Rd=N=M6*`L&gSzFholmnDsT5OiRujCjRS%eByy;8~|NSMp_IsN;O zHxw(|aPL-Vq0q4KplPI8~ z2L{zIc^|J-yJmlVkAJ-EeF6_RzdBiM6k7b`kH=j{$Bz`2m7-FSQZdbUEN5)y<>SCr zvnE9Q?Nsab*nM@N9Sm+sO3#0V*Kv|Lk{m>A3>k4U{TFiIE-DQ=@}h`J_QMkr4nV*_ zBEL)|EUovk%SaRhuxX;grC(n(s*TOx@BAB{IZ5kLWbN{JS(cnb&;z8wyVF8I`RHuW zr^B4js#9N-a)b{hr*{+i-G7zxef**#PmqIn*LKsgG_mlw5N}At)mO#FxsUSGWdg7H zCe_69fSv16_y=EW)rnV}76D|SRWlzS13;&Z7s1kK`6z2G;l3T)!NX%?{Rk<-*J2zP z{sV%Qj0D54WSdLOB?qlN2!vS!4DnPFw6(1OoTxUbz|)yp4- zZp<$QvXIB4wdxO9QvSYXB~4|zyJ4dTbL)z?Un099*e^w&A1>rg!q_mPgXDrWiQ&PQ zAY^~>dro__23{)>5)dGl8Y2l{q9ZIlRaiSkVNB%j!#;( z!=)-b$BP@FO&h;pXm){&RgWs*J7MO-us@NQn@^bqf4o{t;L7~8Ms7h>$Z33Z0r4*1 zK34v&bv(GK&Dx~KTIY}6?2?b5rh@DJc}n@s18U*pO!N&1RAPv}x$kzg*nL}qhesLu zdBl`(tiT+m70XUU0K-~XIj=BbDl(|~s3q}uS*!iT`wyrw^4`E_AZyCkt%F7b3cd)c zS49{8%vqZ%wqa72M94?11tUr#0s~lPLO_5KHsnvOUrEF>=n6yY&6wep2koeQVrvSu zh|Hi-dVD&`R2^L=$njIdJGsp zmQF}wJEW12psX7iV^xeeiEw@VMSVRzxe$W4U?38!bzmzdRfBl+Eg>|b{U_vH&HSgW2`HwB4Ga_fF7J{;QOc^K<94Du#WtS?gHB1fyar2fM)YR7$ zUi=x`fTh>XWd1wg(4^jDi;>3D0dQ)mN`X2f-HUXP=N**!{lb~k6#aj5TmR>QXFae< z=H)#9L7Iovj{n1vlL>=XrF#7kfa^V}H1Ca~y5@x?rKsydy9C zfJs30*s2At9JOLl?{(GugW9OvY|sC3GyS4V4m@3A_I;bgJa zk2jB4Va~Q7j`wC$6a_v@S+CCI)~3bVF{6f^Fny6UCS{UCpjSpnQH5AORCVa0Q!PNj z)2&|x2u3%i1;M%U^3}$BQH&S`adbUXb3wU9Vpj?@g39YEIa7?`{hmJZxWtkQA2AbI ziR3am(EzP($Ydn`o)%|C+bGA^tp z(rlaRy(WhDG?iilC;Jd*bb@Gr$ic7%k22Sp`e2Q`O6m5GrOjW@B8;(vzr!d&;h`@ALkN~gl9-&mQj=d-p|Lx(By zrb~2csln!E`6}$X>B>v5W~JJ9Kx4)4Vdd#UV%Vs@(Qae9A&<|X#Zd$5dEEZaK)`;J zw+Xux7IbMak>__1?JXhhw3}N|3{PCelh5|)Fi|4-Ex_)3wj%Cg<)%m8XnreRP4-j! z-p!AjL?aUSgNaxI7?sAyhk?t-GlqUb=t0U0)#BHk?oapm&xpFrkvs_r*y zH`g~4Ih=acqF2TCPxm8FRZT5`-}`DQ1#vo{$D7~T!bF^P-Y2JWG|@-^Hvp5s)Rn8N z=>2|SXNh(H&(sBhr`3j)+ne;K9A8P`hY%@37`_Q&n6T3tT*$>p*!}$SEOIK$p&(A% zDGd>X!B5wT%p%xlPGkX)clFFK!}rc<`b~eLN%{^z2UQ<1=yFIy%;%9J<9gUa)u;mDmXhom0G0h#?UMsvv~Re<0{|K(z2+F2k1$1iWDWn~M0PDw{F= z4_Z=W!_j5M!YYvih+8}g!b6mlf&(TN7hQ~D+wRM?+VTGZOqyKGT5c3h6PJ&tr?u5{ zlE|oPPnu1s{U;@&B)oYme7g2g+(+tM?k4D9B?bKe`ew%s*=o`uqsGJYA)-C*Du?Am z)d(uA{c*VX$&k0(doBC1yeRo>q&wuhN^=I7^2Y7CHCb0iBo6?PArJ#W|RM5ngu zgqB=p?r@Tq7oOM8RK=-sehHx79p9eCUhE9rZ#r4NrRMxlnX$2bWw!MaE*N_c5=Up< zAHPy-_IjMJDzpU^#C*1zt^IT{M&BagV{8=h@wJg{^8ERq?IV%ln&upQ z*~O&^&$IPgQXk&fduoY?!a8S*wBtPn9Q6iHOACMj<;+N$(3C2#7w??^!@+yYk;G-c zyU%ZoJ(vjg5*$T%k6|4$aH;&z4Z-9gZ9FS0EA$|m*uJz+^=LL-xSzNnfb>5T0l4w_%wY`s$c6w8O7~#AW*bY=NIgBIvx16wE0n- zWf5*FtBdvE{f}@{>1e!?4AHMX5^*omYq9VeVCWnZv2XG_?M0KqhxbGHUUa)^Bqa;a zmp0vY+r)DjDVHzw{}@*o!Uj=Zeh@11e>!nMcC$gze{B4ZHujA~V&Q_jw=b;_< z*GFNk|Iq=4kAb*o&@sBbJiPM0S*~&1iuomvLo+kxw~;Df)Z}mmN|Sc@+_N3NP-C06 zxX35SecBLj*fi@UkIt@Lq}$}9wnYUgwoOwbi!!unsIFpx1uc~I^-V& z>LGeS2GsK4X+@p?z*d{5{{vfTzDyCG+2;>%aykXJCbkt}b!f7oY&Ok2eR-RTE04l3 zWeJsI9Ayz8O|@wfci$5h^Z7Jg^nr74bv)5jgL$Z^w6w)(EA-d&Yn8{LWC7I_d+$SN zi=o&4Y!5X>z28=-T2yDXoD`dM4lGvhBN-VPmB(gsr1*nF^ghC*dP$kJliTl#t8s|g zma~PjHB2#8*q!Ia4uo_aziOq?@qGNMC++TlBtOF68B!L%b-^sDxUZTu5nwQ?SUGHT=x;Wu zibtf|+5G+FOi_*$O)dXfLU8j>DxcFf`B)~gi$Pxb6fk)I;2_=aZcmBnq8*D%^LsR6 ze_BebCkv8>9@*O>gGBOPH@Bn3(eiLBR~OKqU?Ki~ULmu+HN8r=*PJ>w_Hfvt|w!&Rx%ZTlfg z7*(zPh@@o?8ntf>5<8ey#WWx|VGSX?Zzxf~{`W~YITtm!qoLYv;o-<$?0O>ov1irz zw`tGtd$&@Fhh>qMC5uJ&36o<}oNC&8NBg3GmI(;HEY@$8F{j;~zzt|iO=&OG@j3e+ zCpx|hXimrq03okyP;`6+%W0yksd%P;d7+39pM)N*DM2V%mLlW|hZGy~lv0~b`WBCq zZ4Jbl+Myu*5aj+cohVZaM4&t{X7QC)afobmOgEdL+ugdr^Fs(!Bfmtqx}fmU!pL3q z^Y&oUR#?o1Is@0kUGA;($#RwLz{|o3pZ}jF-w&Vbe5!2c<9Y;a7rd_5?K9AO-Ce6{ z3~P5GBvQT;IadcW@N1q#`rbvqJobCMYI$yG(c^YME=wy}tEqegZm9S2YM3XfU0kmC z(zjk$Vd?Ym30pEosUBwkil$$sE%7*F;N`GIZA}B$MWp}Z?4b~E!@LvuTX=iy>C=th zvqjpzY0d|vmjN`Xl&L!^=^<^DNOdib$u0HX(?t_Y20#Rk*xvt5xi4J#qkD$_2f$Up z;3~MRRZ;9DLo)8=O81U{5$b-w8n^hv7plWm8RirDy>G4*dHc8;*7|Sw}h#-)rp8@P^V{_2O zK6A2C_4tHSh({$VNGfkzVMnz~%|VPgkzYw6;=>z>%E&}aGT3QbT)b3gFfe1US7-K= zDf2#yWxZ8OD8zsV+3~o)3$AeNwD8zay;9>h)%F#w;%szLEKx~3Jzce`zv{zz2{IPdl)UH@Y3NZ*qcVnjh_xz>5wZ2tLpeez*+iX>YpN!|7LoDfqM( zOl9TRkos7`M7qY1Sp+R+P+hQPP-qfT(}a;1ebwPJD)( zLLDd(h)%U_O)}Rn?24q`+m6Fv6$5lr!_F_ zJAHkG&%_x+T5#`X+tFJvsQ;9~#T!Mg+vsr?K75=o&TBE){}OAN`JHp-H&R?$EU;GWa4kiIBoeoDuUEISaqtZsSWls$Ff=>0sOTx!tdkS9kQAer6#h3A*-Is+*^V5L0$V_jeq zVgP1{DMI7S%(BnXc(9@xY68t|;7=Wx!ZK@@Ro*iH<4C&4`g|hZiq{5&g5T#N8qLOn z%M$UI5vcz=JqmjQ5GYE#35cBED9nupd46wdM*=ETSqA{%`nw8WIpqF=2Dch-x-INd zf`A{ZP_@wm15dwSH+@NRV_mxlBZT-(Q=fOyN-RMj@eX(i@lB;~P48S~?8692?fFZuDVCV0C;kv~S zFqWeU=IxPa@}DT5upDv4*%k{Pez_uX8Zly74OjVgb$1p|_qSYmDzvp-s6I*%wk6YC zu-Et9+tRtmN<%XL14i7EY{SOzd1`uWiFEeWduzX3>+GaCm@eV`FnLDA(>Sx+8A1e4 zlLh^^rH!6xMx%THh%X#`O(&*p7xGv5m4Pw0Xag5;&e-g2d*%(UTFO$j>&fwU9z0)d zd728dW;%j>skDj^+uVf=}tepR`WlP{U_{kI&fA6wMXYA4Gs6gUrt57wlCL z;w01HYIcIE2%Y%_2`(`pH%LMdB4SUJ8WzB0lAU%GF*;QE;F2$}{4p^vFHho7Rm~vU zSCBEF?~F***9J~3uy4L2i?vDMGk@HN3~Vd=KFx6XIb3fy-7Md)PLohm`JF809vqJ2 zT&;}}Syn7w3at2loL6l*Us&0MHcMWW*%T0$G&>aAPrGTC!C;Nc{J1F zd)wh`Ao1tvCin4@ovSG&mlgWG`}qn_^+e=!*gL)Qc&Xtg)Aoi!sP@_CAe?(@awXpS zr1tEQ+9$uT;eKnSSsfTkp8r(Z@;t_D|GYnehr6Tis{~!}3HVzzyC894UPy85XMjDm ztfAz-IKgHl`400^@YB%J&3SvniV-K-ij5!BE}T*+kd?WDDlK38etV6~jn!CbiX_VDwCA+@yJ&A`kytNJe z*X|p45F^q1#X78)GLua$&o?ng3+K-j?mHvu26N4;EjFpKiDYjWkx}CjK9QTkiL|na zft{9^@IsA|hgPkn!M-8$yC zacqn0zT9p05gGY;DxFj7t{H!MUcT<8{_d~fK+-S>3)er2Kdlh#FjN{5HnpIs=z z`$PHbun`HHg0tk0)s^BJ$wSyT*Dg3X5neDV`2?Ju$*mSXg|wgk<4X7?d+5!?zudsUf~oLcBu-vaI!yg%d!WOB zlAoCNvJiVxunW>lsP3wAr$`*JpRK|E)r^0tl9o57HUXlHU~luqut++!7(w^@ej~vu z`4L(+&5YNRe0}yG)e$9A(sN+adrV3F@BQC1N|6qPQedNv7@=qy)U3iSZ@Lco%_PbD zfO?6>rmNktY=o{3uBg7?BK_FEF|4FugCWMdV*JTXX9XZv{!OU_^1$pd(N^a$+8coJc z-b`ZT>yx^jG}6Ge_Hd^nwW-Xuz_T;Qp?qce%`yb# zkN#NH-fCL!#Cy>hC2ybqNIY{5?Oa$VSfbG^et#uP%50Gv7kMW;%<*^!L~6d}kXF0x z>l$I}fWpf2Y1l3?IE3ofLt$)(s)L+gOl5=RSV&1)6lnqdT^~GQ0lMG*r46=ke*^(D z`+p`c5W+?us7Mh-qhJcPaxWOO)}__8SpBb(2e&2X>|H2{+AI>h zQetw#Bhl1Nr+@GjGO86=+rX6>M{dvX?!)?fzeW>sD-}?HBf_njzIyB@INbB+@x|j$j?;B2KwoMhc zk!GkJ-w9@besGC+2#Z(ho+`XVhI4z+v^!V!^okundJv;l3hZ?+LP5En-xhmI3PV$>lbg=@>Y(aGWBD_HY`4o zn01O{c@|qc@jk!#A12wDwk6C@`Oe$VN1UHjqkDo+Ry#4S6aOL2B14%?BpRH9Ss_S3 z%|ln;LgLQ`BT1!>S9>eXeaW6X!H*-^Xm(yNx)(9MI;h*NCDsJm`|bapNo)(&SeB%Y z7zt2WFat-n9*Z_%2=zO*6QxapI2YBzUfTL-9HWWEK0wj8@uF{gB$l}on=B+M2?-sY z%ZV}b@&dmt^0?7hB?7HW>;^SgGGa!|x-7--VB;T6*TUF8e^K^-2O<{eCD}+04%@bL z(jeCiyZxy^l&JjAn+)-4t<7hBz#vo}T@NbrLHbauIS`-BnF37<4Gn726aLW-u^1UL zJJYZEeR)*4Yc@#Xc+g*1_)h-cTDi<{o(hLa(W#%R`rYv&XjN~q(Lvm+n=@ew*n~#G za+)3U2x%2w1R%hwyO%%yiH-Np-Y$H>+{JQxJ>M$Pzb}b>7wX=UsANIDc|lb zIq^D*bM{i1(6=!Eq3BoPAZkAz_fc+^rM90&-pEN9)ku0suPD2PUgD5TcAL?_KSD{u z2fdf9y5RP3i`#H!5(zDqqW&VKO�$hr#bxLp*Z2=heu2e1-b#f+mA5$D!z}l`CiR zD(Od;l7Crci8lR&E^Ib3E161J<^EGTlHiPo?PU7`wrHJQO@pV4DIFu!4C9^4xeMR)+*u7;xzIIDnW@q50Cc9|wRywNE5bl67k3!YdXci^|<> zy*d>Z@BHG{V|2lpYj*6pF|o|mIl*WMg6HjHLe$m-iHb*SKkKaN>1ArnQtBwci384m zN!tk`P$o#qm5&Cbt7I(1G z)6;8oy|Acgm^)ea5#<3ivD2S{>xIj_6Q6gb z5FSp;*EM;(!#S81RX2)1Y#4tWtG3+{X-M?2)jJ?k6)iJ1uC~%%GW(-}= za;Wl1xMxT?jkuGt4mF>c%$-%zZQNyrd{q-yiBH?+PTde2r zUcP}zU_1N!dT%Hh&#!z3PCyI!9=zgzT=(76)e%|M&T6@F3;J;82?vmFcgXhleTS;5 zioKm|CrWG=?(grDKQ8!Pp4&WMM?Ukt5&g5Yr8u7LyRh<{jbza9*05%EnYyWVYjg+g z$>HIPR+a^aSPoHOO@rWbd%~P!Va2#61f2|HQG5G^fxyCFCt}(>WWgKTOOXHASzu(4 z_?3S3fiNbgNU##|ENL@V8D0aMt@0lk=9@Cw-8bYe{k(Kn9lP_S3n{R({^+|o+H4?F zvx6UIG$6`}ytNPebC;*pwNZ9=mQRWCV5V$4K~#8MeIo3&`J3g+e&PDA)X(?1JhvwT z7eA=2`Ke-OSnv!9W~c8(T`@B?4p}|-E#u9GlD=?o;Fd~2WcMJ|C z{-e}T@Q_QdXNP}9V53&Z5b5MHLuOZrAZf=^vBI zVywuV5XvUCPH4vXQfxT1cC$U$VD^OL?u-0^7bg9C#xr^greHAfuqk{*-5?E4YMECm3v-|`Y)Bt;=7%HRKc5y?|fg%y4GR(-z#;^e<4U3K#+?N0Kwkg z2ik1nT5@Q$Znh3SK8ZOU}#&abKBYg;)oGvqF#_!J-K0Y5d%7btGc9d3TGm+5J?H8h`M$(!@YvDz2 z^RSmKSJQe?hrKKjR3a0VeiypJyT^Na52O9_z`+S)7njQc)BLcW_}4}sV!@tChIY`u8fVAwii805EbvX{o=*?PGk2Ax^EuT6=J!{%Q*1a%ZD_ zrn4uC?cfl2`Vx*v`pLA~``<6jrA=;vdzpWRhv$Nz2_rB~I4S!TLsEq~8%m3j5jsBIE zL+v!S;wIa-ZI`@Y9)Y6Ttymi5Pv+X_LlyXsg!0R=2ryFAI&HmVTT(9B=ON zT9-I)dM7dNKA_w@?4S5Qtu?fWxbABv#YXsmdWbz@Xz8Zd7(qC3B*wwarw~at>>XM~ zd=TzU9tryfnX^sqj~_98oBxliuMCT-TmK$H=>`Gm?(VKZknS#NMYf)|bJNQE`WI5_8lxXlos=HBnM+b4&S^bx@Xr8x}pGstb#XUO*+m$JaqHLi6FJu&x z1Hb>=x2P5D?CdJ>W$9P*+H*EdzO zEVfArIXnXi9TkXHOj?YX6tr7ac|U~YHS@%-9Qutq_WsAFIC=m`G<eFG{-MnzTlE}bssM$5^| z3r|WyqoV0DotvB6-Q9gw#r_}to}DP|LmLe(uKhkm#FrXB6GxSekRbn${R6YNN5=d9%x=R+kA zxuFR(bv#ihIHsew=}z{8D*T=!uUJ=&I)Mj_mxL^F17jANNWAu{vB9gSKD$ z7|PzY4IRw*%Q8s{(K`iC41C^imCxscTh#Q+xFap53qL^x(uwf2v{zCpz@me@!JNFD zn8JVA&W>Z5D46O_{N+c<*p-*)e3b_vba!$UjVX#J3Jki0P#{;fT)kI*97jVKrYD60 z87t3rT;|_ok|Yv_kbJ!Zp+jTf6!Y!Zvhweq6@$saN$RE@r90-Xb*$oQPx)o2(FMKL zsN_&fXRU(rImMjfoVj>3%D-a> z0B`Qk|y1h zJ`uBLZS73ns}~qi?;|%WXT|y|SFAFl(VxE%F5^+ zJN$3dF!C%Hk2X3I_Xl`(7Del|S`NoeL=RNzf1N)5JhOaRQ<|T(dlEQ3@8o`sMapROwo}lx_a5 zY`u7iG+g?k(lE{hjzY{YSKv3=r>_ybCVhxYAIDB9B3JQ2jipG$rG4_RA&XA?is&Lm;bPfMDZ)!em*;K?YWA+MQ;{H1;0YGpD@8=(W*nYM;r(ZxS z?SqdO{1qE~riQLPDBtonWBuT;rBuQUKnw=c=WgSrPwd6oPo?jR?FtxkTz{f=j`n$Y zOq^j^hCRT!EtLXQLfC9ZR$KEg7T~bOi^%eylo3iaEU`}$DsURj1fOF9H}pFrj7%M+ zn!YfOG+i2$az!SrDD5a;8hnRbJ?N`Jxz-@n#gkLem*R@BB3Q{ROa)?( zAI-tPVy-BlWl`bALL~3GkI?t4cGyN@690?hId$z(c1xi3!-vZl#$PheJQ-`qVpp( z1YZ$@T1G}HSa*9xfyfO=QdcgH{*|PfSrwOJSGh^38$~u#AufVx!qDewe*S?|+l+rU z#f%NkEXaM*Ad`hhSJ7}bs=7-V^t_%BX7Yov(O!U%Un3OBr*lF58g7> z_^HIf=GUWYQ;lAm6GWeMQhcD%{|r@nt0oo3Jca~&2VLIK#HzN9FN!eL1rZB3t$RK; zO&Cj4vrt?|?>xF;dGzo1>^YjEROuP1f-ri?fiaUh$8Phpc>yNiDrIOHtst2SYpEOk z`moC7IwqApDE!)qb2R6doo-?+A_gDrlxjFLdk_u?sl;tbRaxB(?E?E@YDbd)`tlCZ}9r82C*%Q3B!%R1ow84XwT;vcbNu<9lnzxz1Uo1SBUsh%OXlkEH9f8$wDwp-C%Vm`JJ2Bz<60QyjV5jPEj(-mBTfc}Dng;;!h@B6vyp zkST)OSi=Lo2$`~{6BW_ud`VCqWC~h1;4T8w%D?B%`^raC@Wbf%L1+$yxzlf0V#qu zHlG%nDw{t*{+p)rkGqzI6xAOt%v7R8-`?uYw?7Be?eb&DR#?whH|&Q6tJO9ZdoDIu zXYRJ90!j0aX~hbiIi+aQM#smfXE@YK_os9DbRL$p)fi%;l}689n(Si>$rL*5k0pIs zyF&G5BPQj<*3yN74w@{k$8$CLn@6uodU_XcD~d}bW9NFaDV6gR;oCCb>rfFmmEFrVxZmObwA@Jr-!&C+$bciw&bSnl7Y>4qw(2;`Iu%ns|_|@}ncDkX+DLMbb!i-RJ!^hj3%=Y&7F>3N%Hd1)a|TwBsub{ z!Q=AdDZ+VIm)5esr|(ryQg1fN-R5~+SXo-lRm|39Bw{B9Mglgn(fs6NFr4Sh#jePM zPn)~msD;iR<|!dm>Xfas7amh^}SE z?3L`v*_Y?+6A{?$_4V&cUaGoQmkV4ZzTfk!s;WxsuLr80u2v!Eoi5H9f^JuLMZ3Ok zm4}8J!cH%>o&=Bq-ekxBc2}X>aOyx*r$fhG*)_hTI>6@Rs=K^qngbx|AsdW#1g=|5+ui8=GAe5$dC>dI#T*CkD8zn~cT zFkM2+!?0$_@zArO1fq>8c`r0Sb=7atdUHCTTJ$g;p^vl!jOwG1{Y|ub*g;a!w4qmr z>uyxEL#X@Zk8Bsj>+KCV0{5dP_fX@|qsrxBhy%zqBBxr0?ke-De-##wyLZKVsh=^(Be$@4?@w#3ntp zx+*fw_s^r0D#3aG@pnUw=CtLBgoJl(3$=Z2T?BZ@aeSM<1+Z$q2ItXpv1qFr_Al=$ zgqKl7uQNRPJ+(Xpx6oG|9)+g*MND*}zhMf;n59weqcTen zbk?lYyT4GoMKip=@1^MwZ1Od{?MDLn528?#7z(cE=Ja`t}noc-QeETI?GyMiw`hv5B#GY~!ZDeX@sO-=pyJwac#wnniS zrj^BDHEVAisnc9+jSDbknH+2=4spw(-CZoHn@ugt+SYm62Cl#Z3X32!IHKq02lsG7 zkTE^yPQ(=nqkuIzyt<$z3Dcj*`M#y-p5|V={5=>98zzvvt*0f{e8<7uY{#UfMF40; z#RA7LN%9t*Th>4UIkN{DXcSomgO1Ty-`;fBbn&LsDT7aHD+rf}03T^he$)?}zFF_& z81P=1Yckz@Na}pqu0r#!bp#HCN=|C=*lk9fT6QpPC)+o4#oP@9ER2i`Yn_a2602NH zyE|t7IeWr8Xq5ZtzS^5WlI=i$#Ej@6%wwq^X(2+75vX9_s_(%bPV?!@wGHq^;?(s zVVw7?jI&I51!YR&b3AB%9)10`W?S@Sqz-+Q8xja&+Skl~riP7&UeB6Mv3AUICP}yK zpj53{j-DsUVPDOLGnGJg`$t^2rE|;&6awd=Mcu|XW$ALJ9YE6kH98uwlctaFxow-f z=#QB6PG;ZC51{DH=WB7@cObu56GX}3L+k!qfiopX8xNt_@)|1&sr+ieQ5t*?^Ze}H zE(Gk@Lf+35o7!-O`TEn+etvolmrM7)?we?fF1|N+CJd@v_VnKxRT%qKTmnhiU;`vz zB((3xYjH-8y88vK^*jt;y4LR8-WV|-U6Fxes^7S1&}!%-b*bY19-UJmhQ>CEi} z(8;HtR;O3W+h>CZrn)`t<~Md|h#zNpAanM)FG09j+rA$FGxI@>TP70 zubb#kk0oxmPmh1@4W=l$p?PB_5lX0KCR{ZdzbeI{Ziz^^X`~>hLTLM3#N1^29QUI* zX}R;hJJC9No3`CmB%#w@q6TL zGE>{FK(VJ38WHbZ&kvOS^K(FkQ$zS>ueLv8)#H#!+Qk%GDmwF{PNzE~5uob8tL`1u<;T{ZL`WeQU4uw}7)u3Gqi(l5Ih&$a* zYLRN^M=dRO`$dOMcq!xXNzhJ@Q#-EyRbO5& z3%GS*KMz1#@vUzhuxOf`K)+X4$Fy{wt^dGNI5(% zwo0oy1otm$qR&L2n_9S=ZF+-WA??KPO^@Kd-0h;hV=r5;{7#G5a|xwE_8K0lo9;d6 z)2geH+*3mm%7mc4CoqVK(>D9seAV`}9Kp}|1sfSBHZKL0KHah+?vm{LK)qLQ@5pks z-PUQz+Vaz<<+(SelQarhkPXWLtnyjPa$~q#pHY5c))=doqX?A4Y<`D|I$xtbV4U%n zJDpd<<=uB@5bxv6%uKL>6Uw*}%8#2Bk?nES#2#hBDk>F2_ZOG)u_zdO*R6W$MmciN-h}N9s!(3j37uW1+cb z$hVagB*pL>@2ZPjUL`dQkW_|wYUet6Y)$sp0Yr<#=1I5<{X4c)c(vU|W>jQkWRxgg zr7MCiM7k3$s+lt@u((}Pp821Qg2#dy#s6Q$V2-*l#6@E%$S5g-Z*6T%H9Kj4ToWQD ztMo=31-bG>>@HyLoy-`YFMdxfm6gv6QmljJ@$Jqte7V5&b>Cf8%;Jx~iHmwVbL)Ky z>V#~w2z%YS0wh2g$H~|Uc=+*IMo^9psOt+D5R+KLiY+tYogtJ66I6Eyx}fcU7^9Ra z_<$;Trz0g?+UFeUN2o496`fHW75j>`gkB9PYwo6(e!VcL4EEsTxWt+)>6v2&I;%v_ zu>&G%$y-sm0T9Lo(Np#J_^P}$!hYDd5?%1pl0frAFVY&J3lVKYa4Qr$0lsU!{p-gl zbnuBaBWX5Bkpnih(`da40{PeYCo=dm*TG#r=VOxYK~#<$lf4o{rWfN0Ywyv}rdK&w zcC9M;TbtbJci(Z1E3LHZ;eyH9CAh?0~v zT_0BF3&=hY5EBh;yG94CQk!?FlJSx%-T4QeUamsMqCKl_p0;~471~)BzviH3Q64}b zwua3!_Gg8Xu=zmZ6cZjX&>NKkH8dzDGa^%oWlwMTux@P5Zh#}X~FI;)e|k!S`@PE>1Ua3nsQ%IcZK zZ8|s-RFm!uRQZ;0@o@_^FDO?d&4)wmphX%@P_zmm%rK=X;Gx| zUxA=zGS%z!wDuG8*nzk2v|?0E-ntrN|E z>t_NG;AMW!H03CuUhX!Q#^ub#w)LZh#pyFpb*B|7+Jd1?kTj;t)~iWzG8KON<3kg5 z>QjiXn~=Kh_Q!MQ9_GuN)>FcVi#}`)v|@QBOk-LxzEl}VuSlCgdc_6js$V2}jOmK| z?rlhiAlcg4_PxIT{Ug?MKaL+531-_?c7C*!nhR*tgv=gVZ4h7R`ikZ;-}Nrq5(#;5 zl8T6s;)K6e9crh_E4*C>QW!PEZ7bH9X?OF(blb>C^FXgfqnoj_uHM_n>97n_Qm2!> zW*i0$%R~T&d3pusJv(j~@rrFCHky~Z^gXQlw6sa98hyRPw8{Rj^A!lxmg6ck$Ov(b zk6f`|b>KeE5ZVotKzf#FoX%5exY)MSm31?c?773D-(Y$CRs~7qZ*%{cE1fGW+-7Ja zIy9#Y^JxBleMNsU2h(N&I)^mq`{`4)_ihL*6EU1uPJ!`N*Od#p{?S3E>@Kj z;raHqPm+Slo;!CRJIZjYG7Vc!d^8I$i8BrBOI+4WZCPXTyDpu`?JIbxF`$<2)+Ao` z0sNV^47{fp_j9-BX|YtW37aB(?Av#0Y$4Fs&|ha~X1J^Ld`w@W6mO0vRK%IlgrClO z`8~hk{Vjj#8Y{qt*@Sti=h!Wm^=}cqz7PrL<3a_mv-tdmLQyFxiO*t4h6+Sl7Dw|G z-{nXjuI!&}e$@crkEN54M1h7I=+Sr`=QKj9vS1~JcJY_LLF{(|LEwgz1mOMs^|Q(X~)O*OZKH>l;p>aUUcurOE!vVlbV$HNJkqtMok`H zj5Vn8`BbB%FQ>co&JL+p~X-iL0vMMe_O^u)8hG~N`-ZD)v(fbl29-fwL$)aJe$H+|6d9WRuLN-`*~~ z+{7-sPuLbpjJmAH9*7>N)}c11PFAAw%X2k}L#8X2b8^jnAJ+o{$sxWrPSv{Yj*!j% z9Jx#|a^HNua84!vbs4VJAJXM|?#0lKZZDYH(rBNWU$*dYA1!n<=Y6?J^4&KP15&nm zkyM;L|8rq>c{&m3;EyMN*nPqo3j1j(3gE|&1a7AQXBMA_`}42qJYjtRjf7Jy{4?m>(P*3|nCg!o=rv+U68> zP;1~g77)ANPOmJkE>H9nfIs~9U}?DP2pTyxeAr6kiWY)^Hz%p`d!GeF9`;7}Iiif0 z+XFgqbszfeaaX{WU#t=guRHRX4fNRHJOf^r?9SeFfm!Uc}>J~FNttXGJoY* zu#M|Rx;KZf;pWaIpvN|`?uOLQ#r&<%BL1p=r;WXh>dX^ANo;@!AB)(2ze9k-vSxe& z>H@rs*)hbZOEbj#C8_1{iRs(k^|Xan;SbXxR7)Zv$a}oSC~fJn<@|cMi>|H#Zc$e4 z{4OX$>+wxD*4K0=C7dwty+W8Us>rTiF|mMBFbmOvze5E&8GTirI$Mw2#oa|!OCi4R z-)f`H6i{aTRbKT}(Im^*!X2s6;(j^C^B$v8x&CJt5%!6em_6NaT2+6Bq6(?TG%FU_ zQj5wQxVL}(y_$P?GH%r?GaPlQ3s~$K!ZcpKPO;Z|T9hGG7gIn8?cwTTT)WUk{fQh@ zAY-&zbNW<^bs|@KQAI-kr%#w-+k99wzH}q{S^^)juvUCPMrk9wm-GVkI;AI%64+n; zyKwswQ*Z+P$PiV~%h3ZdSfSO6NsF_(G-Jw4O!2ifplbYr;6dfArEp_a{9Kz>N$>d# zDT4@uJaq#}i4#cEM4$Y-d zHN9cDoa$zze~(b3!@5csK2+{CM)(W7*byh!Z|9lz4iPiffQOIv<$e0isO~cp>%V6& z#WwR00`me)AXHap=Y5jNDuz@;3Y1?lX8c;crVq5}ItV`HsSE()a8-GgFsq|n@V7E& z0j`F>q)7B@3)qs0M4XHO!4OYMDZXml%h@X3)_=7%{>i$e78^28kIeHLEV6X!KTWrp zP6hy>*!kW0X0-6b-bQ~!z|ved3>={9kvSIDL%}k$&}QEIP=9$4m)}cq`f}g?o5&)2 zq1JtSv;56evGnrxN#)IQE3j>aUZvvteh0h-Um!1#AbOhxl^R*4M1bo=H>l) zxk9q?b)Y=U=fqP{=y4Ph?Rs+WHtS0KU~^bx@xZLl>$0KtL$=<=()+{aX0~^2YyPE- zy0iDhv{hdX+=(GS|I*7ZxYHV3-D}g|vLcVIB9KLqOg>M@X!B+HY(;x9ne)B_+MHpV zskZM#;nE&_ZW-h0jz_xLiC&z-_jsLcug~9Jtd;JWab3P9dZ9ApOU8BzQ};+TQi+>h zhyCq=X;6@(@F>gIq@{n8Bc~#l5*(lY*vaq9sFxiVsF>I;8j3B*KIeJ9l zv7>HC%=eQ6^YEaf?Vj_@*0SG~3^)hh^ z1^Eh)yrEWlxw~iRyi3ovvHW-<=z61aJALJLlM9!u-#$qc;fd5)xA*jgg7OnV-RQE} zgQy~#xLSj&Lz$+waN3`ZY?Ny~%_eQ;tjT0~jGhBprbsfc3>K|0$US^Joch5rq?Z;A z?JK*ZI^#QZ`{wRnY*qC8=(00BNwOR|f3X0RR%M1>>&>{?zGRG)*cMfl-1t#lO5`YN za8mE)BL+6@r@tdvT2c~KMPcblc(ZG(IM|XZj|xP7&9n^1(9x;<<;-U5KqI=^U3kevNAs=6v8kd!VDN zJP9%4n7KT;N}t{goM(Z!5e}5-hAgul*_)X#9j#0Sa7>2ds2mtpU7ZH>D1+R^_iVHX zw>$0<-2A=(@1JV)wjmw#Mi*TuJaGWPrSJXKW2X+0GXvk6(y;}SLOT!MVrbgaL=jm% zod0<6JkkEXwQ|1xB@1E&)!zg7Z|ns9nNL9Jc**H80IX^O>jn>iFG*pC8XIUpaWNTH zP@v$pc4sTLAClnVf)6t2;l~G;cpI`dPJ0VLeYJBf3o;?BsSc1$r!ECWI#;9M{mLt1 zDGLB?j{Ta?MKv`wn^QGU)$||)t&3L;7!=wbtKIxEC;C%A z-FLL8)77I8;QMKpr;dC71~+&nJE8NVlox89ON|0+;r#FaDTIp0Dxr$y9^^ppcwyao z$v!?IfMM|o@P)7XlL>TnIT>~*oU3=IrzoQd`3fl!HNIe$mT2N>cf&&E^udY<@jGn& zTtBt8L;;UG8ASD41qTOvdQ^hk*1AJxIC;?Y2;BEt%aeF~c6%E|9K^wCL#C)c?hYeG z*IYJRb;|;<2S(qr{xdwsx`GtRD!Pum+t2I@oaw z?o2Rwadst(fJD2~#(Sh_>TYiCME_XRlNstLqv5k|M%x zN)W?fESfz+B+Q@+mtRv923U!TGzLa=D@=S1#t6RqKJ8V`TQr_p1ca_AtuSc}T)1t< z0nmFR!ZQCRiBUGt`LsK|(v9G0l`lkRRrCcQK?@6*nL@#DL*R;b`fT&Wk-e+9uu8YGmB?fF(T=x6Fg5zqNxxql+tAN zGQN_f5eYX}1gtT&0_}Guq~e#tfO!tuAO(>SBIV8fkS&E$=TOAih)muIWxz}QHI=dgb- zqfEIj9SVh-ud2t}?nU_bKYpa}9ihU7Jk2Eo7YL~pkAbFj$g#a=G_~on#xfSA# zrorI^^6!kJpP3Y`7k<{hiAn9$=-Sc*P!C1U`tfLRA~hc`RRQ&~`-2irSo zG+hPBAilo5*hbT9b_kQs#WYBkQA7%%btkLa?1Nz-9&&m`?9j6U!_bkw&;LeZrn`5F zPxbGU7Rgp;)zFasOVRK-^^J>RVxKPW5&JvPir3}Wr;cyp^4Z3@d)0erAhKZ0(E958 zawW7uReHX>vS9;=+$-s@giyT?3ir5}xR+7z!=mSbCE`X5K+}`gBG>LS8O|%ZP|h>X zrZ4=9|Gk{+lJ$gjt~9g)+hH*FTe@2b{@B?3viO-8jW0cp*tmBUJ@C?wl&I)b@Lpf4 zNHxy64y|(fq5$Bv_PZpy3ZX55>jWSu7hTW6&U%ObZtbJ!J*t(e4Z3O!h07L9Xyj?d z%->80&*Vg;P%)$%Uu>| zh=%ib^ZLCrHpZYiRahowU@_YtKM(~Z!1)olEGQ1C5^y))PgmkyD(x&_(Yw-}AOQ3L z0;y9h{#5?s$K|F~3{nxJuPN8x*bR6>T$g{5{sPNjeEBzk#%3ylqsLnJ-rE8QYCqkE zgeKr)+O4Tu+x&x~x&FT>n#P6>`?gJX3~NjM$El>RK!D!t2GNt3AKP9L4t6qZI}2Iu zOeZGjw4Ilj5MJh5v}bMie45fU5gvB>tm>9^np!rI&a4UkJ8_lB5N z_O{%`f*!#1%eX905VWB4>;>$o?CFpGV(bx#J^wjkOp4faQ^AN`05u1^P!;ti9e(VwS`6G zd~c)hYec8uw0_gcR42&Z{s~D)0Y3BW~K*u{oK#+0MpzEvpVTlPnm~ zf3#dy9e^J+*^yOHZ4qX7xVc|%DpHrJ0QUfj`pWkB@3Jy%n4B}h0Zm5VwfUJH$%qqt zDaQ5v!tU{&YH@l$QFk}XsoUqkQ!KFF$%6EPB4sYJ91giLurrkDKz_nZ)K}6)M#~25 zQhzH_yU_tvtW$Q3TmSVgF^OzW1loq+@FqulR9-*!SW7&fFj?lvqQJuZFpO`P9KX7J z@^*#F=^0Aw@<5lT+IN~P?MvPlKclQk3h2)BakEWc#X>7(!oTqDhK7dr#LX*!CRnmW zx1fwNDRVeZvw~RKuT-48NC^!QRa2t=cTX-}f{-c~bb4#Q{u+`qYz8lm{z{tfGkJCl zp%n;I!yDk;KquTS{5-pJE2^;{n*U9M-Lc!B#A1+e(wcxzLS#xbw7&AFNht7pyHpiA z;9$4d*nVXdVsB_@$S-V5n^_tLPFCYu(ETQd=i>lRP#r6(MV269sE)Isga?=kxHA4? z6-@dTMJXx?twj|r(Jzn_%m0@#3Y<)JmfRd+g3~Ow$EZzs>>ohl8|CH7lEQxEpm-T> zA&hT5VZBTc@B<)7RCf zFsazuC0ML0?n5SXmSMc-RSz)btJ~Fq-F%hE^El^!&s2f^8#ut;gWt@v z!oYLe3@TsiiDs07+>EpcaSRIn(i?B1^<43`K{dwUKaJqb##JHbA{c4fr4ym-C^Lt{ zSuX39Hokb`rTg2#0IlDgb4wWdsCt(hU zFMB8La4)@RhI)-Q%bvau`2m_4WYYMPtG}_LI$N5l2=@LF0vHmL5^i3Cv>$&JnY%ER z)g8zdfqb6G`axo&{d>{{qDYdonzCr$_}>TI2GQE%!P6B|h8Q>`P(iqZh=9O=gpeQx z?S~T>hidU*kzFMWXe6TN7`+m>fbGB(L=klEFejp2qle-M%&8DZ7biW~ZbB5*8vvP> z0`193SPV5fwm~+a_sQbHq^vm4@Qj0)J}Wcesz)WN#{q(t0c}{Jq6Zl+Rhy5>75rhu z%qi&6fy8U6R1Th7{ZJ3pj=f}xvo4Qraq?5ejPaoH3U1S&hOD*0;(nO8ey{0x zhrniX=)p)(q`_Jg@QSTp&P}N$Mxn*fVS}Ma?LaSFw4{D9xHoC-%a*Xp8*lA0Bq*V% z@M zl{GO3xpZz4UlRd@o;uW8KiLvjK(0dJSiBe*up*)ghym^&UGH&dt(J)xcs#^`+wX=xjEInl#c{G1y@$J-;P5 zJ^p;{KX1MWXi7WMNX!N^9ELyNo1?>Y?H}u4oe6?fSH}cnrJ+*3(sHgRm>y?!!P?5c zEu=RF?PuE1DV+5}D4q8!^TW@0epUJ*|I5!Ie&l@8n?De1eO!%d^^S}4+Yr8*SK$l2x z$uSKZzr@##qdc}I!zBB1{uU;t8V}=(19iRir-IzA*dhJFjnMNgz?A^h3;ltL{PvuD z?8iJMEGBzenB0#!N)$0w_HHxoPel&JQxxp2N{Xtw>FZ>}N6I^?5Ltrq!T*`UB$jlz z8$#wGrJrC}7Q?xOPUALG;kuhUFLOMj@g<&jl&?VyE-uF_%G&Bl*o^FK^Qhx*6l;}_ zyi6u4i3G}D;_}!Lsr*va=5K(=Pb4Yw#TpV?GKx> z%Qi{tWd3CSXI541TJBTygB)=_wqNd7yOU4JU~;}dTwufV>knO&Dw`$KHy ziAxs<>pQ(fJQ16!5IFwR*{~vamLy&d4nxxQ?X3c%ZyK~YWA>}p=)6Zt+WR|zLXn~^ z=PT&(RfydSfNB*u^&%BFe@imsbiWA#PIJGx61$-u_wGI=LszK`adVGd<)WpX#MCk8 zMBYV2IBD-lO;AZc1!ndjc%pikALmy`&gJqj)z9R#g~+hcfK@cmVfN%PSy~ z#uuIqgb9Ng{CW-^;r=`l-e8o1o3s537g)mz6tX4?C{p>H*1r#X-PHnwo_ z^|r2epIdl}W=W`Yix&)h4*Mf~*WnJV_9q8{Hr6?x+xLNB5T41y<6}t32}UgSI}grC zOuP%7R}wvOW&Nk(WiGFNfnJHPdO-Te(=DQ%>#JF@s2;i8nL*qMh~P|>NMt0@A6@4U z`(NAs&+Qu$<$14(d~7V~_G6D_5nDltX@^5RY%9UmoSEoBW;3(PkEJ=00m>@ z9~VerFHu347e=(n&PjnPx`zIWD*+ul5>ws||0g_%)H3)39y|e}AN8XZdxIeqsp5qa z>rry=+hITTGD6UXrAithCT$;lwLDM zFNMNqEqQ(gR(Y^;gCBO%E^3FvYd8F#vQBPVC_-#kDrb8@@f)?Z)&lel zJ6lpeLIC0I^)Ka(x`4Xj(-;l$!~{yTlvI^oix%t-!dKKj`5)?LpS=Hp{9yT z-e!&LZWKUmC}TBne5?r|$313{yPNQm66AW`03GT25+ zVk!7B(9<_0+c1q!0Ky3(cSBNRW8WxW7ksx^A-iz#{|F1+-L=V;Xtvm}6X?*=s*8jK zBPR}BxM%Bo=L#UiPdjfh;#JjE+szJ(+$CSxVGW=X2|Dj&Y@XeOmt9!vB3(Lqc=DqH z5`xP+E)%0_8XbVqsHeS{g45oL010v148L>%@oK&2X(i$pkH=ocsO)w`)R}~2b+v@C z_v4b@-rmrtb*lQuvxW7-PIq$|8~*hSxb06Doss=eIe8AA&d%GXknQc**qVe-DUp@d z|7Yv?a!wTrju!$GQ%YBRIPI>m(Qxj6tBzW(d<>-QZIc*QU)P_;0c;n9wLQO=I@&!4 zJF}eluGh~RG~2fZq|h^rbktbvreY1~o^nf)$DU*f2k(Ej^?b~1{E=VNKceYtqBtgP zINOYryaA3#W+o!G6geBW=dpkQS`p`0n_M)X4x?tWbebKuv(n1O$8X9T7oMme>Wc%Q zF=FvxG?-r%C#VAXj;7~P-*6(st(vB@Gq+*uJhHf^?j;~=EF>%p8LW*fzDc;gxy@*I zTVzSa@y{*JcBdA>ySwy-u)Uo&Qm@OeZTq}f6A<$F@LHz4JZlkPN`VdkasgS*>@oq5MsK_GNvdc~Ytc`gxX4Dxp(!&J2Hf22XS--CxUt>cA0!mU zqK0d0Cr#kcZm-ZjSCx6Wyr4nu(v#uf;26rwU5-}VA$?&pt@7)3P30CKCDw<@`^#O0 zS!&)%fsG&PE0qUj?b8Gb#3C+@CU(pJp3rMe!)Ev5_?U{rUe|j2hu}PZa6>k~JII%cVTr22Hiq&t5!)c?Q7or&67{m%5UV9n61_8kaTWXV1xMEPFtYGcsq zwqF}(b$$FhC|sdfhayXDJVJoS%i~_}R=O5w_E$|eiv66O{XIKtv3BJGz_0*&#JK)l z=A}2H@e+hfk6S!R$&4PCXr1`P#B3QP!(%MVE%$26zsfb4KlVP8L_Y&W7wwwQL(^Xj z3b;$%naBbi9C@r4oYtoXimjp}mDiUWZsemA;pZX(GX;O4X*yMPw$|)NaYvD+r0X_0 zAI`+nMCo@pC`TNS;6t`2d}}`MQ-sZp4NuA^Ph^w`dB6q-uBpJU8%wftvjd{;0^xQE zjseY*y0WrW83G%E#(pS?$ zkAl@rTO5QJSm2uaxzhe|b4H1`lUIaCM4CwcntUT@3WIj6o2y(ySXsZpsJo38nv9Al zU`%?!&I?7_4an#1GmKb8^MCC1t?e}-?&4QNtE{KgA-;QyFBQKX2m|H+%f`#jX(&;#hufo*MqlswyJrm!6MulTxSK3Q2|kxlsIBfS1hT?A)>UTiFa(F(tfJw7oq zwA^?*WO3u8+V!^Q2z-Hb86JMwt6*_yje(&e{n2H`fOXT#HAvDuHbY6)kpP^~v``p) zbHYTx6V%IhzQ^0`*M|pV90YcfTm5vb*J6&OzytVZ6|-U=Xf4h%AfXtli^}SViG4QGiDPaXi_Ylamv+~>fbCg2)SzWCMkZ^G=UDr)Ua0tH5R!!Kw z6{6rvr8IHF!a$PtnvRTnzcd*iPxpi&h78tQXUOLEdR$}~Pvvp#j|(#Cm3VF!N>~pk zCy)JD!25qrWWzVVcQsxx50b!I0~c7E?kAQ$#+)o{S=o3K5z$+FDHP0uF>JPN4`mjd zZ<~1VzP?Ll(xsMz9;$6YdIrKMHII9Li~x?1OOMAiGCl9_d&kQZG0&-e`1#C#L{~I zV4n#8>qixF(x#}s*AE^(x5c+kIcq_<*<(mnQpWibw?9mNQm^akCK1qc8N{C&Ml{gw_obKfC`1tVnmyW(^f*Kt7<-kF=5`P9uhlzhU_ zHkSJ>JcP@&=QRTw$(+m0n?IJe-oui8Ni9GJIHwfN5uQ%xhvkd7I6$NUhFX}P-h#z&Rl9u=&*tMN zHe@J z60VE0^FHz~|3QW>^geXYZC290;6c22eBIrZb|@1$N%iS_*~afFb_zD1?S7W~ug9f) z@$9W?zlKcsoPE=8bs^$)qvvOL+aIncnoi{ZaKg<+TS(5wr&(p-T3>KuLA6-2Y4Xqb zR~zewxtiwZFyLV?9C{c>{n%{YS*@qWA%iFa@8&EwE(*&9$`~l4Oqtx1D`(K6qx71* zOr)-v?bR>!eJRsD^Ou*Gb!Tlp+t`GEBo4bAn^}q29X~CvB>Z~sSo8UOgU7&AJ%WrD zNxlwx)Xg=cP+KRWLgc&s-#TU%v4=UT}-~S&~Zy6BRvTcoG!9s#Ngr;!`?k1>}Btdl}Xc-pO& zr^%AfIQa00iLKigKt{+XE3I&0Zf?y!-d9(uUSvHk9bNf{wz{Wd=N1L^m5uG~S|h`@ ziQ?=DR}9535h*mVA?m>35`LZ_QJQQH$s>5 ze(ZJU*ffB5UrUyda_FFi`rcXgy~nz)ja}FMO;rdr=;X8w`y5VLwLbgdW^qC>rRIp3 z(BkMU+DW<>Lb*T?Vr^1@6v@rtiud0R@UV<4KpRc32<#ESRf zz#vl|5ZHghA67gGqmecc`~94v-`#fUGE+7WFr6QcnagS@fF;?(!$Y<7CDOp#otB#! z{r8X3=?KVwPiz-W1GNSg&a})mv=!1``>VHg-z zSQ#CdkhG!m8X9V3uV#uc@+FcBJtq}2wn0V=1Xy>E?m_(Igqm;|#iol7qv;Yb%l^j5 zSfZj0CJ4UNJJMNUXmvCauweUMqrqb)Vwk^^GSxi?Z)0Rnt_g$G5Uc`~4FcUWYY7mI zqz%HzMa554VEnN(!f)WP>ii9oT0?lg`Wq?rUO&DSkqbF76Hruamo#hz`3-_Z6SIi@ zDBpzql;~wAW7T1%Z55rxK&vCDqUCIk;cgnMmG|8Y{%LRU(&+&^w8~xcuEE=LuiPSW z{D+wbPu)e_e{*K<=h1Rh`RxE3ZrSF^SZ#3_+H^Hp1j4cDPH(;K;Kfg#k<=>;Y4e7q zRN|7a|-975#>;>@}52O;@W^?OxIbRdef(DZk z%sPGVl1nnTu16`2;Z2wJkg`HPH4R!dvxs&w?M%)_YFKCdc7ipmme41H?od z49quJ@mu*`J3BjYT!X%KVtNDdMC*LnfUFX-|AeTXNG9R}+&12Z{e zCBw4=dYMzYiOx0S5UjI`D5ZpS#mxNO!}8Xe2D(f4B{3nX3bO%xwf`5e z)a+jv#nn$qq}ArgLSqB3o7{PpyuC_9HH{7@k{aGGLD8UE=*5N+7lW8Mk0--pU0i%y zZ^6bkMw5@%m>11vUrqmL;oQ!jKBB5@V5X9)!rD#6SA!X45n{z2!Vj_-*cPsI{Ip&$jV zOf*QO8QgzD(!#s z9G5Q0m`x{8!(fNI2NauI&N$oohRCU~54&VAiLW#+E1Wf@Gr`U2f5;ymZuLk@$648+ z|G$glHN|zMc?s`Uzz`*tojz_IY?Pgc3Jhoi(*hHI(v0{E!1yPePWQCGiS1)YU>Qf# z*gXX4uaq7i;)U<^km#>)TK^Z|Ncq|ZZP>s+qrB+)nEnBeoSXnINZ;=o7Y4@b10JS) z(NmWh7;=>j^)?s#3Wjiuxg{oiExBwsd~nHkaT{8~YM^}s+fw05CEj(a-s|19?UaX|6oUyu`-5(H*W zt1Hzo;S1ICI_1Pt4ErS1^Z*wRBN+f0T;-t#VR$4+@8>Y@K@q=^r& zrr+cYQ4Z1FI5X9#Ga?!!%Bbe@rTs{tPPTK?)%>8PV%Yq_S^U2NJ#=BpGDh_`I1-YS z|8VCE#`5n}f`EyniFF|eMQqhKiEEFr(Z~RLSQux}ngd3t_3wOVg_<#h(F-Rp$3p2q zHMOPEX)z>4{V4I5Nx!}pxQtL&N^!9R@g!mCRhv_QCRZN z+i)2M5DsjQJ!2v|`YLmI7vA^-1S5rqZ0feUdcD2-9Jb>ELgSB9IdZotaj+r@gclL4 z;wHsCD#VQXv`ZrY2hrB|svrDzqX*+&yE66i^gxXA*|XlWUq-H%vio)ZCqIOOIqpQ^rb*D(jsPEG^&Dc#+L}wh-^)?Myy&#ypdTT3FQwaydy4D@|^{UVq zlCH%cQwp^u1&xi)+J>T{er_ZAv72_5wodA205iF{Qk%CFgpc#*@Jzt-kBG5xA;WZ$ z(71lqmnozBibRjh+Q9#_?s$Jpzt@CD;)mCp{Q$?MAyjbk-^TFfheI7>{=wXkS^QxM zMbMHacsnfVbIMoT>u)Sn4}JrBt9om%{l_)kj)nP&q7$hfNl5;@{LXe0%b(4m$^Cgb z$URNM@A&j{PNHJ@GxSMPUuvS1``DQFOZ!ti&|iLxWT~s(X*!VtO!QZP$zIl7mne}Q zKIeg!RMF30F@Afx=6&{~3_bG^74;&=uf064wbzDlt;u+~dVW#rz0cZ4>i0_e&DU+G zp6B(RrdWrui{!b!oMmBmc1+sp@85sEKDGE>*1XQMpG43WsBH@BNB4ZPc`&?mp%Nd( zs<6<~n8xWluG!rjv$ia^Nwmyr=3f%OQm1fP9-Tbq&iwvmA(_j`!UHcXjAcqCE5q|w z5k;Awf=i41a%7pQ3Y)26gc@)9ulT%8=zKbv?GGV6Ev)h)7 z>+l40RRGg)pN?TXEKKZ|cm~M<3I)43Zdt9fW@=v}cG97JR@W-)j-21@lb+lkv!xOu z=8}yQJUO!lXvc8<^aZ0^Y)SQ$)p|7@f%oUX?MbeE}Zh!Tl>mAYDwi|pN`*M@58UXga z`V$afIeh7hHw(EY`*FIer#7$m|NQ2&xntUzJf@TGH$B&aY~^PE-^^q1S7+PVdEx_ktI6gRZqZ zRBR2zzUO_Pn#9;YdH(%lN@yA*dUUQ}qJb7pnh@reIhGEaDyrFs^an-pS!`iHh#bV< zFg;cUUK`Z_Ze1+Si{1#%8u|r2RljneQx?lSL@5|$5-}j zI#$5I3T8H&A$15s=4 z%!|b}MMP@3QH&56=_76vU8OX#F%(X2?nGo!t85Dk=tjMRDi4Xmpz-cV0au8Hy@?uc zz5xqzRHU*v#ZJBwZG%DxIc4MiYL#v#psy4xR#%tCfHwXlV~!em0p{%+9+t$s2qg1} zk3|I#d?%;ewcAPVTu;7fau`FRnTd~-NBEe%{y}dF3kz9o;NnTh?T=VLuD{QenY_R? z2+W3s>BU%sqD&doo5VSK$oYz0_Ai^ezhir5WKTT&l&<4f-#E#H5f3ziisPX5#VPl{ zV4pabN0ZBpz{|E%vT5^tt98-GN5l-5a~Q{%GLtH&5iudHD0&?0NhRv{vE@-+Bjb=* znLbkKTXL&*pbj5uV>Oju56QoMK0IO*bQXrN^>c3sP?v$#B;~gzPR# z?0(Eoml}4rOeUjKjM_OE2_nCtj9}bhVX$$g9wDT%O3$Z-R!xSv45!ICxR#$sgmu2$ zN&XD)n*LI^a2FfYiT*aB?w+Im?z-e^f$UWC!RHC_ZNMM~ghSM{>+hw97LDIeQN&3W zMyC6Q{;@{_HajEslp~+x^Zh`HAQ6Q69@_uZ^F0XM4Oqj&!VpBOcC!AWqre-a<$ZG3 z&t~hHn;Q%bkV_CgBSeG@F2ad~g9Er;5JTyD?(Rzv$8Ko%7YmYPvJnd=%?YZmrI3Uz zZCaB>04r&0d&cTwf1o4=2N%UkgwPzI%cyTiEqH{3hsUJbIqE=iNB3I)6m8Yg)#YPv z@2*0=D+qnmL{4mj%TN2UR7D)knp&fW|LIY)UI+*Uxo0Bby~f=R2x5exz$fXTcl^I+ z_vMM?VSI<~dWGLK>cM24HOueKiD=F1FAF34p z2zqt~qnbQzg&em7uob-cz)ED=5Ep|!KTKrH8-l;Z5ktw7t~*~aXD+EEyX_iZeK|o9xvd!$f74>S@?FZy zcC&R$#USLZs!{aF!K$m;moHybRJM5(mK2Ly#B`Qz?4TqhBy#GzZgahN8NoHsVjtH* z%}lmu|3?N0q_eRQdvL@$(l9|TYi&HW>#B?>u?8amR~4v)e6fByS-DI+qg>!{5j)*b zsJr4HlSnR}qHpz>KqQ2A`0Pl+#lV`|3U?|-t7iVzHC-hC50r-fPQP55gIqi%3=4aS zH86iSHW~(*x(J(*2`0p2XAxkBxZB(F5JfX-cNcBlxCE%s*Or^~N$Z<4;w3hgZD5qi z#ria7zp@wv44hQzaM+#bxiR8pVhIq#pudZu-ZmRc7<$)o3~LBG5NF`-&Vv^@#X#RW zzltjnLvA*xiz;LPD`xczd`b;0PIae z;obMHZcQ{8Kg5_qxolU6pj|G<<6rS(XvPQt-B{tLzFTY`d9u#SeIBtg40=7r4>un? z$NWY5%eYLIgci9WV-A^r;yQZroen4HEHVTf#}AsG>pfExbZ+KH1A{Dk9apyZSwlv4x7H?Y!f6j z(H>Bp8LRH2+M<$8adCEzk}6ygN|r>%cpo|-o5J{^n>+4)1!2_xotK9Ic8&mqMMg*5 zA3;?Xyr8(7%Q>fh&@{-O7^YIu>$WQ|fm}(TQtzI_S^~UOkXei*$5#xMR~#OQyYqDf zKxGKQR5pviALIU0{NxYnMKP;1)EFej%^3bp>rahGqoehC#p~c{{Wz7Q*!Zny3H$JF zQ$Q5|rxGtFA6%i*iLw9LIU;}enQ~))1}`ftwX1e=#mpIE-gI-`>*vZ@JMV6{T`DfB z#YpT#lin`X==XFM?M5%_xkOVJM`2(RzTVor>%~Ao`ve0X6ho`U_$*N0-e~Q05LfA8 zeM&Kr(s{i^tQAGsb5NXHk1!V>uV|-JQC3MvsE9$#!Wyy2mEZ6$|R($xqtNM zw1nw%z*^StVk>UxsM`u25zqh}MtYQz6p-zajyR~G&N=%P)6JI`+|f7@X5s?l)qxHnG%Z#;qoBxvqdU3@9JPx zK~Gf&J?joTa{A^Cu-Jic7YY;vGLqEuMNy)!y)cMKASQqea3eg5xIsY{kT3(7wS~Zr zp;?=RQLTKSj+-kJB9yrqlBRd=YI%bqCI*^erJ%z{tj{1}Cod2IVMI$(HR(>QzN3OG zSX)~I0F375Mo8hD2q>-zfqV&hcTdXqK9MXccI|Pe=UHlU5yOHx^=V74ouJ3pWNBP9qAo()mKQLw0hPm!h6LWLyC`}iK8&o=93e~249 zoSQ?8kj4u&7!w!mq9w=cOyRkdRM&-aw+7qrHJ+>hl-R85!2}?W1mNPKFrJr$AVUzR z_}3~?NZ1<#J6ur|*6;w#no@8<7b+|zLpH}F0=Wthp#U^?oGX+sSmd+()pLk?718f) zPSNs%&nrzq5>c$As?lJOA{DHE2&7PXhMXTo8;m^07QicA+ z>9`!x#EVT2A1bh4)8pxF3wO!05e(#UTR9{$j=-OwSPU4CZlBVZh3aZPWL7j7cyr6H zGQ`s1mDi_;mf}oU@$RfFmt#DXJP-^AvO!=#-#=evOB53}iPAhJO5x>d5Llf}*gWDEY(q|Sos}A)EjnvGY=usy7Jg*IpR z<$S>>Hbhi}O3d-8x35!fxxq1{0)Nl`{Bd`1$X5vn!o;xR$sqCnq;r$AM)my@Ay3Yd z>@Sx**j9U$neQ@ad04wJ!=h;ZqA>p72>L_jY+@*mE;Az z!_i%rH<@j0a`qf>i0=h(Q5)VISc3(AW%xVHpcNl%Zja3+=IKj|soW3q1M-paGzCkMMc%oEn@r#@IRwax@&kO|6@LOG zL-An-SqH^f3=~BvKDl#g!-$1!^%RGHihHp3EC>%YYy}S@6Cl6@@?>QFXjd5~Fp@X< zEmyUyip4I4Z1DCTgA(IAj|t4iC*b!#&f84-&WCQ;PUEIQZAp?^{<_&;2}3Gse-C#1r8XKcE+bgwSyqu3QI1EbzEhbmUZTkn|$Y44LES#yd!P@5YsvpbmHusFPwU<)A zhtfO(*r=H2gPvEWyuv*H+u+u+vcp8T%`j!9UIb(k@k4#a#A}qd(mLEnzE@-Uu|m8B zB5xxuUtdO;qhe!thvi$(u}i13d0Zq#861*Zf$GIe?B**q?aoHbxAVIA*W=e0KNxrq zd)JSVdo@A$J;&@{v(q<_VDa&%#wKPw4x(>u*IQ33hV9YB?32g^+HFtUN0ttPz6q|M zhN6G4w6G39#%b0_(nw96q8o(*F3O%K_C7v>@h2OPQzIX|PS62|$)E0EjEJX`17X`v z$Aao=Cdp3U=Le`@!?7FlYq^sF$;Ksf#d?REa9hPZM$h@ymT~G?#HewsVDq8h^CpkV z&-Hb$tJ#3n=qbrPzy{fmMV(zF zrn34AO7{1Gw$Jp%5+$Bmm*f2%ni)oYrP{8_#=0$qH3AGPS{0q9uFCR)N*-Z5XOrD; zx58g7Z7B1tztg>UB6}0Y5mJB{M2#gC2rT_w_8@4`?KD%&(808X21ddJ3wjh zz090>I_`OK{W_M=rRYN;OvCnpw9E0(=yXfm7-d<>QjZ?3tVQVVZm>=9(;HXyl;jj! z271PmV@tsI`O!@~1KS>ifLQK6CSa&Fi-7n+$Kq?a4;1!+KVyw?unQ$IkD* zZ*D{x$+f8E85tQ7^;P-yMOBY_4u-bEph3j20k{`O+?thBU-$LJfy*pbU z&V4xvFkApE&;RXXTE^Lx<_l91yk&cg{X(qMw!Z*#EhA+b# zr60VUugQF$E6%?NU|d0Qo0^&ontTUJpl7SeSu6Q&o3GLi1~X^t#9Y-3(=!@HULAbs zq)%aacXqnd&ax}r@1Uw<|>bS+~dwWi(7hdhF$1xw;l{Z03il;uF6B+kC z&!=`!30?x^*8%2kzti#LU+oP{^)r?0DYpM^-+Sa+va`(f0!8i)QZO)xV9JpAb8cL| zZ)`V?aTs}n!9A)r^Y49c5~7M^mOvPEtp5IRD5&mRMYvcH6p#hPBg9Xh9402*$?*~v z^DA6@rCt{w4jRzXiV&C&yn3BQXIIsL6~5|`2E>&PxtD$HEeR}|M~DX^K@!>pDzRJd zMKU#Ri@PUf2d$_*;8-%UbM-Uiyl6lq~6oKb^ zS=KcqA<x4G7KKN)WP z?xQZLu|UrMasjraeccqvf>tP*vW1=YuRK^SO-ytskOA_PJplb>XKLW#SM&I-?WxcXT$*zP7yFab)^P)uoB_ zRM(2j#>e&bR@i;-_S%ien7*Z?#7zHsf5$~ClOJi>_RH<|3qDZ!Xv%csb07b1Ihvi) zTvfdKgsc7~OJAw8(InvF^N)l*c%SD&ad^wd3dAXY?c$(`4M4&b#&3 zyTzWEsHl^ly1JkBT-%$Qi@VG>>NzjVTi{Ldu;_Jk8!pQ-qJbdia(Q2~S$j70%uK20 z^{geXKe4`~tNmeYX67{du%Li9_)Xgc-juo+%tvNT_nMx}`k|v7M0gor#Ic6$X&g)e zDqO-^5kl9nu$9OQrePk4j=E}{m-*D#FXL02?Ek##ruW?u)Nb8Y?tAA2U?;XnJd<5* zWpRcoh1%iy@J8=9B`hH_M;$|D-(|#sK#$wVitNE=#~i`?J;+GNydW4p3~O0i1H;_i zU5-LYUHx!z^`D5?N!c~;p{u^Iw6y%U2GHCLFmC}}Ip(grexSwH^H`z9&F#WT-kqJ{ z$LzeaR=baap{>Tjmm|IWpZ;0FR1=7go>HNR1Vk)GB2jx2c<<3~KDa}(ML=qwHSeqJ z{GjN}ub%Git!8f5H>bs=N0!=v;~D2|S#+GX06&3K#r8zzzyJQ%Cx%z8@Ka z{0slO`b*1qx)po2ei7=4>-y~?1!{}HkeZ#^+KZ#&WmI>)?{L)?Uq~325W&!@t$!-68 zAU1cU<<{>Z_cc|qMvvF&Ma*t959p<&X7yR`YO&jJ>=UG@BYRy($7j|`2$NuFw&`>} z+b!F1K|!!>M#q`X*4EVf^+RcMWf5QTPjRHIXGc38bB5yV<-tre1U!(D%A`9#?>yI9 zeRr-W5;#}4m9tSz3pP&rW{@^`b?v;k+0H6=34Is+97}el-Dn}|wZjsG8!#9&AX=Q& z+}wO~<0@~R$7Y*s9flzAb9vqUahL#?lVkn4{s9`kO?}{28iUbJhW1WB@&O(0W>wp`W!KBud}xo^r5t;nK*TOe!`Ym?T}QYpD! zP+n508Nr+Nl@%ePNM`+O3)S0e1D00Hm2FuKCYq?TL*hTsV_YTK+u4l8SVrB>-`gI` zi9fmu3c?`jdbDxEv@EwFuUV7P($ZI#(^^?~VvmPge_Z!HUz-ESh{HTRWlc7=%I;ngL#HKf zThQ|{H2VQk-LASIM(swRdS79Z-%_w2>Ifd+18}HDE)-EiBN+CUY_sQ*d)5)I zH`)_^*%8LlpPb)Kp`7F^9*I z$0;~0mvIt3`Q2yAu%K^~YjeMr=9b?3cUl{jamw`@4NHaq2N@j{OH@h6i-{JS?mvqR zeVM25!NEhxd#jm5XF-l&?Fkv?TtUx=yLbnBYg?O4ZW~DPt&!ZbUY*$tA^1%+xf76G16;#mbRGo z6<|?t09%eUPwI2J&16RT%>Te|0hdFOkOs$DlcJvNHWrwML|s}_{@VsTVj9+q+#Ww( z3oH=`8{Y>~!lwbK-wtiu9MI@Q1ZQAjVFGyV-TaCFQG&iF)tJ4Qn3*3ge=Xhum+LSS zGjsuSU)i?7{s~1+4x`oRD*AXC`eV5UFwnAIUCMp+8CqE|k0B`!wxE)Pe2aobkBJ%4 zLEYcQ#l^=7Y3({p3A0c^k)Ns}?O2{hmlDf1Ha^W31jve=uJ_x<6Zm>v_Le&t#S)kJ zL&HU_EVupH9o2HuX%YaZiojqqN8G40jK_F@g4HIFw=E<@bhtzR+S!@&$ENQ~aZj6* z@##JL$m%K|wgFzw)tD)P7&tyJBqav$*9;A8+o}6`vlh;Lb(<2M+25xG{xI<~7a*gb|BZ*9$`Cqg1j1ps`^;mH{9V%k_!19^Gx-?VwW z_tYYi7)0Mlvi4R6Zh`gH)|Q&0NT+JWN>MM&nBrEe;~}NeQ%%=S_kQ#u65mds*=#S% zBT_Qt;Q$vBv)L_blwjU1jv_N6fr(;?x&X(9 zvvUx|w&8?9*vyyS5E>;;T7S_|!U%_GX=@vXsd*Of1=DiPZ_clDnJj9#RQW89Cm%4c zYqePJb$Yy&3_;pIkg|lX84z%%2haXb+-_}IePE)wj)Gvkc^5!U<8<&wG|IutlP~e! zxpM?&^ZQJh6dO5AhR}fhios$^4YuCzBhB`RC!7F6nn#Zx5^L_;fT2pwW~*|Y&uVv9 zEcd3ZW!EaiSH_D1p%B2-WRXiANTp%IjyKvC?o94~Jr!)5zKz`Ex6k_y>O6^Gj1N$z z9Om!vid+Kn=RyVv@_83i{K`a z7JwAxzJ-~{Faj*~v;NfX=e1M*mKg7g@U%Op&9$9%zBEFAR%-Rg+kTA3B_wn*YwOlX z6UX77fAXjgUiu;uzhOs90v*ZiE|w)2? zwJiOoT`3~+R8n~eU!CV7z=5rKWKVc6Zrho*zHKG0=dd9fHiueWkzK(}oa^%v%2np|eOBG~62R-EA^h6^a-)rp?&oL_1)e$c0xu@)P7<+g-)A>2k&!-qZ`%5-69_57 zASNZnC6}ge!lB{xOw>YSk3y@H`vppmP~UkDeG?LduzkeuMwCmhpQ))_P5y;5SI;oA^NrX2wm7$}=g8oS`^qh;H+GvNAlvK+KJdZOQ zCY_>hLQOV?0e-_G8I}Yu5{`*Upb_qXT?g3ec%5G`@Y7HDUG3!R@o0MenI+elCquH+ zEsh+&lQ&htr`3+74idh^VVDC^2ROJ4cNP5Uf+xe<4^64o!Tmsxd?yV&`&QhIue6K? z^@bV|JhKCRI7a*}T~4-bPS#4uQp8aPZ!j;_+XLULzLlrW0@GLlwly-=a0`7Yew0;% z#sAPXKZtd^SFp?bvQ6Nv1VJ~k$dTt5ts#1W4TKx&i3{_kEwvuJp--FabX6C~#os_v z``WtuO$%hX5<&Pt6RuH;FBmhUH9NG^`ZH)sn$|(0sH{oQP|ZP9sLwRaou7GdmC%yJ zF%%IgWZP`+f^)Jm4_+pPSyy1zkD2R(Q&ijPtOYojn4$dV;F(LyGIyhJy_xZ0yZWh{ zZItAQt}A8wuCx_tL=Y7&!qi(GvDw7>MilWh1hm3IDpEUJ%cq@O{nw(i-CrNIFTuPr z^JHI`&%$Ae9n-RLBllxBjtQeP19_dqVR^OPTDz)>J38DfILwgI1o=34&Wv8_rNX2| z`DYQ9;|G7vcGW>DYb_M??dpsQ(35pk?#D@uI(Fvh^BSc^?E_*v_;k>#pwOp;|4P?u$k^Q{E0ifJ6 z{-u>)^NZ{394UV@Du-GnL;vB`s+os`hVIP$>?{Zj*r@LA?mp{&sqPMlx6paAp9F(Z zwv>4A4XCiK-Ough}6%#Yaa&iy7I57Z;zia9*i5?Wa_ddGk4$k>Kz+ zO0p0RizN}vijcbZY7U+5DWY%i-txN@q5(TnVk1BhB^=;zM}tT}u;vx*h}u=<-HW=q zE#;nzx=`W8icK%5S7`hDnH=H0Z!1;#|P1M_7OcOhGePk05(%-sWgH z819tA{w6E?afP^uN$K#(nhx<;+JgjreB#}1Y%H3x=Zto%Ejg2SEFT6U-Xd6eE^jx92)%#Na&G~5-T1! z0=#R4DZYlUCTSDUE&Ou+}7*k38bgc?w=0pei~PYp9 z#HK;X)j!_6OhYW?KkY`H=g}WpOReH?K=Lj;y$B0vFWqdCC%#}75!chdTM2{;=45}wPM}8Gv1V~ zDAZx{>3ZVz+z&FwEklFrp!_NVEkPn{Tk@b{5QsmxIYSBIp}GqlBVb3M?8aOycxZ1r z=DNDQK?vHL_`G`=@5FxQNnY9IrYpXavfowq86FRaeQ>9K7r3%zUFnI=BMwcm#C~1 zzcapIKHYT(5noSekKYacP0b-ddyX25hevPQ$=Y-t9>2QA@n)zxt`fcm6zeou?tf;h3v z!CwFMmMTrG;Q9|*{JE2-EV z%hfuY8(grcEB*8Robimcm4bOgw%`uBjr-kMGK0R?HE!v9OqmLPx68d5Yx+jXFw2EuJ)0;+VJR6=Zcsk!MSm zR3pB!s!T50zfaWGvYaR9X)k9Ez*9E69QUkV@1>M+x8v)=b8>QQQS->Y}i_bAO1^>jRnoKX~5oF@uz)W6mkwNSK!-pi)ucpjCVuDa=TZ0IWYFybco zTOH%q^t|Su1;mRf)p_4Xc&mk}Z7VD9lH8)9{W~s9w_#3gI@Om)ixeXYO{YIorXafO z|IX#x@jYnwg6Zh)?(H7|fpU_ctMt4M>i2;5cPzKW zu;G!A>g#tLwpJZ~4>zp2R(?)6m@R^w61y#VFMNx3E-ooS5>h(YbfvHFF zG#BrmIKIER8~z5lT%4oz^G`qP1#`S!IFJ^!vA_Kx>^&@>H?j1EURU?QTJz?;;gWDz zX)4o;=9710eN|HP3+D?VcgtCVdL_rEt5%kIJ|AGoH+<`_SoxPzGkwLp1_P3em7J_7 zaA_^?QVh8V4rttZ_qWyEzNiTiMl{S=OK_selyIa1-W(KL%_S)`A8J1Qf%-mPr!qV1 zxJaXh%bZP?B_5x3A5Y#h#8WRMs_N?MR%cY}LvwQE5KszFw&0OcUe_0dz0Xsba(Uf> zGubVu0+oh@Dqjpk0t5;)UW5zAq_QkOBFfZ0)6p5pfF5_foN4=IOqwONeo4tur)9RV zUH{y5H@y`f@3!H4+rRRbJmK53kDYI=-@(o4YoiB$n>Ul7YjI)r7PvU}#+L+z(`L1< zqP*P7Agvh2rM&(1Hx0H2huy|neHG_;F|!Vj-~J>o64IL7`q8|{_Dev%@YBe|x$lgV zWC;qbiM8);Uwq$7cm5nQohc%QjPtdl)J^y>(g)wGUjc==Bx^P4=T-7k+P^F>56fb2 zf&R?H4Zq(D8STZwK3fv0vShr@trZpQIdCPDBB0>6r9JKi=K+9q;(G-%u*~}#^v>Jq z4n-R~k4i(U4f4z!9v6p6a(@OAp;o(p&(#p+n}vq&~$?H22=KR@e_ny=2} z^7=ifARZ{oSEs`ytssR<<@ilkeT|chL$R;^;<5<^Wmo8a%b^*Rm+AWGK?X;v>MOl5dQI* zp2PlAvh={oo%`)IlSbt6a=SWwV2#dzLlzEN&^UsqKWz{#Ih^0qSY5b4GpCJ><=4mp z3vZKFH+u+1NCIAkalW2$G=Rlcr)i+yC!%xG*V&Gw8V?&NhamdSLNf9O)7w}AB6!zP^g z#?i`R9>54Rb|5F|#iV*O7*_F_0Zxj%jFMAaB!E6!MTALdWE5UWhF4e@XEzKbAqlS6 zLk2cs9<2hB22H+?(m)Q<{)tpCvV zbMn|=1TKR@F2DZGhX6IT^v1jANmQ&LG{Y&ZGSFcxi@(y_0Lg%eLKa0js`pL3VPCeB z?}a`GieLs({y>vfON@aFpP+dws``4(DW9Nl?B4LyYD8$&NkBhFsL-k&#fMuur z0p~AJ40FdY?oh1CLW0SA?X@d2Qsq9hFRe@xi$PMMRL9o7X}yY zrb)-)#MM)Ld_LuEX~%<&tGDxN^=&4h z_LW937P4|4(~mo3evE+RpO}de!umndn35bmlHU0260{&H431gpMb>!Q0JwAY!Z2@! z@VoYTX%2fG*5gS&d0M2G64yas1(+r??BVjGB+N+S2zSc5;AexnNvN;`a&kQ2%^Mt| zJ9Y+27=Bb7V75?B6}>6H;zs5`%@0C`NeUK|68$PL3zsPD!bl>C9IfL8}oq3Du@| zY3E;8??KzFrD}wE7_Aq@pS+LT8j_GAQ`2Obl`jy;_A<*WW4H<$6U==PittG$RL|Tt z`l3Rs7ia(nU{bV!*;M}YN9^CU5$zov6E!x}VePHkj5r1uV1%$5#gDFcX0HS&BAE0d z^soVWVrcS4e2TNe_NaNNh-sLx7(rqf^uDEW38;x8qW&Nhf3>Fs5b{$lp-A3Kz!LM$ z6MG{#%6lw}@IJIZE(i~f2Xq*UDv~D$irx;!kUTLqLHp}-e5P2Okba&YfuNO1q~m&_ zrDz9135mc%MlcFBsbgVoAUlA&1%-br7beX@eg>R{jVQpOcYe=@Y4?VHD#&^B+-RbN znD_P@Q50zYxcRZJEC^Zx7DF?r-b;xyf&t*5SI2LTYbR@d_^eh>^2jQF<1z=u-X1Wh zOoKKE+rLy0YH=dS3_&P?!aeNOajkDM6QEKkv`EDXj=l^5lAy2@0BZRFn|~J(VRbdA zfrEpiPjj9$s)BiZ$dJ;+@!`!V)b9LO-W@VL%o~-bf}9WNFf!8TF?sbTzuHU`*vK$} zT5^?EQv~*T7a^z{F)1AKOiCo7tf8?CH|Bq7&HBM9D|y1yd>maRu>MlL)2(DidS%l{ zYx1Wl#-kPZJ$QubNY-sC(0u8j<35wmAG87aKGc*PQv^Z?E%q|sfTH@XK3oQROfxFR zMqsJ9e)IPZ?#DQHWi=L%mOhvH_8|FnVU<2xGn9~Uifj2_yYfx?mMUuHCn!63(Ma3W z^fmE|`e+}k-`HQ7{PbWr4(|777HdLZ+ZB)^bjQq`MKXmaF=4j-G|SV@tzO-Ir?M1L zFnQX(?ocFr=yrdRgC%EqObvw~1ftiheW^>G7U1WE`iDzD3xBaAA%|l(+!+p)*o|B; zmoV!VR&*(SGyGdz;_Zn<+2OpH70V`8amCQ9p|!P<(bdVjlw22zB;Wb|NL+d0x#W*5 z*=yXl6nnDMPipsKPRmL!wBL*6N37C#pVdUV;l0azIm&9=MRY%$U`)2(${wHZJ-o?( zC1J18@>t%VB0i>9B{FTVap}AmXV^KcR6+LlDG5!$Bg?cy#Eky;O4e&r=6R&Pm>V6L zA3rWi!oAMDj$l3&HpgX$qh(Lct;_My=$?(dtg33fX6_N&?{1o1bX3*&0auveFqG17 zerbo_AgjN5)og3v{m=DuTxqDcXKsIke9+crzp>wY{l8p*9rpU(uW!2N)3g0L77Cw# znKwHVPk*U*|2E0{P2Wf0-{f}Ftc6Az$2{$A*Td1oke~IZI_f`Bxq#kkAUf?NdsQU? zJUu$v6fx}ijKWK9&d>pExUew$>vtS1I*Bl)mp|&0Bg%0IWyx+#vJ%Gd&w?KElgm&u zPjgrdgp9aaOI71bgU30BgUfh@SNd-2sR)l7gW;Gv;m9z%1upiefpckc6_SQ>tmEh< zZIT9h9P)Ufo1YW|1K(&AvsJD+iDa76pStD9Sd&O!FB;hG;;bIzaM3F zV|@Q5%!iw{XZLDDo|%^-Suke2_cg)m=Hk3YN*(qhyJWi;Lj)3qJU$*U+~B$`rQDOC z5|CEETuQ;eN324o3$MmW`y8@)5%#hKR1YUI({t`yBJ*HLV5GRO_XwpVKEc0H3;r#l zmQ`^f|ISfCtXSUcUN1fK;-0jd>R*l2N&V%Ich`k51!>O=XTkp;Szj3zRog~8Ag#jC zq9ENR(%q8MhzLk`cQ?oYN_RI59U@)QEs|1$baxFkXTRU8-*?V+o{OKb+55@+UeCJM zDwQLyv!3f`T%~Jo_hFPkB`nK+^BF=>{`$UBK!O0m?uV&L(2@k~zy$Bth-R@QbU$)^ zjMk6CMO_EIGwLAhtjb1JT*dJGi7bw~pWy32BStwT;kj+)#ak?c9fR^bb{!MT^mj-%%5FqZhEnSH8$^I0hTEU@~gw!!)%rjU@ICxQ028bXX%uTFC}p&Cn%a)Xc= zsK*EJ3kar|qnvL9aPe|Iv7t>DX2OQcD}iU5$#*&g{$zvSVftcF?w?$=4Dc$}5n(#V z4RcUV6&_!jdC@Miedf0O9t<%LalU6OzA6bM53hY`6`oLyd_NK}h&u0*<2 z=dS+^g#A4!Dd}xGxz2|U3{^vf4Ef84bIw=1Ja(2#^A7!Sy<<(Wx$XJnN20DbpNmQX z9-+Q_wqUdS(K2#ld*uT_Lza1NN&P*t=SeUOatjYcM&8?)$d;HK{pz=Zr>`?8z zHCF7#g^B9#wRPW}?aSeflPE??hi|*^p@GMyyrjH&nYXr}(hCJFky3JfVDCTc`mo(D z>`m?9t9O6oe^uuW^Sz3U^c8eFoNxWTe3K^c;Bf!p3ZU@6^z-PBNG#K(^j_WWN5^?N z^2T73Owe8EX8RMZD>qGL!`6J)qgw%i%9yyM`ZsEtuP`O({MKVHTHI`OzBP)(X9@Wj z9Q_`T5}@fc+)H}PI;yivJY7IvyzIGme;I{M<+buV?4_TBucow9PKWn+7`dx%JDR3a zhz_9lH{a;!XnP^%e-+l^?*8#W>9swq-uga2dTMqG<;UJ^Rkon}Vb{frca^%7@I`+k^*m;>}Katvp?N6yhB)|mJbP9zqeH`=Euxp1EEYX z4eARtw)hy*x@COuy=qGh?*i&$VsZ9(>=v)*_5Px?_XUtK6OvZ$eBX3) z2+eKBSzbJ1I`TOkq|f$sy6w70za-=P3cOY3!Ur$!s%#OMq?Gkhle3f2O1?X+h+?tY z^XyC%F`bP#hE=(I-Qo*1(zG#BtEO=6kGshhYB@U_7rmcs8YYC6o=?3G$>}6q&`Hl; zde}D4CT;Yo5Y?rSYEM8^}WxBaA$?7{qZI3~fpfr-jazrS=^@8?%V($)e4f%Q0S!S6p1c4-E#o zcgsvwi}OZAOF$$DW8&Dse`~|7IMAxJwDcQL$__OSbL;46$&u4i?eO&ul$E1KdK~Ce zyheq}RAxoKwCJ$CqEjp)%F}y+A8-7^@Ai8dcbOOampH1-EYfGsWYtZj0yq=@K6&3NQ&KXo#%Eu9HR9f@Y46N!u7zT+CD!*4Z3ZQ?3Ug`Sr`A8c-+p3P<_yjD*C>zT+9Ckazu6Hez`EL)tf>(pV%~ zYKJ2EvM?_%2>B48d>$TtdQFK;8?QP2RHq^1&IXa&=3T2nLWvq84njk_yzu8x z365n5afrk=&57Q00y_HI^{YSOlb}y%GH=k)30*4u?*eaUE8F+R)IWe|5oyV3aBH0s z*Og?y(mCN{!HbX~-CbFRcGOkdm*9YFigACp_7~y;|4O5eR5(TkNjZCKB)+65yzb~) zq$l}O7c(>;5J(v!ksAVj0;3Tk5ui~=a z5#;P;#CGVH?jq=wxG7W$;!VZU_$;EF9KmjOFYD8gRJ7!Gu}CG#>b|VgMkLq!V-NDYNeo=V^56%MFeRz5KDG00K0{8u(@NgSRa7BPE;_ers6+mxT((cuyk zoE%}G#X&4aZ|>SmwA%hP zmD$wUbVCbAmeN#ZQ^|h)h~R~Np>7~IGPtDmVTif6S3!7R8}~K3JFMQt%EW?+99W%} zOut108;jj7m@tA)pc(mMmTpfV+-Vull!;y7OIMkD+t?Z!x+=Qg`FXh$wSD^4B*e;T z#{4~ujHuE_+4R-?cV+F9m{2LHBN!Ov__ov%2|>uA5*n#Paly~@ zB4?#DOHo8Z<#DA|1u{SS^y5)cfnn8#alCbY4mk$mVi98bD0`)Z%zb^I>Y9+LS9W4C zAeiOgs66XrL4>Jv)T|B)q*zzM9TD_lA|g?fxQ}a=iW9x}K&-47@HBZ)#aFcwj$V9*e~7q> zRsMA=FjRR-GGU!{B>nH2{@nD`PCo`hqay5kE!I}2elDqHmyiBN3UA3C(GgOq4vNG?Lc5g2<8JK_Hb$k{GbdhWFT>=Fl9%g(z~w3?bm2@<{q zBJYa~LaD{V3AuP)%kQRr&{4|b;|`68h`@RqpU;2){R_M@T_=C^KN0maB)EQ`7le;w3h@G+4O?08+vshx=&?e`)m1~_Vl!9LLhgS3oELWWWUW0!$Io`H&r6)k)f?uhEB1Jv=U>hlV`KJU`0$`qM>nJW<< z8n6R3FX%~}=GOV}A&FHRdHP{_&)@I14O;$~QNYK+KY_&B((c#KZw%>!q;n1KIUSMU zYe{s99}u|&aNs4`9k3c%pFSO}U}ej0a`^o{b&@!fNEw_$NF$FW!xAD!{mGdCD~|e6 z@KNitQU)cNJwW8^Daoq@QE0?CV6CbNpyr4rBAa2T_pm7tl+Yz&I8dl!Ny$Op(DJGzh z8wG=Nf8uhYIIu*?MVd36LzT^v&U-xGzWqX$p|hKW6258I``_8!exjInid9i4ir?9L zDge;42y^-1(s$!(Ap;Eulxm-7zJfcEoQ(r(bA7@(y?6of@3cdi{0Z@P|BCFLT*1cislri!YW9U>|N?XXFVks20Y64I=!?{et3Pu5iLug6R-NfewPFyi6sr9qq zXSle~Pgo@abpGps6dH~ZCPHXwb|j`F2T~_b-V$`wuO7z`wWQx`BEKPe`N!hShk5DS zj@e`?D^&&fpjow=hSVH|^sR!%3Sii4+@^qn@j0kYs}Z<@l#YgADp2FHTNHs|nVa#?4K^E-Ed2Y1+=w2tyIi z1+d`Var{7e)6YfYkpDWs*_G)CE!i|jYRl<vSY5? zwKV(OWzCX8zPF+0NV3Ql`r1Q49h|+yX%6e?+m@s3oiQ}s zn*u2gC$m3Ik^M^GuNqEANB1)|2&|Eqy7kiJ%+`2v^sjr=qI^T0L;cnyWK~4>u@oD| z^PB*qgWPl+`^B6r4Dm;eKLtASjGEt)F5@}c<#dcANE~Bi!!0X}7F=jRJMvDD_ z%|`C{M)wD+xhbDm{IMwq9-pwdN`Y&vv{J-kz~Nu{wnjwu*In)P7u3>_1k*=LiWe~? zYpL{MB{mw5+XRg_H8@BmHJFGiIQvE2GtO-G`nW2U^t^IYA`1)M3t7eo(ro|xV{Vd8mD>0v@L0y= zj-cY&zvwX`>(|C{!VfDKaK){i2}bMAjgc_ za`KT1HjPS~R6e&ievzFcl?J_QLdK(Yo5`$vOC$46|9vl*aYGYb>=+i(gb*dx4fEFO z2w5;}NDHyzlbdA3xOeT3*i9A5Yft;>g8ns>MPp|W3-7Pt4~JQ$4wwdy1w5wm1BTe& z60k3jBmvb(Ebdjc-W57h(ISa?t^IP|C+-~~&FkQckd%5)v2sedS+%6NspLq4%E%!u z`xx}zocw*q(oWnoXV?kt?^nZ=f7TOlg7(|#yrud={rpJ)802VxF~rd4rLn!B!fp96 zLX4ms&!R`bpF7W9DrRx9(e=LjiXJ?HtQqrnjsSmOy1==lh=d+<3e*)n0j(J6zhaaQ zy=$#V{^*6&o2p(a&)M>M)Q^m}9Eq!VbL|JO`x#C_zT zR8C6NduL1jY{u?#BZaUNc#qw9Qw>e=<9a_?-%Tab{S{E}{2RxbmtXcSz_Hh^r6?-; z&vd#@+~mSb5>VC|0xZOk#lQ@Kx;;{$$M6egq;kObPflLrZoAkY=6pE$yWJW&>T3=L zhyIG(nhj&Zw?YD}sqx%1_8V>Elk-l=y7~{;MsK1CpoB<=TlSxGe zg6xDs>4@b-aKncoF7ZX;%X>;Jq{k@`Ae0peP&X1U=@F5Tf;__&Pe6895iRRMHdB)* zD96I9CP>B^h>ZZy(Fjl}5>_sed~7C(pxi!Jf`>@vWA1S#WV{e8Bf>XXvF3!R(xd_r zS3$A#tSBm~J-66Yi7HexB^m{mUN+!7A!;K+S~mi$XE9QaHig~eOd?HkNKB?=Q0b_@ zqn~nAzwm2mKNo?Oy~9gV^RuoOqqF+MW@lHG+9xBG zeYvq+)cx$QES7xBCgZ97eYR{h$;dOoSRek247Rnnx0sAND~tNg*3z1DtHn?mNFkAS zN+ERz7Jv)f0BVo(r0=chyssK@3D&GEw>l*yOP3=6c6TdUPBYSHO<^>>p$igc&5EGV#r}f9&nm!BSld zfKo2jPDfV=j$SRcu<`O7&+CvAGH?j-FdCHL)s-ea&x0#joQOBfnEmbCk*kG+5gMnS zomYw2rE3BrG!VfwaXv^sQLmD@M8QhU@4%?Yt=GCR$HKF0e|Y@=2dq_9=(}$L4k8YB zR3J4HvcnFa!X-I|GynjxwOjPqoy%Q1I%2cyKY-OVfbn?#?h^0vDnett0a_Zixsh-YGF``6V+R$fp)>^lzb`#qwh{Q2$ z`dkYYZS@w*N#ih)<*VMoap~aS&_NjD&>i?w5^_Lrmvo!_c_=LM79g=VsYNwtY0lVq zt^dx?ekpW*5*YMOcbmlby2~}MRIIf5a$@vp7^06@4cvsO52h*dbO6Zu!9oM9$%bDpmukfHXMBN5fNCk>XjI()4POZwrB+4MeMbV} z21rL*m0DS4Uw7dEW}LrBi!s#1o6^#}rd2zb3mmvIvpC%~b<5jWb0BKmuEV8eW&Xz_ z0I%%^9g9dp7O2sczer=yN zb9(2#Zjc6~iD2(!PyiW)YT~v9*0JnJsf-EACF=X>DXzEEjWjSiFRaOQ#GLf2JOfzZ1mPaog@pw#1^O z(a85`m_J#mm>5>m)Bo;$R{;nJ9i5sMf8I#bhj}@d5E#7+XMi#~jTJI6-}P(v(&NLS z&Qgwuno~8=Czz_T(a|QSP0h5CVbP@OnyAEJEX}1RFKMP+8Ik2lUOqyUE(?Jz$ou^8 znu(fL9mk_!b2dF3E@WNH5kRBbxsOOXVTvZW#KtK#v7G0>OFFr+LY!)vU4$R3s**;q z8M?}(AzpVHup)M~*idPxU_4$Ta(palef04Rz}X_2YX&)re=g2D?f0PKLQ6|uzIbs) zu8N$1qNwsop_o28TV!^sl{$N}ayggQdSh#494FCV_*m4dHP^I1{y-0?x~~@p zg8&33OUUtVr>JkhlwEIUjY3r~QONN|f^Fyu=iJWgIMzB-tM>a&hRescL(aU_w`D;z zS?`N!K#cG6&!?^j@r%DTe6x6&sU^UNp*DTdudGKxM(!Mxlw{kloE4n`Fe%5Pqfysi zr$UvZGZ!yB(go~}8kwMJX+!JPad_s%MeTfW_~hF!yI8~m-VjMq9yLlaj3|jn@PC8G zO|!*3irDD$>tjsjkj_rg*5e>*wl~462A@DrXf$#|Q}+xQh!fIuK9mu`xL?0Mz0^^< zsx$q|1vt{>^QM$?s_r-%y}CTGvcBE=#3R+D>$QJcc)7~cbWM&Ci*>Z%KPNpmzc}+w zb7@+%Jd_2n_h{S?H_o2c4E@5|@}aEs^cR4(R6C|p?Y!<|CPJpYLT@WL3e)4;)Y!<+YfIOG#C!eea=_5u2BF!_(?P5yFpq@3+MP{0x{0-ro(>dD1AML+Fu27(gQ|G5%7km&m)+>bMKDSow}_-HQBW9<8o5zYo_u2W+1?-=Mzvf zj1Y`Xtms$PjJ%VVuL_oD|4v!B&P#zb)&B_tKoKY;BpAhNWaIQ-1V-ThjPu)q;KrzH z#*~iD30MaDh|>i*bsD5r^3nuu-e=1jN8UmyQ9_A~Uj27fk2d;phs#>t&}$?{_WBI@ zB_$F5^Z{)pkRhW!=1812W0OwEh!z{Q*zZlL^rS(_4$T+A#0bcLl@~&UE-8f-_tQ&b zc$PgxEiwCAnif@ScIM&X3Q=296cHg#MHf2U{&2IGoh?$uIc%Y>%PfbK5X4t1(u4&{ z2(GQ2#7NOlsA1s)TL`Rkk+J2GJgPhXpU<}4GHaTh8>f%{EBzXb0A$@Kj~Hb;{fvky z@&cV+opq!1XjTpf%WO<8nz8a*XGZK4q-)2o1sEOEZAvPcQa*c{0x)!GNmX|A-UT+F zoSe86Q80{&Gr!otF1*V`^3IY&oQQ^) zQ?Zq_L$>XQlZyhhqf-7m|5kLk;(^7|cifq$x7wGJ z>8_O8Ap?K1^AT1Do!8+*&X83tJ1}ONpr=)qyrij5Hpit-YRfm0Y}T?Pbs`+9i}+p@ z(&9zFw0F;!TU%5XSNFL`$oyNS7j2gEQZM3$W;+a)sLm?X{0x&NLHiTg zzd?=9_);FlM7hH!WC@fFEqz~Y&F~WX{eP1xN{7`w3xTo0wIL z)qC2WZ*US4-Um$$2__fj)c~S7<#b~E)KX6_Ui{Q15#yW@rJs};?psYR_=SJGi59#G zEjM|1Q;lV3#+3hedO4RZRw`2cOOHAMlT#YP{M%WGPIt?y@b08IPp2b&e5nut2XLOV zLigciBgztcO#Mq-@f_2-X89^XED*}>-Rm3=O^NaxHCF0(NOMRDCD#+m*+(L`J6+3V zlzhm8KIoJE-XY);WVMU}9^;XKSDr^LPXVzIFHF9`MPP$Xc_YUKq z#5+Qq_JlmwgMLtzK_2T>2yQIV```VffslkqWAJrNAjM}O!ww~{rY1j#6Lb13BpINV zNyjGW5K{XrCGjhK#)*QQnT6N>)d1E1CdHS(={6Dz>GnwRiWn{BAu)Tqief!Qv6P`v zf7G|Wi!0X3D%HZQtV&7rOSiB3Y~X;EntmSPvk`UIL@02E12RB>VAUu=C+RmE>!rq> ziNcFWFyW>D`EMZYedKH`X}mY{snDGzJkP?V5wYEWtDGs4KR5)t+nin6iaK0s;YFUE zWrf>!`0gyZ!-PF6bzVGw!81F{_I&$jx$ZU-xYWi)Aq7P8_n>5NCAzYgJrTzf z_)>vP0L&z%u)aoC~IrvpzfUH#7|WRlt4<06>8@{Rtz z8Uo|Am5%U@?*n#tvUhdcC#AHJ=#=%P6gyvh@!4hZF=K$ZKyST?_wxHIhSe%<^p zm&Vt;0xHS=VcDz!ESq@`H)DenT{z2L zf={{uOOX!mhhD_3Jwkv(AZcrTUt7VsrS!kCW8Y%PLaN+4WYW-zsXRj`yBBvHiy1RE zsQ=RqyS_kj(M=R{CgN^58?b=-=H8wgl$#p}D5{F{%`MdXEoCxmRpgAbGl@Yyg`uMw^(ROxxF2@y< z^ij%cOFipzH?a0wiVPhr7#WLf;o|jnV3?W%F4=$%U&2r&?G2o zyt7^Y>9INxD0wR~q6Uyu-hqU;{%NaR&vskd{g!Gq-7D_)=cG|6q^PNbGszBczqg-2 zVGlRtTCk?o)d2%>N!(gRs{duZz75o}1t?6KA5=j~guQrDuj>?lE@EIv=D;Na0cxC$ zhlekdcKqPxY=usX@9ibw@KMOH7HU&uJif?o57s*jiX4pJr$Dl!)^!o(T)GR~6K!Df zp7nf4AZ9IvR#y7{lZ{?2I{%Z6qHb6L*=QfAQ|kqlzIgaB`f!ZO<9pw>;stNN>+hn% z4!=igsKdOg+@V8xH`0lTgtL`;)JVu;gOnOYw}(0{#@>6G!%6Y$r9q3mgN&fkA*Ls& z6rI-^1pK8XPu6OA2#6vL{VSaRr56FGLdsFoLXuoTDTh)i1sNz!@q2ZWVIDz5h)FxN0Es+rR};Z{Ay@95+nQlL><^5hsxuJ zup|AW#+YIh;NH%7r&W?Hf`jFqfgP?9YA$$V<6;&Dh+s@#ZdPEhy};ST`2SZ-ML5f$9c(V($v3T ztS@8+_G&hgCURGx=j<|ZVR0ji?SAy$AS9>84FV?sDK3h$$9P0xe0Q)q+@g0$K^+sm zzlP(D&M6N5th|wDBs6fwY%r><^x2$>1;}vg4)g-CG<;CxF~^#qP-w3^3oS~|(5||( z3BY6+-bk8ZzcH%jRY;UjTqP~$k{)Z)FqIZ^Njx>v6!8s*5J<2!dljCY)@RXwYY=L; z?55B5c`d*ZJG=qlE6y9=wL4k^h}b{s5;jwlAML!zIQ9O`pi*5LTAbbM6CQTt(~ahD4t)e zJpjsc$C^q@cWY`m2%H@Tw$>P)v1Min0gPhxvnU^>sw-?3fTl;Luq%TU6hkn-u;72{ zAvrwOztHk#Z`Z2(6$&3Ox8qIEmNGAQv&81=v>!I!v?h?OvJ=qk9v#8Xe|$n@w+Z^j z=Qsb9MpU*@b6T^FT4aUN(4eG+_Iy70=IyZXbtzie{WfD^+;@!J0g>ZWA^11gd4zw@X&IRsr z)+y`fQmw3s^HRv;X1|axZ2u_fX~J0+CEEIQc_3W9<)8pf={VhhvybU9B-PSKb+mll zC=~UEn|~O{yk6=4_Tpf0U79APi~rFs0I8DmHJY4IaqjKdAV&S~{8;o))vk}PYhuVn zs*gICi__13Edtz4r7VH_bWu{@o1J3oBV;^~{ifl^-=r(JkGVS0J<{*yh8~}#yukgp zO=!`|WO<)F=vC0sFtNwIPX0O{Si*^3l6MysoHR*O^Ik1fV-Jx1jBngpCHKD_4k~!^ z5tO1H9wZxsA!b5rU(@g&H9{bi4+UaXh}TaM!37G-`!3hKLyq`ue_Q2qdZ=R`CFeY7 zN`S2rZ9=N)7~uj|l?o7uq(-r?iHMBF#7ixi;3t*rN?L0i!6XlM4={lBre+si$lv(#vr9pJ0-Y?#e);_jH>K&qSUKFLo{NA z=K&HRjfYM)L@Fy9y<7uziZfHJe5r0UHB*t&S0OH%eS=mxvF_XJ|6SQSSkL^&&>@WD;!W5A;+Qi=Qa! zL;oBGIN$%%w)avSmPVNzKYc+Y&|GqGmK}VhrZU=P@pvrVNMQt#Fg>e1rMpU`wP9l0 z!P2Os{qQ!9iVkCNqgNv*05#l5PTZ&IC-ij`qm)!!yC{#Tk64^_$}?Scg){&ADulVQ zx-SX@E#a1w@E`d^&aL*Ip15qMHM@9iHZ_6i;@@EEq)lB7IP?EwdLDAN@-?N+%dKy*c)t z)V{lAfL*&*7G*H4)?|mYoJ&Y3V>EHI*@yuX+5_@{XnRK1W7ZKt?dEtc+B1+gZm7MU zIW&>{J5s{0HY?&7C_8m`4N*36DKMtrOvPiDge#TO^0?2zhq->e5!o?EQNTtBZ19_j z8y~bUp@P;ZRpA%2x77Yc5|l*Xzq7f`L0>X&{)X6mBPe-LTea?hM5H@O|9mRFGDLUt5Y0GhR{4 zQzZ^L)mjNhMOLmfZ=^4N*N>>;H?)#N4=c7K1tL;TwJTC)kC_rS26^6zt- z(zmjt&{Uxb1mTaq@+``(5<&YoxKk%cp;k^2&UQvK`Odo9JjSj+UGxe8iybD<}Dq#L-RCnaTOaq;bl{(erU|I&aFa(r8`

NB?{eq@DXvec4cb$R^IzH2#}Yc8$Wf-9! z=D%N2rd62;<|if-aIu)h7ffcc5Z4vBywoB0^BMB)aEC1yUmoerEly1?IqomHv+Fgl zvtolXgs%1{Vf?kc5dVWKdq0O=Qr#PB-YDO3*eCTmk(fJ;(R z_n2;rcBb$rdE&Y}L!YHD7#g5NocJCFED+`@*lTn#DdZeFN#CVeHvNI2pt9M8hu`M@ zO;;*5*02rEA>mxr6WRmMsil?k8_emmdCdsS`>3(3|xrj7<*_t*Cb&Nbe4)0C&~o zA3^x>6|G#}{^{@Y1BdZ5?7+9I)M=Zqh!eio$Ni|fM60x-G-M+DW zP))9MHdEWdH(5231#AiEH^nATKm#vmg_5|VL%F6(W zrv?!HbBP@*8TYlgM5y9q-sViw=e=e}60!Y^#20$A%VfiUm&5jg!Lk=#I7Docl0a{e zL?h-71#HQP>1!e%xh-FhJ2Iu%C&@N}S&_f=dE(2!wai!_XI6pvH%+(K__*=b?pwUthR_q(;QU zKE0_8y-VY^UKv?wQ?}J@a`{@^7DG<*oQ#AibYqW)zh)qAbm}XQ*`K|a>c%Z)d}*}x zR}=nGFMgRH0fdDsMO@=a{e?Kw`wqj&Lt*chPqt?>GP6~BKuuN~5er5h4aLubkO2vf zKe6%j7lNh2^fq%P-_y)u zY?Cv%b!Im}LC|k;>Sgykd2Ashz~i=A$BskpHE-mffAB#d^X2oGhuix#jDod)<}I%9 z)Fvm*4X{6yKonzGm=v6dQAxp2=S&-} z=DEkgol*ee>sgjCkp|Ti5H9yHlICDjntGZ;Xvuoe9g*K!f_so@giC`xq)|G>+Y)=Q zs8r-EX?2ZdrC_oUAg|{=ptrTP1=N+GP&w^ye@?Wk-a8FE%$0JtVb0KB1=;FXQqDgT zP@_QwPbYe8`ljzf@ShohkkuVIeu~v5r-nlnc=#SY*`e6zZeuiH8q-eDecy{YLc8`% zBSW%JD*iKL(>qhF`g#@jusRcL_pO1Z=r?D~@QA}!GKzl(X<~`Tza@RTmFHQ2Jd?(4 zol`ZmZiST~5M=MbRN4IExr*Hfy_ODb+dhH6I%{bz$L;79N6baSTRPQs2&7v&s;p=cgjhQJLWqfAG zqIXYx>!0T}OEt$GU>}(2ISn3#ezy?H;fS}Q zDZq%4ZyOJx?uVtmcjWZ-9i0-$3^!I1*;Hx^+01Gjy%-Rj0{;qu=KqZFHc~%6smu7n zyW6Lj?-)wX!7oWk!02Yaqk}3jTB>KHXy%q?Yvvn6oVI8vW>4uXD-N^T!r4F^ku3KTr1Y#o zo@}x}y>C2oj>Xc84lU7XV{Bcpl&p)XPO*~PZz8QU`;=ADy}8bbo!)g&kW z)n0L6CI(ue1eGpT^}sFt>;Lr3VJy6q#tE$iFlc#2S^bKBOtBZ7p_rAGIvUq8a_;9e zJe=4zFKQqoJ%IIL`f&T(lE}4CYKzja!qERIG1@lEjRVM+p15Gi;CLw{7!&U zYaIC&y|6F0{ z6RW3D;i=*23P`F=5&Z;UdaCH7d5*bR@56omz>%*>6*rf&vQJ+&UpV^5(OpIS8HHxV zW?7u<$A=T;A%%9jB7yo}BLL@*=?6^}rWF+x8iZe6gJ|;^4L|hqZD^B;T*}+`A2jU( zX_N@YC@_!-aXC}IOQB4wTlLG}jX}_$J{o?asW`|eX)HPl%T;9P%P7c_c+uhCbH>Co zX_tyJA)M+LXbP3qYJPOD%m<&aLb8Qw`BFM*ClZl9XwPg-DfS@ICGz$kS+FVm{xdM! z$|FEnL;6IHm|U^skRvcWy}8S$&}e{G{EaD4Ya0pjD48@0!3%h{Lm>jew~!B}AnKed zQ+SaVCFfxc?1U>vo?00^YVp`mARpK7nv+!~n z6=L~awTLI6=s1c?Im3j);Kl%qyx2t2DN3bHvoYojG4Nz2ge(8%?-x@Q|ng@z;m+3#GeYmxAZ$0au1r%GgrM$K(`#Y{yomro{Hr>scXJ>gH zW#)&f={jAg3S5k+qhId()lx12W-#qJ+?r8&t=^jhoPy;3HjdV17Tde6o_B5TfQBk| zaKr-qx-vY!Ei>Bq%j@U^kGoLn+hkT7ru8>4-=k}Ea)0dP`Fi79=W)`HU#sh1@p!FX z3(|``^!Nb4NcM7@@nl}on@|^5mv$gBW}dZ6VOlY?1CD7#l}rO~fx7S#8F1-wK0mFa z1M4#k|R-u!3>h>4=Ve*T2q{BpCvHnog*G~G;Qc5ePi=>X5r-TWzc25_Y#c`^(2}qj_{(^mfWZ ze+xR9P2@BD+dSKUbMOoKOaHA8{o4THEtzZ?2}Hzy{blc+S=cQMeox*b6Dj5htJyH1JaMj&j zhhFRQqfT-3TQAYOl>#QWBiL0i`NJsUp#!lnx4k{{rOYY+5#sa~CeVFsP?hH5x_CWW z^>KK^>BUUt_2H!VE!quC)Y*1owb%cm&0O?;y<6Ykvx03>Wgx0oHc@xUIV@V}tf#L5 z=2d5$qv7`&v4+@r$@08K3;)lK=K}0F15z}Q;akNS-Xxg9b97JHYLgnx(Os_O{Tr?+ z|J%(AX8y9_+akru+Wzdh``;(94AI-2l&{zw{EheI{;h6HX=CGd=NC_(T)MtkH@iCC ze7pCq%42<*N+?{05HGMTlXw>MQ*DlyrA&!G5J z@RkY$kVHF?SSAhcK zvDM|khZnPzWymUovp}2V1xX2K9z@&feskJz%4mA0rA0qxw`g_LI&DX%^kl%~xsIDi z)9wA0pZCLls;VW~OOnNhXwi=?UYb`c+-}r==5wCqsHAKHjp!P4mK5RGh*aFn82%-8 z*``#k8ZtE+VTzo!f`YN}nS%c%G3@@(-583l60Pgo9kkNSW3zbrJy&=*iFJ@V(QTgF zh0JJl^lhrk%}j+7C+-QA5whz?DGkCO>qb~wkd56hQQhguCa+U-)x5fkP9|P23xwHW zl(&?foo+DuKb*ooHdSAo58}+*yQaMn*3;XnTD<9sM)(bfIs*~)oab3lY8oS`ZY3v3 z4N8-22k57NqmGBN6P{w%q)2G6K7YAnS=ALJF#^y?q2zIqkpnkn zKq?XwT3w6=Arc1V-S1J+uljB8CD6pMLc44nyW$A_;oX4nfza6)IU*);B-{U}aNR)l z(F#ovza&_yWU!RUJ2lkAY48?}C=iMERPVnzFyaaT zt2hxo0zdBX*dU}w?o=*+&e3%|AX}57Uz@X0x4huVRl*l$#4?fQ`dHl(F>%}PeLkE%f9P5LmX6Q)doYumGUqLr$bP~XD5senL!x}`2aSpzBSC!Ak2mwUq@BEY+Z z6qK_?Uf<&vn=q9qI}1GD~l zTu=v+nRT*Rh8$`^+$G2vi(4n>h_kiUEA?J=XkyaLwYI;CH#{F)kU{Lt zEw;KH4^2z@9EkSS7*XKLb|<;?%Dh!-a(tzqLqlOE&p=XQIbJSwLBD_h=H(SE3|U-U zR3KQ|xVWHuZU;%#Vs&^IK}#k@$E3ot({c5GdL3OfDhAuIQx+)02O4B$9FJA)&mq+*3|ioa}k5JI{i zI+&EN#5z6#N^PC=I6IFQWIUYd5DX7iKXAi4*hv8yK5nKvT&;!dpVH#LH?}mqoiwSS zEiX$A$2>SU*{rpjZuu`14}$2;K>jaP5Uv?Wx%vJ+O)9x+xwFpV=g+4k- zMW9{(`S5z?iwD20ew+Gl+W2{jadp9F3yMlk-rtWAuA6w1(M+XQn(bbL#ZoQlE5AQ4 z^>jNqvJ&WE@vquporI4k*Jw7FUXT2E$a#MhG3fH(%AM+KX>pB=L>hEJa-iK`t=1>R z#b-`kx4y(B#%n@DDuP>`)#ZB^(?zsIywT}1-MxSxI+d}+l_;O3c9UCJUZFC=c?@*NIfOYYO@4BxqEE8cxi1$T z{#VRkoNHFdc|5gDKwmDkVy74(qo(fM-6}L)%&z>~Q@1|K0TRl=!~`g7ox8rUgiAO6 zow6Yq?hi8l0A6Nbu%nX|4kbkVa|Q=jgQb;CFzp{G7)(_-(oHb6c66%oVo>L$9TEPF zRRz*dSD8`!-S!P_{iIcb1RFbVS?HrP#0Fz?*jJjQTI&Bri28RM%yhBZ356+7(gozt z_O_+#CSu0J!_rix2w65$GV~bO3Nyj5aiF_tp06z2;Bh$ZZmaI6Pk9v;xUehR6n@(< zWen3mVs3-V_3^rL9Jevw?Q)EdNbj}qg8`#f91L?K%IQH6%pLvHjbR3kqAHeoB2Uke z^F*CXO+I=z29G~BC?(tr)*2QQ{dc95o@9qgxK2+76q^U8(T#FmRZ_20Ld+qJNfnsSqyNSDPZG z=ll6UtljhE1pTSeaFA}<>8Pc64js6%Dc_nK(eEG^X)f~x#Q zmRTktk0k`=ugu-DOtYEv4aAzdJW~IC^iNlzfhMNM()%0Eh^>P_@1}wOg#!uPS?R+L z)b1hG0?7OqVQ38~_0{O!QYYCb3;`kehs+?<(gW*)Lzn~(!|)7Ww7j~Zr18cp_Z@;9*h zL~sWsdIwgMJ6Pa#@2X7^T{90GVJGe{E-v|dL+uvG{%6f~oNmj(UW9wjT$}f&x@d?a zC0v7B8BCi}8(h+(3NrKj{bGB^7$_&F*xvQyEZb~?G7v4})uR|cAAWc~eO2jj%0j~i z`RjS*g?jKz5cmK+Ma7uBQ-;&N+Bs(GfdyFVRpo&r1LmdnAwJI3Q`EZM`bOkmyXYc) zsmToW^P*oz8<j>X*xO;rx>$*K86RUhL;8rFN=3^Vlvkt#3V`)Et6TuY^dG0t@& z4~_)6GTb@5tuYl<3NE>Y=8uOjz)6VuE>)2+Z>`DCwJB zV4I+UvHwD7ND}~8$UdK^`6K`JTwKlb0_mG%gT@dRxwn2{<+R6e+oACBK*-C1k#exQW?_8Rmg5$7)cEc~{D@%a>nImGKo3-|Q&y6eK1X%U0NU!62ajST~0krY$FW zb|#E2(dX#SEgV%$`u%;)ixdld4I1Ccm=e_eb@0*yiUlQw0127QZAKWwoq<1Szc^Qz zy^DkOns((8D9V?DC$(`*jCLSYs8Qf#RLF~|9k}3Sjl#=hhUX&`D;_p0b`xO9S%_9D8C+Z#PSdgNB&pef$!i^${gi@`o8WD_8{!->-@QU$Oc4m3VZ@$Z2drvM z^6-v(DZkU3cN!@u{xexo$$MPhLS*b@sAf|4Ine|8#I;qUDu}&jRgeeIXyp;Ag{48_ z^zz$Cx3BIZL!l6Nx94Ew^YC389Wa+BDfrtU> zllVq4Lw2gQ{->hh8UcUmG8iznxG^sQ<52@Q|F_`4Ah*|8&EO0Vj$jdE@j3Hkpk(Hix{D&*_R|8 zlCtY77MYEHv+9 z5iZ4a$J8;iETK4>mQSJwmAP^hTO%}2BFa7qWG`!sIHTS*bsRgwNzy+hLq}-1)i|&j z=+9Ae#VJp+x4`2-E4R%ZL-3|Eg#7LNPZ5QNb)?UcFJg^1y$vFoPa_x#<&KOw&2^yi zB%j;al@VDm3Bidz@+5h*8glHNx|wY;VY}p;w4tm}Ul7~IVOIeKoH03yA6-1vcaF)S zO2RsTSn*|Ef!P!ic$#4H*|Lw}%+62($vgOST`R6Yg9(I_oq-UYdD&}&*7MI`6+*Kh{~#ZU2w^vLqL@t(1tq+YH}i11i~+|4^{>iJ=F=WHXZJa1M0J_oIY^A=L>(Uq z(@Q3yjz(h{NA6Ay;RKdR3l{7IM+_}`hTP|Mb`sCfe#2g8WC9Ht5Y`8tZ!fCCS|eNl z^T~}tQk_*A9GR~yEhY(c2b3p0gIWHNKA=zrLARaX_{AN{{wBiA{4z9>R#TNTQjQT* zYyEa6oD~jh7wyj%Tie90zS*V3?b%M>5Hm%Q*a$NN}{$JTkBABg_Pm{YL% zQ2<%#!mhEKV_5&Z%UKdBJTbp4(N?d?<}w{r40Wnn6c^1R{Ho`>WhkuOX1jI~MX*$I zs`B>Ml_K)N>I&@CppzJH5q2;c9os(+`66!F`f;$&QGAoeW^=^;`4jG?Gv&s3gJZcReKV3{Pov7XY;V!LC_hq{KR(>s*XSK#F z&%t{x{}a%_?V$^^B&6L*$5pT!q}KBSb0`Oylk<{$dvLCx<5IoxUO3NXemnTUkmKxg zjS>I>SGv~z2rKYNtQcDN7~{J>W5d#E-`evwP;_f9j~`%Adr+Tw|Ju zsM}lUQ|{OC7a`x~AA>{1?ISW4_PM98X}b>7`-w?@CS=)&pe8SY0*^vTD8?itzAY79 z1YdVoiGb>r8)_*Qgco)_cgL4fFfaiI56jgh$<^9%<#w!vAg;CzSKuND{>;E;3_}d$ zVJ5cg`H5Y$oS*F6(bM9#UwrVAXwlu$ayr%)BkPTmm+ZA>W_Crj)M{~?-L#hTGwq;G ztnTfkM9+Y50gN}y^u|kfzV)HbJEn3ivCN9_m|>bGxyXv=V!72>PBn1J>jNjp=eN^a zz7)Lt-@6Y$j4`KyK}C-k6W={Z5BC`hn4IOAt<&kaV16h?OOjesYo!|i)(PYD&#;gC^MyHpPE)yQhq z;ZzPW5s_BM_Dol35Ab+qv+q4G-e9@i#Cy8;e7t13P^C(MSg!~8#|eautBDr|FyQob zy1z9s);yZ}SAaX8OCjEL~<4sKK=5@fNULVhwj zOO{O+bmN+0*DSB|toxqdt$g9(UKPnb;O~I7jm#vPyW0U#Fm84C@Y+ASNSWWLP34IA zcD(6+X)sbCu5ZjcG8jb5vFnB1h;0$yb;A$IA}J2S~GxghTg zDEVZM8qPHnDG_=d&+*ujiKut3{m)JXhs+?xK#S#|^VKb8h#Nkx@jbX{Iw^C}o=W2^@3gr{3*s)2Vu<$cNALpc0sGb%*aY zB1k&CvB~vF9#kB;lP5P=-QaH17Js@9oXGw;esiXrSRcT$Ufahahab@_?IGiMuQ~fe zv(9p^`Sft>qRmbC$F8Prn5UOBQr-2JvFevqCN(Lsjxa_Jb~gcm%<;Bl z|Ge7ogZYbta_#NyU~YTf)srwR*t^Rz6@|p$hGyUvoYkP?>U81O4oO!SQ7-HBt^RnB zgn6s;L-vCpu2d@Tw=c(*Ge4>;a|}i3BM*0r8RCYKj1mc`rH2qC1G*uqCg|hhPjc#) zyzU2@8ixu$-pq^QQuMv@eXdvQh|%eSx5tCm4jY;aJ7g9stor#>Np$Vqil4VlDc$szRbPOl`)oH3Y~D&lzG0Sv!-NSTfT|4IwsrfO?xKV4$8nmO z^;Z|1*Zf9qzBVI6;zZ!jV@pgjU{`HkkER)C5w#emP0LY$w3#J-QtxahZ_;!v* z;E;~rjAQ>&GUL1xc?aSPFzoP<1C>zIM!iqGB zHH=WJmZ(^!YTzD3OlGqZ85Q0qKQ}4o-F|%>*wJYnxG0WgTPdsxi!~5$&W;rt!pa>R zmd-AGDYltXsiu}=sUoo{uY5OR%F1}MD^}OzYbY}gs*VOI9aI7)xGHtKEOwJ^%95pq z`t#Mw6W8hdq<0#lir980V1Ibe&CYaX(}5fd$71D!g*xab*TC z%ZuLAE(Xt~=Xax-23>X^7bm|e*Ujz~7j{4mZH}Neg%uBg(U~@^48>>ht#-RTzc}lV ztbJ_Iuw=fK0(TrDq$E4R6)D{f8}>vw&ZquDVG@tSkz{cI&3gY&glI@#!H}2*=65Sw z4M$h~-@iXKN_UAuSz?zLzz1llfq2xI+&S@gu)$nl51 z0Kd=l4_dW*v=2xE@}VQNrC@Dk34f&=s^SrrHZE7w4^thKW70<66%F8HR}$h9!gsH< zMfSlP;o!&(dU@a8ban(Qr^GeFLm@ncp@-Q0a4*cg$6?kRKXif?(WIxcNmURN-Nl%V zxcLz+0XeHZr7Vx;_^WFwd=m5$NJza!oR<3SrSA5*kmaLMYT0(P$8>E(kfg;^;be)7 zU92&S(mP;0+_MB))p>bk6+)*0yaROO;`p)nG0B8#Ne& z$8TS~6~+P?qVDMT(n*}&6$Ou$kE7?FDAlX2RhK7pK}P!c;Dy!`qMw}UQQ#o%u^IB? z=V@|0wuRbdsHbtDe3W_9aZ!Ap|w1fZ4 z1>n%sz>%S60wg4i+59NPHHwT3+PBIfBpO>PF!)jvg|hB=lKE*jE+dPF*>3&Wh`GtJ zw5NoRe(={)sFa6oGiUb54tX zKfAK0toA5L9!upFy#hUAW*hVf8dvSK=_2Q~n8;@;imv_VpX=>Owz>^JXWF}Gbg5;z zTINID7=u*ck>y%CYn_1)x5FPx_FL!ij6OdP6DoKZ1>P?%ck_gkUM68bJVdQ-4=-O6 zrXQ#Ft*HKDU!LYKu_I&>xN>Vp6d(!prdMclG&&Eg7rU+3Bar@}_q$KC+H$SKTJ$+` z@pD~V^{jnOLw=}SY2`+-XGdem?eGG5UTx z#H1kLYFT}{Qx<)8)jO!9G%)A}Y_3X+B*8rsxPK>D@!uTL9SWJljbe0Bj>{JpwrOi; zO%@e)->Q&1$n3YrFNlMQA^rYztF*cHI`DojkbTwd{-ae&f_!h5-|aafMx4LNX)XC@ z;>@gl^9TRU8(!fY?!9(oJOn*m)SYhE>k5#d+1BIdX>oip=MoPuJE0aDQRnM3?Aikf zpVw6_!-MzxN{RTdUls&WjM1EK1l?{AZo-MpcA(<7r5L22UXHI()*n%D^wOr*j0MI4 z%W0TM=GMy@{P$)D$~XK!ug|WDV6%n))6%`y^)b!LCIQYAApwIKkInk}-Lf(5o%0s& zg2So$>X-3HThYSQWa}{mIwp7{QMLkOJR&DRMbS42Gd^$L+3GCbu}&L zKh1$vpwwfVHR38ei|{7S-@d?zxl1|PFcW2cdvo&mN`^|kubO&(W@f63I2ix+&S$*; zQ^xswg2NLOcQn)gVPc{8A_2$A0+!zJ^LlOf38budi`$k)nks$~0f^33 zF+F!JJp6XfUJr_0W(n|8>h_;Mj}3sALuf+r0-(SSI!jwBNx8>y^KY_bXVUlr`<8rM z{~mrJ$fd?`rG)H-KWJKYrWU))}1_8Ca6J73(NCn)4{Q{(uhrfr3i7iuTZnb?s)zDSb=A1echfsD!)~b3h;LIzHn?j%CRATg5PL0wvh*?46H zfn!j!X(`1T62ERcXj?r%#1rA z>-j0)pXSGgT}O`pKDIrlSj$!&SmJ5A_PlD8baHyU6PO10?nukpCR?Q2p@NAguYg({ z_Nf>KlUmWpoR2i;OCi|`rLbE>iM*TOT^wF{CiPUFV|%I*cf+vG+yJ@ig^?_8a9U1t z#zB+j>@Qkla2Pv>dqCmpkMx&Z8KrM#j6c%1AHk_dhwPgk*?&zaC|CVTr`Aq?^Z8rz zb99l>O*2C{K^*S|X;6o6s%_+#r~+2WtGYeWl9P)03DMvd-m5E~?eDO8@Ft0qo9|Xs z9H<=L+?NYdM+>?KS(GQ6;pe#sI~^f#-qsgiw`c72a4<1vRAX3ja}cqEv^j@z4&+k5 z?_GC)CFf@Al+#9x(vV4-%NJm2l{?^=H2S5b@Q6Hh<;gD2G)pw~Z-zV%);^ZpKnQ{L zAlH^tTg!!8V8asT=#u5f-z6upg|aW3Zy6jdzoum-TQwY1o?zGg*_v6;+R}T_D#^-8 z!deBz2qL!``d*$0U^)CuZAS9FKiZB;p#ZtEHO29y74b6qV#)UXrYOV!l_37`QSXN$DyTwPtPRebh80_ zeM{Q8+0`oaX$8AL;g)cyC-}Ic;%HOVK?eOFpy%ByA{s{gW9vm<`@h||^D~r|xHC*^ zWW9<$h59=!7YHC+85KJj7) z3+FVsZO*?xB>o2#0RaDkeCX(Glz&H=oSgOl->+aZUP^x>Fh~AoXu%u~FtF11-k*3o zwnVJF&v(J$JiB9$dy%Toaf0$r!o z{=2Gb#3S+V8V@~mHiC~?v$n^wkC=I41K`Ja;fHHE&E!5_2wFYJ7lwqSXHH8^ley`N z_?{d)(sl#CAjNU)?CtF>uIjDFk0}}VC>(?~>_BIL4+=1PT`g{<%#*@qPIk; zti)fgpoeCOU?Lf0NZzBifX6W4cfJWL9*`B@NR#gt`2}WZ7+MZhgiFWy)0ro83aIfj z4#%xav#k!yFtFup33BoqvW-XrF1K;-7+E}!0!x~_+4t{tnvCI3M2G!M=XNdX3+}lE z(pXt?pJ<5*h3~37951&@;tE5vz7dG;rX=x{Y)c(=Aun^^OlLW^9X*4Idz7? zs$+UVBcC> zMmAQMs>L4B{#fQ!HDRYlh1u{?y~*Iy4{J*$6ZBHf$!nb|pvCu+WwE?k)*eo7?&i*BVrPf)Svs1pf$5lp z-F0;{peDrfK-6aa#G@e9i|j-9Yz;jJrlryjHOx9Bq#qqBKcg zl-$o=iJS=CIosJJhJNFV*loT5?#6IvdU@_Ynyh$I@_#y=cHXMN(wv+^Y zp5``Xv*V#9ub$1Tn)>Qc~!ATML`*gc|?bxzFXIgir@i zj^GidbtZ2t%jFNL)r7+Bf!48(w5^$#UKG`k)96b8r5Va$IG_8dAO=iMl=L{i9}N5X zh&`ElJ=r=CL51~3Fh-<(*q4iFliY~IidoM?R>Lv)t)(>sO*(-t1I(vTX-`P6uDP@a5(V@4DjEouF&hGBLl?L-iMR^D}&d0sC$1N5t*>B=Z z*Jj?!uvI7Gf2VRH3)MzzwRIR3 z1}5+!!!FxnKbCB(W#qsK^+M@C97h=$!P4%zepMI{yZBLFYts3`(b1H{Llr|5b09As+4o!ei}vXc0GFbeze|aOE(=5qh;QU2Xd?__dHhg*l)sT}j%N$hgi4bfPsg-PH?$Vv{ji8%YhxI+-z6MzvP@SYQmpJ zkA)x*LG1K(|< z$c;9Q%T)M;9=)FG?vKn|@(Ga$8d}h@6k}ZE$6lzNLKN~=r#o+*pP_Vl5%FjW{WDu%(nO(439q};VZ|N~FV?Jy!^JK4N2Q!e#@MhdKA*dz8I6~@CcVWC-)be%Ze7&vE|UN;L*do)GzE zqt{)YAVv1Q^3(Pxj`MsqT}Zv)3@QxxCFfHdzxQ$954-JL2{4c%7noUfw;+Z!%<>~u z8~~bLk4p0#k-@3-uK^XF@3pOLzg3lp$kLY0vd8OC-;Wl6i@|V)37_hZyV8Z}$<32I z-2-f9t23sF_B{lOyo&ff8u0M&c!ysKUQ{9roHvuj#eBlpQDb*GyLxYB+Gyi&+8&D{ zFOi$LT_WK<_HT*45OTBM`V%@wKotXB`#z$)72|VN^^@HNxOsQU;uV`vD94spYpFRO zTB@f`jR8YCv)q!NGB;(0^YZ06NrW$3mXObJZ#2*obZ!R5C-YwIVbr3rFipH|qkd@8 zKLa15eQqb}?(dr_mP*pxEggAV+8npDTqD_r6$yP$fg&%HA5RMh2M2Wt!%AH~2ScGG zLhg6Gap5f0W_raS-cKAp(0P?J-~$zT}If3hR7jhiGxl0RvTDs z0MsecXE@>O>^{-qk=Y$g8m_B;M3(y!)eHhYGYaxF|DP+9)9X3LE?Z$p-xgT~Sq%PLGyE@Yg!a3rgYiepPiHM{rjZCL$^84gLhnGk8 zQtBdvF4cpG;p)Mv>3O)wIR8c`ht1;FjbFNGxc7nh}N-7lgMP=Nn;c)9+ ze5Q$buqBPJGp;@Ud@Uz>zD<@&rA&>!t&=zA#Y0~h54@_XO33TU{`>dslZ6VW+n5sS zApii1(#6>Oex@LF?vW(B$1N@M?&-;dALKvY1+Qs{y!;v4mx>_<(GhkfPYwauS-M`w zXh`(ioNtF-Utb3!aPS&kw@d~ZiU!9tf%7p^4FjSfQp3aCJltdZH$!IK%8XGlNW;8> z&y>w-w6@h%HHkzvL*lgk?@;^eD!;HKfgzx^JkFW=DSmvDtWyy&)db;hoz~_+(0I2 zlI*Wvl%#R8RSugIDY~emt>RbG7`fUul_hBUCM{Fv>mlJ4YNs7``$l$5q7vnfFrW}y zFT(OYQ$l1+Og7)!>YB#I%i=As{LsLZ_W)GWX61G92n&lIqb&r$j_Hi};j(X!{k zUfK5c_nVDGhmwXlxSSTo589EfdioDbFYmYMsrC{JGZ>?N%65U*%$ zX}L3XCyu>@j+-a$iAu0zv@(tzPKs>=_GoU#v$Uw#^>A-Yql8WAem&L{P#fGJ_b$-^ zgD$Gfh#fIFdv!#6vXh|+2n~Vpz3RFYY!&f--jM_V7hCjUVae2zDeoleL>)Yg%wV(v zwH?n%8=|1SwnTi7m6V9rA8Dz{+1ys*5$_5{3l5zGKq;XX?6lq7+?>Bt?4yZ=5PGE| zw+VFc*E!TQ*G>E?n&HzYsmX2i6r;dvHBlQIn;njR|6s=>IaBJ;Z&Rj?wZ9rcnFIcm z)zynyzZGP$rxs(_$|)=J1_Kp=bWy{HR;YdwUAYw`SIHs!pPeqEaq^~M_@?sR+np=| zR4jP?+Vch~*1W%Wy3~7t+u29B=>3yQ)=UlspOD<>2)L586c!fMH$b$U;LtnlYt$QW zaZjrXBg9BvF{&@hcfEa2XsGFa8e<5Rdya~+oo)@H0MV%kdHk*$>_9zOj4)jqU;;Nh zz1?Tfh){fKT8hpQzE&aY@Yo;9g!UYO$miAQGm^>+^E!%JvNiux^tNd#4nv_vW09$Lg7TN9#QDv^))i`{}c;7j1+}zj!is0!NM%M zjDb1ofB1kLrNGR1Sbm{F4gn#~ew?Vo_|h$n+aQ@N-8+CQ|Cl5Pz4Zv76_hg}GxMe} zU~Drk5Kja7_s)}#jHS$@)6EtP7?Oh}DrHzP!$LLdH%kWZHz}pQ@N_DhsD8V=MJh59 zS5&I}|K~4F<6(d$|5|2aHLFQZ9x%uJYu{_?2YF(Evxei8vPD;tJT{#6ZZY~jJ`l@@=rpc12oN)KGm8} zcZ*%SC5k#ZB-6s-gOeVc-kpu^z@aFfXp)9(0ih1VOgHxLwwqc0_p(Jh^Lz8ZM~({i zy=cuK#Y5nZUzSnyEIcp@scf_ z&7HMrW$HfM4D+H{|Ex@5bjuD&Jg zyMK2)_i=J0b2Gl*Ka>F5;6IFt9$cqZE8oA^B2NsW_{`aJ=AbkI149gwkq}iI`}wk9 zT%wqD6W;c?*;CKj^E8w6VJ??pAmVa%WVLnGb@O&C@OYunvDoo46Y9Ce|2B5b5wq0= zSEl1n%IDhDuQ9B<(e<2Fr|I`rJEo(&6#;Ub!v`JfG(%Wv?QM7NqX-9(P*FvE;!&}Z zMn^|Y*tHn<&5JJ?Ybu}jz|f5RBb5|{Q_fhh#&Z;=LH`6F(gvE?UXN^xt70b1&CQX! zc)Jhg;L>{zMtP0W!2Usjm{IRGY0{d@uzNG*;n?c&?Ky3)KIM+4ylKdJL+S;w=0qf; z4jF;4NNxcEg4ecRN4)=zM6CUYovZCY5aX<~#?huEIr#Hl>~bm-dsy&p0(`g3G6q-* zCicWtL7+^||S7$A0IF^ZqAr%hOU02J>JnU`BfUyuo zKs{XzR2jgSd9S+*^nEubESeH(Tz;&@vD|ziRp3I8YlX@QOcaNZJ5!wqK-Q(mMxVye z8DR3?&hs%hrUhS=>JNz0gxms{!}nR#8OCyvBgIsrvJ+D=qKStMAY_&kV^z9KQbJfF zeC_bI6BGdzg0}j8bT`f{YgSTKQz^h|vHs%UfdIiN011CPluIpH4!~qt9xT>_#~v zZuTA?)Hl*Li&=tU5YxGa3Mt9S)HF0g(^gZZ^NPd>J_}Vit>QKv}$q|}dA!A|$^XO3n;90g&M(P%LNx*w0_FCw^EY9ZS zXUKdDG}8{c4H^w$Y`$5vD8(jOzQjZ%jDfgaCWj+$fY1!nsV6jHpa$EcpsOW@Ok&sH zFd--}bSfY3SLG?F!p=R%Sis8HwqOci6@-H$7fVhBC)3<9G>1$O3@=EI=6B;>)~dNd z7H3pdu8S6O1k*GMut&><(PY(_iUbF%K4N!HmV!YJT9JhULEY&r&@lCKSI$V)JS_mhiyA|#Xl?u5h617Mr1S{BK~yI(v*{(^*Tn_SRw zwM`L1#gJmzwE3h}Zl>}xvDy&>8fr=+@a@=|`*ew9urw3Pi#lK4pj?fSk%`G!Bky zDR|cnwi->mklRRL zSD>WK;@L_vmkfgJ1g~k5$d0&fS7qrLz%>k}9?&KYhc?ngz$5u;DOaXBz1Zb&o(>6a zl+3$oL(3>O;L#%d#`*i37T5q2^t($GHuoC+4(y4^ud{p`&xoST3=rk8BPf(!Tm|i0Bdu>)5 zpF_1#O%m7wE?qF47mT(t)a%b@za1F7^|X_pb5|?kS+8a(!i09z7G*WBdDGl@i=@ft z@3K)~^umSX<}j`dUwExZR_#yFb4~PO0)~B@_*Z2umLM# z@F#!L)&#JdL;-UB?GS;HoRRxU_(v~<99$3vq$V76@HPRkY_YUl_oj$>HKM2r85!4j z_CJ8s51uXAD;y3PA1Biv_Fm4vk~grmKYR93rlXd!@hJbnoAuH|4L1dklzm0d&vQ}Z zs$zGjhlu@o15p;yZxlZ4Mw>uyJHh6Ej;*Mc(Y(+d2VB^f4 z&)I0<1k)^%Tr>sLZwVofpEMgD9>&2=25n8W+@)$?Nbo;$xt`kS@Opdw+|4g5p=jt@ zWcjkKgx`!BddU|kHY_ViWxiE?v=1`{xYD#!Zx2BAC&~{E4@>dz)PLKm2zj>qx>4DM z01`}Bq({2&Y)drRN{q^zpwIl{|QgO%cOv_Vru+=an zU0E=o^Ekf70Strtp@wE6I71zHzemxd!)Gy8DoK1N{Uo1y0&N)d>Gy9pzM6gR@Kmxk zU0WX?;iHB@FK{rZCss*WMPb)kJHsG_&J?awN{l9A&dLMq!hmK`fbtYIVqYHemBmOghYQH!6#Mo>r!wM zqtB&Vr^$yjM^X|9ZIjP!gUvuaGyrTKN3}Mopka z2vDVatg7AZI`cTA!nDc!H21jiYUbnYynqn_7la=;&mST2KZh_JW5;uGHAKf$!@{AN z97)#lrMvHwQD|dhCSb%OPPu4NUm7kum<0^C>sd&`e9<GUdk|KKvC^1N?d$X$$#{|3@H_B)XSgUez1lsERBXB_iKNZXubt>U@DaV zZ|QpNe?K7ortHM^Xq$aue|ve*Ome@mOpII%Gxz<##Y_^jj}lH@5FH)(4G;-0fl;j8 z=a%1`K0o!)Wd*v6acn&4PB%}xU6nKtzH+)hpkP=mjS7j8)?vAqw4b|2?0<`s{x|88 z#|Uyr5OrbtQ62ZR=&M5;Za{r(y@nzcv_08+DBMtw8EO}6wbPZBvFpZy?-BzN@76Qc zG{E=yA0usZ3<7EZ4_tnavr~QlO+{C4g?dYo^*aS%N&6osx7~lx(bvQpMjZxdmC?xA zL@JznuHNROV&m}(;BgXmSW(WZrjlJEN|%k&>2tqa+GV)>gZwS$RGtUM|Ak?d5~5Kg zZ0ueXwmGu6{M&c*Iu5Li!Qm{w4KWy>b7oAab4F{Lt9Fo(Gb|>C73Z~l0rt@Lnc{Ti zcQ;Z93uO;c&L{j|f4^#91q+N@XV?S{3Uwr$C)I?A-6bk3k^k?Y+;{qaw)EzC)mOKL z{z-jQgD^ql?+NOTdw%o$=7%1{hc~1JV$%hFOf;}iX7iFFfuA!={i`xEd9Q` zB7;*?nDDA^pDTU8uMF6)=KKB}-=*Z$>J#DB%_Zr*R~r6L+SIFQ;XFk0^U8k{p!f34?8|?$mkRvNi+ektwR1`I`aZsYMzSkr`;}+!);5y- zs&Vnl8}Vz$+BN=6Tyc2)d9JV7m%hAv^T0O{$vZ%QvHq98dT+vBCWoLY3_#%N>gTe~ HDWM4f?e;TI literal 0 HcmV?d00001 diff --git a/docs/image/ClickHouse/gluten-debug-clion-debug.png b/docs/image/ClickHouse/gluten-debug-clion-debug.png new file mode 100644 index 0000000000000000000000000000000000000000..e217eff0976b0da73c1c387a7172000346740de5 GIT binary patch literal 290162 zcmY(q1yEeg(m%Xt(7@vE!GbIhB*9^k;O?+EA-KD{yIXMg0KwfY3GVI=!R^0Ko_p{6 zeN|g$tIo`s>F(+7>FNGWn1Y-HItmdA002P$ED2Tu01&DG0N4>ESm+zoUd3$aAB=;N zgb1K&g5&@Ipa6UZ3#+(h9=E&1>dtxX-9KkJKn|Cq#J_wKHjW?q$SDM%A)tvzfq$oz z%)F3neCF!RXA+~^ip|HXTHmVNpm&x$*Q%OLs9lHkE)Iu5$q)}Y988re1dk;IM*$m0 zoTV2sch>IV-TAz=dizlL7VtTO#ydwuce&$i*+KrtM&{Xl)>>|K^jrGyJm?QcXh{6s z&f|-=4XJt7$6~kNG|r_G}7qnta??R69>xUe8}V&gy{G z#gL&p*)Yn_POl+8&-nONG@UoqFMH@ND*+WM`GNwrmupY`knb-2-mmg6msLyFlw{xa zataV%w{l({uhiXN?tJc>BJh|WAg_ahoy)%-vtI9V)T_0g7Pov}pU+c2@hPQ=YDU?xLMaRl>hol<_$2_&j#LoEARo z-mc!x_*_JJkerXQxv#-cg68)LE7VGqx1DyG=?7- zqxyX9^Wvgw#Q_`}#y}L4!Z(y}G6GBmxcr2qf})5c zrig=(i>6pW3bN!=oM|*zc?mevJCz9!`ykt4Fy3+vc^L_?JIegbckZ!fo%-SE9%A#l z-XrE5b}(huzlAKG82Px!-SKhV^~^6HE-NKtZ#C28vc2U^$Nfpc<8a0!23>0_?L=1p zL>>MGIa;U2r`~>BDZ*i}B_k^_$M;#=+-j&H5wP-vk&SaZrQF z?*ooK7N?en-DF;y_QxviIqeP4B1xH4VkpsKiUL8~vLCU+zz_%|Y>bj>AXOwPAk-gp zAE2tL8gxTp0Lr4s|3~koa&f4rz=j(DTEAF>Fpyg`6CBu=aLjzNwVTZzv{) zS0_eQIj?IaRq9GRUwmGd>TTM5=JrlsxYri(;u2#GG{h5yqxs+(Mkgr@_K~hqVt+_;=M&7}E5# zT{9KGSOL4JM2sb(>O8|y;4pK;D-*Eb)I|;F->eUzL?ZbO#l7?Mqvm{2_=l^bU{C7q z+vo0X;kDcMh*x~oWo08*ZE>dKLWP)rM;&4K!;)+x4v$Bo&j1ofNBb^g49gQsif3-vmYfodY_} zUsb{Qk3JNJmN%4@e8FHx*})@8qR#{?ADYoaWNZr4%549Y>xgtpO+6KXl;~Qhmk! zcMxFylHIm9sT$$H<$|JQ^Yt5X9;J!-qahuy%)j&M#JE0UkUUrw3o8l)Y%gt#$QU}7 zYp)#MaI^&xY&RMOg<#q*l{cdZ3H^As+EZ_8nW?mPf_CseY|?IADnre_vHmw_RwvmGFoDZ-hMh9Or4qzbp4|m&7BaB$59OfFsr% zt-$dAZ%%l814z`S6<18GrF=U#rTV|l0{pIk9$cZYIPDMWNHqVjBe7P8Nv~tsf~NZU z|9Zu_%8>lo&dbi?tNn-dU5wWNVOct|CiDc zy67%OczA2;Y^F^XownnrE}lRbA^Qn%Xzz2+7WC7(wT@6sCV+*Nk&)3=Uy%d;CrJ~G z|MH$FkF2MUMMV{+alF~@&@m}Pk@wSYsOAhaVTqNm>aoS=`9!(I9n?CJeOj7O48!OD z#_+oRuc!S-BQC?JN%m=}sCejbj&Tu6<_DE=5XQResc~gQz~m(VHwRTPsv`#Nwv0${ zOpu%^t{99|pD=LPMpIVk|F}nYedi;+aNdYXLHh?6s#z)|iiz@gB7nd^0QLGZ4w*cG zzF$g;IXpbPvZsj8uuKltQBocsyTAnlKKvBgLO69zmuC<5Qz_*SWPlLg1@g;RTTxl z3oi)8M35MH@W>U?4-MUjL_&5O82jP2n)o?kf}G#+V!1hzjS38gwa1APZ%><|lP#(K zD)>i_G3jQ5jgXQyFOEqYT(HfC0gn#|I*Yl~?-n+aF^IOu8v7DN@l9wFIr*5f4h_yX z44nj7S#A($=e8y;{0l*yF~+-KD<~1i+f6soII8Dfk%|f)PDs+f97&Vy7xCVBHh=BI zvyp_H?ta_NaIo;4;81;Rs0Qo+Q?HIwWIQTN9DqjLHS?>;XG^NLfVh67cRB=mU`(3Q zyQ?_Xc2EpL7;CdOQm{eP-b&m4t~v8~F-FH#_d$Xt_8Xc0R-7%4Ih2XY#Fuq9^{jsv z&qOD3)w?6u;hkkxkd>v5EU}Fv9{>|OvMyP7W-OJ>p{F3MST0M7)&r)jygY-`MQ&wh zQMp)-knyXF+A#vdh*OV5oqwb+0U|8wn;K-Mnowllnphwn8yA|+U0+!3ul?s+pIoQQ zL^Gd@a*TCc@B4sJGx-GYkSidxqKiu|^ZQBp5+B*8C2u~CRQr1#FuiL# z5*v8Pe;x#9X+UYY4yW&WFW~TvcRPXYH}SWpD6Zcb@8_uBN$B;Zf4&au_FMJKWs2}4 zV8wgBe1?3JWDcxS+Dq%GTEQ z_2rp5h=I=Mbblp0JzdMf%B8O9+!gagmGgI7S72+8i9QiAaYjbQ?U@H%#&|3+^x$xR z|Kw1s!_$pD?fCSxD4ed0HjjP|zL%%^VPdI?rQOZcO0_ROMGoVSvs0VP{(48#D#2w| zekdv|?0^Ok<}<}NmATo5#>UwJYz~`L@0;Z}6o{$o#I`>OKZ8f-#fo#orfS%!Ez z8vJwvDpmB?OqTEgJeMi0L!&9Kp*Y%9OFhC>4a zE|V*$cw^NUeb|Coa!yXpZEN1!l})Qe1Twsr;MC#mNIAwM1lS3PHi2Q3FM!J6y|==| zR;^5F;w-SCpy$^6KB7s8i5h@NMP(>CR0G~r4Lf_nShFmN=6kLuC$~BKoS{UwU_^f^ zLeHhfSS4#n5Kf!Stl8rE>xp4p7Kkx?I!d0&b8~1>4cTvkhJ^>jkmk7-KZw9(z0qKF zp{zgfihAbO)B1>cbk8wM@Je3rPLr`$MxXryNs}v zO$L$L?fzy}wXktoU6K-Nhx}d-iDzc$$o9Ilrk|V+r>9GmTU%REO%lVYdl$c|BwkP( zr?x#HClGxtTULOupGa)gerB#@W@aA0o|)`8Hw?z2F!~7B6=E`v^cxe-R$hLxezG0Z z+SHOhOi_He zILA`uTsnQd%p~NX+C-A+86O|NwzIh7Sm?6*Pi?hSr;*?M4^XV2D z5^_>y0@kK|cr^B+jN$o_53e7#ot?5CM;VK2M@FJQuaQh4Vc^Ywfgs*DbY}|Ki3A07 znXK)`bUjJ`jqY~pWwf=I9XP~z*S)5j zjkxCD36zwUesn(8C_M;~WKYZFFsllf-}v+9=zvRU*F-P-F|V%ne9!y&F&z$d7Q>f27@k#?Tbm1K&+EuJZBZHA zvX&L(Ew;7o?OF7HST_Pa!&6Lq_U_v~D0bGy<#!q5D8{enU6<{WwwZxS(A zrO8%X>s?ZPblx#{d%VEA?lA$Lr^RE^sIfb_G#}@jJ+ra#c?~mj4#$7I$$9eja{5!6 zBjEV`dD?%5__P^%v~xInJ`WKj!-Nd+RPVl2Kd*ZA@vj7|Jr7DCW-i;0GLOs7ltvU| zkbc^E=?if@QmNn_BC6#^V0!-IuY&KRWq;mpL}ws$88zQh2K5P z?agMBtKK<7(#zg+Sq^5CU&^8-qG(z_y(Q&4-#x0nHrohq84osqvdF5}W-wm2@!@Ql zW8L-DH@3x`RVufKmXTG1S!sR!)z8+D zV)-2JUUcr*0(o-&KbsLtypPY{0`Xb3+w5k3he0`h+3xnTIa{eW16kM?vNh*&k;M!s zdGUu}v^5*;9kpw|VWW`N`}0!$Djdi$d>Vb`eU{UrgjS1-5X}c!ab9!jd|df{c6sH9 z!rur$vTyO;8ptq{5pHpq%$~y_b7xH(`5w^wi?4 zrlvfFwe&N%JD?V<%CreIQERZ$sx=)CJS?#*osc%u^D=RHd`$VwbH05t6cA?9e&kTD zUhOy>9pe3Peh{7LT5BSWU!`0B)n4>j0W#Nm*4E*1YZ1VrRA;DVGF0=^>(-{cBqR1b zA?BGR58L?P;4~bGXkOfPsz|n6t@7~f%xC+T1nd?vwZ0)h9Cpn|9p3>&qVZ+p=yCbi zFSgp|(yET&PhXY%H-NroM60+%#d&Rdr(#ACW7ElWPLq*D{bgvdje3^Dr-!vm zgjGO4$I;!FB&2~O@_{HyRh7`_@$u1X%_ZAJ<9!G2+GqXr*oMuDQ2F~|d>txCc+{%% zQWmi~3+NaFt?=mRC~CN6z1Y4JoJ6Kgg@s4S^YbMj#B@Bn)_rwbyJ@v?t8&XijhXzg z^RmZbve>I)p~m1Snah?d?by7cbhXV~l#0w_x1V3&xNPRVb*GmFtfthh-#e_ZJIz=# zudXT3G>RPe3N&n>b|R#DOe;ib_>&8>k6_~y}{QOeJ0-RTe84v%l2~vv@oDqf1&5=3iSz# z*7FSO>c{5hW|Q9qrxB%`M_kLbg;9NH(l^uUs#dGtDS zWHX3O$D#=YTppfPK5>UYj=*%!l$ydX^PKk7@!&f_gk zPzf>|20yreIz0NyWwoe6+A!Jio!X*}*mN@OH-XcFU$;t2OO?0il9u!7z541q7!7bq zQ&S3iz;pYGFlCHeirx>~jckuox2IvcP)o7B1rb^`)mHtDO(cx=s~Zbmh}S!~7$E<1 zcToLy-8qGduC8pXUx%@*lkr7aBOF~$X?Z!sd#deN=(l)UWQrt>jJ!`@gm|8+F2621 z@>eUX_A_b7-Rdpb37HI`U#wqPg+{yk$%x59t{+{F=jG+j`%TVZ&LA0ic^f0y>muz% ztc#&~*3*$5g3^lM?W{+{RU45bdMeE9Eu6q~TcdG{M~ODih`1!d){ zN*gcK-pe=Giu^sdWf4e0=5_ro?Ivz%h z1WBia{H~(hoZ;l^dt+EWTfzDAlabJwljOfJLoY3$AO z^YfI&*CN?0_lx6o^r>RPJvkEI30Oob>-B}=Y}cc{q&Gf$7O2`5w`WVY^?Nr}kj|S` z=20yz?XB9y)D(>-cjV?52;b#-*o_f${qFPXTt|z3_mgh*390=#1sR6G<21HMj9v$n z#m1^MUXO{){ZnUCv-Mu|=Z8|dRU4bzsJ(mSw}&)2c$@1OfDh}A1Kt(MnWy#+kUSHIi7RIS)t-xPaosZP6(sJ)Gc@ZEg*5$h>@ zXSLK3*)=G;kuLCWeqoMHSX;4YVXLq*Uqqrz7`5J&xrgn^Jkia8_)Ou{m0(BTY zY%+UlYU=1w6ElGThHo;1&1m3aYcQa^v8ib?i}&pAVr!_-_Y(0YJWuESuyI|vi$cZR zGGA#+*)A&Nt_Q>WVW|K3huRPNk4oBW1oGgOuEK+^;;Oua5{0?39k68SI*5(eo7cJ4Ge12ubtkw6|0RDwd|uaK>BDz8H^49t*>*+ zeW@yqZXWCUCHhElI{gNp)%luB{*P9XjC<1-NhtqCn6BFXpZigP-2ubel7_U3$s{#h z(xCMuy|roR+(|-{++=NnLs_}q8q0lUWr*=`ew;D55}T{M4=*iI&*)CjObd35-T@M1 z)sO_iJldi0btv~)|D)T@X5p!^L;W1P$?xu~V)?93oK`Q>AGuc@-PctdM04L^v7oA5CqCRKI^>{hT`p@y}4F=iR<#}NKZzl%_ zZsQImxJf3zgiWs$K2JhZrbJ>Q2m+afOB&qgW{$InzusDs!2d<&! zyrz*+f(47sdy@~e=%@1?nrJ`RMJir(b$YzW4}qVVm`9EpS5@Ui{)4j`0*rIbYpG`C z&qkR%Tk%cuu`xt_7nw>uNx=@_>M|HaM;)wanYdBEcs)0S7h} z7UiCLDo6a}4@~0tr#b<@NS5POen;2)r`hC%D?D_ua=y%uPA#22m=^;*DE1RDHM6k# zrifv+Gz0!jL$@3@9oK4KL0Pq`VDLhR@yO>I0xNxCB%Zxgq1o6#ECI@MGZ`!Xzxen<_JVY#NULF&DCjQQxZ$T@pDQ=I?F(V3N~7@7}!iXbkeVa zEtV&{&Glw|@4VKdxLZGYOc4&^G=)9Prg1b`Yi%w6oJhI-L-R(>`fJAgbSUh?yEGZO zlIRMZ_TA~UDG50}l&raUp1sbi*;QkxBo%_Qj-(6g2e1hudyUyrr{LKP(O{`dVW%Yh z7X$ukR$d@ZBW~K*dtMh4lBR|R{`0OmGPmux)6>%$6VA7+l+>CLZv_46&-KY0OgH<& zX0PIhjcsfo^9Rij3}f$NeJvO<`d1iQ?Xa@7U~cx(;_hewV2hpxqhMX zD=VccWt57oUk*mR=|(;bu|gckOUhLmzlpjF}oAsuo)i!MBrwcJaqbu>D>>8p-b zE&NUBY-$UdQqUE2J46qO$*${`V#5S%wptD}gDdPdaXz|He(mbfN1O))>)5#Lb?G-2 zY1oe>{uc{SbTben{mHqLEgzcVEOK8vYj!QAW(r-d==sUmtr0w4>9SK1DBT1vK;e&H z(go6#fCf>_T5ZOoyA6sX*}Og*N)jm$?JTLn!_8makxR`+X0v-`sTY<^A1P?{aPP>2 z@SQ{|nW0I9nF%KsXXiIb`!e4fYuUoDymay3ykWnd;S^4XZufl6wG+11>;9S%?O-{}rNe3$wimcRh6v#HoY|4T{Zr?`=a>eI!K+w^!5 z{gr5FraX%31fuJfVqFx2(Z4le#E3Gqx97u3U}#1SB`$dT;i>PQ_5^)4dV*Y5KZ1T& ziu44)Jwwlu*&los^OBC@8Ay2UPK5?$rl-HlSo8h%E+fZ^a_dQ}$T>Yb;&s1U87{Y$ zD-Ky*Zem5}NHicI4x{@@d|26_Ugg-)eB1CnAtRYNdwv>oHKeD{dhdN_v-wRV!qwiY z+oTsM#`7PgN&UKFi!WbFxbO(a5N|2f`y=<cjf4t<4qYJ8C&FMR2=qV2d{;lF=l@fy`lYO_EVq$p!JQ{1jUHlC39+RF;`xmOX znr-)1cezZyt-94(Bf27atq%L%S>hC{P_Ye4bRqg_)<7ZK36pA_19R)_@8(mcYc8y9 zPKIl|Z_*dlsycS65X!lTM`-zOrUoeW=5%hO&OG_a^69)Ur?5}X%-ZPfm5{IrR%J)k z*1p>!_Y!S{qP2USpF4QGwOMItKq*bp^Sl`f>bCK`aOo+PZFnVoJ)KKqkLTsv)`%{DBZ1X_s3R$z;b_uf$$Quf_}yc~&O8XK?+ zi7gJZg8Hn;#A1(C%bGe&e^GL8Ct_0IbyI)VINs~=w!OaDYNky4RHxAtyDs2b@p%4v z@rw6`gOGA-Xw~hkky-mRa&FpVHcQa$WqJWsBq}RwWr|M5YRTSX*5G!vY0cGPt%$h^ zx9DEwahQWVlY?Z~RLO>v&;54Roby+1_UN9xm#u)b9obA+DGb5!=awd$^%Ix3O)GT< zGr^Vb)v8BLb!GYLTm~bST#mAj zMqF)p_@qMY*tOw<9ccFA_MPEM9Mx+Zgwn z6*vx`9eQE2$Mk5#PK)b&AF6H(n+{W=>_$ zK(DK=p43{(Iu1Z4hRp3wd4GUV;%&pG_~4{&zR?z4uP>G-j+3EBqrUPb+mZc|n5fQ0_=aswjUe?CuEZrocSf+%n#>>kqm_%}QzJ96x zyZuI#ld4n4sLsJYpVy^~T4c>@Mv4z<)55)jnVRa{jm|-2e|v}Hh|~{(Xr!BCHzSIZ zMK#te4M-L=N%(Y{yHef3o`{PLeBWc-?<(FG4riHcsEK_QDRJJ-l0lYd8~hdxsb@I* zeOm6YA}dM^qVlsYX-Z53x%Xa{3!*Y%(GRF3$~y@qovU>^jO`ta#`;>^S#B|k;QfGa z!39lCO)S7InZtJMLGP>0Y7R})jD?rqS9bMo)<@gjF9M;ji>K`YqGIN&^;|xIgsd3F}MO z^F}cy^_q+vWDGbGe8V?7%}>4LPp&5|GpM^0)@9{vG+ti(d-A06=x77&8i7OI1zxp% zE2Q}3Zrebt8jUwUWEN5AHZQLBSop4vQ7vX0WG&M{waw;VzML;FaOkeI`_apH6N!4k zv0D1z^)R&u zDi*jiKe{xIear8f?TsB$ozG6J=KV&W!*89+wbeJA;H%DeJn2zE*^sY9P!k3IPrD1AwmM6yl2;)F^p?G!upWI&^;a{TMxxe|7d#40*7|z zb3>FteI`M;tk-oUT ze5(l}F!0+`UhdoSIhR<=oYr{W+!Flgxh^z7lPc5E;g5re@)mf%iA)$OPs629)rS;m z)5f$gd%)3ozuwsQv!C4OwRO*foluI09g$?(sAD_8i_(I|sEY=1`dzw}_5yzCNX?_b*lzEL1V17*1lRZs8Uv_tV19i!XU=9j8I-4e6G9>eAKb&~ zX0TyD@RF;oAg%%qHloGMLq3LW zE?T3;17foA0gM@R1T(v7u?lF^bq0j#i^Vm0FiCn`D}0ARZ4{l!GLK;iaHWwLsBDP( zj3YL}A20>`3uu)-M;yimLv!iqBu|40$UfrkaW7rQl6Sj>F?A*P3Op(iO4li*#tVYgW5 zP!NX+JklA85QTS*WPK=zP{DF6#`>33OE-){NdQHsv$dui?%%>_l}D9S*d0&0Qn$tm zXDHL^xd>t|F&2;#h>WL0fo%0F4w9M2lt67R_!T41bGI4s+Kw-9(xET_&@+<`d&bu- zOeum85{sKNPC*8@Ci*ZPX2kk=?SeX_UU=|@vF#JaXwg{rC>^(THbe`T$*+EY(|+XI zW;A3!B090sg8ClxZU6~EodgKCHu7_L>x5Mr^7yOQoK`~~4BJ#{G;0(muON9UJ48Vl zDy+YaWpXLhXo!}p2q`=wFv??N#S1{YZmWGgA+2ub4@`|sfgBkZ;2?5s_ApsU}vzK>6WkBpg5m)pTo#R$Kz^q|FIv)bk6;RAp)(O4< zLgt8Zwa#hF?%-LIOx1f{Inr(A$-%faSXL}@+Ql~B$QUZwe?41xUZme3&ioVgR;$Av zE2Wvg2v)k-5fpYuYC_Bu4HS?kZ~gz^;w}v?0HAt=9S=LdCf;9AV4k*t1qG(<%h2E! zkLurw^FQW!|KmJTsF?5l$dA&t_3!_Ms)b*g3!mKM?i>;nm&^SNFoHgJ5{4U?^1nf= zVE4uO^N;$SA^mn?Pnq#5Yx2O|2M2B?8N{Ih_9ov{gW7S$u~RrWD2DM z=CV=J{}PobPhs?re&JA3L33zpw|=lq6n;1&GCho>|3Q_7=XtU#LIHqPuvA1`$drf| zW5lN0-rG-+96*dmT2Kow3P7Wz?ElI zqHnpLlPezs6B0GTYq-BkYJVTXtFLETB!TzwhsgL=MUIJg^;IKN%8p5S-&F-%bmoZi zEv1df)YEVVKZEmiObXtFhl;;yL}tgMi1CXM8T0rQnrIMx7DmH8M4-r9zq>m-FK-?s zSU=AbQM$ZgZhP&OR|*Qf!q6w!rOukW1rFR8nC#fKPoa!pGpTsy#OMUvYmtTh@*&O2 zi^TTwcV zn$wwinnX^kC>!R;$@5uoaCkXV5`CIENA_|57?iWl;jtbRMR%?#rLJ2@j!DoLL`PUK zRw-iL57!bCDSsij=+hy@>tGW{;giZ~qmk*~BHwi;eHFDEzvfi>E<IiNp% zw#Yg6A_UAbExxiH1kQ|cSvL7RP(K8RExwXJtlp=UN?;Z&-4?3U| z5jZonZyvJ1j)cw1v*+tG9<5!Ls^H89F)*gxY2Zka0LYO=C0us@UwGGw*MgMwTnH!jW@C}5&y`Iry7;Kz+) zCDWcIO-2ULB-&0jjLMY0K6Zhz(1K$+!@O{_YIRT~GD1~iQ5o;JXI+aJ?(nzaWi}YV zi9V=N#kchOjNXvo;z*0gmb!Q97sT-}Z|$`*TWc%33ifl5zy6dAT4bGdmWXsKcx%;f z3nN3w)bW=W%rM1}HOK_iALI=s$ipk$Mx1~V{~qR780 zYfgCJ5l5##qy8I?{;WdF^T;!2fcLUnjWl+wltn7A-+f?nfQB)h^G zW%sqir1ec*$1a9jimJ0DqTL#D5HXOf39^YB4`Dr%4h{9y&v#jS4Dt|gd7KW5QZW}- z>JBq8JrQ;YhYjmMpwSmq?ExZV4E5R}-C~PE7yt{sO-4Y3Vi<;OEnc*gywFWob2obvk76SUKoC-r|7@C?LtsuJ=DT-f5&mvnXdIg;?81W0 zjYWsE+BVPE(yT>F4DsLWVsn)A7W)h^l|SG`JfEIErnjowre4)0$pc#c4wBa{}(@i3OcCnK&O z1hQFe8_!rwCqAqX7u(bCZGGM{EPl7YM&AIQujBpQ6CeRqh-0?+Il>!gKttWN83OKQnYzNFz@Y z5s&+JpycWu{r4!UKzUm6np`C7%1dj;B&Toi`UKI_mS458M|Ir1ZlgbJb}xvP5=HcY z@1=gVwBf!n4CeMHV%NzC`Z!W(d^bmetJi~IK1u#z zLOg|8SLQioDTd4ri;{}kocdVCA3`wfSOb7!|sHzu{uS<+zU5;di04+WlYk2~7HZf`sW4JECH zUD0x5F~|%I3qz54#>t^C)k(UM^cjE^ukgv3>XGs~t|t>Qn5#dX+b@to1~)Kp2n&z3 znQ$!REg297e_GB!X+*lQO@(y87DY|p(7fjl{HzifF_#P@ag`FSH?B;3akl@ry53ql zbx>-TQmh~rLroJwv|B-!)xR%a^*)L1Q7RB&kvnBLDBvnBPA% z(12++?XOK>sUy#?AfODV^)cBl=&3^ZTXc<7v*oR%7`Jf1EFmElB1(5DkaQ)d1~PmIG6}ZZMyB7Vi3RzF^7s;n>l5J7(313SD-`9 zlZ5ttrISrt{8XD;TlH{o;z8PV9;n|dps_zEt4~S+g5IilgXEp+%_gJwvxL0|(`-F3 zGN0ZTsm>Qvv3>lL*#Y8m8^PKEDs6s&;#%WD2?p%ei;1cv4C#2^IE$;9St_j#}`{I{-tXn|H5BVOUNDJd(i?qef z`)5*oR%n`OOUw}BbMDt&#W5-Hj6?A<1bZa8Epy7;6?`d0viDUGwk9sp52@`N349+2 z@wl7L0IyoXi-HPP*uOQ%Z1v~jj8l*Zq2ZqbOi4!PRO+~2hl)amTg=Mo)Gu8-()bi{ zY=_cF6rvv*Okg=6Kc*F_*zpp{kO`$t4->4qW)c>=QNw=*b~d8yG-ukL)etc2hBZfHvW_3(Y?)yzaywpq5fUjSv4MU;>6 z!E9>J1D1$u^^2VRTvECFULu(I2|uFWy4~rj4gnQQ(c{m#hrHlbd9}b@Qq^fkakh8Z z&{!7l<3Ee~Q*0e-Ly1R$zX~vcgffL>9#3J_%RXz*w^km<7j#OcDt0AQsBfuIS?e6{ zrQUl!Y!83r(8jCQu-ooK;JE=(Zmqb_?JjT>bF0y9s>SPOg}fzoJ&I~l?zoFfUus-s zyAA$~-f&UE3C#gZ+;m<&XxkLl=?%DAZb9$YYIJ`6wLP%d{gUm1L9*QHVlyJK2v)Ds zYVh9aDN6rQvu1a&Y0;|p@#aO-tX#cPyHnrVLfXM;TFOlQEWgv$Zui*DLXD-YItulX zG==rl>w4x!c`w*s&8-uFI2gis-VO1jJxDX<`G3h={)CJN_d(zUz#}nR3BSJ4IbD z^iVgN1>coiFPpZf-=oheod!lrD^{pxuD~vfEXT=W{3Ao z{BYEaXaIl68vmcg(>X+rA@NO%*(zNgn}ZNp;UbpMGBbG^sz1%M{XJC354xS4W@5zJEKpS|c} z$QSC(*J@ zxbBWwba`HHj(Au3?F|<*gD?cnlTPIM9JhbIeE%#|(Pbnmt8V5%8t2v`^) zadY~^b_B?lXE2mEv1gl%IwfP$8O%V3i1~>V>E`{o8yeG4sLZ+sI1qY^V9M)RfQf1* zr*$tPC8p!$Fi2UBDUxms>XiffY%42jP#HalRx}zch?bM&3!MFlGMy zNuSwlfAJ@K?)Jo@lgDCq%*N|Fx;V%4DeHC>196^^)&k2yjj7pY^fKpZ|B;T>nIkWV zyI7P5Ku)AgfW+3PCO2jqA4%zXFbXNDs(Lwy@nNJAwrQ&&1HL`MPvYyPu6{W6=Yo)M4ZX7h>dCb;74xpt-Z5I?iC6Gl`Yc?46 zZ)VVbnB}&$X>1jib&AV&qLMG-L((5L1sChZ=8AQ z2#cT|%$Pl`llVP58ES)6r~NtYAHK6ykJC@yoBL3(tB$_o_K)Ini@Pk2x8ZB%ev4nC z*vg^W0gWOj1s_%Fn&v+4@g5wW%HF)iV{~ce)-K&nA%-VfC4bFO{wIMWkQT`$j0n(I zDxEfp;cv8pMg_M%_ZNdBGqISoTY=iz#WER-na4bjBR44-BxuuJ5p$Ofy2jp>LZX0uRYxm1E{3+w&$;^+nt6RtGKQRx9>7ZpdJLv5(sC$S9 zLIEoFSIOu=?&-NDO>OOVgWiX-lH28*EP&+)qKJO?2lq0=3-d~5$iq|Cn$P|(2L_w& z&HgE3e&fgAX^u>I(uEwe5#n35*&Q!s8jcUs$-}p+7tbMN{JdYxCbD>&S8Yx8LCiA^ zPDisNjh@|H+Wg0Zcj>AcRU#V2piby z^*AasJDGOGA>-n5`*VFAy>belIknSxI6hbAVfp|AFrRr~4os|Xd1Dw>23BprL`Op_ z@S?-ftT&ZafPEgGXh}arCtfPc`?c42HWrFXiw~x*ePL3cnXS}rReJhU?Q%*@K@=fo z6qem_`tT!-t8ypXL%`>Lu=A7C)7_!il(v@E*K#axAJg56+0Y~hGeLsGGtnpx?oM#`;O_1Y0W!F|-1*MA|LNE6 zhw7?bdsnZ$)@r+nK~}xp%*(mg1(#noic(}Q5 zcFuc`Gcs)k*Ar>~rf1Mj++9T@2YE;I+kC6mZ+po2kUgqaVJvpt_MX#mZx?g?=}M}+ zZ7MBHA&@V@gihvfH%)xJxg^t5Bh8^t$J8t8ee3=C{VT%rYP*~5xu@+&vYBk53=^h; z3@S(gfJ9}9$^k$~uvF685cag-02Th;Zfq3jl8!-}yn$3zogMLM803|iybk_4e|y8| z6j8}jWY)$e5P<;=I)|cB3`$%eJ22h;)LbT@0nk)(Hf$Gb`9-5*6Q?_iv+Gw5MmWTxguP>nWvx?BNS{s-9nF28dP3>EzTMWTsUk14Fe25 zojwM3hc#GkR!EKmr3fmYDgl6&DKPot5k--d7ON~o8JyiiDB`<8{b9(#KqLS#5kT8L z%SuToP|(UWE8m)L6kBL*N`(ucNt-=Ly8DCzhiz0em-{S1S4w)B3ZSsK9I~7=D+L5; zn$v>mWvKepoR1Q~Z2QtGa-X1bB56xaPv=w8>O=D-}pwW$ipKLrGu+uBwW<5*JCI7+^?BL@Mb zY5>8&-9o&mZVC&eMBSlz?^|6C>U_!uZ%QOH6H+<^fEc4(RlzSc9^`}8Td~vM`Mf`| z6-u*mE*yG-!MVv~05eLy*1mW(1%C;Oh6O2qAg0`yRywzQ6DYMuVU)D!k|%CfvL?@g z?Fq@n*GB0icz*r%-~;L~eh?rlPQZE`XeJvR#X?gMhiRMIY(M7#$_)~|Av0Ad2!+8N z4KD1untf_4KimA?{-ab2^xv}faR7pu1@wrT#6!_<@7H4_WpYq;R``-WF7PnckrW%D zbY~~t+&D?mRq}`lC~^72jCV7!5~v7sDN|vi%3W|we2@i#6xsv|C#VKAY%%uCSk1$& zD)gGGpf#+cgVebNc^b-Ozg2&_$gr;FW{y?myO1-7E*W8Xh^s->ex0|675g7~1 zh|=)aOwgtL8QiHws;A;i=F*QR0@-x%Fhvk5*YNf^4#vk2tAFHuDm2~)>nr3Pw|BMl zF`0bN)~okOtRX5K8Dd5wZ&}4*O1N?kYc>4#(Oj@4FMoF=?N`-Kh_xm7e!}y}o#VbH1Ug&a7D_FJ|lA z={~4xA)~4DanPpS)eg#@3X?KjB0kn5vN2GaJ00~s9k5*b@M*DviR3H`Z=45@|9M2Z zl2pjdOmf8P&+5GMX~dXu+r~RJQXNF*2oeHo8>^Ndmuk3htNK{ZpX%4~XeHCLv#ZWDh@1aFhvDV=*|D;g4L_}2 z63b}f!5iagF!JuZ7v!-dzo%QeM3|H;BUubMh?j~|Ny$PG6+aqOl(c}q4DY}q=zA$= zpidZGv3^z8F@oi#WxIZJKwN#gSzFmz%9yW>DT^okJVLLSRUjQhE}+`xa=ob7G`N<$ zYlC*H`0I8BoqEl-)AOozAAXFdYWSS^C<9$rM<>K)u>$4x?)#bw1LOK_t~4W(Za93# zSeBk@+niEBG0KOGDjxQ&w&O-EWxI_wm$@emH|ag-6y!POc+bynhW<2Qq)LRmQ6Qlg z0TU zjof^jOnbq)dR}?eQ`9=exJY7f5C&G}azz#q))HFVaow7Lo-BI~5A8jeOQ)u)3_Og$ia!p7ut@UTPD z%u0g3*}9_Xc1bm0>R4HqPZ9eQe2i%SXXne#{g#(K-M3upayDirw#}TZghNvD^5@+f z5ko&XTL5-9fbttU>3X~U;Z>%7?()O3%4-s{&qjx1(~zPU)RE+I)3*F-V>Cj)1**|n zpX{w*VJu@|ADbMzjgH|qczO}Fbi*x7ygF_Envs5&&F7Pi)$N>^lA=YnoRM;xdV#v+ zdnx(84XOzvj^3@C?K6Kysk1fiI=?A7JV$+0C0dYJ!%&#r{sKhAoBisFpLHi5SEmLxLC#ZFqX3kIX_dg;^^LJH_Mad;$w=`s7t_iS6qjg0KvK>{z5} zvGzW5zvsph_}^|@0(fCe90u197m}wU?*!n{B5#<&MEkDW#lJW6Y2Xip``H}=(IkQ` z6%_2)7l!3s^X>(|(nN>LsE2MneY~rh-VR=c+V8V|GaAI@;GsVr6A=;HHM@E%wiOGDn9Yy4ieF!!k3L1qBV8WsRdu?NnAq8iA7}+zX6Wg$@HC$;H=90xPEk}+ zjGBisLT}K3D0Qu)qhspygCW<4h);t_9CF{jL0NR|RJ@3l|B_I047)Z-5QDnE^lE-p zZFSK&3-x=N=-4!7g}B)@qnKegtyfeyFl-Nx`kLHt9xS>7RC=3i{Gc1DpL#_}fPyi$ zJyPyNCfJynOuo<#3$EAs;LqjLF^s# zte3U*NYB+P{7Q6<-_y9I;kqwJ)NJx!W#4oOVyk5gQocW4_mhDYB(SX2mKVJtC>uhi zAglC)yZGJs-Pici(yF(s{Yd*ix59ts)GdWe8v(Mb7$nPnzjwRq{;dBl`jdV6w8}mx zh2VE{O}=$})ahOtpks!I`?gymTq$t<_3$+s_M`F{UriTBPv5mxMuyuNE8pEbc!jMU zFS~Wi^>%jP$zKux@q+y<;MCOE_=srV1<6YP?AD}NOJU8<%q*ncDJ;B}5vo2%V zOh_@$xroXZHeA~hh7mja?|UmRXbkx(zh|Y~C8c_!h0L^7!mb2bUH>bw4{Y1=3Y0`j zCD*No*%|5SZSI%5v#bPoqw9axZ8LZsSmswe*pZ+ykE(b)D-!Z@7#4PV5}tV|(|voEE4*t^fsx{h zOAAbVk6f&p(J^tF3auVmhN2v`u&^-wP@c)#TIGZS+$?^RNI;spO6-2qTF6Ifj0A{t ztA16f4EO#FYmxWcl9ruGlmv5!n5?#t# z-keBOR8%&<3lqte{d&vWa$I`%--p{K0W7nDVTvMSuctH|yY|N&@^^yh{=}V%($8O1 z|6Kn#Jp0CohoriQDS9_tbrI5~3^&hCkcrx?foGy_m^)n_dwB zTG_3%QJ>tuJ+42Gf8rxnvw~`+$jjO6D%!aOd1`GA?-J;heWAik(v3Z*Io$Z3iqcXR zR<_dii^uW5?&gx0^6XtN=UuO_4Z>dxR6Q>)@(zmI;@+xQLTybBw17mG&JxKF*Lq*d3$_!c78rHpf~iro%cFlKPFUxo{_s&&NBr4?x@Yjs#utX z9(SF(9w+SSUTP)qX(*xUPI-CsdBYbIPm_we{J#SU?C)=GT{&rJ8suectO5?Fqx&;6 zP+TxW^GkB^EYuu~rhJ6MNnT!lt&tyL2aPer4oI0AE1IX@?CWGSsONPcZr)bTJZk89 z*}C;|wlYQh#-Fr4`Q^ofUUuc}{sh8~abSI;QD3dc~ zrDs}wJEl8@f$Qbyub;|v;5#}0a~Ko z#t}a978e&+@$v-JjgDdn`d$6)dOtodu@p~Q_-_idzCov8h>2NFIa~0Nyz6mKfuhA_ zY|yCt{j61)rgVC|Fj~~k?GkY^QDRR8D5ND>k~ z)|jKL1Yb5L|JCEN)$jR!%fp+~hz1i>S5|j2otM5i*p?TN%@VqEsZhtkTvq*j;y`}+Y#2>kE%(`MqTdD-Gr;rIwDN?cS64fBlM1SfQ}OLvZH{~*R#{XF_(BsqG?A? z6toaHgaRFP0|VV3xo5qH;DiEb0ijp!ylUTio8hpX@;;Zav2i-=a8233M|(=s0a*Nq zG>a8QAT+xm9Y{$PBN%}=cixSTiM+fSeg=gWjM&fq0mKp%()Q!DyS9GkA?}UwdjnIV zqA1F8mdvep+D4YRU^X_})#<`98Xe5L7#t1i0T`*M6nh z7N*+)38$8h{$#RQJf=1HqHmzlN^4oAB=}>qbMNI{d>j&f@m(40 znELb3t3LJFKEz^OiBn*|=aCQn#)da=VZpJz=}rvpx(*rH_i1mb?i!`pevO-xlQJz) zZ3HU|3Lk~BD=coh%8F%Pa$=^W;=ssS4A6}^0|E|wq}345INzJLKQ3YiQ|vc zkY@-4$w5L>GkI*Eh=6>FNnMwKpGu@@EJ~tR%YI|4qx$<{Q>B~z)@HqF$CsT9Hv!l0 z8@tYy@&LH(E2SqoRsd02>So(*c%zMGQ^N)>fP#j2(v&`XwXJMBOZe4zkvsyq1Tx9Y zuj1YKHg~62>FAJ|leKbLzqDkhE#TSw!Fd}tA^6|TyqWzHEK1bfTR3^g<2OkiYd=3f zxHO#!&mlywHmZMN{(a+YENpC@E(Lk5J0@`Yu;P6l@GFu@AMd?{<+PnenQ8bKVcUk}IcnR)IX}fQf%#*SavjVD5CW3Ks$7V=Lxxp*GDYFej&|97VpEM`zyYP=;%c_YS0y`6~VB zn2x>!umYf=#77(Vq^C>8j0GqHVE{ZS?S+MZ?|%JEeu~5zJ|ZU>otWZR(YemOrv{Au z{Wm{v=BLbF?<&q208h25f3~{|+kB(`?}(gea$<_HOwhUOFj3UZ!GvW%EmlHYE=*BT z!NqMe1{{sJC+zVo&8$@&GvqZI*~Gh;)8*B!!Kt*w^Kx}4yw+mpQnOd}s}Dzg?zN*s zAxy-dE(p_Ksd;Vx2U>JACpeZeRcs!C z?T?+Wz6sgf{7NH55yDgL<_Ycd0n&+1w;IhSjv9|!DzyNzVFQ70r6aPD!4!;WIKea` zAv`5>rIt(6+LpDao5Ky|M&=$K*7o-H1R(zx_3WNAMA7k_bmW&k<*(X}cDMZx_)1+v9R#$#a&!>cwfHAjlQMi z4jAxLIy?N|C)MI5Z82ua+~biAJ8T~!dEuJqK@dC&hi$Y$yK5Rr%!8#EXVEl(d)?LA ziAdn?q{Gt&E)@_45M3oOel0VHUr`zekUekUk!48i<$#%g(iltLZ&4X2E~c~e?i682 zKXsKbE$ghM{D#zL!H&taxdHu;|mmBZjHW9 z=Olw1EDpH)61uX|${@*2q#-Bt?+n@JV&<4WXJTyf&LcDw;TcN-4w&dNYgGmc2?k{)2Xg{ zl!MmHWD>j%KfkAggR_ar4{_tO&e!KpWRMn@Q0FilWLygE$Kqnx9(T=TueCFw>|$c# zMK;4mmXw8)hlhUs*?!t@+7=hQ{+mY`y!LNh?>Xfx-_l5 zy+RwP2apmZRQOp52lK;Rg)hYBv3z`dI!dO_zlmiSDNW37Fn*D*bc{nd0#(=RY_Dg^ zu6ps9O~3OO5@nHE-&zRuh;bxGb1m=GU`_>uMkSt1NuxhP@gn%iQJI6*nQ)AEL z-rnJ4Xsz)sqP!ol!K+(^5V)g~NP=GC`|WsfLb_~B3%4pNkjw|yh2isF^oCa^=G(=F zUrsg&^*9w~D_^T!^YgQEmitbB977x7NVu`3CVDO$LKdr79c^4{+H{!iG?+uJzOvgm z0o+!Hr#CDShCJ~n6WBh+$2HB)5FNNXdPGjP*6!Baw-X^L4(J91bJ5eY583*@L#U?& zeFD3s#AHN51=t06h1ZOuZiE496sP?ak!DB~X4u_uYI&3d1aP@&83`%-+)@ah{ewkT}nH9 z^<3Pl1ftAzokR9|t?yint`aIrx_>H>zdLW+Sj9y}D8MpzYQaklGZ{)ogBs&9qi6vJ zNTmcZy}S9l=pqdS?w4mg&o|uud}a3!nwhq_T%VK9@p)o|=K)$%wba+m8r*+K703P? zcNG-_C3AILjoM(Gxox)i`nbDm)BDE`i^w_s6j6wZadmUe_I_MA%UY}EX+}qjp@PK> zlKFzO#OHow@O@KzjrscdjPR&!W06VQ$_7R(|MQ1Dg;z{)#QF=cWR8#j z6NOA3MdX)n=aN*SBQ(w(q{&Em;uPC2>-x~UT+__MK&0gS&A8fpAum$&&^i+YxhcLh)rE;`Lfuwwem;e(l_W&5w24m0R0IAIdv>@SFT-2?o$8v@yL0scBL>9I9I zAj)ZD9NpRT^-~rm#`ELzDu&gCbzv6n>C?u`s;12iMocKwpVP7L3 zaU!KKSzzP<12Y#Pd#c#zp^AaDB9sXLl_=CeXP|`yLRv<~v1>%$=JR;Obn&zbq_@a# z;*cj302pcOC&nT`Rs<|}nwu&rE{yhSh^UrCP5o?ZY@vw{uK{BAZ1@KLj|+hO%Zwj8 zj$kRk(^W2vfzYRKPui8mY4sN&l|*uddT^M?r~Iji)@IZL8nj&IeaeeKZ3Ba6MrfR` zSnnly8(1o%V>PVuLeP7<%J4`zX_>oSOzi+)fjSXFER_ke`3h?l; zhKUZBbiVKI^bMV?*o803ybo{xgBt;cSRp-!p9^dGCf3#I^Ms+Ip+%u$NV2T_I2LtL zG?$%-do41TV{G1V?dYs(s&e%F!Bt%Gf@ib!G@@)7w3y!ey=O6gpa0KS`|TFb}kwkT8ZTRUo~}ln6Qy0eV^^m5q&nBGVaUE zndQr3&e+*B6ztkhJ^S|MQ;zf7s$c+sDD;@%z_=vh=C|$BoG9fdiiv)@TysE`)Wwum{d|ujpM*xWDs3SJ~vdt3_RPvl|_zUF&UbS#e$A)4#Xbh!*qK zEDI-tjSntbj&DpfZ1jY1I$x`KCAcj2+j3q`=zF?PBJeyA=rvUacHPc54g4yAsHWeS z!5o6X*o;|Zn*P9mm19;?#2B(l-z%^Eh0yP=?fzEwTJ~Gp1O#I#RmYbP0&GF{cwr0x zsh^0S1+Ha>H6Nf06{&4D3zpiZ1E|%mULTU*{K6#QcQpM(&@i2B*Q#yAiwjWbruc!9 z(seOAR&cM!?su6+UZ3rCP&yDT)&{g&Eg-^052cP99!+A#aXVmCBtQi;ttt9hNyRTL zE;e=jJ5Ar4;!~XQXeS)TtN+q8Aqr{@88Z}ooc-64aqjm9sTlS>ODo}!1YiC8XVn>$ z<9!6VbbG@^>vu86E!u4~%sf8tAeZFL6R8R8p{P~ZsvVCKvy{Spi6z1a3mb_ljC=%l z(FWm_ciebrO{I#7A7Y>(RG51x8F0Aw?4ckON=`!BK1a)NIc3WNRRIB1@TgR2+Ny3X zu?U2y{mJ`ncyg4eG7Hl(L?vXuBo2shqXLb56}73vL?mo~)5_uAWgja3q*+D|eF?Dg zUMaf;t~!&AN%p8_alG!D?wd_B;KX><7#%LT6*6O*fvNlQXnXPW%ucojvyWbX- zc##V}Tt80X_0&(1%&gcp8e2JkU)~@0yz0My+e)WS+wTH?3N#yA_76l=fM~1x4DpH zF8l;huHyY@(BhDWJ6U=yd^`F&w@KSZ25+st>TP%a|1b6^&X$lBfO zsf_Z2LKqGxBlH~y1rA7MB9*m+S7Bv)S~UL^=pQDGD~(tblK?6*lLCc^#mSfrj3%XO zz9C>G#M|A67SrhP+HVThoQERjcrIEKWi{&t=yNh>@6~{N^PLVp6#z`A#QZr~AwxMt z_iv0x&9QjE1ib3X*Y7p-j1U5O8P9mBOaT_Mydw3$1=+M|pd@$o*B?)NtGBFpu*?Ku zm2kufK0tt@n21qn{6`rXBZeI)3jVwrseJFe@jmx>Epc-KmIlSDfZzrtg-cNu`BeFt zF1Pf{8gJ-W_`kq+0<>HD4Z;DDJ5-$1-2FhdW|q;M55tdj8TLi{;j8@k?w0C zZaqQR?Rm4iw7ou~y1vd`pheVK{vSXiwr2~$F#cbUFtMP^(U)0tAPNhGb=Gt?sq!+d zry$NAPC|I%dlalVHTaDny(+8yPK4{o%++;~yC_2Pe9Pv23<5Sr`SiFcEFfUh9PZ9C z>v6sbiBZx^mh5(MLCeV&5a#+kGHX#UP-0;Ln}-SGE`=xz^@#EQ!C#W=a6O1N%)zmH zISEJ0Ri|df#DN4)^GxULVfuyW1@Bbd)=HNQJeN1C9Q)7@N;HMN0%9{pIAh_(y zy&mQm5M-r_S*OgR!U2IuY1R6r?>BQ(Lqh6S%eFj2iS~4us5r%4K0Q%O*E@!r=clBT zdYyGv=TL;}bhYvL-0#AXBY(6Uqtc)Y>tItC6ze@V7pKzCVW86r|C7>t4ySTr!tbR| zU=9rK*VbCPfRr3c3w}-hxnY-+mW>uH2B6R(>`snFE;3T=nhpmD7EoJG`F!T&OGrq8 z)(oaWOIMZr~@KB%CUqjfayt-Kj)nq9%vwCGXRt zp_0#yi6TZYiMyw~>)wRkP8?u>PVMl+^2LSGI|OWbtyNXY?=F#xj8IIa%3P8=eSE8= z=vhoJT|z(9m-ic3+6~7|I|K{fR2kQ;_az1~uGFE}yr$!U!Sy>~qa*Cx3Tq2L2WNJA zWMk+FP>;UR)PFW)A<^mbU`7fL3zXH0m5mElU?TFl*zPGlZAV$Mg~M!{g#n|eVXqiB z>(ti#O||zt@^!nPd__FlJOw?8!x3nWLYxD;|Gb#bHt>Grjz&dO@*E5l6)x(NxYnf&c*_fd4aAv79OL zh|-WnA%q-zI{a{`x29|!3=oZxr^?`?!2T}@F>6;+o5k};iOS5Er1U=Z>>Wt>rNbxg zJq$$C!d9kK8ohW{BQ^XreY=pKY)e#>ScHceSx8dzM`&x9MHF>;s|~LE^HKINwkRLb zXyBMR1Y&+d!s8gHv_Fw#r=uZI{ju0{m0|J^pyk<0?>U3C;(v(%wKTy!Dl3EB7 z!T6`8tyEQQ!g{%`cX^GGqv}xX;V+ZwL~#jsb#qS)4nFPd>>xphK%n3VBdVyZ^ym7_ zi-MFr+k3^1m$rQRL?HrWZWCz>iUj=C=D={DzTGKgh zOW5J_Yf00@?VVstPZP6my*S9yNVF_2t0>#&f*ZwKI?8EF2LNk~YPdJ_d7B&yK147eEo`;fEsN}U#csMNXB zkQTou+Qp6^^Y`$Bf@cZ+N4jD$Sy}0t7hgY)e9g)&x?svwl>s=$t|?*^X{d$H??+Ra zT8{~$_Zu@BY*|7sd_u+9P1ks_IF{N13rkCC{0iN6cz}PB*%$y=*rN94AlPj5F)&#` zzAn%#;4v{Sgy8x*C9Qa69?Fjr{C61zk0X$o%2&+M_%)sthDo&+C2{HyMQS1a78-zn zF}8|d$C^sY&%ikMC!M&9vKj39L{_Ueob6EM8fRag2(tgDRf1qM9zNgOtB?aP zAEwJsm@*c$Uv6B-8t;HbL3cbJY{i@t=_6DWl*0l*Lr`w=&UIk;4x}7<%w-(vp00H&! zVKm>vE{9L%tKT3?NQOGrTpXnrU2i#tCHw81gaQ;-LWiAFtLZ?qg+eNiX(CNrNq zM8Oo*S?F2gT@(90=1Rmg4#Gr&&&bfr`TPaXbY=r=1(vLugVHI3tw0u1CU~u@(>1nz z#Oxi9!g?9iF{o(ZFb!r+X|ZYBAYTO+lRIT*l2$s+)MJu6*>T78EqlU0>gR%W4%uYYg%Da@ymTAy38 z2aP+fp0Bs!(8`xbvJ{Fr^&J=>%9U7_ST>z5zxB77AAW8lu#}a>qcYEm3JHX|+YgnO z!_F*U4F5edQ2zc^@1f@zDvTze15hyFnB@7tKV_XAjrJv*-`uXtJ3P3$G5qWDZEr3+ zN|ih?dWS7xEP2`44F5f}w^DC3X_e zZ|*WlPo61Xf_6~;Eo``uravfe#iqF(U6VXq@6+^}xts(4M)TQ?M6agl+G}5EjJ6pR z&i0>3+Dn|X{?Jn)1e3Ub>3(;%%*)MTq{Hc_uueR;KO3$C5jd3mFi(R1B*4$l#Dsy^ zyExr)9zJng|J}9JkIf%boAH~K)kcfs5V8GR;jQP~#RWsGZgZOv9YKey{k&ajtu|^} zFrJO|H|Igis>Y2v03ioSbDb>@ah;z9omA6q8MT{TQ$5~<>na1jB!x&1rgZKK;Z`Vl zj_-45^*jk5l#<4W9ktc6VUX0cW>8g;;YOw%g{+VPwx3(H4NT+#D0tC{@8}p&iNJFS zRBuB^CP;=+px^GT!;Rkz5{>(B@|?DCf_!kBft*D51H85ypE-HHOE(rf=&+DPGVY9z zdK_1#3=Iu^=F=$0MOA)kK3ndpw@>{dh^;|3wS29d?7H#o=!x)hDrKqd|I5W7z-}C zzRhvjCufQ4X}WI6Xs1Vsm+F35VQo~Bv1B?Cag(k0>0iCB?>4QR+}v<1nw8q5$Or;w zo6ZCTF(H_{3f`}}uquVEw|1!C;kfW4({y+k72l%_bHGsE5JHmBn6V1MC#=$Mge13* z#anzXZxiI5S!|FYCMN3SqhoR&8cTr*fm62ve9DA=D-J7z_6?Vsrxr<9R-d^yqa6Ba zXlwb8Cz;>dRSJz3pEr!df}7?I;$=fe76M?*P`$+%jvX;53@UHBznIxm zCKvoiN=D>x@Zl2;EZv`P@zK9&dafRe3%)0Q5%M@*ByGxHVau5MfR7o&Op<+Lzvh_R zWlyl@*zuMBo74{$qWr-{L_`cyv=B#UGpnJ=4L!|6pQ|2Ae1bv)Fd)_ivIb9w2{RHg z6TVGa*jm>A8yCe7b*WO;-Z(rcpluxW5Zw9Fwtfk+}EN1n(3c}9~7c|Itgcd8pM z5ulbI&H8-QFxO;RURiN@Rw}96Ra9Kc_Pg`KGe?}vK$!E#q4n)=!T{8b*6q3DmrDwRpn;}m zZvK^#AU>qk@7_*YK&AIw$b-;Ezb0XJ13?0szDDvi_2&T1UjziC3xum2((3CahKml}ZK`BG;Lpx>h9bS=THt4l@IhWDv zR_&%#D+<&moI$v`bVAfQnGopjcZgA<29m7P=>nI)zGwB34Sc&l<>Tsh!=KeD$BQ{Jgc_I3zO9kwW@*|Wab*%- z(kUrMaq;V|i&nkUwQ{z1NPfWaFRdp81Q-zjryev$bMbyNRP%eY3<-Sx@l-IkvC$Ew z6rk{1i0!cS3>YSBVE;YwQpH?yKQHs-ulfc6J@TKNkGh@G+ekvS3KWyBHFj?yr&z!s z%!OkRqZ?%YwFGor6#O!?{W^KJcG?laxRC8&r_T!U-5NiC+sjHX8})%sc6oVH5p9PC zh|>11g$+C?xI-v#KSDSXAws36I~&qaMAOmY?k8c){m=e9o00=66ex<-9-g^`ILmge zI!MZUn%^2T6G5O1+~xuUdqFcQ|EHhjaaMw~J}K4nXwh5ktMg%+zw`g>swibIfv+xy zO4{2!VKM>T4A_HSn%GGIH8BQm5OxkVRh$@t7$po=B2`6)eznXOjfEEd=)7=Qqk(a_ z^+nIIozZG0X1}X{J4W1GANhE|#3~FhiWA~y*qAw)7n!d&k<_8z&`B6(1pc&_U<8C! zKD8CEXTGEu`nqg-!hlrpjZEimTO<+3#-fEhta`!8{LFB%6m=@vCYw5WIg%43xS&AU ztuKWNltBMPHN{^`UuE9d<`)dnBpGNG@A=#9U2g=kCah4vhcg zU5Zr#6*_#-n?4d9DQb~TH4_IRC7x9k_yq6olVrw|TcbJv_f%)z=%U=sr@5#qqFl zE`vCX;Rhc(0Gl)=eIG2F{^UQ2%IIat=coJgMImd*ubW&kK1^@yQcnXx$>)${uX>|Z03A}%5 z4Xpo&)$9b06)jJ3t|K@d>%J?jY3M>0p{> zyj2S8*{lx^DeJ}I<*u7;SZb$;-3X+^JqLuZ{@t> z7ll!>Iwh)Q+RPw4!BmZlo*mBXViCF>{3v%pO`oZ8D8PVk%0hq;DLGes>89d4y(bp# zA>1Q%V7${zEk3_(lo4iw${B^vR`DS~u5u>xgNSZZd}85LINwncw(gBO{Swrxdpoms z8vVNT>4&VcjJlZboMl{-9{yD*Kz+d|VI44X`L~x=PP8PMowFiY05AeK+&ZT9@qgnSc6iP480l|v`E&gLD03)+&f(tXqR^8g`nq18a|U4-r-~EGEFx&E={jC9PQ7gG3val{aIWtD`-eb1 zQ}%RuKwYUoxI@&Fc%q|KtSGUm{DYIW-=r+O@62aI3P&V1)`Rihw?p^%r$aI8bZL*? zIv(lDzoDw-&R@wy4rqj~4?01muT4hyBLNL5hEMAWsxcy~Q(%K)d^L0m0vH+^zBHbT zKm}aS#nP_1-u77{Rz?WQB%%KD0VjD1!B%8y&U zEgJl2W`_ccDZBs6U2MC0543aVLNPa|^m$vc1aLHgir6#6;lwoXBFG9NyPtFCrep7p=vt)4A zwg)~*O0W}*v)4Vt{uKLAQpjk;-GLe3-sQu#2=vbcexGsb0H@Y(z{us7<>O9*YiqCJ zKV+@#ou;j*_?72cNm(A{F-~+*)kBDK#P9I8!Ta#x@ROqEOi$-#%%%V@ zaTM38ngtzTxJCZ|HWQ(!JGe6hkT%2-!XlCWwPeorH@n95f9f^#7#xa(tlJ!^$*ma1 zzm88HB;L%$nA^d9FM@8D)yuH-Y>%@3Tq8gV;98*GioF%`WAU6mFua@AUGVmn6JjYc z`e0vG+zaU%RQ;UORBej7wmy%MkqBvev(B%3} z|Ee&2!SkPd4!Tp{HZNJ-Lj+erb%j%Y={rGc?=M7jNl@N@b%1p6pkV$bPz%c&)f|#R zAbJ4b%3}B1+TGdSP*u^#`x|-jAT<>%EDps~$g41L4nB=KFHuN!&C3gpFnvDkt#U_pLu?bzWOtZXzbL0Fkw?ZKNNvqGc z7&{~44=;*;C_7oNM++ttV$A#X37qd}Wm3Cm{~4CON9DgQ#I7R-`y=Q8_`};|LLy{D z47hpw*8UfE&qIbn^Vi16oAGVr*Szi?lh9nmLbintltmfd3m$*51d^wJe6&ZhP(q0f zfI9a4<5$g$=zd2q5F+slMb!sLEMAH|`>?E2z4F zo(KcGW>vHQV~GCORe4dow9OEU|A?FX!nJW8Bnrd!J-me>)Bt+B1|;8QU$SA~SCDZb z@qu4;bKU1kusY$95W{-mREBTO-%8L&R*xltIS)}Az1>DMvswbaIUWyuSjW*|*6Y^n69-+&OO_gg{WRKH)$DWRB7-KH%EqiYj|9R4EGPC-pbWxh3 zO(QhzAZ&rQGJ^J$_5b8qn!$yiBd0kFiQt`+OPqnFw3kFq{2f?3@O>*(qY+wU^Sn=DRE?7!U>F z2ZP7>x-M}+{=ec*I{(K7z_%F=C&NwzU!;^sXhW?;ZK$vL#7mj4`REEgE#R_03^lvs z_E3DleFDr!vxbW`jJ$owv2Q#4yKW7Uzzh$rG(?kk2vTjg4lAmkEi8%8x^K4oL*tc@ zi3<>;h=8|^4e4j+TS#EE$NzmiQj)wa%OpS9*G7YjGPhz9IPaVi8zMBm)!~#Nqr_zE z^*gla?<0^cGN=zK<{9r0@aK^1;T9*0zCik|h zOKe!J-BJsTHh=k~Bf?P!$~13tsHFdpF)8hf2Uz#Cemuwbi8h6-%edLfT&ZvfN1V)S zp<@Im_Go5(vO?m92Nlnm5@!?Wk^{nJy_jFAj82EbG@Ezlq-%`(h8 z!**;d8SFi6UY6g@llom0$yvht)#R|Gav-o9*+OY60r}Gg1oneYHU1>?ggxyFB2uxl zACq7V{pU?F~<}AQ8z;cgTVx-qhz&}-4jSKl+wNJG;KW>y{mZm-orm`TnXs?1gyFu|% zhkq{rH*YB6{Z5gCVhN<#jCT(l6w(!O~vp{G*kb{Z;+!@tXg0T>>F zw78KIA)cYjMI88PIb3{(|1WpE$3LtV$5wlvUk=~%0du9-T`i#bF|x-WjtUYe36${< z5dqP-E0NOTqgsJtkz!({C52cP$!=|(Jk=%4fB=~9^=k$i`vYtxFctXsKPdo;8UG(s zUmX|Kx4nHBMmmO&6d7O$2`QyxfRQdiN;;&wkw!qeyBm>^PAO^Wk_HLskZySAd++_- z`<}ng=j?rUti9IS&w8FNk1K-@wo#~@Z%q?ZsaGxA#`0GOGbWOUDC4eHUg-n(11fFI zH&$pHCZ0y>K!;y&mzMXovSPhwRZ-?g=6h6eps9N<<$>>bQK1D6ZjB0!qYgB{grdO^ zfVg_e>nDJJgq&7p(%&GNiC(`SrlRf}(zdQiCUW@LKu14c18<7juGRsPyzINQA}Fr( z6LjL1k;B?mFJJq+Fjhd2wlEC|I(wV=g~b?U+f>x zWy)Bs=TGp*fHO2GKv)|m9VS^ZUA~iLYoT+Q?qRpQzaVNbI_Gqee2;{B&It=T_3nEL z3bWNETX=+KfiWq7Ky+Ml`0PnyDuJ7C|Ik;~cTbgbh-7-U&`VMnPE1V@wClaRhr+IB z(>Kjubme-YDg%%-)9eOS0;Zup$Mz5dC^5g|nWnccE**AZC3`TiTuRCm48(Hc1wxbY z_>2~I)Dp})JF@O?#eJxYCw^o*E-SSJJrlXJbb$>O+SmMTME@eKC-=M8B({E#NT1AwFo0OA7U1OCG7apdV}V|XkJ1RfU>rLr=xsb$YG zSi&3@U)=m0y}o~s=3u;{ue3|L8I`QznQ8P@N_h{YAH&~|2Bk)rEH30&2NRs4%F0@e zTUfc`LP&00*5Zjzpxfjei-Ac%&?4xz-R$!)@c~@lTw-`zUp{UQ4$MGSK)N2hKTWIC zuD8YndztbGiwGb|l9QZNS?|$?Q&-7niE?3aIIw25A?f)<60A3Pt>15lO5#i8N&M#a zykYbCD_%<^KQ&`AeB-t8P>nDq93mQz)t4;Ai{HlB?~wo9Xk;gmOFePXP{`^^1e_l6OCw^0+q}*t;_-+i?7td|iebcqIu@K8(zu z8i$_WK5J2WyIwp-yKfL5x=7>O6M-;|m+Rs0QuF>jC91>kIvQ?d>NLLgf2pRLZo59>)V8Y&)duF4NtWD7Yiz*UQj@jRT57Ay!dAY${ zr0X95CHDhtA6<{;FR)ModYpm;Dd{C7D~LK@5Ea^R3zWX+12|qlU~yoB2)K00cVFMW zivtwhK~n`3ABC-Rm+{g4R40D;{AsbC|-;yDk85hxWrKy{n|GXUgi zxj&1_arDwRtB%hp?5Z8kcUI0r?Bf zv_BJ#n4CYzVF!spF#Yqcy5Ju+*yQHU1b$`~C?icxPtC~s+aM%UM`R2qgh>H0kCgcl zv}-3YsBYN$fn*+`4+1+eqQ0e-L4*2Zj-iCc_rPTh!15fy;O1JO!=%D!@qX5R^D~u; zh_nIdHE81yo!XDJeJLw_WKWeiY>Oh)4-sOnl@vRy`30L8{Jqd+seYKvY$Ssw{^DSc zR6k^?$y;@6f0%a#9$@7?^SO_qG*2y)N)Rde9Tz(LXSuWnTSnpXVE544((*4S1sU11 z7MBLk(YD}ub9k4#^}}vrTWXU>)Xhl6NK#T=#>U3M!c0Y^ecbXi_k}bU1*VIu>#J!E zGw#i6Ck>T3Vu^#oWW5usP7@)AgWW%=b6UM;BO@cflIrBs;v4KP&h;5}R~pRpom>zz znnG>%=eyIXb6`S5sd#2)Ca%l5^sl@9)~qZ!3+%k{w2`R#o3!0`T+0g!f-Nx<&gI5Z zx6MgvOg+DU|DL^Kn}IRDFX*kKLY6t{v{1ls@$i?LoNmW*y%`m=qRcA4e%<}O9EB7? zr^m&3L%Wnoh3hgc$>312WyC5MQH6?*2X-NmV|c0G#3e{-wt<7kzePnQ$Xi?X#A3Ng zczk9>seeIGkf&VtlL;OkUR0FAtQEWI=6Z*RuEN>FJvyMQt6T1nk?!4b-jT$nZ5x3X z8I6@% zOO##PH5%cCF6A?dcD1MF^?N+2DeYd@CwJm;1k$_=73-?TWi_QvOYSKUtkP|El)cxK zQVo5tHaR{1>-XZ$^juIv#kH%e8xQaFH5pQRDk5qLEDkTS@VY!aylJKooiRFfx<}P8 zHQg%gy>vO!UC=D0!TIO4=r?p~RJxhxF4L*q;=cMTpLQ&=+1#yGb^`=Bpv^ z$#*TSRF7w;2Zc|u9f0Q%fyiN|+|82|a6v8!73m5q zMW2MRi>!uW$h(>n8P#vVN6f#=AQrlFXf4ZN_ zo%A6pGU^Op4__+J)9uLOJ8!0DifPk#8rLC=$b*B!kUrD)ruo4}(X86W-k9$5Hh)j< z)M3gQxB7-meoq18RAajQPpA^(()KG^BPHnEmqww|YF{o@lxW*3eI_>#s02qd4#(k# z2e#3D#`<0^Y|n7^_m(Rx`o$LNO06C`@2!$YY!<6swEIS!%-E466yLsvW?Y@9N&p-$ zF*|FjC1vLp7sr$}PR}*;F=${nWRo#|+)$Gaz8o;I{3Q|uo|=^|;&3fw{!u7XOjsI* zc2hAts@Pz=y!dO*<>8EcFfJkmL2DQljfanigS2|+u16_p6}%kE8XA|=$@;RfPd4I| zY2!O4w5&{d>MhkK?{D2g4JA}3?}o}!hb1j7L1G7oOXx6iC@L7nPaPghvbbDlWn#{e zDnDOEvCpVzzQxc$&&U%&H-)%MX3x}@PC+VSH@CQD2{^q^@tUd7twO(id6|Lz>(|l# zHaJ-2=W*<+|K6w0uW79|qDHc6whEjx2qq$* zi>pLLN8sS<;HyvXFVuKWPwoG>LtR)B;^u9zy(ckATNy1CZ<|l+kMumvm40WUsxBbH z$>nu_V^GQ5j5HHPCD(K4m-TiQe==w)chj2D@eg^Xrg>+Aokrg_zDJjYmJN|ID$cud z`Br`{`g*A{`eX4sy-O|8U5-KXuU1bpSHp|+K~k?+^UDfhozuOE#f2pTx$d?ru9Yyz zRmEY&?9{zT%hQAHZT^K)2kS3*NN3o`=Oqn{PlGX%Q&06zyVs5vt8IO4?f!D>XAx1+ebiw$-&QMZ$mq@K6+ zw#&0~%dVSuS_2+#Zf>SNn|sSOI;TC8XxQMO4_?RT4XN+Mb4(kkf#1#Qs+6scRc3~^ z%d{h}9uIX7R3|4}TQ!gEJw z9>vTZXy49#ZU*{-;Y4?!vsU)jyRL>6%P*x-Jw`}!sK2?%QPlkX^owMaq%e(giwdkK z`~&BE+T@=E8yF+4T_=lh>(0+J`d@LK<0?o+Jzr_I5XO=?-8m2xSP}AhCxHTdg#Ny~ z-CF0ItbRXJ@o*fyZ21Kj^~8mem*dUqgMS-!$<+dV#8%e zw9i?-+p>q*<3sB+W!X7jZ};K(UsN^=pV83eC&2~)SR4xkia?Y$9#T_ND-}+;}p7!c>VG7@I_MEGP&I);PK2Q?)~e*}7?0Auo3oBBb)h zbE2d1@L+_YO+hIy_LXC4pGOyS&80USUEej2kb_|HGF=>-TmL)~xqzQ{k32(avx8OK z?WeTO(uy!N@A>|}aOPAQGgeyKK>J$R(fDc-W#h-r|(pi1rA{PWC#FS+M%xixhT}I>JGsF*QvYNhE0ilEWkz($X&PTMcz%dtLVmvckmiU2AlFV7fnHkv7Q=zHF8b$daD@foxnQocNm71$Ai0+C@!m!XdzSRJo=$2>hd zOBSi_Fd`KktQxv8byiCg*ot6El&1*h$L$^O>NY(ysc>U|FZ#jYh1YAOb2Cg+XVzw@&8 z5h206AvG;}tq_jLr|7e1;^5|{16wq+Up~wHC<#iCj;V=}-Wm36f`$k1p9Led59OjG6Clqo z;fwAIUHz&umQpXb!MlZiNZV4(+~z<3z?Ad3OYP|X3VqKC)haFBPHBjnwtdL!YW(vS zIzxk-%fp6LA5U#v!`az#vj_TFPTjs)*2E{KEcB4>#8Qo|;$qef@p%GTnpj!J_>x^H z!x}cQn6)=tZu|F1K&SooIq!;iB}NK;h=c8bjx9mMpM0Xlabg%h>Qv+Ap83kJ@2(Yd&*>krBm(&+{|& z1;B>i;?ZTiMW{6i6oI(liuCLAf!ksYj&7U{y<3q+vO4?QNbc^|4c0h0qLQ zKkYV7!H0&2hZRQg@FQIhmhFn-?`K}J4vlDS8$(~eATceXuT!6yL4Pu%tT@`ocBTb_ zyq)pjG5+jkfB^!Guur$x&#t=04@WLqms&)JE)nk4ZfR++)S(R9*xYjcM+Izi zlwe1r&7uzs)I)5qzFFp$XmL#Ln9$N7R<_HdbkXZg<7aLvhDLhm9qgKybT(#mz|TUj z>*B@QUn)%2nkY)q#q9V{YiV&QfQI2iFF6*Ok?)GY9D>2P>F-)LxT`71_v14lA(}Y+@h}bQ`&uJpaB3_octuJ&(bS} z!8k(QReVmt49>=ZbSzu|Q=WD*pwj?)x4CC0{TBrYC0-Yhe(8^qGJ4by!K%rC%1|h8 zhU3c=(A@5Q8>QaRhn%J>7X}4YS(zM^k6B!PlpIn){2Kp0^?sc2>Czx|?^!3sksQnm zCnud6ZNeUC^fYhA;(RvLXjbl1yZa+sGBc?t>#ak3%+kN! zn|j%<^%gv#;6y-lWDT|44wb&?wu|KKMVOx~N4)SDJDMS*o$yD!jrv|INOp!z@_QoY zSA^0Q05I`mCKQ2vysvs$PD9mPkR(gN1KJH@vPa4Kwjp1Xh1{TyT^uSUwLxic(_(Gi zmalKuS+k^Vm#YtCS0Hh)-~BEHIK~F1NR3yQ)$U|d9!vl%FRKsTJ}oFazjN#pC)9L0 z{fex#m}G2MC@61jxtv9Z@Ue`y?Stn!Tye_@o5sjNDhl`%CCG!&bli zKa40Yy~dHFm1CHc{AFowD+@C^bXgqY|57+P7aLpEH?ImYLE_~rJ2pnw4Kp2Y6*v3;M zBaT@>aD9)`YO-d!GawX*;nVJGYRhwkK;4>1JCv&M2O@JN7T zoAKpYvHMj=t_G5xvp9F4Q<3>aAcrTr9-rq;_Z(5NV%B&2ld~Rofe$pglw{n6%gqj| z+6(nxf-h|&UcyuvW-_(7h=C*s2p}^Cpn-q^6FH5QR{L>(EFDVih}Y>H(j~n0t+p`lhAQ6i47S}6P{?i5s%>1+O_#TJz)HD;R&r-2n?k#L4`|L=jjK( z3~4%+HjeCidnau(7-0z(y0}ji6jC6_FjPvmANVZ82{VDxR{%W+3}uK!2$=%f?S)H6 zGkc|_&S02y_nJ!c1v;u)YzShQ6iY+n&8wJD1c?9iNvWRi{dw>_1yu+EGTR9Wk<1TC zk?!@Dm!1qM98NmdF8$G2sXqgdS6ypP)SB;I*sO+#5?*}1qwgWjx)C!oe%Tqu{`=e4 zuSPg3$7D{$mzbC`gyX|4+;x$qGp}ARw+oBGFqBh%U0q%E+8iKb=V&DG@7y z$`{6(E0_t(La%1N+3+Y#R?%G;9e6|HP&_`Gg&`Y?O|kL zf%$-lFZAwaPik+EuE|A!e=a~5PuegRx{)hQEHkp)($L%#YLz0N>P9_ey-+Jco;^qw zY?9$eL%Z(rIR7cAz;GN%<6LyKOz77J|H7~j+Nx@k-&7^Nb>+A?_$9qjfx4 zb~HU(XgZG_-i>$NYXYhuNmE)9-U7R&O)7tQPJK~Wz6o&bt|MEi{Z@@b^C@O-dPTp5 zzwNjpsX^OK@FtvZYQDMtpFd1B`i+Qr6VqI4bXsFU#8ZEO6DYAa(;(U9zjJ#D0B}db z0mKktt%bHH$3HPstTdRw7+YpC&t{?FGhL^7KlRyO(F?HH`qxcmCS@u-j>;6u)RYVp zcwBs9$Z%?$&Eh->&Ku;GfvE&_6ar|PM8AwrCtt+#SRY{p{@D9fhmhz*Hncw_w_M-M zku7ajZX-B^p!2W1PtaP=y2Dp-G8DS-!1F&dW?<}R^9%KlorluD{|^h0 z_zC;;7+d2r(P_<`Glh6ttwpAg$9!#pl9cX@>I>Dp$IbE&=ZhM$l_yiV=Ci zb)yC{=y3|m{F%U6?>YKB&{j~?{PDL7UV?PD=~ht* zp}&8&qknjil>K0T_GI4b8>V4YCf%FOtKzazgnk@ygmsY}UY%c+X#$4aQBm49iZ zbQxvyt@R=eV&rUXT=E3%t`tX5^Nr{#v9W_HWKv z$S1d{$qsi9eY}^@2z!5=>B_}fgf53CH@jI2e%ya;NG{x(Tx?7Z#K2*(1UWFwh{dr% z{KFM1TWdbAskZQqjaKyO>1n~1OR(2_9&&1mMx*y2IJ)yMmu{rixHv5xQ59lQ21{aX zb2$CSko8J^q0KEGP502y(8k8b&G}wSbF-fnb`5uHGCO4vtQP{;G_ud}T^XV3)Ay9X zaQbK}7PAq73b(SdTDYK+WNZ0&9?< z;cyYzM0)ChR!X`8Lp*YI6(rQuoE8x`94*Jf-Hul5EXy1gls;}BDtsENM$)F+25Ija zRgwAW>FTyNw6Od(PJLJ1-o8=|BLHE_jm^$>M99W^8?$JY71j4fpqa0IBoiC&u_X5<8$XXMQoDH!3l*u<^4NPLZn?c-y88;5KGlQ zB|;W!EzG-8>`YC!4Td5J@7^Ma5rF_x;0yM1y~F)|A)h;U0fKhtYe%lcRTNz(>;8)~ zz2)(7hr26>xL;SZ($?nPuZ9PcKeiwz)#t9!!qoHQH-;kP^?YMp8|{UKdie;PzTjY| zd#X%~7aed-$<2wv<)|p6Ebm1y@my6^Z09P$VD;f*mbI0YAh+O>)3vPBR>gW>ba8wY zQpu@MCAGk#y()KZQr(PI4*B1QyNf>ivz(YQAw670G%>MzsKZSn*=yPg|FHls{H63C zYoyqFjG6~ZA9l@-p3Oqqel47qDviAVxFIF_@K$4iA?y3&pzq^!|GSIC{BP|yjt#*{ ziNiUG6p^ zp)0;8a|5}lk9Ycy3&9k@>H##^W0&|@002bmu^&U zH_9n_*N!#6FrOe4D&uEpA#{vCXzA?pjs4(gei>k((UTs)sQ#p&Y#XN!8ogI-4u^&m zpxLsb0kB)*7;{0=4&mhroCFN>TlvK&JRSqW^Zsm^H3jRkaF~IsIa=TQvHoiXK*!N& z0~?)f&;(5RtHRQvLe%p?XWk^v;zC#Of{qT;FEA=dS$}2r*Du%AA&WsSj1*_I8gq8% zznWR4uf%XL-77?vD9wP57j`s^_O1jd0(&YXT1MY2{{AXncYK2uhb#@kG|>+FeI+B? z9%cQRbe4eCv2TkNtoR%I>RbcgO@E)CmpdD=aZ!%teAqK!gg3+^IP;D~D}^1>`nm&r zMY&h%W@9Dnb|EjBlUbbc1unN`g9%sECWK>9(9&v+sAYVIDc+)(q!sNn*PEHmET!io zxrG?EK}~(8fPrg^quz`(4l59o(==1}>ErB;Sj;wdkod%qef6)p3R_D(@59A65`LnD zTnh07W%@16bxjxh*G}Z&3p3RkZ)x(t3)e3pp&_4$#DY<-?|lxoS2Kl$NjJB+K`{~) zzWg>4v^CUK>tX=#JHyyX)my`JVJE&=C6yJWrj%*sz2j*A^3UTmWLStpI5#qgY zybFDAm!nPi)Xa=%3Je85jzCd~v zzMI!eTfG#*d(sTeE)4k`DwghxI{u=h_ag)SPPPu#1pW~rWDqk9AnWs?kQJ>PJ!y@^ zmutq2nnIjETCOveHS40Ca#!iT$nPOq6aegE>^pkIm_U41zZwINR zxzFt$Z;rT6$veGBpneEK3J8$0jS959`g9F+FyeX&=EnOI0x>YgYGq#3y2nZ1>z2sr zUw!SjE~Dp|C>e9vp?*{1lX`v zhqxoiW@ChHs@#2$wN2{iM|0wmXA}S5YPTx!#S@!ysb$=EnDo~6yUGdJRa(wVPsy<_ zT9&j=d+P4T=q3DZ3j^<2Dx7|@9WpabYAdd`HkyfP8Ai1x`*o!vjyz(Y35I6~ z_V6UMt18*B+b~K4f=!NxT4j|-?5b^B9o4^>|8ZV(m4&F010=0?h@D0xjn|*`Nc0;O z08$0fLI^mB{EaX&O*`+8V9Kv)}A$i#=hILZ~7R-HiFK>V?X$8RnaYT zj(CZ%eQ_F;&T8>Gp4!1Ce7*D2JM;15)Af|>^Y`!G+kPuwH<&Ooz?_0zWxPw<=657N z(My+&QfJC~@^f@jo$K^F*UzuOPn_wWdYryO+8-}G`3J*Wc7F(RRPLd$g{JGRLkoog6IFAQ&*)hx~ zxux%}n&R>*vG(9@8he!4hXjB9>;>OSzQIXS(HZn#!_AcC_IwnS`dNZDzF!OrmyLl9 z4~9uU8CJ8pdCVu6#GYE__X<)F)Xb=tc}b_2>kiJUdcu|1$e-27vJtD9%Cb=+MJ;bGWBp|>o2yj>CV^IM(D^Va(Wry%0zwNt_si@{#PLYef;e{^X6-74{@Abttf zfcK&Xw7SwvBohbYiQqmxH^v2s1lwAs$_%ZAFEZQg=^0^iy&OaJ`l*v+z{T9w&!0oc zK|>e%R4R-1Cfe$E0yY&E3wfMc!TuLbD!;6i=rw194Teswi?IN$`#*Ae{=!*t+5M0P z`SPT1yqG}a&{)IJvZGBL1Q@eIo%c7jna=7H^K<>pXm>3q@&}7nhd5iiBA6w_N;MB$ z?8W5h>b{q2U8i{YQu%0wJS8Y&g}l=e7w5vvKH z<;R!Cyp&3fOiYj;!EM$UphrW#GAg|9Fwo+-9CH+>LU+BTt8pR}u(EEi^0n~aSgosv zzcCe%nfq9M>wqkof*;5dv@@vk)j*X|L zj^(gGsz}kj%Npep(k~^C^hUiLL*cZ4B2EraPh{ zUBTkAN*FX(6+jIyHrbgrBA(sc;M6LUdEuIzV4ssv_5H3@Gyp4SsBZC@UJnnk1-<%B z0kcXA>xv-OxtN?#KE1Uk(~{Jaw9G$iGpzNl7Ih&FwT`LI2^xrhZH}JsVYhD|&zt$J z`=pwD)s@u`xwV5pZ<+MWnH4vVb#nD|RybEq>*531&HGgTZ4jA^$kqBC-}0YOA6^SV z4u@7Iwl*7xn&KP7CGuXjzU=i(IzyliBuAfU{XJ(QrZ-Jt39+p_ISGfun4irH{dhV! z>`8j~MDyC~Bn(|Y(frx>=EUL@EM`*467$iue$qnC2}Rc8ubMuCH64?+U*Bbxy}BJz zy<4a<8dl^CFZXWz*?V*}5B>L4t_WuJ&gfM&=8HG=$*zAQ&MMo~c)H@IURL^_&i*sq zM=12~X!Kt9b$t!8i44x{EexzS9w(6r41rZ`_ZJeb2ow=~ihG_}%!gm5%qsOZFeJ>_ z3zAbx^CH)gd4y$^&8Z0oD)+t&X#!jV_EUt8wG@ z%{30eHJ?xqf#hOMm zQ%A`!jKLYzHik@$sK2T6_xFGDi2z{yM1wVkxri;CP~9~^cM36MCRR$Q_Rvn6_>?gA z(S48x42TC{83jO~G;|3RGQfaze1FDCtApfoCq2sl{(7!ENy*UD>Su)HC?J4wbyx7& zV@Qj8@SH6ksLaOvX3o=>px)Fr3ZGu=@mT4>OI3AR%J=U6JoLrXSy7rYErHwz@Qdu= zwOjusx5Xe1t_Jb)IC|!52@Fhf0E!_walweS0p!HWbo6hDUY*FLWF7GPDgvq^694VM zacf~2OLr6#|9HhVT)dbfBq7nnK4=@J&XeHXPvtDc}(xab9J-E zy>VYTU%eE)+5Gmn%O0^q>3jE=j;Kx{y@BxvXWKyfw`)oGzE*c$m~{IyVJC9kE1tz3bX_QlRO#*X*40M!vGJiHFl9hR>EN5ee=btXZ#gn7 zl#(ee^IHYG{c@(=J=t7@ z#Q&_qmfz1O`*M9)h6t|ApZ_RwAXoWe%hCPWr%T^>#p2v|#HdmnYfa3kw4nIwP>a4O zDsPLUm)ZR$69c=umYV{Mi%gM(G6BNkblfD-}P){!dCm&>4AsS`nPaq`BFk?}NmxJn5Zn^S;$vZZCfM{@h@9gm=Y@ zTS%zPAA;kOd9=HH{?`Q-G-EqiOFJx1+;~ zC`q%!mFwNp=svr}TKCHQ{JpE(dxg4Kd1jxJ?vhE;m%a^k{{q1N+ekcBY4Y>N)Q}cg zlVpy6_`4qca6$UGb5%u0?Q@2?!Tp06@dpw@a_Y9f+~7Srh@n!vzumnddf!*P`gOkI z!bPXM{boaH>!n_~cEdTT=wm6fk%3a=V1_n=R!hD>;sGWsfXnj-!_NSB3!o8iY4yXK(@5^UEhk{Gct+i#Nr@FP;cjeTi0`DqDb*$ z_ZK@!gKrv+dw&BUz%(rYwm#GcG+0~8&~HB6$44uxSc@ob`(FP^#-sB2v){0?+~nyy z7?p`6dery1EA^$K$mC!O3%U2!81cSs@_(4aN!uU_$za7Zt!N<3DS3kbT*%9OrpT`C zW`88&dSdX__hw7Sj z>~0m`mrqdec3MN9%>_yenBdZVD6r9x7RY26%bT9=^t1i2w%+xLf=ksgJmuh9e`aC} z33_KJ;DX|VeZG5hV-GVhB?-Q*$SpV?K)*CyJ6HJbp=P{tFGHIv6CMvt;B*3-K4T>r z$U7Ny1h~kl;j|UJV!F_j931I%Y2FS ztslF%6xE$+zEH;@H%8_}kWZY0IBo{;KSL~jyjCe+1_5KB1spx>7sF5P^&c5BN18ng zJ#jWpH)TuW$7kIB%=q3&0%mYflOGc;ok0d?FIh|_9V0QeMht*S2A#Jg`1Tn*ev!g) z@=*`~Fj$63^Vg6Yh>E1#FuvnGuQl?U4UbFWy3q90teb5&>czH4KTc}@nyiQztsD)t znKvN!qfHC~%#Gk~%7(*d7FJj{-lr-73(+vtAJyberTOE^OlLRgNQJcCbdnnsD8`O- z<7VgY{_U~;?EC$1k_!#`&GsKpE_5-s6V++*5~`8eMQNe&o45b7DT%bg{&;4lrb&E` zzTkhr$#^k zr$wQ~63g~E-Z`kEA-V%{BzTgBFyikI8`^hlw7mZo9=6|(`uZZvf!NywEHGtkEnK71 zv$E7*ls^5x!?mX-nDyti0lr~av<`zh#!RiI(Dja4b^*5*RQ;W-4<7;n8t?A7$ZUB` zj#|2y#T3KSYrPrR(${U9cs$9K@Wi7!nm>(u!}2Zso zq<0@T#q5Y3lD~V{C?^K9o}pL*6ns>$nH?A%OFc*Skhs&{=3ZBWH_U&$IceI-iF>Ea zU*I#LqRi-dsP$VlYns?$Q%cJ`jiMtn8dxkR2L@SAcA(2%?-jdU6t&NJ$HUUn)`(tM zUpz<*O5Ch51$y;A_o+xi}-b@S|=n#kcXIP14YctEP&X7y5 zPlc*hAuc)&FM69T6~eB>tuGGC6AU@yn1bNCdBL#uhg~^Q(Z%|v*JOC?Lqk{I5wSu} zH#*G5-@iAMRtaP%_Mlbh3VWYKydb9&j{|vw?%oM;)D+S(8-=W;evSvfK#Dy`8%$`; zt~JVA>*s>uP&C?f*{l}#KmDtp=4xHGMp}mjT`TmL&IWFMx#-aWa10#ux>Dn(ACTbc z&r}$z*ic(xWv%yGfM^dH=hr1)yiekETz|R#k#C~~S$5R;^8L`=iC*~!J+zi=6jzl z@F5m%AkvucWbp_5R>&&B{sj=XVZa3;e1+1yK1BxxKqws{kPuW31h^Vwc88mkRSf;I z-MNhUGxOJzZJ?X!h2J<2Z6nT63$n&sP?kEP-Nop5tZ^KNQJb3}0AD zKK-zvrtu_(Zr4xc<(n3COsJ>EA+WPUFLyTdrgea2khw9*G4{V*58FzTm7tlI$ibQ^ zp3N`%Txq&kQDQo^+JJojm^yBiMr3Q9QnX>b?<+_fSiao zg3-e`L8O0Yd{DYv)Y~>IobmfORUEIO#oC%vY5xoCd#efq?1%!7IP4=5) zeoSEo^a3b_Q>qkPF}cjIwbnl`C?G7D;CdRyI2>=!l<=aa(X44)SSZz8Hba|1De`}@ z&G#H38nu3IKgR&3HZy~37c=~z|9>_}<3HO6HHJk?oxUcHn0uc4#nHIEkS>mic;#ss z--OhYV?i?h62mX=7?Ky47b|`kC++I%xlCiZ6LKp5ccg;WO8zFiqaoyjI&hKsfhF;| zr!z)BX+Ptr1J`>#QTrZuL`8U7;r}K7xqR7S0e<$kf-B! z`+rz~6yMuTmp#fZqWn+1N5kP6r1@S5d|L6L$b%Fx*aQ(k#~BynR?@#1yQ~Vs7~Kgw z!X^~EyydZ5oS0oC4n^>7AptnjFhW&TRpe!7u;J{%R!dD~|6c40Z7UYuf`hRjpQCM5 zoXz{q_rd_d%ej-*%frxm)v=`$&%q?W9?w>6%ae8|uB4{|R*PSutVfI|I#u6gdz`FR zcH*+bTwVn^=m7+wVI$S2w;W8%YP)Yme8MWK8*NFKdQP4Q^ob5}pIjuUd=QBcjvkrN zV7=E&WJJ1Bg77bw-l^0M0FSd{wbe4_Het?P?t z*vVc+h>PT*6X74X1>Z@M1RJdH6RNg2XSQTrf&%>zh>)|{l~w#{NA-&$=Hn=4Rn$Mr zrg91U=HdjQ+o6hK#OeuPVDtLpx9#RPZVEsM1_J4=hGz}RS~0klzhPep;8jVTVBU38 z(xlXOZ+}|b0P-gSo{X%jF8 zkP&#x#^N*_Ha6Pmx-olv{yHY3)oJf=F;A2$ygBK9bj?4t=rsH8T9B=jLiGBcnxVIk zK42-pNorp7XGVp4Ck?Gpqvh}}(&t)u4gr7|aD)S!iYifg#d58M>QRrJ+mYhDU6)x| zCK5+G9CpYwbN<{-SV)3E^83-}&*WYrcp(^uRW0==PAqe3cTCyc_nSXWBD^&cuZ2|Z z7PIT-OAE9_1dn8y)CyMy29a!~4N70`NcG4CO1!>3{zNzXv9~Vc^ zQ+MQTsmPqZye7S%u`HTkIM$7h&wh~FuenIx!}JfiNZ%B^vwbJ@`gt^xkX7NiJ5sp# za(@;w1_0HmQ;VNU+S=MmHNFKHkW*psqsGSrLwkJI!WbS8^>bZ8Z~n-EX-UG~m0M_0 zVs2Bu*I0Q*ZQA{$xzr%Uf5C*bZr;qgvD7`oo%$tb+sE_W*4qmLSr9xf0|pRvJmCUC znE1DD@%G#8@eY?;2c<<&al7^+_4Y-++_jL<)8w({PW7+7BTNO(ja!hEE-8r~^ z0n-_0l~v@E{Ij)9w&8tOICwS!Kn&Ybs*_kn+20PZ6wbC<`F0|=Q0>O2_g=+?>}J}B zP2q7TF%uWbsd6PIFQ;DCyc-C&vnbRZ6mH{FC4;XH@juO~?v8*YUAM{$mNh1-i{PtY zmlj>UAnwNe_=Ww*s1^Pv+fA-rP4;(rYXC~ry{^og@hB3O&}Z1;T-|J6JMYrlrNC$LV=RSgp(;g$u8 z^knqc+>7M_ck?Y%z1%MKeQ%a#D?5}!earXzmim(XeSjXNqvskj_3_t%ar+i`q#a^xspYHoTr*YWY zF!7=`z;2NKc!cmNwayyb+rJ;`X!D6j>7xgp21F?1OXZ7Y<|htmZk(delxZV*SSPB# z?p$Vj4P^N2vVNxN#e|`T6a7=zikLDX0l?;jz}w5nz`zh^ z!Tl`K*5;_p&Kf}sueEZ!92$y99QI=55ycE+%m{x}7FKH?b$a%$?o3`|Wca<|>!dsP zx;h$`%vsi~8y|5e;=~4gdQ6XHkH)?RBTQ!+Lj4%{rqYeKw#>D%>C1b}Y5V2l+WwTI zHTw4MuT1V=F0QARO{2NEHEj7)F4=!f-m*NJb4SQcN!Zr4xP{+;t5#I<4AE{I%3yZF z^knA#4pZ{%JnzE7p-xx7Bu_!;kdJ0|h7}Km>C!~4e0vvxFIk|vIX^Aee!W5YSA$CA z=0>i?t=?wk_4Ldl(c7SPVFe17?OI$A0*FV%iG{dc?kVCi<{dKI70Eg0z;-+sgX z!I)xqiNvb6xbe))O|nop#6bndJQ`mT?RUcJ(H=*SUGX8=Oev`w2f4Gr`^HQS2+gu7cFra>03 zAeG1#Ex~Yx?nT#K^x3pnLIsjBw$iVHf_Ybzu*7{8e>Yd)G%23XXF%Z2LX^{k*c7N9 z3W5q19hwhHQaW@IxPMU5zJA4HfZVx=XjiWN>9m|vBR@am&O%E>PDGwbxq|yDzOFrY zWUn2|ZLfSTW$8Pr?9fMbiWScwEwXyayC^fqRivpY-dmu>D0Ae}6!FBQdR+5>`TYE@UdY~W~mC&1%yFqXL7u-c9Z;yPx zmRMpYf3Xea+&%7wmA!9SnfL9obMl2REA+-?u%AMqKFIT~@cj1bBUUu)yM%3yRnBkk zthh&Wv~pxWhl6#O7h<=8MmUaZ7%~6IU$CMcR_yp5@p)c4>^}g zc{^c|olE1^N$KOgdexP*s&XnPSX9pASPfU)-j-3iEt&->ZWT*^*;Zn5aU_?rF8Gyy zT@aBS=g#YvFY&hV#nNT z2_S*=?%jL#$9kB1r-c$T%6C3=V75HUK2nYX$?S`axdM7`>pxl1yC@BZ*SHOrPJk^^QnD z0h|CiiXivmnbyAzwCP01!#83yQ!$wYKtu?VZIYK{$`%HO2I8OeTvEmo zjJ^#N{rVu6wVH*97{QCWRUSx?37-z&#>;AQx4-39{7wjaQ!%A5q!W8i^4sVRMeV;v zGo86>sqy!}t>cTAiB0@wcH{T$55IlmsE~v*--KNyfIB(JQxfD zqyfO!XP^Odrf1X~sItoNjT<*wx`%e|+4*Phyb1t-5&;T=`YslYyo; z-fTM5P`Kd9z(Iyd{Z1Yn-UC3~9P zX0s4UKmbIC!|~gm-`e&v?`F??lUbKRnv4#R1c*!m12i-;IDvsW5ipX3Ai1NtsnID` z+FS2N!2+N?j~^wer&Pg_5P(P~UX>A&+Hb=ved6`|8*FwdC;5;8=n+FxO_Sh{*1uAB1O`M#;#o9l=~dt0{XP&G}6bH)!h*8#}Z2c5|rDT$zD%ig^^^Q}g2XIm6( zFN}j;jS?bU#pN;EhFFGmSSJ#g}X-49I@Wvw4Xo$tP?3aie0dmfS8%oVViM=4e7 zt)KjJWYI<%`x;5INhKXj{Fa$fbtYj*Rccd$F1fvG|K1D?KJM*`f@O2odttQW+O`@y z_WU`orSQbA?yss;JHO_yFR7_t$X^>{G&YF=+^i1K^G07!t7KCs3Mp-4+~_*Hah1!y zZ{NxG8!o0RfmPQWs5sbUAO`@HMqmH4Wq+6~>8T%E|MrNh&c%$Z9EZbUL_8x;2L#v* z(&dYMc#=y3fT`J~Io1Q5>O)Ib6~2!Z-030 z#j$D1HyqYAL?DuzM85qF8-eNC;~*GG045O_2=zCK@HNnPiELp> zAeqHt(M}&fQdhIjZqp_wrhoqN7w^9FjtqJ}`s`Pd>E%0j!SaYk@z`2Tkh%$oNW?S? z|KPp%!r|~|pMEZc_dQ znO44Y4=j)9B;!f>xn{l8)zzacDtdu|^86+d0reeQ3Ums)=Vsz$5eT})_WpZ6lp*)$ zpL`OP5`g^^VV4k~1PwqczK{rv%%E-p_J`=#R$<oK*;|qzk#VGLTk`35&#$i zR#)!_fUeFSmHx0!lcqw>0kL(OloWa|F=V)NXPMu(k_vPCPd0S~1VpT^u8u{*y}iBo z-0Yr|FA)KPuIul=_g*+0{`8Y80CHD(na{tnp?fBozOSk#TP!~6xcict<(swy^)kuy zs_KK8(j4oXZk@v3N;*U|-TLDn{gA>7pM3I(LE4rrTPzS!dS^Y@a5B$@;eo1s(P*@{ z=h5ci*~E}xP4)h0G~CuE*Qe#?d#B1%aBYFDcxkw8yR&!+FZ+n$6@?|Gx2}Bt=+UDnv;W3j zKi{|CRokclg27-$R;Kw^iGak(n3-`s92`Ax@K9D(21kLCkTD_(27^Szep^~3_5D^+ ztcsps1aV{rcf9`Et0(F!Eoj6^3j~8x6GNTtZF8;#LxMz*YjN7wXxv0a4+*+J+rip{ zhpKkp`s(x9#bv>4(P(^Wd6~j1l28JZz&Obi=H28)OG*jIq?F6c%aKS#%2hBjiG*uw z7HnK&>I`3Mb5pJ;FV3ZZF=XSK(Ip@vVIslU1o$uuZ*PW>qNcDu5{Cc)8IW$#*~JTI zrluz6=H}F=DG`qQyk&7hlDkpd?J_K0+1#z3%K%M||;x)1*%cj4=TD z0*r%A-YwTU1te&oInP{t$2;D8eRfPBN=R6WDguklOo??nkX-E7w$xX?#5Ys@i7$17 z*l6@4B7)8|IDg@dsi{fV{2Y^fQ~99calYLs8$STwM&7=x!;6%10JHy^gEgxEIgeKo ziy6=JgGd#-wy{GAEY2KnoPPCG{XQMyF^T>_XnJa*x2I!zazX$Atp>~{bC6ZNY;|2V zU*?n0IWO%8Nv*S!Lfg-c|NllYT2cBOE{8U>9i?|?`cN{gjBn-EdT zmrDFx?NDi!uvZe7y7SD%x4q*%*JnosGC?|~NN=es!4XT4l~nHj&5&_alN+_<#+8~M z1txXpxr=Wvk3YP*P9Ca{uc_Vw0D#gACc>|?Qr75{7i(kyfebCqH!i&69q+k5J1&3| zGxbiwa@b%7e=CRY#@TW}Db4;5v6$n7`m8ekwFKkWVVN{d9s2mqJ_E|^XTQc454 zBO{Z+j4-01P$VlYd&~BsfoEf!3=Tr2KKi}BNF#+V5=VQC*)M%@#UcD;@}oI4 zp+oWfO57_Gb1=_{5Ku$Tii(QH$HyK&>Sawq_`dU)dA`l`mB4k7oB`^YYYQo$r1gTI;)}@Xk6UYg)r3KfGE1yNAMgSIy`vM_S z`$lW_8~`NKrD>L_O-aPn5)+63T$(4VeknAdBo%2C2LJ>0)So3n;`35@E(n-FT>@}3 z6`z?Tk$61hWUTH$;<%M+;{Zq`)@GQgvqMO@0;P%kLc}(fTwcdbD#%TuPCy8ffq|4{ zD}YrD8p%mYAVQ0BpC#}|RZP7WLnPz?z)>>Ar@#pzIV()?tUv~e*pjEDmQMnZtDRgo zE+Gie1(0S!rJCdikT~~Eq{kZiS(SV0+P3;hroz{0Rr^a$Ny)%45eKn)fk2krL!E?5 zd0WGMi86C*#!bkXWODvDlD8mkalB&^q4y?cN1!Jl=t-)U)SQL$Z;)U}5c z&_D(ofYG!@raQqMg# zrUQ{1$=@RZ0Y^ZE%*)6;WMl$#o5Q){nY;eQucFeLnUh;rfAGhD@+ZIiZ~xupb{l9@ zr;s>DCP6Zwq$H8%l?en$4wMqS%mga?QBg~b2qn`~G8$I3vjnaf*!(IIlW{~e|3qp@ znpnH5YMF7=*X}Aq8AC(~AOu{#d>H_M?3}O4NrfS$JR?YwgJ2*)$#%s@M2sXuB2vM1 zW**r(XKo6$>k@#YV5Gorh*Q!wU`aiq#^OZG^tq?<8L4|IO2Lo_L4XDTN$m_EHHsr* zT>T#65||;S;vI>MkV)W5P%vpQqR(6y`$%qjNWM6F~x&Smy!Vzk-4iV8Au>tz==@cW^}cFy=&}10f>Ntq+~o9 zza{{a7DZAB!&*74)b1+PCA>NclRLR;B*{=RGL1O`4KV3POOc|WQ$oThJ4%@lNxKog zZ{OYaEnFdT-Os6!X(se3fCTkEqwX?N8u2C1@{YW%O(#zY!S8kUjtx(2-Lj+h@X5;R zgUn*LQ7A$Z00X|2{=UJ?`pTwLZyY+sIS7rs01}lT0svTr_uJ^(BV?Hc+ur}lpJT)u zk9e~)os44M6+aR$5-hb(K{sg%pE>!$pNKgJg zof;Xv?f1XBam(QLnwr`Z&2>2$_T}Z})(7|cA3Y{waR$tySL~^-uKafZ&^<8v^UrU; z|DzvRL(?CA{Ao0S|Afc|=)`S@>Z`j=JSY+S#e8cW##Fk4Fx&QXgJjK@X@uK_XQwu zOMQKPQ|rR|>fx1AAF{nMTzxnW? z?93dWXQAtU%frWgQNblJ&7$wzwWnt9|EW=Iu?G#u%$TP z;KXf-K9g&I93@R~?db6nRaN`4(;PadWsm#bojU_h`r~Acam&7a`%WJ{v?a$G2?x4+ zA6~n4Pf!9uU=k{Momlr$lU4QtvFDe;uYeRGM!Gj0# zGn^V3-R-T{uU|J%2hauRjA26x&9~mVHZVA}W9R-K{@@3njo;h=-Vq3=F z{oUU!MOj5<_1V`>WTiPG(a_M-!CTj^1R_S9ryV(dEJ4ru{QSa`-kzO1cW*B$)fi0- zKfH3~%1Tg3Zas4BXmxdUPDYxBVrjws;Qrlb{R2eIfMl2q6Q~ilt)=0s0|Nt_Vu+}m z=L0oWCl1vVWIJM^z|A`k9y}UpY;0;euqQ9e&dF%)=>4LljhNoJfiHrvx}s<-JoI$1 z<*P6Kkr)C=z=oz{ha2k&*v-4GSMPKJw=~r5U)`(QR|f|NC0QkM2^u1=tlWL#gzJIvedYJgF|sGqxs0;!_|8-)2$LU&*DPY{oDQh{Snd) zyD*KV}p}D!X=0IL%x<;|?_SWk+Ztki)@Y?HVlXB+j zwHy9OoWLGOOGDGKrp9B8p2WxUO(vssE+03qbhKT&aoqroh>(Djgh+6Ha?C6BIJ&x? zj2t`M@W!#mx!K8wgX089nys#(w&`GXUY2v&vv}vuy}^M|04`7~*jDn-{->W6FK<8zbi%zxV5^v>UdLaes)Hj z*ow#9eeXtJU!THEQ5Ub|xr>Nz{OQB1*KYVCQgCak+*ny&xO@Bdz>|K1fS7}6{7B=$ zLk)Fl_B7AZyu%_E7UoS1=~WM-l##&v^mxFqj!e%#omu>=KmU_`mAe<*vk}f~>klj0 zzU*{xzw zRxBKP+}m;EW=jOD#I*!R=XgEyt+%cXJspUU6@Vjf0&bvQu&v~0fA!aSx!Hk$ucNcA zyS+Uo04(;O{@I_cOgy^&<&_B9xW)P}{`xO^db;o3>jlPs{QkSqX!OI6uA~Nr5+LaI zrl!Utwfme-=d8!?u(0TyTLO!7%ietlPakb4$WDt!f<215%#=wq&{A1hY09@qWawG{ zt!q~Ukywg+bAo1F@whu~f7RF57bBZMzILpjQ^%-#J~<6y#RbAAIm0txR0{)vu!HRP(=b`HSwsY2ucK+WignwOQG@zU76smM{AU zhYhe3(;FMA>uL`u-hXiG%fY83ah`tU`0<1uv95R)J8pmVf_{sg%RoPCbMdNdGvkxEj?B2O&N6~hU+VIf8wJVo{!Jq*kQ8$?V^r>Tw6voGTW*;}^##ZP~F^RrKf1_uOaMWuWG*Z=KbrQ0nl%S&qK z4bUank+a77>blyR?Cf0clDqZRm;KMyt=F#T>dBKQaj*kw5{ZCIZa;SHNW;GJ>~sfl zn|H~iy$ISQ=)f&?b#={kRSEvR^K9T*ob;HcH`F((RWEtw@7`9wXTsuWYHCW-s>6ce zxj9&?bF;Ivy-V)4+t>OBpBZRH-S#{?>ennJ|9$&CZ@-zS5G2V_a4;mIKwV*dC9ix) zz{t4Oa^1?0lof9Ehy4IXd+UAV+QG{5EJuc6#5_w26O%JBi2!u-=1s51wP$CkrfCol zRF)Nxkbn|d0ydR1fgA~tm_ROxGm@Mj2HYS`NAxT&?#OXwr{{)4fv!gn@7-@Tn4WN> ze&4=>jS)nn(ca$PhK4!-0EAd9<`0CHJVS#+&lD)v{_3ioC54Fau%|Z`i~T+%I1&*i z02WiC|5#iYLI_RM5OHUDX*3r8v@bM0bqFe*`2R06Ec5{3A7S&B+1Dz1L*+-q2zjF(~%R+HTUk^oSj>&*}wPA^JfFz zx57M4@FMe_5N-XHxQIK`^)S*z&clB1g zdITY45-nSIlthERkFR_dXO4r_RTs~n^R6t7%`OAL-2D8ND_0^xUtxac*;gAQe*c%Z zyBK4IJGO;_{;w{5u5o_!*s;@Zob7yYx1+5!J+I)klP8WJUc7qaR+Q-t&CMs8YSk(Z z?5lY5{28Cm_jEpT?8woR&DHH~_r}LvTMF`yA3h-0GJj@H!RfPSCmy$7{j6QnG(ntz z5$a7%P0fcJ+iu@na7|ZL?YnT{LeTpU^X^3?`RLK3*82}Q$w<(rLOdjZM6IE|_Kj1` zJsquMBU$kE~9+h2Xfz>wjP+Sp|DIp@5)qu1qWEh;HI@yaRW z`t7@q5deu03`7KC0yr_o$ zrY=?jlc+V+*PT7t)YaKKHZfUVT5{(6Ii^|Kx&}4C;=&!_P~ewee!+n%_ElcE_>NQO zojp(D1OV{x;lu9xw+NXhECt#K07(K#$@RMW2D8uc>9X>Y*Uz0*Ye&Gs?c2hk!1XJa zIZ#zi)w%O;29}n`XWdyjd8f~wn|R!M^|MZ0=Z4%oo>$7`l@J(Xj2J_P00dm3USC&x z?v=xx9S;(Fet~J0_U@+$SYBEZi<KjZWHhR4TydLN#8 z<<;rgnH^h7_Eqk_`iuV>4u^rIx!mq6SFVIZ{(^#n*Izji_bptzeV>qT-%%0?cz=26 zL!oCLJ9g~s=~vp@+uJ+3va>T^KXvHHkt1JTZ$-w6ig!f9fe)`<;Xr%$@4NWsAGKJt z_TDD|QW9|hL?+RqX!!o!mKoQA;za+7XKZc*5CCvQeyIN7Ypxi^wB0fhiy#4LwW_)(3! z;tM=mh!+)?hlAc5S8fVstE{X%fAONlVrlPvf&}yKg)gpr84G&1=VzWf)f5T^uD0}W z;N_(y(OCGSUwlC#Y6JrSFy~%Sa%Ove=DAlIgTdg{mTt8>Adwh>dj6LZ7$UE&uX+8oS08t@ULP4jtMl9&uWilGdn!eYC4zZ@&CR=0 z&pi2^Za)YQ~?_(0UM6D@t)-$e! zgH@GpU3lZ?-lb93vMJeA2mt3@y}cffr?8~-#L3s}DDT|wh@mbKnT#`ODC|x&LuNJ+ zk%T~~BWXvDG#xp7sQbbFiHV7gx;V&aQZtMqNGxeeGB!$D8USt62k#fM@08tlJYp9GRM~ZD`ESN)H8s zHb+{1egU)ibsb6Y{DLh^3b$)Yf=v)A-cgM6=+UFD`!^9ZPbe->tFLc3`|8o&&IdO~ z$E{ZD%HonbVxP~~diS0Yja1cCojL!eZ)tgEW+sL@z$%WirfD4=?JG;3qS7+8+jsA` zM;TwWA6!c{Rh4gDIO_}i^4XGPAl2vud^sG;-m$B;zA+~wEffgao#~1Ba}qqiU>lQS z-Ze$2H8dSLsm^_Fv8HP8o9EB^eE;}tF4ow5_;?EMzkT6c!2jXXh1m8DcG1f6qOyw; zY+?$x?guR-nI|j}odDqT1s>eJ88Ea12M%03e{RLIG(77i#v2ebrPtgH-oIO=d_6eYmNy`N+}MTQ?NXoO=okO+D~d4{+Lw~H%0Oro z^?TeO{^BF1f^|_B(dgZqUp=^<%;J+8fC0K-%xSj{JnDJe-6om!^;eGZvTebD&k$iD zw7>oL{{a!1Bt-m=|KY!=-hf#~hWh{S|Epi87yygU&Ha!6?>})VRn#S#c-I;-{ydn2 z@7s5y&CR;HIy!4=4{Dlbv!|avcLo5GSquOmh0)dB<8saX_CrEoGxPy7!wXe_yGC5s zxSpMp-PqKSo2IpnjvBx?8Nubnv5_H?T!7VX%Q)Il_4MKWmYdh(p!H9BbVpWWL+z6% zk3t|hNh1>SEX+mF7R5BFTTRTJ)Oj$p*qt@?4V|qm58GSgWcAF=RTTed*Ph*Da~?lI)VA8C&LWB$;SRBp+O~)QS84W8)TweNRneN7t}4Vo4b3=OyQ-M#B++ z)!{sF;K1Xq2M=yuji6&AuwPMBm^nyKlwLw+u{#odUWuS>WMWctI;)$HJ$y1E z~l1N6w|IdH?FCN3%(AeDns3SS9#o;{Abi!N@FK~J-mPKZdvhrubqDG3fi%NA%YdFPd({@$@U?E_KqT!0-AMLK#Gd|yz2qXpoNx%i@4x8<0eZ|v< zcduW&uEH%1)Pc zDk|EseH({3fP9|0RTd3 z2HLniqpto?XX~vLetd8D?w!vT29adgyEHL87(x5c*d*!aYnqQfd@@8axxBczyyRgN znV%jmE6Urty%@03ceG1nvkS{5rMoJ2mCnsfh{Wv(k;2}kv5|oYIEKc?Nk4y}>BytL zQNxI>(oc%Keb)Oerl;9aYpmVh)q1yE4g zRr|ZUyThBAQUpwZ)oQgKtKT>Dxb?>MuVSQwHS<_R5JMn2aY*vzTEo zlGJ1@==CaDG`{HHR=lgMqUz-jRuBinI-p7ZB18eOHXHqK`^!8DJt8ucURfc+&Ci%#06MwR%cUPb?1Yd z-5u>jIJ2};Qo5^RSD9;O+PmaVrM)dXiUA+ntu~#*%*5#Y%%qf(7>}bxvu7NvYv_1z zQ=PTPJ-^i6crT}?%1xI zKxR_u>kwtl&w0xJ^4;w_cMZ+;TOH1Z{pG1+m6ViZ0ZTbqX0s{DFgiawK@tSA7_-gJ z`9cw+q-fjx%w$19jz!mOnOTmEZ0W_qlHx`8LeS@D%)Yullt|JD`8;k{6dV!eV9u=i z#zTWWt#__oilRk=1pqnMfZ|@y+_Qmx1GI^y!1j`gveLr2*)dd{u6CQ^uIbUP_6`HI zzVSuPp00M=;Pw=GWm|H)DvEzpzH8@@XBa_^KCk3qaNZY*K}qrUIoDKPURGlMnbOM( zi;EZC3jv>Ba}?IrH*~hP^t3%N(B_$++gtRbox3Z>y~79V8oJuy`8o0**pPc3dO+F4S#-Kt?! z07tS&V_|OAA2kX}cMiCRa|^cVn%$B?*-odiys~@87SH^QHxMM^91P#`{IkJ+gZRW^ z=&y?_%1R4oW~Z!nTTOjKSKF=L_Ev+e3+@-KSJ{C;kW3$DAVc1zsgb8i+AA#IH8eM9 zwK*GWs}yev!082VQEA2Q-4z~>r@3~2S8GdWXFCzP7JWsf6}xv=`c^zm)unx1w{PFM z9hG{LRwa4p{S5VYPtZ;*bI<{C9>ck zk@3q&CX?{hM`A?8I3m9C%E_9&<)MHdERHlAHw@q7C;f&H1jqq_Kr*g+K;#k_pqT+E zKtsfoo;66Oa()xIukYI*ZgaU@OG``T<-78?6r`o4X_`hrB&FZ)ce~wVBcric?04b^ z*|a4U008XRvEy(4=5M0RZUFkdOJ7~R-2Ze8Q3I4JiBtj?pr>W!I33oR@n=LrqAe=f z=;UA_?<6{I;5&~0^6u7f&$MGXaw-n@SAUG5A^YaWf6UPKi6i-eD zCjw-VXxtkNS)FMTEWr5g(j7;Sn%`w;xZQ5XxXJ`(AQ*uGED($c$s9HtBQSt_LlJAP zlQB@McNu7){GsbYysJHrw3v1d~Q0k5~gxa>3m0g?ZQT)2A2SdV6$l zMQ3mC==dB+mYJTB<+QN zS`d*WiUcqL0iINTEl$Qb8F3DVDg`Bh#Yi6-ovM4a zCe2}82$_a1qT(L?%!Jqxsm;ZTa~Ol>}H&Sg=gSmE0rH1ad+wFD*HK^zfE~d`^bO<@~&x zF%8Tla04Vs;09^V%yv4XJlk%I9KJKV45andS)gf z0m~mJ zG6p0Wii9lEf>H2=Vos+W$w^H#7Y7>BYC+9obva)!j2TX-Zm!tj8ED zDJeO5q#?gxi(objis0;%BnXMp*N8kCGkl@2-H|4cXJllg+1cFmxReHQn*c5lQ|g9G zKruf*KlJq3#W&yCTd}{pyL)0{f)Ry6k=dE~!s61-_O_ixTgIRD=Vx2D7ni!8y2^@+ zM|*AxLBLE|oggNOAP}UY{H)9@hr{u3e1yvwaEk;a&;S*PhACr}mT@?2oQzbz8&B|s zAdJ!KZcB$Luh!WQ=J>08twlr8nd!NrlG4tOw%sM$#-H^UTAgXMX1$C7ax$2dsh<(ZbJEi@9G2Hl z9XoyM6-jy=IY)42=451Lr#mc4b_viV=o+&5j2UNorqjmfr$?ofgqi?OfCR{pr)6%? zZV9{5Vqa%RDS+bdvkAWY?OVD+5}<|v39WBFRHDQLk`$uCh%#>iu)%^~{T6`wt3=>P zp+I287szzl0cgQB)6@Q7U|>{&mJ)}zrs&MN4!_-szHfi%ElCwdH1=$ue{irr^{+|X zZzKW}foXFb>3_Orf;**DkXa>uf`$3nuP%M&6>KFK357yLh+xqW)|q-6$|!L40aXwO zlW>eFpz-VGfQr|oaqH&w(c%6WOEU-Dc23UtIb;B*6fS1bbdi=6i3SnHn4UsO4HTW}G~A)NZvw0*Nfql5m3(NDvV*J;tm# zIXORkC-u9BZ8itM`iu#ozQGuvB%&lVy;{YMmaF4K15xIXpdlEc7(rUWDry>;MGz1u ziFVFF%2@F8Pe1A{ENnVd_k#~U=zH??^5-oUkRakMw_2VKPYKir7$P%5kto;<$&&k= zTCIZ;)-3{X#P|k$hxHQ41ooH6k^r78=`dm-34|1YT#Y3Xu&t09GY^Ro)xW7jBydvA zPi`C>N%gQ9*;DG#tI7q8BubQo3jUC#x)IHC2}#tl^?9^wkmpSX1c~wB=bwJuTe!XH zP~CeU{G`9{@t2=pi5ZfJi1D>O0D_e9sC#zWCv^!Lf;3RKAOn;PR>1=kl^xd24Of3g zU>t>rx3pXx9U6?HQzAn!B2i(0j=+j{Y$X{>&&b$TT&#BkcJRIz|0G_kN|u=4JgH@7zzNAg{#z_NNKd(yf!-AA7yq$!_jDz zKm%ZiC{rrHugABiBw}15h(;ycA#Ln46ccHATi<#2j~0geKD}|(Bb~3DIJ{LiuV>O^ zlhqO!t34$4DjJPSLPX*ar+EC6Pd{((Dz0y8QZnK5%U3xHB5~u!weg|Vaf2~3KwCF? zDJ20n(3+i-|HHS>j8BaGG9f)|HX9*AQpHC?^soQ`AOJ~3K~y6Mz!=)|b2Hz2`^?n% z$VV;LBGLG%SB}|i4wXSbMCz4AjlzJWh_u|g^>lbj;fyk%c+?+3y97=72UBlcU=qBE zn2Kv9CSs~j>Kn`3)lN3Z+gL0nW6{q){p3;C_CsrVAvv*RZ;VXlO*SEA3J1Knt#gAg z3A)XyZOhGBTwILD1e)qqVxlgIqG!(6Fs!PXqV%jP z>bJ>008?5+N&XLSU5iDHQzwtwY&J6>AtiQKO1T-iXk=P~ra-SnI2s|VKn);@N(76jlK*5PoAPi}*mFyN_|CcU z@zGTsXSXvDh7tSpvr8S_#r2Jc-~Gu?#~wepbonYrA*IoBV?FN>Y_7)Sao`4PI+-|TUU2r%l4x3is{8rhMkX&O&MgyFq%Usr|>>1c{TQ|FUlPP87CQxkw}#9HKBe0r4YlzqZg0WWM<`+ z7w5Nibp^I;D=M#;Sqf)5v|D47QQ{iYKpHn%u8j}%N6|@)BgKO;f$7_%G+J(4O{Irm zj3j8QrX2}o7bOjw#H3B&P8H;Wh{?1&1hW8{D*3182_oKd`^MnVB!E={X8^HCBnmc> za`O^`1g(;sG{h?TAu$kfK?9>xCr_TtO?T`pEa>j+^5^Fk?%eJ6%xBpxv%^n8N?^&G zBTJZA$;UHgm6`jHvR*yqT%Nby6hWG>6+uWMe(mYd zjI!APcr+r@9TpV>h)Gm+1`S6T6{Rf_i4>Rb9vmD5K#q_^0#9^J z45TDBIqjOB{;Z*H?;Ee3?zrFO_pO8@@$CGqp7w_c=rzWquIVghaH}m{Z3ZY0g+$=o z?^_9nLWQL}2L=WNA!o3&xG=P`><837mjD< zY+;OXB1VFPimgbTW~$=@0RRGCZ#W!Iu9t0CFOu|oeWAoWNus7<$~+l?Mj!xyfwVGN zQnR!0Qc#LOV*z2L=WRnZ{su(dydGZB8D3!j?sl zL^L@)Gd(joQM>odYp-?OZ+E*FrGoP#29R1hS~4jS43wJWOx4`+dws!Ru&}gzV4#l> zG{jvcMQU7Qt&NkE-q|psJ-;*CuKX!PriO( zZtlN%{E@7zEM3>zAKZ7jTp=wx;P*rB=5(Dr2t)*6$yJ;6m;jK3g^ALlt${!w7>$GaYIHpFb82=j7*i zB>H3|4YcgoTFx}fg9rEKT&|#&?f3h)=9pAv7*e;V3Dh`BPKMv-jfNvxd3kD91R9uy zmNXz#HaH`koSqq<{Y=S(wg>lTyvvbrFfYHLv;BUQ%(WP31xpqIGXPbg6UMQuaC^|_ z4Td8{+qdWzO-TSk=J)#SR$J=rg-m;tWo50(8A9;+d^Vc{L6XFXK;51W44IT1;=zC? z8Zi_X%arT0@-8L1MN)P%G0w59aC^wN5{!gRPbnlTG5xyoR<%v#EdT@3h|&1;%%tLl zS6=PPZF9}JajlOd5j-dZUSBv6EGpSK(ASU1Qz>7?ktsO|l*|K4Wh5pN=?<^g9|(mJ?4eL7J13vG z4pMTG%&gxbg27-Yj#~;#pA1eb-!3L9L6}U*_!_OjxXUtHY$Kx^g{Oqy}{OZ!>zx?2>SKm1M@uydVzQu4fnp3cCvHc;b zI}I@;&NQwPx*5fSzNLixe=Ma5AYy={pu9}8SlblsG4A*JY&KhxkS3>R#%4b`R8#rx zs}1cB?$3IcBhgTjcO)T0Rqt0*cPg`6;y{dnv@&C7=?}OucnjTfr9g8-ja);*{d4f2XZtr(WiEMmn0>Yf~Zuvk1O+P zovQwns!mlJdiHD@*&Fz4W%5VE0Wsy#Dist{a8jE^#{lCJ9K;Lhe4IiA;#0L9?VPLS zW6IH<)UJDaV?X3rHqtW<^(bnQeA_Jr5;9ZQ7n{lYD1yR`h%CSDL`h36BHEai1QATT z;QBCYR|gXTaSBCtPc0LC0=N@HtL-bd0+e?|aQj&m zUdoDJhDpz5#2O8mQj0HqYs%GEp?}v%@fPz$5m1=>5#@OfTFfkS?-?&6~Br5I6UePIPIdYk*1%E zo}k}lZ?s1W696ebssCXess}G6m}xEU9%+bP)s49Ajiv|uIL(OT{!r9X{O6CJzI)`n zOnd@^8aiqqI!Zi+91Bfjh_HJrEBDrI#NcS^1J_j>GAN0G$$ww~c-7}@(t_i(&9*9+ z?(z-uaK?^ix(biVCQ`s%ijpq+=90|zD{Wi$LO#qI|)J)A$q_kS81AlP7&5?v-u$A7~{V~afo=&=npsmHZ58u3c0Lpto zIR_?w4+av&TPRKr%F%sNl)cNb@K6Cy*8O==)M8n$wm-a}{t#3AQJScUg@fzT%_ecH zq_=nYQ`G)g?z#`V|1Ed`eQa5S?)METiO!b*1*L~an6=mpuQ$5j zV{I>g(h2v(07(JfAp8-V`ZDd?MT=Z| z4aQlXZxP0BEslvHA2FhsE!Yj(mwLDAY!>Xb>rS>+u;^nvNp8wtm)Se+pKDrf^;(wV zKH<~q(4mwo#z53BU|op5&ziR29%j{AU#{KZKQgx?EfUkW zUyi+qtP=lSrM98r%~9*+E0^Ey(bz7t^?zD_SS*V-L^Hz+ZQ#nZr2^8xl@>VfE#V&{ zov+VR)Oo(@*%u{RL6Sc|-=Uh-v8p~ybmq>Tp z3NN?(XRb(H>eb=r)0GWS_&G55(d(=9e(%OOtalHALl$_&M30iI&%~F!~08+~T4F6Wi zXC^at0_aC$R5&6u1SF_JER0&#Q9H85H%!!a=pZj2{RsXA*Q$YHDiN zyFPHqRC=#2vD#RRy&b%K&T!&kzqxt4f9QYjV*smZa90Df&vQtIBb7)8*BLgd8#V|h zPnR3C37K?Nr|f}`VM#_e&oH6Ag)r$-xgAMG!h6hphGySZ_&=E_jG*iej(j*>{oeE+@f7aIheCXXFI3JD9Vd1 ztJ@kr`oc`G-VtC3_>(RkM_LrmrEw8iV=>-7*(y5r$-yvJ579!nWh)#c9T?1EZCp@W zZF+c#AKiQ8X2V}$@cbZqPCwmCxf*ad9cQpoWik=i7zl!x$fq^@ zl9^taUaPY>zhWoRad4>kf&V2He+tEMR*}h~T)W+Tr%OL3y6}n+`iqo$rdK_QOft!yCK&s}_c=2TW< zU&lCV-Bu$$elT_RcMuqiq2KQ2a}{fW+^kF|GZ67WvG0A_UpElzZ=6>2GR{tz09t7c zczdm%J6^C`OUZ(e5*;@?pP!`1?q|9eAnG;y#&F7ndu{jg!fG+VM~(@FF8d6XD3CNF z#xkZq+EvqMr-icE9jS5}8L$pN@N}}U8{UC|{CJRRm=!0o&XuofOZ|F^UOh#|DW_+| zK_5OGwfv_&`tbVnTPE8fd+;-}*8F8D*XWZo7rl$Iv4k1iFzkStt?}sAzwpInk58A= zu~)0%B43(wOIDikaf{}#Z5nJF?$&V>gEz=2C@|F{LoH!>!)IZlZA zOJ2MR(A~2=w617zQ}$_-y*N3hgf~u7rrGFSXlrpd8#n**>&9d7Oit;S4VPx1duJ{tWD?Ab zY%hNdUra6UyL`_5lensHqJdAJH(SC=q9n-^d&0njmfm1q1_lu|kWdB&LXaz^m8?1q z)fPjAL2qDMw73cay>hmEW^IeTjf=+1zrhgdTr#nBCuL}P{IzH_olK|LunmL2QK%5= z_UsTIFFXI~-=?QaLoq8GyUn4g<|UY;J3mMTbEVOB(-^xz@T&?YMj(zgKvi8`-Oy0n zXC4eBY7KY_k@|oc8;eH(d(!;id%qEaExvSPJ7f4}j(6!Q#JRS-%#4Bx8G)RcTGAjy zIYI!`hw!)n-W{nBY;5vJ&%5%X^=EIq$lCZUSPcuOpsW(Sk;J@WY;FJkpg_J%kxcMU zHdU!DdfhlR=f75c**01&Y`4}$UXQc#)MBkafB4VT3(-YiY%cR7*;|}x1)KidoA@=^ z>FF4-0n;k?^n?I^4lFCvO!tqUW**f$?1}hY?TxKP`I-LPW-Uwp@};8nktrjx#<1Z| zzlrIlllNPWpjSd6y*tk=KhHVw2mkv+`drtn({~^#fTYZt1p<{M2u#%Apl8_X-vCB#P|&Ae&x6t_908OUpI4 z2XBTD92qyV3-^R84tDmto*+2yvudGbn zdfN=%8gF0U=aGrH96`6GcNl4l-p_xUk|*9a<`%lZxsQ*}VWikZII=Qe3!{2ny9ztY zHEzD^jTsfOLjj+Z%C7)4iRv#jMCs9xpZ!5!l9G6NW+$@M*nD1lq&{HBMkl4DBpiQH z{jS8rL4@+HNwe+|8brZ0ktGyn-dfeZR(|)N#)S4wyDtW8No|op>gNgKN&c+1_hnoy zRJh+JJ@R+(hNdV)8y+8W zAexMt8jVRBq%f5kuT?DLk5ux6*;G8 zghl;S-qxl`@sULEG&Ga|9bIKQyu-cbF-*5i#$FnC_Qs)sL$891(~^-> z;itL{b`BsBw|(z`{jl~3><_-K2OBQJ$FGkLIh+WJ1N7w{9w^@dX{-b7S-KA^iWM&i z;Gf&a<2lR??|;|Noq^1qH@x4%&Z3}u)4)FkzZ%=y-ET{+fqb5Cw_QE3u2be{0tlb`1`psm~`gjyK33hd_H9YI^ZrwGd`ZTu8H*dE-`suPihE!BP zl_J|!1(jcPp#8b^bjJ$rQy>Mv&V`gxKQUX#tDmR0Jp6F8VIH#5IPFzc{Hw9?N4xzQ zI12r}$n(^{!88C;FDNXWEC_v4P`(&qFKRuHL z+v@{Nuni(5&1`hre(ZR@2v}+MkAVj|Qq3>Hn4UabqV6x6Dp~%Ld4 zNP8hZ9@zf>lvNfGLtLegM0|>V!zRn0)v3?sMO#0m;IYrJDN)8_*xRi+5ohg|rdL*c zk6$4aTcu@^G!8nMvz2By;U0a& zg|x0f7rMGrgoJ~$rj@qe4h!Y_FE6Qa4YjoDv$Z$GL(2{;jZ-e0>MK0TSpqjp%;L7z zzr`IlvQz3|nZ|OhzRG=>Tasv@V!hGxN5H}6DmM1}wK|*qihvqaX`y7I^RrVt$)uzt znCpsa>x&Z<}3vn28m3t!-iw&PP6_u23kg>=FyndUA(7MmT z5aw=wP;GrZD?K)_BIKN$tfZtuR<_B-8vPESqAdV`(avT_9G3q-Nte}%fUD@9b&P3V z+;M*;hU9yRj7W+Lyy7K#+!Bl9`mb&=B2YKMJ#n+b!Wc}q8uM{LZ^RuE7Wu2Er7PJF zxwMk;@~h(omX^ORk?1K3MNi&7`?h+{RV@PuJJKYMnH4fb`)7W#&IR)++cLh5Km4C= zo6V`SpC7MFGWjbM1p{YwrXkeO6?Vv^w-l}a)m~>&4C$KZmC_h*grF5gP9$SK7MP{b{dknKZK4e!X#e(JwNc5lJtvUyXWX2{FiZK$a-M z$f)-H$aT+QjXq&{ZhV_5SNe!IjX4dNM@L76PDAFZKIUebFO$aN+>EWj3QuYh2BCL; zlWfu+oEDJLYi~wy+TMDoabv!7_waDxQleg6SkS8X^3$52#cH;j>OT2M7%hNTw1Stu z0ULj5>)u6+)zjTy!dGYoJ*gPQz?jDHcAiiY=X~(7-f;teG68E@aRTGo=N87E7Pf0T zOsC7wvLfF^bQ{c;=ga8+1Fc;P5+-v4Jxg~s-dQ7w`~4n+V4qw5O5X49{ntj?zy{A* zQWj#GuHGEylK1a30LFb-qJeo}o(#swdh25|D&_ypcOYJvu zQVO5Xg*@>mC( z2B`jQN{lKK40W2U)a`sH30@gM zMr|~OigWLDojUn@)QYHs`ZPk&nSrKs z0o-ujL}(`^m8^y}CIuDVxSlaeR?v`yQ1}!|&|*b^{VRn;qK6K`0C8v^UT`B?8j<`b zCK#Hk_!A!gy7<3h8iLGUirE2h%{DsC7;vp*orPhja)2}&JB2=*nQ6+g0_kJ3utM!# z$miF~eR0-|Mz+IvG0PJ-gp>}8WrZj}=NGZg>kD1Q;61B$DSBB@9^Cg|m9l)0Wm8Qu z3441y$}`sz-ThfMKJE%KF$@5iKD=oh0^BwP7=&Y1=GT;0FrztCM@5?)aQET0x%jkZ z#;(knPIx?w>zN611y4Bu4p3lAU5$_q%=2!v7xOOQZokzKDGOT^Fj zJjW$kZf>&A=Ow8Jobct@r`D>CBSrLn>=tq5Z&UB?(!gN$&w4jHla zMd2|b2Y+Z&SeEOi%cK*h)ujqjvo+Ze1mH;kX#=>kUozm3-vNmLHG3hT`X1_Kk0KBM zC1J?qZf5Mt0*EMHnGQe%LIk<9Hls$Xn#_AsV&TC3QY_ZtOQlP4kqL}q{A*_pgi^cA z{{&=@4Em0WF0Qx`HS)kpP7K@raMc{c0fNrhKIlDFlrs0=%L8!Hj4=WmrJD3U0!B;v zIw4Uyhtmu*6VjobWXi{5h(d*>UsU9btmW$2KaJIV9})xzpnnjM)727mF=bXq36`KA zx`sM+kNU@`0S`?OUAYPodPj0skH<6K&gQb`42s3geA+BNWd}jaT#7eCAtYvX;~5hK z?9+pFk$;bMVk>?UTR-%K%WX93Qh%BckqK{6V5c7>MuCfF7|BlRo(Z(N!q3mk<3OZ< zx*~O{Azzov(m+#*8vJ6i$vc})IK3W?OsS}X*gW)^X@RNs=!J4&^7~(M7Nm#?38tLg z6lf-e$XdcEa7Sz3qc07S4MXuL4(zqW(&vMa^0c+IlP-#YmQb%!8F@)=WU)mw6{}0o z5p&RFVT&A?I`?4SDs|Ls3fybEJHz5_+9pi{6%;did7*|1I(@^LGOTD(IU#qM+kJ0n zoZ-iA4Cs{JTcSR$tB@e)a&B0VqL-cSkvYv}CY=Yno6H zWRbO+L3ThOX6W89=j2x0j7@qc4-e{7gjn0Rd#p9jZ0n|%O3Fu`7NuRK!@b}mPY(|u z&g{gJ>iLL_fDr4f^ka^o8=_I0C8mBkBr!O`o$pG|#@>>nv+^fUu04x8NT8uPA#d#Di%%}fq0m!hTh$Ll5s1!PWi1Y}R0;fu*rWJ9O-3Pq8hd)ifr$dA8lW!Ow#z_So%(swaDPHD^k?rVnc1Z3@8Dz}d5rB! za~T8f;c16?o_VI=vgCBBrl+7dg;i%fWw8<403y{@0spJB zx$O_M=O4HJrkQNIxBj{)XFEi-65?vJ~zC^ZhtAn+Hn!AiP+mmF93+ zaAcaG0J>CbWFV7AE*AQ_ip$2~#N%WXl-UK7hXA z^AafWX7U>$D`HQW`c^-Gp?HEClGB}3`*eZhXD?K*CD1QyTnOMLKAKG0lx-BkFqni07vYo6xDoPq}tr7C2@P(BW@H9phP7eRj-5$WsNxIkZ`Nzw?_8Z)^N8*dEKX7HPDj4Ctc@($kK;+ zgfr5L83GbtZp4H-FiScA)RvxClhGB|#7PEJee0EyOg*tW$*_a z&zH@aeWI9da9H(yJAL!I+k|$ei6(Nlq!n8mNtzDK-nyu1`%t{Ro5tS+Nzcnqj}eQ| z4)WVCix&4FKj*whn@O2Cgd|Tm9#a$QZs5oA|WOi4> z+$?)GkCR{m-2KTHh@{2567V&Smd#is?CaauXKE38I3z%{*@Qd)g4+5OEg2A$hfP>J z0Q8F$Kw-i}^{Bg`ARH3El72j3a+00Q4|x4V8qB#X=o zGz5x9l8T`Im7AL88lGEfl|f)jII(uJamSiC)v!wQ3A)o3s72vWi#!4$sev=mGrkwr zx*#^eIBG>_4|2^1UI;@30}Zoi#K|T6xnvVNDOBSNKxpQuw$d7Ps!A1AK!Q>=+T65wTtXl>@E|T zE41EG9bcphg(Z5V$oHAXx9r1RR`>yVV7ANOrGw%4DO3Cw>!v9^5s5!~(xfW`L}|d$ zLf0aX-#pZx2`-(wSxn2WS*p%W*-cc5_8ZFC*Z$P$De=K|w=n+zaHK2uQd977;ee^G ziToce$92lOVutx=Kx(F+eJ&eF01yEx^GL~RTw%R#DQf{>`5ie5m1HU)*0zcZhw}mL z6ff7yqe?Hy6M^|#tu_+I!xNgG=mxLIZCtUA&J@(azuoN zDLRUtZ0EROeOIw@;%gMn#bkrOMdrkgAK|WkY;c9xq$Awjo<<_7$uTrMm6DI-H$m-3 zg|ryd24`Xp`Jh#&&Oa(g0w{yS19{FIRWGFoFin}On|smDlDvA!#L5ykbb=3@enNFF zdsi+bP!-5Y=>#3<;TmjpHFsa92C{4RqD4Ny1s49^XBvAmHd;+O(Qzgp=3Y+lnP8|? zvOg5|WaseWkM8uFZDVc9ZF~_;xiL<}<0{W>tu!;BVezam82=MiMt6FcxFJ<6iQ2_}LroTID;0`{Zvb+H4gM38MmW4tCo4lfY3sc*f z01_OSi11qC5jXIQbb{JK;Tw(|Q8i|vncAz8w>IKUXm__T_1Syg>g{CfJV83L^%ilTE~7XgQ62+t>0Es$TfEH+wXWb-Ob759ARg z*?aa)STb*ecc9|6Ucd@n&LnxL>monL=?PyoQju^df1Q9@=xXr@In|#*Vz7O0ZX14) z>sU3UTZ|Vhk5TZiVEckeQ)J9@REQC6o&O%&UAx*YfZ*+7!7y0go8yTst8D+5jk=H2 zTJMZ*hOn9DR^^DhW=3bOGk*tE(L9BM+F`Bo6!2VvwB78s(6$$DEsM) z5w_`SJ1~;5z#t%|sb>F$d_F(W26ZdR<7+BefofMoE>cD9_>4~?6Q8X9+S)BdlbbOJ zRFL?7Vn%U;3hl6e(zI!!C`gbe*J?>J{7%%)t<&&>_j*%cDdrXtd(_7L9b2{*(we&Yn(soW`?6Pl98-_5v3m*7^ zy%lnm1rL>7RFO-KK1leLm|ersX!tp}H3mV~=4oAutJfkMRZ6uE)ZpzJF!bobLka221>4QmsVMx+~u0Qq>8^v9xBpfbD+_>Ua%!U&@gwu ztCSrgu1Swn(q#yS-O)nZ8(2OOL#=(y7s$bc^jjv%*EYD*n z%$YJ&k&j%hU#vJbs5x^70mfM_o#>DKcPUx6YkJmAUJcN=!qC^Ix|hhM18xr&^7%rM z=sh&`&ysGV#|+F6^$l5#xv6a$XdC#;jTCzJ0=}-B&*A!p54oq(uK3di5092FtoJ)M zG7e4d>1#j42#dJT{asIu=u4DlOvL5inWx=cw4VwQ zr?>#mj}mMYTp&rs_%Nd32x6RrXje$@;XS`f_zjy)g9> z|1X<&!XNkHl*hqpaaI!Z!0L zoA<b2HxXDkg4+1CtHZS-YY>~?-w zePTlZxQ9N-Q*D%01NK$g-w#%4VuH`m7*o%AC-NyvmxkI?!wEhcNgkK|eINvHjuBKl zo3&~Qs6?F+I(yKik7~ur5>kZb{Nw*)`;Ke=n-&_9DRKwOAa7f$^t$jjh8>;3O?wc9 zuz@J!M^EluO&#FuOkq-EsIib9{si-*;>0@Z&XTk7k2IAw$4_zKqSi?CMO zS+eK{$;-X)id4x5FlB~D*n&-0h1J$w#Q>T2b1e=4fWAfE1WGCb4djMdH{SbDkD@1M z|JZPuay@J@OU7&W=SHpO`O;&3^Guenlu{FFgyu3&{8f>N+%Hb^-k!39Arvx+#!rM6 z1eT$KsFDN;P{82@s;_iT&a${qRBurV5_}+X1>BK@aEg*VT&VarTCSbMiFuw|zu^yK z6`lPYMja%iirk>twnLWzFhCJ>>N$mwhb?3hX@%lX5sReU%-pDchp=D(*brk<1DI^<~{@ z$FL(V{M==oNrVvvKWtFfTia_0X001XCYT^c^K3hs@{J4VdzSoR!Oma;xdLV>@wgtY zAwv>{(3{^EzyA8T@A6wwSf$?Y_QFD*fY#JK3kxPS#-kY_LZZ-x|3m?*N|?Sxj3P+E z|71-t*0C0l9<(tn$D8{`A4e9%H4zz#6Mdk|SMF#2lHpZj(aS?b+8%IoG*_Pz8(-Dv z%Fe!0&Mft)KvDzq==ucvx&cGY6lv_2LzS1*a!@ce=FoQx+ztOo*uDlXeODlL{4T5H z&_{C8f;)#MdAu2er;9k-CX{AFDx8BJk9CVYoPUf9Uj|1c$gQQOqfYpt5)PY`y0Pmu z63X{4ZQ=^dz|sJ8PcMd(RpI--zu})(oqf2_O4>mB2?Op%XrfDENCC)+f}aQrM59bW5M7x0<(wx(Up>hE&T}K7x zsT{R9A+7gT{&Iy)jh2n6G2n6$Comd44M2Tk_l6-@)%QGsX$ez2rY(i7QkH;>hC zZ@;2uE`4>bMRaac~7`SrPr{4gGS-2e-#PaMnjhtE?$Q^_a%j35qQ6Bkp!Sr@irv|DwN`y$Bsg2y-Aj zO1t(8)ne}5#Y0e|-inmo>I{N^x_fhlbr|xRKYhGhW8(6t62KC^;cTVttI2nzLYKex^0=?e$_9{xQ~u`snW{C03}#Vi^BPKFLlu*>o|%00p3u+HNnW5p%rec@}N zI}+L0rBr5nm$N_5lCG<5p1eM@=v2Bpn>GXokh9P^DRyYZ$KN{`+z5Cm78m28a>E=W z)nv`#uA%9-w6xsm4C=uS@1M6{QPpa2yNw>c?sat7Z`v-SZ~6v21ZYaIAV(ic)7a_d z_P*b3!7kQ_JP5g4_YkA{N{Qxw{qqAHfWvS${`nK~vdW{K2l~exzm~w%P!4wZu%7L6 zQJUfKyKMRZJ)*l274pN^+O{HP`<{`zt;d69iyz$Itl8Zj*1(bs>3pYvv~lO|$6*cs zf6K9-iA}w+Lt}HQF{^>o?}V3t?s?A zX%Rqi$(to#YSI)gWK0I?GDx5Rp@vV>O)^xtUioYJzvke57b0YatX$acSk2Z-`)A1# zCR0;5Bd+tRCXFSM90U%`ELQ9!yb%aY32n#Alh1~QvJycAwI7ICVcYeJoW z>yfdyrhvBWK?D|Bz4PMe%)F4h54c4*=OBU8S6~X zA4?TQw7prfv9=@B3c9$vqMkf@awx+FPYeJU!*1ErhPP^m_2Z}84*!kLAZd|Zm1r7p z&%xQ5KBrGlaNo!uCS@@qrBnjKITl^qAExJhXEXymQMT)99s7&=I5;0jy}lY4EVb5@ z(2$5eFLIhbK3edxmc|BhrYIB}R}mdmeQ1CDN&F=~a>{4Ov$eLi8(bI!g@F8*wx4Rc_Q!3iQRmOPKCXvF%8{{98pmV&l=l=C2X&Bu2>}9`TaKsGmnxoE^ z2#sH?x=qgZh8uPZ7@%=0?O#p*4?VD#b)SKVlFtg(2UUp-NeZ3)+ zS`r0+^xmrX+TcD{wJ`?c-Me==d<^*ZVMaS{p2R8GfSgO>LcU&y_bLheDJA9a!p?*qAo?v>@98+MqR+1Ti+1E?p*?q!kJPaFSE zQ12GbMO`Q0atw%jzqK7NIH0j`{dz~G2D1$kkXBS+KCUl|>XJU*m(^9NozQN#e{0Vz z{aRIOe$k5FfccJ!PW=Pf>p#9+wu?D}sYJA?QM2j#h#Cp#7vC>~GtkXO>t^PL| zcKj5%7Cf=AWjx{W^f-!R&3jh=^*gQm1CD~<{bpDF+A>VIwW`}S#r|-snw4VxT0RRRn3ReJyxg!zD`zDw3@*>&w=e~;Dj)(b#!V%V%PYFkT zI&-7MW`&$NWar01alz#)%>nzHkRq$A{c&lU!{1WWGX;Zn%h&?dmQFmsWjTJL?DOb> zobIebK`4vS#N5LQKh3RM?<6%JYO*hJz{>G|;O`JCEkDOuW^6-(aNq*vg$7UnH3x9Y zd8EK$i%~`<#>6nZuH;OQa}hF}_`y~|7uO-LU}I6HV#;-ZGlL(vLJTs^rl$J;$9+B2 zjAFa~v_Vx*uN9Wf{<9K$;6^5CB^ec((jbt?ZNaqgbvE-829>Ta;h4;XV@8LY<1U0? z7D|ajUunlRry}Zfx#F*!fs>;%_r)hC;0I541xy+2xF6Js0*H>rpiML{rODtJO_{H!iLm6+Dd_U- z4*Y86ZWYbhucNyJ3j?2wMALDhk#oTC2b)0mb9+duqDHMIiM{wIIqP3Wg7R_G`Zz6x5r zvIjg6(1d?-u5JDueX%nMlZwRvpwgk!kyTTP?+Iad^V-EqRGpGi{MXMR*U9ZRS`I7l zeli=Yb3cf2f~EoYbEyM_quKlozr6=?KAile4mr;-o$iQf_1o<~<$h0?;c;rcauw3t zpB-=cx#v1rQLB8d5&n^-*jKBPvnw#OsL|=_EIPd=ayeYvmTa9)}A@7VOvLq^@5#dQTQ>JkD0+&5uTA2aemew(?v*7{#P3UV_W zXYh&eh>*v@1E`JV$arIMm`mxl{`GW)aYsdsSoG8Pk9&B$cd*(Hl*5RWhwF{3pTU!F zH+c$QOfDRXyf`brVja*?!upjM-pkp|*)3;DgTjM_6ALyyu|c39eNP61xV`PiF0ua6 znjI}=T7baTeC(A*SWyB1Cd5qKM*ZpZ)W4YA_n%9l@9V_YWn>|yu*>10NN7U%80;R&1PQo&rtT5v#yJr)v31FuxgW*{J8zL{As}mi3eubQr z?@cLWw}emtpgdGb>34&sMp!W@@qq$?MgU-1Bo4xNRT6nLY1hM2De72v0sMC;8TLR< z5XpWgRO&MY2QG>v0iwHA;5&RP7>*MSK$4Du0Kyv#B*Gc8<$B~zFhUrFG{PiDf(aNg zp0SrRppViq(-;s{!cZVFrKKcA8U_l$jHJa42S^8qm#;zZLODsIkf5FvOw3@X1E5YO z@$8UFrI_e277Uub59#I^%b-;Z@yFvY*C!<Yn8&S zl~rnk3(2jn451opK_ipAj+^N(gisC!3`ozjxjN(p@a=YI{Xza z@R3ns{r=Pd2Y0K3$560FDB^X^tdje`T!n__4_5w9_ig%MNk2bzygjj~L9XrsV8-p> z-?e1o^&-s!wnz=A1W{u}>xEW;QUFwoG5P4%|}?poI2IzPPr3lKJJ?_(CFqS`tS zc=M5E4u$Yc!k+SV{irJ>BZNPm$tcwiDIERy??v+RKoCZVUWAS( z%_^QpSUdZJ!M%Rj)o~!gWUhY3OYy>t!jG4f)t;F-Y_z!lcb`6jLQ=Z6{9-YK0$1H# z6>We25T#{#cH#_=DO4f9B8T78@p_th7C<dU zc~wbhd^1H^mfK??(OuCcVE)v`yodAtEH)CFU9o+Ok6ikXt0N z>H07)(>ngWMlY1LV*2msitz2=hJ}ju44=QMrY0id1o1mduQ{&_In&qsae8y@_0#y0 z?vyUPc*J^t__PJmbe1u=oxvDCF{c+OX zm<~QX${+eQjtuc)LIj(cVkwCl=dA*LxlvL=uygZNt=((u>hNp*+oB{Dq%txyBjt`l z1W0rY5TbB5aA*nclp8!2qQaF|Q+6Esv4mMokffzV zz<`PB5O(7;cn1K_bcwd~%TBFUF2Pu496vANfpDNjnziq_dX4K5VKxXm>z8k45@`c3 z)vTMjNI+U*cX7O@9`RQ=fa+k=*xTlZSGYhp4jatjK)FpzTicbzzX%Ax-QG%!;2m)f zK6BmS(5?=$EWJx?H$u!F(_A%)l#;)Q| zL^)S#Zk(lmVKi(55?l3ZeP9HBC<5{O#Nwci{|%hrpaVtXT`^fRcC7!$#)D8el!lZD z2>=L(5)cM3e=@B@BsKz~(0HYpH<^Sr*CCk_3z9que0#F{Bn_fLQ5|v0H`{4~M>h5j ziF>4!fTw{ipnu=;ZIE9}u?&~D!!sh4a@mR|M z6smGkq}64uM4ZHE9@S>e=pL`P88?hzaqOCcCYrwNDkUP?{43Rd=lV5pfCc~ z$w}*0TXu~OSGSk9n^Ku^0ZapdIDx7tL8DV7gDYNSL2_wy-9J53E3caC$cX@!67JjF zcBT|Ws0bV7mR$C%sInd+$nrGnYksLcd6a|ayEGg+-zECBZO6XTDw-$j9iO-PTL}2% z z@sj=qb-Ku3$KX{e9d)U}`?NIWp-(|b!e}2IZD<=sIhFX>0XrAIrY^}~17QPoe?QQ& zaKbEG)-+7m`caZcdPRrT9!mxYk*INPtk#VnkXH|Y%HKd+6ZAY7EdQI+!w$PYuM~Mx z0s@J2yb)lSxf$STgQfDNWY0>UP`v#Oi-=c8)Thu6?oi5`+xT zuk@WliUKzfczu*!b9?$0Ij#YpTAmh}M6pqhM?(etAwkg(MHq(u1V~8IQz5{~4LY~H z_mf631@h{LEaKB%TwFvZeeBHQW6R)w+P+(Fao$ykm9ks2DZF!>Kmbw(ZeSoTX`3Bp zfz(Fyb@IO9PzDpMw%s++Cc9c5+{P?Rsi;q4GU`wjnYisnU*B6YeRCciEiXpWJ{~{i zm)=5Ea~wp=$lRUq98R-62C1RoXjZiF&?2Y1gB7l}sv!9&aEywxHV?HxnptQg5>i6a zZ*dueO@V2uaOZDw^`&q)@w&f)C;&k?qMF|S`^nP{`nPnBg6^#M3B;?|3XlMVvcyag z=B~GR-ArF@p_q)y(aVyH9UL7U9UO2Vpg}-VAB}v5D}#T-PH?LK&j}7btA6eM|HIT< zMzyuIZKJ`xXmJg0#hn6$;7)NV?q1wIxVt+PhvHD&-HJODDDG}2``yp}#+iSTkzXro z&2`UPE?J1q^bZW<09snuW;0@?z)Kc$P@*mqcVN0SmD9XUXG5AR9FN=L}OWNN{B{z z0}K`B>d7CKJ&kMxNuoba<#cU5dR#;3m)_ogEU=7;S)t)E@G&Tj_)EBqh=GM-#*#6_ z>5#6f(et8+ZW|Z%7CU9E=-g zg++kM5W&so1mNSt0Umh}^cH#%0dNThA^9a z;YV_k8Yt2zjg)IPT&l?ZEl?Z<@6*~()3E@*_p$DdP-vIudF6})M0jF=X$2CLhy_&~ ztjQUF(_2iTZqWEVI_`v&$Z)VtdxZv<3=H6lDmm{VvkrsovhGqKXLR8weP;qBk*^GO ztQ9=?80>#yaxh_x!b(I8hit{kqhaHrq#)z>Q)vtg!K&ekGB#5;syCLzk7~o^1jdFt zv6cOa)GA+q4i>mfnhp^ACU=JLKoddxuJ209$}_zoD3THqg002927@C%00s$`7|?Ow z^3ch#5HXuvt_LB~25#BsjzqSY>^;CS^-6_xnPZcpOH5l)n4X^|o$T)NOt`D4#h<>QTwrZ7W6FA~+Ys=IZMx{c#5;~?!zur`_yfV|^0SSH z$728t0@h`c;0i7t-udSJ)5F6<`?(JwrVqtv*hC5mGhg*Ny-7@Rb&jd2u+`&y@@a1@ z1$W3vgqNysR0Wn0HkeqO7|A%Ch*m#R*chT=#H-2s1Oyr+$QO`Yg@O!Kf?~ zRB3QUrJPumul{oMacpq6IWbg*r8Cv)Y~i|ISL}gCl)iCp`iJ$1!$>nNg4lDzrIYFV zFzdwXxeGA{+nsQ%YHfW)5LnRbay`}mKrD^C(%|UhbGoq1!GKLE)^X5fXDz^Hcw`Bx z9$2W)JRA*!i zaL7g)^GBl!{hIbTyE)|Om20YK&4aav&)@TlkM}U+e%dF#-#DO;F^0Vj7*Mrwi9Vbxa>WABsRjljkOT6g|7TF?R zY6;jh4MxjZ=5?qIqN}i2`SNhlQ#YeL$0;8pCPrCUxC^pCYGrPp1msfMsdS}lD*u;x zZSwr_1U~;sRD{SKK!mS5n>T$0(&cdRQqNZ3`7yk2W+XcqLk027yjR2{jUa9H;FbRV zG(&vRMI_1<8-Omm>BicR8F4x zp@OH-+-!isYLl(U*`s;nNcRmdmiP_+L6LZx$l+14wTX-qR&KxSc|+W|f&vx!UrFAH*Ijm?LbGk6-6!a~Y< z(_(>4!ia*pAjwILt=WtW0*fA5TbOp6M2PqS62nJ?R&K(p zs&e4;h*S;Lc^>YGs+`L^uh8cjUC0LP3*@V3kSQ!gYI3)fhkeY>T*L;~IX_15sqypB zs90vPIo;)~bA&qpZ6ylpZjUu6!-dISp#A6P7a`MfVoYpfO7zjuQCBXm6!DwBToRby zR_c{$YQ@T)!1%y~tB668#@;`YG5Vmf*@(fxL2?2zGNSz~zk^RGXW}<>Ir1NINU@Qf zYj@T*>eHO4usk03@G8p7-%b?~b}n0lDS^;zK;P%dszNdnvp{TU#E4kb1b|)HcEAlE z8bCyjyx!XZgOwZ>+usqIe6aY65a2%z=#D5ot7D^>Mw)i4fb>Z*wjMAj8Ujt17yMMz zrmHou0fdNDlOhOY4HE;TjW@T?E(y=aQCC`)$^0p_ouhlXLJ(&hf&5F%$Zd`zBs887$-y! zjIj=ptL@6Apd>s%!su@WOQ&2W=uCr)D)sJXvCdBTq_rP?*^h7ao-X}a4>!X@`mVZK zwH9wRn@@(Im9AhhLOYu!m3GT(+#VzHhwkJY;kQ*NJBwVOyXC;gXX^#qZ@TpbavVaB zJCUpCLq;D_nr%0$Gt1o2Us_Aw_jjmKd>;oZ7#C}pEju*(3m{H^c!n=F)~l(K!it69 z8LUL3alR*nbv*(^Jt~Oe!B%fF6t+xw1RE9<26T6xrm~jD>)kC|n#Jpj#c}u@-`L{m z=6&KvM)luJ8E$Ve`B43!tDruzoRikqHK(c5X7@^;E++#Q5M=pXBTkOrEqB zmwR8JZHhI-YPj5;v(4w(bzSb)b*k|F&8F@z_a6O@$I)^ZYewx)gJ{)Ii(!6G+X#9G zEDJVP-ly|qmD-I~mx1-`_e=pXBAjgMs_I%@K0cU(IO?HnclWO^vcY(dETa=e`Z3@@;%i=m`e^lV1K_gp#D5y`Mzq*63?)EH6Kn#%doQ z9erEvu)OOHsViWQ{&@OhJ#VJp%l^IWeRj`7Hnr>a8WIny_%{|(cDloifD~TF=MYGi zwFZ-p7B=4uWf&qVmh+!!1dZoUnQ}bJczK`Ib6e&~rOsQxJtE*KshkKgX{3Xk*!08Ky%>ZtUXszxK?R!vPg941@q zxE+`&Os&zS4LEN4Ed&`zexzoj6k74Q>&MIWP8Bk+iX@*sUo@ZL0-j=4R-W2cAg_6kSF?`r zdrRp#`Gvii(QOpPg;L z{=Rj*4{T(@4LN2%=DeSqn0+iqCPE5)?mVYzIWNMU`-!WQa&lMT>h`<=F%dC4>-R}Z z%}Mjf4Np%^4@*yL?a*~wwQJ^)PLWIeNp&WGlt7LYNQG!by8x~9L$oUW;CY{-0$EM7 zMJkDD_7Mf1GCtYkD(h>y76h}~>`W&7ByH`*69h_={%nj@6Z-EO;^^(G6a`Tw%XHjt zc-eQ{E(o0e;^5_dIBP#IO~C$39s;v#!O4#=-`h~4rhx#Huc7aAd(h?CrD%>q?1$tQ zlsRJ129S+WhTPuuK!H%~{C|zz)dVs4~nm1l1FbyODmW;{aecmTR z5Tjmr1wXAe+o4om(SH@qkBbF&zOVDlsWQOAIAW&=tM0QPsi(lZ(D+`jd<;V*uoTFw z1P#(!Q%#oOeWY8E0WD3B0#K9nc&P)}Oph97<2I#ZIX61LFim84e1_?pE3YD$D_uW2 z4YUwzps=Z5UYW1qVZKa#keh?>qh?@xq*2-%G~o^0v{l$#Waw~i3?C3SJva(S@e44T zq0Lw|Gw@?tCEG4d&p8`yi6WS*Sj`nG+Ey1M*D|VTxT>tIEPwP%OA-+sg%{s`@tf*) zn0L0`k18k9J7u;#tyiDWY;l&BEbv)Ln(dcz)6qfrf#tuNQj}8N8PMn?N?npa@b@pt zIG_`&3uwKPRFWI zov--juw?(EYSrqS*KVfp^N29|hoSbiHldeS3IU;??!LD@9Xeq6Uc95Tb}nwzio2cG z?Vfy7MvY%jD@3CsYb|q0@h6$-NEzcK0{1pGS{_dir9Ln-*{fEauM>;DPjh~U(PjV< z98BgkVTds)>HF@GkXV&Y*M8Nx_qkiwo4Bzo9;5Ykf$7N7w7syWKB#vln8s;nQjg`V zlF#pKJgB;pt3spmvdZ3c!>75fHC8ih+TnQr@vFI<#CNJsMc=IfS!18j!6U40=WDGV zbAud1!r{T~&9>_|RrWr|=I>W?`BHL;HA(?jy|13`|1Lpsnp_bq>@RtVBY(WkJKP}f zBWzlQqJ;Fz$FEmn9MneAPj9Ekp^CbbO5~i4PeXo>ujyYIQkk`u+;~t$0-|>M1E?Gp ze??pg;h^6osmRbT(gNGfRW!mX?OUNYty{JSk!M(Of-y~gf@W@x%D`B)%6wHT2lZI= zY|;Um*3V|^^A@AOi*Qa*DKnf)z^G`-OyWQV^(!S1*idD=As`8{b@?OW@N;>r4x&rP znPJPS@s&~=5m-gLz0NqiKU$BUTNg3nfEE=IHB6h;Pa%BsH-Q%`JE(IR04hDM){ib3 z!8FCNW}Y$8PGrcsWOd3XPd=Uku8pxlG2C902a zIJYa`(3ubw74@e%S&kLu_TnJ{Gd{d%5ouTJkC2Mhaf&<%3Ca5UdXIqv1bhQ>J~~NP zU6F{l7_-a!a}tur`Erd{^^*IR`o4@2+Fhp!`MP~@Y0{f;)2-CfxF-~O6jSW?GKs!F zp2f%ROpA3!f{)MIPw>}Pug+vYsS6|WaWd~W;Tz#!g*DS4P`gYYLM7=N{QEpaUI*wevf$e z_M2_RRj2DzX;K9+9VGXeDx;|-{%ZVH5O+R_)7A=eMJnp)g&D1Hk5w5Y?U~xbN=0Z2 zt0R9!{Ay~U*ih%NfD)fps}cLBFA&7?Gtd}WvOjD4btnXZB_`ibyPhjToXiKny>b!5 zspn3L2$vWCF82sQBj)u+CWtu=q{ZRbfllhztA)$JB?Tn|YG_IvY50)AaWJXD%!6uz z(=z-s2=dz!r-vng=jLDPj>oCmrSXRD_#1qYKr=J+UIju<4F1@s7zRlftX1X_O^bWVow@v zPp0DK8K6u!WAxwQ>23!>HD&EKoB5230Y(&ajj1&4YM08==G>4qitl#AvaN_eOXA=~ z98gD_csA8FQfG=_OE%B-8?ENwI&^$Ygx#mt=Xkb~LR89BwZd7F9VfHf|MH$R%k?o4 zNJFP6>Y9!=0LHVq8p{RgJWVVdG(luMxz$lrL8V!r>8s=*(A2> zL6-z`W;Uz3mQqLReI$~Vzd(1jAkyiX?R=BoPmH(49Fxn}2eUChUxac;D5ViPyQxa~ zG7QZ@qm}gY97SaVRNy7-{pCl7#Mm1xZS{=JLum?9eM}?DK9A%(N8%Q5q0NlsiWKH= zS8t(YCmS8@MMXuOot+honN+4PoE!4>J6BshK@hm?k0r2$85I^1Mp7+BLH7smJ|x-^ zu$oGNR0>N*z0Xtt!Tem0Yq(`>|2Yy)H*yS;2xtFj*!;}Au-{`+-W0xL;t@1JhH+FZ zf0~h{>tL>+W2=uM;3b?aFF znJ}_gS_|?T?F}!J{ue*py)gxu)Df=&4oD*eGF*0LmT)xNEr$ZtbeQxZ0$Slvp9MMB zeyQ6meq+YrD%V>{K9bCJc zI(`ct6wa}CKMZ;OGMXj$cDP>s0LfD7jc&{L_xH!5*Li9-m)eczO7svG>^H;}epZ4Y zZm*ZSiHj0O->W(;4`P^g?mB8chPY?SK)~DU+)kb>87`jYskFL|60wt&|F8gN2XD#N zJer*eSklG8P)4hVCx3G~d`IqJSZtR%vo&8|O36@ZRj9}v<%$yOq)wApz3$F8>)x$( z+|Di^Uy;*O2AlRLg_=r4P3PluJxN*VLtjk4%Dx}fyaTr&|L->Q`Cf+|{JpTQr7fJM z@%w9vTE!y$oFoW&(xBzNP*Jgnn1dj4et!ORRjuCgKJu9RoQhWKh-Wgp=5_bblLC%H zeLr&3-Dj~ACSaC5F$=*~&lbInVh<)r%&BBzrPIEv%JJ;G-s!x~Go+L7?{!@zKOmK* zcrl-n=mld5B(`}MUM8pBrD&H~v@|!?GZwyN;KU}DAfNaoL1!k!NMO70oIxV1%Urb& z5x4z}HiC?|#$?tbL|=v0jT`ghQfY0VEAiRilWgrq$ju6Z2%oWm-cJs?{zjVKaBw*} z7>7iq1KB^z%3ruE7*dntV~3B4D@$FsY$V;@-a-&xkNeg&2thpo0Rb!4F4**RR-AD7 zVcqbv{Mg6GX918#zZ}?7n**m}E6H-+-tTy;qhw)|_!Y#_d3o!jo2Hh%=0{a< zO(8ji4>>ByK0hhzjIDEqq~Y|05A8A)hP0%F+u5a7jx2+s(2CfVze>7c!5Dum&cZLx z|2h;yYHM|f^?+YQKJuU>0jEnJJaQ{|1!;c&Z{^S^Vi@MLQf;&-lQHtn zmi``H-kSkMq)lai&W=E{E7ubCu?oQ=C5RFHdltD&hm_`D2&+I-3_INB$%>R{fz^GJ zUP2E$X>2e7%@8d&n>oussaeil1DczMl28iE;ELTD*3caDGk8*j<7sq(J*d*DEG6hc<;}QF@0#>P55r^Q*=k< zzCtXm5$wTLNaRRTz0+LVQG^Ik$oqMr3dNkIWXWOZz|C%Rz0t+kLus?sg(LktNDEdS zLmo^6Q`xh!u1*DJ1+k%0g#Cr+%z}(&s)#fcYRVf>!%@X0u#iN)5W_8)@UbS>QZTW? zme65sf2CE(C}}mj%o3>;ijj@ixP^#P}ggXnKzl>@}tzOW_M;yjUJg~ws2$z@w;BayR_iUmQGR3CaflrQPwqxk!33+#z)1X zfxCdW*l;3RV}?d)JAZRuQvCo+Snyhh0X_GAuhuXh0C3p+ zdTysj(G`v4u2ZwMv*pe`<5&KQ%i+TYR91|M_3*O>GhNRUHBv3zP@!m;6hBAY?tatD zvirvlx4ItoMn{&eL4P^&TyM-=tZva@gGlYLB8n;;4JJq4HXa};xB?mA6L*|n#xE5o z$z1Ab*?)>ziWi+eT;!t*^P)EQ`lrb~l)hgnE^{-Gu`TMidq0ngkJ5(2;v>86(E7c< zYb8J#B-9kwZTr1$9>JPvCo5!Y5jG^nNQMF_V}9xeIY5zn-xT%@#So7+x;4BYov^`J$H$MN(&F@-=N1i27P4nPIE$u}+ z9aU8gJo;j0MfJV??E^NbVYL6+wr<$%uD?!pNEjM$b?e;la{fN_E)7i`kqljWT%%{# z;)$7L8~FX4fc93g6-%|62PfDNu}seUVL0^MXFTiu8!}#KE(NK1f|wHOm_5D>ZDW%b z0Xbq&7%8cU=Fd{$tnkIh>uQG`Z3cOjNik8rn65URl0mA{VdG$sFP$Qx#&_UiZ0N0o zVNsFI#=?~Hs(HxHrwLj3yezkGzR>3ud@gt6vMe3Wd!xzQ+lC&f2ZRCNhwYP9$MqYg zpRHYqe{_9gQD|e@a$>iH=8%(DU;eFijmO-ORA<-+ zhT~umdYIU5z%PiGF4>>V6!3W(e}<`hI_9~vU(NaMZ0!|V7Io#XrmvrC7Dk7WmPBLy zR-i`2gDxHlglL@W3)!zol4Uh4Lys#T-~t0pur93^t2=6q7o}Ewry=~(`3SiB@(sUp z7&EYCxR+!*yzh=#G{P4gP-Ik*<-#v%zGQf z^PB&r-VZ&i-P|G)rkFmPOHX@dE7`oKdxfmulAmuZy3{-ftshp}L zD{CJN`62={JO4>5PGx&vhEife=27P5b=)-1f(qVN3JcdRfusqNC~wOI`x@xHp`3&& zR*6zYEtZ^KC%#n*z?Ru}f+~n_7H(5>Srui5lyzF`7j_udmfDU&E`s-YoQ6q%KoZxR z>u#Dz(sav*^dFP06?E_&H;qAK8qDaA%w&g*2i2jWlEHx>&L-x8ReyT_k{xU5CZrIT zw2Nnk>R7F%jRP!P6ygjXw@y;4SX399EVOgftjY-LRvBv_dwwoW5-7A+8+SBlb^9!2bx)YYtO zRK5fF%ria;>NE_fX!s)%nXed%wB0 z*-y3I>5+JoQ}0jJ8C3$>Li{dgCj_*7zg%P|VzRwX<;ow=s=#1)hlN_|<0ksaCfn8W z5>*iRNtpjC6q0qZG1aOA3GhE?F%^V`oT3~|^^h(%xvXyBF(7!hfADem(b}r^Be+`* zS(IEXezsSO0Ct&m7H?AD^K?FFwaMn?bz3;=vlCm+&U)s;=|=1F2o(fmdTh@_xf_CZ z_22%1&VOa;cq^^$OYA91i%OrX(D;)Ph)+Q#bZD_ah0V_#xE zehT~H(ECJei9Ef)Q;!$gZL`guUvE-F1d_hr?(ICIdK`)Uk9Nv5?_Rl^_2T(siA zaiC#?_WY5I%$5`fX)fiTc8@$T|l7F$z?%xCJlAwz`shx>OC2k*)+LT_?U1My;^Li{)Rv2!xy zkao_Su5%m<98kQV0k%YgIZ@6d{q8xixbM(JTQhG>iDkf?Zt+_XVDZDIf6E*PMg10q z^6;laM)gjJ{=8A%2nI7UG`l!N!)zs+t!*?ABT>ZwD>F}y@!|E{M`OpQ>wO(TI*r3@ z{DNYBaosc$nqGY&qmk$(d{Rea%KE2Lo{~vDDYoGntV-UE1@aNwtAT}Sm0t4TZ>64tY}u!u0YopZy*29Mcn* z<6hA(+4zOF-JYO2a~1UjhwJ8+nh%(w>YiU6<}AYLEPz<17AA!rloh3PE#f%vrjr@F zduv)hzXzqGsp^ua;w5!sh;Sn9adfp<*?02sEgLnYtC|ELpcMeA>$c>jphcV(G{IEu zL-PWNMkYQ(bkBuZ?bYTF_zU3w)oMs;FFV6n0FeG`OI7_ zO|ALUe`m3=@-L3+Fvjs;T=6v7ukKLFuY^Za@vmfBPm!JSym3T>931MjV8O*6Fl5(0s?7>Oz_$-&-{*#b+955 z5&_*^FGej!#Mqyi7%X4`Pv@<~{O?h}=jAEeRrTu6qIDz)eZN(I)|u@#xr{0LK7~T+ zp>8SjtV%qIv*F8TJJ(M#h4Be*S+pg)ULtwmLdu!H>ocF>>hFQaqyVKyDK=x8$HyvY zH=_HFg9tb{NnAfk5QyoqB8tB0LW<|DBFuj%fAv`c7UbZ>1p_NaY&a^AfmpV2kr$UCWGhqD zMO4sV+v$5DqSfv8a39J}&@{qS^W!EH@m_FR`+a*@@(YQviV_$XPUH)th1k;M5mO2g zUB7<|<27_76wlnz2&JF@->V4v3iVI#8v?@R9nP2imx%;s%q18<4E8oK(3BOL5@^IQ z3Htdf>bn^(pjF~3#R0AKZ4~6VG;~49ki1*jOXs+%ZH3eO!y}xT&{J6pF6Z(O&QaM7 zHj0qzyOI41^z%C9Vq-b>m60hz%1?56YM%$^B~KH63{TMO)5NRvW;`AUKf@uT)qMRoJVD}9l(_`5p%K6X{i4^94YNlN*BS3}sj-H%v7B!F z`J$3l%PV{czWz5xueqCy#~Rnvh$2~ise{Q>k*ZzCUHZ{m4eAbF`gZ-ksm&P8kTNyy zcyaMwkl3rGZ4p}3h&!+#d5ItU+;cjk?e$HwjM9k(n}Na{Iv7ks_nl z9zJi}canvq)bDiX+noo6Gg)gmxQPY|=fo zmC1|?H9Czr-M8Qp{ns;NW%UWLnt-cKu8I}`$w}Auvn7bK`PZG?i}))LrkagnfNrkb z-3R)gX9T5ge{AJwezPReH3v&Zpj4M?=I;$ZEPc{q3xN`$XXO}d630hP7}+SnRySza z-rX6FI#1L+%a`uk!_l5=VOD}$P{jT=01>+XzqMO`IW9%<=`r3k|4VZm#tW6(`BH-1&|LBWEbe};#qMY#9;>NrgZ%s?z!w1IYVjts!d4fa zZml8XzU1L)5hsl#SmcbYKL^fDTcYb#jYQm)`t45}r=H+{9`(OhkyCY0Q3DF~^Wqr? znc$b-tu$^74|rT3oUgw_6D%y(FLHjTmxT&svp+ilvAaTwFnE(!aO0-?ZM3F)ZDYk$ zHo_(ZVRl0s8lt*JBfq~tq0{MW@SMDYGQms|naU*iFj_In&Z6Hz-5Zh*6eLyH&U&xc zij*6kJAL<-`73RFFVs4Hff2C3P4;+VvO4+zUYo-|Sz*SVgt!%PXFR z54564!Kh|rN($`Ewg|)cBT6ldYi$NJsZO4j9*@!S?s5VIb2~s|xPeiVI@nQGB`vK^ zA20dtD=scBoq%cZgpL(UR(ZQ~57;lS*?i#T{F6>oWnGstU3=i^(%SlZhlg*n@Rmk~ zdZni0yKe>du}<^%k4IexHnY@g&c1S-ED+cV9VYf)j zT3l*xG4soQO7B-o>%mj-=|s_~!~0gj75msv?@pzOedb0y1)K$^!z)v_zG#E>)fv~l z*;brULLF_N{O&6>ooe0IY=u&^(Y@vXg^<9EZJ2#9BrtE(GBEwsQP zr2C$OlUzh^5nN)s+;lq`&3F*wn^cgnfA>M3wZV6_Gn41Mc&^Dc(N``IkH>`lyVEn! zt2jK?`!AaZg#jl^wAanj1LblgZU;N9*IXh_602qK2f+;<0h%f0&X z%a78i8{DTt>?J9@CwPq|~1Yk}iLj$)ioPH*y%65K%`Fq8kG*yJttJIp6RX7O%!IDPjrlQxyqbuG2FmNmMFsI^Se2#E;#c8^aqLS?` zvn$;T3i1|zy3nHV+vLBnHupqTXP-W)7&S4HpU=++jk^92rGIGnrlCBuu$nrn@KXet z0vL#WcKGynnhaEXzQt$?RZ&rzI^ctwWel+?UT_wu5{jQ;1<7kcBRQ; z=GY&vVu!ht;$2&0J`uzx)@riDZghDL7a+T_`01?tW0dU;vyGPnQLG_s+TuxvG@0oO z)k*#M(UE1}IvRoGb68q7_K>h`$6}m1g?z+TlnE%3%k$$1HH@!`U&3gp8U9zp03wE4 zw#K$z$*Bp;tAnv%SAyMG!s~m`0&cADbyU}|$s%(nC7D^_MfV2`**^+#+CJ%9-tro`-@x2vYHzOU+w`D?34S z+bXkcN{wFEcP;N9=B!vQ!wl10A_XLwRU%4>Ku5%w%oAKd!1Z`cfA4$f}F%0W}SaK@Q|t(aeD%7K^oL#T0GnvQ>5*$5TzD*=QAEe(Q~*l~O{sqe3> zNe}D?7hbx1pY;xCr;CnKX|wy;0oi#I!mPQp(x)B;=KgoE_4Vye1(#@+jVdjl zJ+tFz>&0(NUlYG)4SF$06qN&=tKJkOvYgpFoi|dhPO`sZwTINiEx?MwNt@^0_s zYZpcrr++tEE9RN)C5+)Il%6J4i3ZwyQ-RhLczu1%OsV-+-^mv*WkRR4$t_nfzm%4Q z$|}Z2P}1f676>=3VCgYXuEv=1Da-A#Hyk|p%>Bp_{^z!0QGcU@{`7Bqv^+RaM-hRk z2>PSiH7-lD@=vSz@{GMm;kSp~q-@N-EomHNEZHeK2f>S)BXN}E_dSiH1fB^(6T~N|7j9AW4@vARTE^M6klCyVX37^XTT$#NKpQXKC|B2*UMNyS;)oNcd`f|A2%rA zcK#`|-fc5ra9=-L*l9%p&3^f3nc201<4JW)$&)~Il8k<*vttiqyX(ou=F=M{=KAlo zpKkXtcMEsh7OFpBMZBpzAk)A@>jSR!_VWw_j0TDwNISPy$nWkZDkAkm=NnDdc=B=c zS;Hb$@W}7l;Iy@faC%s1>cqpcucdDT=)O|?*wF)I&Q0$}c*559_EX&^D!os=j~x6T zm)nX<%&uK$4y*6>A(z6C=rO&_Z+@F_?)}a}urJbbevvPRhK7bJe83(HBm?jgO$4J^ zz*UWy?1l2e&J+>3tlbrW6pfwMeoV7|H;KX!xLEc;?!6>k!q<-AlK+wk zQqn27hWFyd_unpQNjth!pC0M;KN!iElq9SRG~tgptS3iu!*R-CD$B5O#P^T~)%s4T zV77W3PX=2_eObVVqlm-b0WcMHau$Mztj^7uv<#i2lde}=gW+>WT>I)D$X$uew_o6} zeFjNvZR<2SHPxTupp!0!hox#Oe>s%e@|IzBTv=P!g1|Dp^u|ji4PjDXdKf=#IBbSA z)=>#mt{Aal+uWDldV|*n{ETq1=qf4{G)Q1R?A;lG_@j@>;U}Qk6|I41gOQ` zxWt&utR!?4vOR1A@M!}ErfoxMMlcS;=ikJ8C;~(*e}Wo9P@z!Cf^ay)2#}TRwy*+S zDal*7pvaZv7&H!$jz*N=8?aobRQ|wAaPy@?4Rus-tr<|uh@ch&nsW$#ikukhfP$Fr zwSfiSui|xRID$}V_SzQZ?X>O$L+hN+orR1!85{k99Xbf1c6$7MsA)@OOHI$n9S?8#E0$m&1(U;b0ubPbhqG;9rcK&H_G2CI>B9Wu zxrTun`{dAHg{I3|fMGgNVx|Ld8to&CR2n>Tnu>_+1WxPee2ENF`%%5nNLu>&GoHQ4 zUYxKQa&hUxdDG#c(w3Mi;*XBMZ2te-i%d}-+3|pR&ouw_`QFd-#DYCNyH_5ELzHcM zqIvUMn^8IiYd@PyL=rqfderrt-B*eaJ@kFPN$xM50!}=%5rg~iR=x`N1(ML zIZ)*nm}ftYQ_KV6{I6#gW%W#tV38}u_P8gh=Xp+w&h4>Yd-3Xum|b+ zpQT*IB+&|2no)iP*yQ>vFDu4agZz7#VmJZe4J8c}gx@|@)gZ=FPDBapV;bfB8T#~y zu#6JB_kTUL%W;t#6Iq_e+p5qXAMKLDr;Kr5i_=wOTc^!i&c3EKAF$`p0Y+5!*YADr z?{qf3pI7ZWpWn_C!kzEZL{Ttnu~0+$U|_@uuBI}ZMFdxr;Ggk3?to-vQ)wDKCstlrere(|Ib!w6 z^H?ip@hAQ+!9RN5d{v;r;=7umXN!W+CabkOjAZCc0ulQ3Xk&7%td8L<_A(m1?oNJ` z(<#zo<4t8(S5>)4eNK0a9dc6OMwak3wWw)BNh(j8u3JNTMLXMnbR@#af0LZY1(iMlN=*-Nz(sB z`n0%Q>{7Q9>0vPwKFb>oY3pV>-g=iC_m+}};HXQ268;aTc)y_Y4+2-`yyLmugq?3& zOD1i^fDLvOKo6D2N$?>Zax$$q`@HU${ zQrK6L-(VZoU&h%FaRE6{B4{?I0`S%O1$hhu_c|2($B9rr0d zznw>cZV3oIYKD9NP;hv1GHxfIQvQ-pKTgZL_*&mhU~5EAH>4S z&*{wo$^psM=5b82LVBy|HF#lRzu~w68ZrJOQ(0MAtIO@|00%7>oDCadiAsu*`H$v* z@bektZ!K>XWZw3ZQHPh?gTqShtn4Nhw6k+{M%&|uPG-9j)}1;bQUPd_AmH}Gf;xDa z%*pp=65VVfiiF~!Hc5m{D@QtP$o+~XR{4-Kvx$K44X1pUb9bir$Ieeu zk2*+#bRguKUafRIk(q1B6pp<`{#PuEjts?YhiBCtB_`ZR#o^N(^GB=4#C4j)Dps5n zL7KT~4jEsA_;qqH9nBw6l6Ep&yIsR)R*Kq+5WrT|J-#y zzE$jt2P^UXhWPs=dgt^Jdv8~gBD%p)!m2-1p~8o7LSmY6NFK;4!w-!mBIFm*_P@A! zdEIyV!$&4FY86A2zI`KQLUV3xY^4yjNm;1=M^lUF%|P7ahoFILGY<>~K0)XT|MOjS z^nj>1+!DGCj%$SheiH5xF%9)-CId!!(01K2-&csLZimj#f9F#Cx>u{P3Cf~ z<}Ws;KP)&%1im=LKI#%P5D@_SNM+UYRn0FKH{=uR?&VdY5DFaz@dKAPDCwP@7BzD3 zvcX`|(?emQFV@o)CoY3}InbK1MFERq0B1KU*pR3*!g8+jbtN)-qUt=C_YBeRb#Fc6ci%zkxRODT zP2C)%@HBafZqID8eU9vU$$W|&QXmtKSBE-!C{j-;$=LK3PD8^q4^>*kgi1E+Hx3&2a)dw`sj)o?F)E}|wm61OLe8h>(VgGjJ z-hjw{7kuEs`|o_{l8HjJwJ{Efw#K;1(}_U*qXbDVEg=$Ok$XZ8OV{)YkHDAI7yJ+? zE{4J=qN`~1v$Yv5wp&unM~{DtAZOABJ76ev%I0l`|Ew;$c=F| z?;gb$PC|L=g!%)PHotcvfb0x}{k0R{Bgan~fGv9n0XS@0p0RcN4xsESkP*ieu@mBo z$bpc_wWX z`Cas!hbM+7IbBNZ=B#^3CakTlH`?q)h;uNa`#At9Y3hp-%UYfGRW&r4QEBWqezm%- z-f_x$d6k+zCe)cIp2Mg)_yD2nC=_gXGjOeUV|<$Je%J3NB|m^U*cdpG6J|X0JDKH* zwo%mNwCYU6-QEF7d=*wQG>J30*1gA855nxrj1N)f#E(ppDx zQUE~`cc&5w&0E8#lH4^DKU~0tQT>5h^Ya1I>jKP<$VipChK1n!!cnttXdY2{8D;q7)Ouae;0pbG8 z4&k>-nM^mLkaiGDF($%-d6AWd0TJ6gsK$M5YC%EUIy`pa{*iEmIJ{b8X;`$jngLL) z*60bViEh~GTyA#q^77_7&yCg^g)?*_5G15S5LR#1|A)u%ZYoTd*~Y9yx8D&vlC4^BDZ|)0@LzrLk8tAX$>fwn|l% z>EixGa&dd#Ye(q&#dFENx;$=w>-WpGz@T=kCL8Ut+O7>Mr=sFpEnIbc{q*2dP# z{|OoGD3)am<^^ozQ096ICe($zC^pM}ejBF{sSE!9sQSm~$fB)n7>;e*M#tzF6|-Zj zgYGyT+jhrxI<{@wwryAP)_u--zA?V_zkbx-Ys@|8yp+firxuP&5VO$>BtA9ZExz z7|c_ue@8sJY<-s`cP*p{lfI2dMLd0r60h)LA?B^|1|kGDT;A9-t*x&50Cn_zTb4nU zp}qj|hhM_82xhAGJOGEHcq0Jn*i-?m)j*w$Te6_1MJI_4JQq7YEUH=Lf6@cDBQgqf zjpQ`hqB>tlK7JU&Bs{1@`678oH3M{XICy#(bH!?*o4ziI`=g{(tXPx{nPZn_^iKWVQS6ePl2(lUo4l+~AUBk~aL1+vL=8s6jE;%ZQ zgxv3Gf+c=0SOSaLN#JBUonCs_I13rWb#Fod78`(SG6* z=;B~ne>duytjFI5Y2N=&Bp3W1dc`C7>l~jbzElSq3};Ljt5p+{O&TQ1s0@nC(VNnQ zADhHDEY@&V$|(eoHRE7k)$Q9a9|4F`)Y*9kjNYhErvgC=xb3e?J4yB!# zND9R?33*rz05+IVw_VXyeNRDaz?DGGtfo(b2-EKEt1wEgTkdO*hNBb~o4}Qy$ z4+?nA@|HuwJBf1L?Zq*&!R0!dFWO#=UWB?IJDPyJAOUxW2eMN~L>N3Ra*Fk=WVQ^d z{&Ubrq7a9Y0+T+s`3Tl;$fMtWBkZ)~AV^;0_;IBr#?J5U-fGih(I@sS$gYYpLGY7d z#2qjRX4ZKFIXBq?_NN-)^2Dt3ud)mK+I=pw|D2Pj z(q`7Xp|8WVzqhAHJ3k4)nyrF3gS$$%y)D1Nnry@M>Y+AyiUhRw1XbPZ>aVxqJo-iP z_>&~L<8#6*u@TNmSD%;8Xd?bij;&z<(;SQ6VdkcnS~VYAP9F>+GU;ogb8 z?=xs>TI1($U>9)M{@>EaFYFG6EI0~=7!nL`f9MAUm$R(;Z|$2KSSbR+tA1xU|2a51 zuzb*euJu?gGYYrMt$tsr{mDE@jHX?0yBW#St%hE6p;qzfaXFemtBPNQJ3BD&sWw_WD|T!>IMRC=y0$Z2SHtqbcpsIcN{^%qWI(w>{0MaL(B z$g8OEx?F3#%+en<{WH4R>VAG7?P~z+&CTc(utQZYyBF1 zD6FM#F~x_wU0J%#RRq$TmKWwZsEG#6CDxaBt!CJV;VqlW+e<5R$$konrA+-is8xva z8ClW=uLj4B>zCqs#aP`AM(Q*Pulub{NQu7z248QHvsGG`NpHNb(nT6mP25!md}gvq zbAO{ycs9;UZDgnh7_1e$Z2cG&`gGNOY{_cqPxJEfC~>R3jwg9w-e9T>sx3}Fz!5fp zx%V^I!tzwA^|(xeHFn1%BF`Yj4Q8SFbQUS;iJ-?xlqx$g7}vnqT7nMHKIAr9)Rcu@CgpFanY=iU) zVm>I@hG2%tC>)UP_+zR%^_p#O86!2SY&)GS-j}lYm%a&jUO6cZRF#&xKN1?gFB^Vx zXJMERQ^ez zmxJEz|I~uo_^g>>jIh3{uyw-RlxqK5Gc}2MkOoh3Y$s6iGXp116i0u!QGo?u#ty4; zGwaqiUHW*^WNAPs&;ZcmA*4VG-09_rnIYzk?%PSbuXSJG@UC>lv*`>rPor7I=rW@PU4D`pK1KXx(bb0;Qz+H_X`%pyNkc^Kb0l>@VhCOb8n(b; z7>VU{@pNAw|2tl>F0 zzYji*1D17MDc$d~LZ!b#GQ}eXtkrKlPi9$@u4m^}gae-3RoU8y1xF`2HIRZ`PAh&t z!JgOo6tg=Y`aU6vJAuUgF#*Narf7~vM!)Db5LJkQ@kMetAf;i!)fpF0y*$(0_STv} zMZW*s9nVumm-oJ`(#F}@m42tmQD*DC9rsfblpK$>p6llZD0KZyE6V>TrSChBaJI0W zMt^i0$VRn$nRTNC-}g-Mg-HHjBmsy_Niz!jVuXPmIVg!yI#%}k(`YsQ@!X2V$K&6p zxmeP$Nv)Z%m(5nDcIPuJm{UXJMvIT2=pi#PYH=c|UZ=}umZ#ttF+w<)JU|S=RBR%* z+jKsJtGNnP(|yO#yU}H?lP_9AX&P&80cii74=h4vONfI-MIwCn>J2z#505)!OD}4` z`Pi{O_eq6)4tM0fZiRuz{Es{!HVp;|o<<)U`L4aM;ryl~Y#9&1R|YB2=qW0VQapY{ z#L&$?%s9l8PlLG>Bsy_nzCup`w;L=tmefc{7F01Md&tMLxLXr4H#*Z}#Y~HdI0Wzv zxhn6zEUr7dFLAc9FpY5&@L5mNJ4Il~?g?%@%DI$Du%w zJlU|$!euILb&G?a!*fU`Ux$KzHUMlF&aMm$OL^*evx&yV;0VUHU9dY3XvP@}Wy#3g+En%4;*D7|PK$_UB!Ef@RmQ$R!_10sSMa z$>X#i1NC0V8Iic$RW|Qiv7aB^U(?&#dl-N)ZSmJH3b@!o3Z!kG6MQ0X=i#cBtQnse zsze0j844lZ#T6}5O-R9KD!aKfB|IhNLr6FOzy(QALkR$MFkwlUXN!OPb~S#4h=vg> z;%d9q21;^RVqQljUU`20dYc)%$L;z!Xh3K63h|3Vx8l3x?}}J1>S=HvPyR`(RB5d% zIwYfQJ8Vq=7Jtycoq(<-wD+iB^EkTje%*23Pud|>$XTmElT?NX6>3!7{KtWT3M1%* z%53vqu2t>JW@Co8(ZUIce@|= zYvkL(pchP$+k4|$teZ9~)6MQ{N%?2A5vK`^CEMFbdQDA{`yVg3RBjwy=Wx@Fw8^lr zq?=6*!B*<{G^gQ^Jw|V{(43w#9^r>_8LAm&3!9B5tKnopZERS1l}vg;Fb#Rx!l?ug z+)0Yn-r9Kmc0+|F>JIskKqxFQToNw|h*Q&zg?|g3{QF&s9xj-(6|m!Ra%LKC=q%7k zKYL%g-b0myz81GXE^qgT;YHt%CcqyHa;!#FvEsbEQlYd| zJVIEbsMm-3>G!tlw;B=C{+x-nxDfXCk-cOTe2TdcX>E$WSF|!R3Yy44yo4_hWzMXu zIcjW}s8oXDKPMZK0kA)qsbY>0w_S>T=>Y=#w{5!}UuHg^eQz|LJ43xUQCkjhCUuMe z_)WUHx?dm1!lx~+U-!a5_%-0|yBz}k!7^S5A|(QMLql?`@av1QMKcsSv(caW1kr^L z6?;r{iuBP!LIMJsO$V_Z$^i9p4QdQfQsz|nC`$n?c@}HZF$~NihHN5U1dH-+m7b5SgrS<1K;w`eNq4RbQCyv)6_rC~HL&EvBsp5c z6zRJ*6D7l77rDp?npeu-^m;M*yj*z4hLDK5usA>Bv{a%6=FVuVeHU?{+f>e@fYLBX z3v}#a;10DuG&W z+u!)4?Rci(A8X9{(zS19>NZ zJ|)Ey?U6~`a1ASLS;Qd6laJS3Xlhiark%KMXd8Rd{L4t!q(ksvMJfPTnE$I!UthBB zSpF_eQ=?RxCc_{gEGZio_MdYcID=zW!>`iiftXCx)5mRtZaqg? z{&Qdt}{8hQxE6Q7h_^wOrD?_7oOLb8h_`{us>vqeh#LgrMZeUS`I^T4H| zo%xvU)5sX@2@M7f#d4puUIRGe*oBAL<=s(oq@nOgzT)wp)6DIK4r>}mTIAt^7Jp`@ zAr%1NV1<(qmwKLMxH8EWl;G&oqkbX-9Suu!; z(Kc1;d0*UYZC!=V*SEUm+ivd$`XM2q;=^+Xwx;s2vhru8X3ypR95(3ox~!^pK75B+ z{(oEmeTG9t1#Bcu{E=!eJU$M7$H%ux%;DI`$g0q+3N?nLHkbY4Ckd@QuYz-s1=@Aj zyX|y;)#xa^A5@*!Yn?iyMobJ6Y;$32t2-!Vj`Z{PC&K_vWlJ2ZM!JS3ai*LN8?IcX zFw0_~o*VzFz!J(L(d97*>+*`3*W5F0*_YEf3lzC>u-1rq?{^)^gA`|Iy3v-zIq3sc zmSwzYfqr!+0jrSE&h6?i3AwY!*{TKnpJ=-O+${DoLbaPot7De!(zCUO$Jgu9WQ4Pt zRBX}Mf{b#1j%{uf3kqv$=4jQM-!oeAG;I^N0F9^$(i=y&E>1j;caAWJc?3^LW^zr- zE^U>*8vJrqo$tm76wfcxA3=U@TMn;p{ZWwH5zY@vK%@H*%=cxKCm||qD-%1IJvo7L zOxLTYu2$Eq0}a6wQp?}cjf>?IQ8lSv3xmgOzxu}&qmL8DfE{8QKWFchfOyW&F zEIy0a_{_ZCM`AR`#DWc9E9bOcwDVhAKIcdJU;vf6Q4jwMEF^ybrWa3!%~su|h41U~ zTFU8aT6T8!JgF}$J0F#40`Kv(cah`#`}5_x=S||Z+4Z)SUAN=%{FIU5b{TQ?#{jPvCsLYWB`J2QJYMSX5Glg`zhp z+LaepjJ6_{sm|#C)HpX$xDI_>@E>S6^=hpX`2y8ArLwtn7=p7(MprP-};V946$2nwm1_jN`Y&cyEgpa4Ncz&n#bq;p=-grIH^#$}=?ES}V`7PW_cGGVA!{>_5 ze>(kyWbw=8oCOR$9Idy0*L1!>m!~5Ck9TO;N+O~Ajlwk$aa^@3e^tNbnCUZ{H1|BY&-@QNybaA2(GO-ExfC*;;Wz1`LK(yZbtTL6qM%Re!mdV1DDm zo=5~~Uh0XPWUF4LH^81dg#Ay7nbIf;x&4q=9vPMC+-Otm9 zV0e~x$M+ENjFjdD?fi?_p-osZmvc8Ww&~cx1r11byS#o z9L6b<&8-fr`7?mGI4DU7U5g?#sk2L={=lXRFI0jS$jeK9Z52 zfy5Sd8y<$}f(&y|FJZFx7kZ~a5fd_GpqUIx{02H%&H;>V6vCcf0+5_-j1d?F&gK+n zyb0pafw3a>)(=3hkeQ3fqZpB~CBh=RqpVql5zbh>a43y(_FyWcS6-k=+`-q-nlYe;TS7e=q&2Etf zy#pXGkNNLPuKFHN5tqMVM{hs@`a;qLBfycuTj%WW^FThQRU`UjaWU2kPtp9CuC}Q3 zv+lE+O#PE2gHSi}_euPx=grjz`pCh6y>v{SD7J7pp5f|^3*04mm#@35{d#Ru_MD$v z0j1Iu*$j;&I)C$D%^a3uYe)Jh(pm_^JEwDjqqJ>65I1agZ!+ru84 zPPgPQBVCL=lX@(8`vJsl?9wi@gNnyM7eBeaHmo;~Hq&MrQciPJK+6sc*1vpyeol!w z{wn{@`i&f13scJk3<(k~4&y-L+6Wpi$a$X(oh2TevcU&ah2oVf3fzbjw+K?DxW=Bo z7>)^r=BAoF-+ycCHhz{_w~2_?o-9cgJX7%>EX>>I!|ZTO0%EM!=P&YETP482U-+$l zTHJzo=Cdy0xYO%r=1@9%6{$iafGG4PW+FSoE+^M?Te2h@RD9Uw4>5-PgrEc&uz;3G z-ojKX`sz(q0Lou#HXf&BNn$Wz(gA4^NeCO*R>n0AF51eA3%q#i)tWI&t_)~ZxH??#1qC8G+2-goVCztCP*A$Z0TVL!buWe&}awYC`~A1s>d)1bfN}; z%k+~O&yik3>got)w^w%Tdke?iX${$X)3Uu-tPAOq2SzrLa?Pr>dJ9c)eZ7wjk?Vd` zp%)|WeLp5?UMlzhHX;M}Qf!;Opz<_vfaUkSsVfQ(p%UbuB;}LV3k6>}g5*jtEia$@DYRP;{FyF zEIIYEp(ZElSOfpb%KQVxLv4uLU?at?Mk%{F6hps4R5bM@_b}j}-f3iZqEd zL0LZSLwNX5^F}}?c;eX<4eEk{ZL9|TT+AHweWsN~7962a3TldrHf|~;gCnuG0e^&} zM-dZK9L0wP+o5+WQFDRmr}sX(`-EDAq?@NBfMC8#Vnc&;(J zSNM|V$Pk+ux%f;hbG~NDPUGDzHOMK%X5?c6lK>PhpFdU85z`H-avS z+-+1NiZk}v>DI_|Mw$mHU(mp%3s7cWd+ewtvh%eg1-NA+*(1QVL!{vP?l=Z8=weM+ z{tmkj{H60I9niE-G{uVv=pT$Lfk4CBdoTsKhBU>M3+OOcheIIY`mm&n{KM}>za!F*H_8h+N%FwG3^3449 z-v*ANiyy%wj+U`U#oe9lYTTS0eouSJY&X%)E^DnDQ9>l3o4Gd*n_$G2>(2nkd;>bH{QvQXcO=c!0&v!}en++2(T zL<+9T9eDr^3g7xsebD0rhqo6wze)bO?Q-93y7%#R)QEe9CL#AH){JnT!keza(2-@> zepN?Rqqm1{9gc)#1mal!Tl$3y+>J~mnSFut?(DYCz@1!;m$eyaqy%zqRd5&aPoR2mkLGWeZ@tlXu(P-TSzmHtsAK%J#O~ zAI`Oz+%6H_FKd?xO83nF2u5p|T(!y2jE#(h9j?(oNjsoi3DE5@^VsiCSea$TN)l|%Js%Jid(?JsvZlP`bOdXA6DOGT-mT=;B- zG~l~|m6eRo-7w2Wrd(6^&-1c<&!r|qL4E-tkeNgIws4o{Q^CC=-x|=%9o{K37>|_6 zDEg&#U&ulh`P}eN!b)NXAPq3G3vRTx08XNup&g}$R1-{r6`}SRa=Ly_& z9h5}>eWR=!9}W`}jjap=K4I2Ki!|1UDpEirPMX`q9jC1m>SW(qmxEcSaI3dj-Rw~G zyLWmkS!wS?k zraGG(C;Ok)pVvsEVy1lVC$aB_cq(P$d3W>Ffh>Cxl4H;U0YB`m)53qU$JFu;nWR#+ z#I8o>L3fcs<(~Xi)!r7d&Q}MSDOMh|;auttLH%R;_<=EI(N3RY5V0)_$z67J65gyx zEKC%tdkKa0&FZP>+2|_*?YT#52O4pn7+?Q%y+)} zPyUq`*UU8>e99O6s6_>5uV@22Agv#k)hSda07+zeGV6Z>`u9{-)H^)Je51`ILvQ98 zMVW|F16pZmVHJG?W!X0kIMcKa0Onq4u!0#3nCgsqun#(g>=5R6&1sKY-RHJvJAL7o zojWVtXXowf>@k(K5}I}M7Mz+2o7Z@KSj~ok9(?F*;)g^R87Nke7lmAvHSnxFQni3% zO($>EPQK(VNay*?%F~5p#6BvF$>GveawYojMc0HfufjK^UR3Lth)}$AcY_0`l6?pjeHl-EF=( zXfyfcR*(bK`UQN@VeS`6{QTb%!jesGac?U`<&qkL!BX8R(?T(&aT%B|-OC-8h$!80 zFi^5jZN7E|GpD%QLtG&mtZDa7+htLp3bE68)Ghd`|?K*)^>vlvN9&Z;# z^gKTATRj<`%S8NjRqb{QXnUhgPgC*QwU)B2(-T9+1DHW~z8*i(?q^;GTRpL{0Z|le zMuBcWaM4_n_wBf_9S?)Z9paPNJUh=WZC2LL&q0OzkM;MmI;)V{fyLF;&n}W> zjyBJR^7iMim-ZaHZQxn|Y zWal$Ed8xE^x8Bgl>3vEN^_IB(sq5=0L!p#!+4PrQ7jtWX!Bo z&*dqDQS<9zRg=I}?PDe{a2VwR6wA!YCiJn0Yq-@qpBD@USKh%#!wME={gR`boZ0P- z;u%-BbD&b0ieko^_UtI2*;Mc~DM!pBIGcJBi6{7d|SBT3~nhfz22-*y!0z+!8- z0U`W>OwMA~{g$GOI$Naz?MDD$>i+ZQbIa^&g>q*Ji`+gvM(ysH~0)}rt_jor-oPOI)M}?$Wbi~x>yqa+oRh~L+{tuot~2|L4Mz-0@6uE z;PR9I!_3T0D5+1X%ygAj?cU~f_pUj~U6_!y_dU!yFv+i{@*2BETxD!D5z;3tiJs190eZ9xY`=f6C zKXrN-q){V_$DrD4-3c!jJMuu49+9?Os-4!=YNix4&jO&h}m1rZ9JRE#hS51>2lwvisdQ7+a zd}ZT7)&Z;oTWA>Z(Bih;BQHb^Xb`|f>8=G#Aifuly%n%h3uvQLayqicfe{$%5Ld^j z4dvB-&Q3*_&lq)7Mn^B3&vUG~CaPBnn3Gx$;MA!6vi|J_Jn`q_ceytBr=POsdvi0$ zkz*+M)ku3=o7YQB=u_+b-qc>;idfoDuaUCmda^TEE!_TRx55sCvOkcY`@;kjK=cgz zO(`l?AqE*;?zv|KWKFYNrCljayAL0!?)x0*-(^eMq|_Ao2GHN!4fOox4O}g{Kjtj4 zbMxwF)OEGqtk?*+McB^#n1ItDHOX{%UZ0As1c3b;2A?P9p%n}Xn_WOcBwRqmGfreV^6*OE}O zi)kiNKgk7hRuY&b8_G!Lnyk3x=xPMLD~6)1IYaFF)sEP|~p7jgu(rcrEq(ud0R zeXbQ-I4hDotu*nC^T~_XH>TzvO^3({jZE?P`+e0-O^Pv5ii5mX!G&ZLe#=#c&vu`a z-QYX%=>(*2lXgDykGbu}ljWmtA+BA%XPwOkuB9mQ+-7I6l5%S#d2hu-YGu2ycG(`+ z3l5>GU?v9(Jh2G3#R`Pmd=3pZgR+z1cf-srE#(;OP7Jy^TW+BuUDau#53_1AHdvsDc7_m`CJkHyK1W>5!s zkB9WWb)4h9K7~o}`W|d>=LoHFczx!N{MP0b0YY|~ix#+-o zWZSA^g{Ihdw;3NBCq~r=zpm5i;VfVJFJ zjc6GEjMQF#xg$56KB7F5)|!sQjbQ}y`eYPC$5qg{IkHIi98W-^JWeVXuQV^wRk@I^ z1gEp&$G6@kq$$$MSBkO{atrV0F>7@48o2g|(9O~C!uks|JK@#jjIx}zSEXJN z7F^w{wa0^wCQJ3}n;J7Fw+LV_Z92}hc^juChljn(6sW`*0u$0`RTBqV;hnTLnR zAv+iu9{TC&IS3ND+Il|oyzcP4nm9~5y{*`9Xs9F*_FW9JEkl>UdL0A+)G$t9H7j+f zXFxyKIrnPG>2N%uOd99YY7v4Tr+mi#f4ibVI!LgkYQycVt!&{gKu{?NhXq+xf?QaN z0^{AK2kOTJ%HnpQSsgXCRDg509ijEC%F-0^pq{_GV$!5Aq(dJb91wR?9b7DSt$XXn zo1#Fjnfdqitd{00i;VQG zyX#`xI0uY%G|!)}xWuUcLeE{Nnbz4WxtqBf6{Dtpfnv?&As|wNjHQwMZfnDZeeO;T zKUSYzE?2!zJ)RfkQo6o|dNZr}ueL1yE)_2v61ob^Dsp!^Z>}v`X*eU9RT*?WOpS`0 zIoocfXJy`Tp%C-mP8K2lw&D3_-NUtGs(L=fT&);`RO7H*tzG0j{A|wAda*yfw{i6&`yF&J>jEG zZRb$`|G$#~v>?*iVsTbBw#Gm1x-Tmx4`hxN2(tGJ9OX)-dP_ooFU8%=Fs!d&f3REW z*NE0w)4x1wGL?mo;BicMGU%#KX56%_mGRiAkcs+5++boP$EJow!GRSA7eF1UIz*I` zj0y}PFD6!t^`{5r;)cPcY~sKe4YKk}1?o8)-`6+1^{Oc0ln5JUB&D<%#hQs-sZ>{` z)Bfj${_9GW5K@KO#u!e~M$`_0qFDNWTmS}ha)yq)WYchg03RrSSxlt&e%tMM!;1*g z;`kMtP)Uh~@l2~4cdLGpvZ;2SERZK?=$F)`@XY$PH`}#q@7;1>_wC*Es^rOIH}782JrB)-+4f{^bR-DgKz?ib7v%1aJ_fE3oh^1n1~ma^YS(9zIWeu4FsQFL1y z&sB0h>wZW_9J-s&*f;)bm}8%)Xe=IPrA3jn4T6JVMZ#mnX@xnIy&sh}#Q)FYA**&r z?a}&7ly&%Q&!Kqs6&ru>=@?(>ceuLMF6Xq9;qCOWA$;9Gfh{0ftH1;6a?>U zLm(~KVZffN-ZxdsN%ieG@E&$eeq)@Iv@>dd?2x?Vr=pXu>7BFk^Cx zIWQxw#Fju*JD1JwB|TJVRL33@a|??Ekz2JZ#V`jz&%d zI@qN&##k;0E`9J98an+g_>1W=EV3$R9NPc{1x1)KoALd49dQWVk9iLH8&imq>S|vi z!|gsQoywLeQuH|RK=}zbKXk34s~{*zX#d7zf_|~=2LEAv< zSc;Dj4t{sKkZnYzdEb9dQESG5^ z9}YTC93cl!vUj8Eq>!v4fO9cZxPR$&i#m)@%>9+)Meyr4*MO*KKT0DiIALNT4>bh3 zcpgr6qc+o?E;VO#TFtW7q*)wT2p0a&e%zWBXIcYc$<^6$+;48I!BR0u((!!uMy}i$ zU7J3Jg}mMjJ#@2JC5mO)m9hmGr6A|CwzJ>b;)xS_;;{F2wx1b4Ta;DhiWEQ@IqKHO zF_ld@S1pe7Qf6pNXA*Oi*RTJbYy`+7x@xDhNucCZ0F|r6s?$KW6l0Asr7iN((=P4Zmo&FI-$tL` zMg7qd`np20G&t;-gq-y^9ayp(Qa{yeYbcf|l0&duqQ4l^>$2}7qHDvsYxkE_>}yix zdX^BV#<5IbiJ`5G6+^;4xd?jAZ24#gLNH=X);5-F3;;`K^L${@O3vu0%qRt}OAbYomDS4!AB(dps%QuASA43o#E z?ku3OS?-67nz==<88OqznaTxY34YA{k33U&9+myJyY9oUvEABmIEOs6I`nV!^kI>r z!jbwD*zHscQ+-M3Q?&foSw81#cVWYR@V_XM>53 zM|p-9N)lL~<1AMcqLq{gZPSN~3v0XSzAfiM4r&%3=p%X@8cR3?o3<9dW7!eJ1)bkM%Z zH%Jt9;Q8Hm8A*71pX#y0bQId;*hQK!@QR!AK0Q5mxn499BP}np3-Zs;&pU(M{o{9y zMWBTof@`XE{6!=q;h2b-lAr(~0FL?~@kH2S9{G?k=n*OG+Nc9pb3B;DuNdaV1@8as z(ewWXcZijt;?Q1^g{@%XFH)+a)j<9h!l~c&l(d4bTHuy(IG$!|DJju%OCiKeT>4^y zfq;*}ET|cQw+wOpcCTBQ9~4ncQwZGYA7yP zt7g}nYnHon#(vDmUi~L1B4bxYtpd<7p(YgSkJoc}hMN1JH%Y#DPg%0n4`m1?zDPXr zg^w&0-$Rss8u6_!SWNv}Qol#sP7lSz?wCt#c|J$#2*yD$Me1^zC^NFN>QBnA%YxW2 z^)lzhaLV`3SHsWOFCgjXz7YY8bw9X~5u*QqDr?uZ&&|5wzljQZsCqiVxL`_IFKDe9 zCbH3UI3&0MpWO_bcyb5*2z+ux2s+#d>BJWKd~_ZoXa!bTFFzm|PB>aBGTyu!Di;%^ z1XJ>@HYXIqUL?3y0tQqurI;inQ@%*y4y0rxSQ9z5>D~}5831O&FF?t?{hMJA5{?S= zdV^G2>3^6&*{USpHKg&WMygR28hC`Z@j(kDt+qmdBcZR?V%lZ3 zIe%^}45YGfb7b^A5}@^<^~*-FjW9~`Za|2+riMk_(eMX+fCv zQ0ZV_hsTN3gDLy@4TuC;L(-ine+Y>iC9^9?vAJaY&2@z|_CpTKLtmR=9}r`rKr&AK zFa$6qjw|${sMack7|&7kVIUTH(66Sx_eB6?($WMGKvtl?wB={o;fLt?s=?y%MEogh zF^t)WOdHS`;oMcN|Gka6@0G<_q)9VHy2+Q>Od*CCyJepK{DDtemieq~+x|ng{%UHB;^LTD~E@eEoT2TnkOk9p=FFxpXdtO0tp@RTL8HnUP_F z^|v#8#4l(e!>;S~sk9*E!tm>}f8mY`#KL>^{Q`E6w`SXaoXk>g3wN9ge}?*Yzu$~z zc|J98bH$y2k`(6uDY?41i0#P9uRfhip%4kwbQjhB{#ElFsU++%vq&gchb$UK= zF8&*m44wA59Sz*--T7v-QnPFKb-2CAqRS@4_Bi?K$YZ-rZfvIQdq3XoG2?#ypm{8i z|2Aai3miA}0uK9jEc-}tbT&OVQo=qrHgURDkq{CKxV}YRZG9uXJ^3$!qJ`0WR`cr^ z&PbHYqoz==Hs5CW>5@4bS(2Z!zkj)E+euHjS`1BfL`fMbUlTtl?<|ZS;RCuch92R} zk*h8>Fz?2U$pkM13Q&l+kCaFNp;-JEL_4;>x=2~Auo!baVGOz#afp+rLA~7GT$?*M zs%u{7_wWck#|UrU@+z;cz8<=qGMi+!BU<$YQEfxL`_pTzSML(4NzYq-@29eK36fRL z4#Ize5(@*xRb^*u`(Z;QL#~~XNrgSUY;-YuaZP?s@ZhpMJfj}-@c|< zPkEaGL$J#=_4#~7af7VQbZHHRXO}>fe#L3|tquoIFE5oS9F1M!PCt!G9p{S|bR2R# z>X0xJmq#RIWSh;7-E&7=B4qQeR%bbZ_ztgn;ADbb6IT)08c$n$d;7iyY~**JkH=qr&2p3(_T1AC2?#FAC$YbQW5*I3N0kD@Ycg`+6_)TuGdK})Y$&mW$V@M zy0~r2MZks*RSh8(-X5~H=)rQ|_+S^EXsXX12Dh*) zI;se4RSykoWl_IWj0$PL2~1$0))ZRmn{XDmmn{nXGglqlV!TFL=Eb|s?2i*!ok~x{ z;fBKbZ6klRUC)>>8Wp2JV9D8sv>o6Q6pSqnxyEW`0<9E29Q-`x=gsJIHhG=3(S5j~ zY3TH*RBh1gF?g%rdq=hsBeF#@|8+FSqdA@9QfkY2*=*CZ1%w5WJW-KQ{0TKHH(7-Q z3hPf52>(B#z5=MNF4{T-_uvjeic^BSYq3(GNTIlEaVrqqy|@=I?(R_B-3t`AV#VQ4 zzxTg+Co`GMxG$CMQp{}Aykh}#*dGS7q(ZTt4+hU)U@k!R!7*4bq09DH#M-d$EcJVj6ogXv)V$PlqudBC^7O z*3Zjn!|&f_EAup+F1PtvU$&onx9An+wMcMOXm6yCn?|He&CH~$z}h`XS?Md_Nu)7I zay&h4YvI{Q|44H7Bh)d31d^i%@GsA?7-*3zozf69K+tXl8zKJQ(%XzeBH zSIu(CJyf-7L%@~*ZP6UfT_4LuaPBZLiZh#r^OxlJ=#iL_-YdedWg3R88WE)T6=t6* znLe`_$0&slb`<0`7x$Vj(;BL>F9Ra?y#`N2Bw*Z@)-r$u7V zr+(QA7A~Lb@_w0^qEX?&b4nCh;O6&Z3w`^kjtP^xYx#PXcvajrq^hoE@LEY>6U0nW zmP(hAO+rF4LY=hxjSG@Ke6P!EM@~W-w%QJpn5~S=i3Ei!DMf8``E3#%Yu--1{?&Rt zS&9$*{26<@^ugy5J6q(rXS{5)V45={+xN!c>FZ8wgA`xe>w4Ftm`TDLRcgxv_rA5a zuK3z3dwy@W($cP#yc*4PY=oDqz0Y4F_eEQMo)t73zP{Y=>-ya!-q>hiuT#l6zP1Q3oVSTX9P%|-XeX! z$l8ebuijx1ubA@oP2#X}1}}BJOCPftELF1|`1E?!a8lH}xMAJ|=&cYd;< zuF;N6TXPI{aZj$?*bE)S#Kb&ed$|~$;?|YlZg_?US8m##drZwvO%WgMt}HDD#kU`W z5|iW?TwOP=om-}-FD52JhtU6=Jo?%=t2;|^CJxL0${t%=nSo4gUX2_PvJkr00xofD zXlys06bu}yj$5m7CDuUGouK%*A~nlcnh-;@V~&cB_;{Q_N8F269R{sV6m(f@X=yxk z|KN9EkkmImc9h=WpotH3pI|-=fyesGYB@>|Ew&tQ%nrBw0R=eyD4;Mx#tK&pt<485 zUd$+^^3NU2B3vRuBA6_}Um^HuS;H7u(dCxqT7V&_Qf@0}Pv2+VEpg%IIpI zcAI}X+iO)s9J3617^2zDcVUMxOeWkuh;e^i0z(Yd$K`1jB6bh*#g8LE92`yQ{-5yR z-T^{+Zg}v=t>1*E^;@c};oo=V2Ihm2KWF>ap2**LXp8^(z=9R|4Xr?+bAR5>)$8v> zg^lQLRpp$y=NDueAPEg;(%Vl18L3U1rKQ}SzNp652yj7{p=gbg@sxBl#-a+3lCnKy z2gfge0{ysEYfNXeKd){+U&K0BR|=<=Tu{E#k)HW;TTn_(61m*b(eaOY+uJ~!N=l82 zJLV*qb$oWV-0FHYfNrt!iA)XEwX4j)5DzC3E()agp3?2MqqFc0PTfxQ&9Ld^I@QZv z=y~V9I=Og2?afUz8Vv--Rkznw=>`mKu($aa|G;r@HnsI}v!^k^>{pi?TdP>_I$dE>}4Uu5XDlRNyHn3J+bIu>OmgkupyqEQJ z$VEn_QLZL{T^I&lPIr+ko2M3b2}BE<~?o~M_=!{Ju%i8ae>3NJ=>71i+) zI~hxsdKUF5NXR;zb{}ADW%u(H&zGl`z1@mY-DID?hsCfTfVZlQ>8izUkeYt;?}%UB zom>VvS&sJ&&yQSqQW6H)y15h-fb$y;GZQ%knaz&_#yDcEh@WB9a#|;bE36AroDbbsR1(Ixn z*}aM0NYMJ`!Hw((OXx$HM-B%zwjgLeotaap;MWa)YBmmAFoYNKsZHd{QU z$da0-h>%o?@{=mq>S*D!K|>Q6Jj4{uYD`OB)UHG78o^)U$N&%;(qJBjK7^j52qYpS z+TdSfh{|Mp3chgAiY2kvNWTOMs;B2rs)$;H<#wLMa~6-z{vYQ_)wqqC6?%D3*F%4D z*$KlU*}mR|`e7Vn?xBsVI+qOCA|`uCT=(?;Sk*=`o}0f5Kw*h%aGFQkEe@4K;yEGx zU=^7(ywc&GLwVfpeVh=h{iV7nTshb+fA22x1B5w&ZG8UmO7e>UqO#5lF{s^d%iHEV zxfBl0Jc>U6*F(kL)U=;Z;Y!2ryw7#iuN5Xv;XvtY=I@ubbMi=H!FKST7+V;eI$q z{zDf{>4?g@O*OQhfe(uN2|1wP$a@zU7!8y6#&iuAGT8ytFXZ@MSQ#w(#c@v{@^S2W^nmdvo7G13o0}aIlTXJU zy&(sBZxdD8ZqHmRZIv4v8jMz=2P(==DPnfUTy870JMt>*?*E8mM>%pHD*e2^zV4_s z_8!}omdZENxfMJUjNv-X$m%4nK7VYqTU=bkj#l{%UxZXC;_q6_i)4xb6LPx<@~Lo< zIo5pZaGaL5&@Yn_^Jg48$k;ram~8W9gvEWuf!&~HrFy1+{3aU2o&l>6WLsH%7UdR3 z4BHphR&+PVzMd>`gq#Q3ueD}5J0~+A z$|=C(!AazeS=Q@4!U4$3pGs1BsK124rO@PQE>vMbetE$%828N$tzrh*LHc4kP+B+^ zg&@KXO)@bt>EYKU8*S9-JXn-XWg3!}l`ZJ=P=CC8`yzwT)99X5@=~usZ7U9}rV#u3 zW%FfyRX3#}tSS^a^sRxgbMa1g$Q#onZuy21(N8W1j9wLU{Ihp>cz9>m^{A{0C=w+Z z+r4~rr*+~aEEi|7zQSvchh;r)jd%g|sGm^SQz8w}vbh``m_|avP~wYm&tW{F@Wn>$ zg3r;1%P#M=_IDlLVqwSDR@0;KS6>;`qBY8zp7eW*y<8X$7M5`W>QoJ{=h#f{D5L+nseVENjpFs?1uLn~-?Jb)9q{YEu z7oY>tpuXq|veFc2jMWxfaD6&<;hmjn%^U`%b;t63NiMaG3WJ81F6&J!H#_b;|8?ue zPr&-Pm&x~&F2NOmqWayAzvDBuY;mmJrnARKUB%mujF^2~Sk_BzE{ z7pR6f==5AdSr=CKN0_&Q$OJ)wUGb@EYb`%^ZB4Op|4~&rRcp>}d|uz@JYB$eYzDsf zh3Ai3IU~vg^16!k){&Y<$bJQFa*e01i?v^!+! zN15WneAn%SP>3x`baeFV>q*DGvxZjkdE3GUgWNbSZZ0YZI)bFUWY{n?lsRQ| zY4_+yWlfob+#1=f*g@BC+4kn9o%PPoAD{KzyDIHhJ-WvNK|ty_fPc`Q=g2oTf&JmS zMK7_;F%4^DPFdod_x$8ESHDFcGs*vM*!V5FyWU=(mX^+&Oo!eI(QW}aGf3q%Wx2nv zQoL<~PpTvvltTc_s7f0g7M;T_!a$}=hOfr`B55rK9GI4WjtaC8cQtta}S#s4Q zQpFBs#fz*8?x1kvx4USr{AP?VyUiLCOv9jHP2aJi!Dk??-yq*HU6X@*aBq=AvY|9s%-YUjMwQ(yR`W{^%LLyY82DK9< zhzC8xwr)g-tSr6+v$@N;Z`&jn9`zI^C`!^&{AX6!4`8m!o>tUU3v5lQ1 zb&z~2q-XDoEvEdtux|y#yAgp=2s!L!;%}B{PQbJl*Df0uQS`yGB9ZniU%85eX69UMX zaM>HvVL*IS3l|V9t%ia2!LZi6RsfN2k|m+;{kR~6Tj0J;?M0^jQ7U@0<+y}S0 zpTDf1zkWG9SvRnuK~F_M#bO2uu_`1&nLyY2c5!mzvFW6IMXhn`q?l-jm8M74Lhf-m zTvK-Tj6e+y~r(-gTl${v>?9O&A?>yUd;Xx z9gGO$2d8UmjV!E+xm+)8h5VG-F-eTc{Mv?qL=Y9!=7scko-#_t%6O#B=nAWN9f>{t z#?A;fE|BoTr6;mQ+MX^PIAFmU0U9z*>xXLJe0J`r@OA4c*LABp^f|ycX{QytPVFdY zPA(tJ4&Gn&PWEbEn;Q#&(ho+V2n_7&qYdaCEeDyy6U@eCVjbQ}r$Kmj49#VS!9k#1f{b=9QO@ru2;Gi> z;@06eRlfoP0&Wn`uzj7FW7J4as#l}5H6W-(!#Y)B_{{9=r)Ot6F781HDulfHyu

_{mw|TBl5)wvtnBj<0Q58R%~o+uoq9qN1Z%HW?>OUN9-epOFz$kJZ-48Tk^hn zi_Vg)oSb)S8#9YLfjukl^-_KtUwTGHz5*ABES92d<#Sp}QYoCWO4M5OhA;%u5AHU9 z-DUSx$$sIo=#nrrlcgacSaSb3-|qdSY92yMh$;%p_kHh~$y$Mu{ef~Df^K(!Kw+Q1 z3Ri*t22Yn&L$iKo8CBn0xJDix(nB87Pfpfa(n>S5*hSAyY3Bd~(>ro3EbhO}OfAJX z(!yc=%@`O_T&x~{(aZ@@RiZIuf^@0UT3}tGFWw9rqMl!!j9okymKus5)|+BX%~wpe zrurrx7pwGkJ%s0Tnqs&?k+b`68|vN9f2=%Cg!k##%hBO_UjOo)O)C395DQ-edXv%k zyzcjSu7fz>?Cji>!7k$QM;y5rccoHfo zN+MBYkOQwl+xmhL0Bu~etLsOlxr3WZrMoqZH^T*!K6G>;v8t@uv?1jFG-i+~YA0&x z8y%(t&K7h<8@uMK{Z36t_%6;roMayuKfb#7yoK!tFq5W<55Ji9d%W#}@k$k3*4vzg zCJRp;_!m;+abgifxJ>fyt4)*3jZ2v3x~Fs<^w?jAbsw=QNuMs&Y!vHoP|1b04V(`U zh&f)@evd^hVwiH(0cN5s71h;0EC`+ZUD_1NeX?I8Cnw?2b`?Idv*ZWm+^R&!f-S0B zB&1arm*R3V_H%BsACG2{*L@JMN~6e>wW8&7J1Z;CriPm-Pfw3dj=aX6T3KJlqnQz4 z5_YB)t1r{+esTy`aIZr{sN;q(W`CN<_~oaX~S&sJf2&>78eclaUgN);*BwiqfkQ$V1Q*VND?Xu0X6Uk z9){2tvaUC8&_{{WS%>x8dwu_b4i@K(g!6~{)*{jYq1A0^42Psxm*R5d=2fFpBY*;W zf{NaZST8To+u)-~#O~e*`@_W6u$PvrL1yO^XBs~Y9A-HU8?$(VRUHMM?Q_&OC7sIB0 z5wTPPz7&xNFuD1E@BUF~Y;b>LEs8q5u(EftxzuQP z9P9hE<01U==OV0!Ur_XZK+9s==rwufyL-$@PS#DM3&!s6y{^ZzRanxP>3Kc0>mu7n z4b&{*;pXN&+w^=MDLmBK>}h?b%JF)A@Ar^6_!@$*tpmyD;GxE#@^wK=$O8fRvoK;n z@m~uGfrNxrL}f}Y2T%7``;$3M{{96@)RaI73TnhPOnYhZtzIjQ#5-O(G6YieL9y4v zSgaTT%J9!Wzq()Vx?1B*5#Nq`z1~f}(ugl>BLjln zWM}ewpqqS`zw&U3ziBQ1`SVN5AWMRNoK*+eEf}~AhNYag9gbH;hElb8{+!}Jcl`Ze zI7hWJigHsC&vPYh0Qh-3*r77>b!6_E8V|?;43q^Ua4|jq%@S%lUz?Ukg&^Psf`}bD z-R{lt6g5$h=-W#WxZpWSF31)$KBd1^gg`#L#=_Zotu=qGXgM4q5FZ(3yF0CVVaA;e zN9?L)rACL1$i)PPe*R>Bu1d>{JEdZPc}U*YbhAW~F5h|OTAs$*aaKll+Br;MvvS4p z^7rr;{XPog`Aw+_?aEU`)rJ0AZ%JgdX3tocBDUk z4wT?P0rgHr$F!Y3mbqo~I|kqYUsUz91Uy?F|7xwCexk!aJ35MBndM9Rw*B(r|6{$) z{d8@5b@jExy&1N|&2MAy7~Y5U`g>Ej!^c}Hm`$3xh#mb{ZrhO)dwIuCesXw-6)dTe zk|hD5Z_3dDh-=Csk%eqhAy6l6FY6en_+}3ZS*?6S0c^r<<6_$SHTN$&MsV8dx=mef zRDr5W?yG011Qv-+o7YnK7zL=FFmcDYh=CjbydILta^r)ev-P^Fhv!>A{MevYO1B`B zR?x)J^P_?yuYx8rHpK$%w4zp}+%-R-T-V@wE0l_crRV3t(%k9edU{-RXlz`>{d4Cd z#jSB`y!^~N8gLT&cu^94Aeb7j@B<`m)j!=BAD}eybr#js3o3=u(p6O1Xx;??#x!kM zXQW;W-*DZdNm8Rzl32wF^&Pskc^4Ip{iZ>H3m`z3Y+E(6S!#RU$|~d2wI@D+WnGg) ztEvWCT--}kaP;4ECxt|X@j<>BLbnW|WvyrH7cj-HyWf>rX2+un=Qm2Ui5jgA8?&36 zRZ|;0M~^q`Vxp?B&vFv;ME;^-G&Cgyj3`kWrRQ8BnG*n_z0LWwY?(HyB^o!mOj2s% z{1&+I@G4Wk)R@zUz|jbp+9j3tzzskxc8Va}&35DB-e5l$)?fA|$N?o>r$tr|ou8%qJut_y?IL&z6xoqdPuwy8!W@GoC#!kW`CP#_;kt1QnR0Y|8Ts4w?CI%Q zTI_t#_WGwCE%~XFVFO2;i^qJ68qm^@-_yH|{Bzq&yQl{OU&nnV@4sBGm7|G|o+fQf z43A`#OOX(xQ#2_6glY3iLcs_K3*vC>g1$yOK!JHMkocE)fBScuaeC`p4YM%q!9Y1d zLA(i4LohL}zcz?&rRPmF7$^@<2L*+X=#lo#R3vC&aH1$NDsrSPF+x7@L=ies!X<=I zBJ`HG1a{-%5z5oBMaN6f>GuN4QDmI*r5W)vn(XUVPvh@(Qa;a}ZQ{k=4;Bd{t(2>V z1qTNYU`Nj@FbY4S;~kgp%2gjKRwp=8SA$KQTs*8@`0bV%zfW6UMQ4=Cp~y(b>Nk53 z;F<`MO%Ls&6Vc)2f(;Q+?)G)<*PsP_3;>n)%Gr;5HVDq7RRZJ*8c#3ljm!5XY4zw} z9L9s3jtwXEFUAQ{(3(`otr;_bc%w{T-W5?L04k1y^^oDPfvYs!nnMFZi}7>DBBU1& z06|evWJ5yaLP8V9M{+k)+z26A%HuZiwn2fNWPG@jxIid|Oy54c0oST&k*=LPAWTB9 z!)YMXk-AJSiz0ypP|3SagJ3!wVc*b*0}Qb3@I1xhq7xkzF@EDPn&=SKx zOJ%J;D{HZj{EjzmZKMqf@J~KoOq@Xn#wq=HcC&dr4HB^*Ni{L`u=Ma?e3l5lTuDMV z&oHOfKw_)P{bl&<%Y;YC1dMv|_n$b^??Lf8dDvx1KozIOM?o=!Ni!PQaZbi|=SGn& zR=%SLkp@BtMx^IvoZxX+m(V|d(ww3jH-Itz+f(0i%^G)k0n@UQG7cB$LmuIv5Qs2&MtVsBV%G#q8+KSY}$*v769fXb88aBT|-! zrR{gGN&U8U`vVSjZ{xp4qNdU=6D~p$0}_HW4w56M^W|cuMc=8NWCr%~Fe|K2bm|0rV z*4Ad^-vfE&R#8FCix|AUz5Tq760=W5+|SlXsHljPbaZrJ#o_Y~pUdaO!TbXR08z+i z5&?(Q4`IU*{(Zr~WKE~2-~ojKKfYun97x51{Uy`Y-ET2Gyl{;phaWtWHIlAL8i7$@ zI{EtzZ?VcH4Nj4nt9y%A9E{^2Qn;btARgibSjh|AxEx#a+jHHPi^{0C_K(yQL|xkf5fL5ozF@uXe`9aSQl( zP(z7SQpa(&P-!kEgQ08EZ;@Y>zTBT!9_q}nXuM5-rHk3q7`4KTG^O2pH$y{YwXgg) znMf=Wp_l8JD~kpJ-ca&QL@ss2wHv6gliiw9kQNDjK4p)wq#l%O*po`rkw?nr%gD&! z#={X4QPA@nX{Wz5vSaI~4!jcML&brCV+liy$rMV`=7oW#kE{tQjbTG50>_tP{5L(O zx6hMrKVul~#cqheg+CS6XS97vvMtaELF5G27}h{$WmQ}W0Ld2GARKu^e58o>%qVhq z^VlXXKC$yabt)w{cVD+dNpaKm+X+X2Q3Nhw=`?`r3Vn=Ltm7*+6t)<=je}Pb_ptKB zCAquf)#I*)-@pHRbNjTvtB5rFcuv7nc*0!i=k}11a9P_L64LeZ?04Aid2yL^oIp@| zS2#)(^5|{cD+5L-`NFRRHqb`7T`}&AHJ4SQfnz&m(rXZogKdKmE)|SQcXRe2zQ_gjD%jA_Xc+gI%7bc*&}c;|%6$C>S;1yMpG zz!~6^94Wtvg@X}RqD>v9JHBYoV;}{y0R{j-xk%x+cLYL1pI{e&cDwbA-1oUncHYVN z4%UanDZHv|ju>#z`Jd?w>^54~gXDO#Y7E5CG$;<-a#+aDKD$1FxT)+@3KQatu6~$8 zx$VIH_Kr!dkB21Hm*oqV>n+bcND&u0E1H@m4GC2HGG^ye{q6C-vyUi9+xQ0o*G4>= zR(LQnQI_b<91Skac0N-?$E|9?d=fd$MPy}l)!TA$!nJTWuVy)@1`#ECr^nL)@`E?o z;qWL2znnaQH;M56Yp48}VI3fj#2%r>IO}uW40er%c|1Oy(2q+@8PP6wT&+W`OifWF zC#NeO<+k}5;o}=bJApja?kwB!u*AMYb9|H#!=#}YH~=(vlPFv|VwCUp<^~R`{HrJ- z+P_8MbL`t=EzYO}UJJ9j4Qd5|xN5p=H1aVh_xQtmw;p=s6^C`p?5r%z6{p^ff8bLWlsUn`1t9b>5H9cyH3F`6gl&H+};1uZV8m!ln(|t$5*Xwd|&_&tY39iv+*orGQ*`7FSXDF2Qd6gTbiAX?JX;7<}ETt za04>7So0>1IooJyDGCP{*~ret$ws5i>YrTBZHdpq*=T9SQvH_vPPg`htD&U8X&5WS zVsYX$gOAYn<({ZN!mPF|DzeWCJ)TXnzNIdLrIB%KY=o_3rOw;r!&Y#e!FzA1x4POK zbG*DoFje_5%a!Bh#DiFzIY(ZQDh4tsVGQ^8ZxNTK`$WkfZQrETz>2iWGWV|+ImOFi z_s=iHOMj^ZeP1`=AyG;x>qN0k_&P0~x-1G7=0)N|n8%kcpJNjvr2AlMtwPJHWL(?N zQhmJ2rh<0ZRBneDM-BJ(eu`u4KMVz*my1h*t>K)lG&qxln!^3P#yIS+$2r8=DW?Jj z=(JjXYybc*Elpv<3Ls&**Gh8a<6r%SWJ~V!6y0Ap<4lFpGolz^91&!~Z`a!IoF|I8 z)d@d2J7Od8d1eUawN6BOGD?$(Tq8``Jma%s%Et9KEOxbhWZWA&|EVt zOtKqqTe!Iz_~{M7n_*@y7Su4P^#zk!4jSV7N^xmG*(AVVhSlCqgE|1NI}G8;?^8@j6TWlu%CgFh{KT*jsn77fxV(n0`7Nc8<2#t#ZVL`3&t~hY(Da)$7Z4JADt$dbu1e_M;H0-O$BFP5g6GhVRQ_7nin@a+KfS!$VklLd_|>amcmL zP9gHO+UwjG;v&My2jNBoyiF;qwD&WTpycu@rtcJWe~ZrpDuvanV(zf#G#9CaO%J^M z-tYHS4x*1F@}a~RkrA2C;OWSg5)y8b>4*9h?+=i_6c-io+AdW)?#OyH_%AM+)=l)n z#x{>Xs+(=q*R2$o@xpZYg|zznh=Q<1IgdNRsK6>E@(ZB#t;1~7fSDs6E8jOA7EDn} zzx_TQ0mpR8pR9N&fB+pUM*M0WGdai`sSr~_Zt6IwEum^uJ_W$G93J97UO#97Kwv<> zDeDs5&BzgXyt%KhcY_1Zjj++ZB$q(P6hsUKv(sZUL?kq$)ci5y-W-RemI!qt$afcs ziNQe?G-Sc%!ItWbu;%;I0(gMGYDroSI;x}Trv&dwFfJM?1WK$5OfVh52>Aj5a-u@$ z7}7aorIC=}#+m?V z*xtqP!=M9};&%L@eP>xa6r-<%akGBLnVXXvALVXcC+$6dz9AY?;92&^y(}Jh;bq8~NtY_ zAxnH2DIb*gtXP7z8U*57lL^?ZhiPp@KYAb*8=TS?@Itl7es`H|X76rv_H)WkPS|{8 zX0N$Zf~`6qf!4d^_|j>xS`iWgOhmIY7y;1F;wLcMah$y(|_Fn9rsiwEgN4oUeG)77Nu~Dv!CDK!Y^7JO$N|D z*4aOcp5HaEY)v|D?01HiHMaa(s9Cuf%~aOs4R@}%q!c+vdb*-`{S+*zk1G%OQ}uf3 z0O)RZN+S6Ow84-z zZiFcD??hNPq|5EBA{Vn3&$ve9qc%T+9LKZ#4MZ%%EB`yu|Bm_hG^368aVVhCIQaQ5 zfd=m3MS4;`UC?`5F3+DtOz5>GVhEA*c<L!P{~NGvQs^Vf*60Ux+V3B+=l@OI{|#f*%@$_UH?NTdbI4&sV_}Rl&e~(hFcn+| z89M}aJ7>qYW24hT<+&>!gdIN>L}|yyfN46w^jB_pzC9n0&c8X~^Db1$ln90dDanQB z15KI-|3b9_lb2q){J;HUVNP~!u)pBHaRI;Os|;++pd z!Cr7!d%}A;K}EhlaQcNCM)w4TVzqZp9)*?k+D@nFS0-1>4vT{yRhoKB{lZcgJ1s-q zmTUWG0{`U^rbA!Mg*^IQEzA*qW9+43s|C%P> z^;Zj|rfhKKC&zdC*n4U7nl~5%n_hj?G24)~MVNj<0TMFu&c)p)_mTf*WE*jKe4LG$ znUsX2W^bCXO3Vg?_D<_Pt4_dMov%g7`R}nrK#53bfUwz=YxZF(Qyr>gcY0Tupa4~x zMDReEf^{evnWfadf1GgC{CwO3jS=)3mtJ`iWzFzR=q zL>d+8D6NDhc|Yu4*=Dh&wYckqiGgtt0q|=U8~ASlZMvK9KO@QnFG#O!d-ss4v>+I5 zi;36SV^GH-!1q9Ajj|uuIHj=3A{BY@3FO1jzuvbZ5za)Zf)$+>bDp zen!sAZwB1>!Vcrn#@=>4R7WLd1rUz7JkzoD-y~bse#Abl9!6rBOBbXsJN5HqKVRw4R9vIpa`C|(gtxgF8SomWY0=AmC z^}X!5-f28el;WVC8|ld+y2hl&AKdh9GdFMzQKl&qqM%0pUoL<+Be$ra%i}{sX_^%< zRI+A?N*V%A;NasceteEdgChwB^acS1%_gZ~e4Ro`zM+7iyWjgcwyP~t5F1-daw!P0 zv~cCio!<+$2$!NHB#cO-ts1)f0}uZw?QDY#6BhtRM*`!+sxT37C=w7LgqjA3OWS=o zDVhS+Y@mQ`jVTzkT^JWR|LsoyHmJHfbA4(w?$Fu$x*Q+X#+s5V2|4Fk9%^Yv$C~vs z1VSjQDh@c`G!BK8KQGD<0TPJ57eRm!NDn~7QzH-7DDl_PCqmT*B$&c$GphRo5zP6) zOvmzx32`u_1#JxlFf=ienQ`^9zg#CNuDQ_~=*rXeRF?C6Kk1i^#+5I=fCvQ&QKLXm zy1}Q%4in89+PU(GPM~-k)L;M*43NYRK$Vc_MMX@#A4y{2TRj7aE6-1?n;{ZS_)MB*V+ec}v}clts=)ZDop038%C<}Eogw&34?ifao}2qvYK zr$L~drjOuT;&2IP2(xlQ53i0-BSwq@^I!1lz*a^m1nBgs+wqVc8y*)0j*dD817ol1 z0WSjq!EhMi9Xms0icD=(hsWD>a`pR(h%G$lf2s6K=VuE3m=c!|t%UPg1gkr@aS03Lrz zh;%F)%ex-?*dHFf*+1KJs5aLvKA%$gZa$444z6Q2oo}prTrah*WopxXt-JrEsEbTZ zyWK-W#`OYuuC`)fyN} zUKu?=)8x>^^}#7Ri(3OIG4rvbX~yb~a@`@;pF&*m=md~_69I=A2TY2_>@kHtlSneK z)L*UFN?Kdc^j9C06sPHmZ9(6>PwU!0yoj8yToF$Akjg11uL9STiCBvZjWU$ohl3^x zwk17^ccXyo6SgregBX4eA}%pBAmF{eL4HFBC@h6z%hlMYK6vuLIt>~LhJga!6^q0L47n`ze3Z7r8`Za(qc{Ik8ZTCPzJ`&i{no?+ z8DYh|hy9NIu7?pTcXA=$-**=~72|B<%8QE3BiKBY*;$zASey zCOHIF<+Q{Ah9km5!EtB}@!`W&oe#{UsNa%OD8*)oua%k_+~Fc3#&u1$?k6X*=n<;K z6eQ>7`P#h;{!W}OTm>cy=u~2F2*6k^GWjndA$`c!ZrlbP!Xn2uj4BgwjE)z^XVnwU`4dvUElBpvbhfpI z6t%UTH_%ug%JgXUo4ddK)#?iKg84EYDGt=@S=GY4q@``rPi;3q(4Y<;NHM1r93c5v zFB{UN$nIyZX-cc-aSjr2e4A@J=N&&qd*ft`1WKppO-dmk{A~%`Z|@xJ@KKy--!|p` zaD@3Le`sP{i@K1Sn30Q%(|#X(W^^19r{%kX~L+ zeW5q&EXgj1p*o7!1fh~@*%ZH>g!JAn4*f3Jo}-_YtXL*6IB^lQ12PPT6w%AYBj%|d zK=SEo;x>hL!cjxtuDiL*y1L%SasiB!E*1P~d9T9n_115aQGQ7t`}5=FCj1se zs4nNZmZay`}JDOmed9>{L}tbF$?Pjdv&j zh|z;)3BXjGYe*3#z_WpQf{;*5Tun^X@hTlDVf#^FVf68(9XJ%K+u1Vg$M-5R6;Jg~Z~!smlPx;1j*XI^c$W_* z(e!{BTPnUy1^yaIG{Nt-vNr&9tHD}rMEHe7d4;LoJKucj3^o6kY}7Os_)Ze}&d>Um zz zm>Xl2wzjgOY=RW}MF6==Z{%&n9LxRR_Gz{jUu5N-Id4@Xf9wBk^zL1xQpd=en0%Jm zhx)K-fCQMx<}Csj2%ZM;4md8}CK>&qJo7wZ;XQ&eXRm}GE3U$4lh$o>rpRk>eyOG=TStyHICoHDh->?oSVy=I zpO6H=%sjFwsNGfh#Rr2YG`AEg2VF)&1skcn{V+N*;n<=l71)t5Tt02`_K>TtZr%RY z+kxu!Vz`fE3;-)VW|1r-IGdyIt((MUmma=s3>J$%csktvU1&SaH^E~}efMTuVdD!5 z@eH8}lbRxmjhnrZQNVD|E6h~6Nw<==q~j4L`SpYUbzVyW8-Bn?v`aEtf~KUPRC>Ei zS~5`jPw&@E)f#{m>lg?CAV%PTuM-#7gkhUjyR^gz0n*<9rV3Xu_Z%iUDqIYd0@c}Y z6~Fr=T|`i7P?5NT>bqxVR2pNtJdQvlG-xWKzxI`bAPpyqETMR{NBV%_M0t=rlPo|< zhz-feJR`km5&$TY#z}tsfg-Lvd$Of!=wS(!aOPhR4o39;af|u)M&-RHvCIe<6=z&m zfiVcQyu9pOSJ$ai)gzgg{AE)?-U`D?_I;2kP&~Q2yIeb$=@}t@MiEwpmv3>SR<{;2 z96#^+4{fk6KyEWo2n-tnV}8lA2{d9^QFA@m8iNebgwd$*rs;la?a*I*Iv9(1?7%IE z!9wJf6-VgdKzNgeQqM1lLI*}Qmo4<4;YR`!(8VAdq6$jt!D~h#ymg-9ZpuMq6bd52 zmk5zes|V2yhZK~^k0}+zkl>7k8@p+kD)7=}>=LP>wkZsm=twF-V=xd%^Jf7@f>LPI z0AtcYV!(&=U}O0rx;kmYxQHx{u_!}c!`>IQ9bzbCO2XEJ8t@H)qI*i4 z7>Rq`5*|!M4W*62dP4_9H5ZqJaGT3Qx+TH_baE5$lRyTu3AHw3ELg#;sawHP)0Wcg z%*+YsT$I4LqJZQyxLwI99C^visDnfH&bO_LpI4{;EOx!JXYsM?w>yboF+U@)OytB# zd~xE*`1@6lsZI(PW-%xqCc{qDoq$;=2V+0hM1%Qxd28|@-uu}d?nsq|z0rpj?-%9R zju0Z>tb5br5HW&MKgRhT&6N(HO~C4~Xkk%Q|0<>+TtFSJB$$RBdz+erB;{=m+1YtC zDxW_n7Z^_*=)9$ef)Z6Tl{P8nr}}BdBX_T<>vjE9m#<2%C590nw}6Sf;;kFTJ+#3t2}T=Oi?I zSikx7_m+G8>SocPc2D?WA-pw#doD$dSl*x5O{$!9#P`nY}`?;Ok2o4odzZK(}yk56FF#O`i6quOfWWo~M@1eP7|w3s~mb@F>?2*rPV ztNwhouEEdJ!;AO-(ezaTZM99eKyh~|F2!4{I7NfIyA^G54HS1T?(T)+5qmW=fO%=7#f4#-%$H&LY z4Ig4?ZtN_KhGi4%xAy!5lR!_W7^Z=v#MX7BiPdi#L?Vy&EB2T0B(8}*?~p|2Ip;HX zqel+|{h}P^JFiN{)@A1tTMh83JN+==k(No+EH<@PCpd0JK$@SO`}?aCx5x`Z(=W;h zer>)I$}MD>&|bV*g>m4;&t;9Bk&(zDE^aZQ`MZjMO_Ih``xR|n9i3hGFO6=R@vGUX zK-jRJ_5)I3Pg^7O<P_lnpjGYhFg;YuQ> znyQNBhS8lm?<0{e97mZlD#_JN!4QTKH3OBan#YFsYa1$)s48OiZJZ>Wm=nyKzv?XP zQZ-M&ZKJZlTCz=e_#*e=#AxH&UwL>jlkwP1*rH!$H*opxXYc*_goH#~UWL$QrO)Fr zqFby5O1$~0np=MJ>;>KxoqJ?^UW~d6a~>Xd`8a-S6{dYJr#T`jEzzjQd(&kNo0*Le z#t%P!^TD~?RM*lFB#rU{m1EG+P=~=EF-8z3DC^37 z!oSvErZ_ZI;-d8am+{OHEMpJM)^qBMu3|rih{7&C!i48fjsMk|5 z39_kK2#l0u2_&?&1(JuGyS0&pnMnm(CQJy3aC31B`R>kawr8Dy7Ci%KFrC_&8CWbr zG|N@qi;RdTlwE)j=!nJabw+?;NWFv#@~8-b67+R=V^VP_@f3ny9mlH!56%ZQmGPp` z{h03O(><_6!gWufloS7j6WHI%-PL;fVk}+^`XC0qKJE{Sn(t94fR7mpLHSN8>{eammP3DQ@8-^rA) zlQwk*dt7$ELSYbFugPS>%*TV3ZeBZxwu*J#zvS%Um z_q4`FP1&+=1E`qdh!z$E(hx&*bo6J7&7|F5Nw;DB{UL?6C6;@SW0Y&cR3g!*h;TeS zJjoUzh@^4J2-wJduu~V|5=_%~Tp{Mx+4l1oxeWtuFpIV=a>l^!21fGe=xF9B9zyI; z5}#8V;xw$My6aI?T|Id|w%qB>!{Dl#koolVw6?mcG3?v16t#wV?_#97QRK_j9+96PrInZa-+uhRy2QF>{onWs7(F49z6Ba=_+ z@1kLx?YfH)U(we;32>9**+JL{9k?}Beb-lQe=MDBt{tC$->n2pPZ_sZIqh{__;0xf zfIp$%G)_cqIIGAJ`|E{V0;$lskkA$95mD_mAJ4Pj8?uQ)w{-(-yM1h2?Q_T$YdNBk z5F9A)+n;2&guHkEm|s5pj^q(uV#ySDJ2=#JT;V&u;vS1b^1jNb!DK8Ip_;yM=m25i zu=b-KuYcl$vHSY(ZU>ot$~Vz)e zwf;Q43VIw;`^YDHu%%&=bTGr9$xa9kc!e6GoOnoKVaM4@TDR}NM0B{j zDq=<|BY-?7VWBz5cRA6D6^9jeifPP``9=-4jBDdT@l2KIip||v4}N%de{4Lyc<%(F z8|s5Wt{>WzJ~e1Dvk|TuebC(lJLFdD*6^FEZ~vys1|HQq{ir1;{cVwM_X>TvcjRy3 zHT@psV`<6G#PqZM%6Vzqrtf)*i?y!JXiDm-MXc2WVxUPtK$n2;Ipg}SvaO;Xd~s1Q zl6@l!g*68COgsIHd|Ve#Gw+oUW|m`tFqOJ@gXF6hAPl{f>^I`jm17vSJ?`P;WEga9 z>)GAHWyk$~`4yV~+K-GJ60-Ec$sH{cweBNbLK%cWrChTpi-J6hlo~G<(ZHzHo~&(w z_sQvF{;9oFO!28?Ps`q=zq*bN_kv)8qdU_ttt>b=c0@XY({EbwWS;4Uwm}}!*B(i4z}5Fg+v)YtfTuK8`j3a?9jd~LXWx|oEJ4BGDCTgircc;P4(x(i z^)(Y>Lcs+LfIV>i@BIzt$OtYG&j)cCo$sd9I1{P^hxEv+!9f+fzlzBqM2*Efjy5tY zi*d%rlY7Pv19y6W$PHma8(XrlFsp1Q&{jSfB|?wzJxYLk5g$`GjeuyJJdg)X9q;XD zQ&l6fi4|_Q@kiygM{1Ld^DY$o)4w9_-8Zx0@3CGs8C%BHb?Tuz{p6ZmEkOr<)Aqel z76*nvZ{PDnW)@P^vjlyOPOByHBX3nFVeSV~h9KwtFSwY~-@}I268`QssVoXTcg@EvD(1UBX9$Y{DsocN@-%-s1oeWn;3Z(-T;JK%Eihh1|9^CjOx1ZJ%A&%J6s&i5FF` zcfP6a=qlb^YcZHXrLOuZ<||_3`R#g~1N{5)5@iIY1pYS2x>5wO4$h1)lqNG#zObwZ zIWj@-u2>9r9R}KR^1nCKkzpXtrJ{cQJ2y>rkA*hTx#iD4jMdzXf=xO8u_!U~7_VE5 zT`uz+*LLLOt~yQml6P^Q7t_ye&+~NBYa-xJ4|FxW|Hp3cN>pa%$~$T*OMEjL+z?Cm z4C>mp%*-2`>0JADU1Q^F0ziLZO;7~p+pI*wdCVp}xCDd}7}?e`v)ePi?-lu^a+vzm;YC4LqbHT0RLXg3WNCJg>D*kexpP+^ab5r2m$(LKAF)Jb^ljno+OVdz9k?*?;A(A5Y!F|c1 z>*5nVK@3DEC7qOjhnuM54?!rXsHoV?gT@Ek-3%LVR99CMSR{|6AAQ>{D9n!$JO6v?Z1d+{7sP$U(ff|BaECkm%aokAFQ~cAKiH z!TQS_#_fN(hZST%U-+_PCw%CFGOtG48mgaOf~IZ{4*c#vjI_q@wl~3qUS)Mk4@wW# z?yaq@kJ(3#t#{~Va8ux;fq-GsuSAl(doqvu_q5T_B7T-wA31eqCLRN*gF%fuKmQO3X0zMyt^OJ`>4b4j`i=?kES%55Fm$jrP0*y9pf z#z6m(CnQr0ryfE4s1jv!1yBQ9IS^>X*Tr{`y&@x+(MlL94L6d$wewo=bv!*m>z>4_ zs?TdyGUDQtr~ekF$-I>(MEipbkHByd(|Rdci0-ka?QIj<-696Cw6?Tbgs($qhJ$h$ zaN+IjT|n)#y|`0n_P@zXi7MYdm$(9CSsHe@f$#? zmBlin*CsM*e$-?MKTz@OR`y zeb-)kycqx;HiUyC-!%a{UtJG6`5EStpw};ti6_*RsK8Ke&}??NaB1A^#+DkBPPbbl z(cQv#nCL9$`(u@6%o=LWg4r-puv)N?`#@>*u_ha|#T3d(79Td?u^zj{2k)+lm$^ z#M*nNo!Mx(zwD7eP@Wv!no}3R`=eV_s4u0buW3Eydo&E26}CRy_HumIdbzvbBK1y{ z4C=Dm@y?jG5%u@qmXUsTl%}QTWe>dP7!D))&WsyMmWB`LOT$9`^*tFn9zi{6+-f_1 zXWadG`|1iA=;gqnRCmtUkU|l)YX`E$DKJVq6GS1#AR#0kt#ypBwb_KjNu&^AuO-%P ztR??BlTcR1s)vf|u3%cBl~Vvm>Vl+;Qi(XK^r$W`TDRP%gSxz~Brh=$W}iCO$N3QO zf)LH<$p4_g4^e&QB*jb)GGph&b1}Z(?hS#lnL4LwyE1A^)9q!Q(HeG5r@7h}Hie3-B4wNClMm z!UmIrjNVO+?$zIe03Gry5D##tXzjKIFaVlQ zYI8&J>EO;J)K}<7){kx+?FCzgAF&(xcY*0iXv2&Mkxs5(%%zlb?{bKub42`$Iy!`j zGSlP+Sm8MaFavk_^=8*%;g$J9BzF^1WDLA9AIO2d6R>ICpI>9C3(cCyPbhH9pf3B~ zPCZi_p=3O|_pM9ruZ}nQCNSZf)LS;yd=V~AfdqLbyK>b_p|$L?)`YRRQWy&rq;(Jy z@`t1>LvA{|MI6xH;lpF!Rzn^oBdp27L?RJuE`>$5sa`@?nex@aLAg}JUYdf5{2HI9 z@id%`Ra*bgRa>gFzblS?VAolaE{#{V>rO5Wn}66palGe zs)YAKcC^VS=W=&VAB;KAAHyt?l+nYQiLGSGJa1FT^wEqvHsnSn6vG-;>>ck2e|JQ| z-^G$>Yvzr+{vOMfNhNJMtJ@P`P)boA`W)(fJ}Lqzp02JIXb7KH z1%DZTSZ48hlAwy=v|S%fRna0ECg!O!(4N}bJpKOn>8U2+qUT`jmxY&4lLJwDMi_>g z<mdccuj4ezrgn6Yxyo;hHiRk>S z7FA3y5HH^Jbr(5ie{ui@1gZ%SXHT1~B*^q(eZBcK)zV{CfA`@vIXAMIN)h=MeNTa7@Q>LaALs#`Ev*WzrK1Z&!5u$g9?or<6hlR zj<;Kf(5i805va7;re(>`&rgw9RLlw57u0JhVDE!L&fsj^0xByjDx%9?i~@W<{Hq@l z5;AetD#Dpz`3*GRh8)uS2a(I=(nbSBx8}k_YdPv**X1d<+vr2iVK~CsS97SwY^lk} zwT$ue+s^fS<4Ist6D5rNWgKU8wmBgR$2#~p)rxIwIkFr-^X?9Mz|l3KK^9|G{$8)2 zwp5Fb{L-;&3;aA-tx)vF?>*jHMTt_tIIn;_9J+y(MUbYy;+ z=J*bBFGUev&2oHid}F{WVpCAw!CKPx>d&3Z^2J+Mo{?%iq0*$)S5eiCJ~7-O-7bi> z<1<=avzGK%5)~gueOXM}4w#L`M*nu;>ntb&yPjRAUo>@VO|*%|P5dDTHHSshEwy=d zMzn-0*?I09v{cE1V(au~AB!j69Pyiw#bH2w0}lr3i<#mn>j$)F_J_;R;aSVwxPCey zhnP#!1Sv-sW{{Rh?+=Xf)THHlc+_^m_}6|~Y()fA)OMG%VD8Kf#~Z4+o~BiI*p=V8 zgJ1*!v#@pWFgkdg=NE9KtsvL z75~2kdbmGZiciOd7_h%90vI(EN~YN=GAM%X7U(b=kQNy{ zA^->ljm0pIlGm4PA5hor-Mw^}kC5TL-=E&w?3S=fRyH;mXRFX&+cEp^uXd2%jjH&W z?6^YSXLc&L3l5ug4o{9u*RQ4#QLrB$)yG{D^`N z-s79oCc{MTFEI}VUbhZB(+Q{#{H*#YNETzPxLaTIsCQ4<(;S*fF@9VdtFcHB$6-Z;o{${5U@a#Wx(4(>_x&&?2W`+mGc0$Y;^h_h8>(<9}DlDg9gZU z*p5^!yN>}thegm-G!<$NaI-onN30mIQ)^|IO}fNwPl3U4P*u-Mz9LY6N7ZjFIDJU?G27 zR2(U7#Tg6m0j`}Z+N5>0`uAT$`ZC=a7x6Be80@6zlA`5)NiGHh5~H0JqNLQ!sZcFu zf@`96I$K=_!?QBK51FZ+E;y+s3b&Ot#gl$Jxb2GXz_$48tBpo~G=vzqpqIjE)j(DJ z!!pP1&O}P7&4FW3R7c^KKHMd6*<)i6gCtdW_@12HqP}-5>jaNmt?$aKv!Tp0nAMh3 zgkqDQKpZw4yK+nn)S<)G7Cpj2*w4;h$!C>1Fc4T)-b7UE? z#C`mgXxp&3$JB`fIfdzV2QxKiiXx@qqYG}b+1Xv-TL}@&T+$*tr$GX6A0;=NmHb)E zJ-07COJtAPSL|9+$J003J=m63A=NSa4wTIWZjSoP_>6SA_8iZY+oLc!aov$_4%WX; zU~0DTYe_1U-2no);He6AbRWsCuc!r)16yJfe}rTk2JeM2)YBrnn}u}Gw`FDvCx3*x zsVew7>6<8#vHD6;JzX$_{42JUX4vs#mD9>wWyuvB`vcTF@v&mm2M0hUgdcrXck;N6 zKe9XRg)fD45Y7U|)@T$z1WEPL?P{P7@Cn0_YAe+xz;Dy_h5c1>?mx3_nER+rkI<@L zi9Ts?Gd88X6#r~dS)QnV>`_Le+=srJ#t!*TAn!&>=$eX9kRmmq&{NceUQbf2JRm7U zdkP75RxcctG@nRrUQsvG?NcKrU{keYDbWVJQ3%~Wy!aR(J7x#}?hBoZdT*V0)j&6W zHm|Ei-t`i?{z%|J%+e5m!4XxzpbpP^#3m6PPn}iS`MC}s2a=lEDI!&wfMSk$SRcjX zEkf7^R5R~S)K}&q#f(P|U_gX}$T#zo-xFm^@6n>G%y*5;Yl3<3(T$Q+DqYNW*iI5sY7S16e^&(U{+%jMKn4_zo+iPs; zg=1R=&I1!A*B@FjOwm}O<+CBAu=udgcqvU(jD$)qOxzUKD~WR`O+5ax77Sweo5W9&Mzt3Aiw*)E>n)}j_+cn`Tg=b&1)OTJPtKAv*(^)8 zhVIIpEuMW~VCEjcoVPygrg0%ia^<_n;|9hyYRDhFk;Dnua3YmxHjSVWd>>o?pH)i+ zV}Nwe;xY-_Kg=SjN|excQCzR^ksVH@9tk)Uz5jX!65F%U^+I7J$i?54L~db15ICbA-*p_{>a<{VZC5k9q zfXyv)rU}R&FPx7GY?BUhh+9!I0LenNp<5K`-iI)Y=@Q@Fq@BZMw~1y;b*PmcujdmI z(vWxWI0ga^+=MyhAGHBX#DjA15vbou(~0 zjDp_HR50V4kLv9{jxE-EfTgZEQl*gN(~WCT`SzVH7(N&79+aXG?Zw7;*%;;YDe+NN zu)y}j?Fmz`CAy*zn@HPIpYeF{g8uqrxydR}LEb#3dc(aNMEq8Tmx!Xx6NwjsNQ+EU z#)28n#gjv|FTvDufK2ewRkT>abNE-BSVG=GM?yMTxkiQl^z?7rOIKr{Tr^ru-;XXF z2`X>|>)ItxsxXz&(&xBxAl4YnMtLGbi)38pFtEjy)-ShCQUTGV$|j}sT*VvXqs@0# zkQd>o_t=GE!4jT@Jqi>^clMn9M1r0CKa2J(jIQytO?Ide39hA7694*?v0>Un5 znpATs5vPM$Oa7w8`%zzOLPoG)g(j!`NG%OaJZ&QHlh0_>aj;lh5#+^a7k_9uC^VV=Tm1uP_!=25y*W|E1J=fUN5)*ipH zmz%<3zCkGL!)^3(H5B^$F+}}e?3<{>RK<;#Bf_5#@p`J# z8r|IcJlxv9Y8(EL&C=3QwSC+#tX6^&biL0YF+DtCwSIyO)FT>B*)ej+7m-RbM+7T| z(!;LnCD2(jwk!d=X73-Z&%S2ljPF%{S&IIV(ts0J!r^wX(okv}K?tOSueaf+rNhN& zB8&a+2GqGL^L78-*FPh*-jFKMDzs|hbF;$JhkeE^@Vs0F`x#RHNYMV!6gU>6zY&e9&Qq}y5Jck+9_k`;S}`1rkqo|FbNvW#AxEE4EB*FZb*4l~8qX?k8yCvK=3T8Yb z!FM{BND4li4`H-fa&SoMY9F08bHaUY`D={nuQpo!uU5)2F@B8b^JGLKjvR?^8;zJq zTOPY^YGbLQ1aV5Sl33Lmgupajz4N}kv}TitvLsY zv^5tyANpEIK9p;d_LMcV6{SrHsQjo@nCUOksn)-T^(_~*WO4zo=Rc3YqOKx^@3+RI zn3W6V7_Rxn{G}41H>RTU)`&V@b7jhgOSM@h#tR!8t`9$p z=A$%C1Kjp&{h^I{K@T?h{shkV@Qg@oomMwP?ag*&3tH~LW=c`t*7^!>)ne8$U@=wH!?X)?G!7n{IXh5<89CBz})mj8Q=0i4&kr&m8D;UeXvrL~<@O=b5c zVd4@-Z7yt?+6xQK9(Jbf&W8i5>1hB~yRMYU>T_zp>&#>dU-QkXC7{FE>4m6wMIwCo zB;uNm6Zp!&>f`m17?dmbS5==O?5NV|vJq8_gY6DnTyR+JxR2<5S`x!%NJ?v6sx|op znRG$2#3zG;d*cXJP*O0%sH$E&IpK2@xm_sS%$u6@IK2no^l!Z!0$)O0WKh*0F9PA^ z`1ogj_mp1b7I1W{-2tzAH7PKM{(2?!#RB)A^Zaw~6s6W5g#xS*W01noNG%0IHZ^Xp768m?r5FDDj(`_rEG!KK;)tiPVC%H(%}KD7GulFmlH*iSz5rvY zzLA8Xy9v-D-)+NZQwdxr!k$j7QV&ZjR|EO~+#y!XiX?|Bn-##KjrYCv7^=tac_s65 zGHK|<>(_36KE_5eRiaU$H53Q<$-6>x+qWAg+~SZc=vIc$RMb!}40O}iW7DaQW%Y*9 z*qxo9Bhe}8D4(aW5EWhqINg(P`D}!JuJ5zd)>^<#q*I3pI7W{$*74U)wqRhOovp5} zW<}2!)^=OTOn!*iMIM{}p9_OA zO5apAV5r0|6Kd#bF%>xrq);mH?4KPxb{lu^o$G>>v4nMNOs45}TkGhEK4?#5@ZEM% zclozK4^6QH4$uQ$`M=~aW2+{V>*(YJq_EjfiPt2;j|%_n zh%Elvv%T@?BfMB85U@L7#$7QwWn1&32EL*~vwRIC{-kaG_(-IC3x zdVu%@-1c`}f?uI1EB1U;r;0Xz=PMe&eRHh4yw;GAR9M;c%;-CS>@afmBZQ&SVyJL0 zrjK6Ff8l9=a-<1}V`fymPV|cc;5MC+=DD*cAj5Uu6nCis`Xp&Ma%#}x963oDk)0MK z0eJyXZ{n81!d{P!Sq@n7|6c%036WQZsTxC$@|_$VsRR$`<7lM(wXg@_2&&XZTpDnd z&+>%bSd-#%U+yZ83>Z#ZHT!mO=Q{1<;5`{wONi zg~`h3Ptx#izSq>SpMr7+C+Y$S@Dh2!R7zAc@5TV1!lE3_{}UW?^Y~^V`cp_@fUnJ1ah4AXmW#0(-JI z$9?3~=}=x$OjSC_HhXfwFGaxjy@Uq1?ZPUY-iNK?wA5GOoY`n}&sSgg;_Mcg+I@ky zI#e~HP#t~m2a5l_2_i%@Y_*que@nSph!Z3;8sR?L!=>DOc`NrLqaaiQQyo#q44EO{ zB$1Pgg#!x>Q6c3W9r_0%Bd=weVB5gK7ASL|pHw(=Elr@*MA1kXQqOoVj@KTUvO;b};U^*~yZ}V%c}qY2-~+i>?s2Jx8d@CaX2$g19R7 zV8j5FSCip~V&43Xm?sx@XaiRQao=x_TUsgH}+@DW|@d7Ua&Z!dKm-xk& zJW>1#vC1R_XYz{QN{fDy2BM~(TZTl8F`%Z5YuVb2rS4X9V!BcLzoknOcNi{Wy`Dq7 z6mpi8_V_Sr!~QLy-ie+2KX2z`AuUf_Bj8M??oCktHl_<9rlwx0)8KP~npMbKS?ap| zci+B08eX!zzo4*O6Cu#FBqDC3#bI$EhBLDO3IaxsB)J98b^ePSqw6(EjT+C!YtqhQa-n-x_wu=%k{=N(BB z)qom;L%(zlVPt~ZadU46d3L=eX1%pSx9;xkaq;~6|H|+E&p9%V@!jPdl!b@yu`TpD z-cC($r73guI~3vJH46V%Pe=A7^~7qU8SWbFerMpyKs^n?9SELzZfiW8%Cq|9FvX1# zM}~R*N%RiL*6F`@xWJaaHgG^+jCPQJoss8+rbHv$?lWy?*ka!ll}TfwGt<8z32v2? z@P8)3ZbHVfdxK7h?d-j`$^Z(6bvR+KLxvhv1Z#=Xd@9`~ZCg)75&HPIHt`Sd=&7c2 z)GJ3wFT4t|i(J84WP{d;+cw3w|B_7n_FYe~p>l#^p|e0!vE$1s?@w!`><>BTmtQ+z z0E}Zd-E|`X@XDk*FSU*Hbz;~D2Mm^7Z6dXd6qN*sdoonrL4Q919XC#F?R4vIY$BL( zLxvTNKgYA`=BRaoZck`vf@w&^-;-QoX30gVeIUjpkkMOX_L?1g(JV1T!fjaoUbEFy z+5+|MZ^p>dDwKC4LeN**ac1~C^iz1LyQ!14%5lB*wj4(c*=>eHsb(X*k~LDUpb1v? zKtH#yG_OpqYEMt+k-J#rQl5VZBbAam+HYoU^Fi#VYXCGMh);ZGiP5S3#+9qGS?z>h zSdcbwpTuU#W^nr}CR3>lsuGgYn1yI43_QE4#s8lTna2gNe^J4(YA6Gk3NWkgTxkwj zvJaH>5hZr}U)sbRO||-7tL;+EL|%jN-s2&HcN7r`s~H!X|nOtxiY&Nz$(i?e>|-^17DlJ z#&pus7-^6Wrt!932ZAAcOWWSCAG`l@p`iQKRz|S(yFXAPFw#=vI5t_P{zVZcVsYehbKWseC z6;=<493XbSXTGBFU54{?Tx&JkjO~7&-r`^H>aT-0x}!@K|3ZhZ?YPolbsC;twgBi5 zTEnJ%th_QIubR0_H`VX5UYM_Zo4uYlmDV2tuIyg>eX>7F^#8E{|HOvd#UDdZa5kJy zKI>%z9Zg&OCnvsWZkKdJPPSpxN9na|e^;}emb5~1_qXF>%D+=U89%VA)kL3pK^8WA zz%~-1PnNG0ja$*$^=*!)Q|%sec0b#N*1DmO&M(z#A0+LyVO-b6j<-pfbF3>KU=i1| ziMXfk$F*+dCX-k9SClW&2s^{;+j%U#Tf4ou7;q%dhIRO;?8xl42n0srTx^Vr$#ku> zi-=3aUfEw<=_s%BQi9)n#?XIPL+vD;!h8@+_~p_7L65!u%o~Gvv%~FaC^g2cn&nKR zl+m!a8Ng;qOA}qk43adfWUOILVZ({UH}I7{^XHElLLr^kOY0QNJG9@ufD&;2()p=tL_2dnrl=1 z8Tnrrv>Y=0#v=I0sd>dDNyKfURAAxzFE8Dx!PLspYK@zp{a>1Wux*ZYRwMHyYPar@gAYki=I)I|N`n@3^^v?z`;AXn} zoKnty3+%9Ws4UwpG_$n<@-W-U-xV8 zUXOwN7kp^?F=?F%vG-6&V>f4s9p`&|9R-$BM#UL2^}MrS*If)J4m5&LIkWzTrE2{y zyXl(Dxw5jPt7O^u91$Otehp8r;WU`mp<{-I;o96ROoYP*rp|zE@AdYrhNnQ2-SAy| z+$r5`blMd3i)Q7w9W*z4tB89-)3urPJb1~%nRew<&nnX?OG;7y$C&=|r7Gjs#rwN? zE!~pjt8o$zich`(Bjv&@)7Sp3=1l?M{)p~}`FWPP*S%U8ffotN)Ykdv7O|9sJ`8gM z*}SMOs(Q(%5b(axnDvVh*Oz3?Rgo>8MS^Yn+1cYcrAjq1>C$TaH2KDoJi6YtRuXwQ zkt^(WIAP0Bf>>eM34;N)a9AfAw0+g)%kFZ!r$U>|5OkX@lHN&}Q~{O$tgf$+&n^u5 z?=Y{50Gs33Emq;-nLcFAK}@>7-JZ4a;3V!aaEEz~b04hNVI2{5&mWHwP(jBPj&Ky9uHDuNIsTa z&eXp?Tn2R6Zl2BG+R@72-`;O^0Mn(Lajc;`m#_YJH#Zyh78{R5FsAvxzwRl7?qFWI z6mv8Uz~0?&AxEHv)MvzK!a`X^Axm0QlY=HqROEA z+W|y#D@VY8BJt088aS2B1XnyzPJ3ZBF1yo?AcHstr|R)mm6$za$+a;eG*_we=TG{$ z%BpB?$xomB{5B|Q7UmbaeUFPEH+kI;)fnn$(j!F5mIq)yt3{@yD?MFV8w2fjLMrq0 zF&KbUm> zX>NV!0o&NQMBLk;zs2(>@UGYo)c;l9nC~1}Q{}9^hjJhQl9X0V@;b>xXKC$Hx_Urz z_9yp0+vgqc3vl~2#JTgFna_}QZZmr)03~vI&Ci2H#Ns6my5SISl_<8kCxzT!3 zj-If-!~*6>K94F&5i1S=u%hw7ip2F^hGdf&YEV>Zb1}aGkHzusAgr>WuDNHV3yeZl z(w8-2Tmz-%>v6H*Ddh@TK|2L`Q+fMpO4q?)OmD6O1y%tD(SLn6*hywMj9&qI-V z_xE|rmj6ZD(YJ3t*nS=f(#1-gJzQM+#NeSobUlicOYtds&V4I{0feY``beB9u>o*d z;>QRC+e5G#Dm$Sr*z;GWT?;vXmHsE2i?FmqYdAzkTSW>aCaQ79#+B7$9-nCw;(+!F zfMlA3`Un3?es~=kB+8ika-OM_!=T5=IS(xAsdvGTS2;=g#k=Q}1)GVaszz_bbj;o`~=E z2v=b9A`Iql3-%e8Knx61qD#ar>%t97uSgK2DSETg2sC{+n3hZ#Z4^2UI^zY~c4$QlI6DRxN-|u3M;&;CMub<^L!Cd9W*p)mU7PQa>_98w0eZR23mM+D%oJ#m|az?sm1WaJ#CfmX3*Tbz`G2jXG zg*cGp(`!y#Q=Ari<|nr{|EIf8-z|7II*1z7^wr(26#NRD%z|_PaE2oc`1L&L$A8ZZopYGBc`2_eIarl+cjs@nD!t#Rw_@Y3`f6{* ze_o=Ab)T*3@8fq=qpLyI1X?$3`+ECJa8Ats6(d5b51{b_Ln_R!>~(_!%DUkDG#4+b zB)?yuA&KCL3%_jj^-{4)zqu1s#=1!D#)98aeSl%jge=6PqC-&7sC@sHJWI%AqZtMS z<;WHCs1)N%7`3!ks_vTKNR)V&tTBQ>7pd15e6}qfAVTk!R@LfRzJebfncB#PIbcz` z)J`aa%Df0v$`l>Mp3St_f=P8U^lPFdXQC(IHl%`4zZ6)O zJZ=tBtS$c1{1;4>@UNxlohXtNO0g*f{=lXCETvHdh*aV-zN0~+^JK>lr$Z=!k4(Vr zT<9z44GyYvczv!7SlkkOIJQAaEh<$c0Rllu;;2YG5ecDZ9^`ztyhg1)njN@|aOjBm z$S!rXgc6fRRBB?Ok`EhyNoao#S$| z&AXo%7IL(v&pWUrCnko!=VfqPjR}iESZsF7;{!I9eP0Jg8^41*9V&coQ;j!_ji2tj z0?MIOnxWgQwzBMT35Z6nBA`2V7_Y)$jP7Es@l#{>Bc!d-CRyL^_Cy=waAFgtg`8IE zdD2v>NDKl3?J9G4?R5CEt#r%hu6~kmbb>sRf6f6fJeb+Ew5?3rT*Hjv?4wke2+F5T zz0MEP^?yEgC{t8a)I|P8MO!)cdOjJ%*jfXvbfl#i8xys8-G)aZb0vRfo))J#y?oPQ z*y7aTb>*7F3PZjLR>L7pZMM1DU9yq4oO?HeOP$72-i|lGNmub4&p#t!*i7XST)GH% ztS)u8wpSdRWNEYhyyOMh5}J(ow|tmT@K$_>zP`4#G-s+QSX`#3QWP!vzy;I;uvr~Z z|3u)<#N?UI6z(o=)CI5$pSg}bpu%!1tUO+^%u%N0ZPu>sx?2r;gQ8K9(!#D;mGp-s z3~^s?cFA<5(eE6C$Xu~sfie00{i$k>!0h5IeO(x7k?%e8W>`ta1{e(1W2aS8vph&g zeRC>SL{n109+|)iUz8&AyX)S+!4skpNzbtVlw)zu06jJhm|LNo`&+L9Qkj63R%!AsC#dUcNg#=K~^8%`)5L`kB@r= zaJXH_fkXAEfrem+N|YR&ECRgF#$W>L^tBQB42Et!uU%$t50<(Ryu2R2zI01EOW@5; zwIy%mhzWjtfh|*C0`Lk@C3l%;VJg*~o)84McR6zxEC|EjgrR@3185_GMrO&D7~YSe z8RCxD_uY_7kU+tY9-VFbs(Tp8TL4Jl=bsD)+QFa1d1wXY#d|ofd-t{HE0D)Eu?Be$ z4^Y6LUhpaZC1+7$)fYVTlRPuz zpr`hB(buycyADv*ir$fJNpd|~%ac*U;ELi_Uan3BpTk;XR|9wUcbLNuAQSADP3H)J zx6eP3pL(u8{T&%CHl2R#&31bI>$D=DFW|CkGUpd5&5p=fk4mTqPZJ{_#SB-nvG_E} zQ)xO=7m7Y~S`Zxak{}F4rnu`9PF9?$&?*AgdqAHdcGi<5vnqUooScE??iGXRI{(#t zn5wM()h@HJmOSO~XI57F%dUH>!27KLvDAskNM^~=a6H|xhRW;V%g(MJGT4+4T2+-< zU**L=sEu@4{;1e5Ii!PKDz=QyEzgGE^?HX9^cCeiI1@Gn%L{EowB)VK;bsG zJ!K8_$rS2a0DVrBLgChN7>aKFJe|SiXCR%iU1QSm?J6lOLi8mmm+(}{&PMlFGnuC& z!#R#$Bs&RWPcZ7o-e6lKYvwYA(g)q{<+Y8KM+k{5(oSa-Q+~fCYVZG}=_;e**qZG? zaCZsr?v|jz-3NDfcP9|sJ-FN85Zv7%Kydfq?)K*1`@J9YV`i=H=_6gIYS*q^mY$^i zcfU70Hb*Z6EikAC_WXSIcl}6ExI)m{tJ^)d3+kWN9c-fqVHj*clC(hh;oYmTj1{L= z=wPFc%dT_)*!3Q^M8S)KJLelZ@(drpNt5%B+4v_Du-3JImnzCYPuYbN%w$XzYw_1& zDMKV9`S91ENNUFePIAwJsaG29=n9&wF7;q-CoDZ_etbCXW*uTu^9K-1bz2)zyUrwH zoIbfYJp2chnJ8xn7bDA5Y*1!1FS;)JEIm2xo?*mE$~mYk^v5zp9{Ow%fc9^E5rmQ0 zPvDdm9hgku)qJqrfR|$l#L@*I5qu3h*^u{Y;+U4i>nES#Hl(Z6YcM9|Ui$s}m`()( zYr-^{V=1NgrwG_=fKeC~tybUVDxGEt;^EkvXn~Z8j*s^jp_e0{e?9^JAHhI+@qQ%T zPA)31@uAP1RcXF%mlMFeMscu)P}_u(c@YFl~G$eWW=J zp1rzhnVC9SKp9M3Gd}t@tGSZw%teKY^1sKnLjJFh**bTOh6qW;+_t*yW;f4tswH>s zO}Xy}6*?Zr4Hzn8*RFjTeWX6~vg2T?3%AMGFFAUz<3}Eg=^QsJ^0mg#)IZ2visJ4s zZUk9>1?|hrZg?GL4+Hsvx8ZvpdiVU8(|TS~Vhl$vfu5-8R#f=aQ{uz;Id%QGD{!I-Xy|No7){R1&3H{?PXb0k0g+H*_DyAb3J+3?ddK&zb(cPZC4WS(m8$K6rr!*HIPvBMMO0BTA zv01Ft!i)nqpLvnyZ*0Iphie6h()D00>fFFb69_~FuXMVc9LS$c6AP^@EO6UwBogWH zc$bO~kc-%V84Uaej}DKQD%%8Ox31o7^K$Pp+!y366uC1j>tl}XCa;g9TNUyxnT>7Ua6%vdcBw2IFRh?=KK@Qmp=sF zJ)Kg_E=ICivT4i}m2e)e+-@Eou(15Cp=x3SP}6SAGyOw5|0T9Bp{{#M;*B>+dc#?v z6OaDTBg0p!#axu+-)G^(87OoXnY*3d zGm}acEXC{_?=<8yJB#gARZTw8Q=BZ;bbG(ww(d$csGF&}oFpG{*)*TFxLufFDDSj> zD9u=M8kff|_z;~{QASF8y_7|a#ZHGV46x$UB2A9wYFa`jPzAD6m;~qld_{@f`a>jo zOA4XaUrQgirY6csB%CRZFDu!DGqr)Zz*iY+t@v8SMk|IqZUG!ol@cQklSQi`Hq&Rx zX;7+p_{2IO5ntnFPXVtK?!a}+?QCK7mjh3s8lL6tcLQ{k92w}d%t>lDe+!b*{iEYi z7*@tZ6~lW}dMB7`-bn9@a-0ZNsa37p28Tg~RF9tnZrrSPsz=#cF|oSwZ)j{00q=@+v%{KTz`L)GYy~;1=5>Znf0C&E zITR@gKP`<44FwV#UZm$=%fjN%G1~|IPpsF4>>@nMO?k$uZs{=`aoKem=EO9KJc$?eRG9CF+B&p{wDVqw}k>O8x5`n12Q8O z(ZaT55?a}9*Q@kb_~glfGbs0dw}TzfHz!TDZ%> zr69r)ruB*V&RmSS>}wTq8f``dE7e8j*V36VcFWPb8?lSkZfW3)P7BI}c9;eYWhDN3 zO4X3lKkHm2kX}ffrov6IvusDcNo8>ReBAie2bJDHG$<%pBjYg+Ra{XhtBby&00D4@V$$q=e2?H0Pz&B(F7ysakpJz&*@$!@`7b*+)<%m#QY?KQ1 z@qp0DTbGxOxgzr??GLyvJ<=4J$sCS~hQd{+tU5h@beVpYT=o7Gc9o++!NIZA9RvG~ zihmDMF#DC7%WmbD7%$hc`nl6RHbxZc>1Oxl4GpgdJBGR=X5Du>S@qXRmJi_unGy~n zyGfGQ{W#>i2fX~Kv}$#r;&Q#y&3L;9CC4&YnB{wK3pG_OSb&4v`S{Nf$@^9nlUt_X z(_X`z#Z+nid2E}TPSb)_$$QuN{keWjd@^HA0)SBjA@UoN3XMXx)34LD4lnDw>htQ@ zIH47f!*cfcx`#8bpy{UC?rQ+}`LB9i7AXXeqn8zd@U60&*Y3PZ<#mmcUKAi(D;@%m4 zZo(&d*eFc@&vgXTxe$k8|m zZ#}V&&NnE*qKRUKOY?s`xig!SF?`qP=P%xe=||2;*v9;J-p)j>{|oYhL3?VM^gl1& zx%KjGXh_qxw!XysM?tb9KXFZEIZ>Q90n%>h!z7dvSeu1vTMc~5hdk)8mPj)nNl7Fn z+~JxzX99@j=}b&Ts;ZNY6(Evt#8nJiR$}*=C1g2N{s(^l!)XhTHk;tc*$K%fa~O|4 zzRp-$6LcZFZ>--i)IydA?0YTEJ;k%X=VhdLPCc9&dWS=PSJqD@Cr5zN^cud!2{hDd ztq>CCpM5=c51zckxHQ(RinQjmKF;cXi|akCebKF_Fw+4cP7)Jax`NczP=WMQ4M zp|W_6D92BYlbc8V14>?%ym_!o^_1d;Q2#Szq#>lPg_C0>3s@>c(;*g+v=RLxT{4jS z`}O>$KlSN;I7BFB)b@~-=dNkoozFPBm> z#|@86V%~li5mnr$tBHdXEL`I%KEv#|H@YRFIAISu8uw)g^|8)ram5Oj%3I4Ta$ygE z{WtnxjZQ}ozQSh9x#30lLN!T3r_f6qOUHK|yDj$shu6G>0vpeZOh{{vs~aEAmdKMy0bBN>hWm7UQY zU8uibmweA#peXC0&GyhXLHUtpfK5)`GiIPxZG{C$$VW)1kHO1A!kig7-=COxk}hp{ zXHWfqE&y2GPb?=29}Kb_Y`=1Oc2f~E1tE>mi4CP0zMC8=ceMvZDqmO@O&LSZBlO>m zn9wOB-?VZz!G?P&D~3GiL~QZK*~^p@TeRsCxNnO-1@y8H-AoIt{3d(+fzecUmy=JV2nktSMJ@3O&LdUDNO~Hh*~I3Gr$5zb#Jvzw z8W)jpJKPJvO@RIjIx(&aV*PQcM!>YS{YIMB>HT315v0HltxHy?wNQfXJ!`J(C5M6C z?-;`#*`u_dyC}wf{i8@Ahxh8JekwU7C4v9n&JVRnm99wg1A^k&9wG2C`}bKVuCg4^ zCF*h$68vJJ>`b0G`-}X@hyH@iH!Di=GakvpMd-96AHU==Vq}BY@A+U6l>TED8F9ki zJLG)dbCU=U%#o@VnH}hD=7?E=3GK-LaxE6=v*_cF6zSWsLS-^MKlypwpFfs;=7jhn zDfAvAMtEf}){hsM(J)B!{Wfu-jEA|ob581dJ$;4Nygu?oVjK zF)*}eBIEx=Eo?mO+oqsjgn?WXvWsD^JQDh0 z@YbQbYB_O9OeB&%HI6)AkNUzx(a1!8z+DjyG|zuRCl?F$8#m`=>i zS7MF2)AF7MBR}xSG6<3eR=qYT+Wd~iNkI_V)@t9da9_N*-j98XmNwiMzNG3YMh4qHwwt7&;)|Ij-2WcNHy8c4BWI>fWo6qpKwP2JntCs-U!d5{=fy7-%U8eH$<*#Q z(2^N9TpZBR>E_P3!_kzV_CGv%vGa~kfMN&Cj06v}Sl4Q9eDoW(D!@E69d;2pv`=ZI zNp&e@AgOdaMy=}o`JMV(A09X7`)Z|jWOVX@2b|(h^`KZDnN9aw98YT7&4Dcep>(zd zpTvv!BE!*Efs~B_Sz{OTSRbwt#h&zkr|h50kdw6MgJ=(p+FahoUdnsQlhmr6?LqCo zU4UF^?8Y67G@Qco8L+5}g7pWCXkCY6g7{gYJ z;iyziaJFV3FlUfN&?g-(*8Gf(50axWH)4#IMURr2YRfl_3gT-W!&2Ul%qVZ5)YiH^ zAzG%3%mq3e#?+ZI;sYR)W`0I_S}}>^iZAwBCcujjomDdgNWA4Z8K)a87cH3_Rw;!54_|=---cO3|mCw5uABByD zo7P2+#=ECi3>!`i&C5jE_oe(H%QHE&Hs&<*iR{po2_TDrOOeGiNz`A>TI7fnW=QQ`%bzItzJj7l7?Z>E_WGoiv&Y$Fq2x7YS$o}-g(=zaIi%{yMH8^pK7}8J z3y&eLP$?IOTk@-OnGKdyz1#=nrT;2S=l_a%tlbLTQc9IiiVW zy*pXVdB4pbP+A;d!zWLS!iPV0-aYp82=OsTxT38E3(GEaJ?=!cyPtg~kdKv24TM&x ztgo+WH~9#=&;(W5*y`51UX6pBB5piv3i#c36o;d|d>sm;ziTM>7VWBqq@;)9M%WEfOEJPv&7(W%db;OJasO#3tb~RM zRD%lzG?JFpPkPZi-_jyFT}JDxlVL-*8+m8BaLX8@2$Tl%eUq z8^*L@Ia1I`(NgbK=)hP3AYu35;Ly}7WWH6l`LBT0iG-dn1&YE_vRwP@?AueMO0XEi zX`?aVtT~Z;K!vO_NO1k1>kAO@qQW2W%58w*G>yn+?2pzZ@qv&sra(LLgr!ECMa6*U zBwPbur$rkFN{k5j4RbXfwl$|JCgJ%bg6!exW`XvbnPC#2UxjZ&je=cE1~ZC{8A)fBN)kSZLh`oSzq-5X&;?zP>p~WqCN~ zBR>T$aJ`S$=<~jA5Tr5dPyAe(#ja?s)L&Rq${cI)6Ic&>@LTeJOWpr_4ep#&qqPD4 z-F1B<=bYk2(6-%OHv93o@nnHfhrhU_ptRi&Ecj_>r`Olvz8h*bp2|6|(Ld>1?{E=; zNuxV<5lv1$?R%G^`T3tWbWslabixsZ^2uv?9RIh0_YXN*7!*_nZ<%V7;@hcjhDXTz0l1uWTGbWkNSL zak9v0&~5Lg_m-S9m6?0?vUOd2fJOLoDb-FVjZtq2s59)i>Y&GWJ4$0WS%pxjRNLV6 z5(}1Er!#u+c|pmsdK+YU*J@+_ILqcyWwqY1>103o?9Jyuw0x0*Peaq>_L52^o5~4P z`?`1Uw)uRy+c{Z9G?pMPOT64<=b~yN>TG)15d4CY`vt!y<7t+~pUwRw%42gS8!*tY z8}|POt>*F*i4>!;RaKRhFCLdm1!%aSkt_W>u_h_nBPm4E4{=7L3}R2qBMRB`_Q%2W zuKRn>A15~SdO))s)IU7HVP+E92dV+}mJ+9-zR3tpF7XOoVgAmN@2H-43h zGhs9!OQK2^9U6&hhkeJAz7!UrfDSaJBnJmmjzDBA9179{7JU_|Wg}vzkGBl1kZ5%p z;1ow9-&y&id6IT$oM9}+Ugk!ZO6^9g*90Y*2tyxEC7+ELY1uc!R>B!oZ0r3(t}wfY zVY%nX!ZQL0bS6*mJZoG8H)bDZdA}V7+e4Y?Vi}StuU^fRw@ZZGw%6sEBnA?Hb1>Fo zC&^r`(`?ia3JJTvI6Ea;YjMAe^MAYzcwek2lvWE&S0v<{;Y;@an-81oeNgT?l+(Q= zJw}Ra8X+Z|s@aPaKi0+tmU!8`f105Hmq^AeE(o;V)IC4b9jwG4+i2KsbiLgaN>=N4 zYAY{Z-cAuAyWK6`4ISUvs8?(Gd2SAW640|*tnz(-I{bJH_+V#c=XE&GnyKscBk8aK zhQh*jP(yuk6XBEk=U`Mc{2n1*IF2LWdo~tQp`o)qc%D!+7nAAD*x$j8-OyK>u>qyZ~Z*Ftdm1! zwtrfnv4NfVltu*eEz-mauAJ#eD=2LgU&6h!RU8YD&+Gv9ndlB zE=pSDeZ4n2mrDc`nJ!}}Q+H^O4Huhelh1Uw*Y^9n)kQ$mDT^cU3jVu}`xP5poiY3FyA$=dtuUddxQ`oH8VsL=J*l0h7<&J|rU6uleI25#A6o$-~0BE2Gwr7@=)P#dyK zegKqcveVw&@UCtxF0Qt^-@zsT+1P}=IAjam`+FCKsYy$3w-8p?Wi(jMKex5D&3v!E zDZNL%y~}+3BcDlJRa0X|@g_iYC@93W!UNjU6rYdN_pKW{<8yyBbO<<@vD1k?TNmKw zR-favxB}gQDZ~B2M7V-NUw$t-?96f7t?3&W$l8WI^C^TL1niY+R2nR-w{odMAa;_{XXVcef@Eru1grVjNxwP%7A8vHM{B-(QTL$Dg7v6!v z%we5Oo%j3gcZNm2v`SrA_m;g5BM6;=ovX1J29o#d+x%eIdxO>m|0C<+pD~&P^d?s^ z3^=;{cSpUSUUaG(1D4%w{3U+U7yf?qW4h|elW#fyo%N*r* zRae+8W~|!Z-=el-DgWRz{wL;p6LC~6iC5KAQ`JE~vg$68NyRgTlrAZ5V`nxIwNN06 z8(=sTtYSsK{qph+wTcC<}3P(QdofO}1yx44}w)tM|htY@)V;Gokjsr<+u z;o#$$%GUdMKB;+#E8-}NR-doZuKW8pKW-}f{q=Y<_CZikOkr^Cr9_fS+z~>Io?fF$ z+w0)5bQtxlb7A&*osyoeFg3ZL zNR8k3?s}AN?oejFl5OtGOYh*K4)f8{H%<}C2nGvO|I2%D*{}P1*HeYfF#%(*c(L1Y zTBY{t44-3b<6Hz+rEYir*$|j-{z02iiXTBEnLBECk|-VR51j%66y`^vB#s?@r2DaMuV*nIzAw`sr8h%rY4j?q*^L~$vULST(Bs%mvvw7|of3s?m1<^C*l05Jy_ zeMCYNF2AIt1k83V&*sbVw{?jG0f4_XJ3y&rao9yN^rSewF|pNDag31?g*(&`d!JcA z0EVoUmFJa~){T3rdKz5KNLujsm1Bf1r#lRE_=G-9XRI0U4=V^af2OJw&prDR>juz8 z%+!$46HGNLpM6qAa`pdNxX0g#Nu!;n8dsJB^0z3+$l1?3*Rxy%T^$SU=KDJhrQm*F zu_Y?$njK^0kOv_*hu~67ujfD1iE?YEp+n3sUbTie%=%a~FKI$=m!6548d6>>`I5`J&Vk&%%)i`p@6Mgxtr-W6c=z2YlRGv0L47NqgO zAWY_FeVff!M?ZdRiFtM?V|+?h1e!Eb_3tZfq5odELg=Jg$*p6~sJ@+6-WQL=_ZHzg zYpu4X+wENVG*%3*7byk%f`aa+4LUpy?40E4;UR=SMpgg8TK=)u#L0jrtaNsl^Pdd; z!Eg4F#JX)opsV*fg61;T;BssK*6A6`m9}GQ!bw0SS|Boo!f3+#{Zvkyl?#rW`z=`d zV3XIyUtu@MvU03=sb|Cz0WA1F)1@4t2BfhY$^eOvFM^2pJ+}F!O||Fzq}kF|tI1** zm^2n^e8Pz&^>FZ_mj0A17u0Blv4?AdewA5ftn7h0eV=S8)9qV17d$slK|^0Zi6$zWV!Vxt6%?Nx zTw@dbBy3FA($Gmb~Pnk^M}$ zZZ)RX%SwvV+M1vS3j-r_@*rBe4uXw#u>sODwZ>jQ3CdK;&WHZE3>|MqauGvuYtAF* z%G2+o!p3M{Y^eXi&AM!qgpO!C`WFG}9@57w0LjK`EC?Jndof;r+T5|duN{ZpgF^904KKBR=q2>CuD6qeTIvd3^?%6^Y`MpKHb_VMkHE*6yqTXC)X;EImi~utJQgGGl9Fet$c> zQz)(X?mw^ygK1~UdgkQtxLD1DVL}%uTc5s124&XdAL+*yyWU~M1Z45M+Bf9QkSRX?{MqL@t~FeTJQP8E zdjJA1IuMH6y1>O^`t#|T`y3sF1ctr5C?=A8LFg%2-H*!6cTu+V%h0 z*tiky4)0SoS3yF#r%5W;dYEos^My7v@!Y7?>VFd`2SS=+#rDvm3zKKHIxPJ8{=LhT zw@|shT7rQATmT}Ww-_Z%1gl{;MpV3m`o2Y~?QX z1T1%>)oe0IF&s-sNMHi}l|InYGf3{g)Ay<~3`D;qE?!v=mWaw2vO*re2>84O{57Q> zvmPyxGLT5&NIoVe5fSn;IX{u`FPAkz$@ED}D!G~^lnT1b4wA;&fTIC!s|o7nrt4M@ z#6t9>CT4k#fJ0q*#`}UKY|_!+DW($$z>fKl0-;cY*M9KPKOmfT;t^1Bb8GkowXej4 zUAs0NLWi_M%&h#9BK%T)qWXZT^mo~-hv zb!djOZ`|y=Fo|f5o0}IGkz* zYR&zwH5~<)er5{#f2>z`ZDVQV3i!Ow+?~u;+qHYIvjK_XvYvJ#mB|W@hsqx&1El2U z>&?c`bGi3p77cLa&C%L_``%m;t#aERb-KN2cT{)Hni9&##KdGhUk}xDt%gcAHZ-{0 zcfi`1e4Ymy+wR?_@uXJ!xtd1TKhPDF*VQHL9%?Vzt)x!;B(R%|&5!h)Y`w8ly0(+s zFsGz%3ad?xNca!gLXx|u)q1QWLukP{m{U}o%x=H5_;?*F-v#rv+>*AY(z%NRCIqhc zC#5J86-wd9?4S1%D6rFe3rnZkNXZmT*N&wI$cTst75dSmkNS z15_=dpMo7uy-w1)J)Ld}SMU8jSq0b~Uu|!OV&cr%6Kw1>$YD}aQc9(ju{Bt`ke-F@|79HXb9`W$#8Muqbxa7vzMc01l3Z+!p$9U2jbpTG2~bNz)Kqi(_7 z&5g;&^eK38)pjM;^sn(SErk9zBWK&&fe1`K9tV7Rv+70xxs5+s_YGGby~>vxtIh%Fx2|0JXxo1z?hOVslzIUG-B;5O?uvj z(sqF%1yQxhzf19eA!<4EUpKSpe4V`W0du`r=xj5Zyh%9b#@s`mvf9Bwpbus+j{>nG zShT~vsI;N`;W;?L4Ut&iwew7-Shz8!o#ImpI%Qb9 z&Qf{>xE>mDPs7P~w?_?X zYHBCy)f-)AH=6Y8o9kaWIroB42%i~6!R6KjAP~L1o13WKH!-V^{6e}&FzAtA0@{(# z;Q2!Z7Z@xVi(GtGf-TEkrE_Mk0uFBVTRhdouqusG+re(AuYoePEGJz#3M%TyV!)db zSn)(QjcNUtS%ZvsZTs8b!$afWKX)nh)bLb}j@s_-`7|IrGm+=)sXxa^NO`xdZoaI( zfW@W2Vy)RLV%Dyg&L>EB~xV>@3^`8#i}QW>h4suod+o*5Hdd8+lnhk`aw zb_-ZiVHq)PJsKKa6nI+KA0}^Z`_^`+V{_uuZhyA?rtacWQ7{(OhnC<*u)s6FhWgsO6=O4D+cP-@g?8)`!YCFxVk!?RoZ+kthPUFGe&P6i(LT->~ zl|dGs3kRxDhvOZ{j63Dnu6x*WccOyd(LtpH(1)l{m3QJZXHERufpH~Kh3n>m`FFcao5WtDPK{tl9!Tqj= zQlaj74ve|{ve+2PVFePt*O9TwabhjjwSe8*8PtSRqV)EV|K|eKIMSQ!?7LzXnxIG5 zuN=);2>mi<5@(Q*2%7>&c!Q{XwG005PjogRdE%`=YE*C6L)3F8MFmI^B;z}v#9hN} z5+vHRxcMkV4?U@}8Vs9Q&%0kskrE_bM}sG{EnVidTKb9OgZ6Xom8z;QecK+#M zn&}}^x7*vz)3XfBg$aVxG&40VsxP^>A!ii}Zx5#Vm(m{P?g$l!`fw50+4%!%e3>nY z6ye@Ehd>%r*&p~&0sy>MYrtF+@PV@-GJ(87w#O(`R2P+=M$kPA4b7lUYoegd|+grR{kDLnRw4%N7Fhtrn)7Q*C91N~hUQsPUnj!z5eD9Ik?@0I2|h zTMF|JxrjBzF+zSB6d)1~AgKzN#rm`~m-ZvNXH|>4dzXf^Ljw$_e&FR;Ol7sad#9(_kD5&M~+u- zqIKv$q|~?^(IU|-!u(+PccFh!j0h>k-$sQ4slLcdEtJX2(Tf}t$kZ!3y8FZ)a+L`zTh`8Hlpzc5p-GzS{qYRH8pBx%szg>^AI{pLu%T{ z#FnZY+~}GXisD*Zit&^lk)MJl2a(te#~}JY^7`Ey_xII{5*Tz`Gvyo>HJYs*)%46q zdocuy_ovS%QsmI=W^1nBnS>A!$JXI@h5td-{Df!ld{&*7URoV>4|dXRvJ!;&MvuPi zLyhI$t_fELV&JQhv*)nqftFR5#z}-QJIas61E3DWcNz>4j?hhcpwO+}aqnVyl8X5M zi7NlOC6R=enR%8}*p>QAR~MT-(K5viLR-%YEWTaH3G3NUM<8xO1ZGGE#{7tNDCVoMe$HQN?GKmu-KQigNo&ej~yDjd^#=Jlhbze#X>C018gx!g@$1f zOhoI~(R^>xH3LQV#;5Lnatulii;qZn>4gbhz%*LIAFBR_c`uDHQ*m1llNR z#+NrMw=5b5j$Cg#p^nKfn1V9z!0|+Z8*tD040&~bKHgx7_)$8--A4-%ihp4Lf8YK< z5sDzJjeEC<2)DR)m(S2PSKXVIklM9RJ3P3iV9+5SQe(Td0?#cUfI zt{E~ARe$gNrw$kVMlN$w6%9a+rMtbZ`4}n0|8wIOo!VHRoM#?Hrf%$m^RWUa?uGx) zbc^_kV^s!;hU{+aWb%Dh@0`<*+&-EVZr7N(08pFY)72W6)#KvmdES2y!9cG_gXgN< z5C1W8qnMh|tovuAS#=QgpV@SP{}EaBxEmsmbM^%}RT(G(0qI0ou)J$tPm_=$s4Ar+ zjdGf3L5*@vn0wIE4!7W6nrZX@)@~CQ>zLq)tHvXu);xy_+z~i zp%jRBrjs$8v{n3z#UtRluK5Fw7vF4JrsD;}*FTv!;4&o=OoI6*B}f?giJh7#dmH)P zXwBe%6S_kJy;(-^Ztrg8+u4Ha6Z6rfOqQhZMf-O>d#lv7d%Y8fH(I&%REC!BQZ{nYQ0gFNX2GZgicc2tcxkOw|>0Kxe?4+X0#eI97gVyUM& zpVn$w`y6F?na03J*gf%oD=8MfN?L=v&hVD6R61s3(0eUg=#8i_c7xcK2EWyiq0@xZ zrYATha)v@Gv^qjGdlQF-?W{rT_OQMH@g4=S z!;Y=HFL_r}pQwpEWKF%+Ivw^lT3x&R-qWRpAwT?iF3+sSJ>r@%w@85RqCXxbdlz|f zei)qdAki24VH-L7al&`nILC_!C?N!71QNsZ2c#wYzA2OSE=3yp!omZvulsnyEqC}y zL#zaYrXL-kNk;TwQxpt7p`16)sOT(;`Q_H+)O-6HT_<43Q8!3GSqw$F(|t>)&2(5b z8v5uNakIxm#v&J^t1nDq)T|}-6e7|0Xr^5+)#gl0YjlNWV0w`wmAXW6R!DJ>bwBM~ z$GUXeVIVotA6ezg2eT!Rw6E1$sG7U6rI@&CoG{pbcGh35i*T@VsJ^UOzpslbkUG5lgL?J# zgLQPS1Cr&eaz=gD%Bwy14=g6GQ{B7zemJGRVJ2&xuQ2e}RPKR@rz@4*Wx8&>FCL#S zL_~9%$GqlE+4+^3_5BsU=~(FmwfF^?$!lOX6-P(Mh$=3#M@Sisrx{c~?dxz_y)!U! zM1@E;fSyD1`@C;fW=x-zWtVgk0T(%=Mx%{tmoD|Gp4~U>7I%hK zUno@{_Yn5y>8s}NQt10-AF5o+oCMUQgj~RN{~%H9)5cF#{dc45la|%{nIE^wGivp& zkXk;vhHpP>Tt&L%IeFw&4rr9q-?ag&K8lj0Jly>K8p zsN6=cOe~jg65K?gBs}D@D5OuxkiR!#b!OhlIagQve#O+sm5&cmOZy$Vm3wHR%K~y_Px7W}}bmqoa9%^g4*nVLD0Il*ubVK~6_nC+H_sK+&Ec zn9bx4X$p|(vPQUz;ZWo*x>8!=ASHSDZy z-7O(CO~Ced8f^!sTQ8zs^Kp0_yu8#fTkD1sN8EY_a)ce?G6H@jc#;B{dwM-;wc3MiV2Q8qhc z?1eM<`**ZfT2|J!mZWa?ugB1cb4c}a?2dB)D7vhue;XuakRU6$N1J{7p93PpV@*1xZYN%yL*P6XD^Ko`&M?DfI=!ub10NKV(BUw~~@#sEg}iXB@4*Lp${D z2Kr`m&q1Q#_8N#Le)J5`t|BDxmFV^&m^1Z1MQqX^*NnQx7--yAk>BM}A}k)NDWK4l zIQl|T?|2Er155NgU;|vl)C;IJR-oF%uA04Qw%}!$3mg+&{(2hx-g8}u_}jX;Mb5X^v{f2`r-IO-rCcQ4> ztDx|rFNVFPc^)*En;MdI4>HP2vuK!POEWnWSQ7&Ae6@}tV8?l=#U*>MS(G7<@e~fd z*K9J7@*HLx0u2ZwHz3Y;tEGQUgq(C&rUw^3qQLnvG zNqbD_5GsjdWRjurkSa>aG+|%uNJ0|5BokHLm$UYz{nhE;5Y8j^_N2_9rKcEG)>QRl zT2YH!5t!h=PpkUq^v7rj^%PUnD6$80dHZd%=tDk?I~Pt^zv{CxwWuJ*M&{5I)ows~ zG3%w`VryF_Ce7vr<{jm?SS_en&FFI;f_+i^<;7FXV{CS(SvF2}ttpu@SY)1*eq2=vi6`U6 z{lIkT;b#r#g%S#ozq|!N;Nz&s1|CY@ibD<-HlQ)^aac^0RXJi5iXz~!r|!m`dH!NP z24~s&RK&*j4Q~{mMu*OmU-ca~3uPj-^n$1WduDo9bd2>Tj|*tVivuK^SC-jHRyJLh z=n0h~5>#B+K@bRM*qiiw{H!3TAPfet?|J)e5kRryljHG}BGKUBBUgHiC~C4tsUg|bjjkx zeabL`%Nseox|YBb9%WRG6`9n7D1sgnVS9Vu6hCIWem0Kf-b8{j3y`PZ%P!{-{9!5c z&tH0ZnVYQ7FEA98a3(ilAy`{x;GeRP(Tt>L+3sl`LeKKZWxrP#bnzF~RaX^jrKCg$ z0qzUtVzfg~I%Tz}_-Mkw!_m=UmR0AjKciAllZG4Qij8xk8n9Vp3D6=9gMOXHuFP_Y zVr8tjE8->1g)$CFpZ#-Lkrp86W^JN#!lev1L~+6G%m*KR$g8ZzMmpBi7iE6$c{2Li z-SOH`u9~T>Q!RqHTNq`UqbpJy^DbFs9YNil3O80ouepS_)Kjk=N8ht5?ug4cVjZXy z>RDtyOb93UPM}Qtoh+w-))BxxQkzP$9NFbp!$}51pmKF=ETxX|YJiQUJYFmIP2FB# zG98FSM@I)QjGX-Iv;_+C#}ZCc6d5J6sBD>;tejz-LUDKy#258~QxVS#&a9Wvs2sWQt_pU&CzzaA6sem!S7o_TwblwZ7fy-5UtBUx^ck`j#9 zE@bHkdkbNPK)tufCC8(d)Xygrl%>M4I=h5dpQB}tiYL!GVU*aH94HgK4NI1c7Emah zwN7{@cc8Rg`7v|y$n~;O_2(D(+{iAK$*YUhdj06pP(mRX7HS!+o){>|8(h>9lvRn; zrF{Wl&3}~wV?nvG zH}_U~Hbo2sA;S3dQ`V;h%07$xf}4-XGt zYrS&h$dUem5%s45AZKm>i4%Y7x#v5-JbtL-YKoaCuEazJYW+k^0k31u#gseaU>rj7 z_l2OKjEO(BbLZye4?1Mi7DFSWH?FmwJaMe2t5=jPA!z?2B9JJVtcoB;5H!dH`=p$f zAqaF`(Z>6)uG)mSQwZ@7e)PkfX99j^7;i?^H+R2b?oCWjh!Fp ziV9PPDt11$<)Nnd>wo#vs6>$8QzOB*oS1Q9QD3VhUbk`EmaSV^C8DXD&1ZMNzAK41 zQEuiEz!+nkXe1hqFf+2gNd;MsEnT!gMmn#zUP)_g;<>PhlA_iCc=WODGpEmZ{V#uY zU6(btN#1i~Wr=s3SS-fO*3G6bAa9Y$LS#+Vv(G+Jr^D~;c|A#S3xG+LNr{c~gXf+w z#_ZYimRKl{H`El&StSJL|MY`n$1O!cdDoK=#VhNdc;bl#56p@i*QOG`|NZZK21jq} z-1eI6^>Rs|{a8t214$0|1z_2qS0v7drLuUH*VRob-O1UH?Ex7^Tz8* z#6kIdfym0C6N?lypGYaTWbpz!+ZRSWM5{tz=B?wAj$~xF3TQ!&V zd)B#E%Kt!Cn%jT@*|B5C?3vU5`scs6uFJ}$fg4#0BjmgAILI7N=AV1!i8>y7>y2G0 zjsu==fE8qjpMUOY$KgGD-hy(_8Jj&lYT{Nvh+M9G;rVBqVyWG){b}1%&(}3hd+pUf z+lC!=wr^dvkkw6Fbai!|Jba+71&B$B!o@3gVqRA8TxE z-1XPLrY$U2KK;xSQ)9{9ufH~m*u#rgJpas-4vEpUfhD+~9Q^dt&rS+Ek*bDYzw*+_ zeQ$qx{!BXi1r_lyBh=N=e*Ew!t!>S2U^pT7d|<-e{duN ze+lZWTEFSB01uU@u4(2ke(_R8lntz-X+RWgt=;|FU$0%ew&k&%GiJ@%_2*YpE{Uo9 z;SZjzmi~A4yq-i{VM>B=SYg(R zoI!l!y4BBYS#tjL$@f0~RMiNvCEEVjQ)}0(ax_DlNvG2-S1unu@bTbiD#fu!AAfAi ziboh}(MUR-yn4lNmw*;5S@G<%&pOB$Kb+7W`sBc&qbFUCJ+X7g)};@8^tX3ToWG*P zJ9qA^tFL+O_1zXEh|QlrZ^!1f)27snjt!qUcKqz;r(L*!Bg$H`ZXksvaw+!ELk~T+ zarM-O>Y?F*<42EPICm}sCqo~Nm~D?exn|88M>CQh?e1zn zdh}>pbE`|y6jiJ#ka@6Xy1Kev{nM^7SGMkWa?QHc5y_BdMurB@oIdsW@#86Bmt#*p z@z~rt^ZH-@lmHV&o_%^}MMcG~H}+VNOR@R$=RLk@?X)R1BcnqnkDoa6`6;UkGD=j& zrU(`o%h3lPd@#qu6UUC7KYLb)03+FSU4Eyn*FJ378QVc;{=cy>_KDwQB9!XSS_zNJI@F z2AQs|u0OuIYc#FV>V|)L`4^ubc=wC*r_%&)jZdQGGzxz%$c>^?qp97m{dxN{PuDg~ zeeKmhXI!MgV6))id0RJan$plPI@ovS*rBs$Pp330OqMHukR5v;efkI9Te+E{P#*H} zn1wqH3qpSGBIc;Beo8|_gC98xw6(RB4dGYSY<8aV_=ZdYrbF`A>iR^T5D>QL6+cN02o)fK`bgtS+kybyZb@4O7cd0f-_N#PC!HrtC&m z3rDgUWmrM^_!$P66)v;DiB(qidQ@?|u5s#;MT>v=%D;T{-rHx+U4Une(`=4?5~HFd znQWAl-~Hrobhn5CAh>`ic^*zf(1^GIGVA~_>IbkfU?2imasW0efOy&}Lj-Keqf-Ku zJDUCQ7?OJu1xA$+6Csd&CU?)j5+YXEtj9KB@edIqM0w;ssP3*?zx~a>#qAiWfLOtn z918>AiD3Xg>9ucD01Hu1SLg43^P7k}%78anUb=Ltr?-!Y)~$FXrea`&Q=K>Jc6W9B z?zjItYR5=z&h2lB)tJRgmyLDAZ?s$@vJ5UgyFtO82Qqi(HVyRt=jAh07T+C8fb@2C z{qDEFQE^pcxlKJ1*s&sDC>wPRGUTlZ)uYGD0_umhdM;LlH%aBF1+uZY0S-uZZNW00 z@uV2#mzOJ6te81#`o|xCm>C-+n;98PsOP7Js%n5&5?H2EcW1}%e)F4%J7&C{QQ1BD z0q2lp>&rQfsk`ge?|%Eg<8};UgR;SXc(NrTFC$*?t)8y#-~IOABkm|?)3#l@bg8Sa z-+t6nzSjO@S{|B=f;9!c)4U7GTMcxKL4>bB_z%;RbL| z4i((j-SO#xeb%X1xNza~&p!Re8;RCiJphC-JUsmIfsbrDA;bhXlN?UF8H-wh0r2qr z2hLq+POIb$LP893h@F`;XOE5z&ze3ZY9u8RsH^Mf!F?%2=ggV2apT5RDsk-0B`CXU z6vn6+wNMLkd`PK;*yC0Or${`e)h@! z496EPT=?Ac&%E)*8&_}kj2Cs4Ygb=y=VzbnO-ZD>wr1mo)lWVB?Cv+-O1mP6C_6J} z%pMyZo;7`n;|N&+Fg!f`*1PXSnd>Sdk8fRl>csK(?p{&q?(QzC!jfZ&STf$&TmI|= zbpUpS$HvCy-Z#6iySHiDbXC<%5=avwDx@Ydeb%h8v9TG`r$rel*GpGy0QB_res*9l ztXoxEw{iXI=bm}$wcT%xiX&?Jdpi$*vNx@f^s&gO001BWNklt=6nF93^(kOSc1hvuFAqB$cH%!fE_=8Wf_d#=CjW}+kW}N@gs+3-Zy8*&L`7jqhB^(_2$a{-kzg}_NRoE zz-q;H)9Ex|l+f4LckJMPxGB&4r|GeDIz8$AUxmF)j)Hr8I*uIJ=bGI2^@jI-0qE)J zIr7O~2X?_D3!i!Jxwm$`cI{@55*Nult4$`8uIs9>!1na?9Q|Y;X=c`(`!{UdY!kyL z&VHdV0>f|(Y=OH)0m6(KGk)-_&%^06XKmf~XqJaY#*fXMF=NIv&pgv{<%<)aeL7># z?Cp>3OpgpVH#ZB>1dbL9$Il_9ma@B373>y#%j`uAB@)R@#*U7T4vh@YoH_II<*QH@ za+;>iOplF@4D>5kyP%Pw!6TpS9m{w}ac7drWG0g#iYgpGHcWc;pve9A-~as6PhPrs?vn%i?w>Vt`;Hy1oBsTZW@y3KtJY2?*I3XP@xZ)! zAZcxFE&XZo7#J7;P}|S|5KowGM^v@karGX)vBPwUlvOihgYDPbl2n<#(RHnRMIA)YjJb zbax!y`$5aq_OuaYo;`2gumAO5tD0u__xGPTwD;V(i-MSmIKO`7=glo&?%V$vOr)}= z>3{y}rNi&Naq;5C3`Li(T(xQSvc`tmRL1S=={~&w{nl%@tcfjOvAW>?zK>h4-_S_) z>NRWEE?eACQ`OURbKl+rSFd#nMb>Uyziio(rurIk(+BtMJ$n44Qq+>;XEF&fAxbCa zx8B_2P1v4`&inU=Km6f0LSBdv-zt>zdjuiA3m@VDM%)sGlPw$}OJY@qw*g>z>h-fnWvPy_%Z#SoBV;U(Q*f(t+qfx>~;U)jJ3%@Pj= zU?9Fu2J%X<z3ixxQPgAS)XnaU>@jdAoIb-1r{q_@+MJow;gW&x4Wg_+oft|;lZw~- z7OG$+0}>WXP*PDa6a2RLn^J_;>p^Bh;2e{Jc4gxWax(8a{#%Z!afyjcR_0w#=Aqs( zC7i1(C-eEBU9gg2Qln`@p2@>xW!`u!%26ofc(s#yyHi}R!@OU!)&a?6GA_9g*JfO` z)~YDMsuD?X!B%eZ>sgnIlU%sTv0GQ0N2Id#>dl6K`r(45E85!ku__~-$&9qNTuLWX zLJSf>EMk^aiMx7-8mG;u`~TT{)9<*BBTqCU^WIvpgV=WhT)=$+SCO(rnU-iuvTS#^ z+|zEk&z$$pjJw-DGw&S#DRbWY@ZRZjUQf%GM#RULK5X6qcRt2Dd zszNRI-i&x3@>bQwMpA0Ok$Z;m0f(%rdv9iBWZsO(U;Lsl7>zGzN;^aZLI;e1zo53F zbg1{?j`cOc;*wa*0&qixRx&m`(7V9F-htuLvckr;O*gyyfkCI%^Ku>mL}a4O!pLB6 zECUW2LYo5!fJO1pWvevhp7nTesHC*0v90}PcfU9J!mA>QEZIuL=N`uwsw+y>zO57E zqkY2@4$387pXLeyAkMpxMNh7Lo8wQ%mdJ&ila!fMF%Vm@{_NQP%E~p-utFv~6%P$QULeE4z--!j11M%Ly5SfBt2j-8wv9(-&oygw8!D6ME~ zYwPTO09o&v^Yw1B&0^lv*5cKnSy-CY;UcqKbQP%fw#MXq&n-Z&WBTo zvZ+-CYkx}-MgJPzF@@Z1qYxI0HV?8tuHSp$+u_Eo;Qyl|HohddSU_h?LTXW&uZ~SjZ*Jf4=38(4;SVQz2O@}~v7s(DKXc*qNz;(~I(8p_{mu9O>D{?lQm>6V z^G^auCY=XpJK~EkZ|_=nY(x9Tq3aJ1>_2#5_vWjge-@sdJFsW>Td#lrAAkK`BIO?C zef8B>;n?+c>)VbUJ7(Le>(K=0ZMUX+C5?=Ak2ntO*?s8cm+#)VHaa{UC@FvS=*wkg zWpX%lV9%~YFMa3kjq9Vs!~Wuu*S`1Nva&Kc66)BucW1|*>z|*Wn+fmR)A8n;Z~o?= zK8Ph#ZEKqrlJlRPJ7q8>=AwwGP|t+tage8rcW&eWT|ap1EdVz%jEE;rocI=7_*|aL zzb|3tU;XM=fKr0fd+*k{b7!2z-|pVMXWO0^gkkn{-MM=8YK92W*t~h`@#Dt@unC!- zd$-P?JEK76KyQqKKrW9?Y~H-}`>!9D%m$|)b$)g3(=+;oQc+R$lb`-|AP|(wDrhL* zyKAc>#6)EF{r68u)AzG?{^HJ^uWsG=N(gkoXj$9*lhSZ8@&}iSh`3J{dHPDe@cXi!6eeP1ei9+t#v-jYh zozn2#>VBjTB?tt8kI~q_XZOCnyZushb>5W%0Npcfg<|9S4Tlcwttu~x#pZ6@yx#NR zA%R2eKk~|vw%XFr=-{>;dl^(u*PSa@uGo}U@A47x#6cF16bMA&Gfg4TZ-4>H@Aoq^ zGNo)PEUx(LpZ?_brPDp#T^1OC(9+WUgEx+U_`z?dBH_)OH^27!8xp}FtEcnE`E%zS z9WMaMhbiOU>^Wy~lXmXz*uM9W z$2I@}Hf`Fnnr$Ex386r`Xdut7*S!jr51dDLzdU#DoCCnj6_pi#`Pc9Ge13!Mf`;<_ zyS6z(OhlsZzkkB;`QG{Kx9{A!dF#ekh(Ktzu5Ehz_;>#0pWhpc%&R5ovCHic7WIn( z9WcK8%FBAZw(Q)?ta`idT)A?ESrIVkTwv6Uf9@Mwzmm3W+4AQ1U-hAqliRiWY4I4Hb-rrHVrX)U}vw6Ybf7xU6 zfb_8Qw!3Zs0VyYwu?|I&RNQ{=VVzb0LZqS@XTyUX*p+UPIB>9 zZvku#{+dgqzz@w$b+P*oHRwRkbcIG~d0C-PjP~_xZmq7Ushx_Az|yHI5HXXsg9Xn@ z9LCIuLa>g=2=zpkn`|Cm-es~ZO9lhr=2M78nMB#_6%E_T@{9nuxm{298az4#+g9P3 z*^;udoQ1C@n)?9I0rnAMI2v<^ii(N=rU+2$a5$&K`4EFb%+^alfk4{*a$!&iBxIIZ zgC#q1sZZ`{Fy;aMQqc5lw6?Xeu&ko4u4ZJouX#gFj`XymwAdf;^$d)xYi+Bks+^C` zBAb|HuK<^?!I+NDE2WBy3LqS3P?AxA17Ik`a5QEyl@u0DGYNK(r9#YUm^Aw`GlQ6m zr@$|oo9epnKXgOGAP~5uAdp;$IgUjrh~UecUsV-q^p+u{X~?V(x%mB69Q0=v;u~bWeUx0&pqsz8#A*0Y4crMclkd993rIlcXAjlKs8RplVo?n7As$KySgG@PM zYBteQT$D2;$gWO;1qxW^uz%>VoLu6teot$o6*B&r^wquz{wN+J9 zy#qwTvs&lkEG~voEvD&UW&yc~HcBatEU~zb$P+oXB7v+|QA-e`v@F}UZDwXPK*(T0 z;rvV=C|iPIDKRxOx2CdMN@>{OkZq^acF-X((v}^x?Pu*Z0`O8&2(LViu`4frzrV7& zIy^BM3qlt{FZCTep-DjF+ zLsiAy?)wfCA*2)Wndxvw1T!KChCrg(nX&%f-lQm>i(CKuPmkBs*Up9(cJAE%;LgoE zSFfZoFd7;!DKFl>?}fg>PXv)9D>^?tJUFD#2*>Q7{q*g+n(Br5o(u$*G!GF4feDnD zi^fx_(#lGIVbP9VJ8xb4?0)AR2mKf3=QjSwx0{?5~RsaB5oT>tUAW*P>_xeY7uU@%y$wt4gICWT=ih~9Iz=0hb9^L&S|GDE+ z;133adv|QQ{^iB4?oMEgCa(Q&fAh1}riO?2yG=^ZMCXPFhxOi_&}0xSSit{)%2NOC zWvPm?x#HPVS>WZFm(S(7{I4orWWh>@Cz=P8fB(_PY!tk(XE%W31%(1o<(jhOY}u#h zKd-58IIwF+EH-=V-a|l|7#%us;si_c(B9oSD5*E~-P?xThhw3k6DLlvFc0tDEx<9z zcF;h>~gK~O@##%RQx_A4Y?I@N;+qwtV zX%t1@d+m+wn>Wa8UbF-i3WYxSmk%?945GZeJW&!k`N>CBjcfPqIk+%4-QC%#Sd5L1 zoH%j95PWd|&TR0;O(AJrxAxHCgZ*9i1_p<#S~u0y)u+ay4AK(*wsjj`I<&j@L3i)K zL_>2^eQnLu)C2%9U_(>mYsX*h>+HPz+2iV_*2CXDPU+O+{(&?CAy$?bjSZyUefKvF zP0imsvVU%F_IB69CuH}qK*&~7%;OqB5Xj=9(s$nZX;Fb+r)vJ<7r)SF&j0wE-;T$V z6O+^H*ET)wz5@_M#Wkq2|b;jtAZM21DUMLD8Xo zI}aT?bmr5Gz%UjX`tYL@Ak0Jib_$mx5J2b+8kLuq#@Cd4Y-8@$z54*5ZG)tFXy5K7 zFk=9h8qOua_Z)wDpM~zT8~`H`kDmJEBd=}PU?egs<5(HxJRT0Ut>1L-_C2t{GR(Gh z8)qWn`I!iU;W_47nq-u>!+kpiyF7M`(gvwKN&F3moJU?4z+qP)C3)c5GUfBJu`TQgu3UUv)wuS>qpxe5&xnH6Y+f*%V9!M(B1I0GbMtW} z{N!e`dZ)t*7=&Ct8e^8x2Z24<2JKoaw=9Qb00x85G)zAblK>#aP6iPL5Hpj)k&)4L zYu7$_a6dow5EY85ni@*RqElmY^NGs3CVm`3AR!R|NEpHlO7b-~H?3b=clGLJAhpCJ z(*}^AxhNExh8ZwX7>t4~$8kWXldqEgrn#93efej#G+e!U8M34A1-CB^i{MIkTrA!e zdQRco!aA-FQ8K#~D4s}^E0_>a6bA!_gmgO1ARM7}I2WB7i_Ryi>YDiRxa22>yI3d? zEKQ%^4+a^+Fp(g~ESNuSUZgR>WU#HAbU0?UZrKf_tVC2mwcBi_`kHb(m6(deW@71@ zy6XPNJu6m=FDM9_0yCM6N1txlqvD{?G%#glT;{=*E_Y*5Bt}L;t*vVxJh-1b*w-XV z0*Dv^NU}X0o``}4=(9!O`1g-*+qB;M{u&z_d;g#RU^62!v(iS7$iM`P###`#G_YCC zu!oadAj;gZGIOq6mq&X~nO|<#R-a!>tc*Q{iW!9h(=ZE6Kmeg_-UdUlB$QzIgc0zA z-P&BczNO~YEb`x-h)|YlSD}=?0zS}Uv?D*Aq)j3gG5l09sP-h*&1xmMI$oJ z4x0KZwGwl)x8tB~n*jg?qlV*p%hfGs`gj5!#4wDS>guVfu$4(!nbeX>USBI3@-3r6 zT3T9u^{Zc57<7>5XC^=S_z&T!2|(Ew3O8@udhO28nzGWOfPZ{=0CY_c!5WWB`LcNo z#DGRTkjk`fFoT*v5ckclmmrWPdPHU zGkX;x0I`$F3=9mAEZDtkyFDN6f83kKfDRKYkPrKYHnbnBsjQfomgwkg5^4L&;_dX$H>(Y1_(kG-Q z&(cSQfHIj(e)!>sep4LS(-H8Sm#^aom->W2pV z9LFM-3yJt#U!Qlp0DuWuLV0<4I-S1p`9w;I+*MAtBt0Oa z*ET9k11)W>Uv)k}CT$zG5!}9QtC2nJT-?KDXl;*e?AW?hLe>shY1>dn!R}q#GqJpF z07@e0c|5q`^<%Y_72^|ALBDKnZtB1LxnnC3K1Ly>lu{yqU?O%L$09HUveym;0D^-U zAYecV0Eu|?-tAlS_7lg8g}XxpVBm%Ld|%e0T>dZ4B3A(tiFntY+Y1?I^5NsQbsJim z8oN8Y7;xQ&O>?u6YnMJ-P{B|rR9987slC0oKLm*7!N^#%#Ol5V|mW}HM`ycC$r{Z(k z=A-fZ?d$3rTbl25_W+P!=ic4hKFR=!&hGoqLiYmz#3-dq>^L^*_*yRSDf>5{ za8Cdg6%~K`w||dhlK^1*?8%S2dIlI+0!oD8!TznsjusXbFET|b>gsD^v(u?$a%yI- zy1v;EHx#i>5Nm8~{QF=2(#Aps+}nBW(SwJ8Pv4eTR8;(@zx}xbegGlCK6&zFS5Mzl zukt?Yan~1*9y|o_AxeOyHAf*E1aaAmzd=17S6pJdCm@g9$7%8xX&AmhzzCF=ta}oX0ENS4u#4#UZOC3W&kCZ zm6X1CXm32e5ROCv3W42z`8iltQvT8l2U2sh;mEWStCr}}Dw2T11B08tf25$e*b}QF z;j#(}!!U(F1VU1lt(espUwqMh_tuj47v-=7xFr0MS3U^C7YN#pG=+&w+P7AkT}v1~ zzh6qp0D$B;mQqilg;;TZ>6x^6e(8Vd)q|F3Lt9KPYEV&8m9u%l*2$A6yC3!IUpCY? z{qmQ;6ig1_!|uBeALawitE6eT(O!kZFbpXru#H5HL$sPLF}iVf%@p~^5Z&E1Yu5by z=Rdd5?boSOr@DIj7-4okxsbB!YpW+l2dXMc5L9G(#zCK6X+7EiNzTdBr=Gk55b3?m z&BeBnA|eD7>~!6^ZEyy`vx;VigHY7f)xGh14(x7-Tlf(R&MQms(-Y0 z?b;lyF84_e8!|LB^tBY|IL`R^xKb|KOY?@fcmH>U3K7gxlcCSfoQ#t%wqPwR!~p=% zP>A;)jlA)L9hDX3Ya1#uiCA=cMu;`5-N(&TbT_fP|}5=^M<3zshR4}=wHGJIhrk~aF3H{$A&js0;Z>i=qzD}nIQ zM;}2pL58zmeTxlyF3;spOrEsDAWFA{;^OrU5d~s;A?a8d1Y{Po@nmgvr4R-yMF5(j zg0fo07tL%J^W;-Um;0CYdkB~1t*$P!opi_pYuESA~0p)EWyR9jyb49Nb0VFjjS z6`Kvm-R-*&*$RaWLSuI`sYJ$B48mqeB-2HvDUs1#{1^3gqsl6|$_f<5&m=Go~|k zFc_r8h>sIB_uW|1R*ELaLx`|#^9H8i=;)|E{8Z?Z zKw~4rd}woJC|HQx7FHrNJF~MhB2aqh$oC&Td@wT| zUWm^D2++(Um>(BcSDI3s5tyx1I%gZvM5?yB5>W~QLU%+whsm7fN6>%};${9?_J1qz z8VFzqG(n&ax*>sJU;+`En4XVkHf(5{ni#6BD=#RJ1A`+9{lup91>XLvw`*%^B(Vu< zAvW{wyZ=3wQVcT7WiE0WXUM1OIGMELSON?pOlRzZKycCUKprQogYEh(gR-gAoP~M2 zXRqYgnUv)a38@%UmR0668Hok@0{-ayyi$zd0K zKp(COP)zzPODv9qc+Yj+j0tTl3fL|@7 zHOI%y)YRwa&nBU;uA%YJvOL9Y001BWNklN; z*Hk=w+(+b0PEPJ=Z4VR{EGz&35*k$v%|HM9Ur2TsX?||@%qM3T(#*nO6xH>O|LGUM zRNx21Saka2$&<;9#b_|XidjPlAlbF3q~%mMwdkN~p3?*b7=br__~W%}n|+jtPCWkL zx9@`zOWt2uT^L@r2xbPC_BcNrm9>q3`*;5(0|o$?;M}@(>(kG_0FZ=cbzRec{=2`I z>@b1=zJ2rB#fuj~XrAe>H@{%z`9*{T%e=5f3d!1hLP-Y0wRskRESMtUiA(3s6JS;2 z+JpOd?VKFGeYbm2e-T!i8Ul>K>u>$2wYAB|ndn5{Z{Po5fdRnxG!)d6#LY{3wlpGS zT9}{tY zBzWRpQA@7$2VgF&xf<>m0DwVADM{G?K;$f&LAi&n9PV6EzGYNMI-MGy2&JLO7ACXI z5^Ld5XmUQD+Ol;+!P@cNl+4O1Sb}Yr7W(hZ|><23uXi*Ss~f+cs!Y* z>V}q{{;*)>ehyANKA+AwWo6Z~vyT`hvjPynenS{2zJ0=%{|Q6_Dq;Y5>&+i%r$YZF zPeH!t0g>nOT%I6HXm>o(#r(IJt@FvujDk@<0l$`~JzGVPT$C{_2JCu&vZPo5GdpR! zuWxYo`t@Jk{9;pEV>CQAJ0DYM5DF=!Ar05@W+ecL$aYAd#cB3U0?a2=>$6iA_PMM) zBccFDpZ)2xuiSgf)eI>CfJ_XoXQvB=AkfD38{7ZOe+2*sO%rK;>=6kGinVAk%Lr<6V(i?73kUah{AlxL0X#K6diLzu@XQ>b08$|3YU9PM@}UkCj1mzU zoSe7w&7h7fmhPzB`|xr~KgeY4zP|pfZ8XeIjn2-;6c~&`InKFr=S;(p#D=TGG=Rx2 zXd2vpdD@ugegi<4GE&om2n}1#ZC*5_+;e0qvdHkIxl@KfYK4Mecgu5zx;(I z{0zbrYGPu70hnoOa_s#1PhZ@#ZR?glXXf#V(1o*SXQpQbfU|PHB(TcaJOd(1Z(f7Q zdynqZ1q(tJsAGr<0MO-dS^NUc+B^y*1Voqa7g%jv-`@UT{|^9A!e@XpKh~=-pH80T z2L;d^5G*vl3iITzc}*^}f<=KsJspA)w5!I-PN&m|{;X}FcQc(mUnT^09D8zN%*vz) z1!Yr`^^i$cmX#S89bda)vz<)4Oua~@WyOVsg;Uc}mcE(Uc`1#W%JPN8sCEJD%)->< z*a8=ZCMOFE#O56xT@U&l8w3Ea?i?z|nj9aC+0sGZ;zW;C#rv$0@$rHJv32{d`}ZGP zmIDAP9{~XXSdx0O;?Cz8>Z<1tCcUHp9)EtROixc=`|M1bgk?F=$dqCtGzrnG!|B<1 zDUI67%HhPELLpcPESd5Yh^|C%tF5U$<+(zz)9IwYI=H_2o?CXn>Rf@efi=! zONor-M5m{Sh|qjeN~88>$+7Ho|Hydjx=mIpEtmuo=aK`7FMj@c@J7L|t?QdhK`L8- zAF}T+M5*T>5jkEC2V4ZA$#r~gZtn8=Gbs`PV1hG0KcD+eG#b79=_$4?5Qt0*v6&TL zw$C!ZvgO;+vv92(906d;V^{^vd9(m@QLUU7@}l>%uJ5DeEIT?o59`NHZ+y6WJkZAHqS<-=c$j-~tjY1Q zc}tKCxI+d4h>6w2_|*DsO@UxReQi~EBIJ-#tQpftIqAuXkh=mb241;HuI%Q7T>@z- zr7&bRodO6nIMgA?rlqWEkgTSrrvBj{{y|A&=bBo`DpN3Umj3b?WoT$9hYDG{b}eL) zAh~*HPmr>*vbniA2I2SnYiepHC&o0V$?8{pjd<3#>_tmJ!TOCRgg^zXS4`Vd54yTv zIkd&c)`>5!68lN1>FMb$>o>0-8B&guu&{SBwfu$xF`Z1^y?gh_fqn6GM$@LY)>VD@ z;YY-Z#i9wkf)bGk(JUyfU)!ccpt!7J>(;GxHI*06o(j*#(}-PNT`%q1YdLCYcvJ{c z67+RGd~5?S%HlOujcsj?EZnhUTVu_dKm7jpM3#-_{eiKgue@~V(9C#fG*D6&khZ~@ zWP0IF@96e@uPmf+YHF&nZDWa_eUMHj6P=Gow(UI!+|atA#D_kxlF8)F?*4r* z97@|FG~8!M3pmeoTn3>Wv-nm+;l2>=15J;kJ z6F(jr-?e*BY2n@V&Gn&?e%sF0_0-!e$TmLT3<8GU2i?67x=ml8uCDIrk%KRM=f#gs zoCG4}B8HYfZ)xk1DW@WK`p!q# zQ$y><*2dbe?%r|825AB-WS*Ir1Q&~>5D`FTYvn9E_11xoTBZpo-eDISKA1V;Y zuv)+Jz3~{WM~OO0vgoTNHlG{VTp#s6a`7p_q?4y4QqJZ{N9pWDh79r?_Tb`Yc~s%W zg)t&#;|NQbAOHXeNh)i0a!g}+1OOs43xxn6L@ve$0vIJS01-QoCzZ$oVGtA{fN;4R z!DfWVL*qwZ-99%qV6crq0HUU1O@Tk~`VapCfilt*dcJ*Y zna?yc)>8-wv!26hbt|$r_VnHLUVsoBMBl{p{N^nsKGU=<3jmflw_ZLbW|dbs(sD?M z_c}mAkDYsU$jq7z#|x463Nop5Vj?t@P$r=vKr*^^=+&VmOhHszQ$9N2fH0q=Cd`{3 zqg>Rkd;`4P5S~lHK{N;igP{>;grdB%Cg3mp@ekhm0R#caN3rUf`bQ(v2rSr9sl-HR z)Uh)l5I_QgJ4lR(6G>iqk^J}#v8Pe53`-aDlzPf!rH0V=G0^2N@5m8E& znT>>P47Rj1Hx&CBO+x?CV=HrAS)>U*^vh;>nhZIv<|Uz_ftCycfCW2NDn1z+b?l60 zcSa_5<2dwn=NIche=eo~D3JvaYJ`5SHJD`>K_JovsX#EGVAszl&v+pr>snNTKt#eb z1skhP(}av3cQ<;f6T#g36Dg&H0F&+6@I*!g*S0j*7d?ggl$moUqd6UF)ApLr*9r~f zyMS`ZwFdCi)ReJ*TWM8IMR{>|_g&1EtHCp%7$3wh_ny;h;~Csq$z-0Q5Y(|MK3} z=)zoRGD0Yk)EC#TR29Gc`ddF%ndG%E@AXaeNG2Pla!s5V3}PvnZr!+!PICX=y@!iS zXTy`9d~|YPV2Gd)K~wkZSpk9-GAT41AsG>w5F9(5uB>YN$zT1Ar63lIkB$spICnBK zHLWp_Uw(1bNzL!uy<^Y*7s#>u9zE#lc`V2o4~^{E-128{|I`LKIWhM8-~WDia0D>O z2>0%F6&4om@7Vj&i-(h$bTkr~kH;Lq&#zp!(b%`|KyhI(6pjEXJ!e0^dLs?y{{08t z&)L?*)Yq=xTCnKAuB|WZ-|g6`$Gw9OdWMN8OOY+ScC+}1>A5_s=%rjk%k=I~*P;BD zoAq3t%fAQ7uRof$16TtnBr5~S0%8Dvuvi#Al>wAkRvaWNt&}1_27xG%Nigd@jQ|$l z02>ITPm>r;CtGY)AB-y$3_<`&W=26EbWJ{$b@G{+z#uSkcTCM1^G21r>93rPl1Q&|-V1ja(6wrvlNjPyLJ+SOcUNYF zJP9z|lnBh%g#;$aM9j-=UL+8R8C{!4!w*D;;L^f^R4OTv6>~ae6$Fds=jVxtP*hZu z`;5Z5g(W41T!jGW^X;V(u|+>vqG-5Ni}K`bP#`sQNO7qByyPpb0bU@{seGzbhvU2H<>aDZe6a2N%!8-pO%60$8M21)GB zteg(lL?U5)A_F}F0Fq=BK#0gFl{;35i*-oS|I!Ql8tUtB8~EhFw+9aJb+>RN5gi@v z&8BFv5OodpGt<+TK0TdOD1>q7g}v2{YsIYxf;4=a0No`m=UXBoAW1h#69`1s;nc>4 z3_>y^6R-dlf=O4p$p(L6A{@08)>Kw5#G*R>6;IkVwS_)YD$5~BCB=nlE2W4PvgW$d zp?vM`X5U{jk+y4U%S_XBZ1M|LRuo7qLt<;(0I6g$Szl32(o`gbFsjN+>~z9$99Vq(1FK-i zp2ZJc?G$Gb@)VJoOUlcN%St0t;Z!QM&M-Zj|*;b&kgz3|`uum1-C z5Pd+-?VGo5-@2ihas?2e$v{l>{s+GUAYcQLDH;2}|IhzsvIQcj7cf($&!7J2^HcWX z94FFw&IpPf<=z>xW(-22b^Y>ZH!nH4ENICD!msZMcIw*apIyIr9)Wa7RR9o}%Cat9 zxN!B;PXVB$qWo`u`j?4?3^S^9`pU(N*Uq2MeI83@00cOh+c$6Cz4?V_Cu9J?dmsEx zhk=&Z+j9s0-y+L1W}nM*c`mC7B2qR#1W|5W*TynX9P}})XB#K1eOtG7 z9E>Fr%sewOHNTKtQ`=ZrTp@(i&sw*>oroqPv$G3n002_4SZrNe`}(PIW!bT0Dl{>R zAOu)S8TWe!zJK&Pk;k_ab8`p*L=u$A=2~mXo;|UEmCir_fIv8CR@F3>7X=NVf`TF; z#QL^%z-o4SYBm<@8=h$2*74FyFAWR~)VFS~EGde`8G#Mvp`oE&TV6Tv%ButYBWqil zs>@13$piv17!Mu~{piO#4jei>Hat+aru@*s0}s0HMj|)!kH-7~3qX$KJ1^c79_`j< z_tx(INuQOu6)rgsA!dX;KMRYQXgnT^?cK4FGxHOZ(+>I>3}R7P+fYzaVi-nAz=tK} z>(;j`cE+b;k+`aBY&mjx-&eP;hr_dleq;Nl^;1(*M5NHHtZghRF7=L=VhTm#sT3hf zUqR6eFManxU(PnJ04YX)MNNHSX_*kBDCk3~Shs#Nkuy0RosFju30X}~Pp@Dbb+tL$ z0A90heY;X>YG!UWoi*iZ5y<0#8c_f@dzS$ZDZ>3?S=VyWd9TEM+ye4tZgVN zE|r+?7ZZ(6$J0py&>^$MXeO7Fl8JG0cK+bT%{z8?q+&CItnu*lbdsxU8p?};CbBer zb5!Ns8}G@k6DHeDw(ZF^6DGUKm~7kj_jeG9zy|;D#Xszm0`@5gLAAHdG z7lmNaKY#gZ&r5*X&#{#5`4Ow8N8Q)%@}f!al*h}&&BRN={n!=N1 zi4475Z|4)_`~{2TVr!X6Kr+h1G&X3dxpOu-yUt`~9wI`!UHr0%dv%eXUYr&0iVtV30{crwU(I=>-+pS(Z5Xm|(BSiG zDV}J;Uj#zakM=#dh|z|CeXG2(a=puGu49rvU0x~=Ee%gX(#E^ol-B|>j?#)jAX65- zKt#W1#W_xUo_lD9gI5nC&~oFxl>0>lCJJiW6sw+ zkA~wfA0F@LxtlnCqfXMEsarQ2uNQ?efmS*63 z^AO`VN=!>_7Q|m|thYh=i}0mu`j5HC8>d(SOxY#EMPcQ>mRVYjMn=9tS`y&Pi_1?* z6(1xF^O9@F*V1zEUC%y*1iRt-;ntWNxD8!+Z+hM@Y)GTgTb(vmST(zR{F@L9$jD8P zIHVEB?&*i6c(@C7Ipe!pFBpP-Az9?&YFMo1^}Q*B3W2RMg<7a^QZktFLjTjHbtfBx zOupJ-o~{oF5`#v}SZvgqwIYxQAfiwx&(F?&Rk5+KtEPYSebb8V6Pj2c_YR&oU72en z{+g^P|?kTB=>V=k*u_ArPaHw@oudpXO!08@^9Ct{ZOO0wEpEef##|v==+Zu z?^D<~a>Kj3*o!6St)s`z$L2>-XNA5LXA}+QEEi1-87#I`o?}Tq;OItAEm0dPq2SAj z$YkUTsn#im_3_$x@*i4+qB z1qDeQ>~ou&Mu=yiC9&85GIE$n-1=*c`H*%svDf}~B_+dkqoZuu! zPB=t=Xv@OAxjD^tIE1iX2?LEAe{3#=y?B6+`JgbkO+__oN!jZmlL?eXiR2Od(=?6BZE z*m%=-J#J~){e7c@3D5 zqlg2_=V*jt9?`%C7uGAS;y_vD1i7C-Gbrbd>vJbJE=5TxPY>=-8;10J&Y!el`QD+# zKVL%Cm?R(#jEoqPYdv0JvB=;d0N{QCMcn{J%K1DMKWv;>;50NH*v;L=dV^LAf-h`o zb{4Tv==E;i8uSiU%xA9hc5gO5ZDiEG-0fS8}AwFE5F14B-zF$mE zKbhE4kq{8@yFR?RqMivgyB>;tU0+yuUWg_D(DtDJf%irZF<rzBuudZS=h%V)U*GNib;w)ww~OB8 z*_p&05*fJhUSK00XRMywP+^mDmc{Gp(|DR^kn)H4w=<`DO8e7|%3PBC6D?ZG9<9)kBm?u>RcMO{|B)BqAc>=c-Pv03i5* zv0w4?!SyBlZEd7T-z8wfzp-G#i-mXZ7KocqWeY#`e#E?o&W&hl9BP24VbYVY-PM3RN{>n1c7 zTdehZAr({r%G6N!(U+h2Top9UA)1k^?0z^Vmq#(U<2c!%Vj8U zHeNq7>JZ{Z4Pu5g0j%_9WHVCsi{BSJuXlxAhh<8zTlp{7ntc?|Yy3t=O6$$1I^TM8 zCNfvB6Ryisg)RwjzZ*b?4GxBI$97SR$~Hcbyu_Fs&L&&M6EvaA_))~UxSnn9{@Nx{ z%t3rqAX9cXodD|sH5QR#r1LqSfE)h-!TQqF0H0c0G4iXq`4GnAl z6+S&w8osAMgw)ouwlsAVwi~&SpZui8%u}Y7#FmJQv|xcGm9q!Ri0sD8|RWb}t#f(0O` z<=Np$B64$a5exgQK~pVn_#V}mIG>Lel@c^jEkR=W=S>3+K0Z2e8aaXAbSuEvc=wmP z<5c~n`@xsp5tExStJF4o2wut z3(~lY0t)0Jb%<#&pC<$Pg&?wAsW{@KmD{I%m<#ak(OyBcH?T{FQ@C z_Gr%UJpcoA5HA~m7)rD0`H9vj4p{@X4g-^a$r8SSpU}m6hx;x)oQNN0dKQoO!C9U- zv-R3PsxP^1UeAuczQTS=HPWF38#2_1`Z{g%?=>JPwLAzvF_p>T^T{k; zM~kWAhebD|3$XaX=`q3gm#;~X%Yj3#)kewz?5f=tF&&&PBqm}KtY*BvTrQ&`8Is14YP0QEL6SoKvb5acCV87|)T-j37yfwKZ5awY za%einTa)TR{v%c(UNV_2>~*!ZNaUG#oemB`0k0=u_mi16s$G_ay}f-yb$ddj)Gv8D zr(zB8^qdqRDl=x7`UBhAVlaQ&;$BbUer&NzMAUKG}?dYmXm;W1*mr2o(v^0}$D+gXW z8OI}$kHg_)wrQ|W-E7|U@V;DXaW8v!KycyVy~AfA^oT=N!vrI}w=|f;cI{U2r}+qh z8z>(xk^iT8@3RC}2z+x*NN7_7u)cf|OpjAyu~}<&FkygpH_3fP0l?eqtVm)Z%EMSU zHvnMR3I5Qd%F#yYyVP~t^$&)gm$xPU}I-pF{R-JHN z$Q6=zXLP2E_3PwzV3+43$*$8ar99e&E5exHRi`Kitp*vQSs;-_V-K_lv{|aTsD6c# zZcrbF3Mp;R9{{>m9TWi3TWgK0<+$!A4DF9b(m-3@8Utkv7}gar@DDPI7(ICJ!iQ=6 zMr;AVrR?*v!;49fLnTML|E}E(fLN!6NT76WuNxt{P(j0hO7Il&E57kq9LIN!Mvm%T zEE%o%X9MtCoJDMe=pV<>UD1W?A+4v|hPDk!m^|mMM<#L~)wajXfOg}kuz}JUt~Kap z>KWw}!;1<0eyiD~_<*WL|O z@i+q_PXheJ0?G8(#iy9pnZ5gTe4SAsQPjLGjU-X4%7GBw(p&)Eqp^nR{mN+{BY7Kb6ER?`MIH__Ip#LEtK0?oE9DvS)DSipRU}PN;nzMvI=YBQ?BXkQsR#m&u^l!x zPSWAKHWyK~rZLO9^d6H=ty=JT>IE$y8g`8qT7dhg>PxdUK6Wt_I|4h~Y8 zW$qi_KVUq-IRup*@9!;~2CY8?08y9T2 zf0k7uLk}bXMhFnMmET3@`IRmmK&K&UjBW+hrVhr^#Q_u=n#aEcR!w#P;gpXW_7I$R zfdnw^z_N)Ti!r5-MZTo%M`_M98WyQg19>L^80K>sH#n_-=@mF{kX5!9_GNF!>L)#y zDsC9>^!BlB&4UdvAOj7|P*{gI#Au>$TNYr~`ej|UCe55pL7!eHe^r!!-cRyhV!;$D zs?A#A)4c||`m&?P&~BTHbV#{tA2F9cEl>HVW3ZqYzz&RuRS=#wbv;j*xD85?0A#4= z|B=&Am>P(tow_xW%1uJjQ3R28ueB#u{3aJZoV3#fLLvmFJ3X2xleYzU2g0!9B#n2f z%N#gAofCG?8M)2Y%gBlE7Hn+Ls zGr=0B>`AHMaO)SHLl4QJe{q6_#75j_qa1d6#*916i#xm#F7*mjhG&=4t+$me9%Mr; zSm?ap_yWj*UHSA#@Q%yOV}$;=lD4BA;(OkfUe>j{XqrkgeVNF(Mv<8N@~)CkOdqj0 zzQdyaaa*EuI^wA>q&If?%)scrsL%YxvFCx<0go?r!)1T*WUQmn;iP|PJdI-GwHW1< zQVdKaXCP14=N5T44b){e~ytSa)8J;x-n2v zKB;URQm{0Ug%g%uq`!yr)qj8EKnL*~^7#m-dCb+1=J$@WoxKT+iXy(?w0+23GJH zb*gjYBtqhq{NPtWH7>%fsDw<{`=K(v!0E8C+B%v%1o*% z`G<)p!sOvK1x`Hoq@V-E&R_>vI!2LU8_yA?dt`CU7&zrhQFrl6acV25>UZ(DJx$>- z6z_hFckhtfK;T%FPK&A+A%Fc%&Cf^xQryY{M%47pnSADWbS_=2%#9EMn z&{DT`{o%M?om<^esl19VS zq8gR;Fl4`rAp3oup1ANm4Hki3eT_1{To@+3CJ^PMCg&-ye6o_f>n0CBP$rQYuW|6@ zsnDzNGJ&X2reoUz5@7w02R2+Lj*r84<<}nPV9`K=b6ahjz?r3Q4vmV0NK6>Z^UkZc zU=twocLXK;LOQxC>udg5e#=NWqiXkGvfvj?baJ-AsgALp=z53LsN+*&G#VENnS_a6 zIy4unw}xZI+XXW<=sggn9Q|JVhrAJn@%ws#ycl(|1w}F&Z%yD{$1V)Jbv)v>o?%i!sNs>DR79TKMc*ikJb__%laP7kb*PMjgY{-Z)!uH(zFvPTea_++BCRl2X#7=|WtC54e?meFO?m;Ev1HVqo)BvrI@MCbbcg-(hE2o#Z_NxTmHMOE?- z3w_evyY6Rm{}7!WD|#&fqa@H{QqvJR)g1<#TXs)SbMJlP1SR4~|HxCNU1Nlk^yjI^ zwbl zqQV+fJozX1{RDocEQGm1F+r1fw7G1KjtL%Ds9F~kHkZCHe$z7ZE@1muIF{~^vrp>n zR8Jhbgf_ZIITlP}3rUqE$g~}&SuyHZ^3QQ121LBP`G1M31ZjQsG4lQ~*+MXjD7YB0 zhF%WP@nW6RT=_oN$*VdNZ{M$;^9OFCS`s%a&bYu<32L*7W}0FesrgShEbh*L?%oki zoZ=)UnJy+TM5WCKUGAAOUGgPr3x?QrgPx z`m@b#njSOg;RFWq5t_XDJh?wmiMpI9&!Mjp365;iyx3oF>6N>qlAVJFsJFOWW)_H{ zmQHk5?laadsC117J6K>g6sR59_U$32qTenR>#$$xBTeL6>n_Zgwz-?8*Qj-QN=T4E z2`ANIv`v?eke^p~$p_wQ5%URUjBBW(hKk3zpD{tFY{M1?2m+Nnk3^zEauRy!Md)kG z0wn#t9inw9cRwQIoXbTIxbwHZXH^K!k9hkOb15)7G>|@qPjEIR8}T83X1DT7m>h=p z?$NxCM1?5$zT4;|+`Y-2F^v{xiCxNE2~Y-Vn|@}j&J_!eMcFt}U{Q)7rsvVs{66To z_#<*eZ>%Yw=h}r8M758>_DsW-+ui#ps6?;e{EIRa!>B)n`iM!{LdWwH7pm9*MgvQ- zg&vC}buWgCIw_G;92+5o0aKCd26dtj3O>LP2(!&n;;6D;^fb;Ta5XBJDmGx7*iQe6 znUnC@46XN4G@NB`uVYf76*g<9^ANxuxcH`W+zn_?3e zegnH#O$&m}_LLm3TP~De-S4mpE)3=-lBexgvGoTktMcL%m9hGnEBekIL6I-`up50$T3*H43oY#*)fz`MyRi} zAz&w3*pPxJ5t;3$@`7V~mt<=Xoxp}EVa`M|3fo8&%krh{hCFMM4IYA6G*E>t`}~}V z2urJGl!}uF-im%6KE`30M?r+9n(C7&)O?4-6et zG)0F*)D%$bg`Oq(5THa~teT3oo)7c<*5WslkX(PZ&HVtrso_7BCaTL9-+l$-nIqB# zU#TUqp#<4LlOHSpXi_>JEaNwbrTxdovVQ+HBEJy+kvElDKj#C@aHKdc>gJc4%(g#F zDD=jD?2AbS7hcI8fqd%ZKRxG@!S&9!RZa-39n^F&>Ylp{F$Poi;I-RARhZ31j^j@5 zV0qcZksN*%U!MA6o8X4*h#X5WR(IX(reRa4>6S|&IsINSSk`AZz!%#AKh2>yPOOQ= zOTY$>&bovaRKUWCHru|k;tL^4e7>i&j5aO!mhv^iamv86?5cZTQN=@dncmDDDCUyV z(ALIeGr_GkV`ynkm3~pql=>z+jKGMarNw)wc=scNb?S?s(Td_ZecW|*yJPde zepc7)5h2g?UwQSh6fH4AHXxxskIGfg${ST4_I2de69&~4?A0L8(BE`$YD*1n)sFhQ z75TJ}ERF;Zo2$IsScY~|ze`3co2k5#oXFGmVe#85Q|NF^!~eMuESb@P>}%R*FK`pR zG1x|xaI@>Yut2v}LQ{k0H$7L7nyBYsR)65Q#{pieEa-sRlChlGwK6#lfgum11R2 z#mel+jRN^~$Jrn9MV)o5;pTPnQ`=YwAiV%oOkFUlOaa(NBaAtp{7USQ(&)}Xx4lgN zUYaA5%Ynxx?mRQDeXFEa^B(yt(=%H5H=^JOm7oS~M-)W2fk`Z3@xgnl_!A@3bS~Cq zkwVpnTKm{@M^o`rbf;&K+SMK-3&rM%R}evVm{%BcjCtA??-gzOwZ%{88j6p3z`A-F zDt<)%nuf`;uX&K^12O)TfC(0{dZq~06 z(Up5kU#{t^t-T*+zHuRl_x1JJOn|_;KpP-oWJi=*KQ$j9WWLOOtj~=HD7iY;rLjK^ zBCQ+I6hJ{9f@G484nZpKE-ksOd!b$~*ydAL!}d|~`4;O?2G05N!&6(dI=yZ z+IAei+m(_WgnYN4u-=j{>->UL3OCH@v;<&3Ju7ub_5l@2-X^FqUUK z>A#B~A}NF>qP|2kqPu?{N!CA-1SL1xME5d3qx5O z&B*AynC#2_91y>+a0k00>wJ!Ny}p8%OBfXynP&$n9zr`-nd}sQl+u8efw&$54+0%* zNZHS#SYuq&3mksxH&P&_goX5{KMQ*z-|>To1t;%IKcOnrJL4#XC{Vw@lHv#%1mXHo z!1~dUCoT5J?It4m4FHjXL_(sOADV%E<~O)32Dv(U2AOEAJ@9>itHf4_tu0dyKnaz$ z%{GQg+zv4SA-vYMJ46>9AsITJp3WF9>cXyM+qX)7KeE>}f7-Apt1!(isF({h za6YE;zgJ6Wb7rlWW`88A>p#xe0;a`{g<1ByQ?TmU^XLy0C(9homc6FWm$tkhivCmX zs?BE8SG&*KDby^+sdG1)<5PY*9Djfvy)b3wl`hz=X=Nul#n%Y`ASdd?i=k;?*j)#%0O$X8(XwK(oI`7~>kz!B@Mw|+g z)E-l(;r%(4OQ1Htvr089;v?sR8O_V1vS=L+{vzd2uO{Pc-S?{8DRnv8dhP1$rJ)?G z*S0$6e0t>XJyCg146fzy{0(CZ?fC${1^#6n5eqlUOTZ6cD@st zyXO+Gg0NX>JzwT5Kb)$f5We2e3oPjpvLpY20{)m160ELjEIB;r5tpuNx&wR(`PwM( z*!y6^9KUtz4IVGK_V7#&hJ=K0PK}pWUtApX{612)8ega5 zJ4b1y78Vvu z_b;cPFPEPitS8nh?aqgb4Ky(%Ez2{v^O)r>=N`L(Xb*b7POzuzisV9IU(wE1aU9kZ zP)_xHjX{^buOv`?6k6J9K?Xc^cRl;!f!=UJ@3Rso1YEXD?}JwVm(dp%9m#sI2IlxE zeI)bDvJ*bi)&07CU_8y`@pv-O*z8UTxfk}nz4rwZ7MrBALy7fxh4`L&J|0n;&1q2` zQyG0}`PiO%4oRMM-oYaGA9v?XpKpg~RSkCtS;?^i9DH1E7N0Z>xcInO`Cw5ro28oV zr@SdRk2Wq_ohB zo8jlc2m~gwdan~HfR$;jTA!bLjCJs|{+T#P{#aOPuwu^Ub$>XDZ#+nu>g4h2DDa#p zwvktq=x4p&{2@bNk-EX}Gnxy}y@ck<^U>U$To0jogRSg(E#uvY!m`RA5L z1|;L+n+MIUZp;a1iAXA}gWp++NZ8}JcR42W91fMx1dd*v0SXTt@8sCh# zp9VX=XQL{mxExf)>3wxgW3%1?Hpw8%4`Yo9N4xvYrEk}=xaMF*x;;Ox-Sq%={~>xi z*a=>YnN&g}!Hf~8Gx{Zw&oo$IEUw!jnBCDQjsPmnuJ<`lEa`gLndLtGxwFDgh( zGGXvh_5Ytm&pl459b&{DM|r+)L%vaJ4G=G{w&$-Fmz6P!ESwuhKOc@Ph1NNIFV0%a zMjukBHMwC4D8E>z#Yb#Q!|udH4H_X-mlazs5|$Jfmvs~cZ$o9VIKG>`)=yQG{{0IU z=|DCX_kp*tA16@y(^!_Ut(G$m&p;mYL@6^;T~!JZA{t`I=>YRx*h&!$B`Rm`dRm!0 zN0`TFTfizW%VeCll4COv2_f_&+XeJ*LU22>VXG!y0lZ7BB>xnf3dC)b9LR|xm%{`(=(yX zz{SIZ#2pu!c^<2@yYO-O2xy^CVTy`IzJI)9o2sWVGeroI0ZM(ZsH6;YPuJ_lF)}HP zy-s`;i(&;VLCASD=-+rs;j&86hEXPrj*Ind4u-j2b$>KkMeiy`L$HruX|Bn@Sa0C@ zZ9$7}B_2iq11nJtp{cb7yZgZ+mcJWX)V6SIpvyT@bY{~dpYEmnGD zL<)(nvc1lfUoGbI|;9`<_g_otv;2ejlfDtXf%C`)8+CIUP% zA$TJ)x9kD}gwax|Rc?FWxB!r&3b$Jy+xz+o5X1EV#wte*5Fv&1iBB#@l-{Qv@hZK5lCXDIx)H zimF~mN{$|9@R-@wEy0Bo2aTdnNdo@Y|KFj1(`WReKn8<7OC&RjCkiif(BB}9;qbX? z#{^b!``zFFK>Rr?Q*`~Flao_I43^X`$?%>c2%k&VE^xOBkBJ70PZ=IAPEhRD5)cV0 z3D%x$4>9U3=y$!yt~RvF4Ko{G#5N{1$=p^HC98J^qvl$qfs2D z>}-EIEti(NFnNJ+aVW?c>=Qty@FDKY6tUII(+ljl+6^bQLn+M97MDYG>j}7QwMTzl z4|W{j-=!YPL-X1oDbMly9=$nhG+;q++ixMO9-IH4*ay+czZG0O=09W3YXEo z{Q=55DSL_B)KiMAk1lkHBb*IkM#p}B>O(sd%{1eoU#<}Zw=&i#S(H{ZYO!Kt`47>I zUPJ9m(3_K~K4mRq)&Ca`>3C~I)j*8CtF(Tap#sDayfU_{^(GwswWs0kqweTQTXA%@ zk85LT?9PWB0=xI|*k~et_lKW{rqc2Ogt>$WMq$lp92^|u8D0On zx<2c@b9Fnt7XIXr!diC&cMRrGhy;Bsy}Z1}S{GKis+IbO2&%bsUpiI1cwO%LE41pt zBiz#PXdmyL^2NqR;~}M(L5$VKs*ToP*{NZ=9ggp_t`nW`K`1)TsJTG?&{}>YO#mA* zAgj)Mt%=q2i|L>!WVZ+UN6r66B;ZXqitMlFWeOsyPKus5LUd4wf`?Tc#WxCr*(<1< zO>V?j(fa>bfY1F@;m)`90q|M>NVn4m5l#`Z4e@_7Q2=^4)Rl@Me2g35%mJI?3IVm> zRR-|1cp)Al)6WBVm4L3rs1PYn7EJhq2&f82kB>zJ1jTu|QF0M~MbZ&BVC^R-r!>HY zdRU^W#gFh$+!_s1dp+%PU%k_NHrmuT`hJ>d%ns6AhZc&5el<=&NiL&b@PNd2p$yBi zRCR{1?+?5Vhc`eF%f!t?u1*&*%YkTNERa?G77}&nj#lkPyJ=TkS!C+w_P4mRDhz#yCTx{~^*yQ7FjpSn?Cf0XCeDtCjmzf0&c+h?Hho6^Uo?)c?_RhC*KjHTLddsw zLjXH4w}*+lwr&Td5fNo_I9)j85Ke3>rTzrXgaoK#H=B{x6$PS6Ta_CWGDug*~+XJUT|7%6!~D^Rf*`5KViSQaP2Mt6s6oeRBP1~f7syJoq zUjr!%rBr1eI-)uYAsY9u&_q8Qyp80=kpbWK+a@m@g8}fAQ%bJb1eDs06aOtFdv05b zXfbf-)%U57lXkqHqiy z&tEM+_ZJG?kg#~GP)Whf4kYDadgw)}ejX#i+a&{7=ie>`Mb4*VGUiqP9iZd5WM5nvFwFk^1~af#p~JDZ z^wCKA{@+z@j{~+LbMyk2i6O%XJAuU<<5~Hk+{u)&=?s~@SnAt|Ho~>S`=e8wQdlJG zb+~ZQ-~=KiFnk>%sp&*40cjz*+Tq)RvI>@IE4s2{YR>sl^!uj7Vfy?hx`t4j?R&jd zj^nn9Pdao06X{RC2-Fq@!oLhb~qB2^;7epnPb z(VGs1W5#0IZ+dm-1IrNM05XW>kHw$j)^Ouj@V~5NzA0*o(Wat!>Rf}_n)IoRE>xl{ zz;Cd}!$;?G4YBYr5YiF&(ACHV|~BUq4G$37%oe*4N)U4^44qACxaqp4 zvI94O^bkjUID?Zm$$AW$-vZu`qB9JhAO4KtmobzI-%wf&1!*P;5ha6Em_T6XhYvDJ z*mZaENDV3Ju6CDDF6oHQB{IUYDFtLcv&g5mdjm-acjrh zOYjJkm7LA<;f&<{Nc#2RoPdBpj%e!T48B%6KI5dU>wWo@K1IlWdyQMav9TwbLA!3L z^lwm&5^6W95j69%=c<=>6_6AGCj{sU?Z@gTT?Jy9)vxiAQ9$=jP1CVQoOAPS0irP)25JP>Aq0|Y-TzVW7{y6HxXP5ZCO<}af zCV&${>u=KQ9H|t84vWwe2yZ%6$7VEaFCh`u9}i7|3Q*ln-0d;MK#w(m&k@uc#?Sxd z0S-QJDBY;8wLOIHd{K4zJqykqRf)rfZ`)s9Ui9B~a4y?#!Pi*=ryRH6_HglpwmO5> z)kp5f`$_{>?pH6KR3=@=eyAF=iIr5}yOXmM83DfE$GxCXw0F)E2m`Z?8i)5r1r#Uq z1#ltb`|olqxX}j?EBw*V9{ahec)zBtE2vNWRB`O#6jieG$~)h|c4<7G(L=REiGSwB^{`AoVR?A}%yCRMPR zCG5I4-QzIDt<&MWx){P4%bcN25#E0o3U-jZIH?9}E?(ypd;31)c4u%}`Mljk-Z%W% zt*u2moFwM`t=t|22ep+KO8m0<8T(mpyRq}(pSP<|-0F6%O+nqhnKpE8)5^`^K zxCnA_ZSehgKCHB9BHZDe66i1C2sF-z5C_=7gd|{CVnMps$l}`n~5CSG;!aKphm}6RV9Lee1f}=LjWc6B(ShU)!sdeC`+0 zFPAF4j-MGjI@Wd91NWv|8qV#O6p<4KJR@>coUkZE$-xf4z%o#oY7>+%ZZ|Y|L z(7qqb?fN(`Vt*ADo?de6C@C?w_>E<7M2z>pP)WMU6Q>7%R)icC9o=y|$q)E=u={K* zE~aks3j^1lKQtCLPAtrKF?mP!ny*ICc?1OoLDjl8c6P$vFLx7}Hgc&)1A@m>!aU*wXK8tIb{3@4h4$atjl{>7Q000K+t|d7aP!f-?}oG7XdCrn z;ELoRN5UA{EYT91VRT}{%7UVP5&pssL(4bT24@GXgM))YU_9>T+W~iV`^_l0(bU(W zQ)Y}kI!X%b7ep>GIVzoUxzu^@9~?Q{Z<)1pWHC?EmG>sLEtgdF(dlT_4 zTJ_rJ4;-e5tz2nw?#q`%8bmMsU1(LqsI1ic5kBG(4Y2-u5-***)Z)Aj{%Sr}snIWz zX4|<|%y{%%LwJNuWhe6~b|pbwEv@vhXegF`Vt(iP!^K7Q&!+_qpSJ=Kv^6}_^%>vc z$s-XCfn9s;VVA)3dl7p{=Sj1>IH}K-znm}1;Z&~vMO(@L5{sKanlE~lO-WN~qBM)ub?_uMWWoZy}zjk@40XdR6F5w`0oX!Chdf89q4?R;X07zGbA1`d=% z7nq*xq+;~Q#mk-Doe|eLuDeOzBhXHrtO$d~UmP-cP1`0^y=rjS|Nwwb|aB|6kvoq3Cd(zKzbr z(O>4))S6Yx7ia3ks2)m$&}g~Q#z{cP=j9YB%cW=dL3JcqeV9KmoYF}^JK_@_q0dZ4 zegjrIN^ne>?Nxp|N@z4&s^4r@em$FIo4?9cK_x;>SNAq(r|EtyS2-=2M^hN6y%WDEb$cf!<0WWwKIP5 zxn3;2%(gDj#QNez;)?@t3yFjOt2#`#z||YB5rlugZ8^3Px}mb=zCv8>o9T$Ifjm+W z0hw0I&2~Q5UEX$SaZ*<0DK}7O9C3%oTUVvNXdgPoLQFP zS*$^s;ShcJ`|2u)#^%vqC@}==j?+g8~eS_kp+Lj${O*s2&{d8zqxG& z0~F<8ykbU!RVvfZr;E1Bmz}^Yi+O=&uYm+fr?%EsCY|PH=Tr6Yf})miqLS9ei??8U z=lW9;-DhQK*Cr*-NNh&@SemE^jblj#~!Q*As>)?@qdG2MbR6#Jg2LPzNR_bvt z&EF*asu?8T+c!OIp)EJJ%wE!FcMrc5=ts!!r`YP2mUQV*ObZP+qEv~=7LP#P1S^`< zx+MXJxx`a3Kxvo01zd)wP%K04frsb~>6RseHd!mv(Dy9>`}j!nYX>a@G0{s?d*sCo zh{U7TH5yJUbHC9Ri_TKzrqw(KJ|eYH(?wu}OGrsc2|e#6GVsGEY;y%z5QRyG8hzew zel#1uE|%nKb(v~s+hKSMc7SI{=aHgiYP50%*!Jj#_ImF5>gTLeW{y_Z%p;2sm)G^lpIG=FyRMnK?b2l7&$dn6K(I04 zBbYAYp+*n9_WoQNVYh4lI5`M$_I=(fD3I>NCwgqE(C>0T#TF*+a4fpsDJuWL>72=B zmoDr}EYZqm=k+#XUf>Wm$n)P3MMrn*e}o+IoFPS?NzWN-1=>4d}N@%Wsp{@Mg)YKvp|;yE!Mm93DaLIv#c){Rk7^hZrt5aSEPSG9FdTNo(oGDd zj1ehK4=1@QnF?gocLGQ?`F@--EFM^7TZUXIAKFEzLU zWNuPf&>9eb<7j(uzd!9g?Iomz;Th*c|A70_u4;i?;R@QhZXOeB zD?dZlth*eZZ}v-a-7LY5j(SFKYw6WWg#75-I|%is)@wC}?<<`3W)sK9tW6WGXi_di?4yqw=oKgZh zk9Sa-28lipwRu896@pQybN;F?8>919JJ!x9oeLFbLkiUj?I zlqD*R(#kLZjI;qrA_I2W;eHLznp1s&|Cd}S4reM;19mRj)f7jl6@>x#8$w@;Ag6@~ z%Sg=wfsd1UJjbJJnq1&PhyvKL z+mgrOZTQmn^VQ$~qsgU<%UnI$(0myTE$M?IX@AWn>HItdd&A~-zCZM#Rm&(S2NU@t zU8PJA=&jZ6zdhIczRY>r5P^vj9d!RNH`}y(M zMS_3yd7N22mHq?D5t~^TJhA))m)}<(uOu;2L#`bkFK1euCQ)JFxy429$8qk*(B~7$ zrq63-!+cW9MDl82f}vWVFQUsOcr;$AYu9nV2!3pp=n^Oz5sv&ol|fW#R_NL)*N`RP zE2-ZIAA}hp^4US@jnME47Q9iU|C!Q=d{YtwT0orAVi9>EGBjn#5_BX4HG}YvHZp36 zz<{3#hs7sUQg!)T42DA|#%LHkR9_kV>HY?g5n94+HmX@NQ}aIjkv)h>&&e$KU;gSS z%;v5ZX75i*dfaV0V{w@pqe@|AjE=e(kL<3SdpcX#*T1PN}z-CYL@9^Bo6 zyOZGV!QCAaB)Ge~+t<1Gx$pW|tobnvGu_jttLoI=wYPKQM?D+sw%cxr@2mTGnyn59 zgdwahb&)xxhoy@ZljV=E$2C>aB&~~67Z?62azyPJoRwwZH#5(>{ie5bQIy(94W7%c z&0w1fuFpdk7`-nmHjLvMmHJPIV-F4n9nJPR1v>*97&tsIgFvGph*n=q1@O#WOc;#T zZLx3IGJEO{44v=(hBb=#7biZlZC!P$uQ7e|&(p>o^<1)ob~BA-CF z*%6l<4LcTAGW9(sXJD~RST1JHkS;%w7M;OR(K#BL@DxErn5-cY>u{Gs8k6&%{kWoz zE^ypWHcFH|DTo2pA3E4IdctO7Q;Jaw{$;gH-}}P8-xb+s?jjP%Dfu7D6#w%rD~lJB z1xDIC%k3Ohbpl%fSAY_NE)OUYNt%VS25Hrq&CblUJzh@CxtxY6G5USMFDuQWT81I$ zUBhJ=UJQ$N{uoXi``3M{F54wb|J&)FD4aLm{*`rkx`t>A?y90L#>WNQwn2 zBf8!qbI{T-?Mx-`$Vf}c4wqu)M{;+$M^O5y$sH0 z3Fww5XzwOSc|23}jx!k#Yb28}YhqoKKHmEWbAWyjrBZJcY7{cKDNe79BL%tjG^Dxx z^S2u1$zAyv?y(%asktG9!1KANynrPBy1LRSCAs?wAABmDB-Fw!tS6t0nLW3L(9o*V zREB)NEWOc|%<9(r5i6Ac-B(3)b#(cqrC)#w!>juZ9JbKV3<0gt z*~x}k_O0KzZg1l3KyZibFu}yJwyOeyl(d>+lQ+7~>mWDI--ns_U7Z3`x6y@o=%e@` zhDNDSi}|pqdz>Qoh;kRH(+Is*0fN z>7jYHeyzoZ|D-_kEbmA8d%%xTfBq-y>BBf9m+o%Mm%Sd>C^*)hw&%FPx$?6^;z<76 z`MelLgBI5_iGesO6aVkDo4IJ~ zG=}~PEHnqocc?)I=s)!X)L6w;C4!rel6g2ABo z>gq3+qSNN=QymiDYaewZiTr6qrSg2`FGa6)vJKC(+;XhRc8 z0L4{67VhA20oZP~+TW)Rr}{_b7>L6!hB@rR-m^gB*lVE=MiXl(-aY@tjr;Njm+?L3@X|a}7T%2c>C&JUmQ9MGn;@yf1xpL6%7`1nMG{G~~}I-wuSl zoo>}eM4-TfOpr9ZD~(OiJunAdVKW)d0WsP+NoGy=d6g!G+g)AJ8Fi#@#9cS7Yq z$&npsQN71_%k3Av;1F5Rp`?@+sgcwq^`j8zrCi5%p;DN6aSBP#970A!{WLM5p|4+K ztTh>xtRUd#`!N~y+r88$x|iO)!YQ7w^zVf)=TukwL^RBmIa9~Z%Sr^P$3B$Ea@{hq zDA~d*pc$E&AbzZl_>Y+}!Y`KAE>c@~mymISP}GySo8^ukB>NoWdmjUjVN%MHuw-c* zraO7H>b^&mDkkQIe!mSnABu`LnqSn~KdP|*lH(GKClc%TY0m|`Ge?=5JHY%;SMlFd z6HeT6tp-!hECf=6TN&+A7e;(OZ=%85WC8aF+r+AnlXk+r>E_H zr1tMtPf8nI`LbOD1jMN8dv`R-&4~< zpV~#lf-K^nB;o}y#VGwLOmVibJhnm3qGuRh@4G;G$90p4+`Q!k$%D8-@ z{Y_#DIf=%xPy6jF3fgVyhaES|c^^0WcGKEbbV@*-2J3KfdJR(e@rezO=yGld73`Dr zO{OU;8Bt=0KKMta;^;PUvkuf;faM=Bv~I~ihcn$RwnZLz=@E=7=S+#1kBA2nW7`IY zkL$?XUz2Mu7>CT$48oxC!4O;Cqfn82CZ-3b7mGWgDs4 z_oV;9!J4<07ZBM#;j(~ewx=Od8FVCov_JnUBNmWjFcKpgw19<%t#a<<3AL@%e>G{d zTXX7Y@Lf`|o`el9Tyn%i1n(Q&9jQQ27-5jD)6~_zi>&9juLf~XrLqH}IZzQ^{)(wO&P(;bewxBS+WgYrZ^ zE0t-Rc)Vrh#ty|Y`Q_bz!ArvUt;tOa*#W0ik#agRYrrt6os5@>gIs_XH;XN@rW0NZ zPF*`$h#Z}DgJLYxw1KuTP^ZRtC$Hu!b$?Z?kWqfcL8n)hlNc@LN{3Ev$bQ$D}q z%0-pa7N|UOMm?pSOB`Mdl1x5=uq0VxO5{czk)ZH|JI=VN@E8g)J-jZ5k0TmO(?$Nh z`+~?&(ePTn?nqz614BUbWP09u;}coDcfJ?yAMxv-q7Wj*=l!78C3gmptmZ0wyk6Uy zHy`w}8Yi*^ZeNnbNh)JA;v>hrnfy!MBU2Wq7?KXvzU0dfw{GT{LDyS3TQaCmC*N_~NyOgml!{w;wD`|DcrdB~ zM!)=|qTnxiq!Pz5jXytii~(>VP*?#aqrQ_Q zB!j~oEk;hhnP3&Bh1d-e7YieVAmAo7atkuN-j9V?@8tk^a($G&m9+|p)u@CfLeyB= zF_b1bQtPSS;yV$+Viuy;M+pB>zbz^e$=i8a^jDn zhA5;Mru6zdh0fMRj}OAKJt-ZnV=Mw{G@0K38((WXP4>o)md2u^p*V^tjY@ab171L> z?QMB+FlDl&3N8BD#g&M^=}h}7DoEw#e!9%ncCE=?Ue3${H#@M;gvjwJ#EAG_YS)ZB;z{!#%hM&E)w6%{kN6S z@pPIE2vt2lpWLmE+aaMDH`X2w7k&R?^LT6KNKuUJj{r?0Br@OPzIA)DBoDawl;wHP zCMdKuA63Mxycwt=^}sy)bEidXoUq>?d{!fu~b1 zXSE&88FqWJpU$HpHcJiJ0v=6ebz2uj*Cnq?Y8v6r;9`!&#XrzjG#Kk3u5AZz|wuqDLv^xX4o-%@eeL zqL1u1k;)fS;*B^zznY7EgIjMl-_y1%=61y&*eWVIYAPy@fHXcSWhqM>ZfwrC!^6dK zD|PR?8-q?BhqjvftF3oM={%2IoaIrZw&OVeFtal%IU9}z?Gx#jgJp0u)HPVIDl#fo zh=}Gg>{i=2)^nyt9$v>%Xykon=NFsrt}uX&JORNUb;W=U3}DCHlw0U>Qg;N9w|x$J z9-=d{0P;5VrAE2tw_Gdr5q?pZpDUzb5MFIKcHn*oh|oDn9h20J`S^*f`NI(*Lo-HA zs*`b|)28y}Fo#$qcQ;0($5Av+qqVM1xLbA9Z)}V?GS{sTX9dCL`1q>qFkI;e`#4ng zPe)wk?w2Klu!v+K4w9OeeV>^6e-e4&#Lo>Gms5zBV#EeOR6{0BQHc^opv6gn1DNj* zqzrgte>WJC7yZ(rRDVZ+pNU$|w_Oo@6%8MszV{YRMZqe!*blIx!$m@fzk;CCGAGL* z5PM_qo7I9$rBQz^YNm;}C-g0pu^@_ymrol+%R~=<+51%?Lp&U7P5IsSpNo^2ay}$H zf>Ukm!8H_AQd#M<6~zT`Igp8XRNL~yQuED(u~E3|4)pMHExjH`8mua|k-rUxoX=DjF=G5om*qfP;ghyZug>(1D z6YtE-O#jAY-?W6`$BQ}<}hcZE}F zgmd&~qEug3nBRQb?gmyBZt+}5v~I+X&3ZU|GTAvps%3!51X~-+4zGuF6lpEZDMwYK zLFYeJ21`@R&nr7eXk3@Bov&)XzGnskuVdR|*H=uiK_(Ujj-md2C7PFAxnY8fD+_Ve zMx`QZ=2=ONYeVoidXoMkK}_uDjy0@xA`;ZX(h%g(7Q<7)*?TL8r;HcZ=Ri4nwHqes z`)0_aqbt8LX2;_{gl5tOV)QUXd%8G1)sgL%Cm}~S>S9RcTDxlV@7^$ubLdaw^38TL z0pKG}TFq#fa*BXYj(gl;DAzZEpO$)*{FZ+lG^aEF-)oicLXMVwlgcXlOgN)-2 zzfidKxgWoTX|&aSJMz81DM>IF6mQA}72*7zD< zJNAoA9lD1^g#?c}b%u!`gOKYx2U%FJNMw3?x|`SX>B}%q_`9K>7Wwq#xetze16x}! zFE5G9^@#;i18RQ#4#eKuXyM3fO}|eu0Q$!g0fBv=+wk;fu`cARHg`vH2?>}GXvd%M z4FYhu$3R)ajMjfH^qsVK9#d0aENB_H?iu&%LUvrmxqXSE_*goBua+6~0o+_KilFry z3lug$V!$OKfr*t3HNNWNQF#qZS-@1*(`(<`vlBgE^`zsKB$kxr6Ih*}Z#pTV!Di55 z^{9&pNVPwfHU?L_f|m_yjd$+kvi`PEh_xGv?d%QtPJ73Bi;d9GBxR1q8yb>R8Y)(* zD>wRv?LtDKYq!hY?2SliAm@cS(*W}fO~7pX^f~)gg&63g`0uf>1{d{=@JBi4Qs*|h zoHSg(D8J2;)#h&ZLjN`-FRiLNJ34xpF8bcoSGUVJ-(dN|>|3)3NN5HnTQiATo zV7rBT{7gC;@3uN;?t4j*JyBsbg+LK2Omt$KpVD~kp^y#BKZO|LZigwbFq}sLR^TL% z)in}9)}^e~xa6?&Fn9qJJ@K-6$b&F&L#Xq(=d zY@2TWQlI0)-Lwn|!{1f^R*0u-A;R728V&!Z99hjCHoj-0?DNH)%tZ9tcB)>$4;pe2 zW5`gT(8vJuY>?J46QYyfZ_ICq^l9ZtAfY{wB|ss_4%w1JvI0{EbaAK`Nb{(gh~;YZ z^W_QhhO(a!XpE>-jEw+N3nr%;%o~;F6FwXvxk3D$33;GorK)Mw0&V`);uT|Qcm%W}YRV2QWZumkyqzM)U2?_9{_T|H$1m^pzDFm@aW5Y-U zoj1z{iM4B=Qr>Qk)2evw8s7c1-w3$)T?2HwGWp(C8unZw#iEchTD}f1$E*orrVPDJ zu6p_oJLo=SuX(bUm6o-Oa^bK3@+al{#d4kd5nn1FQNqzIAJ1}(xAlqn*uB+v;FPzO zcdwo6YIV#aA5H3aIA$n01Ujiuzz-OyrrXul%e1>wlr*&@)88u=ASk@=z35mB6Ho*L-EOB1CZ7rl3u$FdhXf&O2a88i1+*_tek;j~wzMXQkE99We zEdhH?4@3dU`Y21v>+zr{hVWCS%kpZ6f2`@8L~#>Bw8AF zQNxy*sq!u8}hcL#B?o z{c(3sG{W#`o+5_-d7YKYXJdA5QujW9q4nw4Yu-vIK7NKdwZvrm!(!6toO1P0G$H?L zyX)G!qjY?|Iq$3Ow1&c#Cs(KEySK+RY_@1}ulxCp(>$8jlQo~a>jXhfe8BA?qEiMW z)HIcKW1>(?{+Tp@{(~Q)KZH)5dZ3{^-q{uDO&dcy5#*#-f(oBMl(m9ba=bYy8f{$< z6?WIQcyoeSDQoD3?Kq5zh6q)GI+7c|MAcqKe4IK@u5@HH? zzKUp{8+K%`>7C%&e?m+`ILlsoq#3*=IXv|~Ic|29=M8_LfVm|;6ks}VnoNrL?(3eW zha30P4nK3eO;YUm)6daM$Z@0)JJNL5{d3p-G$5_qo~{tY^aCDKgdgGJ3VIIRm?#m+ znncP$CG-A8<)|fCKKpSJ(IoRXr)iBL5`|ngx}7#6slgy7>!-K39$%`)W4atB32)BQjhbJ^6GS8ZNJjpy3!ZgOF8IBs3N z>DFV(aGV&!#Mo=J#O$DS7g`|1&qiR@XigoYVa4f-p}m_6Ye$EbsHdX|tu;j0;FhDK zCKv$yVD5KqInh!>{1-cg&OFlG ziQ8KM`cALTiW~&8Eslvp$aXoISbpE{mPNf*m)W2bCm!IvIC*(d{1#gFPFszy*z@x8 zNP|Rj^jz0?KeF+$ldRP^hUR0#z`$JL?5D}kYp>N*0=&)Ia#d?GLgOe#Z0SZ?klEq> zzO|iIjSVlhLU6D8m*Ki}mQ>zFU<=d`)Vm?Z#>vUXM!6{N5-ic%O%F z2CRPQUPkI_8dXdL!XUl`n6gs(D(NXJtisrG=e&6;$|I1nmI4{6D>EWCDMm!+%`QRF znpakvx2v;_aSB7RuG`si=vjy8r!-f1IwAQ@K-Zg19_0AHzFnfEgoPPJWIU`tKjBPtBKQd-#vO;m zZe;jzJs<+2;B&p~St^RRz$GMZ-piK5Ged&a zDW;FpOixS8Iy^a1Gx}|BBf8Yxe%I-`8}r@gItL-$XDjP%=j_$?YPaHaRtAN*>1gvo z#qPU6g^28-q9l2Je!RYRYBsK%!^h9{8;_6gV355oUwLs>Kc$JBbgkQ)$ZW0Vp?EJ` zl^4jvmXxHGtgh;-jehQvqDfJn|LLf4w!LS>+g8(lvZcU8nF6@My*)9PTjt_d@Z5XY z60I68F8p3yXtwqE`qhcfOiP_FO9}b&(UJ0S!A1^U3I)*7n4horF$N~VtXmp#8m#nm z$U8o^qP~j_`)kI~gxA=SV>|tLt4hzo(VptLjJ5=jR$2@T_7dfIAfh7BLf6d&3$+Ny#e_3d9=gl{_wG{@vXQk_okTV8UGg0>F zq#%ytXJ;3hJ1Q)VjorwyEsHB6Tt0d{Qb5pa{ximDITdhw- zN8i}U<>mF(EFf$CZ}o zn!b-pqxr`e98)JBUSQ;X*bOXEXuP#*763g2MB*c~J4BYeLlhcY|8kN;% z>u=!c5}m1()Jkc2@nv21?FItp{Vfb}s57vbb)uo*6^F*9`$|Zh-U$g&!owF|FO;4a z%+R{k99teM(Jrari0Nx!*^jsF5lHDLrbtR(DFAt51VE+QZ>L5n@k^}c>yA#(yx(B( zg4WKm=;+mRueI!}J95Yfk)x22tHEJf;aJH>FuwQ7I5$z_If@q6#ozmHF8Ame-pd#k zlh#TGlOd*%Aulj}#>dw=JQDSN%#wJ$UQ%enbTL{0OC0H>r|S@WIgaZtL)D;V@OwAv zxI~Ht4abnuRMK40(BR@$J@=JJbF*bikgow)^1m(gc+yC@^}n#c*Q9DlY+_?+xyfEB znl!Mnvav0Pi@Lq7&JUt!b|{Li4Mf($uClU9%gQDsBBC3^O#gdIQAI~1@Zd8@x?Gx|g$IysdaBeP z0ker1jW#T4rx#HVn3)y5&%}2;ZEwx#=GK~T0=|hKwEU~Dq zv$wa)M}#_#GX4;w37fo|Fy~QXW3y!Y#U?EqKLeKpYo>4OVs`-FExq2lP7uWyWWy-O zu41yawFRjkR*-Rc^rC|e&(R31UX<~PG48O)=*arzy5^ip0nYDG!&95jW7~4kl=GmJX!pOqEPwr0joPO4pmUt2Ltner&re*RkvWV|aW3XDrC z(0Cs!NhwBI^j&+Gfm}_laMQ%Zn*!!`qYjx?L`J?4juT>vB0mS(l&w z?I3PByW~Mk3ay~A1@+BJyHOiDNrbBS5=OlBTl` zHVy8LQ?%#}`P(-QsEtL9!-98pv9UPYHxwdlcRD~A!G{r3lQgN(lyIyt)k^C@xI7cz zfw80o>m*}E^k14kv@|a-%+~v?@B|d+y?bdV`_nf?-sTz}?1ht{NQ(&{%xSyaX1OJ< zkkywPiB)@b7mrCuVS zQ>qoc?K##}u`0uyJiY9A&Mxd;K894!wwW6Wzz2fEM#woiVV!TdsTLV;ZFQpkiY$4Y zg|!|GJVejoCn)gbC3XSoNIECrinUcTQ)siqY|x(T4N+QcoGAzR34v*iA8 z2r&Pe^x!q8BFY)Fze9Fw>1HYvtPCVOC+sno#-|MQe|DPaF&6)0w{Y<=i>3gtDx4ed zkfr!@mC6iD#?;7%CrR3FzV-zeQ-N<_TkvU%VKKpYU z0yKmQR4PZ6Qrf(DJ`9m2bZTHlN`9XEKn|&{$c?_*;(RsLqq}wvE6HhdJa$5es>DZ( zveahO8v=}ZJBdSRR;xOL!ETmx$fgRLACnwJbAB!Npnv#r$SvQX#$=*0Bu?W$;#3Yh zoIL@b6A(UneSJOk%qJ`fdq{p)gPo5-eYXDVmm*(7v9)`e!&Edv&{wK&`n26@bk+6m zPo^R zNk|pEmQ?oA3QFKc8O(>obwx_mr6!{*>A18>k$!L86DshK%Wv zg8={dJS+6lG(8VZCO8L+GCJ)Cw(>=)<7=JOV2nWea1it(Qz+i=md6-@X8ah&jytKz za1#Y?Td>9Ymx}S}i15{t>%P9ecXuZpo<9^aua)srLd^tQ&bv)$aZsux@ z_t76l9S=`WvUn+~X#i>YT~dN4n86)aw%wBblhM;-D!qOk)sXcNC<97@!e~pjxZ@;) z(uS|jNd+ln7f`6OzGg$mU;*N%Mt!A z&gQuz`(CS!uE^i?YuNPue#z9I%+1B6p}xNUaTgv9D6G-M)Rbl^>Jyi2;OPQc1OC>S zSs?dU{GZjpc!VCW&%>xekF)XcOyAIY=i3vzhQ|}G^IwEZrNVjtyatBEYKSs*-Uhv#NjoC$wb+skXSqa>3H$)|bmYheeKh&qa`1olskpF}}- z?XHu}!@gZWwt&-s3`JKnC#T zx^5ItW3uhamM!U%kHsXgL2nJ8&#=on$Y}C7yf<27Gv<>Y<CnwoShT>9FEv-CgkP&It z#5D@yN}cyKV9z8WPSZa;HZcLbrucAb3LUi6U~ayq-~WK zm$NFYMBXHGg`m^&2N-{5>|@V~HI4gzo{c4f$Pz9+P*PD*1r(zS=7mK?jg@Dpv%&pP zXbWTtSmL^B%*c{((H}5W*!A?XN`E$?Vl;?0B;8h$ZH-Ni*&gpJ0XOf^?T+m(t%!gG~DX3xbbXNZ~+IpObuH@q5r*VPxy8focx7|M zLnMK165djXJ?Tif|6q(r{&zL8EK92-~L>#4-119*94^$U?B-sRF9H>$!@hcC_+YKf!YmbK zw>v3raF3R+{|gB;hckA|9`;`U$+z@7jx20CuS~Z{d3C@b*0C}ysC%3U&G`M!;`_cS z>sIL>nMwXw`Iuj6l`(qO-i{TR^`5tv9~5+>n!pg~0eOeXP(wrBu50Wc`A}umH#5`| z=bV35SXouUEggcz#RqB3a9ob($w~^kyM;r^0Roh(%}I;b&ZwJoczMX zz!j(Uk84n`Q3zOhcwu4TcMToca=W7^Pu3|3p}M_})zv_h-aBva2n1H6GJu<<=V>K3 z@3J!1)6=uQUJY*=n2y`^Scx<~E*EE?+9yYCY+^!zAj3+Fz*;lA@OL{p+hF6&UgBmA zcJd;NL%!zJH8AM3F_B;ZY~JwIs#9pV@o$xTsHkaZip=+Z5LzqM9R%fVgoWj^4BN&OVcGOfgv|+9Gv&%($-p8S%5erBO32^ z(*c<66Axc4W(-~MA6Mu4k6eolL<;%292*xotIY>^Dc0{(l-yYVt@o$T8SsxGB#)hbIvx}=n3vur}U03Q$ zb~N4|1+g~|qKzs@gClsdV3AhO?pPj-&Vo=Ax?as3{EN2>xO3sklq&ts#zApl<{fL&@^X zRey<2>h^D%sTtN8v35v}`lqaj7b`kv&7LLgVQ>2~WP8Jl!OzLyZhP}U>ZtNl;Fn}P zUH5a9vq*nYsdoE90V)gnLVmZ}?pWMu=i`$-k4BLW14C7LS0;CRNw?kR%A$GDM$v1C~lcC19ZC>6+Yh0pZ&(jAS>@B>{ znvSkx2AO+EJOy~*XImWVuUIS?Kgti$*FCUtVQV9vm%|~MBfJER2L*8y$&2Ar=O>jU zP?h2Th>MjTKi!b;ze&m-QGpb{#I(2VI+J`v__o=##&kU(Nk%`sl@I84C2}7q&e|` z4_X>r9M~?zAi>SX4QBr)GyZML+fjU8FV_tMMXt9I%V6Aa>Ay)111?%L+n>~eu$jrK zxIflWgpObHU~&pyx<$jsfP6o9bM}Dez~E(6F!bYA^C#HyAK!+ebn~DPF{o~!mZe7Y zvh0}X+_`8&HwdzywSv_+iBRHD$CMD$0DMpz6e8o{dopn)xHXw7A$qhOpU(jq~^zG98T&fbWg^H*(_Ia(#{- zuu?mb;|flkex+rP8XngTtUWnL2B<_xAsND2!Rea@>2ND-xFF0&6;&;2Y-WLXWHed^ z@c{-ZS_Iu+Au$5{Ih!|4aeBM6p{}Z-ymjb>n=3+l;)73VdiwH_L1bq2brm~X{3Ai{ zlbV9STBp;6ALm2GN|skf@rXwLsHk>0m7+<0Bw(Shuon@ba`c-Be3S*x-+zkYQ(*%JEatT?nPy;+hszi=W<+bE*8mW2V4!$wsG(yqUda9n8m#h_Q zq|W+Usm;;)YcLq3EyVr1v71%OXVMsvyMw2*Y4CuRwJhiMqT-~}$>^?zjwR`f+2mgZ z0Sb5*w|rrKzGE?H*><0ppI6jk zSCM11x=;W4R``~&p43ooKa;V6H$ZvT%GCGeJ>;|sRvqS6 zR4tDet?IVlN@NoJqu9^|U<@j8JFWWq02o~8QV^C|vG*9^TS`ti()f9F83<(mBu7
    ?9@Dpq3=jFqDZ}0AgB@M07Xtvl4VXM}J?*gM6ynEEIhfJsVM9to)BI5OBBk{@7~;-^gk- z=^eRcOINX2R*KlO<%YK6yP?)wV*H!7u+_*&+%+M`q)2(1C2uYm*@QL0_ zycLQN)Tui9nN!rzq9dw@qoW6F&Gv>^aS?fE$F{z=6FjsSvPL{x+oieL3rg|(n-1{n z(`0q|?*B8z8_v?p6hb&soOWnRC6;oQcoX~v0rUvJ!l9sT53B!5l7UXut=tE%Cx<2J z0wxblFpq!T5{K?0re;eGHH`E8gx;A(#Ro))?p<5zbo|x4L?Nx^9E3PN%hPNBtcf$? z!NwHp$>f{#S>Ln-ntz6Gy1TuNAIDKWaEcxlkO7ut6EL(cc#lalPB;cY7}W%6Y>1jj z$tPxz9awavv&StlvaEEwo=4u$@hUomy&#tctLN#dgCn%0(YQ1+$4%Azh9ZVSX&%kx z4JG#OxfaZ{>|I>eGl-<;v8c8h|8$NslyXu}c=1Hg6g?|m- z^ZX&H=6z7?@X1>@_on+PlA%#_1+Z&$rk#NUWt6@-A9vlad7mH)O2`+pbLm{nr|1v> zYKs~7`99)AW~Q*u-XGN;FWr!dcYAb}o8q}%ZA=9<@9X;E`H(4bal}Hde`Q)W2GBr7 zdNN4ILIvT)Tat*s=#i`n|9f0G`DA8t$QjnNtE{HW(PMPNM(zU3R|f5nK%7W8dB z#c{L!^43?6!t*5zHRt5&2mIjvzYc_51V{z5Lw<{b=Iob)bC@@U+b&t91*EMXR>iWi zMZDrur0YsD-QItPJbqD#3hFAj3?&^|r)OfI6NSst%w-bK2IqdBYoMUGWWeTky9^tCgZ;5Z@sbbE4aiyQm28ZQJ}=O`iW(B}oX(qF4))ytR6 zQ?H*;Uc7idyI&?*Gx4^KJFgYegmn@i>*LhQC`EyVv`ViuAR$ZEh6gsZbO@#>?N^Zd zOqG-9^D0c{rRv)Y?&R?~e`9R?eU#`;ld4Yfy@c%4LH-J|ANQNf<=V?YEKn6(K*-EI z0WoNSBZ7Tr2h`5=yJp@Gp6!&WXmnv|%E~k~mNasBU4JLW$@~BE!Em#UT~#j^_vN zrmM37D5IXlvtEAk2sM7|9`0g_$!s$*Zq=9UgR!%-6ZGHf>2`EQKY1mjUPY)yb?zBFx>fj!}M>w1Y^yHH50`u+dW0-XI$ zd1g9YTEdNsqZSri@F`>>R?!>D?p!gfb2(pRQucA!A81{}#I&s1YVf=Lq63#K;*k*3X>=f^ zDOEn{!DYst1cOTsxbbOj6p+K%qWXD-qG$^UqO*Dm@CE#b)|uo5Gk_up&rp~E1^^l> zpd}O{w=LC2B9|(8Y|~l$_&@AT+_V!|@`{V!6gBG5*G~glo%~Zub6r)24SQ|4En2P-IHW zLRf{G_L*OL0m-94hbgZ*WbxJLzvU8cgmebjkVr)(ADx1pvk*;t0cjiN$AGRFqzHCE z0WAEl5C{$cdRn-`ra~b&LE72WBni-iwZ2LQ4%dV|fxfD{+m9hRQAn9G$ns4IkCv*R ztAG9b**2=t`G4-zzk`6wo~`@nmO<;)(pP6UKz@_{BW6#yqwzfHpX&S;b6V|NZ=!quOGCNFogjY=u%Tg4JNRIPLTyj8qQ} z*C{n_+LC6bFPWCu7YkA$f%QrkSPfkHubwd_6jFFscr)=DedLLhu;L%UUhkh%pBScE znN=^I1vB*Dg7II+0bUGEG39;bRd9>5ynhO6`22(ws$PvlQ_e6=sybUSCi7mz=OSCT zTj(qhCrAM$Sru~o6g^HGU>%|g9@h`%%!So?B-m~VGuYCcd-}tTg};bRJXyErWpi=7 z(HxdIlRQjjV;*k!Dx05I>`Ptwc;&zxF18}gtqCe)tOo@PKR6B#E1oU>?@8wcv#04+ zbfW4=PD;8RK7}wo%N8q;tazmMdAS@KY3ITRUj_VHQ?ZO?AdS7k`BEZMFlUhIJ1B^f ze%W!rf(M6h7-m9{ak~&6%kiq*Dhs}Ubq-x6iFr>)ULINhd$}xl-+rCnP3;fubOe*u zDh3=ju$xw5gS~+qglpS%Nd({n5oR;A&$Fv(yzSd$C^U{5wMm>2@0Ug1f(z??sS)FD zCn5Q@%5ievpyhx+=YUxsxJMmq;!O-Rcub}VMw`u#KP7@Ys@(*R z7XP4Tl?k|@NP6r{WG9?>7$wL$C~6zacRE47%SNE{CFeTJExy2obcsA*Qe^P&ShB?g z6pr?#54p$#ok+crr|{HG+rM6~)<+3`_-|N52DYgME>h< z<6=fNBEroaT25)8NTDQB$=1K~M+Nrw-Q4C^tIGHd{XNU7%tpfwS86 zGM$_5itDDKD2P#6Q4uX=!Jmmg(lFcskg;F zit8U$nTl7TNK3?qC@%=h2Z zHcui=!NuEhGz{jd)~*#d^78VEX4~n@+71rP6Yaeb;KaewlLyX?u20pLsqxCUdOwLD z(n|mfAkeXMwSM@l)lDT*TR@M9AWg7etbf!CUeyM-gIa8!rb_Jo-)YUY;q>)?;bj>BFa=HKuNYgUQW<9h)h)9;V~Wd6+5NkcNu6I&_A- z%>lFprFM~h-W$^I34Qjtr2^pQoV&?vqCw{`-0%`M7Zo&C+JlocQOgYqO+m_SzFM^! z*1=UTINR(K8a+zz9Hsj`gMZW?EVN?rqqQx(c*$`3BAU^|z8eMwh4^eCh$16^vEy6Fy5czp$ZOvAwCI(!Zh3)xX$WXT@-7Vv_2W!{qMMv z%`UNe=Ep<^b4vvGz_<2P(Qo9kqnTdwbNxVJGA$Sa4U$^4_)OM4jx)1Lg#E<;C+ba5 zn=vlj^OXe=B!Va;;55$9ad5zPUbMdS4q1V!ZWq6DhfDu`e|Q1kL~bWQ@hcr3M}6~n z+Y;xmdr_R8Y}oL=RMYlUy%I;!GO3YP`H+Wf2RiL=yjTfXuw0N09irkEQAECq+Q%JJ zz3L#ugc<59bbyRoB>PLddzrngMeYc7nCG%kZivDlNl0erf1Rlz)gg($vwF8xZY77H z#ii@l!O&~e+vcv>>xY-_8sY53(Ol*oqQ6znL(gOLU#rF$gM8Yb zU%c$2{RWLtNkCpt>$L+dE+&VJ5$d=AR^vqk84*45353Io?ZE$@d41xeA%Zk_(_GX{ zNlS}I44X@B|I36RgSf`M_GtWMEHT&QsnZ{4*4zD*H&vH{GiFG1%d{cej|*cP9zxd; zz{(!2@MB_^WNULrQM~sP1z^XL;wx0}UR?PK|K&vjYvX5f&vq%gLB0vqdWWBA!OBNo z-}W#4U%yv(zhZD;@|}Vi=ehEBC(QMEbnM|Rw z7w1mw3xq%_;OI<_H^I_;l@rx+lU;vV3h0y7&KeaE>+{pSyZ_??ZeA1x=Ed#!oT(-+ zaJ|J_OW!WVvSe47E0dvi&#N4q_yFDuvt>xcIJE698v#+cpO<+*j`HU}iTjC5#Gxf1 z+Vr5QvO6GWSC|2zx|$(=|JTp65Ik8T=e(Lv z%WuJ{m3nXKxBE(YoV0R;XYY0;t{egPdK)|ajV-^IL6Ppeuqdre6ohT@k(ze*B~1Uf zvns>4SBvvbN;z9m;AmvWN)Xr}UR;oRgqymXpvpuChUy1xqo44)5n~?Hj$bL7!TU3V zGpV~`>Mae zglmI{xm2K>Zhpy5m`gc!TJDbs3AoQISKod?LWESlLGmCEHOFq~55NjTi$wtfjatI$ z(6FE>M1PE`P|RlEm%)dIg$)l4IqfxwA_J1@t3$bzWjpB+lEdK<;V~h8d=@6OxGVuzOS|Ms9W)o6>`90LQsP#r&IiXwNqH#kbeXf|bUfEK=OuOw&*YM?Z zElt_q=Y}#we!zgL?n87BQ@3e=LeQc3<Sh47u0z!}Na-yPGZ?e!7jzrq76z zNCKRHG*>$R#_wwW(C~|P$(!n1CCB4H44M1NfO0SV&`PwUWZXLfC*ZSk|3a={g!Z zSV%zfof$PX8hR8jDVPFh2lk~S zG$^36Akp6cMVF9%jMW!RS*KMVJWi@@f%f^MOgryUM%+I zYGbkELscSOqwG)!LJ#uM@%I>;rA6ZB+dMD?S`|8k+xHl*#5yd>K8%`fx*3e%9BBhkkG*m=_EKotHxkA4bT7;_Y5l@#Q&e z>$RY>kvhMGyvysZOc{MAaW`?+ltN!mufuLG^p|rQlzD*rw=Dw4`G>jMf-^m5uo8Ys z_~op?gE5Jy-S7DIg_I=iYgu71FF{P#^MpPB^vB}{VQGU=+QZpO{ZRxhP6{^4{e$1t z`-|OqjtYccdz)|e=kRaEOPe0PO)@(SEdD~ttBs3nQ3pn7@j<;xCjA$yCq>;a3sYY* z?cw%6KaLiS{m%TcV$tjd+6b)VBA!_CF%e}3(7l{hR_MR&H>H3-fgE{bqelo|*;rau zQLq2o-2JSxcD{x`xmfyDf@E2!#@#}qe>1X9^38WcdP~IRRI?t{iRkEVqbghYR?{vEbi5%@S^o4(DRm1zjxpU$ecYJIIdGYCHV zAL8`7Ja*tjUb8m41i%JEv4Z2c$O!2?w#(CVD_p|-QZol4&pTW9oBZytf7lF5f7;G$ zeQckCRbO2$<=(gVmC8I9I&~#G98YX%cOfek=XxK0ny2teHdNJnR!OhEty4 zE4(QeyA!!a_>XC$l`vK!?u*q0PQNZHjVo=mCskNtFkcCIcc-K|$qir-9q#BR{eY3T-sr*SzUb@(U z(`B7CxsiNuZ}9EAiTi5R#b)R2ZX;n*mzJJdSGC_*R>bc^$4)iWh25k3*o=F0Np zEW_60ga6x6M6IJpxstK3D2cSoBI)v2wU3a$!Zg?~?XBK24J?Zi@B4d+$bD6S?*6dY zHkoylwe}^12nNYfSS#H8z#|-;mla3?AoDiJ=)V>9Ad#y$k)%w}*R>X~H?aL?!*lkz z&qJ2`3bObQSZ0Fz$yUJe%+luJ_u~0l*Hfka>xGD)_nR9kR=HLc9^uK$+^IdVV8){B zb$xEi_i2w(I*yd@OY66}_u^dQKgTU@ZE+8mfpT^<<)+tJl*dL zF{y#;b0H#17v2NZPPyRA^`4rfyzCU=cVPkteZT=`J z>o?hh1%7f4lrJ!@a{VhLZUnO}X;xcyVrGGw$Bl)4b7NejqDieroqiKJ2<~)jZ&%_r zyIoUEu17S;$jl9NwX)=*JMTBW5jljpsT`mpndzAfK%M)4;4aG}`#@adyzbBA)4zcm zp^AoH*Jh@(OZJ;`Dy2WJ48#pwJV791m+jv_lEi)*FI5qv7yOJ&i1#F0es-`FebA%HCOK{^YEDK}2*W z;$9k(QO=MS6Zfg(;{t!l_4MJ)>%B5RXIwt1a(qp$L5b7vP$-Ti{5|_#MKBJg^ zRL*;z=PG{5m4bYd|Ame0DY}n}?1$Bwrb7HF?v#G3O9ln?%L7;FJufvvG(NGSX0;yX z-Q;5i5KIBV&%X6b2n%rARR8PzNqG8Xj5NnH2ZM*5<-8{NOOtcX;~|&Zi;citc0Egn z%dwie`dq(Eu#b5r8DcS=fgX-~^+N+b*i|>G$UFelNQd(2*luU!JRCLAN#Ui52poU@!r!C$-`4d zNEwLSYd<+v_H+4rGl&Ff@p!f4|JY8tR9V*Pbd51X^n}h7hw6zWW8f@J_hF$I&K0l2 zh#FSc^Z7G9Q0;0kbN8C648cwrgBxlGROsl)Uabl6qX_aCybes}q)q1V`tHg^7C;Q< zD-EDP1@UTJ;}H`O5EJ0ENW5(jYO$1-RtfsIVgrA&^YgE7ZjuDN&|ru>PP%JVl&~Mq z=RPxOF>~hdz8mGMT2Q?uJDQry=|cci!+2F7(s>E5@luUoysP$jk&RT?a{}5zf!lZr z2ZY>1|8WgzhR-u`{_ArKO-`VRorg_gLc)4ygnwliLAUteQ8FQ+jbj(DwjGm9bkoiF ztyV#S%*@Q)>qqdIjHJxX!^6YHC1mO?f=MZ(iOu~`qQJ?K#KbXNm@A8-7#o+}>E?Z! z`^Li$>$YIu=JBp)%$_uoE>i*AL1_7QUIz7YU!~m?O^xaPn#Dtl&pNO>z-_mQQ@s5E zW;wt2Q$k4kwnX8AuyC<`55}fCuhx+u;S`uv^z;}9GGqK}M7!mEPuRqzmXyy{VC;8Ow;vyME&@F&NZ>3;g_!@Hd zeI~=&8^W!RTEYm3zMhZ2dGGaQxqQxtHz_t?y|8=XOi3C0zQ{-1hb)3c<|Mx5UT&w# z)=!L&o4LH`Xp1^x*`SJCBl7ZODj1>}R}D?(n68vQfl*)^}=!hmx`u??gp#T^mhC#>~lyJ%L@GUW;v# zAsE|J!o=VIL)q^Jlg52*sEznk^YtmNIs;%L>rXfF@jCS}KEB3Xu~GYNP{6(u2M=G_ zyf1}A46G%mFiX{3V9(KFDp5aD0_mdFqwyOcCPNjH(MBt%k|)^Sr&=k;%adaa>4UW= zW96g2pOYil4s#wpHF7o|Ocf6$iTzO&&743j{Q01y+aCYTH4-;vk+GWhrk7lms};zx z6X{D2S6TL)(fsjvSd@wV;wE%`^3dgf|91s7HrohB4F3mSm`SoI9aD@}mA2b+=N=6a zwKLcL<)|PBo%`y#WC{d`i;R4#hk@bqSQDCcy7*R})lw)fB$@Po;6}Oso7(BQ?y$(K z3iWLY3uyp5df25{x|mciJ=Ko~`0qa`F#PCmkUxETze(%XaN@SrZvHlrDRJ4sKmmsI zGe%#*A5#^8XP*_i_0OH|s4&Kf5uD!N#wyo>17h-Ir~qkLPr)o;8AO8i%~TJ;z6WM6SkYug#;Yn1B&{Yi6OUcat*zci_vZ}dliGz_WBnw6@if51OBB}d}ryv;zAf3XK1>0vnJkA@!^RxVO&Y;2t1ss2o4?yk5n8_~OI$`AAF)k%l$<-1SQe2{kQRdhgz zp1B2R;U6{P!ml`B1kHf{`Tqh(;*`tFg0QRTy<^K;ksSgGv6@5_V9#~~>&g*(Y2D2> z2-wAxb#4Bv()~k4I*dP?)n5?=v3~9iD`+pgMa{_p=bWzS7=Bz2YrVP9cUMwCt3n#- z>!(=PHP)}uve;Ntr0Pwsl}I;oF-V18n6l%0{LqonC`O1qL`_DksrNePPgGF$c3$qg zt{pMuFDWgxvgyio%>Y55Cs3Zj^Bo)W8#nlfIG;@iqW)l2WDnJArdS$ME~14(N0jHk!@%w z(N5`W{AlovbpTS!g1HtKuc5{{*a@B1s~|F7mp{|J-{Im&gusZ_xD9DZV_pU={P&p5 zI3@1Na^2gL@(L{r;^>7@@`+Y$!rp?Bl>hV_D?EiWa1()k{v+d~UHGOq>Be??35UOlCigz7GNE=Vm$T%*IkzQx>{E z;ns|rR( zG*s&As;pN+y1D|9j#$n`rb{yND;pOavprNt2BUbu(8@TRFwbU=B>85wYRS; zMj9aeGyc=1>dP}*Wu;%DssL;D%86zJAE%dGDCI`CxRg|!b|q+0FnD~PiPHb}25fLS zp6y9H&Z7TO*V4~yXQ1U>-08`lf#xKc(TnJ=`ZNnJ7MyuxH7{wg=sW#c7U*I}lj{o; zCy|3rno!HjgJo;)aFbZ>gXUF=baA>F9gH$ut(OL7wtEq?rNF$)~Mn z5)rE{2ze7s8g`jQM-1*}M91OZ(!x#fDXm}9+DfO+VhT_WGSo{ z+{Gb?6UQy4&b?SZ6S#OkmeJ^V;L(s}s4Ot?Nl8gbi0#+i$#N1Res&v;HP!9zCg}^w z*;-)z+mQ5J$HT|}%wkAI0la+{$Cz2Z`VFO#`#;1d0&NiW^qCZs_M{CumMs}O+Sqlc|04|n%O{9ez4M|tfl>vOA%j%Nd%6psnc^ zk*g#guq%Y$o78G3pJ3HEd!^pkoQ3mS~lGRPIv7F!}b9$N7Iy5+f|Wb8z{TWZX;?u|ZqkSXCn#TZ`Gy>mNd*X&ib@<$Ju_TF1L| z$zStOTA*NU?65t#?7^?5(dN9Ai;aB{KmOHl>WBNL!cUCYBYL6`31AtPii?hp)kNLk zWzL%nFj!C&SW;eWe|xA{-StdQ_2=~5^QF68a-! zPiFlVkK?|Md36tU?us9w3m_4;sQIe*jh$1{0D3 z+m-6HgfSOZ<$gpRvX_^D7oI1%!1%$A#NK>A^OJuL2g9AMLOj^;<1U*8Jjg1<#(^D( zm;{$2|5qV|qrGYTR3Q@${y16CgX~8l2BjX28eaWBhL05sVGrx@m2f<4VO-1Nu{f(b zX;6!%Ft^4g90E9CflUpey@LQ?|Izg;2nF*qg)zXC4pBxU36ck0|EB-apXE=tI?vNzf3&gD~T0(!Nrmz>r-zXuJ?rr~D@eybr%VBh$U1 zn4FyR6-GWp@-s`z^>&jZ`}gy1a8O2oUXtRo5+rOy;m}NvD%${lEd6rerR8=RdOR5o zv=ksb0Hr{9n9zMmMBVUZEbnbhCKEDJ3_(H4s*D>C0--gKzmNGJv2pcL|FF5`NTkzs z^sqS|1P3pH_*V(5Qzi(9^CotKsi{p@Na)+IlwsFQ%sLxBYL0;rPplq@6Q zpj5z8fe8O0ORZDycl*p${_z0zJ^$?>n4ezTYCbF{g9by}iWX2gA$Mvs+GL5fO#wA3 zQ+|aq2ro7sji#CynX){GB}QLnmm*~_7;($>F71FXy**nucu#+1*3|?98 zL@cBx=ocWMrx~3M4?}&g7XXu=u7)?QF^^UYAkoejjxMA3fw8SYRfA`<62o)nMAkY% z_q2veNL-CoW+w}exy`BNM8gPWG-Ta#n>*MsmHXX%qmCEIP%{&o%uA*T(uDcPN{EA! z$rM!cNKg(Xqi{AlJh||dgODO@!;@L5q_ne$C0LUZ3B1t(5omz>K8*&s&`4k9G?wg* zRTodXg!pL{NcitrnF=`sWv-m+x^#BGBS`~?Bf_{WWvu8Y*sn=bNH9&rJWW? zRnHKos=h|Zh^e}U64MEcR*1H72Q7QNs<>Hj#$G!RpoQ)2{3si@65&X3Sdsy}A{A=7LNaIve4c`6pc}d*3m9Wm?D!c+70bl301S(4 zF-lKUKqAD-NPz^yqS04mY*T0*HC@i2l3jtF+ zcBwfP{2W|p=tWxaiS1PIcVyx_%XLBK^Gb%TPM(so39$W@&6&dQk9)ld2@Qcm8?dRi zQdy29-wD7biy7AP{oqECE8Ax^GZ}1Jz0CCqKK}3ZxX>-@y;n&HhDQEs;7wLB=OMo+rGhuQ2~y- zj_#ZHv)?2S`ts$8%loJOYnln?C0^=VV|{MCT^mHSki71_r1D!ayv&hVTpXU}b%iN4 zWcaYi?^m{zgRYB}`5M4+Qp1MeGVCde$;qQstD;+g7!X4*xf>7-QZ};@ zrTK6ih0pdQKgJ>c@5FOC%$10o2b}&NroK9^t!C>wxH}Xp5UfQ?aVx!@Pj1mlbN$;X6?1tYW4PjB+p0GqxfRVCirT< zwtiAesbZ_loEVNiPHZkGbDPs9PgdzVOeUT*kqj@ z+kx!EeF#Sj#M6%1Y@GS?W8G{1os(wYW5c#N;OSAmGfoUXB=nGm8oq3Iflb(7lBEzG zU*v>Qy-Wemnz;g6MMg@h!d=waL2!}5B6>1LH-p6C&i4|#{|^C;>UK*@3n{1Na0dHk z@HD?6a_c=T261TpR7x&QT}%f(Jz{KguletkeyQ|o!i;V7U1;Q!=HDDHWZROsqe>eE z)W!zMiG@Ar2g>hg$(XllV*5HUx(m=IcU3~!#)2($RAi2_~4XNc1w;m*hGedX(i*Y5Su(wz3PhJ))z4 zdtx7hMzakcd`vR$BB9U3^RPqv=Vp5a{-)@~R*9rhGT0BjKRFik=`*T}j(@WbyM?T3nXWHy{?1shH1C#IwS$S-;mCiyPR@%jgMLf= z{XLLlXeb{|cm4)zN8h&?v}|TP{W*hTO|YS!Oo2#~ zV&g>8oSftA@klQO!+swZQKiZ>3E4;rM`Zk}5FB8j`zIR6k_0i&M5cezrTZ)Q$1N95 zF6|ttSRhP@;2x*>z4Pf-Gm3Y(a|G_cul}js7LW2J=xaPQH~)yOC@Q4iR71ezCEu&q z=EZ?eEF9OCDjFfYqf0!EE+F$NX*wp!(*1H{o>9L}2D_Kup0Q^G&%5O_gMt=)_2o

    @rCOxF^`9iWy()~f`DcFI5B5B@ z$6Nlnk{F&6dxJj`ET1=N`hT7iOK!b2YWf<2c*`e>KX~L*S8{bngBjb59YrfU#^s`xg zcbQbdSv&9j?1wyXkXY<6dJgi${&&>>oh8cPObk!?^Vo_nzQ9p**C{?Ru!t2ocGN{=S(-TpC z>2TVW%_(ZkRIxE2vYH{&7}NN;Y&U#$D|K6>lW-uO922BfEu{~()XDM%vpx#p3EK7H z4JnU8z|^SqL802m5y{##!tr|g|4ik7F3NdXIDU#kGE(&D{t0mE)2i0jRnw?`wP$a~ z59p(YG4q0607|a)K1ehS*1x}nfcIlW7E`;wIfh1g6@Y_F)zCsT21$UDm>?S)?ty}| zI#$&TvTx=K4ToO#_}iT0C35nb7SET6p(3{BwXJP|bxPs{*w}huB4J>3s{KPvo$i8A zka4dv5_`whY__|dVJse3D&UlAZws9MF0$j|6mjg;9jtVUX4%;6OvN-8x{DeT6GDi@s)Dp%(}L)Vd8p5=jkLo+4c}gqjScQG(lY

    $E#^rGS;M%141paO|PS+T0Kuz`6mGE@j6?$Og0CpC!OZV>)E_{;=S3?iwcHkuqF zjt|YR0*B$F0bL35+sn$l5 zS+6XyNEqo!lsmsOC{amBkTD5}No7RZ$oG-Uov>KO{ImM}C+G1|rsD%dq~1ICN-O!i z>vfbsd)szO<$@rf@r=Q^K#ac1sMU5V+#Mj&tmIHc;2gnRtKoD(U2?2WfQ62Xg$|>* zp04g&qfVfveX&1PmOVXCq4qldSruJab*JP0rG}a;2v9-WJ`1jOTDE?1HMnW? zurYS}`}%nGQ9}}ZxLt4B8?9yJTT=LikMtDg^)&oci=7EV#GOcExmr4CO`P0q6BhK{ z`F#(&xyTO6mpajOme{3fu3O#$(`l0^V*rrNKxSd1ivsSU}cd?~9u ztnBawcrT1Sq-FTwGJTsPBO`^YS9&Z~V|hzYJIuB1Pm;cNzEYlJNvX_6zK@?SHoDB) zpC`i_@2{!2L>Z1-M*ari9WB@(o9`rOsfCguw@9l?l`1srEOU zYS@gxS+PDeYA_#c69~9jkDxNN+54SoHI_BIHrH%5chc3>RbA`odpn~QM+NzCbBHiS zECj1~+Fos${i#KUvXCukCE5-<{GLlI1M{1KYMefQ=trUQb3T@24b~1b5rOx58Geqd zx$Nk$aCQFr+{^UoXulTLbgif~RzLNrstfew?+ZP2oqiU1MWV z1EV(Rl|`5s#DZX8uAfy1cGFuTmwPHWzc}G@xEU`@Nvb%U$N4zp;J&i5@>-D>Ka`%& z|6ps7U^ky@TG)J`ZDsOqu)nQptJb0NaR2z73lt7rV1Bp4)yrJ%Q|V52Q(<^2M4Ml1VPri}joRVg;Qa^94Ks zt_9BjMUayGj)A0&a{yGds34&8j;X zwK;qWRqWIIgxc_5^qjRX!(lZW%dAy1&xifkk)wd6 z74fy9et*a5`3S}RkXIje6{bRuI+j%BTX$!#><}PV zifiM6&sR%RR`SQ~J_3^;ND>iwW|4b+?jr6tMFBnUAItmu2L}|0U)qW+1|KRKE#(DV z?#P%)l1L~I4i50oqafc~lbG9c<7Rs?N=LH{ec`d7eF%=r))4oD6zBDlow(?sgJwHT zLzs^X;@;P(rM0DkUuiX+?*R@?ri$;0I<&YhVZrI^%Hr(ot7TGUc@q#81s0rpnmmgg!d zEN=R;H#_n|l`SOY$5mR32D9OtNTSjtG|G|UpN}8K1FF8%)g^>W_Kg2hE(CtUv#ekZdtGZl{-WVrjKf=9JJO+n zC>Z%jD@99@@p{S05W#o=19Pq2{eHV-DhB;Qh)9{lds+<>2dh`DA|z>2zRmj*OpwTw zLbAmQ_|B~CI5cys2`Ugdx0`s)BZH}lTClluS-WJl>zq$^6v*w$6fYB)if@_=sIf0M zNywQr2qoKhW|uAvGc=X=vV(TtdaZ9`F_#JRvWOxfRXO;NfW9pXkk#-!opt;nCuZfp z97)uxHelSDWSl9Xr$22!zgu#$2YQRP*i+Negb)tm_3yE>pI^nUbNJ{??;LvXOd<&U z|FqTrC!)U;kvgMbJ@`kcyuag%34SGnLu@mG9U6f#`@6R^_X6_0@Msd8j3%)PXJVlf zv0r(T->P$C-X(neZY$D$a8Mx};OT4D-7J3Jscg3DZO1%-5IUf>l&{+xE%9fxfLg}f zY4~#`nR19WT#ESlq=~2`!7NjTI&>(8`De^%?rCi~bmja6r-sXLY@moI>{GLzN=TAe z*u%lW5QXq!K+*b8u)k3P`_kGnG{5TekJ-o{MQojR8txpWx5I&c&Af44$4;Hd zcG3979{#_}E*mYUowTnG*B*8TqSSn^b1o5SE_ypzYm8te#?)T6weY4!mTEn?6cOG(bBJ4%7$_bdi zad6`4EXOjvJJzGW|7oA^T4?^*krTBQxuvZn;dw8aI<*f0q3XzCUPGK#V{ODeza{G!u>3whq|MKRCF{{HcAvRrMf#x_#&A z;h>~s#et_jmWhAq@_V2F=!Q0dJ&Y-)8a53DBHzRbbQ`BRHVZv zn&B3FtwfEvB33M`euhc%aqHqFj^ZoNyUo4r|3Aedy3eQ2=HG~tNFqwb`!_nTKKu!M z`q@|(x7NOb_Uo(7%;WiVedFBd=XZ~mk9{}NlgN@ef@V7rQ~@NB3A;K=pFSuI4QFnZ zVlsG}&Y;U*?yy$rwLN(R)}7rF?~G-3*zHOMqk3O^AcJYTX)7`tcV-paAmg4=8gov| zA9j|TQzkYxCe@)-%4YHaV>jeIP>!?KFiUcoa&6#BC<8;$p+WZM~#?QKqh-S7#iY z)M`3&klCyz{J54^GTp|>mJcQ8oy`;Qbi@to?d>IdjijZnKC1KOOIgv!dHHPJY*~PB zao#hq;I`(t*tAMFGE(~L+7uvc@4`Ch#`G7pH7V^7;OWbrfK&2+j#hD2p%FERgPTJ6 zvl}9rABthmM$b1+SJ|^~_3BG&I%8w^_Z@}(ul0jV6)6S9ew_7P=^gx(Z4(d>TKrmB zP(aGiR6_?^8h(?D9)sS9FO8te-UD|(Tk*=`!{gl*{Y-O*=bi9M$LWXNWn?s^_I70X6Oo3N|HkZTaf?s**fSz{bFN z+AKLDYgO>gUcFl1c&YngOKPfzr-xt5<;TiTBXIqwT?r|v5unH}lw0LW0uyRLfkUoz z65zOXL;F|dCr?P?+;bGvv&UaF-;E#u+QM`NpVHQ}tDfhmOX2xh%?jlVm+<^I zJ>gp$AxLk`S0ukH!{rzD6n>7Y z#FG?7nm^G*>I%XoG}aKo^q##4$*<~S0>9|7$3l!`5&@I5h)-T|FibIEfVbd4-kBWA znjwQ9fMMh#u ztu)Yxq@J2DUwoEt5}h{DR`@B2xKB^xlQilfPhvn0ec+Bkc=)*yC%0P95IBjavmrgSCd{KhD2oUAUe{y!v<(_;5&NmW4=cRam}g;8;3= z#?zs5Ytb*AnSE-Bd0P_wLhIFts<_B2mCG;Ya@hi4lbX*e8Y*wSrOZNHyd=SJA}SOF z7Q&r=c^dy6*T2x3KLjrgyaqQ~`EKy&a%6MgIH2OcvHTMfM3VAboH69^;{;*-X0lN5 zZd0Pf-nG~eNi)hIOQ2`x&jq3Q`o@7&VORS8Qm{d{7x!A@o6P|>N`km+zM{L>T=eHA?a zb2;aNJDKVsv5NxZ!E;YM@$Xy^h`JQY&Vb>uk!+0}<5b43wxDg-DT+_`?n%LM0 z=+-&I)wZdgPYAE`55sPJ- zqF$Ovs0!P_&u@;6!`6yxoE!}K1Et*vp`Fw{hGa`;w~>S+H{3psai9< zujiVhKw=z%OIJxvt(kx;j!7$rgJ0inJUa6+mQ&)ZCZ&lWzk(Sgn@le$v*0CKl56et z8zpAMUjZ7r|C#rHpS&)-Y2Jv2@-3%T7TA9R;V$$^abah+GQ{r51g|t(9&)>0?8nWd zn%fx`&P_GAyxXf_V@#$-oVeK6qd~Kl5rN0$1c)v+D|g&II;NG*N4*-AT5oi^vS?I; zkm$fmxOl-58ffqe7P`X}!RLJkHMlD3i}F&Pt(rK{=(7TGRAh9K%%HP>3l;x{m1|2` zk8d8U5De8IO875MI;WmSNmrDCN;ot@LS;`V!R6N1w76?q)k*8noLFP>&|XJIecC4W zW(zG5Hwc7E!k83V_KMb|B?k!-NAz%~CH%aOv)W`E_8wzbp}Gbn!BF%PE^>y|tKRYW z^FB9Hk|`cFDzHz)l>{jyqKlH4_J`U`OPRsjVOI*)8}fM1$dG1}0me);PCK!?wt1DR z8GFiWt=VI0n0TP>svcR)>(F^@`i!b~M=mjY|94Zw-S5_y3}|RrXga@nq1Wnh1x=N+ zgEGw^t~jvtEBd*u%NF56nxpLfHOyETQH4O%Y@r3vGhBoz0b#GQ`gL31eo=Vs>X4u? z!00W!x=1|ZB25%blrT?GO@oOfyKexqUVH^lN9cPzlg`@-SV@c%mlPBf?CfMqVN*Gt z?u9!<4u;z+kT;^nUSE0OU|~a&*JGN3c<7p|V@Y%U`E-LjQlZ#qOW_NvjbOstbRg zB`p%*vH&oYPYM@@oW^qAJzP0h=JIdy*XEBq2a9Q|6{9kg9>f+DlMxtV8>eXpKz?K88OgNP$CL}>vB%lhm6b*YZQianHEN_c#HDsT6 zLW`F&L#)izS>H^|R(yq@8w2kn0x8Kz*p!Pv-tb0|>dgtE3bub<=b^DHr<}Y?hVL25 z6$2q~RLS5lhL~4zORE|kFZmwam?XRjbh+*FzJx75>M&)p(X2dn!`*|V;elNRIG36S zj3k1ISk*U_chV;F2M-q?8w0S1W>X%EE5l`EXK%0tp+#WJ8b*VSvE6CL;>659doib= zp%y_U9t7FL{wY`dgP8)wFDWQuRG)bB57yaQJwZF45tQe1*47%{(l5&2)887PiyWjs zKCuR__-qdc*LI?(Ff%bd9RSpk4`aEPTDoa(^_~uP+{ZLhUq=AjP$BP|9(s^OlY`r1 z%bi2+M&n9efZH8c#%#4go6SlV_L$Q3)l|4Nn?EK_R@^bP?wdU>jbh4L&T)mp4Z zhAs9BO^c-XbfdWtSD>#aGy#qv5uaHbwE}R%ncd+_EzNSm!M^HzPR^TEpQYP4UwhnU ztu=I-Z+))gkS$YxK4A^;-J8n6N@wkS{%Q1Kg>?RLaIU4bg^A&oy$0}jah!<`DJ-0= zSp{P8edOVWO;aKiqN9Gc+w3^9b}g7~W6)N$j-pt@)d)ERB%g@9`g-uay!2 zgHhL9v?`fMclgDxZe@>6lD_R`{c-(tS^tdLzsZXdGrpffQY1Pg1m3XmgQDE#?IEud zBQd)656Rv-)+1(bN_Zs%6V3TIOg~Z%hIrTfE*E)Ggkc}3HT8M;KI7Nn)FE0a2D)wB z=!Egk6Z106e{nfZ8_8Crd^hRZ=D*%RIk3>o&*kom@E5t7s<5>=S}HG z0k>D_JQRXHAD)BF+;&F;Dgykiq<)4~vY>XG>v|93#^AGSq^3`^_)@da&&~Mue@Bra zX;7E>4)R4;LJ<3I+a2gtH=#F!GBwLWK3nY%*n`uKI8bmm=H;~%DAGEHUNCa^%G!NDh2x687Kk16H{0{P1a9UHzKEj0WEr|cX!C4rsO=+9ltMT z8>N)|7CuzwH{6Zz8rJ5jE))V<)Q(U+pkwUO2z)Q6FfQWmSb+dt^l{2tywIFayf_4oc% zOmwuXDUVJjm7i>FTljV^2<@^`P}=+71s^gav5CKBDUpMj*((E~bh5J}mqALd+Dc>+ z?R)cafR*trBkMV9%Ze&}a+<^sbk!Tl?xxoD5jsQ`CZ-F2*tUnAo?bRS3WIqHeAi!8 z6Y#A0C^n<*BBQes!wgs=1^hl+4>Hwq^wd(-!{W!SP+hX|6h(_pakdNYaI8 z>)ZS5;lVgB6}Q7`Uw4ht~R4Q|p@? zVfUR_-OrB3ATj`z)rSQbsDLAhh*AOB{J&&Wqrhh2{ERvBKt^zpzOH(^$4oS>6t+H} zr{=G#la>2>*uzQ)KPRivLCL@#lD@KmYHvVQb8|^`wSve)Rm6`0-Gxfq-$qs+ht9o&LSle;CxCE~nBBigXSqgIYW%)Wrg`R4?u|Uw75X;DaVR@R2rr;8?p;Z7(=Fm9ZNRF zsz?qqa#l^+-_3ZS}+eXb#hVtSEFuiMumvhOLFitEgEvQX} z8($=#Mw6uOIwuwQj*XnuyOZAD{;ZRIs*=`t_j#f1`5t)AnZW4JlfcV|;JDbb-cnY9IiJ0N zdPx&bOyqQExJEKZp(o$VKNy_52LhjEqzNnW8`h*naT#0`c`DlovdnC`v~NGsbp=;_YaH7OBJnb|DKPrzU&5+KfI;1TDAk!(n!!1N>%@hy=DbK ztDCv=GeFAJs(7T7lr>N&JV6vSDneBkYOL1?uhV1y$e~_~qpnu#^+3H7L6jGVh^PuF zC@*wm;FrkhcWJ1{Ng?yxB80p1wVS2V>3$n$5++@rNj|m+4WoIKbb%()`=R0#!$OzY zyXZ`_T-G0v?3Qe~RaIF#c!Z4cTvnMS@C?D)I>`u+f{(P8ckS=f$#fyygvlTrJpuTkAjfZ$!pcE=u#hTJry@9F@ zGV)t80V4`5-sjGl?O`@GYZMUW2grybudooKw&&bw#pOuyM5`yz9Iy(9*M*8ojR)TCBz-g0CRT4iq zFHw6pk(zQ@BPZX2Rkgs2NYI zW~Bsr2uN35U&w9rp;${*U8TdX7rK}Q6-0r_GZ;9uHy_W;9ke>ybk3-9m6VQafzXX2 zk2!I<0f3maaYsH+b75h@`(k>*fz?6fyoQ^{M#WplMO0XFvjFV(P;~ufzYaXP7AS>C ze!!c@?@4a~xNdQ!k|Cy`B96U1C1C;tdDMvCa#tB@3GZ_{~UM7&m;pFmr zx@?w?ysF@J;;KI-B2NDdb7o=N+dI-bdzdVdb3zHG!;b>WOh_0nn%vz{3@4f@#y%A% ziOTkI76t$w9&je_Um@F0g9jD>B5-_A`(<7u;^0OALdxp<2!TXoL{D=1`Pc8PLT1UQ1K94mAE0M^aIv!&=ACb^@yy&p3aR9 zc&m+COcf0LNoAmAm6yL4+}J7j1Mjv{@B>G_v5*@wDSg89@Z?_Gk0(x%*J*R~cW_7I z+H-VzyqH=!PsCfF5AN>&S*i}Z-~2u0-T%c z`F~}UZHP-nV%3*z0I)Bdx0qNEE`~N5;oJ7!sp(%LM*AVe0Z&V7r0ZTB6SI?dWjIfd zKHcu(VXu{p-Y`__>a_*{8}9SO#N5#(&pVW)Ei}I(0KuaY0*aQxUcTMmxbp%U?r+Z3 zXl2x3osP$SpJ?_@_iYaku=q;9j>!`a6(UB~MhO_?Pz44UFQt8g>35)sa^Rh|c^5^= z047(6RXxfkB7{a%e9(jHwJ#M-O-&_5*>t_A(k=jQ`~%UUT(23BhyDLpF4{+gi1>7^ z*>>Q{L(De@63r^qZIAEV0edm%;>v1^?;m1u83u2X<|nVTDr!WtksngN#46@OoMcoH zn=-Tlz8H+mAT-|j3Ad++mQ*Z@?^|yD5XwH`4O~!FZ2LFNhDC3XJiq6HH@}-!joYUX z#9>^!q)ahPI(DLRlb%qoWsmOrW6lvL7FXU2z4Tab!O_dYvnLh1zZ68^XpNIS*w(ac z^e&@4b>9D8kOFUl*{S`WVyTl@G8MdkW+NeiP6NQa;XZ8e9Fb`K5^$Q$p4d~;mZfpf#@nwMS|mSCCa7MMMW8` zzCg{Wa8JT|^TiVnI}ukd({e zyy001Ha`xiaf>N)3q__Py-W!46M=;K6PGI|!t+SgPLgE8o%iB~L}x)^4_B;zj+Yn4 zo&D#TdA8p7OK@8b@NO|i)iO?wZ*(JDoll3FAP#bZDJt>2v{$#{qxPPu@fIRmllOA~ zujQSQpZA(-Q&$DSPYn%8gRGA_7S7If#qFj^+Z1vAy=NkUIywtIc@BpXQsY(Q?V9!kA7lz2NYkxFAC#%mYUvFT`h8qt zm$`Ff)jgU$G1lEPcWi}**Zbj{=YE}n*?n%w#=i&M^ZCJM8;ew+Db@(^wCl7pOQn}f z({JZkc;JYi$2b2CdDUUBDuOnQ9>$6rj3V*_=r|~Vw}7+vz;Bp%q}3vP$^Xqt^h=0Y zIjBn6w8(?nLT*ul$;4rQ&hSoVzEMgMgAtSu82U_7oP9TGE0TpXC8)_G?XjI&WN@3e zVBh-DxYIKXwtn}{ZqI^3=b%C+KHlmI8{G?B&+pm)o1*R@UL})M_A<{%Tc0H6&kEz) z{HsI5lpbd{J*#L_L@%gVB71#{xW`IH4fXMb;Jb{Xr`g2#!{kntj5q5qHsEfH`CqO{ zUGP~m1P^{k_+>MMAX^GCp;t(A|qc zd&KHxBGcYWXg&E|ZK%vtSykg&ng~75UsvzX#wYSQR-;Z^b?pUQw0>~k93Bo?KDg-T zz&A(S+!T@v4es#oqtNS3M7HTxD{2W!Idn2F<#H1aC}Q;ckC9eB$N#`to1#CmTAWCH z{bL_3zi2l75a`Q^aZ9)L z`simzyzd3~2v#T2n)D{lpY#{RC`4z+<0fFCL+R|cDe|OkU7HXC;{^G9m4S9n!?naK zcA6ai$FJO#no5i;Y17VjZks@vl#efI_$A5Wv6Lf~+lo*FIk4M}^NZy?Ji9(#nyWD& zpnab}wg+?Dl-wL7=wDn~g3C*FsBx@}wImAq1TFpe5)Kp!r!k{^OkR&yw_ZIs&)PJu zCn*|)GQ!XtCdq<2DeFLFTK=i5g77w4GkDQjc|Rr`0uqvVP5ivUPGh{?)w=3c9dOl9 zNJ8E_J}z=ueF%ss*|x1%(3W9PI~+eU_Jqk)z%^IJ}hPITi0O1A^?7WXWTa$(5?HT;U`uY#O zkvSYH$h1cr2I8!*;@91O$1Yd)fD9g9H#V9 zqRUAgxF&lZzv{~C3z_|56U{sq6PA~x%&psL_Y5V=tLNRZv-@B^2~FGkRHP`0XwsVq zv;)6{4A|lffyhN>YD}Zl8DH8aSx5Z8fHk3YKMu(=!s2#ILLIV>`pTl^wPvfkQ1M%)PF>k;Vl_GX4;c27hir z0`+JiaCahu$S(?V*U#RGPNv}@h0o(AyPHiD5{UYL6ceu__E%u?W$XbZyn>{HkisE8 zt}u#)1S4IclG@7ck4nOdwV@ra4{o|><=|l zBQ^O08(CB)b`zstf0aG8abz3r7~c0S1FcS|L*S?=YD9Tu1{L|JZ z)b=2MYEwjZM6{kGa#{NUUWwc|LoLt|(h+fL3UH`Cdi=dvtfi@<*7jlhVC`ug7TA0! zte7K^2U_ZUp0jG}`~7(O)c-5zNAa$eI{$w_q&GG(x;!2gB8d47rIvg^@#KWn#?{Rw z0Baz_&v`BNbr=4Y@9;;Zud)LTr1kxF@U9br6xjW=qG-mdL3HdK^RAbgL-_?2SJMle z!pl#6l}_ydE7C*Ciq|ykkjEZ4IzXZARgFbgKga&?th6gIfipP+0lA4QBP}gWtlu6K zkh7WF%QdkdVcW3h6(_y7c@MU0Pau8%l8fhu$NM(nCI1ssf-lBqD+fSw0p25H`=zzW z`}C={W_b@c&OyKJ)t`6wNZGv8Gc(nhswxvR6W^XMS9l#;5OZN)fi^b{Ia{U}1|q-u zXP{5R>ES>NMrPg70ubp!Uf@} zaI089==6_3@Y>eke(pbg+Bh_LcHbT+);2pXX8n00K`$j)q$`4K-=uw=8cvi9yWpj8>eN2bsXVsOp3yhkg$9d`oO>Q8!#2yu#oEsKrIi%}E!b^@@Z&A)aV(I8O` z`ud{e8hyK{rs`0t=n0ESPY&w3G@C)yGX(pD5Zx*NmkuuJd8X+ghzP-cNf! zYO{3jtwypm^k9&ja|$t&*y{GTosd!YwpnWO`It~@QQ5ec{4JK}zA` z=j3|O3cPW0+a3Pm)fuDVsPDw*eqo;fRFZ9=R;mujGyvYJy@_lbVbuxw61XA{g&W*< zbB28Gdxssq;mT^7Dp?%v#-E7N=MPc-1PDnN&0)y>)T*fva5bI_<6Ofmur;o8S~v~l z0Mrk53^4T&K!gl{5XH$N9M8=V_nMxu#3`R}gQ#ZlCB zosT_ECws@@HP&L_J?L<#mamQ%rVz?+9xgjA0&n&|^Edi9VgsDni|Gm0c#37-SZFp& z_0RO-l+Q%NUY9D0JvS;+WTyAOkwA=hz`_|G0WF)IS6U1~!YK5AHlG_USQp(z-Tb>o z_S1f+Tm=-T`Zv!93Lx*rctiWZDzC8Zd@i;IH#RoHFmmG!?d|O!;>d6z@90rWrmAA; zvs}4xvM5TqbPcV!n%A{mj}%-F24Cx4-*2L)s!eTOp{rn|M-?S&7kW@lC=K7RECzWb zvA>LF5;}^Baxf`8*P$tuOCk^4(%HBtM(YK-tF|5k2vkw|lxsi8YUs%9FFAFbUuQcl z+~Bcq0CFEUJdunbR6hFY$weV27Y!2=`ye{z_L9fE>MADIw~W@dwyN~FVTr8jW$I}} z>4eK)T)GX@KcoO)L<2BS=2m@nt6Nt&^oybI30awQx|YqtFa`3@aD(zAk<6x=CB4P$ zMZ}~RbsbNhZn{lUdWM+OYqYO+Z~}Kqo(>mhFc1faku-_$vn&%5kU@wfFFB%uqNC-8 zm}P(ZeOUcg1LX;b-V(^GC@*JVVF`RZK6dK3(u_t&Q-oCh2Gcykk)6nW#(^W*?y@}%F#Fd6AUBKZi%e|FlEi^IGY+~gQ&?x}`eaq3$Hpim6 zPOUE?AAYs{JbELV%dMWu027_?7aPkOqD_BoL$K)#5eOB^LkvPf%!d|!fj0fJ^o;Mq zk+?oxaOY-8Qlcv^;^=JDcwu2w97U9I0yZn-6b_x*5j&A*w+%{9`dOWqH1_(f+Lfg9c=c-SmxE@Aamu`IYCql zx&I?bIX9M6`dT3|?;P*Wk=m_@_GJsVq^|eQu*gLoE}8W#L12OpeZ(fIIAg!Tl%y%m zJrYeQ$Z&X5>(D4R^XX^$+v{r_oY(T+8tzI!Tv}3ET2fY?b0RdP2~G%pi{nrvp^KYq@+tm836LZK9i-Ubh`8kFoEbM+u{*5393+n?WT$d@nP%*)sj`Sn z0A(}fK$qs~NV$9`Ad~q!E>1<~(`sT5WE3^ps4Qr7?n%o@ zP0i>@?)6FMk3@B)XJtP+ey69;!+sLcRo?v7V6#K3Vq0l>k_!Y-*P+CLs8kvEb(c-_ zFCs6%BlRp~tJw**J)zl^Wp)IJL#Q*Q3Vz+at@@Wc_a#t0HRxu%4w{yliuOsxoFS&6 zSP-9!l8Q2}cr*u76xZ*(m$aRZo8})+I*Ti+zx!GIu@$)WfrRXK{K76eyy0iAPx&-&_#xA7)2vRFuZ`gxog9 zU|^7PaPWDZ9PRV+RB?)4EPnF1HZ7q?vU;np$_;2nPaSPrM>fm<_{AcED{XCJd} zU4mE%DOWGAz3k-vz&dv|@E5-h!15$DHAQ=EX2MAY3q+WbGGi1RUe%I~q(O|_@cVCH^=Z2xxx2AK?2S+E=PE2@2DH8XcfTqll;5{qS zFa6h=|G~P@HBk%N+I$bDE08xO`J3n3zI}6jTlPt{|21X&DWIYxib{S_o&w$^5sHn% zBLDjKS=iU}iC)Tk8|~TGNy2j0umnj4)gdThytCFGJ~olPOLD$rzVF1Q&3AQe)oqot z>^Uxzlhf+%I5XFKiD%y7qjry}NWMQ(fxngvG)X(#gyP1RQ}50>?`fCnFn2l^aF(`%(Xu z{iEpjF`8Anq{VGYc|vXv^EN76T!z4y%#HrR;ug*JK+UsS33C?Vwb?s0pxi_cl0ejwy^&horwTT$P5t!S}527if_i z8!3@Xd!wV1J_6iELM{(CgM5Ir#En>emxG%S6*X@>Eo4Mo#zq1fEb;t~$dytQRN<9% ze6UyUy*B}f;5vN1UDp@@To zpVkuLCGyEN=@iv^tY?QCq|Twgl^92xMy zJQ~i^7BNhdO%yT`a5lJec(3$69bu#L{w!Ik^;9@h(IY;W|GqZh_AoX%E-tPQ0}JN+ z{3&h6QuH%*4sR_0_6zu(3RY~T{#0;!+;6_z9YsrL=KJ$eT*!cKF0U`woy07b-AKz( z3|Y-vgC2QAZtvhgwoG;H*Eq6-lHmOYYJC1U@#_X<8THg(sjG0+2$NhRE>ksqYfrC%W4@=(L&i-}`;e*ts4A515)a&vt?Jvl0eLP^StO#`bY zXJ#~14BQSiK9Q;j=1>6R`*rR3sWnB`m=y$f#{Cmj8p-62Qu{|7Bp4@%I<%^m6cqjC z<8Bx1*f|&ZjgE!VFN`n+u`VtwG@~mn+>JM>0;WSKjD7JHcl}y(u=~}%?yHfwO&J#_1{%35gFn(q{ve>OAkvTYPJCybn zQTQcpXmUJZfCzh8mIyNL`(?&ZGyedZbv%mOAg_amWmnXVxD3~#D>yGQuLBTnkGQ6Y z4OIO?aAYeTh4SMj1hM$4!eqNl%C=U`eoO_SUl!nkj;25ECNn5Q%4R~qeI#bX zbw0+ETN3ZRy-EllJ=b655D}A;PwszscF?itizR(M_L;(ZcU@7o<%wj2@{Z)kKHFC- zfOZ0&s0-t`xSGFf@}w_hgD*S-QM;RDOR&U=Zs~FEbWW^RaDNSY&m~J^vhPc;6xMxl ziSRn`H%^>##`ZmtFbd~I^pg}i#z=xf0Q-&|IE=5hbsL5AOM(D(526L)dNewI4;%mzcP(gwnSAyVMOdn9A~UN zwy5$_5~A7Zy3i%hq3ob?W6OQxN>z+^D9^;LFb*M{asdIjfr%- z9FFO_Vyg`r&FhU8awx~;%oh=vP-azodVB|mmmH0Pre@qlW%PwNW~@Mfowqw~8|QGL|GKWaae>C!KGeDS}1tqnMmY zK>Ks4nsn{2ag_4t!1OP?B)EXSY>P`P4Xreb_pAoxfo9z(6f2AucCQuN0~bPcW>TZ% z!3v+S0$0xzt(0^bCj@wC;I>Dr5R zG6aGI9VA%r5P~}l?gS4`aQ6VgHMkSp-Q69Af9ITg?tS?80}p(&=j*QSsjjYCwboeq zH3BAA;J_mxnBv9Q-#q~N+I zlN9XBAw6R>M94TO0M6At*&rZ$K?t9Uu|mv{DaQtRRAPl8Oo5*c^%jpo!PZjz#ctSL zMoVpFb2?#28(}9vD9okEnj*R%-S6+UHdv48HG9?1Ty8v1B6>5R|3hE+Fq612?>h%3 z#OTxO4=k?-f3rrPW=3laHGg1~$6-)Lhz3&v4ZjN>kk99Vm6{TK(NM5XV96u^e_iscud+&|=IAFXb=lJT- zRfz@t!L=hXU%Y6$}}lfN$HPjawmnUe}FV%o+hk0F>){6goG|u1mJ$lOp-I z0u(A2lIO4apJw`%t+u#`dMj&f`o@wX<(D7&^aaXmbN#t|e(APGM3CkTzkcmQ_2y;R z&;0D7yXcG{T``Y+wAHSJDS{TdpG8|%q4)s6i>yAA_`HBP3&+zbRE~9$vN1`W3Y`&X0;_=Cj`7P` z)-NOC(6KYW{^kP60-edYTjVkQj%LUpszm0eMZ`=8ina*FQPg1lzLcA`oNjxd!Bh|t zEXUyjrF`1-vzyKZG%z`g%1|kp{20jfM{fZlGKy# zWuLqis$aTiupTc%;nQNH7Kzm@+-mshI~IGS&3NElUlii}_Fp^}WsV~-{8 z^&2ZXyFxE%s!Y9XcaSCH*e#Zt5(%AWze>3-1R=3o3@?~X3+d@nC|)|G^LsknPg$;! zbs}K9$o`P&gzs%?tR;Kwm(~h!9dzNz5e{kp`lP3;U}$nLF*$f81c$t?6!sJ)oi2HzG?@xWmWjFtXi0e{G`ewb2iG26hZeYRGT~(!f_s`!bKQ0ZOB!%cpk>>v8YJH z7wy<4yFO<&L>1QmHx#)}^r7#5mvG^w$7$><{$2i_v;rLLtCh=?IX$0@#haF1by1Pc zFq}J<1E6Ro%oO@3e`+kXQ|89y5?EY!yQ-hA=JqvWK0rUD3$<9^ zo0+q=M!$R2;w?;|@4F<&eOCEHv%>Il9V5?rx#9NgQob%|+ViQgX1L_r#P@v~{r;s& zi6h}3eo@WcG(8F2mdjo15$mwV ztzPlo?VVM8e4p3AupnA?`_Y0og}mK#ETvAfXvTgQEY(6LeKA?C%6R8gv-xl*DU^Gq zU+VmxRil%d>2GjA&UJDA^(Z9b9cwxOs%<^%YXFdmNqh9R@*9CRy1eZ?;zi83yYeFm z(jSJ>8jbHRhsd(r5X%E~evx=!=Dnl-BmMv_r#sLYBlNIiFLTdh_q+qPLw(FT-fN(u^G?&_=mYF&h` zfmx&*nksqSN@u-1t93sueycMci)9iQ#*P{&q9%Z(xRDqITA@O@0y$M3cHWZEsj!~k zgSRZvGYs!BNH(-P(1ULQW>%0GgryW=abs!mKS9q-w`q2JjnmgD1 zQ^Q|3dQOhx@p32{Hd+3QGU3Uz!0a}fIZT$Lt!v%$dgx^^g$T@Lbm3A_$|%2^YB9D~@M|y;m1GM|p;< zrY;xe{k=u20oRSkE%WmLGk#a!Rz9iG5s6Ue*^=5UG69=;}0 zw%C7T2AF&7VENORXX0uED-wG<*p0p6GT8nIzMQv=inMc_@Ab!1`Q|_9f7TRo{vExT z?&wUns(~!7q^`b&6G;Nyc5kr0E(6F2Vl~{W7*r_&*@H9YM&}yLuj477CfU;5cNvab zP6G#!Q+3IA9qw9`d8FsGISU*hXIDuG`0hTmVrP>-UrZNy}iGWua-HGt$y_dat>Ay}`by{8&OYdvD4>6s>|KhsgsUj)Z z0P0e-rf%hW+^>g0-srcS^+wa^%EU7C(@LTH3MdjpMMMnOir(DB zrNfOc_u^7&4|b0B1l(@^BgQiH z-_-v-><4u}2frqD?&ZTq4iRyWUT6(LFfbLRP()#E8fhA}JNzCga>cC_io@NT!_WJ! z8Db$fpQn&6Tw!-wa(kXiJvr|2@PH|88_4eaX683QE8}>7wa6w^y|w^woRFqmA1&a} z3MR1XHGPt*MxMPJ&;O7nL=bEg?04LY#084?rZSJhtunqoUgU8-#KTA^sk=C>@o20; z=}VC#WYhcE#I(>+xZZ=P#69~wz|LYd28S7J))KLlp0|~5fbLI#_%l?qn$XC~+*`(= zkV|_v|50-GT;ni#%2}JZiDf@yK>Fj+>!7{q1KIHuW!#}TqOzvMd9O4srLrLKa&^8} z&8Vf>bIeTM@V};LA=YO=oFy{rr8!ZFFaY5uTl2Mk0S#a+m$xU4@gJQrN)W2IYv3!fP z7y0$uAlif%h?gKN1H*To3d7Tx_?=>(3fgS>>3)FG+axu=>r0*K4#(xKVUCaS(^{vZ`@S~Vtq+`_kpnSVXVmY{WK zLkFlmS*9U&n=~iFz!HPD0*pWQ;fz$`8J9biu=-C@4d){^X)g5 zK>@Sm_0bqv%bA}QG++Kb&y3zVybLNr|BX4;5l9;*(tC1g5)#N2NYuNdU!6M==y0t* z%Jtval@>BX)a+i+%=F9vpCA^XQzof-A^X=#iPqn(xl!BQ_Ak(5VBqW#JN3O< zTdD?}qoft_BmxT5Pm?3&umf~x(YT({WQbCIA{4L1gv*@ z@KG63YeIlUm;UXqg|-B@NniS+-U$U!{l->xii`I0be!*@MzbivRes`tq?byi(+BK! zO5K8^N8=kD_OFlrT52`BNa`(NNdS8giX9p4fXez-By+klhxski=5?IlD<4U!T3@J^ z@Mo7ATxw}xcNriX@cnwo3?2SVhwkr(d>UG`_p~nxcl+%tji5XxV%FUv)!D&-C;KPM zJiq?DWa~^7JxJXU0YekvUH9}0aiI+);v8A*8<0B*P$~{JLRUGvvnI{^K8J_%&lIdW zKW=#PtkPtNOHkFq9P8d0V#WU5zri-9K`tK*`vH`aSig4;Uef)xd{L?jsm+wV9uY1$ zi~ZO3)-(ayV?f(K`LDyKFGQ>RdZEBL-C~ zt;WOx4$_HQtT5_W;lU~m3ZVanOMwpYc2g=qM0G|5H=qFMK-v>AgL(&$l8WI-HUot| zkI|iE`Qy>01HDaWc29)ss=K17kflmxXYLm;yEFrzt0??;*LN6PAUB~-s(}40YAf6u1b7$3Sp*S;jM(|U5+3p)}P#Ur0X6BQKtXoO^iSV2KXa1SD1f~+h7rUKjWZt zQBvvI+g=09W68UI9!lYgS17QR-G?WF^5T~@h+VAiDtCc? zq?DVL&W88ZTIdfY&PQ$a4)y{RH~1c784EVM9~C(T&sR~ddG(d4Bk}Cbgl256Ae;oK zP04db+iC7rcfZVWbFPk;>MfSHH+sBI9{&!hwCIFQ_v_tx3TfKMYP->_X9OHpUz;xS zMO)dZTj7^%wN}e^HxMI7%f<5%k@KC4pi1$x-b_EiTC4Y_NU_j|lK{u6yhVL2iv9QW z*T zN_z74Kkn%0HqZ~eIys|+Vdj4e+}y)uUZP?E0JlEQ6-JVfPauC#{R#3mgXr~<&KlS zMSXKi5yyJ_v)I?^$8)fQ9UMYRt}pGeUG@~N=a)~DV`Ey%9{Nn#6?M{Yz`2k$6t*T! zI^EXCX)h;&Y<<`&iUn=eZq*24);)dLa`WN1DuZHJvd}q-D6FD>_pCZEusDH4P(VP9 z$JtQht?foe)NqPukL?>F6tkG5yn!KRO0UYQbJInS>*=`h{OysqY%i}8K|??I?Pw)U zzHJF1u6?#3QTf2YmB6N}b$j!jJoPZUTY(IEU4Upv#UlFdlHH0n?k! zz3*Fa;!loFCt_(q-q_xT%o@TihP#GX2-jBaAdd5gTYEDR0pUvau~yy^Ygx&+Yi%^! zGewh;EZPmd(`|d1o&)~QB!-F@;T+E2cgW;bz;tF;>5hC==mq7VhcM@(fxC-phoo;+ z=XU1c<^Z*Tu2do0Ca1xfQ9OE7BOvXbIz6wQT*maEd>u|HYIQJtm=4g2Hv^p#yJNY1 zs(Fkn8H?o8s=BF%e1c=M-X2~DGqzf`rH7V);8OvT@t}v%F_5B}H%s}LT+XZ0P-8N^ zByA?71AT=dopR7HyXbljc?<{~(4b!y6w;Ap18 z@*vR`;ILzxe%gprHT&S~umH&vAi~F%(iEy5S+=kpKAfpAI+&?wy}4xras%V4`5yx2 zuPBC1JO6sZWoX9_w3y;poL9wc%EvS$Z%~=bT)i28p!jTT)q-e0DX%SUeN77Xqx^!z_pLefKiDVBK+dxeFaa5W;-wg2Go=l z!L)J9sy9WJ3LxRc)L~oB82L5v)ir8{-yL5+m|Nz z=7APh1o=!d<_hDh@`5tP^1B}WCwj5B;zBu%DuointPJ$me zTzvI4?-$a@=l`mY97x(vm);vFYM-wn>GQK=kCdz5S=(g|x*HtYyVo3(ebz4CGx_UM z%`ok;A<*LD;%TAB4WQ>PMtIkUDXxeq1G1EypZ@Wuji<>VjbrNKqCx`e(QJKbZthTY zx%`5*EzRKcO39b`%l(}?1T=pcHM9sdv0efV42%3CTtr`_*J|xdpG7@4Hv#r&?1}NV z={JR_G`;<;kLk-bHF|_}ChW9c^@%B~xJU6&vHAr366An#WI)qmlD!GHjK+Yief9Cv zQ0hAfIx3Lt^_;k-8{)BmT`xKxZhr}@CoN#|dHni9m;d^9R(g9#$m>d{fhQH!FfCrB z>boyDbH-zkA|P-7y((p{Xy9U^j|%|?+I48iW_Y|ihF3S2N>1J;W6l6X6GgI2CE(y- zvbUalH+_Kn7rE)!o_6}>sMFxU@j~+A;)2~?3(6ue9`;Gi(S+WHczV8!E-b%5iPgd{ zmJ0DjR5Vi#5uX(hVJtG_e<=JD5D4_mPAR6Itz@!^nxU$KM-M)Lm!BmHIm*^qA0%d3 zDyFCLnw*WYiL&K{Y5M&etyvHq@7kR!_<{>c%Um}%F9D?{q|0bJY7<0rT}LhH`|9N< zcx~%bN4n_zsmahxh}YwH%1LMs0%LIWCAXX`h>MIylJ)UaUu34mzLxprGN6IrFM}55 z{aUhU;yzItztVC)9_Ur-@p3;5XN&x?Q$Kq*Oj`c9S}ocF?-5|#G?Ub&dFGuG~gf(H$B1%J#{niyOyK>WYEbO^Ekx8nni*O5>t=Srw zo)b)iZ3R?ZhnJTXxvjwO*F9g{hM`gMy8s+$cno15Zsq`yNMvy#3ak--$i$Re)$W)1 z?;}EjrVjxVMjB(1Q*5WEIrBohkgEu(o z++|idD}Ig|rGnrv;f+giCQ)h4gYYuGyWx?GPGS#;PK zoNR^f>gUgsJe{2~1o$zDuhS!GHC2;w_)ct7 zHD5jkElz=uZK1alkrtVPgH5;c)9qM5^(2oa%TS~9S7DDDk?U1<7OkewT`>XK7DXv3 z<%e_ikAu5z`SJmXm&gXzTL<+cB_9Qx)Sv>B)fU@n!uOHv_L`kb6nu;JK$=|hbA--K zUnxLQm(vY+Va7p=*COfc{71O;s*4~sdo!IViPwwYg`6%rk(eoj9d?vJxY#tPoWHxn zNn5WE#>?T0<^8ZP7AdemuqLAI~*8zYE*l%xvqEl@yh@eL9Q1>&Ad+fQQe?LQr9uiMTz674N^( z>(<$xHa)@T^3_GEw<^?l`R#UAmL5^6HeOdm@znpb%Hvj|sGOghBTurj7?M2C zOSAZ8tj1x6GOKGe-V7%oc3)+l>T^@kF@S${&T~9dqV3Hvp0X(wJ*s4^W}gvq+Vg;U zHM*BgH%7&jUSEGp-d}xfwyEBS-}i{N7^)kz<&=Pm*;@5VnKj;cIL&6K zA1}JDewiLj;?mc-ZWlamvTta4mO(H;SgKcT_*5pWkz6Ls?>>CaR=SPpCFD^?<;}hV zVw{ekI6tr-nK)j4Jnb{8I4e3*BN59CUlzl`ZiF`!7clvqLGLbt*qZS@yU=}s_IKNj zK+`Y|5J-iZslk;2x)TME8NVf@cwIknWSQ>Sa>?e+2#bnA#{&v*t(V=T@{qMNo|vYm z%aNzW={NlEm@qFHo(8<$zF+ePZKyE8Z@u7@`Fk`|9@(1!=99`6(2r7m)6yup0swGY!zcWj`u7tR{r2~OF)DumT|4qkw`I)Xa&HopLx$93aL+izXSh;q@J*fA=m@=}J|{xN ziWuQ_Q33kp3Wz^c2Nd;gIKHS82eG@xlsLsakJ=qVD5ehBMdh_JzZy}A+M&g zvgyEZ-9*JOt^|Pc0qnG9J_!W{rG;h8>&2L62>e$_J(z^$Lbp5qkcANqL!{RqlT$|0 zgs}K&cztf zyFK9Qdf~4XUN+K*N1dIW0lriaMq5S4|DmnwA5Pu2mwZ>EVyr z^OtGMxkZB|$;=I}L{cSOsx@Gj&^j4vYV_icYhk4_c4>p5g|g?(FH|NM$kKa1)-(Uy zaiZ-t1$@>~1rn)ko7on0PSe^TYX57K&FME8Hy?b`RMI< zs<}+X#~XnzPAd!F_ZJRA7ZsPcYrM9Z>1C71e^YWT1AqOM(&p_$uDq-3Ei>B_Q7GRs z2^f*Htp?0qb$nmOQ0TM2`yreJDh(--zg%7t*>ddc;P7?sSD`U7 zZDkGOLGVC7GrdAGawt?iGL3J0^!|vn+G2!|wf9G{=y?ajt1{R_-`xN?fc$DXM%su* zq=rZ!qj46d^?j!gC+0hXxnKJrp-h$Oy+Q@bIag~Hp_OMk>^V1y5D9N2Wcd@B=)Lz8 z0yZc01Osj{yBHdR037_S>)PXS(|LqIf;@dIzr#VrBy-H7-DZDvkm*p8knKWVEpPsB zCUJpTF7vS*4xsHNX=WqHtVZ90%IP|sJiQND9Q(yOibV@Y2|&t|jZ3tl#$ST@xj8JH zV=HZn6y@)#4>~XYd?p?Yg=Q{{j-DR58!(6ev?5lryZR}C#@DEVr9k;qEBf@TK7%2y z!6|n)9CLnTDm~KnHd+$p3-g1N9HsUZ1j;)(IcHb6ZW@{29`y!>vTT1eAnZ;r`gGGy z$IR;a>v6!NQMbsbtCH?qBMi_xnJ=&Y^(&}nY$<|w&Abg2-?NQ_j#`Rr_tE`Y=}(p6 ztWj|hlXgmZbM2|$oqc!)^2*_&GuY;r;qDm&^T6E4WYMLYsU&+s@-CT0c(Xgb@YBr$ zUGX&qkh16-vvsVdsAL&ZNcX%*NOO2M`>kdF^HtEM_rj2-6i9{LrvsVJ-Sl%WzUXz3CDpIq+C)k(nwA_={H|!V~|C z6=XkrJtn|+b}1%~4OMxGCoqxU_H81F&;?z;9X&rM?RlC2E<>jlD~h?!7^spF30smx z$!EKlkXpjZLmj6;^2G*0H_bow4&9jdNTIJ`pynPsbGEOAaz4kePGqUr!^ zcu_$a@!DN?SZvyG1W@P5e6mV0;MxGogeV{^e84uA=^z$DwIdZIS^B5SY$R{q!ADN7 z!VGCE0VQ&$2PhjlKAZ5%M`o1Pz@?)$_z6-fh>AkF0?{+7Ayat&R80eab?YgKVRp}Z z+A7#Y;M;{yz&9fB;)8Y=~fZ!-hS^ zqJ3K18`C@3e{#2!q<1l|Gt3Ql=u+{Q766lP9LX^7{d|*M@Lamiy!G-vNGcGYF$DRI z-O$n<-_Z~ohJ_1eP1Q-TUrTxBMfGm0X~?Ej?(m`U*T2_L!o{Rl^^(~|C9L{~ivtNp zcv&aPO*X9Mvmq!iJacYX?Z}KAq5XxI-3l3R zZYMy{&Gu1y6lRhacM}u8=6Bu0AAA390w9jvnm3#A1CM$#j@9-Hp`F!}l8K0ge#i>R z`LMx2gHIO>(Kp#*?*!9=G+lr8VT|s+g)lmPBOTIQ^;{RXB^9Mclr8a-Igo_hQRZ%> z4WTlflG$XyrD=;~PmT_}0T>>LuCe&V&!bP@4YTgjd4i_Oc)jSaf`zZZnjXqPZEvSB|8y!x+6hx;?+ds5k?u*3U2o#Ut5SGe$Gh zaxYvW173!AA*)d~+WP0G!$0=Sfj_raCM9JL$7Iie!Uf0H)k0D+3hoOfEPw(Z@-eOW zU)utMkZs@!P)2d#{2r;a(7^hRThM67HoQGqLUON-^MaLSleusmG+&uRgcn_?XEXi`gY(jk=H#+(8~SZi}^|M1S^^+_o2 z=8y(c^pbJ_k9gqI&;QI2M$8*O@accwKlaDzLq@wSM;mkQTd5}))#x|~^J#2;LA*Ht z!|#rr-^T8qvepLX^zZu%!tdBo?KCQ}1OFS?ISbFj^#f$0ab6;OAuF%`1-r6!7gpxd zmppv9BodTw6Yj2>`=HTm4j%QmnwK1Hq&U%}1&16}?MRvI;c+O}5COU9zdH}e(od6Y zr8q&MlRk6Hui7zhCs)KIUU*X>`v{1E(DV6r7Q(ke9T!8Q(x8a;y&t#Q*Ci(E1) zg#Rt@-|+&2cCK12M^n+&ZQj~t;Io{IJ$PTdCzgpohdpXPB3x^;>dPNu%B8ETi-}42 z5_T(PZ_mZ-C-Yn46G4=JCK~Q@JRBe793^aDolZ&UtSS^MB1B3UM)l-(^>c{BXjx~bXd`1c;ix)*$JWNRd)%ccZ7%GY0viJ~&i zG-R?cw6Yxqr`Oe{gxHtSU4kN5SmhN17=go|8yt-N4Y3m>lUlB&PcBX-v-%p9Hmw05 z?$E7k@r_MXBN|riVUk*p4lCF4Eoe9)U$k5Lr%e9O-g)=u#MC#KDW>oUpxLtrx-82d zHa7n!HNb|aOr-(_`?BlsCINynF$IL&IsnmMFQA;Up{gkAd}t>JM67IEEP4%JPn%q; zc?c|7G-|)jPR-b0LLl7cd$9oNKIOZ~sl_^1kK?k`o(M7&Z_V3sy+_yaT}iU|qiIx5Zbh;LBp1`Ihj?{^SstyK7U0_DAlYt96gZko>G_KOaW>c_nQV=n#L_J zv>5-%%l_Te^@59@^fJn3@$Db@KYpf3lDC@qiO_bZ;i;oep|N2{r&N%RW0e<-e~C?r z8$1uc4dK=p`tg=IZ?`9&MYG=FciN)h(@Bt*^^uH7>(OSzUTB>u>M+C7(=DNO)9o+- z@fa!bFdq&>Sx(91IrS1xSlsCdv8CDCsT1 z?QiWsYkgJMFlxlaH1LWwvC4Vm3Qzha9}ka5jR}{lkYzuuFO4H1>kmPjmA2V)LXiiD z`}%JhuUHaEX_S3&+a)M&7cpekOb9Rt*#tayzNKo5S~Sh5dBG*RP2ybLbM%?{Ayi~hJQY#ar7MCPf?SaM+ zgYT3t;Rzku=?@J(7CpbPw`eJcM6BQd+6umm2N>B!vT1lN3ll} z#vTU$Ip4&_+aoW~5FGe3j zYG8M}x95bJCCE?(XR{x6{DpbgZYOnota#m8cVxav7(wcjIw!sp`3q-oB)n)vKCd~O zqnvgBg6EYNC{He6^NVXfd z@`|kK&z|I7p&_3t_;kxxBZv$KuH^Atu~*fZI;sUMOv z(O18NHs5Ccz|2dopR-_P@5jj$Os_b#QV5a#J{<*lcAiC;=i)-5**)Jw<^;jw5(0 ze7;p2+rhzsms@A%?$mu~`Xjk3`ck#K7bo4ot0DagL#Zf*w705WXB)d=D`T7SvsILj zfM6OrW*Ltze5F*BA1PxpJ? zWtQ~ThdzQhp3-Sn@gMoP_DQ<)HQjy$=NdxwSy^S5b9BLOi27CT*9W~hR;;>BE~EX) z;O}LkeOV8%r`o_F z1F3`+7eCymKkhtYi3&KulO+*1WPXQMWHUX%%|!1`I-@joV*xYpEgmi5@gJRaO=?^mYV;0jviGS<-q3c#0>tN30^l~Qtj2THRVKriD=j}_Ermwea0Z9%voBhzBn~9)&jqcHL zTjBU#HtUAx-ZJmwWYF}$5sZ07If!B_2x(Gc|Ly!l_W|;qm)|vZJOq2u$pQ&beOrWe z)#5OQoa4j;8j9Nc4MdYfqJ!BFaF6hMo0&Qx4ge2B{BXLeE;JskQiD|wwx4<};Es5P zi$@=(&1>ysaGP@c^*}yP98Ded{mgs9jo01}5DE;h|J#~b25B$3bnIA60?mtI??U|1 zUgvz0;a`95oo`T_uwVy5q%h2(5%}E$V-m$U zFm;DQh>ZR0cmVmquA&#>g}ITC0ch2>m&MkBg~Kd z?_1p)i<$CKV_F2Yy>WFz4OB#*tPNhuoK`$9y5@)#085q<7{2}6%=OQA3@#p|jr@~e z`u8h|Lje#~mS4Ucqoh`~GXd2rrm!d!N=t{16HP!(HyDH_7ofDPuz)S+gB_P>ugagL zREFJ$-NOr66~8BM*s)iFnvgu~nC?8k8I*fM{BKP4wF{0(VEtRTeFi29n%Ik9+whdw zg}xO@@|}J0i}3Y=mkvAHRwgaUJ7yG*w1=uZ|IN;)u|bNpID^j>0xI~j>@D#mON}(~ ziui#wK3Fa8YpEfqC1gq4AsN#uiZ zkYuT;A$pc5&H(&>P<4scNzsGIRMb>2^X-(ZK7?S?3eZvE%6{@UKxA5&4#(O*&cRze&Yep7Rbmt}mWLm;Z5Jm+y8Y*p$xPe*;T~VmN zk?I*jnyB;CMBg=+TL(AYbmVjTyNyJDsB$O`A*UEqkJvzm5vmMG*|GKppa>H|Y$th@ zZJ-$ZjJA`+Q^9%hg5!T82M$TT_HlzW_apajge)hS8Npy%xX6}S$aGsSqupx7+_l{H+i=o;0{?o`x)BKfhtzl(QX$dri_}3>c${0biH8z z?GY@8oDyvn>J_k z6qT8ga)Pm52P#g1O+SoEqc9S*dK{`Z4; zQDyxGb&Sta92KIb^~%EBYVNGh^qEE^aV@MbOaEc$(aYtrdMYA}CQ4$JSqNLmHVg!b zKwzXgL53*(GhSPabW)xP*?s*gM9MG?CtO0_5jDbJT5!gYe8SX{7+P;YOo=RO#Jcam)48S zcY{Rz3hn4#RiKS_MgNvujsGUP;QsDNT?-94&{(xlE`jygo*7%MZnN7Z<`TnD7PHdR$-Rt;-HJ&XCK-sMgc< z0Yu0NASoV*X8}qd$P*kadQC1sWwCk^`nTZdWF>0fKq1y$?DBBhLpc0HU0XM^^RSdf zTl79Bb2ZCb+&~Y6DlRbTu{%bmeEiTg5EMxP{NfdbF9)kM%5*F&Edi~swOqMZBic$D z+Ou;ofJ4cWkfba+lv5#zbCa+2b|_If8;B2D%&v#Cr6Z8UN*)qgZ)W4TP@KQczse{9 z3aZnsA6o$iR2l|0-v=oSQ#Snum%)Mx))WA=?Rq|SXvSLY<2w3K&m7M}3>6g6X)yur zLD|p$q{-!1?|Ic=9hsC_8Z-L z!^t9od*{P?*yfyM)fUWaap2c#q}ec^X)n-0wsAySsx2~?6dtZu>2NA13WHxRKbGY( z7bib)%XAbnq8ntk8QgF8P$a$enhHyDPp&P~tTFr@6EI(U^wq*byAF1HzOM&AZ+)yO zF78n683YOuwpCRYb2L?HdiODTKy5w!eb&dmb2TAMe;6G-feIQUz|9|8UA#?x16$4~ zoCjF&^<7NO4-1l4?0PD52d2u>t~UwwVPk*Pp{kedTyhrD8YU@LB_Au!P6M#4=PRoV zg_|8_V-oaq?&qiVT9YsGgl}x}svZ|3YfN4{HD?RHnfOvIWQ~CnP;DBF`p0o!51m0c zfZTPdD&#-Jv;8q?Ay`K9*e?JT(sF#i5l+gr719&Wbx+8q+dM(O|K5Z$nX~@(iz@XN zfO#rg9S~k?`=jQpVPR4HMIPITJ^*N5f9rAgOQj^YaeK1$cGe#akM6q`r}b8}A3#X5 zM6Psj5m%F}oE^`i<*?fR%v){LwfS=G);@K?cHQ(^^x@DR?}&2V?G7CkQooiMa4RUk z*TyF>#^>hSFnSGDi)Z=B?|hv`4&2glU8ec0;f?z6FLSZ)86qgD{<#RcoMdF*gmlO6 zHOxt2Y)z%VjYE#bRzF28PRB*2tvCaC2wss3HkoZtjkviuXhBW7S(KbA&ME`G5t0l4 zat8h#gU_m8AePl+x8(cG-O0=0q1T6^Xuhd_eN@AX+oV5-Ie(t)!qnZqqn?Yn94L_b z8sJ1x?c=fU;R%hwhIIdP>emf8XucavX-qQM>D-5zPZ#NMN)_d<%d4mKX2HjpqQSGS zr^R~+B*K{9cMz4DjuDkMiT0D0dPscku5g*&e%W&I@_ysgaqI46ZSnn-gLQGK zCiG)KR#rxqFK2He&D$tr`Y2WT?Sn^mTWV?=hE68%`x{U7GH!rW@a;%Q7@>&ZRn>JI zpV!k=Zp(hQxQCh92+cmF$W>R5BW7GRw67-ONG8VSbv!5L3W@Y9a1S6fLSBjqA4gx{0*d_;e@hL_zg236tivc zzC$b*ktU(gkz?1bdb2bxWLBBbsR^}j!FA6940CZm{#hJvy%NOAQ|UVshtQU7GWiBJ z^cNKwyK6IK*b0&og48wib#=8yN0*+l{hA2TR{>&XZRq`A+IG6TsUjl*h=~?Zfxvn} zu=Ig|Pz(`&d*oF;U-m_9%^HI$$fsCRYK@`G?6^x3^3QYL`2$%zH0323=0c+1zBsHoSJHFxp`onwRlKA%Mzv&Exa^nC=d&1`~c zg$VO{+}zyInVmw;>A4MV(;^}=E4Gfmeu3H8N^0yUeHFbG ztgiN+RIkd{eI07xD)sPXEUa;xtjye>=B&pKF{J39Ao8gv!Uoy2WzySB%lLw$^0g@x zXM2;l%*J*rPIc5;ZAK?E*;c>UBp?eFCfgMtDEUbBp?iIvtSr_jH?~ubjXJFuY!AT1 z;vym@9@y#t$RewCg(#0(&>)?T#>T%6_y;9ygKm0zWq1?x7Jja5ti+W}_=_Q; z@|<+WUV}h`Vdh{yNgb(9Yy>4X807jeB~y--2K+zkx~OC98{Acafs-I1fdX=rf3pS6 zf+apaOe7s2E-4$Wr+dL5o%W^)DCHte@X>Jc#rRqs`sGD;w9Q5$GDO1PE&}(r8fMyQ zFuSh{ZtiU5?qA1~H|5|LbB(t*4KMv9)+G|Rl>CxE7oG-D6%>?t?qmjmfBW8{@QR+! z6y-Xiz%RCc|7H&CJjF9}I{Fe}w=}zy8In;{d9|^7&dRR9{RU7ro^$-uxm6m@94~S+ z#caK;yO=fgZTo4 z%DP*)Qa=YrxZzJnnbZ@p(x1MI z36W}a&%uTK{BEWVu!?4_?mr4Ati-X}?eva5bHqBh?^Em;ycwlQzq)Y3$1p44$B(8C zTZyK9>2)SWkM6|FBdn{jU)#r6v4hd?%Nf7aZuO)G9+XW7%F-MBF zoD#O)JQHeK7{3qFc(T@ZSELnRSq))}oYY_-NolQJ-`#_`!E7?zjg8;@JUI!XECBMl zw-ZNufj$EXE6OYJVRk{H58$8iEG&wmTH2z2_-r4>JA&FTF1HSWQ(6am)1Gi7RoH&( z>*wh9Ga3J~U}wu<@64!_bBn8HQ9s$f z=|=zN6kbc4D5g63bm7dVFW`FdZA7H+i=&~r#K`KJ6J+z z0Y4ym@q08aHIB?GbZ^TS)rZCa7mL|O4$LHkF=QU_9t1%Qo{tHfd)iLdyL+Tu7Fl$o zi$!k(QxH*nEr3%#U3{-yy6f1Tf;Xx0cN(BUz8HY`(CkzYPGe1;y`tLf8lw&KhB0p* z&xBzHVu)9ovxT)UB=cC9l2v7ATYfdJtErhY4rGzzQmGvs9o>88$mc3DszXdkTvl4z zd^^JXcWaFYbx&Te=*7ncek8~0 zwShNfOnXJ_RoJITL_Q5k1p+$UVe5z{G(^5PWS}Vjx3Vd=6YCY2Wr3A12$WF}>wd51 zy$pVLT1@+Lhq38|={qPOq>Ia^0g)9IeMI(?Oqp(N8GS%SbaA2D=5StVz0SRbh<*|?wPN)ZFIyzUkp54Kt9bTz$tV2!fXsG{vlDDp`gKnqLM={$b&Zu#v zznDH!cupPiI)`hS66HrTx8G>_ydLzHQ2|Y?H%p)=wu6W!bn^0D@Vjl1hMrjkW!~e} zc3}TqBpCO05_mjGvqHd+mOPQ;|B?08VNrEoxI;G#At5z{bhmVuASIxZf(X*xjld9s zbO}h8fPi$0gtUNkHw>LabC06m?>_g=!^0oLFlXceyMe1ZqO^&)#o_xE$N%sGi8tuN_D@bjTZ(%>C~ z>TLcb&e4lE);c_gJF*mE%ry75UA8K9WtXe!kred&D-VP)&7^nY+U=`(H};sAZJqW; z$nsCQYg}mejnx}51zDAFMm6!OCVIvbEpyo(|4f~wW3pX?ozgnumhaFeU< z=M=k1EO~Qbp+jjGt=c20+sXu(Q6ik1vBZQo?R&`rWAcgApMg5>*MCoe2;3a9^$?*# zK=wJ+h2hC`nnXMbwnLRJA-9}|m>!lfP?3ZPZl|4~#}L0@X20HZ%3G{a@4xG4%iqmv zPUwOILRaZTVHgz%A<`|t1epXsM*=|=znC2Z)ONuJg=CBLtEA-1f!4{LdJra42gUGv z`MPxMfG|uI@Ke9;&6(7r+K5GvK8oO2Oz8bLuZPZ1A#Th2UuY}?e8|Kf%?^KhB&+Twu-&tyoZf8 z*&MKkq#d$Ytq~GVk9F_2{N;B4+X(%|PzmoEhnex*WZ8?DnBl0J8Z~io@6${ExcGVy zm)iu01CyT5eJPlPLtn_P{tnX^6Q86_rQg4f2Yq>m0Ag5POy04Kt~4uN<4)1Jf4(Bd zmRHJ$c&_GKSzcUxsxc;EXtE3v4uORCVYknQICX<}PZpv}Jm)Z006j??p;;9V zNZifu#N*ZelPl%uzSD}#(6Cd;a+!|19uRPs7)e_GY>1icZ%}ne+Z7gQu3K0N0 z2~!n5GZJ^-D~;|hb%W?U6hWbiw6T)@ZM1Tm{wRob5NtN_NBBy#xXFbIAZ(?$cR62@ zXza1T2+$9a3{hT-`A|#^41893T3HHP#2^Jp(MrUu=k#Ad0Xc~3gmsdOu$VRqDXL|4 zLe^FjCIV+b>7n4Eb@jguJt9|EwtV($BL=C7*KapAW@>h4&U;=Rn|250+S*ypx0pHk z7@RI9w7?g<%A1=R$&9`Fq8Mvz4WAjY#`qk}UuF6^TsJPv-Av|pQy1z%pqE}=xPB)Z zdR7XewTX%A>#Ex%6BBLiZW8C4KQ@f4cIyz&EVo zxeAg6F5GP`GIkFs(pFeqoprIR{TiFV)!!018O#)E^!zTFruQ;cP^_V@1vOFR{5DIi zsQC_dE3}ycyPfN|X zOY>>Hf!|S(k-eQ7`+D4yo=JUM*9bpq=oS!b-8Vmi*$vgYtkXZQa6Fx?4uH>#$EpeN z@!Q>{QA^xy-(3%}=3EpW*b*TjQ%Qb#vo^A{6yP?0t3|75Gc$aCJ@3SKVDk-%*g0O&uKc_HS0}-d)!; zM(UG0DV4ow_O?EO0h-EH7Rwc-dVbecMop)^<$5!mYxK_>_hNQ7t0(SGU zGk*K8XgH$Cex_dWRbEhEd? zd)@CEHqpG&aJIC+st`Kwe%0!f+9WJ0z>K)O=8#Etxi~A4&lfmVyPr3L#&7R;{Qky1 zQSs5Al}KG!ZxV@=43wqS(2jXo?+g%6ys1+y_F;{`1i3N>%6u$b5HFr-CUN+DDJhwu zWfJ8T5a#0I60+xy>w8Q@g>z!mV!wACJU)m-?M+{l3N#ZL*U=HM@wi?+91?%*n45Dv zqPOq+b9kmTr-+*wC@WVe!*#9(>IAZG(%lx+fDVL-oEZZAa!pvBIWbvTfMACciQa-Q z9I3R>u)xQdOuB1I- zJ5_T!8Z9AuI!4mm+>C}t_3Yd9SjjTS<1Kf@Aek-w=fcM)VWz;9fIneBqw=&!c(S-R zfri>J&H1(31zVr|to}A-b@pSStK zB_k*pkX=?*Q`vBtV(#~K18r!PPlMWlS&UQQ?`jw8nXz`a&k;c zv(rI(*?1KV%bXa^#gg;pOI`5IWCWZsvn`8|^2GYHDh)*jbc7{tRpjqr=>M z@oY-LP2F!7KBMfUJ_x#f2p5Sd5}S6VU!l`*1Gd;iryW{}uAg`NoY$ zdzj4sxB$L%{?KF+&u<`b46}jbfC^?N|Go2(+P%&}D62n*LbV^|Kr$@Lya_=+&Z3lX zIV+kIDYo7@j*f=95AGZ-j)hK0G*slw93}RxHhOU)K*Pgu5eZ3+w^a73IMw#A_uRU> zzR%D1ZBxyBeRC08YvjXdgw0)tj2l2|j2m2nu^7DkiB`%X5qIJT3d=HB5L*BIb)BgD z?vesff;G9FWzJJSxc9^;L#WwlB8q7veV3BJ!euNlu3(5s?%#%|AU}1Omd;YhmR>Z*uI7 zjg3d?qRafeodn{CMLArJ-VgO}RFOoO`?_34f|j`~(8o_dPQ^DR3(mWLM_FdNQ@xKT z;QsYCF<5mS&2Jf7?9f137RXmeJDvH7l=a&a9BytH#T1vrOF1s55oawYaZG8CJy+v0 z=5gI;r*%-u_Aq&e6$uo?#S)KS_nqYzSXqUjmR-NSvm%*pkWrOZnUx-HLN-JXq^Agw z95B!Zg<>^sd0-xmxeOSjT-D0)Y|peN9hzAII1ZhJMyLDq&=+V z+qbUJzr4PPFdx)MA22|Q5UhbFRboij=2H$}ZIjEF8Ht>alNStD%f+fY1h2`_Vdz7k znu#|j2U%P`%^p{0r6MHn>aDkd%IsG0xagxy?~|o$YO&+?Cw}B>l?vh)Uvo;}adg4F z2{g3*SvSaMH zN*x-J%gS0S3H2C!+Cb-vyW=v#Oy9#P#uuf7PkQM=Y)c-&C7pSavk(qgY~-iq#G}tb zj;W4N2&O3Z9}gA)Wjem0IK0z*g?Nw2^rB*J(c9C52;YGiqB9Xb#3_RU6cD5x++v!D z$_mj5sSxbTxe&lz{3_TO?wVt_OZ$F~r_*gSc9iOh#x(xw3`01?H1_n%g1z{m#_er% z73c)-H1U;|jN6ZwgYhY7F{_(i7A{9Y7$1N+qf$POJPeWg&}vAuKK}Uy?}`>BrHv`w z_v-2~2Emyag*kZzg%Xlqa}5{-bHppZ->(+Dn*z)qEo}pN{|g4iCM*VyObxboSs?7? zwrn&KMnvnGrV5>7qYUA=B$gn{BJsSEu!M)59du9y7KglOWMmkZgG-&oDrOs_JeLdz z!5HRG->(sg8$Vq~MBRNCA;H~3h6oBoSBJi+$SQ6~f%psYg)4xCAP?Ip%P@(~X55*m zHi8wIddSv8AoJo{6go?&x=5c6btnmmaOD-Hm+_64x!T66X^(!OAcm53e;lVsT8d7? z_Z#e$cyPbl`KSi~xLcYx>aD6CVOU3onAf=I*K1|nt2Dc&jKNh(DGx6LP@u7iof30s z$G}{(`tWy5R`1NOBb@ONW<0byVKBTgoky%nQzWVKklz=(O?^_3FjQ0p0Uq`_QMv*n zjj0HeRNa@Rx^kO%qkK&e(6nfDwI~Wwy7+cJW`;IKc_{dmcb4-{IE_E-^OhqBGWxLk zAt+e>)orUUT%;|~UCBtLmO~ELJ7l8EJef$@Bc6lC3_r9Ear>9Cf_H~@zZg>qmGF%U zk>}@lgXtC9y5R2Qrf>AV^^GN#v-vvsDN}a7SAJOVwYZ0)Dy_3bgl_DeqN(mjd$en0 zm?3XpP~~lg^>mbS6TU|6gx5HKI%nWUd-unQu<&i%CP1F0bI}rXNk+pxt*chvdS}Nw z&8874&6o9yz@;Y6?#(K5xn`zkE|HHtd|CP>LV7IX${w%0YcJ#DncZ3U^|e~jm5+bX zkv%l0S;{w_uLj#Mn+|N6+$ppU!f+S}vCb>)cFg;yq2*jFE(P7VZ~DTRzLqdkOAMad zv*sJeKRjtifQIFWHau%mNUNJxapK7P+%FV!1P*EJV;e3Z!%BbZQTh~?pr^B8@)EVf zCww1tk4}hL4vu?iRZ=N$5oqB)oomQm7MA&v<2>Cz^V-&ZD~)^RTq{l-bdJqTB}jEl zuGaERuJzC`zty#?Huc_%+RSh(;E~EVBU33-;^t2St%DkU_!I|iz>(4WR6O9{q$U@I6C=&(}K&vjSXy~Y`?z_cDRtCH}ci0-F% z8^ynH%O=W+ScPwXhW5({UPsHW>jV6NC-h+kjhWVOrx_AkMG`-E8iON3`dJ;EKVJ*} zoeGkz1333*ses{G<2(dMR$mU0&;h-JNCN=YV`wqG*M*=^^iPC14vf9=4G|G9)hk@* za-KYdv4~>llc1##MXF1VW#JOa9vl^5ZX4=4pl~%0Dtf1_m3qw06)?D2tb+eyI@u%V zd$xiv>fh}VG3Rf9xHL%8j~Y*Sld9Hvh%P`L0dn>v zb%it_M38O05Tlt1sVu@#fC8cu4n+W+{R0QWJ_dTC4y|;)k~SGo)Vna*!y3|B&;_!s z-ki=Qt4i;n1Hv+NtV7FCQSR@PZ~-HJ6G#fe?|IEr-f{`+<7~Y?8uGh@vEHRWEqKz` zrxZp*wI5jFE&Q8sa{Q_E_(+#gP4w=1RPAoDzU9fy4U%rqYez39t*5xb@4ba|yqT)-GwN;^DH$jUJ)2!i2| z3*>&d+KXaz<+r;y|B(e$AGqCo-`arRe10Wm6RcpRD=s9=b280Zemh}hXE<;k?ROU8 zVwG5s9SI3^1Sgqu`(rw{MUZ&60n|a8;gnVDA-g~vZ03LC1KoaBjFaYgg zhR1peGh-rS2y$l=I3SGFKP%?MV)`L5d)a4Yo0+r8ZUx+Q;X`dTwV}p+%6Le{Sym`= zjJvD*rn1#`@Nub4dDBI~{Oz}N)tR`5iIznT5N(4ho0*!pZgCOM)ttezG9haK{;e!qLXPm^pz z!&WHQ28?>CBeZq3ThEsxWMn3ATdsV!iZnk+%PtdOIdBi3L+fQeMg8apvlSXW)6%}l zF^6B<4@F}y2TFd;&qt+Q4#p6or&dt!ehyS;0oDY&yQI7&^{IJ5=AyWU@d*7MC#!#h0@gMw zxg|?OXqj9r6nnq<_^Fe-Y1Gr6hPxY*HKPs$5?K~HF_k2oKjsX`3 z@x3W4ZgBT8v_N<`A*3GS_P{M#&|-I?wbfZ&^!Axhq#yhJ(c`hhi4%B^Z4}Y%0zA{j zhPP9F_DwubkaOu-&K97i2r4pj6l*oT=<5Ji11}UuQ}LUPWcu8p-@nmAMgFlE?_!u0y`d) zgvpw?yPvlOAOR}4>;t;^vQpGhb-2Z&BM3?w>|3$QQ_%nej07< zYDTr{l*W=WgGO7gmFg_3i3wlx6<{6iIQIm@U~cv70W$I{ECy;D(ZS2D&iJd zP(_)fK;?5XtlgKid$&J9TdT;I84Z*@TIw8|0%1k{le?$#3RV7Ldkw<&lvX|K=ufoDD!hRRj`X63aQsO zH)ou*IORB7C%-cI^x=mp4XW<#w|kFr52Kb1UGs9#+ugm+oAn!;N)|5J5O5;tUVoRk z$=Lu(hrg$%@3!6^0}ZUoWn-N;6%@p&hb9Mi_1tzQ@z+N;hN^vhiuvbt2paeWY>WWK zlOJOxXbYalzv^ogwGK9`)bPqZZ;JprL&LoA$pvHRBMQ5>zK*&!1B(f_g7CW~>Fq~x zxpj}`I6ewr+~m9Dm`(_D;04EG&k`JbrKeRY)br<(rj;h(!spfqMYSjKc0S*97hagH z%kwagSR2_GG7@!Km$tyQSaq014p0EXeTvvnT)*3#kUTG=X%e&1OrIkMbMuW-&!Mey z-HZv1&o4|-p)x2h3$?!Q+$VFhy#l=Nrkb6c-||Z#7eWry*`GP!I}nm^ThL<=I>151 z?xc`Ft9Qy6CnSFGk?QRryPLzIWku-k3=jPh7+S!Fm7GAS^!zD)SV_d{qb2mkuxFTe zM{W}0Zf7Tq3TkqSDGbIMy;U4Plu-QR&5?<-+7T3=C-V|8AWDj9z6+~c4b@Esq022( zJ;sOwQR&D&E?muRmy7%H?Ry&t65BTQ2yUy{vYU4@B!sj{ZbTkj&u=x}fBsBCVP^7& zy^YQ3fUdUo|IbA*>ooqow^NO5pS=3jL*(CqqEH~ZZFY3nbnkKzRhQ+`Ai-S8Y?yGp|2YVb2ObWT~4?tjK zL*mIWeOY?21Ki}NLf?M7Dwl_W*dv61J*r6m``~2G3m8DI=qE1xU(kuWyagsKzj$gk zCV?Cc**9l13(c6%-ySWHDt`zY%nxCL!}fjYYy;BTArFW{S)9iy!FhC}$0)sbWRi;q zmBz{!R(+xDs>%2tE`qoIChw1@cQFIKj~z8skPpZPQ-~ij;e8?ncgr$p^4f~Wf5Hvw z#G|kz=qOK+7D?8-zVW+AQx{k265sP3m1x8EyC36#TP-X4(EzJ_tA+tK;iu1f-CyZC zxtqMLuvM%tA8W-+Y;*XlV3yDRjK>iHLGnKJt86$;iQ1{&&2T%*{lOFX7Am9}?;{xe zIg-8SuYTDLRx#=JPb5#GJFQ$t9XY-EF_G-+N>jc%Ghpdy7T5C_GMbUT^kZP~!@Y%o z+Ooe1{2R{=`@9-_xjJ8$DnBfX+Zo9(^@eYhS&Ns)_RUA7i-*Be^XCsnkCopH^1Z^n zKhW`q)7|b~2YNTAb8^eG{{6x|GB-ZL*P6DSf~)>2F1+>RRjl%o9k#Fr6Q;BRj1#>< z*ATK#9pTcjwC^&1pEPfnEEA=*mfvbM_ z(Ho7Xfv|VDC`2ay3Efh;2ZU|>;xBH-bM0n}9X5#xgC-+03#Y_C{DoW8=9VGQ#s%Z8 z=PxavKJJ!X`58i#O%YegoBxrGRWui-}?7?`nkYCsQ zP&|~@f;W*8D4i;dm+rQXAVuV!h<9v|gj_zcEE*AA2``GlM1(bx@Ro# z8O>d`7ydtyANh_xw+57fAS{1Zuh!bvd$=7nx}DW$F4O01s>o?qQa}1T5ku#Bhzw(t z2r??^C&TBxmQ#mH8;uG9?xImL5h7Zjzvi=d=+8nJ94!5jYp(~g8b37V?3gyd^L*BZ zx(@l>@MIT$$8q8!e-2C91~c0iVES7QG5>e3Ce?NCjx(FDjmn!6dtUbrrBohvpoA5t zCixwWe1AgX?Xz4$7wW0)(s}eK>~%t8G875?0P8_aT?5pEmW3*sSE1wRCq$Y@7D$1( z451xnJe*ME3lsw3bT|e(_@nnI@dn5{gkys1P@oVfWN?=Yj)0HPi7bLi{wh2@Qhk=0 zjaR{TER!O_}O$5mbh+niUsTKs#q4TRX?Xw z+&uf89i}How*7Tw4VIpwh(RPcLOw)D@R@iG96Sc;s=8>c8>yo z)lj~dn7tZhZN0W(q47K3*;_H9y!t*B72aDsd+yHE)169V_6+t|mK{=KlSx*n zuFjfYYo0&D@&@LL?em{~GxMO{2)aIu_VZp02ry zwaL^7EL>JI2+-#O6iB0L&iax#krcHjtmhg|JAiDgh<9#P4jom2gI5)Lhrug*e&mEP z{`Yz{-rlP}iwKQQPfjRA-G4rMUgeo^nTy)sO#S68WfW%g>CD&rR55LKimn0kIfmomHn7 z??tv6>VF)#i&IYrD(^N-`dpe!U~%58TvipdUSE&S?@Zw{)nC=3JGsK1`+-fd2a4M|)99#Dd0UVeU8GLmC?+U<)-I#)zp&IMf4 zJq}u40ml31Hrr#V23d-QmDTU|IP?2;!i1q}wE2_nKJqEd?25yf;m_=bm71%vi?5m} z<$uvsmGM5IbbZojM@#6Y5hj{Bb$2Dfb_F{bT<`L`-0*vRyQQ%2Z7(aDYW6AYNe7Ne z|4ur5{;k!vV<85jqUVVxI`_DI^Bq*Nhz%XY{Od`u5Z2#QWWcnxP9#zo(3$y-4fN4L z@}@B#VG6Fca|w-2SJ@orYxhONUEM-sBWF=?D`J)o4^>>pU=M6ahR6t{z@PQ1y-#2X z)*Pm^2VrfpS=*%``|1NACkw=Sqm6Ffc6}?ERAQbsui4TEN_{QN-hQlRO^m`mU2}X) z!e0nwJu>|sl%n5YJKUZ%)cURj3!8S2wrJ+7DRl3R7r?BZpYk~SNpXMj4B*v@6Vk^h zV&mcchS}0NWF?g`$uU7~M3%%pM_VM@KEYKH{V`vXQdzqU|?cATt+0s4;ttDW%s-`%iLIKr1udX$H3~b29X)h z)`}5@m+eH

    8@8(S&g`PDgsiE z``G;SG6hC~c2RfQCg91a2-wt27E{NA*U2kcJ`a)1HwQEkVpsrJU^#wCSS7!x;Cf>> zbT+(gUG`JEacd4;%1di}!`81=x|WTrRmt=gGii-cH>0B@N$ULg^;`p_i(8ynSyxvh z(5H6Y<^R;LV-+;MQV?4(cUI&xul%t>ROCQ+JVLkm&N@K;RVsPs_k{o|kwfHO2X& z1A=Ef!!?v+LQXSx(4h278_03LmVMW$NUBP1Gcq)shuYZtS$i^M;h;JrktI`OUVB|x z3n(*EIXmO~#7OpT%yk174e4dBYFdZ?sflwpkU`as7v_F~>Zz zrJzF`v=ND`oba6v5-<~hw!hD^HgIiOn@19XNoS}_KTf7%VV#3G!Kj(p+4sG_L+;ua zq<8~7f;@t#!jg-E9#;sYzokp|34Ao|6Icp*yrlJeF2l0hfu8qu{?eTWmtSSoo@%tYR1Y0^*&7VhCZEa`z zSp>35d2Tc{;Z5b@lQRPR5`f0rpO%(}oUmSVLu2E`#TA;Q-g6WKbAi6gcx{QAHvsd3X1{g1;!Vq}O({l>MUC!hytc zxI8s4H%G-zXR8=cslVC0xZ*M*JlzCj>vvtsaK=zqlLWA$@9*oKXnzxnXtF(sXgNrO2}LQ-s7^hNv@{Lk9c^4fQV&;l;b zG76Gt-3R6y&*rsKgMSKxZaF&x)S#W?JFbJDm96!ghgZ%PH7q@2Uc$-t^|tuk?5`p+M&abhl)w9 zIo|D+39bQUG&sx~S*t*UjDeI^I(Xo98>3a1S=t!$kELdA_)bm!ZJ z5_Z@u=_9!#t_e0)S2C|N4bAN0jQc@a{#=Bgbp~289Wh&*@w`hKORgdbo&I6px4QvL zXR25;gs{xoj|^M&c!jpPx_@*Q+Q;i+(Z~|&1T){)UbY~eECTaY@WD1HNFrPLt0TSE zKiU50dMvK86njU-H@8(I?rOYiC*@1c&rh1rpo>19L@uW1eYB7=ScP4H*Cbz;wUH%7 znd!8fk&Z(;UCCYX?D%}BJX4>BBK_N2+2BBWV(Z_25Ckoo!7c3qu6D)Tm}7oX*4#iy9di;#ovGuq5T5@Z-y z(O`%X@AhpvD>nw{g?hYvyR~Ed+P#sXx4K$84_7iNQ z?h)*KQ!Fx&fg~r7Vi>>W%3^lU!f5_k);)pG_-z}o_cb9U;50=< zm`PN4@7FGih!AOvNB&0pl^0zyx=_n-*Gw(>ldPvK6BDrm8QTDgmveT9Be$RB44r5OPo$bqY3zmFg@jfI&c)Jx4eTn;Ox=!~F!jZOR zSxXGV7+%)>f3Pykg2ho*d`#%Gkh*A+3_=>&tUuZPo}46K=|7rh01nGSs}KUA`w0bn_miTWpZq z_Qu1ZD5YuIq35+qwVBOjircrcS{oI4yBe4Kx0)MM9c!uI8JNyN`?-zCh|lAD7Lx3}`d zacX+2-fu^^7d*nx$nNs@vO91_{T$MA4mwiOm5iA9g(cTw^k}ham^)l95=;Ji+jw4{ zx7W*lP`T9EnBCZW!CNSrw9jrVZnbOtG#<@bK54Kol-#00GTNTH!}f3fY};S*NgP3nFD$;piDr`=*t%dKilC0Bi#hb@^60MlUS$M zsUH*uF42_=3$ug^jp+`cX9FB0R0OJYg;#^Hy8(=Eg_s%d|B6eoAY1F~?&JoJ5TF*$ zZHedI&KE`{NXY*`VJbIn2k6hC_^Hl*4gRRW;7RkKvMSWma@uU5k_`$pSHj$c&H+rnX z)9cLUqoS9H2@1`{Fm;?0AsT;S%Sz$U1}C*`Z^v>|sfzxbq8pN59a4W`i?qF_pgmZ%go^4Z?Osd9nEf87}}EP zNEp~l+K4&IZbQ?Pc!?Qob+|XzBJu2pTZ|2ph|W_=d;xJ7W_Gk-U6h-icqdvxxcr}yeliutgkzvn;)bkq$dD8Zm_Fi? z9t*joQy9;i?jMy7tq~Uwqq~HB>4|t3jI~qz`)uLT=18pGH5i9D_&wd#)TM8*ojs#4 zXII{_pRm*SpY_5R1gj{(phoyPe2JC1L3Z)iXPMZwfBky=l z-oCp2e<3*$>}y~-Y_gXkHdBd~$WSl_s9Q{BLyJjtQaPfw;rUx_r+U&H;mI zBg!WGm7y<`#aST;JAy=Jh{ARD7Dds%ysSNYJ&6@`< z)q`g!7utl3S!UBZErLr9#hhKbJ}g2oaf@JfHaIvF-j4G;N7Zk6+)nw0@K!`t{~E&{ zV}&^ao!Ns%Nw@UNf4F}oJvQg6*LvZ9vg1>eKJ1TqTJTxlxE9B}iC58i=}FzT3)At2%3D_#z85c+b-mCn%daHXP1EF&ZI%w1a4^wS7N?Lvj# z{8Ybx^sLYXhP$Jz$d1gIJt!C@#ibBYN?CQfuFQU9b)8Wy`hUum`$Zm9g!lp#=bu7` zd+Lp63BqEI?lzagEYR_{_~&L4VPYZMf6!-a_w{MOGwtj`8OX8#LhrQ`$bx}^+t!;b z!h#{L^1675^QbDHg-mu_CW>bvBr0Bhw#Jtn;(r#-dLYC;iOCDQI_GbMzR+um`Q7ZpoEH~IyX6_b zkgeUB9Svy%+c7im*Au2(>OV963pqDtcbHi{i>jn}RhpIfpfJm2RW-{ZecP|ZR6T~& zg$`h2tohU$eqDaqzdCI2ASLwI6{Hiu*GVJv|8NTnmsvdr*jxZ-b=Iei%+b2g)&5(O zk{q%nF$QR&i5Vl+&jiyKN$SjY}R_$eBW8md>+f2-L(?* z;rjO+rZPpm|A}3FygqO7v&!`bT}TJ1uQa!}KUf7_S&clDu;^^HhJXD!3m}3-ZdSuG zSW zy9$h;Z*OnEBE(7>qBnmRyIkv@rlz0KifZR{?a&_L@+Q80ziO#!!$1Ca3&n)f#biq_DPy z94A*;Uqm+(Fy5Q+u%V`egkvn08uTS(9y9ix2_Zg+hTYkr_sx4Ovv+9v!EnafsAfnc z-t>nU-H0S6D5jrgKR={e1=GjWyQu%pi2)es(S=aWnLNRh2gFY+MLm;cF&N6sa$w&3 z+bR^Q2_n_eJ9KgP)~UC2rI2=~Apf~Nr(@k)oHGRXJU3sDJU2}*91aiY5LmM}TxQ+p z!r#N}?NViVjvkkeBMmedQ{dCW-&r&V%E=|k`gz-;Z1*-g8em{yP4Xp(?pAKrn}L8S zy6b<$qlV0X@pavfF0sh;$(1(bs>#xwy$LrxaLSOS<1?=tSJzj!ClC+F`2J%B_c|}hqV6H`(OM(J7XOy}g zmR2F2+ZE5(@fv7IUB~MSd|vGy7g9~f1A&*7U4s4>C-F@X45~PX*rz9gF8}KUHgbV> zQQVHTlEvpW3P*#9pTgV|UBDRs)$sy}gmGu`DV2y-Ty)gG*A6lR{D6v*L<>tn#q`a7 zUa4H^6rjW~^bdX<(AxO5nP$}a>(qAYaKb{9n3lKth$f!52ghkD;M%?;XqbQ^(0Ok< z`Y!eASoF%-PtMIt3(*Brt8=Wzpv^PjfhgNNLTxe@KpEuupA^7_`|(vRUlq|k&viZc z(Q;KF!lAE^aKCdZ%!SJhH&xUSecah7cnVoO(tqm12z(K~)Cq$?3j57go%;8cNf6WH zGL!3_f-{huY1`Jm@Ijsy5Uie^n`NvJw^(B$_4l*j@kHYy|{0{ z<4=}@9&_$wujrjY?hikN-8%7F6-n@V_l&2LE6a-1#Z_LzxsBqcP#n)`FD(`V}bg-#7AI}!F#}~Aw@5FJT z75kVgG-(~%ri=-4#zcUK%Szj4Rji$E+x~5}I z6AO!fJd}Pnj#HlpG)6TNNn8o^B*M$_gkyiKa#+G6A)D}LGjOwMGp{p9V1)lruNU-2 z&xL^%&mmZ8zi0*OfcC8Jg(I#LC~&!{vmTh@3p|>2}*r{tKLP7@<0Bmo9E<*`S4c1xKh6!2B9;)k0{B0G1}^ZWt=MWFrgR%e4%>)zte{q_kQ zE^Erut*>^eENw4KRw{TZ(^zdEdKdF7))qGDZvL#Dl-nmlQms_t=Pkaj7T0~ErVqqT ztA4xT+n6(?SrVl3XjJmAs~?d2G#oUvTAO>2(XVEOjw81mmYM}Js)XF?DDcxWGWb=} zL1O|<=N%Zp;6|?h*#K+S)331W@oXWo@F5WHSVK3L+XSe7|NPY@n#1+7+qfi#Bhl#Q zy&oGoDn|Hvj>uGp|BOM<rlZ6Sldqe}*nh+G0Qn*|ZCJ71_ zq=&_9*_d6#j+V0Zrq7M4Zu634%@UbfCX2B;dClD_f|*Iw=T@k4p!bsz!*Z(!3K5K; zT+`yLDop);eWUK>PBTRMvczYG43Gx>eHoE-aVF=uLRG6QaD?@pMz0J~jbBGmDenm?74O7jbc||=+&icNthQceYB^P)Cnm=Ed>Bf~; z6Zc`EMJDd`i&pPAE!NskdnrGiyfRQ;{Q1xwZQDWgMg`0Amg#KsS!UXjc#Z3L>rUlQ zv(+CXwc_j?l3h!c8X{3c4g5Otzu$+HR@yLneAm|Pk{m>_PgcMIcd{Z54_|4pFf|Ud zTZ^cT_V4#^ofdVh%D!qx56)D)^T7RFk<#UQg?r|8$~94U^bk5lGcA*P3W z8o-41{S`SjmcFY@#&08!+F2~n)=OgiU@^~fk37iFCEvg8=lpWL_*abslu#ySki>8vWy zh(!1~bD1|;Om_pNFV^IiFSsA06cIP1*_l<<T;LgJ9>5jrtuQ}htIp1o#?f7u9gAP~pW8aWQ zB}LFp$X0|mEm!OHFxyJN`KbOXa$s)^9uxVp>-ahxTFc~gn~>J*yYq3Hw{W6 zQ`+kxEP4_5tXEeTxZGIpJh!G#0r6yWYyL)?sfdT~6SpV?#P|l!8>M+7?b{7ahC!P* zlSQur9(!qSl{d1Qmb%t*J%k_u`x_Xd)<)A`*H$hEt|>+=rf6-aUax5C6JrHPYA22p zAm1`|p#HnUg_(Wpb>8icKg2v%6iB5ldIrs10WJA}fB<0P{CRIqog~%@*lXdL=T**k zHzDJqkIcbA`ISut9;dB5l^7IDX-yJAM#sudpymdGdqC26c0WzOZ9jHAIr?3x@6kc| zQG=DlDNH$Aey7Mnv*+bmf>&o9@B$>XAtV{k_i3VCyAk`UFa(lHA^f!5lxfgrW5jhSC<PJWylEqQO0 z+#lU8cHM&kN!Yvoc!NC&q%$GU_P~pqTV?++WPMhik12vtrM!a`>D2#O3D4V})9ly% zSdeSA^6ARjCx#rwgk`tPY3*!6`D|$R@aCO{Flfaht$b5ZHZdi>1;i3R=9rTGvTsjV z&vT<8i(LLu-FyO51x>m2t^KpRUh=`0x zemgXE*6!}^N+3%6%Vv(#NyskWz<*BS1X`r{B79>jTIyT@km8s2gscpoM0$)IQ}=5o zB67u7&kwOYUF{MH2 zihQy8G{k-qz(IPXHQanPu;DvbL8dC4p64d4E|OczgDID;gHi;WLgQMMCi9@t{;R;0f}?!y}Ac z7Ex&HXm0+gpk*_Lxg#^mjTRb<$4#XxoM zlLixmj7uhUtF%8sh2frW?BHQx0Sr-Jt4Q?P0D-ewT&76!;Fw>0xD>&Zdu_^ex*1R? zvc2Fvydr%U>Tf1yf4o8~@vWsM(G&3tL&yW<>FIb;FH87c*F#reHKrN|?pekpz%FO||8mHAzyiXH zkDr$Ov+j#m1=^-;`R9zdu2A(Ai_Gud>#{>q=IB>ge?yc^oXrYkgtF+E?;a>w(o+is z@|}$>Eb-lv*6XmHT6ydZ1wHQ8W0Z;)G1uzq<=I%d@8q$cVxn4hP(aSCm`T=x0FD+s zf4Xt3!J#0$&@=faO#W{ivBRjk>_U3ok@s*jm~3DW7qS{;g+zKTnoDF#P-QLZj4AfEtXhI2VA!)y83(Xt$z5V!o(<5fUN1fPj zr4`nG&*NU;mZ7XOuMeEM(~zq%AdCg8YhdEKwunBhrL3vXs*P))j%6-XZ5Qz%hcBdI!%%J?wb`;CGj+NW)V#Q^B?bnB&{SyRS zIIz#U=x6W>(m%{^h~D&|RbNJ894T5M(_+H1A|i)}%5aF~VDwgNqRJOjbHk4aB+>U) z>wkJ3Hbt$%5UE4x5Uy48Gd<^JIc|eK5>%9T!&CkcLkko-NBFriW*8U8B`x`zKmSa< zt!}kBo0tTIcWSZr?P@OQ`QWxjP3KY)pdBGB!SpChHacdiv@dHm-?0hCxjj`=f*~G- zwVKM1L;%|RE~uX?>Wt`$o#f@GaR6x~(t^dV>%{1Y^mW6=3(7QnoMo^CPIS5H%retz zXonwq=35rD?_8^^E!t;Kqa--DD+!PwzSws?B=;Je|5@x|t(@mK#W-Uv6DIEN+*sFW zCTQimRRE>C?_PQ7he30Pg3|73DM=LskBQd12tYXuQs@7g-ZcG0rL%-#TCc%&^G8+< z!qNeJ6}74IsrzL`n`8UwT^XS-KAWMAq)8!_=^Vse%}=gKSG;S?2skwUy#gd}Qy^tM z>Z9^~OtAd!2YooA4@t*os&j~!u;LFYHD_~S@`S1{M@l+GZo-J1t<^6hu~>3-#Moeb-ZWyg8%VP zY`o1iF3c5nz`qWQqo$jp>Taistigio5k(x56TjI@90N+M<6rRELIv18^2sH5wxwcg zQL@4ELjywL-=H~M3emA&{>+7=4Zc-0PE@^!{juEmWxzm%-VjWjO$^ABnWCSp;ufTH z$$+9#W*Kv#L{Je&w?3=9`xD1>_B^N>meaqf!P+jjv$OTyA7?WV_G8=ze=DOY;6JP= zA>XPJb5}GA%`TtHZ*AT55_;_SJ7rKRMagJ!01Eqc;@CzRI6O*a~Qt-lj^95)xA zC!CV<&gT?uUvt^(_-=Nqe==!$Kj`5+35QOy&#BGrGRLdRQRXIqV7&8oD{CM=@0|Oq z%YGP>W|H@`>u0Yc{W|l(#&C(F27^5Ro1-!R;OsuF<&Le>ND6DkS&RB2 zzpkar6+I`5vw~PY$9Zo^N99rbR~{nx7QU6+lWI0a5Xe8BLXKB-YEp&5GB>D(D+R^M zZsc#?)Tz;4Je&$u#;D64(zkA>wanxFR29yeYhR_rJ4?5}x3}+l+pfdSg%YxPeydH* z5er6~%=NzY|GyG|6Yc>LAOYAk)y-Xv(^PFChoUT@jzm9T>e!3k8*0;osjRT#Nee$FuS#LcJS?_dic2W^zNlJ#^E*d&hb9k9vUcnpQ!nAR} zB!T?c+30$ABeFp??R5v3fWPea5<}!{$6*c-vu}77`9O^J(o(sk|BrXzhM6H7z~X@w zpIBJ9#YTo!bHk)V?32${oJVlv&CmfbW75(H!DXfEA9hPK25&wQ21oI47rKqgzc$AI zHWY^+G4W4MWp#4)JE!tE)h;X+$dC_wo|OLjs+EThEm&^HIzWxd9CN)dyA`vrs3=2X z)zssXrTP>|WZewtgi){#qY%a;%LT?!*R#P0(kLxrAEKLed!*^NDA7w`Xc*Y=<@(lX zzWjhX_)!9M>)h~V;z!ypTG$zB zxsMQ-j_n*;$&3d+xjzr>{#LC&))uM{g%o!S#6#yy|E3Q7qUqy|Vm5QGPotxT?R%tf zuttn13%Dl*aOYMTacKHkM6ep7JSYPl*ziJMyFAZsda~yTV|?*Zi&3EVhSt7(6WAV+ zR}W!teF-znc-<<&MU|PF@8xXj4i2xp4?NAbf!SL)j<@a>p~r z)-rZ3Yy?6=Zh08)3bNo%jw@VqT@r}h{d9Z}utC;#T;sCl7Uw4XABHR&SWxgU0tC}j-0Am- zprfe_36`fvP|aY*%9GagS%VILSV+f{#u#2Qr>EyE;w~bk%4H0oE895OU7JQ@K%VnU zOX>8QC|?PF;D*6f*5E0_>tZqVE8G5dbq(>qD-F5~qZ1Mgy4GjaLR{6Fj>i1$pRr~t z2uY7F&uEV;T%NvhQLSE4VLT6VmUj}RwXo;#q=B;pQvgrlQtQyh4d z^0oeDIxbL-2#9C%S7wW|KDw_*eL-=swtnFbM2!Gy&8yg;y<7y}vdR^vdq~(|&Gb7O z#?`C)$fm%%sXP(y=aMF`_9IZ_&8A4upWLKykS<;UBs$sYd}`PAe0@~2;S(otgGmR^ zkuw~#k>L##Ey*SiEp-bcTp zYm`mduh+)B^etC&FYS2kTSdPE#f+a#_uW(&A>9&XI22nTs}m5O){XP_7`u=^dHWmB1 z{;wetNl?Kc-kEI|m!`M7_d7v3@n{Hk^yDB8tw37j?R+XzOad=B5S1wim#Q!Pgvm+8 zg<(5pY92c#hRPBygexTM&1;;3s7#+N*=vtZJP<54j5_8U2QWw&MLy>G5)efJiYeRq zf=A5W^icw48gP?yqZV&ecis17*~0#fh{|X$(FXe?X5ZSqC^_Iq@BNlUBOQ2C-qg0b9%j{S|EY!t8-KR?N!xny`o0X0MCdiklk z<$OEoD>NJsExbPGJF?+WIj^etips5=fg{y5*EsHF1FN2pJg zM@-~fu~BLpf&@Z9IlFy$zA1fP5!F86&uv1(gXk8|>#9^4{Jn`*d+=(r?VZM&vP{>_ zL08v{LT7ShQ^i#f8ks!z&$`i(p;GDXu{@oP-Esl-10gLo@~ zoK2))(Cmz*jK>4oI=l66uW{rxGsD5E{<%ClV%db>D<5U60SA z@O9o_>mB$}q=O))F11y-H9t~#(~wP|K1iW#CmK?^Cm1hzLn?S77IiIZ0v3- zWoKfN-{drU=l3XuEw*$%x6&cVl{se4>B!1wXy@n{_`fM*+yKcmEIM((^bB8>PSaET zCX)nK*gNV;g2{mwrD^>sMRcnJ*rI?^@#6_w;A*AVQs2g(d@QQ-)IThyFZK0TyX}}RQWY8dpcb1nI09>OD4O}^jMrSu? z!SzUetFX4M{+mOaOj?>`wEvSGmh;BrU~Qw0p`l)EMyOdX8O8s$rl`(h^Ybtavxj(F zcqnehd`vdaR`#1j=={x+GH}aAh^{J3w2QF zD@G!ZAHyi%22}J;wXNzv!GySqsBE1lLgRS7TXjt~V_``8^nv1guqK?OSnon!NMzK* zWJXy-rxCp^o+|%uHGi7#1qq4nvia+-l^+&-TE+b#eaUfX9@yH6o|ZJdG_MH!BH$w* zT3buL!tJ`hF{Z^Yt_P|*ZAQc=!K=Btv?iXyZA zSsT+r>XrWcBcGc#jDv;q&7g;YAbSXqij=b9igJgbymDLrw&26{*Y>qi_88&0rqa8d z*S;l)_z|$;LcKsX-1Jv+G62G5>f9j<`DuCfSpxk#3CmLA=08~m&>msCR+N|3D6*L2 z++wB<&ajTsmJBeR$`rcxKbsx%f}kh}yM^>I&%$@Z(juB9quczc=3@mkD74n*#1PQc&~f?e&Kaos#CMg?#}dS)G-USvEd zXnWlFF-EsyTSsWve~$T4E3m^+3?u_Y49__&>V59O7b`Xj<*W>9(hqV-BpSmzQ+!p4 zsGVbsPv@3A*cu<72fA544op+DDvkO|X98XJ@t(ggW@=pH_PCEwa8KM5$54x@GvLqp zSLl?YIxbn9e{woNQLEb)IP68?XzOrptblfTU1Rh-->c^p)WVcJ9NuWz!nJn)1Z*{K zJN*p?Q)RJ{o{V(0_xG46MQCyNlJzma3mncX@0unGplK4 z*_^tZ_HSfT_fxNyuS5VinlKp`sLk_Hc`1#GMLJYy05L(!s?HG>fen-G1lLXEZ#f|4 zF7W?@JmheHzpU(J+iEb&8i5k~m=_b35%)YeXMQ(Ls3M4}POkC9NnFG$Iqj|=hzt$$ zZGCqy984eyDo_{w8}=1CVFYEitPTl4>Rb#m?B|+)#BRopIxl%w?2kueu-OQQ<#Wf9 zReO{W#BVhVO#gKS=fm#h78bVtgBeDD7J~ZD^281)yz6cGsa6(N{1{nQbZX3-ts%zu z6}s;koM->K?k$?T!l^WWTV=LNNRLA0k_ z2K%(OQr>RIHh%e4P8VwDNKs>L?l~C{)6jo@SNAtxQ797RXXVop20)MdbF~Sav#(?E zUtP_c0bL@9?|yZ~Mr~_|g+<}t+>@JyA~@rzy#FZK=Sx z2*9P{H(RrKzj?G0^2F4-I?Sa}Vc>V~?BD>N9A(Umx6>j0vwtVsf0_9&_lZ7jh^AV) zE&~GV38Vt>t0!x{e1HqvOQ{5qct)?W_WmJgWBNQlNXX7XQM1%_WTzh^cED-N`o|ST zZ51;J#n&Rk^CwY=UUKK7(^4A*8O-5Fa8vEEy}?}ZsgH31P;r({CkT4%va-d5%R=aQE{`$7F1^njxnQC zlG4Cq(yU@{08l?cIr|kc!1O2}Y;e&yX9&Rjcyf$wRUhod3<7#HJ@x@{r;xDwc2!0a zxMth5bgB6I)whx>71M-xn`4&v0T$3W%m%)aK|xM47IwZgfA*!+xul0ic!od{u#8Hg zkjB8ryRL_|1ZCM&(S^o+J9>Wf(ERf9^8fMR4YLQ$p*T^=fZqo24E1Lo98oQ4hTb)t zLN$AAHA_3S!pi!c_NSLW-ha-7S(S>-Q;=%k(T)-)9g)Okh&8Ei`M)~Q59$VBJw-W4 z;Qis8QBVgUF^d=q5NZLxKOQzga)Eka!W4jb@bvVA1{4`&Xp^~`L(9{>(RFW4pm98) zwHdm3kiaOfL1hBt&^%U&n7J7~#r^`wQL#oeCzxddTqs}^a9A|4FpBWIGUn#sV@lHy z*#bfi5>&7s1N7_sEk?HgoK8ye3oiDwZC=OA9e&3b+gkCPW5`&5QtSVV2xYydo*G!7 zhQ`71Ff@I<^U&k1a&%Js6o~-Hvh3e0UM+Wd?7tdzatP3YWJNsFV1#aQ=;)alx~Hw(fmbMb z0Y_bL8X~-1dn3@UYdR~dTROMb7swWy@(u*;lpELQIsY1v@Y=HXR6S8-7xeJC6$%L^Qosx+;?D3gcGGeZ-h6LDx`lJ-v<4BGJb_zz@ zL?6w!{hac41!n^4oL)pZtegdt96p}g)gB1B?p+C`)i#?gtgv|o+)P>lL)0Pnvxe>S z0R>JIZ-pt=)wSlVC<3yHkJ_F)8;i;(8wlT2fY4FMKULWZNf^JrSD?n<;l|0<|c)w!(H>Pb3My>SD)Isw#Sw|wtwuWvM##) zB3-?^_r1UOeT8zzn%#arYggN1g4D?mZ%xlJ9{cL_WXnPh-~DO3A&ll06nBXhXLoFP zdOF5;<$vmfNHU-Fco}qm%fiRH@anVB1#Z;%pyYp=lt~PgjLJy_(gFu_sbqA?gYog` zpvfoiDTtO-pYn`6M}S7S0LLZ3-$cCsaKtsRDlz44j30whMktIlqhI!9%!qSc%z6Zc zC`m#*>K$!PFCu^3yxBjR#m8pYEW=!Oj#55JgOB$nYXxWqb-WvYJ`%mV^9zwe>?$uW zZlU@M%)m61uP&(5`S=p=_YzBSAIlSvYSDh(tSyq#WY=nI)hO;y^Qft0S%|N<BO5o$p>3P8oJBDe?w3y=F(wDB#w~&{?(QU+IJL&Yh z_}}}om!Uq#Fnv}CvzDFAbCW+i{(TPb*zz3H8U{d=idA_^`l0Ew2~i8a`C!&5qakZO z9YmM+jPwCO4S1JOV{`KtIc{=}?Pz;I(G+?dTAnOQh%lz5r76Gq6@F~MNzWKS$#kVJ zoJT}PCNu`Xhhv{_58T%0o=*n5-qQ(VgkPK(^Ad138BER1%|ZNJ@|)VauKfkooKLp* z$GinGo)>?&S*Z(a2R7r|Qt6b6)LrgcHgvZPE7348M4und)7luvnfEG{)1xv3K+s(L z{B+9Y)(x0$rngD%C#ShZS-b&!&UI6z@zVqCA?i=oP9`6`sL+d(%%P^g6@>J($JQ=h ztmd*^=_J}!Bp}>ogW&0ik(W+ZzsEjb4nZC+*m#|mTQ>kj%p#b)BY}FC4VduIIgc~v z5lPO4X20=*?hM`Pupdd*$TzB1(`O@c1M;tE2N}!?Gyi)2P8U`o{#?5hiy?z+vDu)G-c66Ig`pV&*mG*X)dLx@6E|%$9+8I7oW_rGo|Bb*1R(#)PK*q0JZNxMJ)Gp z@ZTWL%0)c6;-xq*2(!E7j(KJvHp&VgS$-5LTU;_4YOD!5Sb2X$0)h;^jp-G-j@sx0 zmWzU-&t(d}O&LfRGY(b@!->OQ-!Ys-)Cs{ToMR zrtOgc9gXT_Uh0vy=#95CoLz8=K_y}5UE_e1 z&S4+&B(Yy&B?c?6G{06Ioc<IHQ50ORhHg zU{waHm{k+;njO{{s6o}zjZpL_mf|LVj{7(0o8;@!LXOV?t(&j=yKq=#H-f+;U*_3V zs<$Qx#oXgQuj6e)eJ<1gCN66PLk<2J^Y%=;SDWqh-RNO|;jaojUH%!_VqB}N1?>p^ zThv&pv-t_rTUNt3AD1tmY$!B2G}xAzTI=6(pC0s+;TKQ$Q^^*(l-$_CyczuWDcvty z4pzNQ*?%sni`sPF;qCiXI&zb=_2J%*v1gM7MiJ|i5UB-3{?BhN22Nx zeV3u;i^q@q8NwOWFPEya_j1@@8Zg?XPo`AXaNZ`Aw8v|{+u{ZPtjMhf>E7vAbK|p2 zmCS?fbnE7t*ZHkf8^^3)qQ*xD*99evt)fu&i#Y#lc&&S*pDRrL^+_5v)TdNI0u6{S z8J*H^O~9*{l}|}G=53P(>qDD5yK?VI61o&E@Xnf9E^Li*b-tKD`pQqD;)68R$TT&Y z5Bi22@W5W?wAAo?Lg&*`T3lReh#)%8;w!k9zfIhOi%Z1mdsb)3LWorso%IUXc_s4jP;ld_Be${vfK=o5w%IEyj%n?el z$xTt1uc|#O(Lngv#qtObmuNj&tQWQ5-b8bEg^5MC)o<%QEj92g`nxqbIkU~=5Fe>J zE3l7bYLP977JiF=Bfirnwq{IR0l&WaTHegDq~~Ah{7dQawrJ{dN@It4ET4GWmzL`|}hI~%p9B!FQg9*&{HC>w#=2UnMq2g)#yjE%s15hKmVD50j{}?5xs|%151Mv_iAk;YryiNF_xqK6Z~Ow+ zd3=3|E1|}aN|->V#u`swBMVJOl%%W=jVLco(8Lxc*BgEwCiX2wFcD2L(a{EH{QS;u zZB@gFPWCy8i_F%}Ax6hPpRH)3VtyHk(#N}eYs>U!T#96wz^H;xy z*}Afa-^aP37}xYUFN>*=fnIQ|U69i*8`39rHPrW9=Og zj>1V7D2sYK^Tv78X;j{v*%D9NW=^O~qe(a|cf@>gC_36g0S*%8C) zi#&{Mlo3=!N)v$-JS;2hm)}j%KSaK>jH&b4t1!ypf{h8)$=|9-WuYd2@h%LP(kCAs z%%t=PVk!lf&PzH;i1P4Pywyr0mh<=>sniIj8E8=PHVKJXaAu?#J#FAD2JXI_#{P{3 zXa>?)Eu&jWS}t9pVZ#qrV-5bKV%oR$mhJDt`VGV%rA~Q}s03v4w_+RyX-nYwUsJM! z$g|%HWZ`#5g0sO?ieEs3b{s30CY_Sg@7JWzuq7It;qb<(pg@^nu+Shyv|zSWR3`Eg z9Vp^5es4v6b@{d4mmnTCCrjI0%B>LwRMp^Lu(#*i4^6ladycCzmi}9##e&Imopc-B5{+L(GUP zuzk&HeNNEWG(E82>;Kqt-qAtx7)$W=3tQA2I6KP1bs^*_q(61mynZY^(6-ydt2FfU zWHe8cdD_whX>2&Yu5G?^jUPWE@$e)I_jV79!E9<2AGweqb_7-X%GWEY`_L_^C5-d% zqx14LDDNMHsk1L_6_E$co$qr-qqRFwLRVsUJ98&8tbTYc9u6ZHMK_fCz19HclY|Ja zFLGM=`JFe=nqGwVCzmSO`bg+V$lx1#o>J>Iv+v%=-7@c)I`SW<1iiMTg_itz?zcz) zCScKC9eA*(!Z{DMr-dbbVfDeVS^4MU{svN4;#6)!c!9d07-;Y%ap+<4 z$RdP-KMnVnV@n4^9!hH~C+!~C_o)4Gw$sfd=s9>tOiu40^RX2Ho?Oq$K3=f(@rng zF3r0)tW+^?g)4Jkx0O3X1~>SNt?Y~94_NOow|$-fF7CPAN`vR1U*chohPIU-nrfH*>JZX(Q3AI+qCB^|VKqh; z_0kpXGH@E)({=i%I++>yFvw{PM@b9a^A2rd>D{MU+8l%@WYjV_-oWc5X?TRY704}X z-%M+PL7nPcT!0YO20;3ZgQQSD$81YNB=41nr7biP8FNEEg5i2Pt!%amhweHY#}XFs znBhN+`7Jfo-JnP&2r+i^OczlCuD*zdWpKsZEY<)KWThKiQloRgVl785D)@1jTLSKb zj=w_hiI|rWBnsF!y>X+!JCDv?fjX_HRh}bk%@!0*IrD9WdW)KX3Vv|E{}vu5x!wDt z#FLs+;fF2Qa19!u+CW*9zH5)Q+LN%6u_! z8=B;3G&>^In`+}tOd zs$!@t3d?(&rxeE>n;qWIL@-?JfVw2SL-;51RwC<6u$U@KiTp6uBd7RS>lEF+sJsL# z9h9o1s?_4ErPZcU(WWs?fGvjl6jd zBvZY4Ul{8L#lWYKB94hg-w+XI$dQFA28XpNV7BSr*+5Yq5t%S$62wSajwyqzF?l0D zCKvmblZ(=qciga~S5w6P?=z1y#SITdM)TiHQru2Tdl?COw+DH6otshI7kkoppStDU zPd(q{Ayw51VFhj>!S$H)`<{XclFcO&>udXPYS`%M>E#XRxsL;#*T%DZmjslUEy!83 zw>gK;ZHwLvY;e-jj)6oew1@=9O_b;=3QtU=RMJKHTSeuFp^b1=Fu}1Tg9QH1J!^+q zK!2^#$mf!MxQJSM&7k$tur; z0d&+{tW9LGmhX~z2r07OiAPM}iUMkU^tJ4;Q0!(ZD!Ae}T`su=f=@A3iS(A9I2tCh zh|q~_%2EA|QQjjjq>!j+;!;~Y7b-$?Zx~4aV?Ui4p*|Kz8O|TYR4xKUZ^9Cce0WT+ zdxB0jYCA9ZuE|9LpXdV5fFY|&K!HUJBv?NGvNa#$ltebwLYUZpiz^RW`75uc#F8wCh(;(P&q{7wu1`dXg;qOw!60>_>lv$AaG7j6*yx#mQ)a+aBqS4^eRaq9j z=bf6y)4th&81KGxF^}>_U*4i0%nZ1N-x~AGD&+;9Ndvsj*QDHabPlTy>;2#tH~FS5DlSl<~xtyqYb*|FB6YhT?!vE-Z{a; z{@D({S1B^(J7|%7j$ggJ=r#sH1^ic4b#tE=N$iif&!{sL5L}XJ(n63m3r!~Rmya|ERIJl`j)FiK-MZIGcSi?`b1H0#-#6`^X6 z07B61P{((_QVGlu&TME{m2VFZJTg7mD%oyjedlU(+#PAmo%x=n0+N!Ic$fKdZ;t(S1;_Ty5Ss#^zKeu5$S=B65*~H& zcC=qs&ZuW;;rb7vVwB>HaADEoX!J*Nyu@|n3PjbD~l^-_qo_7hl9#Z;eXN(@g6CRJ8 z9%tQ!nWMtfHE1`UE|(P~PDeAHiRi`~EFwN2i)Vvg4EN(&NE%}pIB2h7(bt8(Bg9NZ zD9#V{Fl0crz)DY;E1O|%QpkRUqyhKi9`XbU))ljXEag?Oo zn0?4i%&c_yV0#fJRWbGS#uCJLAl!4F)PyO*;6!M+B2ak+HX*UNq&Vn9@*}e!M`f5M z6ds?ex)`4D`pwDm@xxQ^ed7_95C2O|b{Uzvn=7L8L)jh-3NodV2B%9vrS`Y6=Duw- zw3}We_!^I>o|55m)W^kkb~_$)G|AMa_(RJu@GYE8X5gt@5GOD(5bCmGKJNWFH&_hS zdF}sj^_F3AAWhfs;4rwm1{ox{y9Wyv+#Q0uTd*Jjg1bWq?!n#NA-KD{!#BJ4ySsV5 z`N74%neMKtt~zzP&X}&&?mGRb)39C>jcWN}Hb=UkbrfKKGs6*OSV)NsgAtA)Z{w0; zHqHz&cVqBcO;41 zxivfOmD|-(M69UFi@%m&#XsxsUPU8LNl~aJr&F5+5cX)Di-H`DhzIRk+ZC~g5ck!iOeKb8KfOPX4fGICE@R@hSEpexCU%WyEb^H#KzZt)#=z zsL+PB%-*~NBiIEnE$#Z2pp^OnAl>MtaohjP@&DulIAb^5ut8FtM$;>H!oN_`HlPxre%K zIJ48ZB$N#!p;gMCyv=m4=!Ye7l1M&qpV9l6ZIonPKKy(!u|{Yq|3aW6C@e-E(iFk? zf1Lh(dKovYVx$%6TMy0L5^~|IzVD_w9>uYW4=BINp0Nb_ zsVG9mPr<{dt<0w_WQux0V+uc=efypt7mQD5>9f4&F-4bX3V-7Y-i;8crCLExtmiwi z<9T8vwU@Pt{$XLsQ~cbF(|ptcWD0%l4@Ym|QV2s4-xS2Fy{Reaoq*D74h(A-P{3<7>7?m>4S3)%BQ2-8^EoUlUYsyd};( z3Xd90Z#2#Ogr=X|UAAXGx;rNA&EDv`k75e$(DZdi&>hIy`gCDN{_DrzSk{QH)*D7J zRk5K%vp&+8#viDJ%>-h>?~Vf97cu~LB8~f0b>((lfo(j!Wy23bTU%r5R9SB~pQF8q zr@d-8Hx|q-{tFA=9)4lQpV(o0VnD~yNd@fpgXaSDSF}0byZc=&-ss=|W^yLz1aDE} z+qJnLMDT=knlbtLbg^mH3dMrXNisw7eEs2r*37V_3llIqt048JtNl`9C6S5b<7~z4 zJ8BBQ&u8l%!PUi=)pH&!4_D){9(Hi#xtDsX?dctdf+NF`st4A_MzGnn$~oSamgYxE z3ZvPjEc*PXu`~D5@AwH3^Zr=+!<=?)u_8=IM5QV`R?pfUl+g42Y!c*$vp2Z$HJ<}3 zeUI}?1%bCXzbh_3u8a`?{KYfl%Wl38ZchagviHUeoEA4WAoKUT>#-gCweuFex07vr zTKIruGy}cfm-x+P4*?F355dKr$otl32Z(99AO{Hlgkb1x1OpcUVuN+Bjvg$ABnmk2 zFApk?r{0uM(9nefB9rzO@&8QDKEarJO98bGjLBrgOBuJBR~HsmKf`6cjF2jDJPu}+ zOEOBS#^a598!a8FkTmZl1u}Ua=$)4ehkLR-N8%5`hWbROUf z9$8}}#U|+7bYU!B<LPV?S3wZ`PMfJMnG!+Zy(Fa+gU2XMnK#j}T&NY+ET<*Q@Tx`nm9$N% zKOzdy5eDuuQAH|AwcDd^Ogup1D9rqbR!6e&V9`@N|HD_FTF1u#L{U`=)k`irJygo3 zw{_L=I5Xq!9RsWYdk@Ae$ev!w!c_BKhkE}pbLYpOt%c-k7T+ylr(_B3{$IiMx4pN0 zg2rtdM!PwP0H1{w(^?45o&Kv=g;GquzVC$s^`Km`3l-m7;cj!q2R*Dhvm-Gk@9VJ) zA(^Rag@2=0;Cn+BGgIc11$SC9CsTkg;-6{qxCdWo+zOe zvdtT<@E6Bugk|Cr%~9#TIJ*RWO3N-5H>LcjspVeT0R?%vR{@Pc*2D7?uuL)Vqfnx+ zFxOHQl|ook^V;QPu*T0@&FcRI=HA{;7J#zUhQ8-P!L63@)6i*WXQu`Ga0kH89KnvQ$royp7KF%>p^=vOj)s$&a~dd zKNx`Z4KhDskCDQWip)PG7hOu22Sks03I%CFX73N+wR`SllY}s}V1ORU3hu3h$fqkI z{wgiPqmhi5q7DuB9nW9`%k<3%IY?NAc_&-vyTEGu86~yW$4Hp%uwZtW9e*?hsUvT} zu^&X~%+KDRF`&@KaIY8&wHQAN!+a3|#_nk05KBNxslwx)NW;T_0xMFuJ5(EW#s}~N zeg;Y^(!d{k^Z)5rJB%n%Fgl7Y&CO54q^~s1X=VH;FGu=Mb!K%hVrNfP70(yuK6+R| zY{>Mh4A9braSCe>fvBzLF}Qm70~vgN?t*-6`J-LW1X@HF^^cn)1*K^6=tMT9aN6Kx zTjfUcrmnyLL-*E6nVI>!a53_^1?xFJl3J}Ynbus4sHm5`4_jSaLA7<9&7Q5;A$qM* zUJq8$A*)<2Jecyt+kH1`?=9P|FDaRTz^0>&hMLLnybkk+y#6gi)KV9o*&6P$nea0o zH+qy3Wh^+j>D3%um!6fqw!K7!Vy`>Y;AC#Ev*vo#W-qvx)ySjqUWu#K)fEXH#P4vU z@#fmvO^up}0iZN;V8Ouhfv#e~#qL~b4ZeC8LB4~t@Kbd3Z{iBKV)`p0lqYOh2t`#0 zVcLMaG0Dcn!$GZ%$hzZ^OsqE8p){+wj1nscjfC7qf&7=397TvsYv+lDZ+W1hPlDd> ziIJoVDBy?3W%pL<-+%by>fKE{nF=?C-Wc~?54_X!xViIvLVQ~hdh_^h(&paJw|cRc zU=dqr_c{54a4cok8+f+#L6YwQjql;Z_}j&s)56w$u5WX(mSo@XYgb8Rhya!%8s37| zl5XqVp^Fz>Gz!t2nWnt0niV_QU?7~A0eDu8%gN$nXXm7nn_3xrxwWoBZ0*twAlCYM zaZBuG%aTa1)sc@Wpg-CXO&A7b%+*-ky85&C?i74=iHz0HyX+Mo3AQDTP@mj9mT7af zagI*08)B$ZaR<wZiBHgzJ=J9PH@Fmm6CVsz4uM06@BCas)w2cX1sBWZ-3B2)$5-q1p4*c> zYi~4dzCF|A^P%Pf<`;M=F3!rP#568DU!TM2+T?;jHm^$yA(t;Y?XyHEqyoO&@0=-@ zD6!TD`(V$G!L!xf;wV4YmoGPX@V#icrAj7Z{Za5=y{|`#QkV6__ecDxRPyTQ_n_YE zOx=@-C)ab8*;qInDWMv=S_^3cvuXMALi)V!(EgNA;f+8Et%dA2thGf`*f*@7;_*t4 zAQdVl4GlAC?NuXiNE1z3Edxj3j{pH+oLY3B7~r`J-?L?ZTFIl-CPeZs;VGFdT6*sf z8s}!}L7zHkP+F>(11qz`?-22~RiDf)%{DWk#(fqG6WOBYi!?l&ytqK?U?AksI&tL_ zw^hmFS=-zRN|;4rGON0Qf6DXHqwM@5u9B8SB7I0q!sFgmHpcnqTBa-c6Wa*>o2H;vxQQOYGzFjLc<%$zMcX+L(MfFR^ z4p7wS{_^KuL);|nn6-S#^1bf{2)gXa$I8i@_U)y6TuaNf+*>^yEySmI<2h0xd414L z_$7r1P^smIkf) zGzoc0Ccf|Nlvgw9nbgr4DFX3z-?~rTRSO_RsQ>i11_>ReY8fEKlO{JE`4I(jaE~5C z4Xc;0VYVd#o#&C(Eq8n3{h}3>6o>*Qle7mn8oH zVduihXLW|d-9WC0puNdX>!wf#uN0-)f?L zueaWY-*8}iE4(+=BQ!6c+TR9S(5M*4xwOS=pO5$E1(4N?Z@^(2{pV_<6U|3zIWn~D zy>U5ABmeRIA)0n&4-{g%I_jiBY#KSffH@QnuGh- zl`4z2<`Nln05-*rW-Pjid3_vGgk~yGc!+XiVC2g^gB};u{SVB>XhPSAT_F z$y{+60D6ZawOa5xzL6(7MD`&Bx#JbFV#wdn=QV<8@(+BKVLC@D-E`=m8yl~Vnb4y? zuoV!({C1%Zy9C&&4@Qzo4o|nc*1Wo>2;3p8wXchT(q|3rRoE-uwl>SN^Lj_c+msuf zzO4J$0chI*tW1f#UN{yeZeL}exmd<)L78d7w5gW`f_nWfc$dGFvvS|aFg*gpbjWEA0$VbaapO1Fa%0WhXLndtDNsWVD>Q0h3^*WC{ayj4+!TA^Nty@5^d4FqA1A_M2*Ozi$3? z{D;+iecRdk#3+z4JX_i2E1Z}}J^{Vx&%n$_(PBCQ+C8p6s<;-aM4lCCV3EJS$nwwm zcSAIRdQ)fU;XjqPvIpYQd&c<)go?iQ5Wd@*TxZK_6BC!qrvqpp_>agcs2hu4^x%K} zqz;1?N)w8qks(}Ex;E=^sa=`YWI2?hoDw7%K4j=-f8yz+ZWbq!RJ)t=qRB~`I>dhG z?7ZduNE5gZ~SJQOiEPN#K;zt+s>zIWHOLU_^Ithas}{nV_k-OLq! zv7U?u(t-7K=Bh!HYhZoFPWL2twtu5H zj`F3BCmUE zW7cZ4qzw=Gsgx0((^FR;B#P7W4A1(m3GMUfy`G(y~^P0RomA5HreCT4X*wy00>=j1+FKSwI5a_kinO&@_*8*#-b!1$Uh?j-Rj-qKan-W4%&y8&@Jd?wCu~HW zSQt#z9eNCuayCOiRCE+EJ9Fvk@8wmkcDFl`Gxr~OHs$qv zR#h_01VMRF4K{+~e;QckY=T_HC@$ z=NI|t5(dc*iSHO~_>KMu%@5SMU>V=Qjs>-9AOI_bJa9;ec0xn{X-EHl|Lo<-Pl(_n z&`d#YvoNmnFVM7V>fl~l=`1ZGC!Y+vu#4U&QGH2`!Os&~MzO(T`y)ww0B}y?ro8$) zK*!Yu4U!BAMhXqk6%UCP2?0@vgsNN44*rjs3LsuGJpa8IP55C`lgyM%G14@XAn9tb zb1;R4yowBf0D?0`LiLZ&F71J=WG#aF9p@jfB-}`pNDJcuP^1t>4^_(h>N+$lIAeWvp!$^S_Yn zK|SkPrevqDLE-HHj?833SLs;8>)gX;V>}dElw}G+1t(?HJA)qEK!zxPuUlmcmkENg zpnx6tLOrHDG4;M|C&4!6kMSN!dXV;xt&jvpbrg@!{2W23O!6RbnWik^?AtsssN#X| zc34ywFLGNHoDr^rK#-}W^gvcbOGb}QM-QN>b}WGWiBOAFRTZiO-iOXC6hnpaZ4+MH z7P~)=)`Jk9k+zpIhb%k~uv!Gn(L)9g;`a5g9%PVp+P%6fow4>6`+OPKy*B(>oDXu^ z=F-;mdh&d9OmDq@P9KkUeNDY`sg9VxQr>6F6L`4GPnt#Galz2bilBF~%`%=; zToGmOQFk`|K6ChL-1Va8p?Ul{8aheH$D>BaH*%KXzw9;nOfu=38@B7`C4Kv2Nru{N z5w6ixi`)64x!_fiW+pBAWDD_^;$k{B+BU+)$+j3Z1osf{D{$%ll317STg=r!Gd_~5 z{bwP#fYW)=I5k0$@w%k{rjA4oD#4P7F*n_kSEeiZgbe_{vN~+!o#i^beAI*`b9VUV z-X`z5AzigwGt-@!*-faqlbfEP=6NHyVyfm$C~^W)^Cvi;-f zwNguqKR0znSI40g+SQ?K_5Nk;rJY2mC5-~a-Qat5EA-m&J~MX1(L(iVxq;y-sL0hbXsT>snQz;Z?werx42%9o~n`i zS0+mQD5=PDM@?9;TJX@!^vorgY>B$rj7b>M>(WcWE^W~<_Aq|9>Pn%{a4ygou@nT2 z?Hq~Mw(Kr*X6pAYptpz$my~EAId;~JF<#^7><2vB1q#zFVxU$YA(d2Ozk6z{xa=0`^PDrA>pbM zv3AA~_j1!{ELXTYK~I;2o|O4GSjgIekAoITU|hghwES{u4Fd03M)l zObZ&oCgRT-?s@oK%fERxVwLMLA8xA5KTgjits52$f&uiU$BP;$3US4DHawh zfO}jXsu?tu#}M2OfZ?KLwXg(s?WpGCX8*6}C>$KQ@7 zFs8x_OkS@Lx|PbxW~%uL3&@w^JOGBdx&}#+U`YP8@Puoc$X z-Gv7K#fK5MYy+iB4-5_<3)%5B)5~B4>C-;qTld>zyTXSp&oYXK>$t3Elh_}k3`wqB zzKP{^3b_UVb~_7tcO&zqzi>*AHNNuMh<|X(>q+)B=-F@GG~<>mjwvuu5qeFT&Z}H_ zXaU8APPz`m+s-jri4K%l(;4(w(4Bd$DnCO$eON_j9Mm4>VV`2@G)#Z-?B>mzo ztO`_e^}jaKy`DkXGxoffdZ$q6UmWcr-RHiec>)YDC>#|r#@0j1>OAFUxAJH~D%@Jg zt_c3XAZFb)fRnvAfT))yI@^^q4L&)x#6FC3QmusAifV>TwJ_eocSIdp3b>8$9#i}CgGR5lH=CF zscl>MNS7VtS@ZCGHSTv1p)V7QFxrf-6HZpOjV-P9z@3Jm34 zXG|WJF{wyOnk$z#Q?l{g=*5jEiQ{Ipm_BhQj*f;$BV67F(naPd(~BdFE3?~q?t2jZg+!!83&(e(kb zv5D1*KYst$-bT%bqxuS1UUZrR!T?B`x5r9KL8zc5Y>|8XErcrI##8w)&o50&^JPUrgj8#GZKYR%W)2BOUM;J zf|AsMOdm9!6Zk_d^kPQY&t_T{eHy-LW)utI6Pa$okms_kU?j3J97QDvUl9Tnd54LI z8nMaz248W-@jH{=G0izDTX{EqgwwB9MpcFsIspm6f@CFX6|oLI=gZd^e_V8b7VX(# z#fRzUL(iEtGKUeCDF)&Q<2X^PqD?kN3kzpzh&WHyJFtB1b=%W{KAuHhJaRD4b2;A! zN%s}2V%Mi52oHx0B;x*gTXGiCMJY)1^)xqwg-th5&ur_$^Y5(QcJLry2gFB!S9ss!MLwnEW(TsM}xL}UlI2xndZ$%W%*{9@q6Wj z(h*WHe)SLZ3ytYe2RlF*1dcpJB`+C>&z*T^UFCnsFDy2{5OyU|e<1%m`Xyw{ zX#DkFdz8~U0*yRtD-#r7eb7JQkL3WAQdF7v0C3+52j>>h_En2wu*u) zYB36dfA}Ro-=?=qKQ!d!&KMe`2!z^{H_0x&2z|Du$h2oo>{!%OtB=J}iNBG=E=V0q zTpV772n`s(`oVWv_V|wy#luSry5Cf$>E|03bHKj&>Kp2vp6^3k`^_fPTSV5lG-8bQ zxWL_>1N+D6fP3%OGm3_E+}TW$%arc{($yx92Yi0_Ik*B;Lbtn$FI=3W%=(3 zM;SjbZneZ+T@MNEFYsXX6aiB!KV8dL zKCQE$j7V1^7{Osu0TNU%j@c&v+T%;vT*fk$os6^3y@smco9hOi-ijc|%~Pi}r;=nf z5tS3~n>wl21?Qw>k+whIAyt%FB|gmh&u-KSRlncO9F7n!6M zXmFo#NCZb7ee{_-26u51oW{&TMQN)qaSI5_=OBxK6eo_J1pjkSyepeSm0kz@mP8i%#!9`4 zw*|k45@*i$HLTPwl+I22mme!SDPgi}7LKpLKsb}{vGF+g3EUo7e6$8RaD4evk{1Ew z(AGwx(>k~B@&v}diT}dB`AayoatBvm1RR>`W*c2?PS)4=f2Q@$_Koe{EHLo#U;c@G zueD|Db>{U4Lx!h=w(Of~>XgcY-Sivjt6|6HxGJ4Ej{m{F|AM0hN2kHqH-tvYy7Wdm zpWty}&6el8Cf6HS?6#$&!2WfF?k_-C`oa;+^H5sD?QBTOY*Txk4l2U5*^G?Ei^;r6 z6Sc^0`XwuWkjExcDh4e(pm3CfoJ8pN(tIxM0<6;1>BN}o{E8okH1*NRZlv~nT!q=e zvB?|FwB9;h!Vtq12DC>(*GOhcwu+sU9?=gEQg0Y7Lo5=-Swla2XKt0x1#QQRA`(hO zB>a~YWw&!TNAqSit+4s#vSqlg z>R)<|t}a^*&0MLJH;aq2gv}&@s}3m0(m8Y0U4L9oZW_@2!+I@Qh&6>>i=au3nnay&xdarh5HaFpv;q*1}-fKg_AQ{PsF&;Da4xNUNRWTrBI`U?6wN_7A=izcNy` znqW@pnMI55k3obQoyt|RqTb2@mDzHU&$aLnHbQcoIU@g{6X7gFZO#jD=O3G2&6uCB zK3I3Rax!=C8Rn@^r9@JMxK#gj)V*re-*2pzQjlk6)!M2i#G6_fQbSHGKrO*egxqD& zG`oOzOS*R+v4<$7qQhAh^<TJD?*^ysTIon^@~%7EF!>4lSIzVw~C(F zoeJIuZR%p8yFLNmpn_ZxzH&jcy~!&R!&L% zqrX2I7*(~BDF^*n-4jWMQz8kPAsloZ3y|W#zEtw}FPL7itZ0?p2<^{L=m=J%Mi9=C zQAMK#ldelwa>%|vUke3*_+kn2i?%PlcYVf`AmG{k)iwyjn;N9iWbkr?ZqMHrpn=;g z6${0@Y7pDVbHDx!yC-{_>&fJ$9%uU(cF%ow$H%rdwS?0L?sj%e!$7U&+=%I|p~tB! zpP)72Xk&l+*~hPf&yZFr1W9#O6;?rUr}zridE`lNp9L|0{-nzm2bvlc()DEihLgAZ zn=yT-M@$=jqU)eT9sq>kmu011T-40^=FL0oEU4w)Ahds@QCog4et6{{+8>85@^`mj zSXU*W^1HUP6RcQ@hR7A>`505#NPpe(tw)27UJK||)eIS~)rb>6UO|p+?g4YpxEO<- z-+lS)PP49gO9GF5Yy{ ze2@>GnVaBfIYx;Dk&H%s>gFY&RD_8o6(K6XQxHd-7Co3BL2(@+q$aGI$mdc0C8)PA zbEKb_AdRc|8CLMd&-_2LU;VLL*FAhLk0lcY5r7)nV&e7J2a?QSxxry;{lYNO)J``O z2fEXl{dE(!J$dX$3pa+sO)L(?P-cTcQqzPOu@pR|){oL*H;ovlWWK$y7k)BliH+&p zpPZJ?{AK_!BLFn^P5*SLh)cRFG?y719AkrkM>B;54223PtutHA@M-Q|QDCUQ2Nw0obo!%1@~nGs z5URqaqwx(1w+-lK=1c;gUc@Ni&QBc7j0@C;^ z13*$zVZz-9auch|8WBV^)56}qCz8S3>2g>6F7P;SwPfOLas*|S^L%6|yH08$uo8?O8*~n3%L1{&5TFhQN^zca8tCx+g?*Ev-ONt!%TZ0+ zSi1e=G#ZH>0<08z$1<)!-?!v}FR`9B?_~#>p7j2^U+}J0$U7{fB=gq;x;+aXH~cPN z&%DO(Olr1=^z9P?qJ2Hh_lE4A)7D0&KNSYi>guT7g%op`vLC;#Wj&}o zG?nIJLx!rMp3+x|BJea*{sh3MgF6&b8VKfkdqmVC@3$0xi{OWfZ*`xpAJ+Ai=l|WE zKnT=qOCKC6y`MQ^9g26pk||jFIP(-roaB={7~^?^on^BE?o7ZgKHaqx^X=cd zRaBfj8u9OobTc1cHvviARgGYR!V)4tV`(R0#oAW$VCptp1yV297mJCk^0jq(^{YvD zIs2$RT@IWq;)uSs?`pbMsw86sa9`un)CrqauD|-g0Rts7c|amiRLbtcc;>l#QtZ6@ zD6SQp<+@Cgd2KRiAsJi77Xi!WWsMT+9@FOY^#&`8awGI?n$3qof5PL3%6)=ixQBi2P zDt>`yJVzBqdN!{dAFmums6fxkVAVQRLVbrJ*WV1fg@{C18&D ztRl>WX3TIPTp)dOhBns`pk(~X=Sw)93Y~a9>o6PtZ@xB{Bej|o)DW;`XfQA9*tyNQ zsw`W`wVoo$h^w477uvImQkAc^R3X|Yj*u)Lp}?wU|5!y1+3J4^9335vj0_K#{ggUN zaS!`hmmMckdSy~%s%3I{xk<2P{>%+=F0ZU?uIv`uKVn9qFeM!7=P@?~Bs)%6GL(VZ zDMF5;GrJ-&xSuh&K&hw(o@U)r`@@H=`$;(mBazv<|QbK z;+&u7v%4E?N!LH0+ zmn;mt<(Sk!i>Kw!Ul1bUb*``HBTVC4_;!2S_aZ1F#*q4@#RWVw=_GvH8Phf5+rOpb zRfd1&+W{&h%}ytYJYC(}t3KF=@4L+PJRXiLZ2X?LNXMH|!Bv2Q05?AlbaC#6?QNmw1TU_`(f zAX}M!Wu&gzOREYW|6DxsRQLmRp=Xq)5&2;%qe^>)AD0q&N4AI_kC3W5je zpR(0Hf8t&v8jc#J(tuEgz&hQ?%Nt~WQX3!mH)QUns2tTvw)(u@^Xj;Bld!3Sn_3n~ zWWz43(rw^DeBS5xF^>7BVWr^{reC9dJ|Vzw1JvUk_GOT?+w?v@4ylb5%UnS%5Y>iA z_^k7n#9zWefsAD;AgPrh(@Rw{6DQAMK223JQ0TR(Su9N6Z)+Bhlok$Q;hTC`^GIr9 zA%PI!^C2kFRK|p&{3U`pv(M?_Fv9!9Ria!vS-!{!@B0Xg)AV;1y_-?JRv9&Ua3f`O zD`Eg9CA2fT1p=fhb!d0h%SUwZ(G&nlSI)VegJpYeb=IcjsuY*zPTDjC<#TRhNkyP5 z2Tgp#hg1+6f>MZ&If{!)VVCID0>-GUbW)Orr8T2jA&B4~%zl?ozVaKtJgXY=fJ8Hw z?L5H^MX9P-o8yUA0l^N^h5JcocPRisK8JWYU>A>J7}H%U=@l`LJOA$JZG6?ouiruU zA@}XnshF37EtE$Fi9$j-KUtVgx%z!rb$~?l`=!foXBjDcM`a#DIxmiI@LqFI8>4A= zr`&~A5_B55l=O-*@yQZl_|Q`R5YVAW&gH!3?WD!clab=gQt|sQnmTBcx-N$*E=(8K zr@CI#>x>0C9aB>J%uAC-emgOw+Wd?$bnU1wtdnBBodpbh`xC|q&hHomj67IO`K;CQ zc8p4tevv-xKTHPJbWc>b;~)`mpm4t4<0S=EekE!==QjR5`5~@-H=MZ*;TswjqkQRa zILCbRqWo0X3sWLvc5>$X;0f-tcr;OX@}R%}+!Nr9aFdLTaQO1Zw7hkGg>NH=>u61D z6FG+%GplKT*r{-ODZ13fG7f^?UT>sFdGGWJm+w^!unAg67^Xq}9zesaVKsXfOO7FjVTV>!oTb;Ii;p{}+q z?~lD}>~y{td7;;)oH+meDqefoI2S{R0s|Yyi)Q^-I|;Qakc~RDgLLY=!{VxqJOy!>~yZY7bqw+$^Duy^(Li5HN zO+t3&UH6+Uj*o;Mk0^RTKs!bX)6y27|Lp~MLmb%b74>+&bP!O(*WsfDc(y%EXio^2 zTKiF7-xP)fEry!{C1nqZXsCm|r;Dt5^OE%OMpez^^wV;#TG}=iyY74QzPU%e(=VaV z8dTM}y1=_}X_S-v^%AiA?zIouLtnbD>fgP$ML~fL?0X2+zCd~LZdGSgdm`3L!ClXm zm(Z*&cVD{Y2sr?r%)8MP;{3w}2vBpG=bcG_)gMOXmdQN2K|IDeB9D*cD6eS?XxT9O zQAUC2KWr{|NhFy(Xd1;m$DN1&B%}Pjk7tOFyuE^LahU=8wso07rZ`W7qt81C)DZqw z|BoS{iF)58Bqig~56{I05A{c>vc$`YQ<>6-9NYW+&|uKx&%annptGyyy1#GG%G!zh zIXkcv?Fdtj70ok#F=E3Xz#dP&c9}%7u zY?ZMY2|3(@Cw<>zAAT@W*BZM_we15L&K6=I>|Jg&)(G)(37$PsKay-c!>)PKi51+E zO@a-Z2AqjKEPsQ&IaWX5p+vz|!wxS@&C2|<-hGhFYQ+nFjCv0^BG7@^{Czo^VOh3gg%}^#pwk_C~ zHHCk-eltRh@YE;!UiN!QNu6?_!*`?534&ieY7Zib<;ET1HmoV%RMm8K*WI^s2h{^0 zWUsfuDlr$KzHD%^cElzb>hIxlLgM@Bqv zQiA~}x*sf>ZF6P!lH~W06kaSt9W44~%b=n}7Cab%m2Z_7uQb^}isdsRc37V}Sop$z z@+5HpjU`khWB}u0N3_3lBfyDPwui(84l&(9n2{w~WNj&Q)+!^lj40cngA|UibP58n%3BNmYf{a5B&#H^M zs_M>DqX{cMv_@dI4Fir7QDJ+#-?2~Nvc4>tY(x@+Zi~t1gPI<)Urg-{HnofWc(l(S z#SI{n)EnOyc@5hWeX#MpPD4hfSz203xjCm7L$b0_P*CvGhg2Lv(m)Gj(9quv|M#rG zF{&qNOvGa_^ftmq=xJd*6oB?cHRp_=Q=QtXtPVafeG;yP8mvZGTwMIS$;FB;0f8DP zqW72e!qnQ@+mtL*q!XGNw6Yj$F!n-rPS&rU%6J7~HGir~BiTwMQ>ncCDiX~+gOtHo zMrwvn6VXxc*&ur467lKWIZ17}^24<;wC`Jdp6Ew|c5{DHPe zaKX7~;)>@G8r?|MbVSh~#43TXfKIpHH$Hx3rXe#|$j}$#;Hd-A8shh(Tx-kKro)Nv z{pwwg=D-TT;*o*sTGdM~T z%XDkN>-T(BXm&rdHzm+_h6M1K&Q+G6sT?T?FN4}^{m>vq^JE>R8*4Xr)$)M^U3FsT8UC~q{U6?^_KF2Lwci4dn!x(*eibF*mYe|L#L{`+ zsvEcV7h`JN>;C2+P`J6Xrg{0wMPNY@+lA_#=Czm8+mj{m_)H~$2%5+7Bvv#B>F(uY zpPXE{RE4hB{EC|$_x;6(y>U)8aZzGo-O_Z9BToKG--rHYL5tSkE>D|FBwm{iXTeDd zoOPckM)0Q}ADh=aw(7pCS*fixJ}fD`+^YLY{ix~Tc0XV5Y=sMrIV2^Usvk{Dl0s^^ zo97xx6+iQrsQRL3*d=qa)?8T8y!T52vHj*eZ@@H)R4DTu2R{YV|5++499WN$isd5! znscG&?e6R?X-wS!0EJyQtDWghfdTYdpu|juOB>MD)jAG+e*>)AI^%yjMs03t3OA*f zWHWNS`0i{*L6`9^jupN?Z{vm@hh+6?^#h-qCX~c+CxWVco72wa_?@4l{ruO@LE2*( z73F13P9I;2*!&>8zt*g*th_HwLpr#9J&yLf5B&V;M{T;`23dymt(&uR<*Y#&G4k!B zSMwM<7d8n33+SheNM+Q`SSHWw!7G$t`j$E076+z!xrXmy4|>J5>USOEfA0dYZz1P0 zGu&!D&)&W0qKtD4((T>jXSpX=_pvK9eBU zmfy7qXlBMeJ3EW5w72;{ttsWurTxFb;rn1P@q(YiB!63KtylC<3z!H zNeGCv$mDafR9Dw#Jqlbta)bH%u59CfcPoIS?|{Mazr4;Egh-TBO1v=s!}m4o4V}O2 z_hVbu6)T`4HI(6J!^`i;gYv2S*slnRNJbzd=0FH$bT(}Ow3&v6tYU$5caIu*oZO=q zDp$F_U%TCk)BD2Q@>6`NtVVYG^Nt+2`3eOSCtNK)T&5~|CN&%T7G6FB7Qhr4pJ~q&JcQ23jG`YyD&{5JhuQt1*9Lxw z2)FthKJ@?xqab8;F}IabFSE6)PYX@l&0Iy3vGI#;qsrz>25EgQ*YQ%T>{A#=$^5d9 z-$!+?quX|CJ4@bfEGi9Ipd9##{AgQ16?F9Mhi2D)D;mMuo;c35-*1DNh`3q|0P3_i zG}Ew!SRkDY6t`cw%K>ry+sa)gd74$TJ6J}zo+FFLY5?YXz#Q~<+~~Q%Gzo>PCzR;~ z{_Sck6vgd(z7*9X;^O{aZfSK?buU_JNK)8*9C;_NhUk~lW4$xc5jl@c3bzl)LT zp(_ap!0^_!RMWBfVUuG98+{uWVyhsYx`3upMWqn}Z2FBo9T(USfd`~G=6quY?fdUl z!gX}i8AvEvWA8v61v6M<>kv`=n#1_a%(62cw*ZoKXtn)ZD`!gHGav9rBC-t7KT^hi zC;w{YBgyqWDCd8cq?dc4A<<&i45{c5+0CnQwH_SolgPKm*F}dH?(T?kHEdK(mrY3l zB~xlHfnUas@UJM_io4oyD!b>4pnqP%eT5r>>KyefXh#u(HMZ#HgJl`OK$09QgO^b? z$WLxI`dRU^hyM4$t^a_>rBnTt+lm0$FE;jb=0>17ZGHBCL86;S%$J4&Em)T;Lk42N z#AkfZf^u~@13K$B20ftn`nOSlBpOuftL`P^UifJUy)FNyR-<$Jrr8!zdT1#zt>Z^_ zhQ+^c)QjFhKbDWd+W^2frrSPh>L8+dcR@BWB?ljkCicx|8*Q<+k4y|Cn#k|zZ%qwB zvf-gdLwAvO8rX|o-uReC+^~jX4_Y=oYMqw1j9w*czl}XZnaPjwtQ;%b2wK<_@hyVr zDPp94w0D1@RpTXbk}}Q4k0K|QW8J@n=%~DO>2cBBhGl>1-ph`m!mBjn=D{`t1 zxu+rTWpe1PAeeKI3>4O5%Sbautg^?74M*2Chk z6l`{M&13^l*zJS;Zd9A5x-pN}cHcCUd93ip$oCB>z;{WaB<5izf%f6lCg_Hv(0MUV zJjA~ca2M8jp@mWy+g;x?9`TD$c|Jm|d4os}efDnl>iXA>@=$CtQjw!y0+WW1eNN`j zpYW4Tq6_gB3oR0h9kF0FK5>pfp_p@MAE}i4Y=0=v12cwk7q21dEM!l}V_nRJuFva< zbiD7%+JBcG9ToGq30B%a3>9mH#+{J1JodLm)c{G7u}ho0C!5!2p9$2((f@Lh`^7wwju8n6wl%H4h3fiEVIij~Axh zf7Lk;PxX%Dp?9Ot(IA;g9^95nkz)Z_oWPY*P*(v&M`s@BeAQk9g z?(X$j-;dYt^%-xEW2|8Dd^99W>ow+C6YEYTvC9?&Ic9>|1m_mPSZEav00Ik7PPL5zB*QvF-Ley4z|J%4fjgm_kLNcM2%Cz3`U z)eKeJi9X|6R$GvcGSs{nj08=ouGiVfLw#=^I@9$0(%kt&TgJ*s8A&6jO4=8W?Vy0n zncfkgMEMb~eVL1+3X9WfhFkgCZ}%S$!dFGfoC2TSCaaNA9;5u)vVTMQPi3^}nMZMy;*1J`LfofI88pETk&$dc3*m-FM zC1vF3zbEMev7?k8;~^E>48MyJ>#%6^`Z^4Sf6ZFd2!OQhJ+`*8F`VTTy{S>^Qw(9J zqq*B@_jb_nP4rBli(14szuR-SmX{d# zG!}qI>zM|eA*|&7w(NHc-$a?hi3yDP{fz}_@+kQIYjau2<>k1@M*-(J)w#U7f4IpA zmfs-Z?-8=fkN||x1&?)Kxr7W@a}x#$y_V>Bbw2RboJbp?ZXDgLBR-S3n===~wGx0i z2tmC1nwO-lE!8cZ+Adeh%H)ra14gO~{o|-Sxe{O$2LV)0y2s*lKn$tlW?QycC3bv~RCV<4Aqar6H(Dy_ z46<=n#e9p5M=YaEl^V*aSRL?{+nym>&dOSoEVt|yybm4KF(K#&S}VtkMfE2lw}Tr` zXFq<=*op;Kd?g1cBYs62RE|?8R7Mp_;JE{vm{-#E6^7uIeWn*FO>kK!`y)HNm0Vgp zQplVk+UITm$!zZ^k?rp%flO1LjK?~Js7mlbI}bywzr$Sb{?@Jdg9qOS^to`0ETK`s zvz=mUtP-bp?<_?cMvyNSCb~}P$o3pc$U&l;7q0`<2aGgvE&oQx-I|{cCMBA~RbofX zjH4pMKA<=HlZ|RtDT_6_?2dY6`a!OX@Nj>2TOeVPFV`>e^hb-h%%(c{9&>n% zdV%+5&C3PoF$2$YlFsX$ItuaVsXoGFNHvG;qkxl(z0tL5EZFqXN`{qNA(QNzZ^avuhT z#B~erQC^_4>x*pcCl-eLumaHJ)KcS1nRvA%YP~IR{Ts6_`xPbm>y@g>Rph*-n`#)io( zP6h+hIoG;kqXl#_<}Bvs%(@j;&2nABsSrjbvL7FkizdW+Z5)JqVQ6X z;g{c^U!7#q7w+a^5UMe~3ig9^t$FW9yg-1Yqsw0PGirSvpj&(HBvXRZW2!{ZQYYKm zojb^I?&0k8ywz6Wi=kl#ot8}mrD!toehRnS{a~{Oi+Jr(7nY*fC;l}?Eb7S6Zf0_U z-55l&v35IMtTwru=gEv(@^TpWQ8HLzF&;KTpv{V9IwH{3fGK z7|=wumv2(RlK%#x@pmAn63kX^4LslIVYJ zK^bsGOsWOJQcbkDr=*o~La^0!d~Dh5xj|kztK)OMe=+(^O0{6M+8si=23=coli&(C znss|t;0%U%tc@gf!|s_hovQ0lRu+DCx8M3~8o;%i#UhvJJq^ZtU%vzLkQGs zo9kcTBwNwwgN~|_{gy&8)^2Zlt`swTuP?T>uQVNQXS0TT&={amY`Ga-Z=1&~Vqn>G z#UQ=#7(cm5%xvmJsL@r4p@{;_&x?IU{Oe-bkj5I(#gdr#d68_Kb}_B%UH{w&%)Ld> zb>F1#t#3r>9()7L4eTPvM=wRXxQjR^slvgD|KBF-#Fkq9%Kh%;cWP*fr(?+_c2Uy? z6JU!2FN-+-PbNdUck(Bgxt^==isPo|)j{t?cldWUxgj_mT19VIQrvZ)K_$fRjCMxj zL$F;i^4k9sj&7v7e_#oLU7=ypNIvvb-1V;K3i+;|ZU#$M$otzi>EHCikfZqVjL_fw z$Sm5S?Rjz6LG=2M$(o_B19U?Q36p9*Vf0oARm9%1E7|N|^{V{^!Z9f{{*zF^z_+hD zFBt1Y^&_;#aNg^}bu_plTWC$pYb{iZbu#9a(Nt@KJE*2>&M9Y$U#~Xb@4LYG^TW#% z2x_JkyqWwpu4Em{wchvqbctGEFWuFc04Bk_1ld^XrWs#XV^hm@?reV^dofe%_t(F{ z^z}Q=6A(MLZH^qi2RrDFyPlftB$B8e;U@*<&pOuR`<-06p26(;4);hh4sM114vs=z zT+|gAP^Ad1IeFS&NewqsUoJ|H?)OBqPxGm390=k%h=-ZSFD+dC+@DT$Fdt3yzR>$c zmRF=oZ_{~A;zGv3|3hjcDdzAqFaIl}P6=pU#H!bu5*Bf{&MZ&Y-cFIqD@W-g9}Nyg zX!=W+gJ04OD~0%;hT5uA-=tJ9rnVzz@CsGij)=N80if z>w-C;r}4Te$;yfZ@u{=3lTv(!rmnTWjED>M)?`-KpKaVjJQS1Lz>rNZ`ASDtS+~Ua zWjB~L{6Sj9%d2#lJ6(6KgM&fk_God=vF-vpqR4oP9{k0{D9IfZMi!4H19Epa}Wzo z@|h#O!1(ma`M0_|6#_}y)*4~{w{!oM3wlY4a(co()3%|)N|s8Z5H(gpG7s+n<5zg- zcpqi6FLpf2^%^{X(o~}(#=SJYqt&UhJt)=YDL>!^Fl;|RfyeAuTeO*Ea`(nEbOo{q z0Y(*OGYZjFe-rrSW8_)03h>D$hu_u1+VjO--Ay@$$J*8L+KcC4ER%n6NN>JEv7NBn zu#T?2E*@qu%)HCmy0~W6zxuW8furDYP)=)WtG`=KwUII7jozlCVCnm)(56o@`3iRL z{N|Nz)YZGYHVW~=gFBr9)XymbKfW!NB1mYuXJ*-=>xo68iy07q+<C)VAa7im$|CGDiLfvOG`G^ z)-|&LiQ`JA{-CZ2WkGWw%;r8_)S=v&0ADFXthG4@++HA%aKT>dF`6zKv!PfRF58ur zgE2C-$9k>}d!*`p?=48fV`c%oWC^ysxRW&XWhLS&d~l^(6Z6vpc*&nBz_VjxS!Frp z`EoWg($uq@>`sjM&dVQVR=S%8V#vroM-R-XYc@tX>iWo<*_F+P5*M?+6&{?s*-2wS ze;+ki6j!UN^O?fAnkYM;XLw*ranHri^6>Y9LOjawcW8bbWam2-_tTf_UR$Nam=TN7 zy~qAsCRH@I$ygad1EEIe&hC7dy^wyl8snP!dQFWvVfzN+u9t^i zoPwv#p@8W9HR#v$2=6vFSy<_JwI2^Tb%tI8r}7Ee7jo`h3TAunW6_+rho5j#kezgN zT+lb~{88~MEx!>qeo3HwJgZ)H?y)wak)gL!%4UbNjauPr(ZtZk-iWLJQK^^TGERT?NuVt zfA`nTl8Tnkr$&zPMg*nCZX3WMP?~)*pU>`^=#%AB{sYJ3LB&^r?6~H(=k)t~BtRdM z)s0(SaBAV6jKy%$%by^T0)@7orZ~UiwJ-2;Og0#JCD9cb`g1cnPzYv2x$e8ebD6+1 zA50-5jX-|lI~n>??t?3f&*tjO-)XfZWU^_7cD1(dg4-Kw+&kI$c{oAdE9p-eq>-U& zQFuCr?M_ek4{&{vlgA|r_3m$M@e^4CnhfISqeuAoQ}ujwpK_U|RN%euuMfg9d;yUc zCkyfMNT;bJVL=v38Id1ve1ZG7yCZ9uvRhzt1v8Gh#YtRD#huF<{jpm)EA!BXuZ*p* zc})uK+eSF&*vj!(OG-y6Yq5BY^gbLO9r>b?sg(ZC7!gbcIu5KDnal`)$Hf9o)N4Z( ztTSD@C7J&}7GNk$L0LFeo{o`S|6%=& zyntBa%gkC~XR{O+ekuG-E9=1^_0&5?C3NcjOf4V-&X>rp4&Li0Rog*L)HV8qUl*!yUljt66SyDaWq?2q-Om)OcuROrxUq| z%G6El>^mZh*Y;MYv?(8%1c{)+;CpRLz;LPDn0rM)nqQNkGqKXe#<;PDl)5BKv|_zY~VEgDR$>@FM`7CR#`BD5f%oH+T=vG&{mL7&8f)3)N3Tih zhg0}!A-3|uag$#JsBAGh$EEoTS8c*Fu7FY*afSjXdF*ZwCpX2mMXe15m&Vju_p!F8 z;2S^NbsZJ|ZxE4YZ`upbPm>bb$MXTiixM`{=#7*zzV*hlkAd_tA6-)gm~F#|!e{P7 zv>Crgx|8?G@TZxY4`B*9!4sWh2%*0UWvi(DdZMbu)VHV1Nbas@%FTpJpE=sn%Vyqw-0Hyd{m8Gq*$Tjl6Bh%t5be457#8R!v&C_8a zwCB5wJ;l3XiG=G2D}zB2AeTM0 zaZ%y&L>|ZU6O)r!vEI8#;`_&^-X{>KY|3}K0MrVdTIa6A9k;lsV~Md6hf_i4{G8|0 zFb}q5T}{;u$$G<<%Dpa%0wsaHc{2hXkkV815|LtQ(U;l3Et{MOF>4{;UjILzx!g+w zO!rxu$p6l{0mpN`^A`#s>fjgZc~-b?Woh*);>T2oN>S{|{y|yFJGab&ls}Z@=3AW} zv{jyNZf|k@_F5?lK2r)r6;*wlt{!FZi>G3Wk#qI%;ArtZ_~z>aMBA*!tr^x+pvGXn zTclPSw(aPU`;O*231G}i#OQd%pQIp=Cf4kEN717N|K}ovhXlm**<_5 zb@*PU#BM}!vur7YDItbk1i&s0N*M7*QsQ5$b!zwJ@=mreC-F zY~{AG((JKU?dPN0r(bfG=qEqY>;NoPo36}WPk9EKPU&C+Fk<9xJfP^W=gM3$KfgUq zSl(4{5fjy?s8=XHIy$+;t{z84^d}3`bw<*hQginpjj_SLWExb9%dym$8P%=w8%*`G z5C`?#r1N~X%&a#VEMpN9pYV6>m9o7FxZBQrGUfp~WR7JBT44s-z|=mw__jGo07gCI zA9|eCz57d#JYY!K(8yGXEnF)c2ckZgZD zyA&_NlXqE%lSCIk~BKOjL*w@SzCzL?d+6)@`eDGE>roGGy=(2&}e) zK(LhD?iZohW54-jbw-9|X&R^1ltBAT<;(YW$`@8#6i7jJJReIv%+ynzMe`VVcCtG9 z%Vw_B`|*O|`TR)yh5_-9-yJ^wUP=~`w=U0YseYr=R7>u^{5WEa zH(hS8B{Vd2_KGdcVwC>j0h~1b1YHv^vVR{z)Mj=mZ8g0j(NU{}lTx-8+j+mW1Gh@;*Drv6Ihvz$3cg!P3D&DuEf~|D$0;96kEnm=y#@tcZ^8a!23|Py zd|YmHQ5+#k#5el>_GmIMA;3+L2(9soux#9%nmmQm?Z$3K)jUJQ~JHuR%9z=yzHskAb-#bOtXV23oGhb#xtRp72#=n=hQsrz_V7|~`; zkfy$<)dj`T`oe&!g_2=qX5)s0rtUA}w1cB`J`4Bzn?{%cjL^kFFxeT%axiDj(oQUD z=&qJvG;eoIn4=EBTEg^xEv&{g0VS>thFEU#5OEhL z;b#?q!3E%{uk8gV$D2a+3Ta{(8T{5eBC{esyW8L3AAo{=qi%gR`K7SB&i268h2=+< znly(-&y7-=KGm;S1W{VYqdUKDP8Xf7uf~9ND>jwqFJ3@z8r?8Yc<-#e8Sv~*JSFz z&ETSgShGDX`97>RXd1*u)6mlXdVzCq4gMyGA?#nwZ2j7*T@LQR^b%%B<6<#pPzP)@ zYp*Xb>0A11AFJ9-L%%dVU1PU|1|y4>6}2z9sX1s`JOmP^4AFbDw_YsX_{^@joe5CU z?3k(!pq1?*MKf)3Rylkk`+@(l$}P&u{@qLf(rOd9%(Os*L^Xos+bb^(n zLGy?{PG6(?D@hS6?Mn|kWYa5i|AeBvez|(`8RM^)u37S}%G=t!EB3s-dQ;$5h^veW zY^wUYRCj&vW^?6odTQA|ZX8xXUg2sVI71DA!cHiM#2V8>i{P81NE^6FgblT>0hm{+ zt>wFz5$`DM3~?9cG`-pJ@#&OdnUZ4U4`8lsVnKg%%ta-woYEk%m3f#&78-1x?9KMQ zQ{_s>4{{Iq&L4Lwl)8S3o8iJDOl@6|$9z^>_)XpHRoj3SgPL9~HxE;A5do@7swp76|spWm3_TASm2;t>NyOuRocRr1&%6fP?S|2lugJJ89Uqa-5^K|9I&Q`FyO^lH= zDQ1g6>;1Q$U%R|%f{qvS!5O*iD^=EzOE;*EV=>ftt|ulib?q^mcV^A!`SFg#C83+8 zqzv-lvl!m;qS7_difO@?{q>p$Oe@ccsX~L^;Ql@YJnZ}R)MG;^70m^d_&~88;EhkZ zfb79b?p@KuUu`)d7tFm2am-PTamUhmUltXR zjP7LTY48|$U)*J?1P4k<*N>*C8n)YCc7Gx@(AxXV$J=_IKv6n_)&8QU>Ci`3(HN?(fv^1zQqk0SAJvzx<##mQizjNd)7c zcwf~%J-CZMeLpieB=Lj=S`>YE8(5U>o<&c!0v=+w>jM0xt@1n;JkKm3uM3fgm8(e> z4k~fC35#I-->5sOp;W$SOf?z6qcBaC!kb!`GhawOht4AmPTn!J+Xn4oX!34a^?XWW zeJ`ZmdXVl0sh@d#)_hrrn-Y^zF0}BaFvO&JreC<0ppBDT-nT-gd7j8k#{uYc5YN+%EIsq2=rrPUP-(urN$)$GTB@WTxcg`95J6dpu7*|#`58K8JV${ z(G=bMDqp%fMl6(rYWBSR=zHi(7A(uU2-F8mqy0T)!j#ynYLiZFh1Io1id;DW3)}t) zL)hO#SQCx-d^Q|;@yn`nCEPPBG`rD?~{#g(h@@0u<@)AC?_i$;)`R zadjQmqXRHBXLSTW(4>sct7Yfsk23Ue0=IT1ndbEzSKe0|F523V&)#1LQeLg^&&^6)jEGX==L)90}dT0x=}tkk$FJbE6U- z6H7bekErBTgc#|44sKB5;pr61)+VTx`yMuQfD9E*Rj=#~axRX~B0prfU5}IzEMrC| zkEGgk#rF*#QEnSULf4C67L^ty;FmTM|EYgZxFYGvoJgkFd9>&>0xhsmrNh}lj4a9^xO_8}t}xT<*Q=nF!9H6+ zpZ_HvdywIe%M&{5zXu^OEsnPczJ^wSkiJ3U9Y_j?g_HOLJo*m`ilPvlzR)rZVhuQ& zviAgB#>!u-EIRyu_LeAD2heJbjE8535@VzBn-_+LG6SSb7hulpn^_;13EzV{3;>%| z$i`8V3q&9MX^lUqt7C8}V2l2kQTG4Ygt<(NJy1UnAA;fExlB%PG5grK;b^&9dVsdO zeEI0GRw;bQsLySI^`kf$McP!zGW1w&M#1PcTc9XlV({=WiQ7iL>f^`2L)%J`4ky=K zZp`jqc;@^7V}iieOOH&u#0NOQhbptrTsyHN zh$94i56QmF+7isf2}tC*T^d}--O0Xb9G@{TIIQtH-oi1ywHr6XW{fqw0tk`=OAZOL zH{t=oQ(qF11DfXAHXnVz-C5wDq8ChKytDGX>_d zsR7eUZgYC6!S}lU>#{XT0xs6Fnl{2mIGwYLyI`WxbelHN)}Xpa`Qp_;mKw;A-=g@C z-i<jSk} zR^~^RWKlEc?0S|IMLeq@)x;FHI02W%+_tuKwNU_j?vKA9Q>Q5-_}KCv_)<}{S~&q9 zO{+vmaKOy$yO$lC)XSQ&5lUsJR@lG0RpVSduPFKThBE~y8)ht> zl4M!=VnDN4Jh+!nEN9bdJyYxjwt%|n-wvTIk*18PYiJON0Eqg&7It>9q1r5Mg8K#h zl%2Sy)#gJfJ>~U`1s4jQ)8^k`&-_POAOK@`iKp9g0TRt3V97f)0RBIP=NaSI9cQ z-@r{CPfgD!!tfF~DvMpRr-gsjh4+9rh~aDq3ryb!ibX|z{YjraJJfg24P>95o;X_0 zU-ja2<^zL1j?Xt&zN-yFM)qU7`^0}c@LG!>)d1w+F&d-+3n;F}!mBdMeh1mID-M^| zR4flR>Fy3E?O8XO+@*o()mvcV-K@dcwRl8hhT|)Kt2Cg+?w|B2<}sg4M^?MSj%HU* zyzXsXzL((q((rT7wQ^+p9Hfo04XY}9P0`P|*BB0JW9)FZe2Z;|UQI!c`0wa}<#|kp zDo}~KqM>?ao;@}#SZD#04za5rX}9am)ihYJFEi_d^r!K8Yz)Tf!>amdgscsAy+7KY zcYHJWruysQ4A6S+oCg92$M0OCA?>iYe<4G5u;x1S{pBrI?<4DN@rzwsd_^ipy@R%x z)rBkS#x%}cYCPb`!tnVj@|#r>_dt%0@goh&)AswgcdB;@@R}kSQ3f8Q7fk=!d>3ZU<-+z_tm+rFQh*Qx zKc^++9+5$waQTO6Cas~y^3fZCRp4HF*pVh{I~sk$CwGB#dqhC>-dHWWcGIoT7%=1a zn)}7FJcqrFn9FP=UYKSntZE~-0zYUjvdG?Y(VLQ_5V~=^4EZD+qg(HM5Ckb{wawlz zo!XOCLPBBdQ1t3eT*)~0J)Q_Y)v3J?n={MBvR?7aioTDMM+U;5VVl{~ zpL2=_2M1nTVFF62mVixvZV!xGhvaNqWk{`Fpq*+3g{;(ndCUWw67@cJ}QAz3j`}`Xjk``?T&SnqJ zxbLV*)#$O?@yd=P^NCGy9dqZa8MaONDGLj>;ndKR-mAQs-?dZ#h1}i zQ{}1mz~c#i3?2|SV6{r9SVqD|(l=&B5VeB|`;H*`(^XRk84F4)+cwUR1mSDJhbED0 zA(VKY7jXzZ&?Boy+&<&|tC5}luch`r#HQebo=OCR;+9P_;h3xsyd<(`sj1ecF|D1W zVj{|SeZ2sB_nyIT-`%d|D#k2{T(*B>Z7zWmqFgKeAIZxV`$-HtB( zFXyV%dTN)O1aI2rj`I7BQa7}l^4NCg140nrEK*BhR$(~lfcfw@GY=8)Vrc8A@m#US z=JpxKI%sHZPHKT|q%;M$+&wHm8kjy6!2sJ0TyNX|ojC79N_aGiLWfTEny}kZ233Jk zzJEA!-d5YfF=`MO`Ab!f%m-31GEV3rkK$^9vHUk~f4na<^-r7+Xo#dO3*{kk?zVX> zCU=icj68p$?QIR}sCSZOv zdW9rIGuYYCE!`bBIq=_>`QI9F(l+bprimJ7N@c6+ z$3>9I+HZWVW>3!H!5VvA%$uT)D9;2KiAcZN#Q|r zd*m9NAT8EgX`7?t*ZFO^W*R7IA29U~aQYXy0$Yp@-6JG|Z7N$If2$9Fk3ixYr%7tb z!4zBM@0O%lr@@(dtmw`)l8g^vTBz3VQ0ph793$_|7u>hE2qxe0zTSI|@f`_AyBa!++4G9*RNFh5M`>pKnp}4sR(mE>gq4y-sMv;3&(m6@ zJ+PKnz3q>;K-=zef{NfwK2OX&pw76V&^-LYF;(`i7H?){aC;w6%vbGlaNE zt$3sRVmE%O@_X3M2lCu`aCsz=LcC-yZ&98H>kkMPmFU2*8O|LwxdQs@pRJW|5bcHf zx%hKSU$N?y75P0C>#>79UOFa$dr@O7iL3KuJMiOD?QD$0dpd10#QfYbbbU=xK8JIG z)f5k63rQQCu|62V|4sD22MzpWAN1SAyu7&B`>?3MxQduvlOi!hLRy*-eUdOVE4wOm zr$mEIt6@IYl$K~RvTsm&w15&!n@unjKbp+9B?ta4Z!t7%FL(_d6KnRmGS9=0YJA~c0&i`f6{0R}+Ph4JJ5k>yBT zbbE&L8XJ3nOFi=O>;~1mw}Bz^xE-%Rhm`oM6|LZ%VvlJ=x+!McH_BFn4G3OiHx?8|VT8blXA|jxIF)JW}1j_$$QvMr` zo`--`FPHy6W|0N}HWZZ0a1Pn8T8gG<>W?OgtWh-T4bp*ZKUt&bjIoXQ9!P@#qi2Oz zIbgO&I4K@p0D>V{y)qOg$1-vkw!uPdY~u^|eYn9Af2`Cx3z#Q7GQlsf4ng06&K$GU zq1y%IIAUkiCi{+valUm}j2|vzjn+-IxHFfuX`Vzg`;~Sbx`jVx=4fbhy?$GqxUW~; z%1*(PcoNBk@^8!i@0Wc@%G)hs3E{NWI=iJQC2`TV78C3!Gek)zF%*HdU-rtMs9HK$ zjdh&+iWO7%)q6YrnxNZRqEs#IEE~G=_*R^1g zKOs%Nr#jDxkyrmJw$%I1* zc)cO%nShcYL5ue>!)t}hr@~8cE??>uEghlA{>vimq(v688kwrBw-=s z786-4s#{a8ibly<9ueXbxB5j6ZhM3diFVT$MS9sh2nOwG-l+u@jI=fFXK3icF zqHVLEfhWazLsW)3nrkpB;qGe0YWu`ww1vPWbA){b;cH3k*IBO#xFMKVvyo@2L&K=v zxtCu^047(2=xo||fCgYI?Ml}<`a`QBc>?ncdD;kbJgZ>^ySPpS%aB^z?!J|86~DQL zTp0F!Y4OTnv`3>i(=a8@H^m~XHxT#4s^6)TQrLfQB8lBbcs;z}a(ZX}>Y?qdcgxTu z_?b9>+S1NWzusP7N~){p7m+M;ehBLl!Y@KNSm}?GpdS1CkPd>F$7v}o^KQn}H%WQ< zp{On-u&*fDoLKs&0$m5U@2NfVM09vR-vnb)h&cZy_qfU*@l89a7ZEw4|NL^nQla)?vQEdgs#D2TCqn>9F|&r9`vBUf4$r`BI)zKXJrIQBgBi zWhkEgV*w8lhd#@xMP_2`q|I#kYPHwag-LMxqw(2D{E>iz2gZ-l-mny*xu*dnkum)i zUcb%;Som05SLY5j{UmfGP>S{VC`)zRhwr_dXkMtZAMh0#?E>9ClNoF54hq@w(&AK$oH)B?R_$1~xW!T=-}M z7|JQ7MP9�I3Ie*vGT%?<2MVt$2Apqf8b$IZlc}t(!NdQIIaI$-(3Luhf1NKO*Yl zde$~3ZXy5r$9A%os1Z_=gQ1;RLcyZEX2sLY7^oUJ5+pyjN`q;#?cwfW&@_$TDdbbO zl^_oCIysDkVIsJzyidBxZxT_ewnKL!G6Z4Wc?+sNCUeI8(%EoCB>2`ewmEiP`ue)v zB8GH9mqWERiCt%(mp4}lqlx@^cb(b^-~UcJ+#i4Kc;D-GBe*~@4S0Nu@`qGq_LKd> z{cx9R?~DCIN$Y`YVYI#CQ*~bg)iSiB5zi3eFT7^$>)Y#==i_e5s=>CAopp5C(&+Pc=u&l-4P(_l=tgwGo zzLHvh;|;s&u|;6L0InTNIT1yDe3B$-S5|D-aNy<6youTYH@)=xb0Z=iJYu~=aUQ&d z&~IuzE2N9AvG9#R@83h6u8_zf9uj>`{FNf+lJ|j^22Hf`^#3-A*BdEY&p6?#Mbofj z2k@J+AfWItb6Y>{5>1;aZ&#Gp=IOpRhCvi?6D_%i4p1WgJ}}U9R<^oKN~B{~z2x zhXV{5dhOnU<~NrcA`WMkAB6=f_&P}Cxj0FjvlE9cbgUDa8Xd&bnNliW_ zmAbDq95H@a-EUK1r;K1B*0?_Ja^bJgm> z`%qC9X$j{+m|XG&pYAqqkQ%oVoj8;!sLM;9GuAP6z9IKB)-{rC?4*ZY1Wp`!N6+W? z+6_VYF|Syw)GL!Q0P?8*#`015?{X91$h(hlfBsnW(0wB@LC0xI3#(0U$I?r?5SJC9 zmnavI_tAi;i?ilg7$gEvNGUr2$s4!KlvX>Kzl{yS8`;lhG~RXI ztTdNlWPse!VpJQmn6<64iBJjsZdU63K)~J_3>(d-%d_X3^=~;;zTR3qE{QLu(9>NO z0g~`Q4GH!bDAU(MY%k7Yjcq`dqMMJPdBpA{$$P6eY4$71yEG-5l z*l+f)aoI1wo^!(jHdeH<aX*#0C{-C>O|HJ$StEq`kA*O}h7jr3rU!RWgt6t2y{{ej z))EsBrO?u{LFJ_XtD!FdQ#1q#mgI|d_@p-bv8xcZTt!)57j;gP!`!yj@>C+z*7VVI z-u@rTKku2m#xjImKD)T^;4qmx7KR!m(hhRVlF2zrlA5bvZ1o;#9sgd0in$*r1Zh55&-xP_nhH}AIa1fBQgsYd6Wci{Z# zYR!&#N;Jwsmn5R96n>0*HX1(N%%rr!X>2aM`Am0UEw7$Yn(8JV0ctzaffYiLr`4O;=p^C-=KbNBuK`w>) z<-$qGs{CAztbxW_-FkDeHYH#D`Xh4`@3Vx`QeaGQ`+l3-RIrH&-X*e>WrFPtK9jR? z`B3>cGDquOsUeNh70-*qnH%Aaki+(Kq zDYE1m23|E6Rff(U^}P$v>s3sBI&T@(6)@Qr7ql{7H(S`AN*4m&Pr{}fAW^T^x^Mj! zua;p*-A14{MNE%W2$SB&!6UL z0-eGntT7O9XHmCx!Ru;wB!mAh&0`hJZ^}R!DdIBvxHHu3?c=jInkM3L|3ly53}pd9 z29sQ7eBt`fPdtXY(gk8p!`4|+50iuE=cW3kBF-2(D%GCw7&NQyhlhu-o?4lXPoAaZ8vE4^LezJJ3TOj~2OELA;0SSK=LKqET7r9l ziZzMsn=%0q5xOhFu^cDU<5;1e^Fu+*=C7RaZ(sSpmq3OipiK%J@!oCBcz0`b zDR-5$T7H_`kj~}S65jJi28~~ptJ68lK=CdS1c{+|!HSYiMZUg0aM|7S;h=cg7GTaW z3n%lqFAY@M_EuWp>FdYONRpBiyp4uvu3lC@n?AC@8qd;?T)L!(f*L!voXVBIGr?OoqgZ0A zEly9>W)Uv?UJL2+2^FsZ3uI+>@`N0ZJsf_YdL0~^+qOpLDh#Yp;--{Un_&s`z}8U> zL}anzZ23-7ECZp@@OUX>=)op|L%ZGGY~O<2puzQiS>4>2k&}z8&eiB8iwX!j8YC!L zed!XRM(Xa)s=EK>i{$JDS_#cd;V-=a!@L%q_MJV@04Br5>MYj1N+$;g3ym&&cUOB! z)>>AK)wAUWt=^ptL96NE`XVomyTBM4fyHc2Fl{B@L;JGDrEiIbSe^S*JdpirXLU?i z*j}-%i`s@8;-cfp+-_!Ie)>`HxfJ>_ft1>L5CQQJ4I`K=adYJNYq(kES;;)nb z6SjO@Sf3(n`(B8YEy#%k;=Q(d5r(kB&t^#nO@`xmJ zuj(D@9=AY`9FEQ)2U!bwKIr=vMmn#K8AUV#_Gm%f*xh4SSu_jx)~XqZ`mmaMfzK{^k3ENmBRoEEp4vNG(& zd~E+RM8~vZr5Jd~u<=Tb8$SQ8;m$?o&(+Gb0bO@O&1Wu3+8-Q*Fhh3+{!}sZ``ipG zk_jlJy`xJw5OTR5dJ~MbV;6giIUKjqh2Lcsmso$ymEqDS4ibp62~&a7$pPdgy>Ea! z%pWtXF*?mAJw3D&y;$|35^Gp53MMR198WBrxNDo`IRo8t;f2CFi2jkARlTyaz_Y-y z)%$iH3SV?I#=bDAC@!G{7bi^K;7SGQPy(r7HB|wtSr^b#JMZkFp4b}jem$=1%wI@$ zDjl>D%wGGl@q|?>6SBR%Q1WuW#_nP(yppl@d|!b0(+Z9M#d#vMVZ?o{;e0^%<&`+^ z$C+6rUFb0M<$UyLA-rTF6Zu=GL{s~b@zqXk1dQgnwK-xlV%v8Z^S_MYhj`Fgp(FVh zISM3+Zyl9tm5Z_EF6wv1d`)b~XpCHqs}-I;Sv&+7am`<| zPo!($e_y)d1sCJTMZh`#=~n&{?Ud*^e;T(G8f(n#YGhe8IXwUV1p^@Uxk!FOKZ;Yo zMXXR^0}@#Vqd26K{7Y0%;pih%68U|-JsxVqQ`iQQKgIJ#ne+GXe0V)RtUYB2Q}2SO zIBE6(>8R?qo6l^ilBzhgKWPxh!$Uqand<5;wJEvlocmh{`%GRDvECmHiSHk_)%rkX z-D-1t-_I?$n1Cc6b_SB1!o*{lYqy63$v$W+kxRkgYPw6x8{9E`aS>x&rX$nk) z(b3TAgW3UK5)D@Nsu&C?jKXybZ(TOPsul zp=xqmJPWSSFzS`NTN9Qi10#C6P00Yaz=?-^)iF>lhsL6j3tBT#9K54qZN{$+~5x z{!;D`-i{ebeLg%dy(rPN^FB=r7qB%?)C>F@d;ASQYOm4=!&$1{z`C^wXSCj_2$>D! zd8e=-g_*sm9w31}l4X7rW%7wG*9BMeV;EIZ%b&<9uow@fe;b1RzR%6w9D?pW z;M3n{KY!4C`Ni>0-1ji+NVD9N8HX@>DJoD^b?w&SrQk)|#S$OzF!RjeezG9D$PM0f z#sehcHxH=W%_C=d80h(|$uLHuo7XOqnxo(|*CT4b55uM&e+_$wp?7F~Kg#oBs_;Bn&=^hATT!Z+$s7JImKWkDv;6R?BGD*p$XPGB z_Se;pnwEAto-Ka|VAeKrtBw8J8*QT|yq8DOs+8aO|Lf9atbKk6A-S@PB+GxGCE<;GAgr!l1n~1ST@2MOK*LM0t z5ns6%sKixUAs!UNplI*gRnTdxjiDnQPcL$Nsp7NOOE&OR4%BfuRFC7v)N!y(T5(n9 zcDcgewlQ;D;eZHH94o+tAavvW%x+@0*nJKkOhsDA&ft&Uqv*{H<30%1ME&rW33NP%?IQM8Ym&2qL!nJ#Cg2SUStNfuE zutwakGzqK}QUvBA)fJ~;4S5~Sua6yHtq2mu`iule^t<`1X9nyNsa4TLKH^g2{tY&a z4&lUN#&-@54(bXj*8wUVQUH7UpbE(>f!h0t=`DT?Y#bYlM?EY$tAkdG(B9=K7I_Wt z<$hz1G8bRN1&0g^^w+^gf70(45|CjVrC^z>a{ZK^t(@Yk=}#=CRc1i^O9y%SE zVXrYR88|qxy}K zdlsA=)qZm@B@h$}Q}U_n7qxzfaV6V&ah>9%k^Zq!tvYpJYIzBH`;&EEA8o&qUc2Q? zvY`*r^-^RIEUZwbC<^Eka{IR4aOy^l`KYhWvKqZI8RsS_DOI6F_`lO2cI)|i?N-WsZyhz3~>|R#%(-Sr;|M z&L4#x{q9Z?sv8lW__Z!cE;(1>PHcrhP0g^d)+7bgQzze8OdZdW$a(UQ9r(LBx*+K z+>>*1KO{BO`Ns^AW~Agm)Z4&))%shwj8qM=ZWk89Uqrs&OUm9V2q7I9C10;LXB=7N zij#fD_mhsA!zvjbRf@-R73Tk3A5L3Jp0G3@)UMM#;`a*k;?oKqqIl(q&08{65+t0I zC;#&FWO3WlQFGFopAXj>6;JX3-lKg2vG!>2u78@uZYJ5>M4rxl`RoSef2szE=`17v z=Qmc7-YDWlLSCz}Xf%FK{q=*N6KPR|YCnVMGEBWroFsFQq`o2VkProGvK>(MIgt9B z#utrcl^KWdll2!x2jyVPFSR;ErjxJ#ZDKU6lvgLWRTbh5t`{{T;l0Kvlr}&7bVoJS zQU)>fP6kGF8mS0XEb0+lft$0-S#}-MWM+|?) z*HMK0a@tWWQYzk7(2__Dh5K!!Mcv&TnKv~NjGheoV zgX7d@Dx6HtLKMqCr%i!m$h6MiuF_}{D435^1M{o9>63JsF1Oy_Eq;qCS_L0a54DEe zkop*3+;`kPydA|)dzkVmrsFVryp6i%4Vp-?U+wH>d)ds-etJyCXETW8d0%O>x1N;5 zg+FHK82@cjOW1$#{tF+Ne6{f~_2lJeo*l&uyXERLc1Zi!VlUu~AQ zjSqE*--yRdv133mAc0so0yY~NwgN7HVw-BL`r(HVsui*S85ykhcgK8d?#^mQm625z zIO43uUX?k)D9R?F#NZ|I?`^qj4J1?L)ywt-{l1{XVdH)joQ3AN#|5;}+^&_!Bp&qTGIZl1fTZO1zo&c}IwRC} zlC>ek7;vz$e;nxC!{%etX+J;RQYUf6a8Z*=!+JAp3R*6<^KE#`*7ODH%~ij6t>nt+j~v+w?(9#? zk%2-!bx;U}G)2gWWa_?b=E7g-j**&~kdDDOba-BncZ546ea}N8PHoSELZL-9cw=SO zpLk*08kD`8^X@Jwznm);@!!K(5Ez%lhEX3>*7*}jnRlU($e@sJ? zXC#V2q1H#Sq(f`_k5 zVt~iE5+p1~v&Hv4KK0gA*8HtwAewmL5srjUu`<+O-^e2Ve=UHJF?e81NOkB?EVt6XUZBZlu*-A~a(s!Ly}p*YzN!(WUL_?J z*Y?_$K`<&rzIzh)YYw>#jl^)DbYt}EHEpawJ9xdbFAA0UY*wrpe5&~kKTqe<#lC8y z_R_$ch7C8!+$7mD&q(V*!Fl*7UuRj+5GzqAL6yd;y%~sG2DoGmGHv1I(Ox5AYXnnx z7+*ydZE(c_HD{0+e-h3oo)8K%3sBlu)h#?$9Dj-4tu?eXY1Gjl@=6vMJjCMKp7?$N zFreLPo-`iN2N38`x z;FXR$N%6W#;4#3sZzOfPr9}`mhZLLABo7A>tXFv)4i?ABE^pFoU*AjNvIf9d4>yJ=MOJ~iUg8Zrf4b?m01#tnASM`lf z|BUpSN+%YfI&OgGWbIqlIM5l?Wq}aPD^`gi4yw0b z7}U_OZ>-Wc_E;&{H?VM{R3ez31nE^}LBKlt_5l_J{5SAx=fK z5CzKMrpgIK5W{c~F$_cfWVD%rpMTWj03ZoO*oo_%v%2O930F{((Z(gUgOfo|tb|ht zqQtRA;t6TSGOcWW`DlZ}3en6xFjXpQ$PecUNkbdp*klG@?;1XNrSEJB&s{x3PFg<) zuva@u>{-^B2|X-bK&y>6>?vCQWxHF+P4n?9jLuk_RW+dFwlx ze~zI3T?BOZ2WO@Va;n}{UHV!q)d=!(40ykTYJ6(^4eQMcB)S4~AkKRJ9+Z}qMM<8c1LgU%iX{Z|}kDE*tW4xg=e z#t#eWf^8I{Rmuu*iP}w$Tla|y{j;;qOSUhEH7aa;Jcx(ERcU}xD@ze+sS@tptgl^Z zbbme!xQi%I%rpvk+@prv94#iJ!;mgkT0X7QqS8b8un>>F%-(oE-_0m!q2cRB6RDQ| zvbsN%dA{;}u~Wk?xV*k5TLA{{?oE>!qv`K=3|<@X``C}Kj89WmMX?xg`d$R1W79_t zqhA6_t~-BfnR8hNzIxZKsjXR6eX#@=lP&~)=QJBWg-g0o&O$E|k}+oM2* zaxNJ$%KdI>yKdO7u3>dw{%ginIus%6W)hLy;Jt5-LsuaB0pC))LtaC}Vw`nYuHvp{ zZ?aHxBV|-x2nNY}&nbr`dfw8#{r$$}`7eWK=ykQVB=5Y~r&nt%N6!@nrX5Qv9J)4v zyZYJT-bRbag3Fewn%mA%yEE(WA{S!bGX18*;Wo%MIX|amSK}2^a#qL0Z}Pq@_wypl z4P#XDtU8Bb=-mZ=KB6bY6Y|P2Q~NL<`1to%)*;bZp3Y}oUMFQqCCT1B%(XxJy4*k* z-XchkB5uHmxh->}PjM(kDoxalCH&JnX9xa?NN_5bI-=s7mlr zmUL;c*kF=@lw^t0uh@zy7|CRN-!uH~C74jnrK@>Yo&#^5VK{xr?PN2Itl+9M5g!6} zsG4$SE@A%9B(0JCr@ez>p5dSHm(u`bh|?}qeSQ5Fw-Yi25A^E1d5vOydiX-b=4OGG zW&u(R$O0HonEUh4Ws&oFH9T6=kVCe&)o3YVf7^4*tS91m@H);`w#gSP6}ouq_V(Ja1?{cO*>4HeJB zTgoSs`NRFRzaI>}j2p*P*L-Jp3pM{`E^pJqrp7eNO#6s3OFi588xdR({i_q=#9o8# zH3mx7jlVM*pD=XNN(anEsTL&UMHi`8|89tj*Ln>ju|k?8)<5Z<@)~_QWcO(5AgAPX z|Dd~8Si7>^@rw)tNQ2Dke@^69x5+nf0}zydS>y$|rIXt+r>f?bvf*L89`&|>_FY-i zt9XJAX>0e!dFKNjq=Cv0d(`Q-S%ZV)eKewHJ|Ml zU3SVu`v-9>2})DmBWK9=&wXxBUWTpV=mJra2y(Wpj~bGh^)~6O6j#zk%=?*_@C~ zfy2tf$JcQ?rNYYEXn%XUR-ev?tY>IwsHS$Y1T8e2Yo3=uk5^J<)mhx_yEtoHbZ)xa z-(s>9s)CH;M{GY@thCD5dhltM=|1&_*xk>Q&{~P&`GvpYi?IC{d33qORrOT=f33)l z@+Qpwz3U8roFAegA(@6HsOke{FL}z`?GR>m)0CId!*O(J-MCOgZ%qB#t5Y3Z zU{Miyc!*lPyfN>pK0yG9Dpp!G#;5m+e*_5=n1sKrM^E`y_oJ4!#3H6iTE`m3@35-d zFssA8Gl#uWAaiiun52ziwE@9Das12yo5=S(&zz0V(ETZ1C6Z%Yq-Fk98tg`T!-|jH zLZnU!)b@63^(kHL!R_Svk_$+4{MlG;DK~77(c(dqLVBnX`$?SgE-}>FIDp3|X4Ka*~}p~G{-yBCUa-1;wPyXts6R;aj@(B*1lwWm#u zs@2D{U5wlNhe-Y?%Bri)DT10K?rUS*kefn9YB3f51^YqnWX5mm8yTq9|85j~z1~zL zf>B)T?3LED<+40vw3~XAT*%BE+B)!lxC0pXUj`Xj$@*zf%|tnV-=1LuQ=Fck9&BCA zYBx-$Q;pk_KD7w8tll}!8a_EMVj6nweLJcM$)eQ6!;rPcm}=8cZB6C8$l+4@LS&>D zR8ydGe0((Ik>m3F4=iwxUI@xmUDN-(rX6FXTFHT8rkMGAb~gSZpmcU-UJ#@W%jbN+ zl0X|WXcTa;8)Bw>aNN8FZ}ij{R$h|#%v8peD*lrC3s0-OPOp9Px~F+YMY#UYg#VNG zQ(JqoH3{Fdwp(yYJ_(5@9{l5Z~zO>^1-@6Q{fc>?!y4nE{L8h`z)ES3kPjc~bJ8+p{J}IE} zQcwlYoI)TiLM2+I^N#16mKbf#>ZKL4Y9IoXobAevhLbY4m4~fy-`3->(QL0vLY>TO z_l3xi0wn?0mOSs5Gbdu+_pzfN7e|Rv@mPI!JNw++ymobxb5aN^CG|p&KkR*_Gn-B^^ss)}cGsw5G-|M<#w`*<#k?1-qma^JeQY0S` z@A9cryPEuqYPdM@vJbU?IzMBSy&N|Q?%26V{;6tC>@Wy!4(sQ<-shxd2{*0!S9}MG zJ1vbWfg6xNWu$g<;wT43B>nlTu~ukizZiKlNsk0YZrv~XuA+tN1G-z2kD8pDf>Mx^ z2>etDH{T8pQ1%KEO1NY}94W^`{B@d?;e`^0ppaSRRINTStcy!W! zp?Q&CQu-$FLuVksD;Uk$o`^B|4ufjVJRlAm>WTwh0BEAhBwDkJ%XQW}6+GeNFH|6yIvI=N%*E{3PL(?{alM)29oAf|IE&JTrN^c%aoF zi{GgAGk_~OTPfEkWK(BHY`+a~dSXgq1J}6bootosy@%{Cy~7PIY`-s-PUk7nvYVT- zZ#lXr7F^joQo^PUV|n|Tkit3z?HvN^u(a%78yEP1Aty)wU);8|IuT5t)x49AyXY*C zSzPi$BzQ^x9cLi7ae>}X=3c)tkv(U~qzjIr9w+G3r2h@TvIAMuF}$kzh$c!%1xiEr z!wL^YDxC{R4JD}jfmRC;fIwKPB!1mwp9nb=7&aNqwJe_^4BLNCFE?F0MGwS%s~+b7 zX{+}7#7p7K&Pm1wg=x=&J_M7L6Q~!xBB&u zura*)H0P*}sbDJd&rcruG-8q)LX#m{RJNkIpF zMai%GuFi!U^e4xgslEjC86|l4=PR&#=T~s)Pchtz1OX0X?e?{uz@qbOj*;|yW5g5Q zcLtLZ=s3L(RmVDZJwM-=UR7;yr0}ezZwZeGUeORk_5x3%>s0ygI zE1l`%M|IF@Gg|diciWV1tv+vS#K5al`UHKDe%j;O{JZzM8vGu1h+{$-v-|%^RDq9u zYAzYLQIT5p&Xn3*2`9gt;(4VLq<~hR#G=vkk$uyJHc!A$x{^Sn*_+A8IFhs6=yBKt zt4_vVNUL7B;2X7C)f<-`%T~K<=-*%OMn>l%(|uxZSfA(djp>Al_^Wrd1odgup)J`y z23axbVuhSM>y7D$2wG|3D2;ZqvQ_CD>s6MS8kPf`|DvX~#Q<&14zuGg;BE<6#^jat zDd}SMP4HdOlLuFXf|nTZ)SBQ>gQJFhh2(ISEFJ69a}q&}fOobUhNVbi5a`rVW! z;|1sbzjAC-}|Tb`83EonbQ{dk`=FqZ$EfUnQ!X~>aWqpn`dpR!Ldl$T8SsQbToryV`C z{o2K`pTGC!*9WcPIrR+@RL4fuzu!IfZMSr|SgiR?SWYhw9S2_Pzc(9rQoGT(Fs=L6 z6Js7zJw=qNqs%zS>g0&s!M|jmTrhDAN5jfZQkv(dWEKI@+c$msY2aflu|~(0;NLoX znq-z{j1WO%wLBSc)T87_e}>CEZMT1BDJr+btup;8ntz{*8>5;QVZhOaW}TyywC}i^ zG_&fU=Ig97jubc9zD4_iR42tt^}6^I0q3t+;fP7H63iWR@-Nr_|Jqds-L&}4K74!? z#mdmU!{bT$ZgCc!ykZ8u#YA+0X_SLTF(85<;MO!p0e`yJcYbNAO+Uvq4v zR){}-148q_lQ+e;7WI|a3~dCAIV)VWNA5I@w_eGg5bK8F0B!fjEiGqCefziB1~g38 zvqeR6*EcuaQ?n_W#Tn|UOOZ8XS^M{~f#!(Wav9aW9A@rr z4*5j$@DzUMca1kQ1^<8Q2b;(PH#qtqSxg8;eC}`yEV}({c)icMnt;-i*0DUJHI1=4 z(3i3TrlZ?p090Wv1owts9-+uRD2f!oe^mN3?QCI?z8kZ_)v`(h{lb&Pk;_cXW2n>6 z{KiBgoF6x$m(R$h<5(Ic(rju(wfoL@`$s=J_j#cnPdQ5VSj<~e*SHhLVUK)pYAz)j zZcm7Vv`o`)9_iD!V1|PN-f~0Y8$nF(yQUDh?^qfJOL3g@aW#8n&kg^b=)~d+RbXZ$ ztR@!{fYJrO;_Ae!GcgOMTAGJgf7JHGX&3EhSe90?$CaQ|w_!=-v8E%SQ(y)4!hA{% z3q?*!#l?C1w1k;eQc(f)Lx9Dw9{+@}(OAE!XuDqp)qmExQzi{%B=z(uISqf+NH&9z zz>QO5%F{})v(78=CuSW);+_^Fe5FOv&wrKfb9>_xBLeg(kZfqE7_H%TGpf4?(|U)4 z^I`m$J9})D=WUZE$mkXvIi%nc-2blk-z{i|dGpnNWaR4V3V`YWu3L*Yz0JwPPnJt6 z35$V3N9|#5gK52L^A6Qq!695hybyuC`S$?IcL43+?Bar@VX9~ZpHJp#aeWj8rd}pO zu9y*L*J%IwMj#B)vQEB8jFSPoG@QlbkOR`Px4xIh9d7mBnM8`vQ~R6yDXv8fJU{0) zyFVbloIGU_;}9fym9(bb?1zZ%MYGq$eL7iZN%wgmmDh>RZIcw^eIrHTd6HPzS9upa z-Xtojzc%y+70+})r^Dw+?I5WiCnVBKcoho0p3#!2(6chLW_dfoPDpsXFL(N#b|?mt zm_HXMgRdw%wf6PFtEbp*GAW}>MWoH4mufU8$fPqo@&r=xJiF@R>eg!X3&S7DxcUViw$3BgCy2(Flah5GZsF^-fdEDC>oy$<* zEMJ=6xz#3Y{z@ZLWE~8#fp1muLo3)Du3{QT=!%+y05pENVn*#72Fvma{;+ z&nn>^@#gjCnEW}|czPg1a~Z3n_SNQ|@8x{4)LbHl67dIl7T--Gezr)9@68I(Ir|o- zC%DWtA-aga_FT4Tj=Q*R5bc#z!KrEx?(yDQE(0EKOzyu-@y%;RMjF1<;?E*aYsZ+= zWKAv`+bnwL-Pw=pfBNl6MI33O8!365-+lPt@~+slQla6LWn?BO%;pkZ&l_(bpj3Ix z@KL*L)yG~1(7#JKhUR|4`9y_^FB)_7bU7W3@cy%`zc_K)K^aSx_Ti0PI}dw||BUkc zJ_&J)3b)oN!%Y3BCaa-Fr zsgO-~b$YtDFzT0kKFn+XMiQp4sT#d8dB1QzDghi4Kx(2YR96z5 zXS}Sw9&$&WEtib*rcCI=*gozEiXu;$wPp(oe{$1ab15KYnuk;S^8xx z@e3wlivbSjwM{Kul|A!790D^(iTH|c2?3=}uyITs(yMVf0&yf(M01ALgB@iOp@c0- zSzF3J4okYx$b|)6w-E@l>bQ|PyB z&d>5EWf4X`pe=$SX#JT^40}UBKHy%RFzw*Qtu;x#P z<|4qdKaiSeVttguAagxZR)?x8d?u*1*b6Vr@@Zzp#dK=h=@6)NJHaPZOOaLpX1OhW zdph4T$GxqaUBy(q`mw#RXv>5aV>!r?Wf=zdY{eGX?Yb z?f$0Wf*tBg6@G~Iy%2efE<#rG^z_6yS@yoa9DLu8Uep`Sx(Z~L^%-re+wh~vA1`a# z2qj8QBJjo`#v6TM*nL~mu!tRVPN%z*ej5wl0E^*#w=ulGPuya=x^%PlpjF=BDGQjx z`)!x#j;95YmzTq3^B7W@div{^L1rf_U4S?NGaSjhe?sh5 z>n;57QXYHZ<{I@A6E(i2j0w2xXY}<+gI=K}8MP~JrAz^>(YCAjhNas4_^`)OvJ8%) z7F$z6C0fjHnV95g<9r`)%f*jXHw~W!a8rF-BW+htI=Z|E|U@k+dzya7fhS1rIiPEI0idP2j-L41xy#-EjU!iksRC&A{ zbi9ZV{GM7lD{1h_9=sRGE>^fB6o{gB^f!S)db1vsY(N0kX*!x+OgN8FR=k}mN)`UH zk+!8q_f3Jub1dV2e{+&LOB?~3P@HC-t*ayCQL1U+B(c$d;=7ct$b=S3ifWc#HeeEg ze>ae^@M1)xgD<>|Y0?=W*LmIa!|XcXUG!Ll6GZE257W(u(_^frW}BH5{Q8?t>hb*r z`HMpXMj?@~9wJZ$2=CW?-}3n-%2(dB85LCqh9!r2Onyuo=~vn>DjSy9+lJO3Q@&en zSrY^s0bh^oLht=eKt<~N0C@KPpyT-fC~$cL9pKbTUDVUeI^{&S|Ipa)DTI^=UcrppLM*Kwarz-)6baDLXXM!*e4z@TTVL|7!uz zRVoCX|NH@RD+Maq_rHr1N57$)mvhT~7_HrxR2k1wdD)fa@LhYHU3I^Ba`S;aZ6EpR zDbaj~<5#?TElwinRa(Tv1-U+C0cT;pI=Rdzexttn`|^1#)2DhTwg@{L+)$gBxAA8$ zJ<+7kZl(3^GK?BPIYc!n^Ujy?XzMa*R)CvK$DV~+z=EDyEQXH@C19VslGQ7^vixB! zNtTuS{Rr2=lG-2n=%!xmr$@B&y71xY0eNQ5wO~QUs=3Z$G$ItWps6t7^7usp>I_zs|!X`uC}eg zX+!{s_pYnM$zLoK)HX7s zh21*}Jg3s?tfx~)!qk%UgNlm>&)mIU+S}%xt%iQ5(GF{cQe5@=OAt;eHdhrHz3jAb zQzu>t2nyy^#kEIaP`u&PR5b&i`O%0yv}ccovlHuDXDVJxQ^%eLa>Gd|W(z;n%^Dv4 ztXKy6!V>oPN}%Pldn?$6nbFBfb{==1FL{sdPce1Inb_D)_V?rJ1^-kB>int)bGlA7 zCP4*LkGBcvyWd-+f`3SP;e9HusOa!8GgNN~Q`i4>{X+-!HL=^}rdaBisAA=Ob$+JG z-}c^)Q|jAcL$w6)Ih@(+ekg`MQ&{r+RJxRZP_Ajcw#@(BKxoHkNQwWQ9Rd-1Ou0Y& zdLI48RrrQpRc&p+WE)}PQG+zBBxwd)4!%aPzU_e*ZVcww%oR9@Uy zbb!WC9rW35rP1*|fn(A4#Z|>lI#vOTVk#zl$%4DtYSLsU+S_6G)ZZ_2_$srC16zyL zcYS;GpeHR9(xQ9vdqp;stnl*p@WHmBfKw|E4Hgq8+KgIoON4^`=coi?qjDp5ri zqVmx(_pdW^b%wL(SKlt3!OFDHW#-F@Mop;d#tj@C9X2{W1REPA_huV(JU`A)FSqRx zD+vLXFGH7SYhepD?rV8{{V~)}d#1s8Bn_bK4E=~NR7UlqoDa91yBsN%Gg|Kd*^tnvZIoH`C9JMb@X|WTqRR?#_M_8V&RzZ-}#W0uvlw8a~uv!mKio5 z&8l&KE(Ir#J752>JWRkX(KK%+@&Q`FI@CWIMkY4X#>(Kch`62?Fm0-p;CY4>_&%|{ zyh~IRC?}4=p4%GMtQd3VX?v=8;n)N3u=Omb2to$W)e6gzNcl6AT`eFm%)#q!MQf#I{l!mRUI6e%0fiPc%8R%o1m7H45IPM>Ya@BPZQh^mP2 z_*d63NWpBj%7g>|N@!FUF|u90SF$G!**_@o7BvZ^48~KfF6@Nm!`B6F%zqLPZC6AA1fxob=&N+MZm&V5R}FN$WT?9RmXcygK4zy|0?`NGjll zgS6}nK{}$U7FhP0P0>nC=4OamJ@msVgR;X9B%Ea59( zZgvo&>~}d_=CkXwjusXuc?a(Bx(kQ(^_o@4fBOazYvtNzuF_~etu9JFzvlTGQ#hj6 z_^QD!{+f~+E@$9X)hCwkqC_+;Z{#c&%a4$h%@Yol3-o7kM`$>#p^;Z5$tnaQ9H@d+ z8YU{#TP0HCS$rblmxy*R%_U`WVLyX43^d82`?Gz`x)q3L&7~5MKg##X4;!ub{@v>m zWj{(U$zMWVIGA$7;r2$pF(nHxRZ&y4&}a(yboE&Gi{LI)loA}oPyb#L@Lr(qs#e&D zEk?#Q%oNB{rX7}N!a+t7)a5W*x<#HPI5o+@YJRt9gxBPj&YSg8Faj zWTReVTI{~)cTV?#P%)3*#!~zA^mKfcOiKltmS#^#GoQ_Z4QjJv%L*zEgjPZ(#M4{ex;ONl5|leUK@!!wgxSaW9HeGWrlOD247P^W-v2za|97drK`0~ zHCMn)0=Z7_(KEQv#?|Fl*c1$W!a0rKvQ!zFlV8&=0oYpDQOf$cA3s(fu$V%MWGxceUT1&Q z1&dpV-`zT}b==EuBF*$VzbO-Sy`kApgJS^m+^>(S^`q?ir+}n}ZR#4_+V^8U{ znJw?n5Q7wAV@N_3Ln=wwmTNZJ+_H6BtS%0GuS|OQC<-0orP+@PckVLF{-c0_Bu}li_s(sJ5=u!oJNMsyyUlSYc-0SL9^OAX0y2xU z`jB-U!$U<(ga<5wk*c$_zwM+VKuOeBWa{c;r%QQk+J4H z=-!VY!gNc{MGgoZG_D3=S%YuY3L}BoYQ$Y2^^4uXK5i-Qe3pJWCC-oFK#IjUt^(yf zE9+5rAD1hF#iXD+d{%sH?G4SNOk$VCY~XY6NjXK43IAT6&KxWl>W_TqQA$nyCJl}Q zN2h(``tnk-v>2L1%hR80D<>!CCi5^_!uLGvdmkMKYi34&y$~^e1=Z*G`V(c>QP0k-gOR%*tD|u=q0dh3F&zM8wyQwWr`nM(OxqjYOblX zyh(=mKH`gbJkOM!wAbiEe2NPfhYN=LA}s}1QggT^HfsVM-Qr;H$^Qk&5~A$i0~Wxu z@jnlN2}P;rLl1P_xkW-Iq83IFNA5=MTmO$cS;g#O-D2t$pwC{2Z zZ~QPGSFZa?-V#x_qP%84wrR?;Z9@>vd-WqMG4OQzWueD=G+Ymd8 zfs2B8dICMaZId=d$e*NHXQ{KNJBj@K3Ldl9=m*+j20%0%)Vs&Wmvg%HR^5Bk=I;iQ z`Z8?U6wr*3-~zuFmO%Ou1%0pMXEjCYe8!KxCj?@*-p+6pDDuxwO*!6{07H9(GKcI< zB$hLpfB|>qx2J6)o(oW{o@RUUkLr-*2{yw%jDbvnjVk)A-QB~(3?ayM-i(RJd{zIf zk!~b3p=HqMO}+`e=D52gm^K1W#6T#=_@U(_e#rI88DgJY?sX4zcdR|OG}nZi%H1`} z6sUap^2IDpV}q&20Lwy|7Ta7od$H2^v*uoUgH`^I>YtGNN2zgz%y&-Pz28cbN|_+U<%~&RXDHDLKRy1cO(Vqa*ECSmq5Xn$ z?omheN=zajU=14TU!4ckjRSbekga@LYpK_R{z}MYLgU!B&KXIc{jZUALsDm@BH*cv z9VuR836|2-(qMjg<8MmKuTTkVLB%cypDHCOI=^W#^=3$1lLBdxohwMp^(&N-wc(xd z*zjbjp_>B1p~7(U@Ei%+pU$JJtTuW+^p!1!V3K>E1??oWfgjKM$pOgQ`w8fN{xI+R zw(`3Cy>3iby%wK{HF*p;b&x_84fKsXD!3D&m7QcnoHOth>;wvvW+{*fTA7(|K8TNI z)oU*pfO59&*7U>HVZqtjrq?ZE9GiigNOTDo@;%U(|RM49b*Oy z1S(d4!b>vaftwuf8mmmS^YV*)cFOe!BiX)VkS`TZX9568gDR`clh3w#!hdmM$S(Jr zgP~}eW!hE8MdwLocrUNv5j~q95huJzFFFGC%z-OQzSjfvvsPe%^!;rX-74d& z>C|JvEslfTB88;uFmE}OG_euJ^u41DVQ;|DE-mh0#!cawe>o0wtmxH&dBrWJa7~L? z55C*Y$WJ281859MQJ_OX! zb(S)-`+?rj_}P0l-EoOoq6YkGwuhlbM9ZN}g{vBC(?eF8kmUUI(Mt$rgG-YCn?1&> z1!M9FG=+EysxkucR}`CRQKYoznG{i|J`XT2<89M#gS7*D z2)tmnJG}VWxB@#iYIC$NS1<=v=CRTb)vyx?;JF`DRZZ_Ug3tt{yrd*l* z4%D2_Z#jG)rhyq0LQi{H)F{`$^eL?}-FIbH;U;@4*V}E7o2(W`1#J!~b$$J(&HWj* z5_)75B4XmzUUr;wLXP}xM z?~6{&uE=A^%XEW$5V5b-(?0wKnG?(8?+-V}^&+NR?RcD6nxe;CL4>*vQb9Bbl!0^S zQ``26OmCt!qE;UtYfMOe4$2y;50#5j+g-QLi)P|&6rAfCj#eM$gNA1r>6Ltah9lJI zBCBX_oZWm})^DH3Zv>(gVr3lyoR05*VPIjFy3drtJK+PkW!qMhLB;Btwzd0wY{iy`8eU8&_xP)%Z+uII=K+Y9fT zN)YvB&0NI0F0bY)K3dqkAeOjg`5=+dy{Fy3Ys}jl`}vw^2t7;UOoQ}1^4K%2NqPS+ z_`L*KFMB&BoxKe~-%S`Lj@&E6!RuecN$Qc4_S`>ccE9t@FCIzCXUc}D)So;E#D7Yx z{~k6FmMAA!Ec)pjc6|JYPCoqH8%-JmvH?Km=F^<4p;@gW85@T}`Gqc1o~Dz8tq5s= z4w`X%%30*z+as!-7Ar?zo1HL{RB3#sN>?PuP#xwHV};skBiGx=-l`xu0%t+@BOVzl zV4%7di%$B}l$c{_>WpX;$ZIN94WU3)KhO|OE6aU%=AB193feIi{J8iHPvr2VLm3NZ z2~FPnwHgf!UWP-gLsK$=aN>RFeHSep>kSednH0Q(BJGbccCpS;RYD2vgcw-`=x<*! z88r0SJaxS>c6F(<(O(wg`4VpZ2n_#6 z)K>t-wRGL$65JuUySuwXaCi6M?i$=(g9UeYcP9aYI|O%zLEq%w`~9!#R1H;B4TtXD z(tEGHR`t(}7oQy_$qNk<>qwUXux8{)I8<^4@o${0Ucb%2gN$L$L`m&KEU83-N+HZL z#WBfh#3D(*w0>+vf2@6Mf3vjwbmbArT0R8$Q@lv-C@Z0Rw#SQD4j7vbM~fi%7D*Aw zyFDTZIAy@86>B?t=6CTPYm{kOHoAzEg&R(Q8I)(;lB8gp!u1X|m>%f;luXJHYWv z5lpu; zz%`_NNHS!K02#?U@S?8)ob1P2sHVfFl)IT+FC^`hhtr7G;O(uEAT~UyKXz)naxU`w zBKnOm-$`8(y1~CY#w8h%Si=j8Z6Z@=!4D7Xf{=lTO&u#@YfB+nj87!U+a_d>Lz~ck z8NQh!=LmmT7LgPs!z9}00dv9Zjy;^`I=3&y* zXTPka?E(0w?n)t}$Z@dr^6I~2EGUIW@-Td@zOM^>SQo3iODf|&5P&6cK1jS|VaMyp znb{n}bJ|OtgC=6)6WD0@l%mqXaJ_f)?Tz)H@}xL}7Nnd^4QG(PTKhFW&;d@@diL-M z+oDgkZ*2IjRhkF;dsh3OUH9#}D?irqfG(#6o?nj5gX!e2y_X287%&2l7oOki16h)g ziKkIoY{v!Dv7s!*jGe*E@7T}D3&H!YhU(|F&_dG3$v$Lyp0xdAlr)nez%(yiB2aE0 zb{-UNnq2?A9t=H!`;HXu74nA`=_MEx*;TncM3XRKW}O~@>FXaS z(0p6^E=J{|!k)EZIKqxsvm3*OC@6%&03+zPb$`80n0PDjG_-5Uxufvyofc#J0ed>N z^6y#_Ap?zG{K0>2Fw4%6eIT%_cPo$}7x!~mFR{37*|qDKG=nPzUfF@O(M?xc3wcgV z%rK@u-$M`F9V7e!Z^DFZJzpm>{tS#Ps<-gkaMBqpfvt(Y|Ndb4E+XkYIYcn>-+uKf zp-k{S2%#Gjcc(A=1I$q-lrC}xL$mS!-UlO0F#YhI9sc{Dz5WPN;Eh2hD_vFKN;)sv z-SC8nO4uG`n)l%+4P?>!Gn<`?`NRL{+UP~h=wzQh20j6HSYM9V^$nezmmNMFo*xMV zUJKiZPrD#*Xj<_ewTBFXRoFX5|60fpX#unVi>t@D3sPtwzqKseeagwM6uzy=?acBi z){vjSV?;q8Q;9Lk>?{#560)&Arjg;;wQfA_y(@B+e6R$J>xyDG z#T0E0g*;|baM z{`N-Bd&692BG6Y^`)B(k#1H$CUoXcC4v_Nt1Uk`RdORgfhZ&{$q(?i}=Q`xWgyRobL$dqLX*0xeTnbKm~>I?|&4{eNi zxriuaOG0SzyvnzAnS(i;z$SLS2aZenA_*Zdt$CK`Z$nBG35eYohDzfy!FqFaOo*>= z6P%h4HR0Me_o787Si8g_{cT8#KOxWbeD+VW z3LBdn*j4T|)eDF#FUJ`fKgRu9&^H_YFyvqR62Y(ePHnnO{qK8^qKt?)axpS92ZA6= zpdp7pRp{)&q?j0CZu>WqF$I}|ot>LrAp|V7|kH zlal=8nEhv?M8ugjW4lZTP^RSmD?YLGBB98RI-^3BmX?B$ZT`o3o{J~kPWnTL z?ZrxO`^j&tjtRs(kEOBwP%53- zcQ0zpDxrp9MVFhKD^K9|Fq~eC+abXFrjc5oMSXs0UCnp@c_GwytrNYM!TYG=c!eoo zy0G7>s;lknF^(R@^MCs1kEJ%E^UBjr4@~B_^;biexT_BuKxf?F<0g;^vph=Jaq2A8 zDiX{JD4jMd$WT*k!`ii&mXso^T*YPxG$Mu;=vU z-APaNs{zyTbdH|i9O}`rEg6*<=}M+{5VJwI2v|6DY8I|oiZ*Cu(u}=KD}6luM?~u* z5ezFD0&j=q=xaw))z~mozHp9)u?t7Fm(FUsR%@!%`Fh5zhr62>neEtKV8cTkw5yeo zP~hcw5_yZ~=?{hM*MqgCo+xuQSA4W1X5I4U56}p29=~gCX^-oBxAzqGOs@w}brFFh zjKnmNY&}kD+i$-m{O(x4$LnT%VMwc1pH!JD&;F}JUYm8p^6NWsD@fM2wY0`{YolWd zXRgELLAUKnpx3Fe#>ey0>$s{3M^}&kXga6M>!t`lgV}%v z3B2;Je76Ch1n23FqaF&=q#qVCb|c5c4Z5^fapbOVy(88#2iz*VZb!0X#P~O@S2?3| zn;qv9O=#dJMCEtfV!SdIh~p0ee+<+ATzWffs~P_n^KtvAZ+`_z=+f3eAhhM0zi`gK z6Xoi3oClWVR#Oo=q(e(=F#Qym`YSD(fo%GzTXCjbd?4!eGSrTNr zszSb7A^S!Ka6$cz$4owhq!Wgn6niY9vc5}|8thc5y@1(1!=BCZUF-Q(*W|=_Yyx6x1R6&(Xp_% z`O$sd7lI`Uu;a7$f1}@1Ez)TWi!~*?dfM(nJePKF$aP!Vo36meUfs7TA*NYQBHMTI zSv5diiNFTu0A$A+wJr`oK!a(SC`Ei@T#5TAA&6+VA|%{OeE+XuyES~l0JDV zrA!);$0iaEXzZ%u#!dj!>pG;?x57*}x@|NpY1auvCoLIKju`NJ^*ub*)Q=$Q0(&RkQE0{=Tr+ z4$Gz*BlLe~fF)MNdk1oCWb+!Nz8u5c`+IBaebuS}sZZKyT>JQIGu-#R940}90v_b6 znG!F*h5fc#1I0%QyR)@5{uB`P4fzB)OaoKgYL>$D{OVL>Y}VUv^7vV%d_j%M`wCYa zbe-r?`6wD}P-m|JgtM|_B9IU5>ijv;%x$SaiAImE%aeX6VNm9C`}>%yCS!-c#eFb- zbwQh9Z&+UiJ0@MCct%k%JVpfQiNhHdQWH*TPrb*tlrr`;;D}bXyiW}l3^kj;M4FJB z&wX+jL=@VYTU}!ZH_mEc`Zs*;4=nca=Hr^Udj>k(-`bS~lVil7{ftAPXHvNz;bv*; zf}WJ-n_NjMC?sP2Fh02YH|YTZMGk_QHAI0S-5{*!4EMJAQP<;Gm-KEA>|xbZRKq@9 zRV8(xk5h)}?#nyS^*G<)c{~X+N#T=NWH~*E!O>Ic_CA~*A%mf=$K8LVr~|||CQI>? zR!NdfMdX_+6S6n?G>+x0|Ci6GKP?qLKzzgC*3hT+jsbzk)NzW0`1v0TjT-cXU&@+J zwbnhb;@Gd5!XrT_mG>B$R`t{O=o!Eh23juDW#aGQh3}~fPVAy!gtSNja_6s{> zj^JCoZ-DYC5149&I#o&*hYdPbOCkC4;o(i@nTLl*Lpp@0>s#JtfnOy=9e0u(g>NKy zV?2?dAMjQ(2Eu469ho_8=1VS55==`I#{ACrj`!J1gMz2t0tG6y3d|}J5-kLTlAN8y7-@VP|tRfHfO~FaCMK2%0UsPY${0oZWY|%m+z~Y-F(;7s|px&3)AuKE5d} zW9ACs+2ae&Xx;I2Z#C!*Fk3Aev3d;5mMhU24H1zpRuRQCSJPrL)a`QGYOmaniil?k zaFEw+vrke!sakQgO&$huAV!SxX=cM47s`PBIXyiE4HR>TKWm;WRcUE4XL9#GdlmH( z&hK7?nN#8>Lo(;sTU*cTLK-VF5J+OPi#HW0wsHKT=KY)qDTCy^a%TBY{u>$`fm|Q3 zU7GmKP7k2?{7m}KvirsuY#%3*X;c!TjI_YVwRv|w3`plk7EFc~{A zbg23WxbZ9%4Z&K8b0OFAjt0FX$RJT_rWK(slb@N<(dM4M!(`j?vYOT17}C9|e7?5_ z-o)`_^!wf(Q4a8}8b{;1;_N>ZeGbb$oNk{Mp84IsmYDAjLMO~ruf2^a<*f6QFi^o+ zo3cQ))1YN?Mh@mM6*hkB6?htN<~9PJ2>ZG3Y5V&NYAa;(2>o1lKfSk!5@Cs?Sus(9 zvqY9B!C-a9Ma2pF{;IL>7iIp>juLVB5A6BU3UJ!1711$de)3LOh`Lm~RJZn(gvM?cfS&5Gi~Z zl#cMN>G7|vA!K~PsNJqzhvW6V+^CB`&xpr=`dkvF;p_U7nqRn4n^tB*7l?O~-c&Dp zrd|YSb96N7@|hW=q|euH6FvK0`#Acz-RpjLa)WVB?v+yMfR_bgK)!3(H(s(xxNPu)^g!_=&y)xd9KrkR@EeoOjZsvj zYv2j@u_L*YMO2043MaTi(luQ@`bl|i;U=5ejgFh- zWPbO!>_Slv)dD^Uk)rKa1S4Cr2r!jQa{*)S0e`E3w_ZIC&>H|kA?NP=uU_&&KL;|E zZwChHqozP%J}`@nppszE1GQoYxlPENkW8AWWgd*kZu*B%s=VaESujMh zdGs;jbiY=xUAH5u+;62)u>{8;KXVdA0~q6>QcA}yQ8C~rk*bVeXu=5-(j=M!V9Z%3 zilBE`#8r75au_j*pv?=j_m&=RD@bPlYsr0P7(qZ zR)3J5VWi#g8fAyWOsc%$+5Ij}{OI;V$Sq(;1#U^EI3JRAdjXD+bn+s<#z-JLaI12v zrVbAg{Cz0I5?qdL`InU;Fs4FnY_r{2r>S@XA~VaT9b=+y95c6>24y@6j>0d9gP}kU z!X-=n-855c;BOWII**PkpkktM*XOa0Y-hq+Wq9&ds ze*CN?WK_0S=O^a^2#cQ~zLVY!U1^vg1MmpA|9;W_K%p$ufdHsXPuW!Uas7DpS&u)n z1;U1u>PrpqBu99w&-LZ-?pQ||SDMJXd9O6yVO^};oDIJ0KL3YAB8AsLH}EF;&%)5y ztwYot0?yLOA|X!0$P$^@fn*j})EDpivum zIfBex09`pgRy z7J(=X32CMdfPR+xlin%INc)GEf8p`9#>k+222{QA3`OZOpOF^J>sJ-Hb{6JYYB-cY za@ME(#H^N%vwzn6+hEY=0Yp^_NRu)7sr{`tPp@yFN3hw! zRcde2a9mQ8djvsXyFn%%30Z|}&r`&fod~*fpXSMKJb8Y`{}w3|n@b!jn)*w2y~o9= zpwYQnH2gdX6P0nO~y1{Bm|!L@|7F=-_I& zk=<#IcqI5XU^b6;DtIz_tr*nxTbeV>6?|zB2Q3*~L@8VdXA?FYSP&ysAuUciCGzdD zs(=frH9c;t(q-(btDwu(Ms0zR>+VA_an)r$3&_*9rVB|n(M%~zN~J=duM=pmkz;?Z zb=#!XNXhwp=Eg;<6B-_Iz56tXR40JJArH+>qJ`F64=)PFPokA%ZYioem~k#^iFBf- zpM+?~2Vg!{fUKmPT=;fG4Zol> zpzLyONq0b(C9;rKYu5C(S%+D$t)WPumgDp3SAQU|Z03THo=dRLcp${{<52j)?ZG@; z5&P}U=&iY_&2t_0#n5XPn9cU^SHDb)H~O45g-WO+VDlGk3gg6txz$kK+MMua%IxXH zPY}=8>(S(5=8nQt-Wg~Lz{XaVl>yr7$SmRA@#CeB`ogn~>*JKW&-=M!1}n|1?d`}@ zr$BI5b46jZ-O~FZ)j7-ladG5s?D0y`GL1TKM2W$mb!VX=t#mQoQdLlEj_BidXP~5R zNEP3|hP)pUda?G5jW==a4kE~JdnH}aVy)WB(91+YN59-{DsRdcaOq%wqG2w~6}YfR zjjf2MhS#F*tG3C%-bT3?Or`>d_bo;uLY<(kI$4@1Yw?%U;p-}XQ=5Q};C_;w)#dNc zL6;d7NHh{vN7Wyz8>Pr)e5XuTn>YB65aHUgI!l#St$z0dt?FltSxO#MWGt*D5ungI z0NlkbL3@yp9f>VP&=TUTZDbpF+v;}tGkD`K1E7gkGP?13D3+&_BV0i<5bP=Yk{2s& zmLTwGExY>LUg-`<9$4Pzv#6RcMnU;QA*;n{;kNY9Y>Wb35T|c#+v#hYGM*_W<7kB8 z$=~h-=s~Hy4j?t^ehOIjMX?&a`EzFr>j>OTOJoC_tfoRF z?6YU>$p+b|Z3W_ZA&60`&3?pj?9Tmj`DA)^ylU_B9>V+6O0ein7+#7irPJ%iR9fCl zr=lz1VR44J`>AWY_qd7d&yODw%t9}@&8ijV($cVyx@Qt(7F{>l*{9rFUOjC!CV&E= zgtYV;{a$lb@AFL*A-REUbqzIiSI_`c-o3I0r#JA{Law3*Hs0Dg%>N@(beN<*vYl1$ z_ktIYyouP`d~|w$tXHAz=T{pFfz|l*(OY!ix6{(Op8CU!ZTmUvd|p={Z*Snczi*Fu zhGJ!L@laGc8X7y8ir0($z=Z1RFfIF-I?vb_fQ3H?jv15icWlg@L-^AO<#t9YKggjW%@r3gPpg*=yS!Sg8z@gva zN1NBn>-c4+5{dcZ)LOBH0}8VS#4OfnW%RYY&)MfABc;l!#_u*<&CvCtlsu%vY1`?l z0_;B@Gjt^cF(xkYBZf~efsgshaOlUjeP&C$8j16buD8gq_C$+m{XbI%sgL;B*N`}l zz&MPzySx}79Pt351Kf@eWN$p_*NVdg!;N1Sm`x*2X2{O~*T9ZkOW&q96z>-@DxFnO zBh$#QP%d?}Z*hP_T(4df8@8*CzAVoi+UEddXM3QC|+ z0HAOg8+gZ|BIAXy_svA*>BOeH4S3#(t)Rz_t5LeY`(%UJmj^Rf=biGJ7Uo1|dQy(+ z#>s|{C*e0S^p|?L%9hK;;2vLl=}&*+)Aped=0`Hq9!) z?CI$a$J-gwycVKtSgGo%EsBDb1ydbh`p>24fArmRf6R(TgBR!%r=io~#DA64M{{c8g;Y1VDK+w~bkv%GKhesr05#pw0?bm48@ zV&b{-IPg{X@E35CQE)J7#^^@Eb?k!K^G((2O%I1D!T+ID`iQC1`;EaH@8AWC?WbT5 zpY^}@me*_NzK6}XZoS7ua{?SgEdu{qD7Eb>@7*t+6$~osYkT;Uasjx8@aLWbSu$YN zU_nIZcWi%CRai_kD0Wf6BURsTz~M5^@@#uR4~n&e&vOI*WZ>S9e`+z52FsV>~g*YhqY7rIzT!>_=O1vtJ`MIR&94wm2A~ zwaq|sJa%JGmsW$5eb!-rNS!uDgTp#vjCpSgOSPAPkcHo&R$07vKxc_tu5?4^^N~80 zV9SrA`wRKu#A^G+#TIsR&Q^D&ocBm$RQ8i1Hyx5BneHxsJ+3SgYCfCEVRyUq_su@w zRcl{Q4|Nh5b}J=9(_&&PC8ga!uZS_c3xRY}5Uvkig-mp^?0)6(R)uh;Vj0}!#oF2` zeLBcZKgqo*;t4UYe4YmijZW@Lg!z_OhW}}gN^p2zFtVvy#PA>qEvE;HU1b|JTnO?(oUyKXUt+}f`DFU%+gooy*X`AsvLyB9ih`AtW?wxuoL=m?h9P(4i705{#(U!g`)(o%84LXCD6k;#Q|tdPX^14r?H8 z<+5o%^O?@FDyn$$sIWSHRrKmq6q57wTz)Xu_OB+I62sMz6yh{4qZ)If6RG$}I`Ilr zgm!b^E2I_j--m0+3o6~P=Q~5z%*4a-T9LgTX!T&xri7g^}Q= z1M*VxjSM_VZ7_8;wVw;CpB9b10S5ov9iFgJ#RpMR{jcXIDo+w>bXz{1vtwq;(T(;# zEz+;X^Ygd%ZDuQju<8L1CzN0mArYNxThn_s3A1tTFRVFp`CSqo>f)G0i?E{KTRe); zOk_uZw8k^~l zm-!d#+b5S7SsV_d2>9F)F#-#HB6#d-y&zc+Bz8EY3|h9g1-!vMkRX??w&dikglt`P zYNhMz!Rd3(%8^giUf4*jjcY{66QrsasA#WV8`?R0Ql zTmyq?D`1!x(Z`(STMFs(aZY)X1sT1Rb{@0^tuvypmIRD+CqXk^RCs(B;ewI-bx zL2JpawRy$B(c!Fq&kQdli>iJ@9esMK*qa?d~mqy7Drg2*DU;E9n|CAs7}8JxIn- zPt(;IbQug88oe$Da?Uy7dC7fTvLH9~iN2ysM;dWoh8?rdD|-|XFPGe4r| z=3XM$<QR6c9MwE@j=Db6(&VTBp z46rPBZSV8kYE;Cj+qzb@LD7)cV)zy_`-n^_#^xa{+Xf#}N0WqmDuakLTU*n+={|Pz zjbu`ij~o(NwK8`1_w#2xhJTu7uas!5@(N2*HyJz5h<{_{-7LRjFYxHVek|B4Q^xF^ z#a<&dTtwgay=3R$k_4$Z92|~792f({*N!sXt53=+3>Q%jzR-rp(1v5EMa`6vt#t^3 z5Xbzk@HnL4G6KV^K5jj=TG2;CMLCaXjY~OQ5~2%6qq5tM0bxpmYL{d${~fVo*EduO z1|FjM2@HXNijw5V4&;)x3E2gLRSr3Y6I6A$Xs(?`D~_B?Jc~|f>l;tQE2T+K8w2)K z1`K?pUr^RKl<*!Ztl!1tVN2Xpz7fk*WJJ;K?ydq~N8rc%GafrEFJD&xb}D@d7|c|f zUE4~{W_>-^bC=P_Br0vGh(h9yr+)V@psN)~_;dE^KueQa*~ z17L+;K`hAJe!2@vIrwBFP6{+`S1!wo#p#N} zsNG;cQ9S!itInW+vxzHB_G-H?aJ0IEHsQ>2?&l{3R2oQ9T8@DP{$VkK&_BuH2vr*k zFBu83Z7GYV#O9k$tEWv={U+B{)Yu9@xOr=lA*09x5@Y^Av%mmR$tUio*!~W7Uu>? zYx-v7b=W^u*R$l&Rrtl?hZ7&JlnSJ^BuZ-?UZY~dmrhYh4oO%19fe+LFdrI^oKpq>`QHQB-8%RHG_Yg zYt7!TKQhMr?*sVTfgpg4RC2`*->CEn61nO@x*C(75R&c2Jczwu;MD7v67oj7)9v*x zc`?y=)a7QI)x}ziyGB~{{QLF51HRTuwQX*jaczNYfuQH}9r+lW5Tc|czaAIY*X>q| zja$H92dx~G48ERBuTdin!@EBDfp zI;B*E@YBy1#ce^t?+kq_UJd^}NUZ1N+1Er#TjLo;ft(vNW938C@Cs1QY ziICR~go5WF+ZUq{Z@_S{UArAzj$^Mfo7jC9?hX7i*^}SH+}-Xzle~8Nsdxhmq$u=) zu&h*4X$bL>5(O&Pz2PY6ZD2a=VU*lG;1!qF$BTn|~dLyuZKx1eqlzfQ1;d5ysF7AEc%dt4W z#}#mNYMUb2oNaZXV#A3a;v|yN^g5GBwGmqu~eg0D$dBUElG+ zf)Wa`(77*Lu2aC>@lcn?#nWh2osAe-(jVI{?p)jl8vHJuu#ydlByAhrn^y=tFP`5% ztA{1@FhPSXR%?8Evkco^0RtNmFP?l~bNSVU%|1HQ=yjR7+w`B>JExsn@oryS1ni3kHqnWZ*X%AA6XZW10RYF1BNdV2iFMOWLE z4{Hih_s`dX@Sg>p0-vIX&BMaeaD$3*qbCz(q~KF=!`-H1HLu}hrG|Xn4x9ETE*}&X z@jZ#0;K=i76OyeZ8##P>b`9FxaSFbcY&mmT!m{7#a$ zd*41bJJ++_Y+EnZ9&T$H%an*ZKE88e$GEov5aSU8KgNWyCCz{ZdiU!_udhd&kLGU6 z2?6)TbU!_xhgqHX@VTn>ZZpo&jz?(ErVEFp6(FZB&7rH_pcrw9(n+~3oAyx7AF3O>%t!eZ{)oyNrcSZ^S^$% zo$sT-`tY@0{q8g$2>E!Q`(R^X`J*;`fHXEIFdj8>i%FV1Ja0_Kiq1s9j*6itcO z)NH6UmZ4(Z^Ocp3UFOr&V9%BjDXDCXn1El~FO}Y<#2NPUI?ot-Tj2;LP@LK@wt#z^jhCC)H>T%KtkK5$UGd8H~CjJVv#H%EGjoSSV$K^ ztTpb)T4IW(N+R)L3ARa;w|deqEFoSfVX}!##P;M?Qrdr$8*K)#J&m-qo3-Slc3NKo z_TWg8h{#DGtpF9o6^@AU=lHSxy1sWGPood%0>96_Lf~ej56->8yF5d?8&G0Kj!KS7 zZPO`Vyn+_J53%3_FIZ4-?#Ldt$OQ$jU4W} zu{|tB#T@1W;OozXEhYJu{mxKdt{0Hp5(NMG!vkQWchhvouxI-%g*YH^thEMkwVf6n z&8XMPQNSB;dp%|o%|eD+Ls*jvvRqGqW(>;l+E6WV-$M`Kdh1r-M->I`&cK#5z|lZa z^`Qn7T3?5*99CNZmS@BShkUvm!AQKHySQsW)F;;q&C3jZ`>(SwGJc=-hSrxA`)`tgfB~^>C;^*HNgG_-0t&fbNAFko91nBx1cdevqND zxAK(1od?ksGf7r~hgo4jV2w-gM_2BN2;io`2hornth0QH?fB#PDL)1cPK1j1NgfVr zpKKzzM*|+WgcnI<$XcYG`bT6YmLIg*q%9&Tm6tOS*!K~H+t?*INS4SJFBQagr+zR$ zq+&2sjP6fF@L$1EH?T^gw1*@VKSwv?LcmSH0i!X3=bN8&5d|-)CoXgTNrXmBf%GO2 z1#2h6Jjl@)@L2Sz1^RNY&#^_SmdcD=97Qd#vys9>Kgh<h8b7B@IpBU){|$~1mMik-oTk|iM%3s;kf7@8NW ztgN(sI9+O4a<18Y@2v$0^SGV=W~J6lF{;6#QSQA~c2EX>q<`g2HAf+vB_tFb-lwWs z%~EnM5U0dh#;+w*9)L%};N^ij<^%)d0FteW|1sZ?^&yD=Ek#DBZVE-qa!W|NX%P%@ zjDypp89+v9cT5A6|qaF*fHcEi%YD1Qt!% zE4Asr1o@B|OXX0Xw~U~YEEb&zPnJbS21|j7H|MDCJgjMlqx+T`k!=(}|G^yIFNBat zPG&4l=eb9qj(QXl>DNBcu}e0H3I2VT7JLYrQseL^jcYNVZR8RCI_W#rhz#1>-8gi&SIg3kp2z1lSGf1 z3Mrl)d~OmHrcW$NK|uw_Niz?mYfMfO;l6!`vH2&GLXeMz`)djtkTdcSj~~}M&HoUXz2>3;#3L`AI`&)hf!6W z`tv}4n^c-c54jPujhM99snuZ`5sRVU>KJ#|XUx713Qc8RRQ&dWc`4)C^gh$CA4*au zVL$3%8oF_EOOH?Z(;64>I5P*RaPKFZ8ks01N=)6(En|f*Y?4D^A)2 zZD>^FWLw_-X(GIR_2{{sUzw59yeYTJNL;WN2 zCVM^w=d4e<5-z~8#=}bsFiK&bvT_uzzL^h&J6IUxmtcpVTn;~te!2yVrf0d=a}eD^ z(r{HCCqY&-!YAo|CVf2E|t5232`CK!`0 zuJ(AUKXoPMlwtj#uBN9?89UtF&~I*jm7k1qn6&iA!^cQ4h8>qTT2ZuCG4i(UBCtOVuSx7>86^UA(v{m&`gN-^iEmgOs?#)>$$^q$;%ifi zKK46?G-=V3{NRDKY96HoVNruvI2ic_3L7{? zRji)f(jJ=RBFk1GNd=0EXWG_i5Q%qvFo|)fGw9RcjZ(H~DsnNgiimOolhwx}QO6IN z9m~Hdmj1UmTH&hKp-{IkC!_4{L(bMpgcxCBW21ttyn_P<5)}IoI+({$iE%ZWPRw%t zLr{k_$}Dl|H}MU-7l{cDW&?epklP{5@Sj%w?zZsA#4zn)5HG!Av#Jlc@e#l!jL{ap z`zgin<9iq~G4bM1I_a-Ug!LOyYdQ8NswCdT@s9p=vtb3NDN+)1!aJT&K; z)aIhhn@=a{Co+{e?(Wc~&TqwlPIRgQ`UHzKZkFaSLq!OV+K%9|+;fj2Llirz5J zwV^y|MPe4%>6)sylArZD1v`Exl^*IezLhS!Q%xF0oE$KFK6eYIZJ9O*uv7i6y>HJ` zptO|3cwIi0YK=P5I+PO5->8NkA0uG0pX2bbIJ9HvZ8N^ae3velGGLoaiM>{LbG5dt zxoyEnio^>vTWty6i%HSTZ~I%t0QqVGH7d6N!X`|`Eom+{fiVgbfv?A_YU{iPjrc(N zaE~o`A+gCo7#54Na|)^O<1-0R5_E)xBeZ!Hjb6i84-*k8hUtr%!C=BxV+w(x{x!pN z;xZM}7^nm{iMQNjfej$j|1j^}6VuJyR^;`c$nbOG%^x`m_>}dFx`0<9_-pD3KP0HB ztuhO5p3gn9)!#-hnFg^A|8--dcz{F?o)nldRznJpRM|vIA;dvZbh=CRKYA@!G zOJ@Q$p9tlR#jkz*|$)gG*c<5>E}<&0~0nGdT_V3I-DxacW8c$How-&46QI<{y@SyIWU zj!2~6pL)NS=bFhEIy!dr)DASEf(`3tqMqlf@3YgR!O6~7f>`2cc(QtA`gxhj)A(hq zeA0h51f<>9!;E%k$>E2K1Jl6BMwA3rL}}4=3#%z`dRd+o>Z~SKr|9%`+ zi5Nqx?sUh{#bieipZ{!p4DcvD#azW!QzxY6n>jLk{1Hd^gI9ir0>`bN$@t80Zv{6(ri<)|-@}miiloeTW zE-Dn12y2j-JUy65s)`S2Kn6>|=jfNLlH}c-UL497YO-;{g2AAE@EsYLAW~^KT|aa< zT5{ynCZCzf&v`bzPKN5uBvkIGqY~h}ZOhdci>Xl_(h?Cm&0QW#(Q+$YZa3XZ3lKes z%TiQa2uBcvPZUyUBorkibiZaAv;(~LfWPJt{iYF$cO`k?_YgHcw_{jYej?Vv=p>4U z*14tz*XFg;s)^*!nwr7MG)0Z1Yb|koH_vk+I_&uN4xPN@z@m2F&}u^}A^~lSQQjKV zcxsTVCwlUuMvva>Y5GJ>h{M{}pO!_6yt!3HmHtO9-9Lu%pS@@^%n{XARRmmi9E-Rd zU-rBY?AArjZK$yj94o_-p~b+7e$RxHKrQb7M8!gdhfXRDriBOw18*au9T|lwGD?k} zIHe4XBnp*Mfd@r@Jje^7~fSyWJ-dfYeur9pr}O zLnJx)qoUrO+3u7Q9i6c{3R2iq{L7{BxfrD}S@8y?e8JDqi+C>~0uh_U$gsDT(EV9x z3FWZn5VoY~t3bfj#=@5H`w4(Nl50APFk`ql-h)_0(>%4yN6}2O@@zExrxcmMvic{o z2lUtoaC{^P#1wSWn&ycyEcH}ws2E(8?6fZ1i#4y#m*?;~JTxjS0tD>bXU1ewGD;os zYi*WjeFeQ>wh}^V!rx7HX8X~g{JExP|M=>iG>?_ej{hj63_Bi4bx_!6k?0Vbs2UA# zvtA~&9Jp(pY#U8HnYry_{F@S$ec%A)5iPN(`l6kumPt)h6RV;&2WPINU*WT=UuMGO zc1DaD&sUq>ic3Wrfk(#c3LH{sM-N8m&Kb;^7KT43?daN2a#2s8Va`vCwZI9$>BX#< zNcC_q8sZsro^y$8;|7Qc+on_S1t+Ben!)QH6(g~U!8~42x9S$4y!(H*@4wM}WZ={5 zt7{D4Vsikj|Cm`H9v$q5sc67+>rt6C&+MR16)TypOh zZXVT!gCpB}KyQ8bp0b~f0m`DGDT=YgQ)n{gA{ZP0NfLo?o`5;PwL1M25AJtUiqraa zb1!7T(#@^;;XPEXEWy(g@Qj(JA21m{$fCT;GkZ}cLW5w`;HiDK8 zMu0*_BOGs=zl*>q1~DaNFmnleT?Zkhst#hdk3~mH8Cz+>$U!tw;lrN8)8E3Ihe44P zIR@P2G@V(!zXrZ8-Zc$4$ZBIG8n8HJjwd!*{2WEH!%rX#0F@(>`dRUmcA$QU=W%}A zco84iD0>Wu40uu)$i8iF^+5MuEW1kS7ZmB=_Mk%SmSz~*uXVD1LJQ2TMnwG`FG+Dc zkjL(kGWo2Da90278Zc?X>>CWx5CH}kTe7<-6S7ZJ@Gc@!g5^gRMbcusOAzMPrSC3R zfn(@rqWAL%4sXQLi(}R2W8g$>tjos-X#|rtB2fe$wu%CV&ex;csXjlI8v@pw#_mlE zLedyL1!1=cm#)nTJr;fymx^Fq6Z#nL@R43d=gRSK1pU9WlU@gxZGJS}xwW-*S_y&( z;Lt@vh1SuwaG#=*!Xv*Qs;8LIe+6TT%rz2SbWPm4*42%Hu;(BcY+a!ck|;R#e7V zf-%^Xcdut%yXB5cqVBXhc?xI^-zx{MND~vV2~}GDa@_2Gr*`zex+wT4s$o8fA}2|_ zN(}M``@Hk25p-{dFYl*)#3?`5wAKQ`!36~`_3{ylRA6+CB1}_>a;tM(z_;*BbW1MQ2vLav! zBFDH6OQ6cwknakpxD%;_W<;lbtpES0ddr|ToTy(IC`F1EDeh3LKykMM#a)YQad&rj zcPQ@eZpE#*yAvQdgg|)ne`fA`pZg)hO!9%8Tll#xC^ z+bD|=S9~xJ9t?YtSo#2?uVwIl6Pj0CIs1=C5IMXsZu9{<2%}^^s09=2>#2E4G4oMhs$RI{NlXc$NJnv0cqIhX5*n??0OfspSg|H27oir6u%(t4mnCEX3d;Oe`>15tJ}Y4xRuRJ= z{9%jC-ys4`eR&Bw`d9xm`fWRYtLzgvxFs+j#(BQUDN+8sdv;2JZ;+vTnu+@2lp%$} zXgo{`hvBRI^$7`SBr!&Wpk4I02>opfzNC);8tDI@oBwl!`4rGyAEQ}AH@2<0x&IVH z%ZWaJ51@(0d%X|0zrTN&&E^I+6^+j7bq_8GcGTqHy5Gu;CTKVIcDJNN`uuy0B}!+? zdmkY4cfK4EjHi^#vTwD(hgDY}S*S?NAo|s3d9Y`Khd;&F2(@d4GBsn-(ba}2KM4h- zGHUjA`Wv_k9G9swwb}VRp6%_$oFpRAQ%H!{mag=I&I@1G4SMhFfEGXLTRay!nX;E> z%PUCEf#BD_iIXe#;tbY)Pz$-tfm-2L$VwQjmkO0&B$>Nv^vUdJQ`FF)zCd}} zwWluOSE29YoKUT>&BkT&+q1K1GU7fML#Q~GFnFzR?k4j?@KBk8kK56@JVulhV|}_p zOIxdFZ@rhx+4@7S4tGzpxwoP>xCOu@q|Q*%YNNZ+>^T&ZwO$aIH>*h8<8$D)i^k_Y zlamh_5>{JbAtsjR$mjQd)ODVyPY96zGo7{gs3#vATV<}M*%gHV_EyZ{aXeZ_`(lPv z3#k3oXgt*S{?vOa%MFCwK9L1k40SH?x7lVY>u@jbxF6p~%4*g|B?K|x;(5H#)dwG5 ztnQ*@w>@*f8SbQ&XxeRB%WJ-r(*b^?9GK7N+FlYE_$(2OS?z3J3g(h%)@|LFB;nX@ zwcAaWKCoK@8ofO3z645`KIZVgpO*>O&NqkS*rRL_KLBV|>A=n2mR`HB4cr|6eD@c}kE`5+PWu(P!&smpo{v76 z{$NI3wlU(4mKp~p7H4>3uXucPn_b9kg+L!2{qKr2S8DYkyB19^Q?&wRS`(c_5V@aXz1>m_Dz(UdZiNW;R>vrj{ zqnhf%@1RU!doCEH&Q5O_4vt01!mGiKCW6umOXs^(YXX0jk6-e~aBUvYi6l#{7&068 zGDH&^zuM7!50RptUtL{=o=J9-@?xx|)m`NgaE|JsgoNk9t`FzS@*7#%_74~|F+^*v zCg5BhcdD-A=Tor9CnctIbI&QixEZi6*pw#^N`-Gj*Bp|C0E>O}q4KVJ($2kaOy z-R%z`+&gP#At}qK*w7Em-CIbF6xN|BCm@?Yn-OJ2I$2qd_Il2@e{T@=*0?axFt)G8 zP3@_3rAl8A5l&|8$g#vY+~?-y*bXpatF=4p+uAZyG6mgdc6YC@rGhNOjN+mkc>qh^ zm*+=K`DD@*%lTM=2p*;$*NZnM!KIB24L7OjPZ($}Tef5b(antFlz&Gw&lgM5I04+X zdQImevDSGi^78VCty^$;jsFJ=z?moD_$qobmNlYY*W=`1VR5SAPG~4u<($skf|jFG zZ*g-a$e)sCLYwtbn;sV)J0N5Dj_~#H8Yl$O5~+gXEZ)znhTeKG);oQ0sji`iZM^TT zXppg3-Ys3H+1<;^lO0qR;eHgu`wNSn(=`zyv1UH6KdxatLz3ehg==zC5kq~DxY=Pg z{}=TR$oqpq9Ly_J8RN+QO6#5_9C5x}wIBlXBlDq2D3W(ls`|mX`bwR_L&`c7C>2;^ zJ?O>5A;9ysYBGHTVdlhCLQ+xps}YvL0DRz|kLV~kV0aTL`_1~j2VePg7TxE21xacw z&xig;UNm;hoR$oJEhj=8aZhxJ;*d`3AFk?+$rE*MVL3*eoTPL#vd_<{j7nU$ob*t9 z8q^f|G3~>_&ZuK2TB>qlX=!PCD|F!8zSX@`4fV9~|MVgid^s(OhZ30O-*VnqF1C63 zo}m=@Ve=^=kL&)SDs!PX6LyJxf^j=?3oGp|?~~=(c)x8>x%KdR3!?tp^6Boog}wbO zIp5OI)Bnn>p}A5XvP}#Jx50LUZIcFkfv~T`>CZ_f7)DLz_vJ~%%)j-}-?)AzH1(uN z(i>4?!)GvM%Sy?!!ku%7aZAk&=);JIM8b$O7^^R(V%Q6^jRd5O|Ju&LNmSDmf0aAm zU)q4a%h&0&T|M6v&9eCQ8vUN`3Kv>$|Gn0o^@$Q|u)n|GVTX#Yb7!r|0dg^XcTTli zY-*xpv(w>xP-zB)Ty0(D`58!TMLi!+XG4vs=6xR3ek{2>323)GZ!o3_`h!w>nq*xYP$jxjB=n*25stBsEHgigX_zRr z9mVv9ADOZ#8%}HZF*dbq?vAJNX8hc4rpAkFFVFm%t@Z$n=`1~4&g?9TJ122*lJc_W z#<+5`gDl)D~ zTNL~U-!~>ECMphqvgS^dL>-fU_qE+9=V7owh?l^5L6!R{X#Kprswgo^q!n*AXKE6v zFIU2(w<7&-ZrKmA*`Zw7;o8TfU$9(j;0}3>HRTJ|Rr*XhzaNiEd$rvI81VeJ)#tD` zSQ7okt+}Nv1yhnyXi;3&lyaEpObe{z&5yOBKq=k&?v}Fwm zT@qJDF1t;Sy^FbS(pug7mx&`;d)FO0iYiyYY+j!ko8_lLFHBKP@i+}rDITm?E>6y^ zPPe16QL;Fd8mC+}s}KE@U;mX`y7a9|d&;#j%A7v-gDP! zvNWOAPF%UXGZ*S|cvNGc4(H$EPDG0>pq$f!3irp3TMaj!*t>I_?7VrCY(f3t^*QRc zX@tMHwk%)P)LdR(3RxBz|CrPI*6F(N5?vBn z!heOP83kq8Opac+17z@{b5~cF{dOms6Mp-elQ*~&4Xp{l&&Q{b%gf{-`^zYr9Mbmw zxRdmZjUN}f=>MwtKAS?w>pH}WJkiYkT}oOgk==&AL>yiTPNmpHd1cn4DDuAJxQfL_ z+aELm$|cfl-s;(0TWI^%Bqw1kWBuEFFNuc5Zs)tZn{QYlI;f(yD~p=WZTwD_A#Gal zzD!ui%KD>vkc+&ls{rXv&q=6Ymz$Rw>^@ydRizcNN?w38``PO+&xL@9lnlBs4d%Q9 zax+XSqt6r(KfRXr3CwHT%q_w&KhBAfC*8%Whi!J;rN1r+zwyh%Irtj@^WR?~mtfl^ z=k(!&8$oYqKH9^$b9^rO29n`aHngqQ?JB9QU7r=lo3-|MKXgncU2m{GsOD{YD`u~% zEd}||MpGSi{S>v>>Mvv>Cr=(nrI~M-U?6y>j>dSxPRu&5#A>X$P-oz|mJi}-LG9Su z?(vy^sniP1mkHlIAV|p3QkzfSQ)VHJ|C~WxMNPRDwzxQQkcz8G+WPUNlwdDyc_!6* zc}Y#fv%*gm$Ff*0B)w5`FBIs+^NW#WXS<7@!8eEy3n4O1PCTr)$Nkqy-7H3ZvRa84 z$6;tP6I0zu5VQ#s6BG5-3<-9?3)O=on-Kdp_d1JBHQX90Qx%iZ(^!Kam9>$hjEuG0 zeY1t3ILQ;FpFr??^g-#Y=0X}Ue=-_71zGAgV2}#wo2+hP5{XDG7jc16%J@xjpd3r)ld<$#b%cu^v@~(g0je4M*Ti6b$!3BCBI8)LcS6NZh zU3Ri2c;{s%YKf60SJ?S!pa_+Lp$foo(BSLscCpq%8zEnS6INL%qWzV}3~GA)j_J-j|pFs+oS4~ZkmXhl`l%}ZO~mhaQNy}z@^qN=wWFzYhTvmUAd zE>6?o4D4;ag2ZhrN5E!e#uM{9fsZ-!brCwQ7YOosZ4NaS;*hKQ3C}(@88!(ta#Q|LDtx>VQAkEqp7AN-RX3B3>1FV4K#XQIFN+ec_7XOe$957z;1Li`G0Om z41Psec&jdSdpbvy7ZeyB-I_rmiFAfg?i?B#8hIuZ@AX)-p^)3dQ;=&D^A0T)?o8rl zv9qkyX6ZKyQcdFz8N(u^U_J}yT3T9W_IZpCmdGLr4V2q_ho1}4TIAH1iB6Y_%jt14 zIm<$`z0}hq5rkpu#dFmysc3Cok1f9|n{5`hu?CQy5-AW;&ir&aKHTl^0`_jC-y1+S zfGZlqlozq+Xk{uDzt{l?;4W`4B9u+l^%4%26g1!vr)Nd?{v=>q#X2B9ro+|8HBt2G zXJnf@Agf1bh3=*$BO->`K9-sME%mk`fme`%TF74D97l4}E|N$+mAnO(Q3f21}p&We|{ty?;0x zBTuK->+amFqBm}P09LUhYy|F>jrzblQ7Qxp9$g?e^bZWpzXP;7O1SS4qCdptWRh)m znk#$xoQ}@r4Jd<$#?=_qg{^;*Z$ST_q_3DL|L2VC0(kAfG?uJ%!V4mWuL6bp0rc@L zk5Psm796Zuy4@Z|yB<{{lXz1K@%L!7KhYA&0Zb?&NI#G+fTDbe>CW8fKmO3wQ&kX} z-=|*6Rz4?ESx!g3UpE0E;rYHRsl|+E41V{waWtHKXHtCEJ<~g+R}m5Ljq44buD-5^ zx2Z&=lw$uiD>+?Vb7!|GaId>sT)a^+;J+4;UddDcoc|^!UH5y$Ml{V=kLTwn_fKf|P)E^wG}kALhxp$al*S+aL&yE! zc@C;9x_sZh49*QVQ7Hnyp*@_f{+vu#d~3u;YU6i z>|fdWrK669iwp3dg`ni*iBNU}@Qe;#HjALe<-JcPZ<7#V1&{ZmI5pOdZ~kA+e^57& zirVUSZ&0;SN$Mk%Wkp#`H8eCda>vbW+XYR|kR%xy8AIhTA)NI`G)dd`@Aq>=!XAL# zfe(G)q5R~EMA_+S7NB-IH2K`J44-GOtYkLmlU5kU*!Eyw+PNtchHQUV1lE5GiX1z2 zMgcI87*NT>(FysVmt*R$HwN_+>h2Y%Gi5jyxh~Rvq@zG;RESUUh6$VF{_b>o-O#4L z1@$`dxV5sr8;%yuw%@cfud%MGs@3hwWb@<8)#tDufJ*#J@=N+2>~yZY?B|iBTy!lt zeV~SOy}KHMVruI2T5h4~3c)OXs;dRA3M$r+uVBxH2GrEiZ)mqX-_dx5%q2OXAS30o za*A_UY5%{A_QcgfiRe`+b-k@9TqvZzad)tTeZ_w3W?a5Z#dA%+Z|Q3sb~F1lYWYMK z0ZKZvY{hdFZWW=XkpJ_6V$vC^+AcT~MG2PDf>NbJ6)rTTANgDNk53&3*b5blP_MDx z4s%t?L@UqLCX4a?2WvuM@WsNM%TK9jqzc-WGzRFE;*-k4{x95dq)$Ju-XV`HwQ5nhWmmCNS_Z|P< zun1!qlTcPmAU6_D2XL|Z-|6pBRV45Z+LixqBiGh=oqy{s#!l7LnHvYNTP|z}e|NvyY%-q( zLqblrLNn$o+>=cw!c6l1eGwCf;omR$$@YG?C2Y6RX*oTiPL$o6Y*_BUdpx~SJ0C|R z>y3&Ts7ghZPyC`4ZJ)wu+$|@Qli|kb?F|-wy*@~HRwbh4m_AX65j zx$OKql?+Gkd#DKpEOX{{x&SUJRQb2fn^rfl*iIZ8w;7M=7m3Gm zwj53ELWQhJO#>%eLIi`<4!cZt4-G+U6kzY&RZlnuJmI0$&4)F+4Pv_!+oSBq$46f4G(T`G z8K#S(;w%{G>+-OUMu7+)NTn3B`;B)r0{P+;)~(l)a!)j^F&5TIZ03dLjh>~EKdE-93ctG{Aud4 z^Aeg4wE-iDEBlPmtSyR6mw0b|{=4U9gy=iuddyHqab{sxDkZMa>@ zcxOVddGoem#k3pz_n%ousaI+?S}o1x{oTXs$-L$))(bS@LPW4mAHT&G)O)*18CTQy z6c-#TYZ~OAWNm)-`~~z^#+tv};I>ET^nKXgfF2bo_tzdgl z5ED<&t-PeSwd~|2WV;tb=-J`Dw#NR=qsIt&`z@uN^JX0_yY-tzIn@m6y_@eXBp)1x zfOa3bFkM8j-Re06y}2AN-q_%`-?AAu^-1AF{`bFLmY_gXSxIs6v%6_cU|Z|cCx=2(plR1WkB2aQxB{&KE;C zwW0n+_W@=?Ov2}&{v)M-MqiANFy&H`VSFmTi)+UOk!_+IhaD*61Tad& zkgCI5ete|2+I8T<43r%FWHUCdgd>u&AhVjlsHDnC{K0{`!x1#a+rmdF9R#~do~6PG zV=RCLAA)OZp{$Wb%a^8}{@6l5Nm5%TqB+$@Ta(g-^_2Fo;sC?s=fpMRfKO<&`$v09AuQTDWh zmU!aacWS?x=kt0P1_yWzxbJ~~+pj;Yu@T1`7fLz)j3m>-%!GS-+!cKuxI+}V4u$`R z5Jy2Nm9hXM5Je~@$0oC?Y1hlF4F9P292bGa8ljGC@M7#c)#@uJ9Jo1|5FJy!zj=qT zqucA3NR$jKIS(_h###IYPGyLM6*W3SaSmrm&+GK<&YK%sEuT@|5z{YY^|K;7;$9em zLhZ^Y5rSV5nPa$$k>f2&$Wl@SJYTi)F%^dr9RAsetp_tk;bergIGBmQ9Vqts?+u1h znb(1_JZ~w)keF$?S@LnXHxn&VKjAbxhFmw@})cAbm+ zy+TA|pi42K9uSY`$IITH zE?6J8`{#~mO?=S7Kwc@CW8j`vAPJXmPkd$NJ`I(IfO2_Mz1~+OuYernoPNSj)R@CC zQsHcf4USE73Qg%!Qm7HUC-ddvjcZ79@TY&YU_|qLvIbBkb`1}@MNljeKFFBeS$st! zV5c7EVOg}X#|)7dDfpAI8Qu26(cK?{XUdKb#UCpf@>2~f!L7`;2pjx1#85ZcHx^<2 z1Fe5uk(*&fL>1u39YXQ{!B{JI=m)@J0{*m2+yvha(FFyLwYp!2`Yf;-aN%!K)fN%u`BDBzI$yC>MOpS5En!kHp% zuoLL~qCGhaj~CuJ<&p~M;(>NWfBCajxBA-z=xD!*$r~z@Yc}dle){rPTwBDC8A0~h zjDw@rJKgc!8S2FM-qEw2v1b|~aZ zGKs}2#IsWXa8s$4*XULWS9&gH1!BO&uZw7ofbH+U^u`e9h41hMiZZ8vdL_@_k2{Cc zVMgI0if-mu;3kfi63vzvxoDvL@fS&fCwudnV3jXcY38076CX{zJ?0vR>HFkw*@SBY z^}WgPX%>OQWd$%`R?^n1&Ck)Fm1j5=k{p z2%^lgHCzA18u7;9;+`5Tp9=QoQU{u?BHWP|cWJr!z$AzJOpYx;b46{`H%*eHrVsQ6*6aE3 z$3r?Y72wnm@q@&X?eI^rU3#}l(2vxiRW+8y3bsyeeGQP+9oMO&Iy~*y;|pNHlu2m$ zt9ta2qZ~)QfKD&vvQ`h^D@=Y@xH!|Rdp=upMYQEa$D^I5Ad z$bbZ`L95s2cBJYx6^c)4I$=4!u;#r>loGmAHN8zcr4O$hlPE;`#x5YBf?v6h3}Rc3 z?PHG70Wvrg(MfuG3Qf24-sYjJ>YSmat6B|Li(3}7;0+uu@5{7!QkM3*aj)s4u)-wG zD^h%Kw|?$G$+=T>M-6Jbc07t6uQ8AZ#WFlj_$T2)x;urqW;|}3K`*tre!e>TQ5CWH zitg0;In=a(&zN#rWGg>D9&Wxg0nh%LDa7xm$@!b+U3IYw`sbXzk_t-4g0|()s)LM? zp-5lrA+;l}jDyZq&H}A?r zeVw9?PH|goa((|j3E!FPZU3B{^7x;ngQB3C#4%t1he<(6W~V$enxbep9EO7(#Dw8SGIG1ppW^qc&kFk+q_C_;aV8z&t;P`! z^0IYp_UWlV7{{Aj@e}}63fe^W`=vfMqpTx(-GD*g@Cx_Kg!t`mfi4U-*qhIKz8NrDyEw zNu#lbsx^+)?#V0pv^U6~(m~XGN_Kt22ZcHa1+GD*INR^{-Q~AT{Kx=4 zKku6SXrQ67_gIMS&;nuI*EVeTq&^uWbz4xh(Mz8}a@p=vXhg;cf&ZrrzX)U+a`G!M z3R@BQ3zKeHi4doDrkq4ZQytbEhi-i4Emx8N58>o-#`wunicERe{7b{Yr1}70x1_Ne zJ?s~NTL&%JOyGvrym0{s-48;`{OukRm%P#b=IqJ)z@iQ#!H%$hUgxDfe_$RfjjFH3 zvx56iufFcs6D;@vk1stJu&{LEzFy3G2DQBXycO+u&Yu&kD3OvjKoq$#KYhuhQ{!f3 z8GiVQ+u>BXxez~7;e@cfJuf}(p&Ts(JW0vhr?<{|P^$5^b7o^wIz`E@)n7 zG#q%dQ8{g%j;N^9lupc9jf1w=Ln4i^={K3rq<096Ros7y%qie2(9-)zY@|=B!4!S8(6=8t`y86k6wN#6x?*`-uOXcoC>#yV(bRC%qp#2O(DGkp2CPJAtgYM zxAz#S@cQG0l*TTsF$cWixJkqujEG1q%;wt{gt<56=pus&a#lj528k~La{^)?S$k8y znQ0mQ6T?IirZ3Z)9W+c%0gg?cVW+e79z&6rDvLf(5`VGY^z>wGBBh5lDMD%J9DV#r z14>Y<@DEikFC#C5oybz*S!Iw4P;b?xmasnNj4MxOS82ou&Tw!xho*eowS);ZAqfIOrqtvSG_Bu}A+Yc%uB*(D z=^LOFqU^WS30P_>+$+ufVgt;Oy>$R%!yK{bpRWZlbRA%V?+lhxMLB;4xJvv#z1!?z zFRnn7UI`BcgEAx&4muXYBfo}}YdH}l)wgnkJo=+>83B_}rATvxFtU|+E!8Q#2@~T4 zlgnUpo-;gjT6*F^6;s3&{rRl*jka7_k8frtQN71FtwpWH-(f-%7q1g{msv&%uxtjh zjg7u}eiBnMa1KyAlYDFa3>P;^KU^cBQJyl(%5k8KJjFV06sk8 zK3ei^y(cvDeAIkpljiNs{0wbnG#bzFoLPt!m1K9)q3#?a@N;iw9&s<9sa!I?WhzFn zlAYz_mmLbS?u~!)>ICIX_-IY!{|Lx>!|$WHtXwykk5moqNb#_%d@=QUdm4D?3P22` zp$wxp6p>QjLZ%PasjFg~##N%I6tIrz$>fQfWLs`Ma!MX0&ckI4VcvNca|bpNao5c1 z$V#;?U|67M;-HeqyF8wL9_8jjMHxb{RwVWvTfY+2AtLPTIb}4mJXqFuf$GaKI`s%f zYNUR0$W4|rnWV`Fbx*spbf0i#0~)SdY)Z5CO)U4Fw2&mp_-}4yg?$gugjaj4HKm9X zSoGbla#(D=0q%9L$HV%6nrLy5yOY0Rt6Xcah$k zo>&0H1Mq5a5$HY;b0*m`$pY+V5_gu$G{t9_TfhpIN^OjlUntFYe@&?%A+gO~?&&#& z81y!|_1F^oPIDUQKrV)4{oXeMvcw)^!qRvSk5%3@`<%cv+~F28j$#|97`ou4s*|@} zFn_CE=|dPXq0G9sTG#7&+=ka2ehYV=T8WHZ<6m2o5udm7x~ca8 zsFI0Y!^Ou<{NHSNId^ky`oCTWqghPWy{|eH*u7rvooQn&+fBA-ofom_*ooAoU&6_;UxsR5Zt=Kkee9qh(cIsE zEz5GjqU+?mPg^veic^rV81tP-6p2hFgtWBN|lkbijuX+Mj$5VHs?*II^{hZ^p zzYXvFQ?;|1$m71ez-ER#U&k#oS+5BAy|a)g^u9dU6EA-{qaQ&A;Z5`f zdalD-@G3N~LfkhU-X3dos&7K@189V2A;`(`Z&Ua2kY~glz*KfXwipXkAUl1z@9_c! zDi1M4g$>>a6RYbu&VmZE-9yLqK0|Cq`27``%JJm&9ceD(;JEvy?>Td_dkX-S!N^N! zl;$PU_y#4$;CW3^ER^N*Tgn|jZa}EwfA>}RuHRcG(<)w_;zMU()+q3Owr$0>Iku?p zrKOI9D`se$Il9^!2(=XWz?#-g703J1l&e)!5pGMWBp``pYo^CVLvZC8^ zpBb+23v#F6*13%d`{H+Me&Pp;2G)2%GKUqPZYj$0{MJWHeRfZT-$A6wnb6Rbrm~dQ zK`n{mYq;RgfW&Z>6H{kQw25Da!AhIyQs7+zS>hzJp8|JaO}{%IjCyYek5Z;%0`pnx zr_aLV$(kHoT={~&>xbj2{CrU!-}W*EFq*z2h!V_lG;^4>c0WwKZrIJJI>!<5{fzH; ztkRtA+6xOYzs(V}JV!RW?4h!<&psH+Fn=LLF^t^!>Awrm^goIt)9v#Ge6{W-HWw2H zKXhn9Zc=IWeQw%%Y)W&vedjZ#x;^ilw#ymoUKCd<_Q*#d3s_@>{+bXwN!c`D3Q5}8+x^NA0n z2bV{DxrE?sS+%6S9o15qsiI32@&O9aUE% z99oNQxUT4?UXNFlL5|LnF?v9y`LR5=tI^EheJIO#@>f~Df*!xV&ddJN%Jkd#to>$d zkK`@^E(Oa;i_ck|&`Qr<3oWVtqDu4ucKACJV8`>K*MRz2_i;)Lx&Y6NoY((bKd%|t zcAfOsKc}Hl-kKPmWnymi|77xivN{DJPx9krRfYW5z-;@MwK1x=p66Q@$VFLOj_>+e zv-wGttU>!rXNPd-*|L1@%TkAdC-{g~|5<(@zHP_NnKahZj;C;y1s0rT+xeKqXyAQZ zP}27^P6%WDlm!9@jWy5e_YU+5qfjE zTyO9>i#F)7yNb7i!aPYK+P~hjgn^GrY<|v@ybJShH|f?Zy(YXw_3nqPqdt%GllCCU zQ0?Q1jkI%=GoU2!!>NS60Z2HM;m+j%|K@Q0bL#_J)bC1;wZnlQd`_#Vq zf)B1t>;K73Z}w=-6@)AGRCppwzcI|nmhU)RE;;MJUk>g-9!p4P^I1(sp3%LbLRITw z7%3_5+v~zJ`NG0#*h%iD1o1~>;9Y52U7s0A|Jx#fEf;ti&CjIwxiKfo@6B z527M|BHJI?XI)qB7U{-dq@IdKMr`W1_`p-|@$X~KC}vs5en*w+RT}qh?r%qu!cGUV z+(&m55x3V#Pk%o`1r+6rz)9Wpy!)p&G+r`X-g@^!t0W0PvsX0hl})bq$3#iux3fth z&B1(O5GWSd*C&|_YM|I>jDp@FjCV2k;7%7ODyxgY`^EJ%ND{x4$eIhiUnKXvy;T)1 zO0rbR>b<0L_-&dZi8Zc$`=pZ+WeS|-c@!tKs=`DG zj1Wgm^O^b-ba8few%Y7@(SV&*J*`Esw780ckQYYIvNaZXb#9JJz7{m6iCmZhyzD)x zHPG`~D`NrOA3g1WQO3j9V(<=YDj7rv(pF7wIm?Wh2q?aAPTlWeDEOV!sO#l!P&hOj zMuuQQNm&=$%acqZX-ENS1i(q|gbrS*<+oh^lt%EUa^(o-2jmt8dG~=H;U5sahh{#*2+n%62xu3{F6y#M733 zLr>mu%srNQP9E~dBuXJ#EnY`_(SgGAiigEdaxg}?tsP%nTE@62?1pC20<ix$>P&!*+KU zwMz8xf$Y2!9)zu9aT;AMf}0ZsXM5ZIg#&i>3%N{t8aL&+ZmsYf-!{LNHb}d-Zq(Gc zRVlSf^KD$##yL21Kf#X`V8mwc zb}eHZ(&jp=?3;FLVuH-$+5>O~T^@76?o3`vuQmr|Laa;!ku8q};An@!99$FlDMC{Y zJf#uaWJ5Dp)`p?+Jbc`LjfA-wg_8Jbh%kcT zvpMyP29{P$U0@)-7Hl{}7z}d668m2+a^%!*wLS;k|AMHJHZEd2k5AbcAeLy(5vxeG zS0DE%<0C|Upe#b55@{hY;!H?unWx;ZI!V}hy`JRgch4c25~%&vSPJHoI$EF_(lR@0 zw#j(M8%JT-63u8R0`I)Tg|Z~{Y#E^a!aSxU%Miw7W5@wNu;)A=hG9yW#SA*UhtH?X zY7{nfrY=Zwe0<_?So5EOq_I&b{a0lzf)002updw*e11^EEPlxN`_m^#X^tGuB!Uy3 z9^244@9@J6U1Xy)m`15l=`$w1Xi~e^w2nOj%q~hICc(IO7Wq}{6a%?f zgL)Gx=$AsTs;>j$<8~qTy;nYdTMC!S{Kd zrN-ww5+1qrv>vu`t;k=)qcU{Uz99YJgogzwQTuUI2l@P=7BPGnK)mv{Jp+nOe4?P@ zHC8)h4Gu7LE#E;LM-tFr&8i#GL}vLF3N&8~qr~@KEuYnWsW{31^?VETNAyVXDRm`# zd|5e~{Wbp_c@kN&K3(CBpoaMQbHkvf?X5r%l4E?jr*OSzoD+un77n0a9O)B!M%(?d zcXteCa^W1lVu)%qJu!{-ld6{IBz=~UITP5JTmsA!B9&P^*MD`^{iNBKWCQq^?mKQj zMk;7mZ9kTfG?+BrO!6WOdED^7o33j!9}Y(9P9V(Ik(A^8ePPWaT!tBmvFyMoHO{AE zBukhWi#WHOW^^zb&q9aoi&e8rLh&Uzmlsf&$zIC+Tgo5|J=I;u#&; zPwu0kGLF`GD@*IXLsdNA!@uKs+8n(t90#WG? zQJeC`M%UZk*$bC1*T|zpXlQB!AdxkDJ4@iT&kij>^oNHH8yYEC?nmzp5nGtgA`TD zLub?cfar~?xfu4p2*AE~$PKN3b3u!*ha|uedxTZ3KdtG|jL?^>eu5N4@ZpAIR^qey zq7Q{iT|sfrjF$0GEU-q{ZbHq@;G6sGz5&$sx# zw;gEMt=F3Z!eB5FvbpW@UJ8(C^Lr2XG4dU{=LG)z5ZHUh-dN|y^8ZCKsz|eVhIf}^M#lVcC zBU75BCi!U-`7!75ns_xSS_SJSG@kU|upebKFzS2iZwaw6F!a$+Lcgyq>t`q8Rs>zs zqmdM8uj^pv6==a&+FA-hxk6jt_wF%1;Cd7ZIlzN+9+t`k8cIDaU!QNt9r^K*O1qba zO<>a(LK$T`Qp~gXf5wn*JMBAuo(-7v3+=O5%IboX7jx}5ir$@t7BkX3z6;>mh-xeq_Sg+VW}jY;K`Ng! z{@ZJdu`B>SPMFu@-q2pY>>6}JAba$qwFd8bOBR3vOdLzeXqk4$h0;h`pYnzl6H}c7 z=~0pv$AePLJS;KZbk)b~aMzfaeKdIDV_n^?f69lFvjvmC(hhP&O9Fgu*&@k$P@<33BRQHR+l z_MfrvVN#JfUu}o9bIF)C0eVMIk~Db^ugI4}k}l1r3VB`^%S=EWGfM12QiCN>{Y*Ic z&NqMFr@S(4lZbEi*steQ*^2l^xbMxs?-?|Exz;glc5N|5wD~U~#o%^(k0-j{VS4;6P}p#2v7t0Y2N7gB%RFWHN{t!9z!)lVS8Ri z_)801T}JD<^}&Lop{qAW>~i()wae<6Ki&oPbT| zS+}*25@*pHEIDLh)XxUKkV@-g{hN;8EY%~jKbiW|*oMOV|Cl{z3}L+Y(OOsqFerxpuUn2 zJ`fpBy$u+d;xS)5F4)Dt(g#N0wafutDvwo?qmSd&>p0R_tjs*U10qLRI!_DN()Ant zWKX~wTx3bpArtVEgxVPin|nfYQq`4p%sO76VQ7_gMeqeJo4eZgFmK~d0|`V+YQW>Y z+_CdA>K`=zYP2*ln(zB5Kvb%}mROmt@D24-9*3oqqWMXfmSXxiqj7rA%lQ#rzWX*W zSzC(Z5n^G~D|Wn~+u6qxsA%UyJRI;reUi-ECb9BryW_3GK-g_g0&hA`(EBZI##VQ?-QpS$`aMI*?A5>S7Z(a4d3yZ$+BgTM&MVM>kjyJtasRFj`JHUOv*laV2)U&*HT9 zdplh4oR0Vru>9EScren_zRR19Vfg93suJXr7yO5MRi%U+NzPE6>Z&2tBTUUF1VJN1 zPq?z5y~#X%7m-jK-<@0=RyaYEiuAd*WU4gQnyiC=FEj9fwxq0!+3Mi-op+kOcn^DK z#{WG&Yd=$vzfHKq0y-J>IJ_Te0BtU|JB?=vr8ZJRi4=dVPiFA)^aQ}{>22J8GYCDx z9iY~g?d`1N5}etPe}AxMp98*I+IU8Cn|t8o9<5-^c3j6VZT{ z6m*Gp`hAW^jhfiR+iXB|;bhQx%L=LIVY`*Uo%_y9TKTVDo3rHEZ(p8~`|x$KoqTyQ-fD3{OC{(m%mV|ZL$8*bFNF&ewEZ8tU>vvFhFw$rGwZQEwU ziJeUBNzT0AIp@#*GuQ0B*Ltwmqx*J4+MmCgc>wl^AIN~irzi{Q;YCu;8p$j)+HkP7 z`rkK1&wO^;`%Vrp7;C+BV;1s%#TD{ii<{B^ zd^o5z^goZ+>{ecUe&Vv%?%cWJClvC}*?T!07N~Xj8JY-S^od+WI{$wal^CDfg90h$ z-s|_?&v5^y{6o2P0npUl9tsNhgld5?N*#sV$M8~`r^oTSclWU>>_@F`(Bd-gJp1{B zW`AaJv&(pJ!W;~Aenl|D%stzQ$#v|J&+&o9Qdt_PVl(~#^}Rs;`(3k5uj4_Y6rZr) zTi)lej`v-c=O6cDg_K5|n?Ger6LpfH^Vn4_!Plu|F|3L<53AQTKy!+x{a_~&u$=;T z9+RsU(B`yuYg7C7D<+Sh-(Ax2!@QCCXJ@nX#)?8AJ{wLCq{g1c>;nj_cX~I&%*et% zdnBLFqmoL3L<5*#SpWBFaup&}6>q0^e%C;WVgZ@6;*Dob9No zOOOa|T;2cA9BvnY3JAiR0-tf{6zf^|Ah4ru!@UDD0Q=v@>~E|~b@l4|thjXU9Eo$V z;3ImDd(%U##?$RY`B0&q=H_m@5^(qQzU9mi@ryE6Ant*aPou=%7dM8&kfcZTTD@RP zSM?!a@bcL$#?5}K2KtPyU4oyAW)k-zFr4=z5pn>8)7xFQ)6S9yjeDbk24nDzPZJ|y zz^oybgztqpF3;m2dRFLhLXNrO7nT7a^RH|9WKRS?TDjgy6Z`CTC&q?zx0KuE>%cg5 z?A8{-&X#d?E3>Doo!rKZ?O+cLeri|e9TK&_gYBv!giW~Xl)w4-x^XpszQEkW z0+@x|t(<@ZC&+-?8JyVB0*RZzLo=}lu?b?o17j_pE43VcE`Zx&wedGZ?FdG^pzpc` zViRQmrw0wKD#M2bb|pr{vi_uSk3~pP!Ao;=W2HhR#|{OOdE*yE&E=Cqi;I@!ryzAB zj@J(ZCsMBrVi!hfT3#2!O?-wWDEP!LRzGe7;Olk8TMmjJE{dxXMoCpvxgfMRf0K!$ z6_Uedrse31F&JsGh-tdhIX1<}z-urjLJrY{p#9je^MVXng6LCw;-?s* zx(w32h<)J3{~=}3=8U}KbQy}?(LAx|COO*A^G5lAp)%teq2v|Ha&{f5HIuWEbfYa! zg(kMqSV>U({r~p@9D57DpN`aAq&oLBnIHOhUtiQZmvE}jrc#W-kLC1MU8~!faBF(t z+fPJu1Bx9E9iR$5HJY3#bpiL)RV^P|{j+#7;boMK+|MMoZ)W_MOF;34sOHo+v)OGDt zyUATtso&jHt+$7^wBI{|5K3mL-^n1j)&nd#|Lnb*(T5ahWftupAXZQELeKf=PeZ0< zn?J^CL(NUpb`GWSEEpnv%ACNO#oC2CRmj1aB>*6#HK^ zn6EXT9YDc^Aaq1Wh`fjek4KJp6JU|&$U`T`>^H=u9>Xbig1PGM!mLksfVeVlN^4;R;<&uRc8gDd$WzgohPuxp4dp|xK%tq|F zK1nk=j(x7xt)6NTb(yy#5X+suvYYkOgydMF%$UYz3_esnl7S6k6gf|QT+E=L_svXJ z0EG+7(B0KhULggSS1LSBoCff1H$C8K*r-_^)B?*Zk}8hKbhg# zVLz7}3&+n`B*&~`5IouJ=^)}t0-0C=%yNu+gg@d<*SXOA0g=(<-nPr;l}wTDfz-mu zzSTfjffu9gSnP|eKw4Af|4Xea_W*Hsrw3N$or98LW+i^4FtXx_hxr0N7c7{@?Y*A@ z->v|I(`4oop5mbsfIlHTl3$ zZnp4#3BiZ!?3KQ!$3$2Yi;a4Qp%_=uNWJOkO!pa*eU-=IeE8M*+G37&Et4xA;y8zy zZq^39A|X&=gtfSbQQ4dwDydg<6P=6nv@@z7gHDK1R0EQeGMt#N!%q7y=nImQHy?Dp zta1RU0S7Y31HSJE$3iybHyCB@TlG|gt2b226e#H(f1p14$Q;7bqS&|1kmaZ5Z)^LmLCsP~gR!x#H0;^=+DJJvpq#$vLP9n9( zT^HU{uVetC0d6aOE)R}ja6rrrDl0q$>7eOQ*1i6c&f#@OO$Ri2-{SZb^(s0aZ zOD(LVoUrVZl8LEdo)^)=M;42as1a(_jOSt+-aYzQOi3&ytw!aTLW3ekzvC1r8zOXdL>IMRc_ACN$MDw9?szV&7=+nr95hsj(P5q1dc{ z-X*Rzm8CYQdU7bBlob>G(HGw8SZe-`||Y$u_aPtXu^x!7jn zM2$+C2q8%Yj{h(`##2w!eVYVkq<*wI6+gvLG+3BUUTKl z7@#38p%X>|DK4rWaSoqIvLzh*qtqH+Jrc5nPN1X$rsDs$m9!adKbdA*S_W^<`rQWq zxHP}D>jJJ%+nf0Uv(9S-_aZ2aE?-*#dQH#VWEuMva-M_V%D!WkL<%Zolcg9TA3#sg zc4DuGS_&1tcXNX5zhfsZH99REME)0EHTs!KLiyZ_Rmw+JaA1GVOVtJX|KI zkR>=xypc2&NLiPBibrFGtzR0n_}ty0h4$v@^lLkl|H5>-PI|k4-YSas?@Gn_+JnOM z^}UtTJCHNm2E8$fXa9zS9p$ zV7U$l<~6D&Illjvd%KiNtFa?$VNb697lz?@pzJlJO194fN6J(Rp!|GhFvlo@LTNSn z15XQsPgIRoo3k=^<)XeCF9cHT=l8W{Z6Sc=$$hRYp1M~RjZJD%0_Y5T_fK3cS!Te5 zU=7KdsBmGWt`LlZAQmjk-MW=^Ck84e^4Xe+V6{273)F*C_-A72%Q?6MbsRi+@txTkIZqxuoGd#CnztT(wlF)^}S;Ty|pZvR5dk zXfH9};nHi1%ufgp(oy)u%BSNHRcrLiqK_ySufi`8QEF9xY90h)8KweAURJi)v7Tj4 z*SNVVZsdiPQe{3j4Gz>iBI5fYMT(ST-Bi?>+bthB=wi3_%~zu^d`^!BCy8dLt#}ijDXr!1>9A~s z5=h(8K9vPp|D^0UptmO>MM_JkhS9ZJyA%)Qp2F=5YdlB~hb>w9f1&q8EEkNczB6W2 z{V!WyVovIF0-1DeTw#{xG6yB&2206@x}!k%QKu`rSH7VhdwO1Dc3c>BnR)gvkKaTqWTf6fF$M&u=)5WiN zNE$i+kZq}mpLTs^Ec_)|Rdo+HMxh1DjAeAp!aB3^HXHUndiRL>gVWOG)x#8c5V$bG z@M5WAv9ez@d$&hLwAtWAcUxRi)5Jxv`O`xfu;0EUX{-f-w$K-irZJ;5bKkD5)=2gL zVvpy%MPM>UPc2|$8+g1zBcPZ6Xwg{qixf?TxFk>npBwc;{zdtfKd<`jW@Qm1xs({| zp}_y$AjmW`5Q+YSR-2%uUXq0Ml^9vf4#aJxD49=aBmka)D3E^XAt6MYRo2KwSj5#p z$oG>v@9``_rTyw=)7e-Q6?iRA?mfx=hD2Q<{ihp$_98SzSl3x)e$v{jr)26wakd)) zXuXrHUb3-sjj)hx?kXfB?6Gqbc~9{9k$D}t)3=8S3onT`$k|iT=BF=u2?k_IY&lGn zuRlt?gMCGDdR=A#73LvSnS4bUfLCu8=q^l)d=TD!rt>1Gy)zc}(ax}|(o`X_-kA+U zg$qhPaEM7t3PrF&p+9ZR^5U*N0t`Hy;RS&(rBl6=qQ}GIQnV{winAaGD;A!rmLP2T zwQmR>)R;x386l!DiVLZ*N+lR8R*IQ|X}q1)zYyMxf>?cF!h+~#lp24uF0&=-;fS>t z*dX>qMohxBBMyeEzvvr$!98l*L@OvhhdA9FH+ijUFd>xFDhqTFVVy{MFAx%sAFoqbTNp1rU4h_Wx{hD(G~%_z<}P&Md@ z4T^TASH`GS_kkI{lynSoseoyfl92_*KQMbP%O96w$rX1KNHWq*m_~`sHl!ZN_R!3R zo+8yB`U-$-@AtHf8Sy3N_Jq|bqPqK(9&F*6+ItD}s3y_{GzzqDb$8(BQ!|ft#r$bk z3ZbUceiUFoH-4KPeo8r>Pk$@k99w>AKeKj5v?{)=ph7L?%9Z(~?Uu{0XpuPa7NAS_ zW5`B}s33Kte>!^E}HjZmT$AksaUej>V^rYlW+s^3->)ujKLD`IPr zpr_po$GVG*os^**;BBuLa`kVx&{^1y@==Z3xVinxG$(bGd&zq>Y1bLa`6yz`YxCK4S#D;r*yFTd z3FU)zXs0~2ohvBK21YR`gBz*|ms4g=9U1k#3DeTaLa>^tiD;RCZbIQ-n0O zSNH6HYI&d2c5jl9;M{h;Mb{FpYTSAJZ92QrFh|8vxKe(-3UZHTH&m zHW@H)HP%4K!})5JLAS|p6#nY*q?{u2q@LGlv4qR2zI%K0IJtH)l(oMWA)(50_oUeR zYy)#9?wi7+BZd!J@yP0`Fdi-8tSXgS36pt4Z}~6j!~7jM9zhkXP?7`H#j5F#BxtE* z-&A3H;tg-*<^9ah?`xm>K#}{~e5ruokYwn7P4Uvxa0Yw888n^RwBpGvgOBiNqFG-$`jE-#jq3uvw#U7@+sB&iEXthYj0$djEkpS7EnskLLQ_kqz})&u?AuJ< zrCgp9?Rel;Ws+QR8uoLbfbMC_2S8QUh5_Kpy7cTKpjqx#xH!@C1A0^Aj&{YECK(U0*9J40iA5@%3VF zsU5cp70uX3qN%js?Op8axLnytiCwOS`(^pDy!`!#`#^itu~giEd^=Q+*-CTfx&3mS zhq@EXy{-LzZ^G@kxxVi8m`ReyVmv@C{4x#(gy9E`1yt}Hrl@`FF@FpQJ|xEbP$Cb4 zNA2r@YZk*Fx`EN0kwO@4O2_42ix(n?%bvwa;Nl74mtJb~=@(~Yy)zp<`< z8-}XKtspWuVZ}4m^bW!{Hp)rSxFYR-L<}$A;qx5tj*z|y{mu>?2JPn+U7HbNp~s?f z>PyMp3a<&;(rmA@a&JEO#vqLjclX@K^wMm`T77!YDk%4 zfX2k%1rlsvq}oLCCmoX2*#Tq_2Bpwq56e{13hg5I=z59kRb`D!Y|)+&lb)sGyY1Ix z{^hEnauSdYTY?T^Z}UKf-6wuW>B0(e8%i@$XVXMcP2j4oj?OEfhVeAm1Ol&sx4nD0 z%f1u)gKzFORYK-Aa_X-WG|+$)Cl&jH%P_7-9t9aI0;)u;{~j;iW|?&EOZ zJ4yYs+hg+J$J?KFD$x=D71Doiw%zAknMFooQeR0*mga(mP@22Gyqro2Hjq_)Fr*7F zSe+EXm+l4MxlrufAOzLj#Y#yE%W5`CmlEF-PvtScT)vk(>@Or!Jmqq+syM17gztT> zJ|B_eb0I)Uo|i>m@o)E%pVj2<5{_5#QRUHJ z^?xr&73`)idBcic6$Msl+yFH7Wu0@sMyS3xBL3fPyJ2BR-=CE5q=S^e2;#XjTTk20 zv*2GM#-KLbFW(Qhdb0(+-Q3;X&9{dV5AX}HPQ65o17!iH5VUL?)ggl zFm3dBJ3CNxPa#B6J5w&j$F2 zue7!lRuB38=ZX@ra-$toP5m4%E#o&f zKA*LHgeR_m1klY!41zVw4&GE?OQmv8lM~E!blMqZmqn|i0t9n*gljf zJPrN3#X!LlKsJm1kpn=~idrhW+kgx{4o(Lsi*NSWtCrR4=%qgzHMb6shnByu{XGiT zIa>z+|A5oQ{p8|KBlVazmseoq8`n!kLqbrXIQ#bsb$D;S=ntLAp8{Zpthd1l4Nh8Nh zExJ`v3q5;e+l!PpBQFPgq;jgi8pI9Zu%~9$8A1Uys${>gf8=6|y_a(IswsqM|C@Rr zn!2IL0^csW#6i`Y6q_+1oovl>+H;OEzOzaMf|M*4CzY@!1;RX`4k!Qc_pcBL7=v4W zeZ|x_dXw*&3^=N8zHZDN22}53kR+j4p|Uf;ylh_Q&9ShT)-9bO)2kW8#gfmjR9)-Up50!JOAOrk94;`x~> z(IYaX%Is(G^w!>rqoY*gEzWMH^3quTO)(IyJ}&pF_yZf`SN)W%|89kfxC&5$eJNjW zYN|ilzGp(Q?55DrhwG#JSiH64P5gcbDD#WJ*gH7LUMiLdD`M{)2Q<#`omW)WSlinx zebWA!HgFmrPB(Zxm173oA2teC%TK7^8ucd49@}*BD9`U<>Ia^ zyiH0sREbV_rzwXVfX{>q4nP;PvmS}vZJ8-g$x|vCQso0qR~#rF$jsP8e`2xzdab=# ziz~j*v9fmY8o(*`ecFegUqVp8VDzr&aO5KuMpcsewt{QJK?I|faIiT(gi;=c)m(fH z|BtT{19ND#Z-81cR6V-QGVzP8G;i(|4CxT0dAz_;o0_nXDm0wl?neLk z5-P6YGGb(b1oo_Y#f^9(_G~?!Ud(KzYO`WxPqeE+Sy3rOl9hF~fa?T!5oEec2lwSw zlE-et&t&A5O&cQp$gFgP{KJP)um_!1=EqV2Uw-_-0$&UeQ2(v#8qe?jk9t^&M zM}bS7Fpf!CM6{X*{U=Lu;mIMRF*a6taCp?mlqTC*YGHQrCw9U}A8UR{daQ6VZT$Z>j+YORHa33k! zH_>^JEJpV)jaDUd1v+ajm@smfv^HBH*b6W1XYN(P*$C_}O>&saT_dMFb93r{>-~G* z9}61B9)3#rhUHNh!M7J7y(EdmLvxMxluv#{d+IXGp!J_t618~$VGj_iceOLY5`NlgY;OU6oz*`S!xs&ihN?iCCRchuGS02LoB_O!UpUp_9p#H^WRi z$xzSG7xb_?8iVINf@i^?T}OJ_4W4_M5l0ntxf#YSHH?JQ@oOGTc@jNLYKR$eHb540 zoG?b%$CYa;;=MVcKn}OB{z=rpY%rLrW#D=1X!sdJE^}?M zF5=84Zq<>W1lGw2oKabJ6)#c|)vU+4zB5yeiuN>PPjT|+H6weA`ks>lk z#NYujZgL<(pu(P*m^KRv0hy5X`}3@x-)StkD50c9Ee^wQjTN?*?em zv}HEWy>ft~*-g)BJ_a*RSW_dNS@(4;_{set?4o%p(|LFuyVu+CMvYh|qapYUUIMGj z%uf;<8=}Wi;jT>JIJA0JJ0WkP+^b|vURn3>b0E(sUnLH2E`>t^3)a;%*=a8=(r8;$ zO=o1kWvxjJy^45I49iuKH+7W}je>wz)$VomvOhBIUOg%i{|f0C5?77x=Sh(Fe|nID zU8-8_Cph#h?_rw_N83*=Dp3?On)Hs31}XcSm{%o|7U~Y%vGV#nDx7=7y3*r0r3Hb1 z@g;tm=luQ%`LM$sG}-HG?~NBfyd859DdAa0gPs}11A`|TIO^r*9%Q}4N|PvNXUj2a zI$;K$kF=6>C02_{URV_1y5A?#Q~6>L<;Zq(QXgiL-KX{~V+du^>q(j|NXn-Mq22Sq zye8Ls`g!;0^}H5PH_CrE#jf;i)o-**w^~-3SVWeqzPI?~qwq(@dF^Na2$RsA{HPqa z?gQhn2G+xzn0KsqZS4s3@TvwQr0-4NHgzvJO7s%AD+MRT$I8y?M`N7#v%VO5><(LG z`=*I9U95oy{T1E`PH7=*I1KWugufR?1{uHeY8iC94ci;&y{81a21vJRMfCp~@mw5< zwQY0$utP)`UxYuO->alT9qHYRV)iw=<$(S-XoJVoUrnlaipJhP> zq5u-Hu;06du=Cn%&F9m<54>A8EkDnzj?`YykF`={Z--~C70T}nIbyUIG+UqV`4tFZ zHoThkOs+4^X=Iz6?Y3(T8FYZPdop~&la`-k3OduKv>h@tZB?4F zpe=qr$Xp#&mA)k0gm`t{iV2vut zUy5MLyzsr0L(zBY!>>H=6XEzq2>&ajL$T12Nk{&j2ZcTkXx}pUkUgV`2|Xr2QPP-g zOO7y`RhVc?z0sC}4*h33Pq$jP;m7WE5NYM;dLAL*d5}mcx~DdgD0X6CtN#8OQ~om6 zdUi1ab6#E-LOP3~d&^iqgl4WGwmpEvV*_Reyz1C*kt(B}f{S9vkd*0L=_{L77NJAL zEF)=v#508cF{XkghL9#ZR6W;GCH`ZKj?~r4ik4p&H4ROyJArN&1~GIHffSiVtPt@Z z40W0@7BRWtAPhC#JOV1KA_HVf< zrzk6dr!G7S{OfL@U+1XsLG8y`qeY(QD+Lnn5hzYM@n*^qliF6hVTBiKZq6pd?PCi4 z>n-r}iT~=u$|9%UVb|0%A+(k>C9aPYhQTIO<^^i0VFOE%i5^=Qyf2#Jc-N+H*h{pEwZhIfYUBBJoAOC!ZVzlUTfq zKtwRa_x2l-_D^`OO2|s>47Be?Brt}~#R6eMp2)nKGFM7>|K{S|C3(>6lIqVq!;}O) z&j?VcQ(WCxxbpsq@({#-Bogg;$w-HQly~2b+Dxgd3>$E4wHa*`Iq47RH{0-d1X$1# zw1(tZV9ssOl%dh=Or-6s;~cPe4Rbihz^ zqJOjg+6{Wc6>7=%y&towsT#TqfRO3MsWaD1h{XAwN&2H21L;jhuOA*A09;2>{Sry1 zF7OrkR-{k^mLTXs0v?A&Hf6u+3$gD`4M6bIkoR{B3mt~e*KxR|^qqjA;MmO?%AxU1 z$6LQ5tYKbPzsmo7*CTVUVB2+l=w+dl-(^K5JWEi*rnLwLw~msysz!w5cP3$@lo9(h zWYU1F=WeA)=!vik83ERTiV)?9FN8_VED|Z#e;cWJct03f+3B_Kds9iqv~MB59YoDa zSIOz0sZxr2kKFh^Z9SDM>2nzWH+b;HvGKosAMBxvYOY;-0FCz^? zz2vX^llk;1J~%G_+1jhKv#6f-^aFgYKhHSI(jQ~Hms0UWckjta5&s6Xzd3GXNqBh; zru#kJ@{;7RI7y7IZ1HF5gz58~W2%)Z-hQjC^|(xY172<}{;}{MiNPC9xNBv z3e-D@Y6+=xOutL0@FHD-hQX0O(t~)=qVs!VVV=JeWJ(Adq&XNzUWofG2w^maeX8bN zxTkYF$qIkv_4MblQjQa$=trIPq4?8Qr?*nAMy*xGqJP7vhwpHM)Kzb5WhFft0$IDV zWF|)@g|S>odwcuA$Bf{MfH(WEDV~qAL~xEE?ZiqA@FqLm;vsh6Ni5o8zhM%b9a>ug zS+YxS*zPn<|M>t$;705^T`VQ>w%iBLwT?|1xE^Y(x1T1&wX~=df+;XQ`p6w?MybW@v(Pd zSZgZ@o7Yx{E9(2%3^?pYdO18N6r1wf&t~>#{nzYh=6xQa5^lHNb<+bTtMIib?Q`bg zbv&aKe;%ATLkaOJZduo8dk^Ed6h?^kZL58N(kHz70-h$;!2p-6@;kI3CHq~8m+jbl! z>XwLQv_n8iXYza*mm)cxIix0Z-rTgQd0?Nw@O$b6a9XeSbRLZggJjU!o}x(Md1~>| z8iUA&Hib-{Nj|O@W{5VPzIpBh!o__{L@#qXN|+C?@iA>W|;ueDiTyfM+vNjTA;`IB~Um^C zboYyY%h>82JH@d=9mZd{`)D52M~=pr9>t!o2swNT0>N%s8mwlw zVA*XomU07MS3hUB|BU|B(0|#px%8dsZaspe`H=|&QVM$=KX>@H@()L1tplNrak|(EOsQ`({(~AHvV-n`xf& zRVFwJs1#8>9}J(ZcAnkxzU*Fot!Z90@CJHc)#M7yN1Nt5$ZPX_Pvjw!-U%Jv<>9jL zf47l(1X|kM-<+;IDHC>rnI+_hzMKu277c&A-i)IBZgjkuY2Sy#X3%XxzCXbwehadE zaS)OG`4SLF4Mt0P==IzBxA{wKDQWP`Q9A!%G}bf=>+^l~vnr9t?PBiJlM8tiL%L5M zhmn@{Whu#X8FM5trPI0vj83DK#jh|z0q$TfTXUKSH-+ecA7q_FB7S=p|Fu~DaX6d` z9nR(R{17ojRWO4hPkWCc8pf*> z_Pcmu?$vYO6Bnh4xD<9M^SWJ%-n^t`sqrJfN5 z;tVvNOl`mJsT!Jm5mK1KlkcX^QgqMb@V9(fUP$S&5f#qkeQzP}02=P~JyXd42=Uf} zfmg*5(kiPY=8u^O4i8{m=%hjg)$h@{REdy;%VjJ#D+jX{&voylN6Wxp4T=K(6}uYt z2mRaA=LK|hEk5hnT%>a>t|A4e69ht4S&_@`mb*fZ52Hn$|5KcB*XzQt=HQWRLLv(* z`8Oy8sMw~UUcdKwagr)P$HUg7JhAK~9I}w(?Z3*;m-DnVoV@Nv$T);0q^P&+YE;1M zbv4|LLcn%!gU^%(bb2+Yn*;zKm=FlF6W8kq+$0UB23=b82T% z2=uMr7doR5zS*Uaa=lDvHBO`2Q-ejNQyPNIr^VlLd>t&2u2PdT$WCR6)$e?2a(Ba# zK$@7OC)YWc3bz201Atu5SMr3Mk6gsX)zs9^lHk&*`M`h0*!b_SmP@o?M7EJXzs1aS zTYHkf^qPIgfS?Sq5>7;GN!2YNK9)qn*<5yeJW2an$;8ntz4xe^3RCy^hab+3jGP=! zggik$y7AvvZao0A$?vVY?__OeD*Z?n@_Fk{Eg%2tDS{+Rl&W)iysk?W?70cnyF8I# zQ%C(;Tu!qXIoU0IZIx+jE4Nn1qekPvJyz1EwT2#k!yQkEvh?>tqL-m$rERCXX{ykAf?qQk&ga#Qzh?in6qYCq)%;;C zd{Ng2MeY+-x!h;eLq{LPW!dHEyfmG%N0%K6r-4i!()5!{XYuuFNUAnSRwr?o^aP5K z^9)~sHmyD^p6jj@;rGII?yJKiBd+JGtHc64P6yW{{!iZY01w|g5S=h`0;Y!$LD?y zpH+8$pUvQG)8kj|)Ghc9-Pu8T3&WhnYo~)}L^(D+RT=O~AY13*HSw!=6JW;4syi z?gB){?}n2m+SQ2?R8$fHv~caJ{O7bk_FzI6OFsXS?!$pl=VOH!Je}RD^Iy5bJ+(bO zJzx~*=EO`XRRuJb!BEr+9$5UBs;YSRO>m|A3~8*rb$B8@)gH351n z#6FEST6mcMaM|f-1ipZmRlV8eywG%Bd#2@J`H7O8|7+vX)n>Kp<>+iCL`7#F7<|bI z#Tkstpyx3gU}<(4LZnw z+-Cx-wHsyPvA-hk{X4Q-UaTPyY;yYS>Bfmml@jF9>=X~GakzNRWOwg=Q|v0-GLm=D zH!;&P6FV?l6w3^994#W%MlTj0_Jc%?sWcgJ(b7i&rMaCp^Qd)1i1 zYP0vZcqfenZWM)3fX8}hjKmk%Nv~D)@Ac{}CkU-C-W2024yPx0JKeR0WAcQb_M5?& znB1@p*UNm`FVa_7o0AZq&7a{rh}Xw(sK5pc~%M^ ze_!kr0J?UPF~Qs)M)hmlGiM~6t+slmvat!Q0U$V(D!ty~kEPQ{uG~N0V#WJ6+nrP1 zyXw6t;&D(;wEUh=Sc6APX0tu-ceYguYlK{$hw>+q`VU`Qi1;z8JU;Z@Q5m8?Y)_rf zTUPYkL9x(s&Em~qOP*iKTqP;rV7Iw>9OhnQLO9k!tBo zXFo3htG~B}zz(Gl_6d*<1!rwXNMm8g%t^G5%TQm@(y`N$uS(aHDADC`^mzC>N1;1>p2pYfi1O*L{Xa z4g?Qgh~;g;`L7w<8exz2@j;~7Pp9RT|9mMi*q8Fw$%#qurBg%G4%J`VIg<1|?COFq zq&-yH{a?8a46Vi^uSU2zd%!qcQ0@I3YV;k)xt-a!7F*AefyJ-$!EhK}nft)%%?9U* zn6==^hv4Qn46n5UNjb#n=-H2xaNwmNpXp=llA9ZF=4qC@rCPW3@1xPym{G{h%|$5ar4EwKVO|Wqqwo0y0o^0jr1M;mA;27VS{nEEw$@bSB8#gk5hvNRg>Hc0 zWgooQ8(Ou>*52k#4yuloWpEz^%jcs~Mq#+%F}W%+!4A>oEg4oll!Azj?@Ij4*blPLqkIk0ZY}OhgWiVL_n9o}%QfAk;ua&UmP( z{QRs4WF%?fJyeq*;4r(6Yc;SF0?ZZ6+Z+1XNrW;5W(L%L__go#hBks+!zpXNXY}IJbNQeF9Qdcq3M?*3vluja3i%YH?;U|2Bvi=VShXq>w#H2AJ@WRl6@)`KlX7H*RzrwK)%46c2NAofaI6daBS`9QZ zt2f+$Z_gQ`*@-$OLaJIeZf-5-w>yd8JHA?3j;L+Iii|&PY&=YWT&09q|EgN3VszcE zlSGw;Lj1wIs^!-ctRoN2Ms_?Xs%tASTn;D+YjHj2{!vbmoL$FWs?@@5D52S0``C98 zDc&TF%)ncJ@Dm?{N7|^5A~iMjZQGB2d&Nm;UeEh_tj!65MkO8vpU z@|WSz5D>=tWJv-=mDPhhE+=CtXExIwf8Bp;ADf88%g`3~pxH&oX=b~DGYo9l) zAPzPjc(re;5aftf)$Ew0mh+NQnVb6=5jb^{2~6cV88<-nS-ezg8LrgqXo& zhH^mmvF!gC;J`~RqkY#~k4(oDdGGloX^27ag5vK@^6aMNb6!c|UiO7DI1 zaFmDYdJP9`wAr0fG*UJ0rVJl%gPI$^Z&pcC7=@W-h<%Hbk!@;sP&*$i92ir|0iaOj$oP2154G1o$xHp?$}G*X#WZ2^L5f zh#z{j_#9?W8@)C*T@H%$n=Li~1yk&}K#K6+gzO?=kS-XbDVtzDxs60P+1`{!pVSto)xCaBy5Ggkvs(xIy|FDX=V6!11>K;)F zudWlvR~4~-R1V-_{-du2^(W@}x`tDKtum6YR%(n>Y`m%ql`{jXXyoIDL5o{&d$m7o zhEMZ#-{0B{PGG}$I^;j@%>NmE+Gip}yO}{Fw6ywoSvV%2i=!NL=!1sY8VnTjFPY7g z35QNE72Y3J|+!-VM!NokRnDaGLC2q@DQtO6a5kN*I|aWW~;RmOkZ1I&|vq} zqZC8XCrY{xuBZaKPpfB+N|17f;m%OvnW-Vr#)5B0MA;}IwCi@RzPN8%8_APzyso@r z+yBNYE7vHQR3H{`gEZS*z(wsrXPYYwhD9y+?EGK!hzjsR1l&Ujd0IcjDfqJ}><)7G z5=>A8hktdR05n?wzwU=o6u`g^seihkxLl8o%V?w`Q?VJ$*EYCK{wnN#F)3(z+^Kgw zEDAJR;FxOT1h66xvdGt5|SLVm7|`so?WOdZH^*HKHZ1190OTBsB!^vL9h zOgno!kGnuP;*6*Aa0CC-)gJ}=|11_Z4t2Ry^%-SpvN;Y|2P>nH?)AOTFtm+GtmqOC z!Z<5idfznZhqL&R6-)Y{OcKw|WTNf=5JeB`{FHGF^t%yZbIpqQ2A{*HNK*NBh3XE; zLex}Eu$n6QTmN>gm#+H4|8=N-xwgyWz(Wvv!Q&uz_-O`=2AQ1dpuq2KTgz9Y@_<0G zP+s`aMi(w^J!W-rU9kt{`?M{NKtsNhO(RC63s~32?|w=4zmY6`FQisyTe_K-w%!>a^8W;;*uO9zxyi(Eg0GW`Wl%;q18yL#BY=A)Hduc9$(N6T&T^|uS zDKn>90fBfKIlpD(=qNh&yD2O)sYt5gX%fT_j7S)dpqR8eGIGt>$9$; zRv4LkwD&?k4OJlezNvuw^)~R9pAljU(3+Tg6m^8OWEk9||5(Ch0&uAAL zl9L4?^8dED?uRvuRm%yz4eK3*@O#nHeqq?;sG(L!^DLjli5XjFDp)d zyO$B7Oqg6k6>c<6M#^jQ1O8wEDKDEHHYOM=ABziXbkmnSBIOq*It$hEO5Wx)N)A#K8eLS;1k%;pU;?NT$VusO(e$5U_sC&LDn zLY%GgEMRr3zFfn_K+7*JPXs(@4cX+AD zRVq_QLoGNcLepQC+*m&hR{oE=;IdZF<;oc2pOv0gPg6L0wnXYZqVEn+MF(GP($fVz z&NLhfW6U%17Q^j*pI5@|1;g!&S$&T`_zqD!zX-Ku4P^E>+<=g-vR7v`Uf!sp(O4o? zA$afjl`0|OMG9P8)0@*o#*>!+>3G1exT554s|o$R{FFWs^hwa{rn%fC%qF$VrkBond@>s-uAZ1AU4>>_&5 zfzdaj>QB%{IvA^Gd-%!sH5C9Vm5zv*Q4mi;ga#93N%F3m!DW_=b0FwJ9cSwcyYAIP zcjD{nLb^~F2SR4#5LHhrqTYEXX#b)m95x}Qyd%9|OpUwN+WcFK@89+OdtNk#3dVIoQK!oV9#APVy zA|)3~$VQHoFMu7Q+>dbBT8bP+x#;+}fH+p{%k3g%u~9jx)%E_0So!|NUFhk~Tv|+4 zMh~|Pa^eH~5Sm`j8kLBViC(|WVSXZA{KW!-G!b12%7~OFQu}9lXZIbj&3up4o|s#@ zv^$Pd(a$CtCn!=?EuDfK5(?e~h7b}f4+C9PqK$)!yy8u!sNNmbVD%mC{zMW$*}M@poM zgZ|J9A;$p*L?#L{I&!Qo(lFKGVfow7ha@S*gnahx5Ae>^G8xuL2|Af%3GtrZk0HpQ zz(P{Y$)EG_ODXE42vEpi72Lds-Xxb^faed%HCZ3e5-;BUQidRlXw_d}rhai<{ z17ToxFhuol{|`-P!4+luwf$j0T4GQ-h7P4$y1P-NK}xzoy1Tm>x}_UQ>29REyN7<~ z{y*#e0@qq+?rZP!w~qr6@K-7(OGmfew9mmXm5|_zYH=cs9U~}<*XXXhoij?oT7l78 zd;Mjb(sdmx!h(^i5HQODG&Z)NQ!r=TrW6kB4WU({lGo&fGUU`Tz)7P5#;GZga&ygP zSHnr*v1X`Q5T@e*!ew0sV2vQSbaB<;_4X@|;&lo(CKZh-Gio99(&X@BV>jjx@QI2r#0ZtFRsxGdFFU`wl~=4NW8y@Hdpp=`Zfq`k|_Y?#!M3ui+k=ty`J z#gt|%>o;Rm)~I+W1Mw$%#!io^yd>sMSe0$uMXz=vTALPq+Uu$`Ax@eSf_CP4wm;Hi z_gVYF%F3L4HmUe{BUR~D%&ua+dMQ9Ed}LVkRu1^jl2nG+2rMfncdl1&cPrPq)^27- zdwonvxwofAw^(ev@h`yWPIqqD54TrcF>Q=i3JV*f2qDPGce)iHx2x401wj@?R2F@J z6@@r~$CQGFXh1rv?y#SgIt0Xnl*=9uYsvnpQ1~4%u;POEbsBZQ7fXbWIH5OU*yHb7 zuJyS(FU`L@na@55xCuzbUM*&FZ{c&OywI- z3>Yg=yNd7o&@-P+8q0UF!x%Ho&*g7Zm!ASFZS-T~r9s&CioWQ=X5UQu3aLvkuuwpk z<3;_~r=Kg&WT_*8z8{2j@EZB0;}Veoa-=!jyKpML$V=};X_U8#IqMQWd%}a`L7@_> z-s7)gf2)lNpZGnPIP%g{iUP@;^ zX08U{%-6}8es0OYBNTW);YfM%Ub(7gLya*Hnj;W7#WpY>@LEswF^b$=r`gW%d53=a zo_gP_vZ!azy!NPtL|<&HaRtWjz(zqEB4 z>d0&}!wp-!+=FIpLJsYm#E_}>^+@MS*5t@p$D5(RHx|=b=`2ckVIT@jy6+&&=hx%0 z(zAjlEVc)xhrJ7;PD<#H{P)sr3Z$QOCHPc&;d{KN8Qm3!4M@a=mD~gfGka+@Am1O* z1_N=kI)7gJKRBZCQj&k!M$8e9RZT5ER*HhCkD;O7NL#^N*&O%BgN##T8v-MY?QMRD z*lWJY7*#F(_jR>P`ZyU<-D>*d&8Z;OErYyMBJqQ829|z03gB**h-Ic-&J+{U$hURmROm~{IPDc!9vv&m&4%%t zyXHn+vC_Z|C}ygvYpH00vbfAw7TNa|euQRD!jcq4bu{%A$=_*m2xi1aUwqAB!<14~ z%(@yJ%5t=oisiyQL-x5jp-Z%>GA#j`h&jDLrdpR+bDF|`Ay&D{sr>bALXLr9I0B}S-Vf`!_=W{<0BSD96T11g&i^{7kL$Q! z>B!M@#W;u(4sN^SDlMjJysFqyLN=dvF8YszWs-+imsswIt)byWfd*G-5 z1EjZT1Af!P!{J+lk(L86XiYD^DA3}h_oz1f>Wac^sleWBBAc2S!}D?zjDh~Tqy|%W zN$PC45eS11_-{S+eYR{@cw(m^BBiR~1tSm<@!FYm2g3E#e(BI(4xTv0iV$=neKVW@ zgFCGVTTiiFF2})zyVP0 zr>*h&WB9$F1AF`oq|v5o(CA;Zn{@4Vxa;_X1D2FHF4AP60 zAlV$(!T!Tk9roBaj{OC_+$K`-(7QfBhxyc1x4wd(t^OG9`boQ3icLqdK*y?ptxYcJ z(f;@A1`na6_Z@C~mWLN(N07`*1sV<9D=z&J#PwDf@hF1TA~pY8^GwKtVEvB+DB5-* znxtm`g|Xm*kyrjFRRKb~#ep*J6lHp(D)A^Zq*yVks^eS4$BUBmg_T_Xn~k3o(qoZT znzx9!Hy%fj*SC<@>GbZF(6 zWcMTeGZ~p6%dV@}ZD}#YbIVpX&p!d&uYgNW!cx%eFj;N*m(<;$n~eK6bVKPpt$>=n zvPxWaNIoS!Lhx}gR#r<5Hzw}Hy}t&)MsSP;jAb{+B{cmQ5OJwkz1{Mymu_H#37FW& z7b0E{r0kYPo&6HXLU8erp{bw+k7Y^wPRE${UnnB3sq z8v<$RYSVpGlqDR%#6JM51O?X~a<14#Bj4Ou*KoU2^A~_U@F)%9Q0Z<2^R>8NbzL7~9}D-v zarQ%pTS`yovR!dnFD9#?gJzo7zwGQ!oPTj}S%n0!|R@mZ^r4;N~|+gGKU;l zvZ#9bcuTD=NO~zCukkmFWPE}#6*b$)rr7l0zk2UeUynWh+p*&JGRtdEGs(YwKwZ3P znV|rb=3#1V20kUKZ&jY#uR*Rnr&h?Cx_ z>b%+A)QFlK3{_ngdKqyI!V=QHg9WSX?`@A!NVsE9{WD5(TQC5oYW*S^+~2iRFt7L9 zomx^p5_J}Zqh3rJaEHnW2jIx$dwP%}tt95!yV(3&zF-|&(8Q^ygBObfM?7VBudS+B zpsA*Iy%Ep1KlY#xq9|4((Z6iNo8_t0Ym_j|pv- zE}n;D_q@*7F?FnLv^q_1Ut9muu*@Vfciv1FpS7Oy zt-EOfU>wBTI`c~tIrh*m$6F%I&)H8g>yOdo_WO6%L|h$=B4%Ko5~2XZ^XygE@piEv z;%p=mqU~?63S?t!b-wBHqk^s6WE{F=f6H|grqEqbS#CZC@xQmd$0$T`goDOGnfKH~ z2qo=~Y#lDQME8t;UbNR=qo=EqGUnmHvW`(w%r3vR8iwN_{|&5^kQ?)k~=j=2KYl+OOMrIXOMAZoeKvMtnn+q(U<;DysCfuS2FB5N5)X2 zv9a5MXpn{PyzVG50+NWYjerUA@2GwFt5WQrr~?QVK<8472Nfyez`=bVCrv>O7QwcjwD|=i@CPN?^lYe}u>b`7|%Jz4`i@ z!1Y5-tzh(gscpvnkS&N z6AV9es51${j*Nn&o^Uq_{gr{9G$0|7Le=s5HE!#I;6G^~^rz%TQX-@1)P^GLWnLLR zu``A%RSDrl!&_PrgOz!zK_;@qSc6z4zf0|}`=ekr9d@@{Gk^-PunR8j|E`I%!68VC z(gXWo;8WB!QEuY{q{s#uIF(gLuMv z(~~|4x~{LTKNi284~9A7jYH_g6msULf7wTAtd;#xk%^Q^NB~zogRxRB6^<9so1dbK zJ09xke=d9YxIbmNlC*8~o5i*1LLYqo;D?EKKns~;@C9`ey!o9*{*I@g9&`UO0c{&Q zU?HsNR`qZvSzDyD#Yv~`KzXyuHGVh0Ww+?R_0M1k8I%VG*=j^#3^W;PkbUuVy8Y4w zQRDjG0cZ-`%E0a7PAD+!7kkcR@f5=Xx}8rfBr@#>+Cua^_g9wR`p_SF??+!AVRw5@ zBMfp=hjOv}Sj@k<=7eOkf#_bKlE}h@zF%C&18h-=d1Me|CRZP{@GcnJ`Oh6j>U68F zoV)KFXSv9U)~T>=2|IR=_UoHCAUxh5tZwZm#TKic$ULk69Mtt3JP%FlpYnT}{&!r~ z9iO;6^Y=KPG;Fkx0P=(#-pcHh2yU-df7yjz(pr5?rk=IsMzCVblj?&LitkQJ9;P)q zbSkF9q|;Ya zLF2bX#W*DLwfz`{9B!;cMA~oJ6YH&xi?htt1{T-YNnZ3Z(J z|684YS`odUj4_ZGC}B(sMA@N{lxBY2!XxEvxmO}u_|5fQ9U%21P#%CprR&6kfP+uJ zf>6{LWyjX7H$P7Y2SCAv_YFNkph8($G~V>k54xc84-6IWA3})zW&@N4u#lrlcQ_ze zgikzr`K(mmDZm>6f2T$uycc$$qVUUsM{#|SCmb>nV?jw3e(!6~7KWTADNabOt~)Jg zn2-<#Rzdx-(i?gkII9dF!h=!g_!9 znHo`N)yw#hfQbw6{ zjAFdcPO_u4xhl>kW#1iS7OcLJJN)>84&acbrF9YdzWuWSuVNW$(IRA}ByVepn#a41kXjs%25`A$V+ z$CSB#fOV z91BdN!Wf*R$aXx!rDgB_Vc&jrIU(d}I`PPuv36oi4dkIU|4~7?<;J99qzdpw7&Bff zFlWbQO1-Z$>sVe}Daa!=u$^+){s>2@&Ig^|N)5ntL1{e@F5)68>Ss;SYqON6ghUIn zvFX0VY6>3Tw>KG{GH^r6Jl`W=!#UtZ_fY{85X(+e;aX6;S&c#U>O8pN%PQgUG7B`C zzb*a^4v_h*%->T*XPH}RIQYl&ph|+tKMeRLBzDOc751=dkHb40^cfb1f3_RV*Ei%r zdd_a_%d^CYCLm#ayUX7{T6Us_WN=qTI|dnj<3-514R5d8r7}MU@*pCliyIWg zGs`#3sNce~8c*9DnM^BhpPhxUWWp7G(CJ@=Muyr)wmOUW@6AXNA$e|-cENrASYjVf zBl+d#HAd)>7Y{@H!-O6&RL?`LZs->wF|t>s-9{BmzqdErmHW58wZB~FXCInqX<)1` zIjDuBP^v#ARa|!y0K~RKiNr#%$ocqoyB(8hw+UF}tGW(`7ucPs7rGwB9M`qd7%3Mo z2oSwudSr*0v8jeX^B6p^C&&HB<>~4YZw0;(G_`||jTs~x>kA0=<&x=ghdKNx9VEL3 zjSXavFTRlY*|X?8r(z(N+mlZM3Tei`HU(37)CLxC(*hmi%`CvEFK}JHbgVFcce2m< zy0N6i3(T15y3zSD!*4O*Jkv(XyqJA?Xz~s%Y*&m07|96HhS`?$X_gvbCn8c3trsb2 z^Z1A`LHhwF>QroxyjQ9d3l2Tp(LiGcPqAW%7gSP5Ooot}*yBU2kNV>vJ=}a=N`5gm zV6qz=3*rH%fT?~41V>{)c~4=03H-nWIYm~e-yh-&#sEmbyU>ymmoQ~cs}-Xdp|gJL zsi}EBD6R#P!t4x`*AQCtui9Ok9t@XX07>`^2@P~6hFA(pkN^xs{e&?H%_0w5|2tv| znS6IgBh1d#qBwb_7ixGUHlB75Lg*rr1=N22JgheNVuH_PXk;DB^V7yTOmXC_KpL!% zf+cG>Se&^(p@Y~yw?Py0XWr*g>zzwe^Jg3VAElBzbbc_w)U`$9zSlRGWWtWt{`2|U z>(EvFG^{%fR!r$Q_S-IDk&A>qTdm=wpWP-iugD$%)6nesTsgtvCKjpevUX^5<35%9}}=S)A2rW+s|n4 z2HE`N0hkFmlAwm~C$7%8`)zKj~*Oy*@lb1m@SQE&AqX%}IIfMX3O>T0a8Li>dtNf5`iXN^fr9(~42qkH#y7VOuU(;3mJjGCt32MPP#? zb^B&1F`gmFKRT2etO`oZOk~waJJPN_$GWr68%=BD5W+CX?Lr11Pb#$Ypb>u;5oc@g zlam0Z`O?9&Gw@*rB6q_zecoju9~yzS&RilK{e03zMVb`}SzQTR{PY?Lhy~`!*rx5G zA@xX6d0aN?#(bKG*sof|*7v2gNqkKo2@Sk+>jx2gf+L6vJ$H^*v8=)jZZ*wo?JCvm<10&-&SGPxJqq&q49k%Bc`rigt(* zYd+J4tlq5W&~xVVmiA+fbdcPt$m^^7iZjfu^Ky&nf=V)R>`0h$wE_pJ?HNHY8C9N#F#QYJ z@u%@jg&dmZv*fNDM~mXojc2-lA6GAZwn(hYQ~Ei=Io)fwgV;yyi|Y>bBN5HevuAFo^kD{4xt@#?4!i}?Zf74K)M1z~(EN;)NrkY< z>s*QoQW4_aaR909d5U*aMrZ$$fjCg|d`bdonJf@pliTMD<9aK9arIrpmwon+=mj>vr85pEB~$>tLV{5Mo%oFaE+wd z#ypMmFes?pt)72-$&b}-16DmalhYNx0+O>HK-BQUWut=kj{3n7ZRCJ1<3TD$fu^+^ z1hjz)d!;ZV+?bFCpybk^o)(aZb$&32x9fM z6VrD80(g>NgKp4BD+dM5LDaN%2+JWxxmRgrU1#-!FD%VjGK`8IyHvIq^-4Z-6bbi9 z_0Ft`stlGO8@_lREF`L=LrejGc-em^b{~PUh~Tzh_>S?RY=1-7`@hB~pEbjdun2nn z!0>v%CtR@mCZf%^Wy02uWG@lHY7(^cV+CbP(u&4>6YGU1Xli zJh@~vDJ0nGjxjIAB`^3Qz$b6#FZEUD&Rg+|C;Z;k5ThqNEU^IUPGA-FsU0%SznB8 zC+*WoU6CS_aR3fP8o;tsG~7`Sm{=1?6j(TpH7X5E6JH|^s}yi2mWpa_tuFZ&YRQ&% zFcnn73cwP>X~1}|?ffEaw%?jRkYNLE|vABP&^@Osh1_bOxmv1_L^30S^?m9);JFN8cN|BIctiLNnD7c&V-_ zWQNi_H+ryq9?aX{Q~Vf}twnyB=PrIffAD*WzSogPfCBX0P*V~GAe~Tu{F?Nb+a_TQ zAt!qDu5{R9p-)ohv8Dnij%pjk?q@o!V+R-@ncg7JAPbkV)H{5A!9>9?3j_izr2 z6;H1amO@t|3*x7#guuMgU=}}kJ6EgdIALK|h)3Y03y`k-r%J7g%sUllO`c@W?iUB- z=oXU1>;YCMH}6nnJq+wAbFZrH?^op5ypLyntQVs*O!U$3+@vFn?#zIIgtk!%6%G3e z*Ara>w%C-#qI1VehrL0IZ#=@;`-_{t&fRv&qdZZ{+h~&##CFibD`-3(b`dbp)e_X{=|6bfxhYKELAKEP2eXH)!L;fOo?{F+UNvm~;_Io+)#^qm&;scbGl?@u}WeS)`zvh0(jyth3uMF>(#koY;HQ8nu zK@vB5w}R?u;4+{Fm6JTc%FE5aHZa=m=r7unuGfvA`gNb;TRX;cRs^Tm>AvlPHeGIc zt) zxlk{wQDaWEJ?ER+U$!ata0g8xd1~6>Mbwx;<5;`$1hK|pChxv!K+s4=+O6%zjo(UC&?~E(-cw`^KzGRCq|I$GhOf(YK&Ces z4mk3|t5{)b80c~Qpj3V8H(h&vOyV7LQsP)%`FOql6iTQ-i~pN;PCV{Rj2dCbSyZ_U05vMjYYmtK5pSn5eLHpi`|^P%~2 zY3uO+S%BS5#6zyzC;o&)&v;sm)oXs|4}2Rfnt#5yoza)_J!X6w)(wA1ulL!xbgbP* z1?#>!S#wKj9H?$&#B;>WPqgq~j=jSAg7~}bxhE8VeQ66e9-MIfA15Sc znug4nT~8cgV~7)6`+OP;WKA1L{4#ieN$4VyI`dpWyIb(7oRc?-5b+AnA@9b&*d6>e zS@WWtBmSIe^43}!SBybXbmWpB#fXiTaZs{E-S5ucl%VXwYp&r!xoW7GC<}}eol7(T zY>{w_#C0Yam`0>vpWl6r?PX2b#3-%##FJb@@2ABC7j3dPtr4V#vE;^m=r(x35GW|Z z$2;=EkAq=cxV`viy9dr+$F56CbG+MuWWC_;!B=@iQ?Ajc7eghDkY|R4@Gc>XG<}SZxSjmJ@Q`({u!@2goz)LC z$#T6Fe=-W3>^q^Y*X}bgJ7k!7!gqS=xp7&QVgu%c0yYBzeTh_ za2@KX@+V0RXJL67$(y;_elwQQ#ea;U*v%fCmh#b4S5~&0QQ(Ff4j-;gs{U>q9$S2-4f?9yDI)O{6#>Q`U)*(TSSwj>Dr2LI4ha*TfygS8@$kOXn zur5)6W_X&$dEJEAM6kbhvgh{4jmZg_^gk$^!>P9qEt<;l5CdEp3th*TVRqEd7mBg9 z!^H=Up9}Y?+B1zk5X%Qd5)pLc$!Iy|6BJchCYTDM5-BK$a+&tgcpizYjDDU4Tnmv0 zoY;+~sI7AflA{~_wNRoaKienQbP%_guvDD+W93+s`9dIq+BacEr{R&3wK_zO_Bjy; zxjW=fRtCDcGZpQXL|2;{a^aYYv+ag&rL=b$_T`PRJLz;HF-%Bzl)!ac%mtSh;(>OR zgpw4o{;{VM$EBulSN7IyWk=w4x>3ZfWvNx}Ik1zUm6?jR9b=VNWQTW((?hT>%_%Sm z$L8L#0WEE?j5=y{qLp~!E_#8`~1lJ4Io zrbt3;xa(y?+tY^{Xv>y1DsfJ|j2bEYsRt|Vn^1->Tlf8H=5pfnW3$CjhEftP($!&$ z?%s#$^9OQ$KR4#i7$y{6f&t98U9)1v(&Rvx!YdUZA@y$Mu|fvbn8GeyJf){Ct+Y5L z`dWf4tg!6QXF*yqjg4;epc1V(1wvgsgH;GTS$=lh- z_?mWsb|&j%+ZgrO*>&_;qGk5F;z{{Rn~x<|DekMXdT!8%{h#Y<%I8+kC)7```wBxk zOF|l=y`-rPJ84(t3#-H@HiV_54QO8OzXuB!R?ovn1*dmcHy7Wb!kZ1eTW{Vu%UhjL zEqrh(j{O``v2W!C@j8EZME-R4A$0s<)yqi&i!zq3oL0dhTRMW;1Uk$>5hZuS#BKdZ~v=z;SA0L^#(mEz|?Ou8eIIhzjFC6U{XDJRBzkJX# z9qhlWKUIJPVW-QbS|o95$l8R-j#wWNchv`(4WHP}B}pW!`G%YZ1 zD6P~OjA(f|q7>RbxF_w0UR96@q7umX%ra19T=|0w?c>@?xItqCUw$v-4gd2%O9sz7 zk5VYg9%5^YgBM1mZBd1iP)Ij_EeA;St~5PR1moKLyp}y^@j$LQNlAe`ZFLdWA37?( zsM`IALr<1T_m3o&1DMKQ@YKdMn zAuyYdIB{h#7F&d}x9oToUmKA;iHfg@`BkF(^egcGbT3(L(%xC#Cr9b7dFtjLyU`-J z`~78z!;{GSJD25Pbb_)fb(@kl%>ubqDP3AiW>K4H$CXj~l31)gk{Kas&*Sh_j;cJ$ z%(Byc)96z*-k#;^r9s^!&6Cx-@2ZkiDO-ftFlJ;BLef_1+-oFhPm?9;q1Z7# zIAXi=?_)|#&r*Kz`4x7gLyLX8%^bRNL-7{eV)}#7aH?_MIAwbsNfzUc9MXviDP?X0 zAfUiA~%CtUa zesY{#|D&Fm^=g_NrI*itC++^C(2=|f{qmSCpzGj~qJMR}Z(_1OcTloKZ4u}Njr}S? zM*fto^5ih54CzE|=j=!!T6_VFhEMMmEF4-(d4G=hICcp=&3o#591h7|u;jB?(QO~c zQhAg%VE6j$Cg{0S!juvs940Or(+|KB2L9seVTu16 zj6u&#PydOY3Dzmc*zn2>gF#^u4xi}Ft!>61+(}$5k!}8LA_9NG{0D1)!FW-iZ{XnE zSiA*2$NT)BoXbaWK+|{(WNg`=2C_T8pC6L9Xmlh{GKrcT(222tP<90MLi!qCU$jZM zR7Lyu@}k)$StS?@(hW2xM`bz7r(T>HsJad&?dR^I=x}hVIv43RCpv=x)s<@)#KSK9l{QmOkU`8J98asGC|+nU*6A~u%@;r zYNc;6@M`Q_Uqg=Cb{g0_R4zCeZh8JCw%Q8VQhRjG&>{#ga@O#@Y%ovT#M~YWYSE=lRfR9>-`&t{C%d9zMa>a!J0`jpVvKL~3)a zL8j|N^SW3{Hs$7idA4|Wz_a{-;-n$d1 zo1x8i=}<|+#UOCZxCDJsq5q$9B`?&qQ*EH%x|B^7^m zPJ*r*oEzHhD4NBARAI(mh_T%QhZ-Om;{x;Rb4}<0C8j+BSTe`Y>oVmr#RAE}7Q~vE zx$Zo%8Y6fW8~iO^qA6^#yulYNlP_yam{E3+@bch*kY_MI#&R330k>kJ#nAI&A$oR&$Q53P0uOm~*ycZBpf> znS@(?&c$|r0@VxNnX^o5)4P}KhZPjId!hbqNqTA>E_PpB$lR5x6cQ!x+xlFI#Ubk$ z(R%FdW;b7`Qqp4M%HF|QdBFkt9njcz@YY*}y`M1T!F8DL+ivbwbF))B`4&s<`u^Rs z5JT&(-r(?Ay%z+~$ULCr?}fE^z;IN^bs0alcmC{qAv_&i<-(ZOw+$qre=swy$ioG% zmt)xDF?UBCq>B~K_tYrJGopp3HajT0psvthPgQx?rO=IC6~Eo(s`r%4iV}|I2xS3- z2hl&lfC}3f7w0gijbKM=Y3c#J(^&`URud$FibkX3ae9Z@=qC+NHssi$&R8myfL+aF z5AQ5GBrGm{#O9PGtog%~%qs~;)AA7U2t*(@Jv2FlT(_ItKCv`h-eH0uPm$1tFTIBv z3ED=K42dD|*&Z!rmA5;#YBqRyWmP@@HToS-7ou)->O%{yFK>S@{((M{fmh|~gj8P< z^!r_I1)BI<2#bwl1hdCPjNF}%dzivkxdN6iHA_{UzVUH8R6Hn(@qP$0Y5g3;0&ItI z8LEmrzwOgF$V)nP1<_#_PCpz~-e#n2I;239)b^PFxANmBW!g#nJVnt%+YTD|Krne} z=QzsG>B3GGd6uV&O`Q~hR#wZwwe%fHkwE*w-<_)d=totBY?YAjUR!(fRchvR^>5_m zo`KK*+N{Rcoz~DL&N*IC=r7Qugk$Fm(3`uPjfq9<*j5r&SYo*UR9!{ec1$cj zbE)EIvAAtDT63!UhhZmhfDr+IR=?vsp0r^Hzb(p{AEiYscfgw( zt+T7dbAOeWB({6@sd~$QTUw`b-NeQod8(Faf6=!J4|@s+io8{!Gqzgok$tLPNE zh%3L5 zn9yj8UBG^`4oq~O;&vv(C7gP-K<(JdOe%=Q%rK`>u%qcb$rn21goD>GT|rh8#o6=I zs;*(&`r(ELTCi~h9&^WOqc+x;vFK!EpKEm7@S2R|t96L>$Gk z7;WC-HJ_u`kN;`|RAVe?Y^RpJos#@e+z*K@C*uz| zJ~MNc<)NBciQhi9NiWwOvaj!y9WcvzBY+bNwCs=E8Li;^^_@b${^!(K)0iPXH=maG zbJe<#>lraC3ju(7V5)(aW#=)}cm`Q@s58V(2r{&v>`5VQfG*5Rfl?BYVN;=1Nx&Ze zaC^^y(Sv38X#5hs;O%iRNsg1>)qJITqNtc0tL0<8H)USCN4QShFL61fQZ%D)`*7xs zF{KXxHvYQofeE0$&s{zF>Da%J-w`H@LbGuXy}jpW#R_jTZnycn-^iD?OZ)Y$P1E_B zY$+{vXF#8sdY7D-EcWv*1;Fg6u;VKz2{DmP#niK;s!HQdSNKCN+MPg?{kfnZ;AbEg zMsd?Qa%yjgBKPZlC?;qcqg@;^oeKuDwb?v;|MIlv?AfJ~FXgp2gUSE&_ic54sYm+V z5C;$=bJb}^K~3u0?CI}AE4;g$;zJ)mY^uakbPtk;Dbd=5dB4WjL=9dQg_-!fmq`&( zo#tQvt=1+MW+XH8R-+|LuXDUq?$8|n9rXT*U9cTv$`yay1eH&QI2wqgt-S0WqYCFobDV;k?4y2(W$NCAR z+M5cfCO{wru=id15}feZ+r*#KwA?@TcRbj{Ge>&~U@!Y7biK>$9R*|cj3yu;QUxLq z!l5aE1B0{M%I~z+V|2e3AIW=ZsiOH!xZfP`>K2c$Syr{Lx&9WItu%_x+)!z@d095s z(V!O^vQ?dH)F69Hm*=m7F=g>@1E}XC zP@CCG-Ojc6s1PxA}ABtgYv{?Q_TegOAd`Rqwy|ujWrke6kNh z0eod3{b4!s4xdLgvFoMPjwdpoix)4#^{0_H+DM)Iy=Z->#)H7rv6s}p=F6>7c;hpb)XpPo{%$jT%0BS=aZey*X`2avV1@`)R* zV`YCu{NyL_o*-EyuX-np(DgY!#5d1@UrrE!lOiTw0GEi6U5kCoUo;Z~X>{5BWUaf} z(*dSs5l4MbkZXUwq8xY;NT(}kU{z1sWVd>V*_Vd|>w>%ROJ>dQuwQKe{9`=m!>N@8 ze^=6d=%+9nLY6OU2Y&?0`5}kD$odt{8y08TUv<7Mj}guPe!u7h6aK{vWp0;6ro-$K zW2OfZQW)sNPlLa7$gE;5_G!r;pWoEaC53U;O11ts` zKUw==3?gkA~L2>rqWZ*SYV#(90AO$q$@o3+iW7fLYWBZ?%`a>J19d zul-fxuYYL{Q@k(GQ0sR21PVdk^g1^i?Gd<4()YH=y~{V@=gcD-1xFV|YIvdYtlrwe z5SRW^^Jm0Fv*9fb4;gGbZvd3KDJ6~C6g@;Ajd_FO`OZJs7yk*fbABdv7D)5CQJb)n zA1owuAy)&b4$VXNhS`>ocb^{wblAUe0wOJP8bb#h>|xWP9PTrV`w&9}`dZsr)_=Dj zOK6>_hI@m@FpRY2wnhjg-}IyKmj3nk0VR=<&KSb6g@mSG7R87rpnZeo6!je2>x!m^ z!7K?NGu(w@V3tA!2cHeP$F(1$&V6g163Oyd9b2G+1YU>DuMOs_vA-7RQy$8CH_Ij@XTidV*CoYcoT{UC3mgE!z+UGmp?vZSYa zk)A>3S_b9shx_QC*$S+cAL}-u0nCJ=niaFDa>;J9$_CRI5 z`OLDg84-s=etU_cyA+>=l7o=0hV%8$LkbD&=7kiQH}sDZiRGaGEJ%~tXLH3ZYWgnV z?Ip|REvLM)^~astKEK^W77$}dO_p-}(wA8av>t~x2rvckE3wL7r? z%A+E-H+L~9WFpx7m6DmaRpcv>C?MEdeMd&>^L$whaX8X-U2TBLq*6F^T3TDTpVW@% zxgYNCj-;HY0Tq$He%y<7FzSzd=*e{CIB9=caCLEV`nameVVU_W*H`pvAli!eYLc}{ zpb;RO92LZD;2(sb+wRrr^)RPdtVqaNdmC_S;JPGf&ac^eN_NFJRm+WX(gpDju%^unlXAtey6C_xuu+X?u|G9$7rBp~y*7?#lUl&A zvAe{-Qa4Tww5O}ph5=nlW0XICm<7>_pLqCsLc5U+CyA>;2)O za)eRC3Z%=wj(<1B59zy(-5Ne62WW*fMU%GKZ}&g>yp-!&laTKcWu$kqxZc&w(Bc@+ z5q-j2aXmkaK0`@}ZaIs2?Y-^>y}rD?^5whyZL@v3+2_6uZoZk86LOwU<2_xiF+WxE zdECxX((U?bW7FaJdSb2?gBzDLk^*yucxh|%ShVceOSkN4{GBgVUr)?ox(&nQ-yZK8 z8@b4tNG6bLQlD=Z^hnOu*vbmsdQ$_|eQZYr-t3>oS5I+G532WC98D*9Tv`n)^je;~ zpE6e|gbojX62TsPD=RDCxxPK|5pgBo;w;1zdQVT`F>uoCMheV6gw^>7st4}GSdj=` zR54vds(7$Nze=ZWxbBq9Ta!E?ckVv!QubBRK%*WtqEscGTkp;UyoW5I`Cb9*^d^_f zj=q~s_xW8_h4`JNuO|3t7`)t8|EgjCX3I&P4iD2Xf$gfgHfC{eX-7wF#kyDaJ|pqh z^(kklNU#0XIX4F5d}mV=ch=f_L?HyScS0v4|HT0;(%9z>w*b6# zqztkvGJS8m%T>g$(9OI2U-;@>Z?(o@xyK!B@A5rYi>eJA2G3|Ym^W6;wzipFv|Gn_l=KrJVEyLREnyz8o-KDrYEl}KD1GK>% zTA&nncPs7=#jSX83GRhbBor;~t^q>6T=)AuKhERGuWXswvu9?{tc6aeV&1?r&>^Ga8n9Q}KD(~}Rfl}+hNFakLw(q6=`MXf3~~uI?kT%G>6 zPGcJIi0@Z5HJ9GIUjNsf`vONtvW?%C&p9xr=T)3~y7BE5BMKxdlTyKR{ee3g*lR69 z^LgDJzVyrZmUVQk7P_G>T4?Xuq3aIWCd|OrUpa3A^tkPOQ0Qy3F;5$3lUdjHxZ&~d z0QPMC;zVUT+jodC4lAwh^b3GJ?!%O6#r7k|ns*+KA)qKObN{m~^mE4xEL{ngns;58yiAmgqcIO|hqJ*P-){v&`rspUx)V;e(l6 zIJ*ZIjo7U_Jb#kSaD77un;+bi*b&B|5h%Gk+w5IEz%3B)K0l25S5XALy~^4UIh&r1 zr`_*u339~8FAzLBv>p{+cU`T+mwa`zzKNMF5WTr8=R=iafb_2UJRMcWPe*FahPVd` z?%os?WuzB5e1%^+p3sX8T??dxjKN|@y$w|oNmce&x-5~8^RX?XJPNSz1u4&4&GK}JKtz_ znjiJ{*SJYh)}mTfn9AZgyE3vZbg>c;bL!qRC*v`5eNnAUaJX7(FkL)ppUDw`{oC@G z`UQJ=yqPlTUh3<#8k;c%NI-UT?@rj>xIW@;K37KK^Z9JGgU(tr`)k4V#<#=RdGDQb z{Du$pgP-M+%0sS;g*`UbI?;vDVApX)l6^LNv6TfveJl@4xxhfPn`vS#s*1DvjU6os zPZRa?0wFGxzF`lgD{A{ljYFXVe)rGV-s?dpAc?2J>xp?=TG4vnJL(41&*P}%CL(R+ zTNQau0ajNIZ23l~Y@jZ<#0z~`?$h+up}XHx>P~_moU~^ELGAwuF;CE5*FwrPT+PpN1*xaZqu+=cbxbgNnNMXWz%Wo z#%~B5DA#$EM;Ll~MV?73e6!0mnqbuW#k0X)>?(pd+V-sR^Wr32$M-SWNKMTj_T3)F z~-W+YEVd9{W=yVhIK+J@`!?c{C}N@ zdz}SsPgLqP7<&&7V+z0QNJ}_#u`b zYA(1_L5H{nStEB5SEYIlM(Yl_S|VG+WXD}RZL`Byzu?_mr*DV;(<*gElLNTt^l`a$ zbxX`sfiweIC}cXDDX+aKb+lCom#vWB{h=-4!fE-Ri=;37Tr-)N3$_WFU*3>`&3`Ft ziS1`!X@Z_W;j8A0u4lsUX#|_^>Nb0+pMJr!|CF2gSFmp#W-TTh>XsQ=sj{nEE0dz1{!?9= z)WY}K`?1QJRI3qtQ)XvZQL_?TWmYB)hKoD+AiMLP@aXyfWbQhqkDfIB&u@@WIN-t` zo6hera{MAA5Pm#>pB0D9eyf=f~aF3B&(**tDdt_cnRY&r6d79_435 z?tXgKg5#y^G^S$0sd`OJ(?w5+&u;%j6B4klQoE3 z!Kc~2+ahsUnvE{y;|}2}b_+$Xpn+UFXL+ELxJmKLHwPH>9|cXIQ?UYaDbgp&*V02-W}=X_$w+%NLbcechi`eBJR~LPx3K91Um&=F!E^s zkx@v{Zw&6;{cXVx>sMmCgtN45MUv0FzSs_~T^1G=KA%Guqr1hp+t>VI;qt9;X7ech z2LI>fw>-3%{YV>ao*y303%)2dnDyS@+V8k^q8-bb11^3S*OlT?t>2qH69c3Qz*>=^ zS#x|r27Ou?d5kuJpH!Z&1 zqWoT^&e(hn8ruO#WPr88baoP?eAcdQ&mEpGq3>rmJZGWNt&Ie7+0V2F{0b{13r-q8 zf|rfY8Dtp@E}{5Rf(l+78r@elKOcTpTlWTNiIb|(DzK0HU-Ukd(n0MOstEOiukg=% z?y?+Lr3ubX4sq%>QM6yg#L==#!m$zd+QIdBetvM&rHA{`BWt5O9Q;4ZdCs;Q>jGjd-Acz2Od->OG~HgO zm(-c{CZ78`)}r@0-b;0^clE)x^G()De#slxNt%kYcOB3E0fFx8;zb4T{@4;KPR{gv zUSFMW@jCf(vI)RIgS;$<M;f;WWnlA(>fxWVJ%gs7`!d@>7 zT!nqL4c#pQc7D)1VSw}#KQSHShl6M_=>^DE|3P%d>;|~cVGOUq@@Nuht4TG+{6)jT z;pCXS@Z{|4=3j8zXQ+6Z@o-g$0x+i&1AM$U8c*H*@)rZ_z2OdHsLRJu+AsvM#-H6B zaWHAFfiqmz|8Cq*}%vUo{`H zvH9*03M2)?Z{sTuv#ywUH>cB%tEr-Hwmn#32=T_uKPt-s=MNztnLBu|06ftrCgENg z()|3RcU83@pHGk#t)Z;xJ6#=OUnyL+O_x%2~TknrEEhcVLr_OZ3Qt`BP#QS6im-rPrCKkIhtyCR2P z%gkA}WX9vlt%6sRZ@Rm>?k%KGE|S=rYR;cycS?Ax_xg_zd%=k@WYVN>-%hv%!+mEyZ5TM|;x^|kT8LFmRJ&jP-0KYvhDCEBVo0!RNPPA9PzrmhQ2~_Zrqz=d z1y=lpEwBVX-t7~XZmvGgIG=Yy&oqr=(sL9e7z*#to?MK<_fB#7RL)PgN908xbYL0{ zp&8S+JBf7*X24I&Js!^MjpxtOCWyMBv3Md&a&&|dLxuOIy{s3gzY`0(SG~`%vXSSj zBEbo+w|r8%$|~G~qO5rYG`(Z-hppa3U0#Y!%x0BJ=@dF!#nkY)r|FZeXMqvmL zr!Y-sv#js%jBvEl??Cd!S`-VeZPMXme-`QX{L^SD|AH^zMON{B0gy{2B>+=5ceela z7;70k^Z$0jY~H44t<8J$C+kHI`0pn8{9(fs)=O7sQzd@pux(iy{oW^5y3&ZGRdPO8 z!ylVG?PN}nP;qd2$w>E3aQSMhE%9uftdF4FL8tY6-$OJuFh)f&cGwO<#-g9KfTI!s zW?>{Ot_sjaCES+I7+<)G!_pnQU_%2XZ}+f^BCh>q5NlSA$SL56f6FMxSs2mX=6vz^ zeN1=8uXCqDv->fBe(Fiy3N}1m4@U{$D+t_xm`bcYZlF_d;E3Y0=qfZ!{J4=XpgVqX zvAB^h%T%QDuI?64j$m^&$5bezG%*H*o6>yHoX&|e9!Gp*`t*ea&p*`5QG^pM2sFRH( z9xumB=I8J0#%8nY-wq*bm)p(0J-+It-8{MK)gDEp+JrkIZ=x%0qvu6Nz8@fy`qJmN z=@IBGIM{s0iUHzPUnR?r21CAy@<~9)qkiROxMhl{5gsEo$^kP}w`L3u0;OF|evW+q z?aN&D;~Z8aU9AZBEK&tF;GFVE|Ms%X&3b&^zVXHFsbXY^X^{Or|5^mM#(R5NGoaI& z=O8GZrJcJdAI8_8#)w}hd}#gKj7N;_G9<8vBGPO%)*HP>X70WE!E(P-8!IbId+VZ6 zca5*P52Em#bHu$jm>$-y;A9c&W;ds^fuAQu_(=IORM$^16qs;49y~VM|7|^!abMmf zWSM!KE0UYI?0{aVt6P^g0%76K-2Q)QIE&KEw$QBJ%{A9hpp<-_aBp}Oh$<-TUR}uu zKtvuaGhL^7kp6l38closzdw16Y~ky;%7US_GBTbTUL(ZQzJ2*l82!T#ruD&ns>hdH z+HKa+9?Ha2vty_ykHybzzr40)Qrg)FsqmLBz&HC_XfBO=CIzBbyas8Bc%#2hmwAw=+jn^JIY`% zPVrvx6=$r|6=w9ODtIUR*o@wE!7MuKO-qw0Gf}cmt2QB_53XF06-?+vnB?z(fHp8o zsH`4cG^*ws5us?qXZBbYFV*nXo_>b-gQg1?EP38qZK8WD@PxwSr<(tQF#ukWwKT^l zpULOxK?HdFZX*&Y;d@cb&qPC1-y)ENV^3eA8JLvXXcty~?$>P!2@B8WNW)UPFMQnh zPE0==j0Wr8qjSi9XAsmlqJwlk8^!X(UoA(NYByE=|2BaB8o8-~ALm4Q2*~09=LUPb z_eeRVU+3gV5p+8o`*`_l7$A2qF=Vz}&Y2m{IN+((5puHVOu8dBrebFM!9z0A0Rv-r zG;_=|zo^o+pexbHWqon3uQpnqev_}BlD;*3#k`VM+H+`yw0mWhl*7c^v)m={T(M== zeUIGb>~!t_%^TObP}n&r^lgFp@UnI@q@%=LJC(Hcj<_g zMNK?XxgOW4r9sdyV`0Zc*k-fmPG+5xJ$okzhGruzar5F%{oXOa`zEup$bX|wxtv_= zUT#=2htTV^?_@@t1?;w!VSYz_t*j{`F{VJEYw)gf%Nn)~&&%?`P60;O{1_-U_T#^| zN-O9*A1C#T^kY~u?dED5C3Iar0U1keVlrZK^P)8KMF*!+D#w;2X)w} zw-+HoAx)_EG*mF%uaLbBU`d)wViV3oEv#ASdi{GTd^G-z-^*L#jX-ZWF9#VV?{EI) zt~=-vTfzB?=Ml>dK+LITE*VI&MqV``gzG@Fb>JpSR=SjGDZP2y$c+eD#uxfwr7!}AOx3bP7ELjZ$H$LNlq!1A}FtJ5VZdBCWlsYB^! z=grqzY1qy_nk(N6koxQT_3Le#mg85NMZuoEKkEC19%~=NLx7|b_QzL4#zw|@ZUU8b z(1AAmy5^7!e*5F5A!Fk|U_>&CJnH*B2hi1B1Jp7NQ?ziz>+V$=b|xNtcM!ams{B%7 ziQ(V5RSJ_B*H3kLqF&@@&UU)~ra<53ujfjzAUzx>JSTj=x`AE|H=aqFN>Aur7&{W5hM z$7gA164zxI_Ra_30?qi}>H|AJn{@X;^Uj|TTxcHbgF9%~UUT0cT?(7|ngQj?TqQv` z&Jm$IXJG}_8@!8j26j|!FEy~q3AI!Gspe9Lb&y7>Ba z7iROh4y(SP%oSb*c-rWB^85iHB075a*52eMSts~zJ@|1lt8cm0U%Z-qSvRrAB;59T zTCMMj`i9njrBZei9;ycq+S~5sMJG~ZNdavCu00SY(OYPLiZ@)4n0g?ER4w-m6ESA3KyXITY9^Sg38em+( z_MGd>r*EO&8@{$rC8LxxW?FQ_Tygx7fdvIHS8$-diR;mx{;TXkqN8)~9ht*4VT~+7 zHvR9%T~UO+>SR`bZxtALdMrY_%wim8O#+wxFBw7sw4?_i3}{>Wb9b z>#HmzKa#-!nQIsI^3Q>L+{&k^Oz6N!%WR%!+uqA_-e%xo{VI}<+@^ILnaTTyH5jA- z6gg36+WTco0KjhXGilLC;f{5;mFYCGJT z+RuO1hwX?XUHlTx2P*X~*!HEY&LHo_M^3LY+Vp5CL-Dl`jAg2!40^ zx1Ve-jq*jy9^is{JomcAak0WOdKvqf=<35^PEyF8P z7tNuQG4hFi&u8s)f!oy~Z;Gu~+Z^?S_okTeCHLW!WPZ1?m2y5%0fGGq29v!Dj&2>1+Ws*!P5n zdFwx2l7h*0}V5Tz|BHb5`0S2XrSo4hv>2|yx>nqWUO>Bv& zZ27@Cmif4Scu%n>x4Ifoz-=PV;Ny-8D%W3M4D0Z`f?Hc1H!j6l@Y4tHu0QR$Nb+xx z5ol9_$!B{{uhqG>bD2!yFvgSKhbaRhuo1NpCBhm}esJQbYd8`AVC{INriKG6zuf;F z;WMNtwIbNcMT;2J98)zq;$q@Or-4h6ZJ~tJI zds9On5(;G+hGCyvp)KYc&D(X3AYYFy(wkZ6#pbKl+xm9r=Pl_?4Aw~R3DlOT_T+wo z8{xhn=-af!_bXZR=O8%5F?e>SmKk|~9Aw5`aL{H;`!8K?p$L6_ga77};B@hcc<}Q>@49L5Lpb&Ob}p4~ z+n#h131;z-lzn8?HGSQ8X7yXU9{@RC(_3*xW*xVCv5Jig@PF`wC= zwi~Fi(`!3F4l4KXAnixDKl0k@N*r{#44aQnt$F&rw+lOQNM8uJe6VDv+m=A|lIhS?G+aWS_U z6;wJsAd9OGv17CnS6vO};?oarIEdxW&Y$K0K^=j%`s3UddW~4}%(_xdwjPbWI)q`h z!fcWVPgN<1fNsQy1$z2$`gL9~S){G{Er4o-Gd_7ya+K&VgRyh=PvIJ2r&0nXI=9&6fBSU74YiK8D!$;rb3)Rv{jwzb zOoM?kZL7SIhP?oL|C7hMorkoD%|WtQB6uo$I4HGA{@9Na#&HMxSsgbyyJA=(+u=#d;(htqX_|mk1io7 zaP9X_(xczKUwk!Az4)p(`Y!fcdn&MNP~rC<67>!1JzrGcn(215@vEKU{pD&7{$tJ{ zgmCF0y>DHjR{srgrz<<_X{eRWkhW^tx)eXh2YLs7zlYV4Sy)d8dyHQ7^28kJP&!Pk zAyD&a=yz6PK(URcFc;Ok|A{W661z^YNr%&7cBae{rHzy1ZbeDvH4)PZ|_QAl=lfLD`l%lGAp0^TdBX_+85(_|FjG^8%PRH57 zY9!3fr5l)EK*QLBnwL2Lj{hX!Ix{;ybWF6NA9-DHL5SxUGJMS^Ngb~a<{S5?n3fa& zR|_yYuPDN^7b^P;y=%oQdN=D*CgOp>cS;`q+W&Y54zT|;%G8>&TN>C4e^lfsP%C{63lLKQQg ziNdX*Q}40f?tMDpanipw9H16$G64XMyr>zjBc>R`S*^>=p4Rmdz*VFWFHjc569fA_H~aeZfTv zDOgdp$9n&Jf;h=15K`-SMnmOHHojymgNyJ7Cy1++-1BAW>KSv;xocVXL)lB&xO6Q9 zummfWX#NVRex47S`K}w3EGGT^f3LFIkb#7{h`=n8UAGsu;HSE9d~p=L_q{FmbGBJ1 zmnbNc|8hyn=D57JaRZ&4)7j`t-wy~W3=rIt+Gxk0JjzpnO>6R3@D1{vnZZ+AU3@dE znxrP(tRMDX-GbgNLo>j+7@17Q>SD$IYeHE^bh=_Ztgj-@-R4w|J^Ll0)6CP06z2={+lm+n_{=WK(9t;P z434fteVpCwdl4vVh@66hPvCxSbn(CBA-v5l`kfJ4hher|-KWREYhcO7X;k zaHOlL3@$7#17NJ_a6EO%z6XDR zWS@SPUqI6N8ag~ztHtPqE~rLk&FIPXuU!F+i%vyNeZ`=-{0-Os=laN4=5kq+K}2Ds zm7>%sVLZyEH<9Qpk;;Xg+mq_l(eKZ>HXvo+2$&5cs)GF7%Bz14PVAvamEBHP`NCsAewy1Jw0ua`? z8mlrPCxMLNVC5q(8kG*-oUn=E>V=OwsM6J6-?kUOyYub=xLlUeZa!zgE4ozn9{Bpk zP6lGj3NP6|hpRpEq{z;oFYh7ixE{?mVSG{&=DrIY-HTwB5~o9m#pr#!{_}Qc6k2#L zQmPZSd*nKG8+VQ^-~RDE_GQgFW;sO{pP2H5B|7bU>c~&I#*fbc`-htJE~oi zE9K3~lkr>?rBFQRjPBQ)nek`9B_tH@^B?cbd+VbT6?==Aig=D7?7hD4D@Mn^)O42X zv#~1G*;|1;4ZnZD)1T&SMa(ZHVMX*p3k#_>uJp~q^~bH0m!X`#ANL6$F15jI@ID+QR)SwM9*Aq6(_HTD9VEl!8BbgUP_U6%mJO zLii1zoQ;0S6u-%0al3K})GkkjqA&5;$F2hVA_mrS%v7I}lJmujP*FLWajR2 z?<7=;CU48HL@wt&C?WwH4G*R`=gG@4jIfyIH zMmiu}T;cdd+Z6t;LqC2&DNTYckd;})Q|9Ah_U&ZlQ}DK*^fcCa`zuw?=*N!@*JVTW z6MY5KlHXlS_0Y~|k$<Y z6bKC7(6?1tG_!_*R@6@yu8e`lk38v@uEWlEP;51X;WGv5!=Msq8-DE!eyd6W9Vh&2 zegIG6hxa|Z@wrM8r2J`;X$E+i)A(KL>{{DH4#{Es7Gx`{y+f^^io=~K8Qsu3aNfi8 zIc$pnYv5+}-h7<{=ZhmDI@d6kce3=zRo2vf1%0J%}WFni-HrM_w zckp}GDGg*oHY%q(MY~szqw6e69`!Zoj4tFn(z|!(xlUKck(-u<{imJ)s8kGaMi@oa_n<~HNsX=J-^P6iF%y-t6@<2HHVLYJy{ zNcRii<3jH%yn9T9|I)vc;`=h|({HM3ZoB_S{NaY_9eRh#B#{E=Z_{qCvqu-O)T-;( z@BT-Layon!QJ}*`+=9JVtWyiATG7T5X@_j|)E0(mMn8`D01x7VZ{(_785M@+P8;x_t>bG<4BW&>KNL7{YW$vT-tvw3w{jYNzkds`k6s#AFIU7h zdr`Z1@nqu>!%m-LNZ31t!~<<+3lmpL14;=1(#4R`S4#7Ad(U#|xE14BVZ7;pQng#v zuj0vz@Ii1%`6b7OW|A_lG z^nLN+YS4#i1uk9fW0cx=+01gs0ktu6{}2Z;fTT4RA69fL<>zGgk7jF(^k$Fm-yAwB zZgbXcC@U5z4yTxxza#!y+1dPc%YSs=bz;xjlI2hJE;8~Rae8SZR*>$syHCwhvgd3a zyGWfoNsYtFrdmQ>eL$<(i8IL(U%kT{S0L9*$dGt9N5ddKI`&0m=Z@IMR1LIgNSu%* z2nl1n;e8eR^G#~Z?}owhn#_Bjv&fHh7||6?4~3@(>(wXc*?-X5h|lb z&)kE88biT;0f!A+#lK?NVo?;tJ8gfE?({SOD0L})yiH3js&uKlOqIK5?E37<Tnf#C|Kwetj_L5e(OS+^P3sIM}8pQkwT&@wXqc#+nfrFS&X z4c3@86)JX%^EA}}Fam)}6v^hHWbY2d^%bXN@YUcK-SyVOr0!a#mFl^My{>_)TW}?q zdfirbFu>Qpzn zCW>D`n%@`}1~Xx2B;HnP9oSGDxGu6?;ns%5H@Vr3|g4vbUc{e!ig9}mv zl**^In~*Uz4Z~_3yBgjdQ!=XpXK^M5oD4v2+JY*^ZGV2a&N-?Gm@bm?u$51-0&8NV z^AL}Ucm;HRWENpJQMeD}cY;mbTNx03T$YT=eb1yvIeO5^D%(&?;K5QX!;{lB-CAfc zi0^cnS#zj={ie*%?t#(_MG|Jvf*Q*JNejaeiyzH9Yo4=$0f4iC zNRach$fa|kZt_PHDH#hZON3Kc$K!#a0+)o&w~DXLydT%0Y9L_a2_LJ&&!jIhCa##M z!DwnPooZa#^0ViNeD;nkS>K2d(DJaE%!p8pt+G%*`Bq2$L7U%f?qO!_RxgjqQj^x# z&b}w-^DSZZ!;#+L{@^eXVbffVX4V`jj!$96=|4n2!nYUXQ(c7xM1^3r$Eq$!awtwn zM5ZS&!DV0Q{JqfmbS!|0(%i)TP9JCCJ6&Y!^>FgdteT(FVzn!^g*udt=YP>@)!2iZ zFpd(O_mrXPz*OX*1fCHACNbwK&Avpe=Lkd!#WoL*d<={T6I98`hZ4hFeUYjgqu5<* zj)_whf0_#>6xw6RVkQq#YR(i<8j-b(Y`G=9i99{7@h#F1ZNC~ z52BNd5pdqjHYshnXIvL}MOKbhiO7teo2{KRS+y}!bDjFBWMxvP7Wti~e`?^((3YP> z-HjuXH5jQ9`jYV3t$TP5*lJ$Mr{0Q+VE2+?H|G_Fq&l)RjynO!k^1zvR^5`xNzH+1 z^;ef-%iy*8lRIHbDDw4l%)lf@fT|Hz`ZQ0fnq1ajJ$KSs=ExA_Yx$W@W$FOj09>_7 z>KcQ0lIp330g1SuVo9e763K87`@^1+#ZRjyZ1V+6L&I->vp7id8kMUAG}yV~WaLyG z7LE&WydLK7FWve(L*a`Z72Q903`;4%I~>9ZTxIce@$+Zm z1N=S$K3olT6LAZ|bDIoMPor}Eg*Z-KmvK?cq_6)RLO$N4wLVNQ6?fRKza-A}gF}F8 zW^~bbF`tciKH=HL^)cYFn=hCH8%^G!WQ-SWabPketi1y%)>LW_s>ezaqnfx=q_59J z4a2Z8U0*)I`nH|8WYeAnj+7l12k}4j3^yja?JRxbHyEyu4b%mmQU`}u5l~T4S8C*C z?w2bRaSnmwQPq4V2JyFm2jw2%j~qP;R_i7_`5*YJxcO8GvnqH90MxF7Do3h<53+FY zsPuaUwhzbrdGuq(d8t~@$2KL@hk+IwicCu$9ho3MP4-oD z%HY}q>gD-YxN$pLTD(aJdZa^0hHFZ|X?nSOR%U@GkG1WDRAE)FPOK`|ZNn4*>Ajl8 zG9zqa&G5g9aCljGI3-`IG4bpzHFbqXUfXSkzGbA!Jh}$5OQU|6%vA9Yi&?=2GEFnK z8X}$jAmxxt?rS7#0t)6p9$@?D%D#j#Rs|AUq^q0UNR!AkKium1dBr#CQNQ)Kt#KMV zAf#gqZ_hcsCM;rO;8hGgFXv164}Wz?!x_peIv=;=TFBpN9K7~C^r5T8ZRaMOW?nDc zFFg(A|qIHiSg{!Vc+R3NnO7*>d?2`(Mw67SP;}6zg5=c zXFl05q8sceD5EFfh$U+a4#W1VAwWP%w=PXhO{hmM5p;gdH*EP#*!2|v2EafU&YOLv zl$i*P#`0=e@)ofVxui#i9QWelLpPs)yOP`1j1mab{9Z?ridTW0cejcR#<#pkRlC5}~c8wlkIFlko^>0Ef zf~wavAs2INkYb+*Eo*{YN;8p6pVF_SxNoO)5COtO(Jx_JF6&S28NI$Y`vrkBqO^CY zrjGFb5qw=}o`epIYSq+X(k=@=8H#!kyt=tOYdSCBRsk~{oYnU*|P_H^{P4~9x@Ez_Lch~N6!*T zl!hh$y>XbnG`0{6iQD)1pznIG;Vj^wm5t6nJzGKan4IB3&^f&k5rNxxcdx#4-vKV? zHI>ctwtI5(fU6_6p!s=n&Hx59MIY5ud7E_ln&I3>%b&9GI4z>j)ALDr{BF>iJ;{P% zqVL=&6}zDWKE+d#SK6nuUK=m!aOo%>Vs@_vKlRcDJj(9QD$ao#do-Qmbtctgi?XhX z1dk0L98Za2{MbZWU6QiyX&6nU;qVq(B9%cqo?sLI!QCMoq%rj*2^X-jb1#d%*|#$U zwG-)>S2Qb9jc+^Zp;>EvDbrd6d(Lj=UDUcp9Ft~GZ^$(&=ZzOfG2?)>><`bHy8pVns1LBeY5`C*mJh>N zEHP2V5mJ$aeqfwiGfY5m%{$Lm;Akbte%l>u+>Vy4eRhb&nV65wTx>ca9LeIw9j!=H zD0-GM=^6jDz7}skV;7NfUrJWM!BUJt+i?7V^JJ>$ab&xJb#LyVEeOB#`6(dy7)~Vc zcm#i<>-~VLgo)yVfnl{^;DZQ}y9DAGW<_PwnEwx^!0D}Mc@Mj?cpU*1Mm2jp3twHO4q`m2ykQIp zeN}foKRvhK<%S|Q`OkRDyM&W^7RFNE5ic2*DT{t^RerJvJzYrBhMS+Bk+9N-sY`(w zk+V(bUf-cEA;cV7b%s5}t8#LgeqV$MTf}FoNXSxn0qs=UvL~Pg( z%vR%3yQt~yyD1*U-u=M-t*07;Iytks@JZcF zrmyCg4xOTd$1zfP@{Y)+K)^~o9Lu8*j+4SN{QqNlIME6m=YdVJzgGokXzb2%($8}0 zK5=nT6wBHbU*_^c2KQF3Hl$zAygim5`OG{aqD7+5bJ27JG*qm%?6yj-Bdudyxl)@XXc%Uod3K z5?pUv6jYLz(C@v+_Mj}vuClzvXwd|x1;sv)OSbig-P{|~?w&XDAqXLGpH|llKYhR{ z|9xb>J)1Q8`l1XrSI)hp{*%`uNXuT~!)J|--L{MU&p|shOHOILev^UMlWaRAKO+Sf1wrLz~_Uh#PeZV-GX7Ko~@_M0v;5R@pkxcG2wsQE>FGpQ}wqt(@j(! zGFm2KqHICdtQ*pjzDYM%d`V zMI?kVV5X9gDbd^u>VsxRcd`wsS9^`v(B9OcMScC*zvK1Ber_vOr<8{8Z5X_ibUail z^^x=@wNT1hjgZgxJVf_XSYyQW8af#T3y)7E+7G=1M7xRkI9lJ!bE$b7%dg9o)so{e zO3nXs&lheIpy|$boLZh9p2joL>E!+Mw56Gru;BX*iH(}$aq@HLC!dSpbd5DndS}2^IMpQqX;!7W%y&nx zj6-JWjLQ~;!T5hAkx1`FFeXt^5y%t}Z?D1oBT#YXKJ#g}29QSn&4nne@a7_G73BY9 zviWAtCN3iEg#ZJ4fB1=Oa> zi1Aj*f*!XGfiWi34smamsZRWyK)^4?UVlJug!=1kcOA0zP`81Mq!Q`&~QZ4pW@z} z-TO85d_N3u!gABmMr0xoe&jJV-O)LA?2Nt^bbE-+yYw|yb$mHCg@eJjVt6^N_SnFy zZ*b=@NHw|~u7CFxYu+p2K*x$ZY>a7a}JX*wA9JJLn@MXiP4F2(p@G6_m$@Pcyv#SwMe3aF(1TLC_ zap}Gio~^Y(?ob;{>|(G;wQ?nV?O3H4=5^SR$e4^WBl`~kMcW4H9LywSTo@ThPl5ApKB2`T+`W|1HIv?y%j@a_St3I_%7q1O>dj z9V0|^`OTZxLp1+?wE!F(xkA>Hjj~X~HOIX7JieZDk)P3lTW}1N^{$fCn#MP1ULI^}-b-*yrjLf{l~OXA1zBne$)B$+u{4 z?M|d2->kw#d4c~%tVAaxQ~y6v$|64@hRmh?x7{^8CI3lr^v(v7%A%=5_A%|e&wqT8 zG$pF5zV}H;_&s;w!*#JdI+35Qnb$9%DfoxUY2?aj7q57|9RtbjLDrj}>S}87g?4rA zjvN0URev26*V8tS!;5QRad!w1+}$-uu;3cp-5mk}f_rfH;O-8=-DPoy#a({;xxe+) zTkpSHwN>ZLOkXosPoM6Zy#=W|MHPg8{WkZ2%j|ODP9G9s?~}eKjvx9DJ75E7ly8~5 zO?Dd@pFTyqO<3^QZ9pt#m6BhPl9vr(_Hy<0^aM>#2DT*-C@-t*)^J($9J|=QlpoEk z)MvF7=#R>eG5BN(`WwHGbqne0f>&u=NWBJH9sjy$tCUm;DP{83ZlSX*E-t2@SJ0U4 z2Rv2E&^uqG8NSK*4Y)zAA&QBOuJ}xDiiF*7mYt;wZ!G@R8xLgG5%PZUzpOA6(Q#?< zh$UU)u_Iu=JSjl%=xlp*Thvm{>^o$zAj%Iuh+utQO|_ zFLFuiHGo?WyIB0UaWkk6V=vwYSx6%X@d4g?3#Yx+{%?=*{hi+4uS>+Mjh0uZY&=;X zUY|aF()me6>^8{Nz8|QvbLe6|Smaaw8I6b^2<2`M+Syh8-$?p6STrjd~;cTIP7rZ~+aGQ`LOI5DrlZbAm-|=to?|;mA_BdWYR_K37l9VuO~~ZJdIIz|H1&Cy@P-V<>NmPeS%LlSIF{& zmaOFu}3RXt3@Vk>KO4~^zI-_l>89Q)W>AH(-H(7X)BK)lnIsZ zgTl@Og)R!=HChrONXF*1kS{|0naQ%pILMx(bN zR*;xXQ&DjuKlMXdcL@knkz3kQeo}Rs@Z@@)z*lb}0Et7qvYQEbyYTgsFl!ceIT-ia zj>yU3che#7R_Yc8dDon1Dhc|Z=&7NSpZuPDddLizdU{Zr=(f1Tlg~7AGF}kzF^3ZUU^a!8i@L0-5Mi0k^%&-k<4xRMlkMg@z9h zy>_**xu4m!{)XI5w%2o6@+o(8>(#-9nfqNf35L3};s+Fr5BB>-Xlo?sxiq)sb&@4& zy~W9|0DF^ddrTkYwwm>CXEb0@|6u?JSB*YR8?5|JM?MZdT{9Um|?@dJUg)viiL@))Z9%dC zP|5f@KBrWz*X%VO&EvuHiDbDmy4QIG84m}0@UomC?XXXFdP4l*mD)Ss^U!9E%2j=Rzql;CUiW3R8hB_+0r4T%D{Bt!6en z|LviMu?ZV%JObrLvxTNY^oK-860QYS`to8ezhK;0jKq=Q{uZf!RysNQw%cgU$s6es}vnQ?H+!;(}w z5!aV6MyfF8r%9LY=c#0{0};o?;ziMYmk`9p>v+D5<9fqI>1}wI?3`E7`)p#?*L?Wk zE{4I!18ik!J!=_B(sjP5J4q%`<1u9rO(Bhv;Jy(gy{{9hR-$lhb?X?-W7zI~@mmKn z=+R$_it!GFn|;q0#rmZU<~mIuChb(z1&#NPRBe5)o2lJGt#gC`NTtGZ?~cm&FqU-PaAUu`YtCZQ*Y3R| z9jk<-g15BU#DTz2Tl}TMqqW@f_DCg~7LAzSmp8bm^X0y)eEr>X_u0H+=b~(`<3VJZ z`#xpuOJwESLOW!flFF^b-a6VeQ#S;f5Vt0UUQ()bd~T0wDKs_hAB_}~MrhV{KSERG zecx$|uEO*;U+j5iW#f*_y@u_1HY;DMqfttvD#Di zMQ-NR9N-Sva8d~~xKxN!+fsfdp4-Ua>OD_m(BzJH3{TOS=2m@e;XH}CYJO_L?%Mvv zh8J^D$>+=)E{~xuXg&YOxQxgDq)mm#jivGae1)KKtXLsywF34~_-$a6>$IB4YjCSdl&sIvZwQB88k7}Fm z=}^wsIyuafqi$dHo9s{B3LH-2duRko#}6vjR6=>82-Po6&wKKys*VlnY-B#`ooIci z`yVnD_zEFO(L1cSWPA~?n!)-h-(N-c)Q7^7ik;+K=)Yg5N&Ahs+vz2SKTE{#If}9@ z;5uTM`=Y$n>Wuey=30Z7OUUmq1D-mI!vujL5S!DZztk3K^Uj0zP)$stdEl_l-5ZFMoKSs_curltk|KbWfYs9p~`cQgtBc zi7Ma0_B}N)*933630tERb2b18(MU>TbOw0>wlxI2-*?d$+q&M@IsadgWXFKkxw9!W_4WMunLd)>}uUw>xJx- z6JbA}kKw6zyOD%J8@ed;o72&tQVhwW$Md+&+cH z_oL^}AvTMLaA!l|cnv zUKlU!KgKE>T|@fK4X<{hFVT|5k+@y1O-&TiW+;aGdOdE?TGf^hn{WLLW7u_NyCPnH zi_0zqjhnODAEHQRnXr+AP- zmE6xaL^^EJLQzFgT~8jj58CR*H^yIoiF+;Y9e+(_PX5GnGBWY@f-m$VcW&4^oh4I7 zNy%F)PtXw`DE@=pyAKVfNbeb=BRWJh^h>+P+WlnTuT_`BvyNz8_x0KC3~^~yv#e2% zNG2i|zuffRiobYCmS&`^6xPAHzGC83-7L<;-wbWu$? z&sKBrq2%^^XF?X>99o};yFD(FIZG-4tK7JoUQ<4sVm$tEejqcjh0aO%6wyPI41cpR zY8)q3G+2X(~ zViLIY5Aq^o3xfG%p|q&Y?(m*(UM4*KKCU*UG3zS}sQ>*eE?GjwW3G_&zyGUf6%AMEKn! zvL%Gt*zqgM7ifd{3d*;D` ztZWl_MZ{k=H0zf<$#ys9^}`+01zw+5(+t+|jHTBaXF<%aHBPkEZ~mPDL@pquHV#f! zbLWkH;YD9-alKFZ13IG&wqm}C;+?(0qvn(_h|vq(?>vYF^j@TiH{rrI|G9l~km!G% zpDXISgXaU+5?8XzW?E&h({V`$fR~sQn_3b|Jye<3DNdmUk0B%c-=ZampXiX-+A690^Qt(1WhBm zfj9luPTg6nq2MVdu>%%F8WgGNm_aKLf&@zZ^K(2 zA#nVktK3bP+W}j=F+?sVxK`f*J((keT6tAgAS%fo0HyIB$Tl)oGpcY@K=aJsL9wv5 zi(BVE0Ohi~gO9GOzSGanGYv5!!&s?WpVkfcoIGYQl$4Vff`5>~AqV^us9Ei4(vJYqqPh`TrtB)C| zA5u^EPeNdh_T$i-?8$gS_YKE@4k?*1tT`h#|3cqZ|Zqo z9A$<4zJJ4oCV$ef?Ma1azkDmj#Tz*U!W4KWK#6*AsFnt1uU-@=FHfv`1tLdQZNKbCM+@s})_&nU2 z=19Qxg}|E5Z#H{CW4kP&mv1k{{^5Tgg3Pk?7y5DZEYMAtxFm%b*ALZ&r0Vq*pq)QN z@c%m;J~aII8U36&gngX3ba|deE3SE5u@RBU05(##TR#gy>{Fx4F;z!9(6NUJG>9vN zT*|ik6-T zef1?=9zbqs1#sz)Z58fP*F(0{G}nVR!uIzWBq9>gnj?Q&aV(<)RaF|`ExnHAI?#CC zH<2u$?V-@o?Cj3k&2`^Tzq@BaT0>8*!;gh`H{U{Oh6sx!}qD zfZzwwa8V(TpPsSgl|9GxFi!V##x5Me0T+6zwCmpw==)n(LacWb?BEMfdtIrh-I~g| zF%(K1I`-95G^o~IvUl7ZX%r9&N4)ck2F0+6ddP??@$KsV#0jRYbU>UM2Jv^WUdR7C zc8fQq(r+R@)8F_5zrZ2Nej@UJYY-TMyX);B^Cb|f*aZ%kdz}CA9=^+JcdPgZ)&u4X zy`T#Ke+LZW>+UL$3Lk=8-h~c90zKV7!^!KT>EFd4PSv#aa(A8;0I>KC)sNoLlt2s{ zm(E<9TUZ)qB?l)Cd*N_P?ZqT34%%<8av^d_W?6h4b44`+cw)O8G)B1h)N0E)=P!eL z_;k4hYBq|PqvY8CzSHcA0P@2vU9zY-3IY`BpMG{`P-~;K{sC2J#)g5j>JGBZWUOP5 zSJr>*4?$4^q^U6?8BdFycn)wvScVCPp;=8Yw@3s$CjNNE2Izl*>B)z?Nie{&+E!zF z|2A|uq%%4;=Fs(Y(0IC-CFoTbs%`?NO&g7d0tDGbocww3XSe}(dmJ=qqccA4j`1J- z5>FwTLxlz~ma5Q;>++!3e2(ljR+jYa52pybw-qupjK$v>fRa9wr2KT}hG867BRwY7 zUejPZnp9qd;@@ntmhxF$@M&@l-lG{HDhX3J!bLv=&1jTXoK6*ug2$qL`+TZF=Wwur zb&>s=T$1faDrr-B$fDEX)*q&VN|9elrQij%9nP1c+v6OBpZEw{v(kMt(*QAH#uUgT zb0CxAsHf8{fCE5sIvB=$&Z~r#zpHAfbJ-og;_bYmpA>go{pPdH{v!&&W)EVfr!Ol{ zZ*zTzki*F>d%M2YQ_dYD(cKb(&{fhdh9NSju!X>$%aL%zRn>9AMvrfNKLeU+0CbYj zxIfI{8472h_n5Or83gXit2SW_w1vgCq|ISnMlui>hyEnjKr`w5G%u_HU>pIVY=GQv z!Qp?r-7qM~ChN-hVS-f2L$*Q?<5XZH$3`#i_c|)S;xsWfk9)IQ&8khk6{`zT6wr!B z;le=CQ4$LnALR+@+MJ#a&S701;_ckX=VCO}z}wxzVl+@XrRkgh@Vi)V62t_*T7-&4!jIYdCxs8lC{5>sK>rt55 zd3g5bVgk;+EIw7cr9R#%egwkSlor~`EzIxV7%sQDxkx4}KGzMG{BS#-Eh6h^eGd%Gnvy$dy_c($@`t<6%`XqBO#)c#Wlf!$u0CFOtBIn`J8VGOBT^PyBTF#wX=>9 zFfEc@1~@}b5lJ)vM1IMwU5M0_zEBIn@oRxNFp@x-vv5Z>9~9LRQaMDiB@J1rjTp{Q zeS{-Y3~;HR{hpdAHZY%uNU9wW?(HiptML>Di=L`k(88`Q6s}ffX2yXz9%LWr*vTnP zys1dwB!L3_LA7rE6{vI{k~$bTM42Av~OYcq~xSzlKsz zPYXZVG8N0>J}p)VYSvO4SBF1+q{`(#zTw{>5y`cm)7+f|+WCsAt|2;|BB$o za8PCd*z<$*7Vjt8*$O05WawhYxk;sg+y)r$suw*rhiHJP%?V9m30!j9@IoosP`3$6y?2aB!&i87=dSkGt8_83l(cH4--1=-dE4)pe0KVWnUO61Qo|0gizt7 zEiJs&o#_$*JNJhZpP zRsw@CR0@!j-Cue?Uavs!3(aFU%19h^csD5WUw-MYS*#7)z%Ij?%6`j$zm}&AE)8U) z#)wbI-|>HYPSXV9Ag1}UduNb~%GM$)Riz7i&4vgPNeaZv!UBN!dJm5?CNN=WYEMsj zj{fTE3lL_`o0MY4?zQ^6Fjsmjd6T$@Vip_ZKd(UO_8CiTIG|Sh%=;~pc;`xd#-v$#QSUQ2h6q8Oh37$Zq2 zBWPIM8=AP=>OShP`+EnND)W=IR9ru$*Oj#(t(Jl3TW(hvI@$JF4ngk5?ywRXiO^z{ z%|VovNfnWCZRB4#x&!hRxWWA_#x*nyh8aTJhU;raGYsTq)5B1lUo^jo1^i$4w&G2Y zNBPy0_M%Dow~l6Z3IB@nNhBmk1bmSnmAEn8irc2yH`|xWi|>ce6n0z0K~YZgEeM~a z<|&}{j}VgH@0SiACu=xf z3D>XHSoc)~NhqPAMOiNQ_TN84x_5LRvQ?33{G^SM~Lp#Q?N=(rIV5?{*zBN3d zM{3@Y`QKdj!t~9C+wU#cn*0niw7|h$&J*m9{aTL#RKt&Zc4z3mdJbAxc;N)v_?>x9 zR##W|dm8f~N30%0Kc1 zZz0rv_jKV7O@kElKt=rj(*iht{Ua7C$MEqC_P^x;eMHu=dOmLPBl*kmPh_&$jl1;+ zw|xJ8useG^2MsxInn~qh^dn>n`z>xINEb;AqKiBqx^!LLQSN`4{^dcIih1>GrZ(vi zwk?#Gn%A8yRTfs&@i@KW1quN3l%tj^1)>g=|6=P{HZ5($gBy{4--spOSCj^y+Q6>a zW04~hHz4@M6{sTkVvUp9fvkan0sBqZh^bx$k=n6L5H1yBJ#Ct1`LI`;*e@Necx5BZ z98rfNKgz$-griyoE%Q^HA<3dFH@oB+wDYRak|YUD7#PHAc`&3z@W#kg5x$FA4_6AD zy?UK}uEy4g=)JysIvX;?f_P3 z<%I4FNJ_)dpi@8(A=}Vd70BCYYntfSrw;RnVsx>FTw_rH>Z6C5E^+XmkB#{Mak|OQ z?{5ju54qi}?;8ty?X2g-A_vjU9lV0(%RTAoCeYmNnLX-Lq=|c9-G8WPw~`=-hTDZF z5OIMpG{jW41cIS@5(7$0OaHEK)zGj(!+*Ek57IXZ>uZwox^kcGGNj{p1%@}Rmo{Hq zp^Loz2bUJe{p%QX4Pby+B`ti}m_awPu4(@7LIui}yK! z$d871Di>IBjobEH)AB(Qp2gb(B6))ho{w2gVtHR1_&@+EF}vsGRxcy9<<@w`jb=7l zp3|puOflsi%Kv`P67XQZd4GEe`$VLnr-wR%Kn4Tr&e<^3@zEyFSbI>MYN+iAdgB0h zfAl`Pl-z?W?>EJOGY5f~7;K*@M1LZn+f`Lp@VNqeFkxXOf5;PyPnYU7>>g@iiA$WX zG;iwh+pRW0#Muj(xvmTq!ufIyNk9P`bVt6hX* z#dObMJLX^CNQFA~Vj+pKpU1SjY)AgmW4+&l1ICTcm%F?mhH8|mhCY**0T2UkQa-QV z-5l}a&F(Alb03Fpq0=CQvFom?jbhsDtoFN}V|9iUnpmDfF$zoAN(u@x47Njj0Q8gc z{uBiY5Si`=CI+k;Ngz3Ot%pWs`QNr1@cTn7@z5tSzsJoNGIl!XN=X}K=+Q)`WmMdj z$!jzD3q``EjO=^pOhKJ`znc`WFZe|&#df1^yYy{Jqvt)v9Y$JrIr41W$6ET zTbCPYC+*f3^3|*jCD=&FvxP$rMw9 zM*t(dgzl4SK=4X$L38_kTQtMCNmw>{nY(&OHd2Xq+#` z%M4#n2wc?8=f5_mrceNa^5dyw+|O0te|A52q2K~?1f0&Uh664nJD5abcIiM}%J^;r zN86y+w^+EpBx^h@`tRm~M*MCk^E~fo+uww$_#C%-uA4VwBG^NPO1@=1*Eg_A93JU!@0I&wY6p1Jbr8k`AWB8T=WRT=2BFzJ z)FCF+Y7$>W9R3>6QzGlZ)s}HsDCV#gH3{GMBAhOEd{F#kTGg+09r;@>5b(Gg0E@Hv zzqP3W09ew(A*3+*Btl8JKk9-M(^)F-I9v|1tsRqUF!5%-UZT${`v@!MNOTEHKn586 z^f=V^G*XhoCMDgo!#e}Z9F!YFvo^L^G?qH(bUbiO{TdO$&d%d$mR)`6gar6KCI_LL z9g#+m?X6-5jjn&Od};2O{16L^^lS1;BMJxDGq^jg7dtuB9KV|N(bUs`82?i5CWJ!E>0$qXzWJd*{@VonS7*WOBrD*dJ7CZH z$@4ADX7fY&C#B;jW-Ddt^nrewuX7f=OM-gkd`*5g#gHZ6?xa`OU)C}(t~BN6^cDkP zX0zLPb;4w(Fz=&533^NZECzsvTFmwrepMb+U%FUpc6icts9%om>U7QgP~9m$iO4O6 z1bssr)hBbor2_gXWzYUTd_G~^piX{g2z?*|Ht7TJ%o}Y(Mf`opja_-Lvxu-r8A_s=Cci3U!$$mEXLEh zPXCRd98PARV4}5)BkFhwbYcnGAR{5?o>gvYx;^NBnRPwt5re9J$Alwf$$aQCyzU;g zzlF@}YAYDg6GZYqjVa%UzxJz7kKsHqKcXmfw!~+iCg=Z9opIUie@iv7S4kTez4||( zE3{JC$6B?t9SeY7{DYX*cpNL4#kIQdPO2O<> zqmO7)lT1SDF&8ZXe{1kCISZ+G|)F0bDrV?eV2Lr^G+0Y#AKO6|xMyrw4{c{anjkbJD~fxwzUf?uwW)C>=z7dUES4{RSnnv-O0i(5&F043Zc7o zm#J1}6d($_HX`N@VzA|R=3fKM&x-}1(v_?2ljanGKNj2uX^Pf?m^=6Y4y|C=p1{%d z38NtHYN$Xs6<}06{m)nc^3TJt0a$*(=BDm<@>S>wSyWsqdbX0m#Bd-oVyh*LoSXn5 zQ*GqC?_>_i%6rLo;dog9F;=4nvkx$_y4=Tkx_xn~Bezp?%pX(;Gk-qZ_1fk|mO z*eCyR5>95N`r|J_3>Aof03#}!E)_!0D0&4EATL8XjA_BY6NsZC(oriE13rlb>eXP( zjHBd3Zy9wr#ZVeY36kRH)hz|CxEx@IGK$*`4aOG-d<iD`3rQ38dzG}uwPVKtc} z-U#j>$TQYtL}`fhns7hlixo_|CI*b#b@aurz1?dKYNQ$Cn5WPmv8buJi(Y?TqsH?%+x>vKDw|F;lfL#En``jUo`1| zHP42XPyx&#@qx=?j&O_s?o?3mE>bWGaDV_vetVn3sZ#~SjAIQ>lQDud^a@v?CuoUc zM63$oxQyL7=AZ|v>|jFfMlo5v#rskQW*tW&PASAATd+qj1%u=ZWE9hQkB^7_SLc-v zb0=i!RvhX3Z=#&S*<_6OzOEmZngIICf@BN*tVC719i)js8VW`2bQ`rO2Rf!p$8V z)~hDU2sMxBk2|L_0=8|N|0cc> z!U-{iHxpYuKMdyxE?Q3cQ|ILn+Ja(BMxN$w3<;dR%Hm5(^nnqzpjUCTf<+EE<~mHY zIp{IJ#;TGMi6l=Sh6plQeul!kVc+VQvH*f?H=>fJu`tQQ@g<|Dy5bT{ zKbQ}u=l>^C%nayOcn{`+VDI&YFBbN0fD_+A#7>CfxSbqHb4K7*25ItGj44kr0Npm2%LXKB#nkF>-VkGWli6o9m25yXn zLJk@ru*Iks8i3DfIb9K-RomPpG- z+~Y6~TyPNXO}wRwU=iiuvn$v$u9Rq zXbUhx<)Vd|3<0id0jLH@|r9P%^VDwRd>QO9$?3ON1}fGnxQD38pLJ+ zwmax>^P&CpLkz99%lJ&l3o-8Y=6<>T69CG`L%c_`yz^Ah{@YSwLkfqyK-HSvd~{S2 zdx(Y+6izrS*PJ8HJXSYJbgA#bTx_N6K3^vsSTP`0S0e^UFn|+zJ`1TauD&m;Twv3I zqQXN`lW-36jk)#p#{-+{2+x+JJ_!`)5!|p`&t7M58~O?JzM`>e2Wm?54?5G;AIN1b zNm~?Nq$`IayPgvpX60tYxHZHL(cSh?sR`i|$!bB;&~1BT)RAyzGi{6#mLTcQLeOW& zkt;NR$Yl)ubK22u-4jc6VejXy8czC=K>0Mf{*C}>W#l2ERX71WhlbmJy^d`Wa19t` z@{z&ek)VquN`H;5eP=Y12NQ&6j>Kh`-#OFx&`#C+(}~*d*XUGL01qvb-j8=7Dl|Q$ z_YI;mi_z|8&dqs{`Ct&kXm@>lz%#$ZB?;gk=gHqG#o&o$de{S+cWPO@H}z%0cz|}^ zrJ?_WfRK+NoX!&Qa{ZRa6ePY+yux>>JBLG-z)hlO7C6bk*m8eDicI2eJZuwVQ0jnN zIM(54`$tTE!rMs0#KaGQnHXiOf#88vWqlP6hl2qdTY4+>=0F8^AxfUgN&FK4ppYZ1 zFFD@FMu|qqX}O%+3ox=rDy^*CXmzphUBItnA#T>zpS_X4IxO~cxP}$7kcj)Y+@Rn; zcJ2~_C-1Ku44ldnTYK!lo=5w~K`jc{(DY{w__M4P652TLceH^jnan8w!a}79~1++c|64$mw+&bx*Xm z8Gu5E?saPH-Bc^HbGCoMR|8>_ix!K9E)@+X%8QBlvmc3~pcF8v*)+Mcq0kXr(T?4G zw_!i@qG8fC-C`OWj&K0`M=Pwg#w12qm-@Y#Kl)mi#VR~6uD+zQ7|6L=TT969=r*ERl)}3Nr)Xx2b-1fn+>}+}OZ3nC!rX_5j$I>mbMDoCnF&jbfbh14iA*)Z#b?)tJPQiCOIF`H*L$y&+As66 z-i6$-#XdGk>Dk855bSIO9tvVk}+4(bT7F~ta&h`@? zA5ts|`%@U-HaT`h+q(t$)AwG}0h(cK_CZ9wY4mf>Gqrl)p` zW1iKz;7X$nUN|ZN`@ml+9}&?ICFQFmgW)?a#K-YYuytpe4yxe zgf0?yx;4Ak&rg!r>u2WtTyljyP(el*_U!tL=;ke^&*NJJ0eurb?J2@#W4DXzIIQ7c zCur#21v3Oi^46pcIjS9f!luxep<2EmHL?8cY*zRhbSza#f<65C{ZFxty6hP`FQ0WAPbL^ z4!qx14I7rX@^3mf1_OtL2R(N=Fq$PG>O|uC1bd5*u4zCj-=-xJnXsMEljP&&@skHj z_O099sQQvTS2vz+!wjz6)!8p$3EpW;{WJdl{bD5|1pXLRkvC?!19~2)xgwnvTRe*V za7`Gxs;1vbl>DV+!>=fOy7WaRl~)e{U(={IimpX^6fS>Qh|!7F33 z?e1WBNf(tW&DG4hL(Vz-$R=_X_wd;2(4r>XQcq)6U0F;Tkz=xx-cO2lBLN1)MNM%( zUhIqta|UW@mWVMtE&9zSqAhGGlNz!CGWeP6PEHqJuWt&vytu3;y*b>Mv~=bKpT7#e!#=NP$L-H9h0}Os^(PW% z7v5(?ID0y?dCoSiHa9gblIq&%6CPhr{Q{(w`dJdZsuR99%<9bck}u;T70$cwUowR` z@cEA2Ka|8cFsb<5iG-osVDc?`)Oh|m@8&!$>LjVPASX5s7_YknVmdbMOc)K_9MmuHT_V(vpErhUE)va!GIHhkkcB~N?3Q% z3v@EbU66{0iOPI8WL)dWi<jwSd*On{Sh}hI-MMq6g^(<9>~q>c2R*Ek>P7)QL?;e-yi>j{ji& zNJ~ffHkKpNrl2#{W^o|+`0YN^x}$o2z28x;4a#ABc{=aE9JrE3|O(j%f?viVMW3ou`ZHjMV+&0~Kq^5%n0>N`N z?UP~!G?P`hJf`$Fk7piUo;&a8>N3);t(z4nuupAEoF?^$L8e)^3-dBhh^cvAyzTGm z^*~8Vd`STZL(VsTYRDia{gbbupp29(m*+;P z3S?r%UzU8;kfS4ByAwG|k5LhLTfZ}ltZ9p%^U{<^Y)C)*4KP{wt!1l>Gj9)o(%5HL z{LSh{dwJ$K*2G9Vc;Ixm2TE*rt*r`=wI9y%q^*AotZ6%}tRXr7DOWk>0{3<+8ZuqO zugDM{b|jzcSMI-xcyaS~l5kYHzQ`#LAG^%@^;$I&t(@T1EpaJFex#2EpeZ?fVjj_j z(c*ngrwxA$JG{SAeR3PV^u~N#kYi+~yqp6#v$(Sz80SU6& za|?*O^r;&$Kne0@x^|5;8#b+F#KL=*6h!-%6tbGw!lJsLN(vHbm}1+`D|qbUw`mPg zMiW$`dAfBiJ1vW zD=^A#_K?``oj%MVhk5_Ri>ynWj`ZOcjkxSbuc8zoHL(CP?O0EId>Epl zk|7#>gCF8e% zyaf&hXpw;?JHe66pNhwv(TM{uSB(xBR_&42ZZ_kC8+Wv#gtow8c*QM#*{jATg+tx= z@LYo6W=L{!R5)rdG{81fdx*vdyuUZh)N$OMtEK!idTSMv#*w695WF$vUVJEy<;PZt zJS^ZA3G)>(0$rse{+Yw94n>Slw3yMRm`|=6Po5}?+Lg)+Q@08+khfh@4668GM7LPf zKC@o)pvfTbGvXE~0rg!})qK$sj(UoV@^?d-|HP42q(eM_EmX3-OLz@h<1({-uU3!H z@3S(1JWljWwag;orr{s%A?gXFi=du)0_>Tf0L82oNPWDdwsPeYY&G?ShAy1Z*wV?) zRwl>8ISLKt{(~Rt(7nkRE=vp~pVPC*A77xOug^KkpLoV9;q=LS>`!SG-_Ava8BNJc z4e@1%vtqKeLIXKJrf{}F&?xtq(ps?CFq5j4@E11Gm45RZ8ritW-X)at<0ZiW0tS@$ zKTd&4!tai*$NW*@{=5n0X`4|*iYAhhH^#-JFzL3?#NV}}iYjF>=XJ$^7{N`u(oF`= zjYE9TsBwI3&?>O53OB94EUWz`nBV%GrW5cc47c=SXkkN8%Zsb3CYkZ4+i5XKHXp^} z*Yviyb$1L_{aMm?w0?y%J?^In{Qmi!dntReJRBI`0DpzYBPwmIs-8&uP=?rj`C3gv zA?D(YkO3Udpv-Mqn)=r7jMV z2zF4?MzHs-U%f@vJlA9?p$mIF_KFK|*_-j$BC=m($l=2gbi+~q8O5`TMZxlp?9=+& z+>VyIub*0+(MTaOt$mmGo?T6}HFf

    W`rUW7SYzesQ9P;eYjHXalv8FUb-iTp|bG zn<&@@=#vFVCh&v5!XO<{6v9n2{o@Oxg&N)dn(;LLjZY9MVdk)%R_XpK#BD1(2OEwX zh3vQzLq*WpGOcKyrC*`a>ptUNd%<~(U@gqlDXsPXfP{MY<#>FcJ3;4zQBqVyC zkMK!__zF%;HbfHfkCyHk?$N<-%k$xzfR`DvLk&rr$aQmhy*4ef#G~(`jZ|>3bZLoc zz4%g+783s$|Mgut?`n7}`SeDz);T7hYuc44i+Ntw^b(0|k`r=<5;sg7&&g>8eRfY~ zecqOp_j3Za=&nBR@eE!Kq>}@FTp|IB3#}#Ko_>a*uhqPpOq&OMqfE^C{?F(Sw(Q3e zc^&|U&r-k=g*`|kKbaLNHx71HiIk=2Hqt*Um2ZuKYmy2F92yT?-uc6XR&1`wL5;tL z3HA6CU`!7E`Lg^`Dz4X=b^T>kj3<^LUIwjbBg(c-lr8TwG|-g)Y%Pkv@;*wjkDDA% zWXC~KsjfP&@FWU^4W;Z$s^*VB&vWCimm~JFx!|)Je~&M69vM~enHWS zSTuq##z&7GC9 ztB5HhZMG|w*CZ>n?9<X}eFFuA7iC}nNDp~;P)&Xnedj0 zyN|2651;2-t0LLvq>IsQBs><2@P%2E`Uz48sWQ z?|lo}eHJA}@rDXHBVbaWZ3Sys>a`FS;orFSPk%ur%nK-vJM39{VKdl7iwuO)gE#LJ z+;0|H(S&{W92N7vCPe=+Jv8g3hs6zKdK1czmQD@aE}?2$q6H z|KSreyfh+8pndCk*L3TpI?l5vW{tkF<;U9z*^?0ZcRRP-b<>T0_cX*yFSgo^J9?2w ztlNtg`@*;mRa%DhX{CY~wLi0i8w>LC=}%jgJ5k=G~QA}(bTj#I*P?VdB8!d zgeAtyxE{Ue?f1BqYi(UszhmD1vEw8B%7%hNeA@m2_ZK$EZp>xf0b|6PfXnrd2deF& z15SEpZA*p2zKet?HPJ{hYcfW17(XK_5IR_n%Fpq#rbl7QCX%TC0y_&o{heP#VcoP_ z^Y^FK>q)w$_92V!H0+&`m2-ZkMKB`Ws7#hK|-ez=QoO<&$cF_)Cn2_Wf%WpY{NoiTyL1``A3FtgqCFO(4h4QmT$#N$Dao~*Xfh9c-QrGIns)WI|77rrvu^4I)Nbmai|K==i8i11bdeTY%XC>dY8SV} z@*p2)Jc?8oIl8ox;iPaC%DWG5?D6z^RkD>h-)7_XT@J^eNMn?LqKQJG(I70|7c;6E z1}I3fUyayrkVGB>+Z?A`oE)Si*H)M~4z*L9+J~DfH3&lk+4M$_eizR!%}B?%JKhuZ zOf20Sila!Ty@xeJn_EbF$x*&RlO}^yQ3MnfkhYb5x_k`iepYPO{`N|K~ zeZ}8~qnO?$)R3dwF^~TG#yDUh_?p2m@WoDDJRug9LNMo*QSy4*Ysl*t+^BT84UB+3 ze9d+&tV?tv2ca+HrgzamN?vC4&B!`wE!E`Ez z?)vvtcQ=tcCGjPCL6E!X-zwZdo6Zn+thlX(6A{b$x3s=4$lNxPv(TI{t{sKDIH=`S z7biRGpflHSu8k#%;A@d3cSem!S$c{l?cu|juIU+b+CPe1%9?Ev7-QkRv1*GivYpg1 zK2Bh(T3B9X-4vTw#(-oB*F%v;v4MDQM~M)GoSa-LEya~HLu0QjT`N+up?OvXENp=r zCRAZ0hG?}-6m05MnrOX~Jcorz&hB>Jg99#7pRS*`e@`X|!PKsvMLasT10Hm4Pi7G- zU%`Rz&ZY82k-nsg{W~`mWzQj9evmKln;69qt@RYQBaNX58Ohclf8BV>6~w9jKHqSX z;9p#Y-Zwv6X9F)bhvtn1+kC)BeeqtOcO@L0^F`dYd=)MFmUMCQ^8z}4WN8X+4`yvg z-a0#1981~xdRX;6wnCO`9l$@Vb*sC~+B>~(eqED}gmnhwgYSk;+o4p`th;2*|J1;5 zocTZLaUtw3j29{9 zl}1C)7oQ$Gq|O#P6MuZ<^k35wdkV+4!s-Yzx6tgdW@hvTr>5toO^ z$$cH#fv+6l#Gx$@yyr%dy9Jj2`km({V-sMcYoIb|frPRo=leBxV$ra<(?oSE6437H zl-ub88J)Ly>04Zmujbc+Y_l|F-XEo^ODc)_A#xuc@1LIhb#?EkYi#e%_Mb17t-RtR zQ%cArm>(UDA!q876Jl$B#XWa2F`ti&5^%Wp5RYS!FK^qCO}D8@dY8+Cre)`cD>Cg= zJCQounZ)3mn*A6F;va>DTTmnC+&9j}Z>mjN-FvRDkBs8oQ;@PJm&f=w$4Ja=Gh(eX z0Q~SYk-h_3_Fi#8H z7K_gu#StmUVz9CMcBv(+#|CQcq|OWin4@rEw_9)&_)ami_8 zTC2snoAzHU348PNRhvFv7G~@THQ#e&U;hRB;oQeu(Lm(6FX+`Qs<9a^6Iv_3kl@ZamtJQ33m`+qLw9zH>u zo^gr^H1I6}K=kH{ia$+4ror}gTWcad^bgj{*OOA!X?43&Pd77PqwpaJNZ`kI*ZxN^ z;)xgEcYdN$%Krx2iWvTAmxy_&xyCog>p(6p)?D>tQL(PLV+*=O4N#+0?H;^!c)mI5 zXp7uqWS(xhpGW?hTwf~aW%LI$rKc3F*HXZuc#*T*UKe2^A_ zL#5T_K8oy!r!H5KvnY=RAYX`?D~oO^Zo1UvqKd*M@4;g1ZCcgixhms}la;o}pFfy~ zRuE=gyQ2U9EX)+1OGiv`;ynM$eb@Oi_n{k|K>TSYN>U{7lZL1#>@Vce@KinIe*0-i zrLLkhK1AMnc8H$a-#%Eoe7#J$$a}XhBy`gFamepylm1Su92uNvjkb-gsEm^klo|7iVEet7LXsrJhH!-Lx5N%oQl=S4Xn zxM}BfmxQ;?`k;G_TFB9Hc9+70zeHK+X-^;W>1LwrEeTs9<$}ldePd;t<&^@3?(d)L z;_wiP&icXNuFp>#kygXA&RdU~%-Zpa^A(1tuahTUh!z{^9`0s@{8pdtxW2ap(}IcL z-(|W_MGO&~Ep`T!R+CVuX1@uBiBv-_p_h6BLHDZYWhv_7&kN;adZv#hztK5YxpM>? z4z4td-h-uTwhq@HzdWZ9jQK;m{Ev#$I3=Cs;f?bq^N=XNo8EWgwkDxP6BclLk_50avses$lT?&;h>;8f+YN3kT zU)F{6_{H)&hiP7Y>pwcd2$Ka{c;3M~Ro$*fZ25 zhMM$CmLHG=>n(qGw%g?(i>H@)cSyt1_Ub_RSek?~Z~IQ6Lsh=_VFr(Bm%rCnCF8(L z(Dp;~$!dM6f-XM2NShgfX{WnFoVM2#C~mCeSqO4JFgB)V_GWk5g9Eu{LJe(c0s{3D zm4#*!&-DZ6Twq~%ipBoTVuSPTl+}=#NuBjfQ)OQy9(=?HayR`EXUz}MM-IkbsIkB9 zonKpP_gQbbzUK}(Kh-M|PrJ#hTUyHQl6W)|zfaZAsLXSRlkMweIIp%y?%w1EP7RRB zbULu9wmO=p)mv>FvJFYI6NjUCKfTYSQO__D_tPr*5_mrk351{OTCpL?dIgr(k+bZp zR)gRFw{n-T?P?w)vdvv!!m{Kd#s8wOSzY4wQ0ver5MW?>SrW%<`tTeuu8J4QpNijPWs~xQ@$@)$ zu~A@O@0XbbQk)b_3NIc8l!mJt{<1#$_wj*cyuF$i+W(d#t8n^S5uaY7=J7Wm1is}! zpAN0)g2u+_Rlt5J>8==SQ}2g8lTAP?bF|bxkDn|`bWGAcj4;zztxN3Ijs5lX9Y50L zR-26%nul`V=EDvX&4NN!KkKU;>u;W{)iDlCtaUz=szaVrzhb_`Kaskf{uqo{UY`?a z_?dgfp2}OH6R$1-4`GI^)mcO^W++98@qEP^8Kf3y6uM=HdMfkNdw-o{@+S0e*fu32(;{hmV3ztD=qzN%SFi0led87ZaHDf>c5ZpOHdX;vwdK zD~WnMh=K^^pXy+dnzozEA-a!uV|%yU&bq3nCo*YA8HtFSSs%%!mZi+N@)tO3@Qprl zE0<8>SR#B>l_ifcn_2ShlK&m_%a_Y3cXL69qt!R(hi5BoBD*?jU3SW8&P~t0&oCUt zeC<}ZzHGhbtDo76MD(WK0}a6vw-c2);wDG|)jz0dtceoPYgXiW5_m|4JWoz@^g_H( zU@5F0xe=uBAHxI+!NLLXo~;_Qh!I(wa(z{_Mn-wDoL!H$YGbza9SGBU zZ9flHCou7O3|O)b_9UPOJ~a{&`W=f5j>Z2DenH9lH3(0;r6D5!tcv?MizYHvAc}On zFL#E7C_gHHhI5d59PYclm0xy<-1bX%QlHP)gS7(XdVAY=)X61{3{0r(8}-|_?f5kC z>CYJgLC}#}Iq1uO|^2nAte@N%OQ@WP|5`0oz7<(iWFd_aB#YUPZF0Z zl5UTwe<$K!u|^k{+WrZY>R(Yng46{2bi3zqtB5~mfBk1=|E0dej}6D7(;OjxZf@?X ziuJnfvf568?^r~Kv_^t(?(wO8d^q$&-rjv1fbPK$P1U@QW9rbG_2W%34xq}{Lo(p- zI3W7qXO45*<$d|1{O6A!j}~pgqV5$v`nr=(hY@tffCB$YLFYxu+K@Ck$9i@G+J)u7 zN8n*FwV-30=LwjVnAHxX^U=Z2uX-)$evbZ&ahb=pOM>FP*_(nTM#yab7kSa3ccerl zxc@wO|L_WVMgE%5%n>YIj;rY`eE58jg^BLD(H*8p=U1!_HoXfxokYNPc#k9{9Q&(X zRFn$8V=Fs0Wm*DNUqUaGfBv_6m6fWI&1VZ@Kg@WeKH1xLi=gXw!B;MsF^~BXZvUGF zJ0W$X?LriA^14HhcNfU0&^wq(zNv>AjV*aX)unbrEo?rXGdWBDaFVHA7dZZtH}Rx8 z+}mO(UDfpC$J8;Srt_@7)ftThL<&Ri6+&?M3%ngyL0{}0-OA>j)`DjjB=KLPO5F86 za^(Uoqr-oFeL!#D?NM5nyv58(?`Y#^A14Q(PYbh~b=s`{oBA39>>pYC*;Z71@3hcx zc~v_@9FyX2w&mL-jLM)P-(WTNQ^&(2j`LX11@gs_H|je|tG_qRWVOwf-?)PZcm8~Z zfe-wEv1hGeO2aNCI=5wB&sVb=iva%YPRB~405A2b|BohiVAoO4fbQ7w zCnS7sr~TY$h?DI~+uHoM#atQ%=jgAGkXoPuZP_whORt6P{$`h4ckB(mtmf~nB)%RA zLyt2J+TJs0W+OuyA3q}52J*KXeUjeiHG+1 zINtBbSqGtoE9|{gbWHND`+Lj`W}J ztKDvwyfPB$lv%hATgrRf1f3QoyUfyXqz1YuuGEpqd~k5IQ>WC&ZLsqQfvxT%pAJ#& zWVwD-*L~roOXqd^@xirX;^RSDYZnm<3HnPs!_J*V(~nj~W^{ue(J)S5-7MB@PS@%; z#~}o&C1#fX5zdKag0L$Ledz|b-~`wCw1%fmNN~_Y?LZn*O($@d#kbFLAxSX*+^>=q zyScWuR$E0IROrcDyTDdCm@4j4wUTOKnAdw=e&k#WXeW|5%GQpRkWH3_m;Qd0U_ z2ITchbeEJz%LH1He>e8^GO`U~7F+$<4m_Of z?DmDRS6X)m={|);KZQH~2#IrpGyfqa8YkNB>pD)!i11Q>06cjzu_{y@o@zmp;*-fq37v3EvED5*F0gl>#tI=X^>*C*RrDiI# z+ptZCTH4?=9aJ?nXekq}^~x&`btz{IT6o%CVo&1=SWHJ4c*p6hZz+1CTMEO~Ll)`O z1nb<3II$Wi^|SBJm!O?d^I>SY-)T|?>%U1bG11WdWq%`2C6Aj|Zd-a8cwJ#sst#T& zfwKju^w0&*Eb8GZKPT2@FpvGywHc@Q@_M`pDb6* zXuzm!N6u8al5OCl29_++)nx}1J?c6%BRx)T|GC@EWT%Bnki2PMN5v28f7`VLeTtfU z?fyMAlix!2Wqxx<5e8LpBfeW2$G7qeGY*W<%D;mjQ)qd;*dcj=9XdPBY8N?2rjK*9 zTdb&>;-+7UV`J&grpC`xJhg+{e?!v`?SsdFQsOLt8?i!PV{5Qb**9JH-hO^H|MZuq zFW3VFbd%`|gU_EIlZMt~PHg&3WY zwQQ3w;x!IqKJ;rS1)X=*%gBr1TA}|-Hwsn-V6XIE0LkLBRw(n@sSQ_6xV3Jkt0Z1zjTX9)&n=z{9U8(m01BG(&IP>*1LG?=V81g+O7#Py= zR05Xwc=MjS{ch=)O(mmB^TfQ|ZQM83v8ovadMQ`gaH(hE^KzdR(PL+b)14 zA>$o%i8Ab6F>R&s4A%+u6dJmP4f!dRJIW1G-Z$iSLv`63-5h8)juEZ|jE=@Gq-Hhb zBkidN*C$DTBbEEwZ}C#;9ti%_NeC5*hG2{)ZQHp|aX8{ONI*@Ac(TN8?$R5Fj*nhX zT}h(AxViCehYUdkEO;W4t)J6$8yg#O_9YvGPmkD4-d-=+TVej(WMqn!@3Yd341e&; zF`@$JNFiF`*Q?ft+(M>4Gv8k)K5b>N3{vvlwMtc*Tv{l<$e@IQ8gfSjg$dO^VAIgY zDsd`gs8KaLTxVO|-lAoEWu|RyZH2eOc_)l6EZ=GRkB!f-3wsP&A)Y_Ju(8ogV8rYX z3edef7Oyj7~5D=U7{H~~|_?^}1y5@4#L=hSt zbMr@Hoh46?2`$@j%0YFGWr?&VvmpS!w0^%B$>T-5;vJ#;MhH(*ReEFKbTfx zH57D_yt>m~KXs$_@fIYD5Y@gtc|ggonNrQvkph72ECBjmn1KSox)|Ky=*{4vsrMK= z+V;8)r9Snn_eYC9{LNWbwsf%G>pM|YI@#*q#$s+GjLpVGsUI`RB|Ft_K z0#!qo>4s~z+JcMy=njdAo`J5BSit6r83nx9=ql!=XRdkyF|LAbq`D1R-SXi^>bL_J z_=7LV_0jJown$_{bPYHW^Lxn6!PHv{6~kH!KIJmSc*>s%jyhYbKLTP4*FM1*{x=>%ze->R2<9!B%9y2-0Q9=sLEn21ebn9%Q1 zL4Gf#q&hAsY*p^{O?xOd1Em-z?HzNZ)keC$&YF%E<)RbGV^wQG8cCN0@22TC zufBWOQrd1TKAFhY+-B3&)78%au&Gl9!Tu8U^+c4_PAbw#diF;%P2)Nde+S*N{XZ-K z*?P-)4#CRxt@|nGf?GP?Z~4y<7sO=9{R8B=eDs^0Z&Q4vXJK7PaUz?I9XRE4S`gcJ znzp^7W(=-kr)f{&x}|~I26#*ZQC)Kyl&hLjGSIgJQg`waI5(0dP;WcY(#0=h*`dSn zS^L!j?BVX5UhERh#PEns`OB9t&MkYQCIy_7OQq9w`JF?FRBdNP%?b+(S|4h8@(lPM zejd(e06>(yv(p^?41!tSUAO5587frg(NhfJY^=(E>pLv_k$Amu7B?m}?EXWa3G-c1 zY<+$|dFu>zoJOJY*+pU9#TM3s4SlXx72g+|Z6`Bvc-hO(Rw4TE;#av&Qx8zui-D1)3$C78?ANujC!8oXEvE(sX@ix(I(;clR1M8!q%JXQ zU6|5qfCw`QAQmk*ur37A57eg6#_^y+BjC8Ic#egZE1&1~yr4v; zYcCO`iBWp`tcs|Y5T{>g4*ByfrmVRC$nVK(-j3+qa~L}2d)1Db;<`4U&xieQ`=I9t zmOGp83tCMm7g~Q@@ALRR0gUH2F#^#ZG-5`BL$0MD*LNXL`npOJV}Ry0(d8}FB$YKR znEZ#&*MBzq{NUfr9dn)hPWs_HPN?D_Qp*VX1(+GmcG*ojuAQ`YXd~t2>5Tq`7A8I^&MBW4>HaLUvuJ^ z6;H1ro_Y{SX^oJ|%LNN!wy+)Xkf+;uV$WFRs6;xk+g>7M2l|NMnnL|~^gEGURLgXQ zytMz$0uBeVxy3J;xbUXG4Zl6_TF&>yeW!v0+y|OVDE> zsYl5fdoY>Qv#_JEEivZ<3#o%riCJskm6M6G2u3CQK?c$O3t#f}We6gbzse@6g^65t zlq3(>+@cKh9$G-Gn=zX+>EHX7tx)9MZjB07D*-6owhiq<@BqOy{E(b>9GlXw; zorhia6TyW05JTPM+lhnq^ zW)xQQ+h7!VqK0##zEhvGR{LNL;*c)tAt{p~fM63eaNf0bKcZ04w{C)nvh z5XYP%%?={!k%2ARHkVGK@JcCj?2#Al>5vm0Rva;x0#mNk(8V-m6A;X zvXv|Eiv*Myr1ML{q`*SeKDa!dls^58pKPGwlys+gc5rVp2M$#JpWX*FfuS>dIIcO&=g%FTn6q|d%SG98_0r%Iq(YjVnomNzO9pY#Zn(!iBv zYdqMcIB$X9Bc2h@Bz*=D=UL>4GzJ@gf5_dfJjI)SeSd}&SD8vF>5_N|Klty{H4mTL zg6k%yV8OjsA#fe9gY%l-f5QmmFw@&@8fXta7F>| zlLcgvtFzr3P0g1Fv;9IqTyGSh-wO{4=cf_7TH4C69nbc9IBhI5Ov5dOndoDyt1VegYXp|y^us%pKoz*U&4)We+_GDui_ zpA(Ggmyp#fZ|ZkMuPwrGEHSN_73%Cokf0fJZWVL3-LI6ggieO*#%;xB$Qf>nAr7eF zup)xB;5&5m$_$m10!6AK{|%}RU7 zYFQfMwi@wsC0U?61A#QGA&%)c3qFS6BdIr=I$GJ!^N`?q%Ip4A^^k_(NzORawtDY- z#A8&5`Bb@n<)^Fr9JI=Tc_uVsyKuK?JY%qN0B)QU4$8MII~0t;t@`d$C1Oo0xO4sty0|f^BAp#NSU{XC)clTs?|8cRh;{ycG6>$Fqjk z3BPR>{@)J&ytfMQal#}v_EMx zZ_GIL9B*I{HVM00XVXqvaJ>s`Lh6~<>|Yt#*m|JD>*Z&>+eCU>i;?n6*k!Ys8P(=R zE#Nt*ugQc6+=2@J6|L?(xsj^Bv_TSG7_B}*TZ@cIh3c1qy`Vjk$?Jl5@61F;06nb| z>FUTHh)A&TMk+~7NV2+`061DYdS_$p$w6N+@JjQ$`b&`$u%P6oie#<#v3ANEjeGu! zkxaL7FRuWfGn5s(K+pb}H!2B1bemOdTXVwrLQbC`7xApU#Ty~e!yk%tLB07!LIL-> zX}TDpDxu5g;;_l$@O?Z@h0l`)0!SZ^`#4O36MNU}wFf)+iIGddO)Y>bFnSmeMX>5| zC*)7?`oD<}RP(6D*ql-VS$mGtqcvF!L2oOoxr|){#v75bP4NC+eizbsGw>A?%766J zN9vQ?7yZ)S6td*Db7>}3IiUTfkmf7D*`lxc2q%t%&1BmX zblHc#$=92!ii`L2cC5@R%)3*Qc6SASwWq5QGe#A5tZmlVp^tex%`=3^dRb-B;~pop z4i^8qrhY{EVjw0%9-kH;{=XION*;H?q)rIDS6WY*A>;LmUGKV1M8xG5BXjdhnMsw)J;j4%48eAaxoq3zG&g9K6agO|>) z5uV&qlwlLQm~zS=v*{aZ7z1>*?&3m@j-fldn7i*E%}=XU9Gu0OF^JK39T(dw_kL3U zF0tRAIQ=eusr8G1U#khSvB8OL2SLBxcC#0!mL z1E$Jp4Q-utEq7;f$EE@M@%n~35n<&6cfX58rjB{{Ry`%kB<*{g=1C-+u%^xz|HsydNC(tDFq&J=|p+8mjuDxr72!F6>Ux*Y$ zV_9O1cE3;x*G5rPD4GFEVLwi7EW(b2>|p2w zEBg7aq_2q>VJp)&!^{MX3_V#XCVZ`j$H>dc(VWF&C~cvt(VuqQ8VVq#&zX!Xy1;C)63~ zplHf8+~u4>Y<*JwsJNZt^yhO3(!!*F3b{zvd) zhJ2G5$A^yvDD48XWJH+x5u7M!a-^vaACl*a(~jGG*1a5#W=tl>1DZ6`;2@0))F#_a zq6KIiF%TmNqnikvY^P>DfW5R-x?Yeh0<8#vDN2b1Rwxu;T?UWvfRj-`%wdWB1_RJ` z#?MI@G0fbrdc7CIh=DjSxh1jX?TOs}zWNcktwW_x7+|eIr9Y*2!?W8w^S|Od3vB#Q z$Zi3N?$F5BFe~<=y%>s^-3xO!GJ}?30$Wb0XlnSp0VyI^F!+Ga?&c}vQaU1M57W5}@vKg_ zyq?tJG9lDv|5%jraw*|~d@?&nK;aQ34y+Pf^E`o6s+@UG3%&W`Fo`v(M?VmW{bO~y zrSX~=jV#2>Vf%>_%S1kInsz51f3i=9vgHRa5S$yM&C})QpDSb>>H!~BoK{>+vXDb; zJtG?IPltFr-&D8OvTvoLSQgqfI(=sc?u#-R^{osx*CfHjtcOA4p+Q2-JTRM z2vx;h%@Ideb@1!`X2Cf>$8QWZrHBOy)iBi4H;Jwe`MTXvkxAr5n~dT^L(j1?hTl^_ ztPbBC)#SZjK0FfbSDM_qnFNva-VPR!s|_5-OZu=>5pY^eA~o@*=rdgF3peb%i5P1l z!9PSk9}t9okPAe^H4Xzpqk-k`NT|03 z5RbM}>}fU%RY`1w=9nZ>m1Z4**gyA}Yr&lFG|>W#>4_M(-nd5LP09B&roOCvLxf9{ z{3)QXiiL^oWV!jg2b11&Kpr)W^L@W`T9P!TjMCpvv8O87yg+?uZ%tBLAcr)Zhq1YU$SN@vxK<<=-;iQt zT@z?xzKw3&83=(sWR&1*VH`5J5X#ayc+jzl_fn?|$)|pQ9lUmgyJl{E^)*OKhW}BW z;0&=HB3?N{&2N1W*7=Yi#9s(hfCpk>f5RKwnqPyemc{WUY_I2sJi}*xs&ew4_xw=? z&9(jb;fz$`r51D%SrqJ-MK7AavS<8VRF81won47S+yvPakx-7fC z?biqScc(VTuhQw|S;RZzwqq_@wKkSLVt_w{H@whFx><55w&k%AP1rxEWLJ!8Xjr9mEYmFbOU@&J^gBza%o18DNd0f^ z<=cA2t->v^LYA>$4jZ}`9M?gN{|xfl4J~Z*gDaEMh92pyLVDt|NJ_Z>UIP#;nbLa8 zaYp4AgwzfN$bWZ}N8TfurzK$Aa^RIVSfUvo1XZ~NB|E3fUPu&ekt{g43YaK*c@Y#zQ+ZdFe;|6h=a5{HXeA@?)8YysN2%e zv=+JL9*!zq8j?Bla7t$Mrt3?b&@@aFm#}%HkV8DJS6aM%FX`bi=6W(Kt?8k&=ofI* zfaCS+Aruty8h0fT6G69J)>MFc*ZomB5`uIMU3Q@p^^DHEhn_y{@|wN%Jou}h(bW|Q zzf#j*GB{l6fBQjN9#>|r-N*K8u^_PSuRfr+EYEFkrG=5HCAVxFzxAE(M_%zM673w+1&`Fadr`!Lw^*v;DMZ%#h3JdX!xmRk zj>WZ+&LAf3R2YNpa{lU{KsyF`W>sW6AY%498m~L+n#7L}IZep6MkGa1zBccq4PZ{~PhCaySIY*yE?JW{7S$J8^3W)=VPEzy3PStME!HYU&G5OIiROqOT02n(=Vy}yr(VW) zu_0;#QVl``gvX1{$YZ6mgZq#x!#h0PSUfkn`tt`K$?Z?E_k}al>6S9NsqyI-1AK^^ zT3WWRLfUWTNrO|WN`&2vLG>SqN2af?@(*LI7#VeEOI^k*mbBlwtw7Yzt3#fS%jc2* zIKH*fAP@*wa)0G?xtV$xPiW7RY5+(Jx z*@uysBG;log6au!kQ!>FOFz`G{dWdYE< z_ne>p$9>PU!t|!sgL%^HyMA>==&JEI2 zie$FK!H~*kj+CqP>mt9&03d5O*A6=mr&?^N&}6_jVkuL~s;YAy_Bfqv&&F9AUZnOu z_A#_WY(^S29eH_1RlJIQnB?c>8hD?xdJhKLv4{{^JapyXcYA z^7&FpilIsV|CdefwwcslkN<#f3`-4ri$-50}xMg0<1tI ze<)TsMk&ALz_E66vCvj?qJq?Iy2-Ow(w60!9cmSDrQp}+WE3k5msQ&PQ#L(%I@A0G zu1N2_xa;foTGaDkB2P?Watt0BB-Qx&p^1q3Jwv)P66KTLb=McQl+S#2GZ7s~{8pjm zoXwU*Lq@>7Ii1{aTiHc<^r^!GH?0=_B#-;o-s6SPfDea@sLzAZ*?t!BSnz0#`mzZA zp$nfryV$!#Fi&ZclX|d|xTA~W{Zs$ZD3=JtlVMx3(<@>6IlKLmyE(n-%9gL>RFA^2 z{`5XAv#B0X2pxPZDO#qyPETcqJ4{9`8$a$(g+;p(%RRb;H~yh(D)m zA76sC99U9P0uF*2+)N_3rC95?-51!bzFE-Y1#7w*L zOsm26Of@nDm_&+%58oI48(H4Gacn#3eHInl&M)-*(3SMK&JFv#Vf`7}Ohm`i18q_c zi+AJo;ZR_q%m0nRAD+FKDkG^ql-OSckDE= z{5g!S{AQc`j(jW>D^;Iayr{d)VL{Bm0GCEGun?f45Ij272x0N7zukt^C3@{*YrBah zp)fHh**bWef0mSGz!nXFnpIJd4s+K9@?a=A_p=Z08swI0t1w}o`*Ajwy7#|KaK z;IS4590u!CQ_;8OgG3A76XwRv7l}+#o|wVIyq9;14cq6dfaluj+sCAR{jebOMgHob zHo3>t&GXvT6526=3Eqf679n0qa6bIw!3S>I)RPr>!XK#hS+kv)WQcKBW?L%sg)EIwznOJlV}KTdT1sdP17)NIZC-%L%)Q zd;Z<%`HBgCMXP42HhB61zx$PBH=0-3P^lV)s3&NLp6e zp8D@`;L6>bc*C?A2TnBI>|a2YO#VMgLPOM)B}0%6C=?Xs0wzeLu&u5l7++Gv>t$`Z zncIU;sJ?p2Tb|d?grvDw{ttAsFlaK28}^w zF2ag*MI@Y2lS}@^Nja* z{mNB#x@?1wT_jof$Hifo(`OH#9lUEDq*_-hFAt&nD>=i|UUwT9Dmh?>!*ZY!V_mv7sdKqhY(T0sn?5v9K+2by{1m_s7VB6_Kfhac;6^Tq^J_1xS zU599Way=$Q5>h23^6nf#B;n?dlAW*!-_vm_aOj6Fu_7@hmDt97znSs7)qrL;lvg`U z>F9O{hoZF^W;Y4i@RX`We+E~U`dWaQO_w1E8iGu6Mt&>Ugw3~V_aapAwJ9!? z#Bxm4b#s$61CrJMZVLA#Fk2-0{Cy__$d9EHcS?CO{R@Zv;M@cqOD5-=i1Fm^p1-hw zD})e4DAi8SxU?NU2b_6N$%LFoV@LooQq09pveLczC#eCRc@VIBOegP$O}|hQ?ZzaC zkG*us#CHgZY{)(@2%>lni5T)ByS_W=vil0~cu4LoX`iTCBP$t`mp3%(G(+6~1|uH# zmzx}YkH=X}a@}agH>*C?7S`5YmX%kvAv>{|2J&=%J4CE4S69Ekz(BkI4-24gDk&g& znv-94{7-55GHdI=MF#UK;5rVC!E(AZalobohgmpBIOVg%U;;g2A`U`sf?vvC z(9OWeh>En|*@nCV&Q@(>$zv#!5cNl6z4Lg5H!9$!cof88!&c?g|fQQs*Gf0z5Dtyd6%rxPWO9=3=5H3fz z+Rgd(xN<_BQ=FdT1?(sN$qNqrF!g>v7VE{?U@c;Ql5sSrvFBS#U5XpY5KoSjK5ifq z8b+gLZ^=4W%~)UznV9CDGgD>>IOVA}^C^ZD&iu{P>3Y^ekxPzd&!p&1@oIK2cqzIy z|6-Xs)sypT9~}E`2My)5VjwqNv?`@K!{Eo-UR^z}R)FTuNjL3&jaRP2(Ql&ehaL?p zA7Opu-gqLkBO;@6yikmVQ1R;kOS`a@3d{%FT!d5Vb|uai_R!Qvs=e)ympX-t_NLE@ zbddL`4*l@5%8Jw`kD;w?EYcr%Ikq5@^0(41`d3PO3!vZchhQvK9?o&S!kdYVU|`gf zAe<%LAU#T*3$agszBGvasbE;N{fR-AO55~V-}^oLAV7qCc#|Z~XtpHK@=4-cq|Crg zmIn;nxl(fWf(JBt1Wq^YI9b`eNvCyVq~Cr)KpG8D_@i{CbU;BH1E}}e_S@lr@iZ@; zG<`~>sN)L*DH`^)I678oi|04o7hFH*OV!1XCI-!>Lwt&%)&?=RMfk7)x(!sDf>(KDS2s^@RWew@N`4E-y3fG4}%8&X5FgHVXAd}t|aFheb-@*Z3U z_5G;KhEj(Vd0E57gVkXR_IAcNf;r+j-%pSqCzmF~r8xQ* z$LF+{5i8<1fe!>as3AT@+N7d_T@?RHLm`}zGrG?SCnM<{1mLsF&LjvRfMrOPrrj~v zKX7u%;&sCLi92V^emU$zUIu`y_g82%2sRa%5a#gVwHAx(1}8?8-1|Ka>t?kNfoDwB zGE3Y)aMBLFWed$Q+#l=jQ&@DNTp`Et0w&^a|A(lrj*I#W+FeSzTcjIQTAD>dQc^&= zMd^@+T|lLzQ>i6Ix=Xs7C4{BByJKOOyZ-L`-uv%jPTTRbQJe(S!#^i|G8cbNT_%Uu1ELCo*T z`;+F!TmgCEhoqd=d$0C~LOT>u+0f>Ec2rUjAyzRj^9w*YE}I1@$@};Z2l;3MRjtKO za%Cxg+hON-;H>Qv3Uugi(L}`2my9urtCRnGYTP#0Gft{&?#iG9t`=4fOZ0FiCRr>F zCJsy6GIS9K{n6S_QxtXZ(JSCx17iSuuQ1mFXX0Y-8XXV5UEN?*tD*UO)`#CLD78gul%jO#@9LA0G#T;~^r-iei zp>KTE!(>UQ>LAS$9i?*r6qt=^A)APq`jYJMyl!?{953umILiyUDo_(k2vD4*_krR%)Rcf1}?)sE8M`=Q@w;X5XPAP5fvmGT+`_430Q?JXH>?VHR<`9RNP=)5iiI zw5;Vb(SkDv+%-p7kS^@Dc(fdO3b9q{Az$~^HAu-WEGQ|n4YGOZ2u`iovbE{^JZMTfUk zZ@(4(!W!zBPJ5Xy<#jP1yPuX8$aCc>hjW3FU_Ul?ffry@JPbjWJa?Rc$iZ&NjF3Ej zaE}D~UP@2koYMD;dONh6K8x9qauaDfp#a?0l{$2j(A#xaWzuA5= zUp>caFR-agW|4{&R45GSujML8QxQty&5|4EqT+H_Od_FOFgv55i?cp*QFeP8mfMT{ zOp~3ETd)l5vu}X^drs3gge_^kBR|c8Z1a;P#?cq4tq_tK2S=5l>i2l^(NXHpg1+p0 zJAzAg&l}>;e)Ynref^Y^`Tjfld!|Y;OLV!6*<>eFcL&tNNs30t#*#6st*b!MId+cD ztnu<0-FnZ~$hisu0FBoDy|6q5Gc&^+iB1L%C#ekD3)u(vruS?#-MHELA~3$BO1wAb zs^FJdEanlj?5*d2P`XTMLF&l%>7Uis4p_ng#a zClCQ2PYo~oC90v4&-*u#klcX^MG9V~P4WYn1}8$ok*`WIoQx!D zR?5NV>)X2rc!ad^E8p?A&+a^V0h;Wr?askeIue%-fG({~!Dkp&Shm{n8?aZ~dgvi` zcy}V5a1k7W#Oxpgj|U0D?3PsaP!OSkSa*erKt}zc*N5A7!e>0;kgsvKuQ0+@zK>c4 za|NS22*?ew5_Vi0;3tPz)I{sNvqwh=T?)IRap678pV!0<53EMtHnJ)_3wX)){1qET z@5H*v|8Nu>cX1ApLV66CIO<>WF@GA163pmKnIu=Vw9cAE=eAQLT6a%S?|u6C*mY|kXQjrX*8;r>cy$GNr&1)g7s#&6(Jzl;Jr9+!*xmW=#?*TV>x(Je0?@KaTS4=+;2a zW!Z74$$0$q-eyW%dDC8q7lcxfIO^)NL=t@W+1xhnl%{1^bp>&(qiykuCSt!^``OJ; z#O!o4ygv>jDJ#mj>S9t%I1i{h-0+;TTlBN%jJa?o0i1e+Z@gK~yBU%E4ink<7cr7A z)Q6ZR+QE;;4&}ir(iitYR6%pLV*c%wT^w@*YM?w`{>g*iSwM(jKB2HY>HDwgM!F~9 z(~0TSnEk0{xGQ3UVT-a(L=+`8yR1E1=`|f zLLa<#juD}sZ9P{imdqsm{`yARJZ3HNjifhNLuD{bb@biTjz%J^m4q&aC>i-cZXo#~ z>dPO#)-4&o?mQsN{}@GV<20QiW_sOWm*D&9I}$5ZpQO_Aa^(DxXzWuFg~a)GZz%r7 zw)09IEbdr);-WrnkDN1#`Lyrq!0d-;Lt9mE_*7okK5#~RXn$pVd{rR8wS?~t1up3i z1^nO~|D@Ep>px$w4jlb;xwl4s2v{%r2By))68=9{lN!IiR^eAhV(Ob>M8UBP5+g&?pLYeL2VpVasfJY`cGp;0!F`ExspOZ1z5>X$QmK zn_Ba-4-cV;9_VTvy8qN&K>V+L1hT*P12{s7;lg572h=Ub2ZV`Y}tH)f1o z5H-9N%3AvaS#N=$O-tw(**hL>ZEd%O!=L2+F;v?kB`JJ9$fiFjN&la1WV)vKq(j=@ zTl$8vy`-YTuF*G69?y9%@CMZV_&Dbeb(M_dHN7>}Tk?lwd`|j?E+>bh!i4+W(lKgK zBAb;olVg@+b^{j%TRk`#i|W4!6hda5`8aL_5zg8;#uBo;XTc%qWJ2TM>Stw65=zim z_V+IH1i$k>Sw3@Q7du}%SX-Oh*EMampY~fRE;p)!pN?&Z{`u9}eRH@he$qf3==wWz zOOi~!an&cax_yw{TTE`ePYD`lv!mS05*amLtfZ=r}KoRJ~N`8zq) zg6qJUv7^i-bhPR(Lq{EkrlwBAyt#EH`eR_L<<4@rZne|p?FB{C8R5*#2J^M8!{G<~v2){0NO7fLU7o?}B^+i8( zU%}TTiw~yyC>zf&;C>ky zp2h%R+>r*;e@%sUBDw2LOU_r}&5ze7MySt6TI}a0u{32;uwe`J(#REJ@6Vt4o0@9y zo^+PA>);rMH+oUOd> zlsRa7f5Q5W^W2~N(@wjeUH%BC zt<2rWJGS58`@~r3$$C#wdE6r1P$I;psIwEzEBB?)`hM`ug8We?L1FPPud%}t4$Yst zT&0Gs0iAiVpFdymuRGC+%G9mSY0K{`j$zVLh@Y7B`O}`*wm_tOT%20nRTUn)Go{@M zKinK=mZV6*=H{lCJuLPjMRsCn%kcqiQ8$9i!#*e2J~s&dp8Hcvd6(9!85GqD%;{tL zP4f!F@$Ng<9W#Ff7DQ}CasA5?9mxGYR)-sxDxhvdjP0Q_t2INt(mi{z%DR8~ysmxJ zDC}+ydZJu#uJw9P>L1CFZ{P(0z8M@<7&Zu3$V6!o@qRF^{)CuM&HCvgRpBG?3(1$; z1Jq-ZV|t=rYS^&z#Y@`#@<(2I64Ri~WOP41RplrBkgFMXmzJu{D^b96!|Kf|m^pUk zrsv~O1oc^hifq&x`s+agx?NRgIfI5;lexMRL@%H8edk&Xe|rLnY^9y?Kwm|oG}D1- zT(ZXRp;*XG6`Fv{pc<6(JsqX!AA5Lh&BBe-LiJXlW^ZApwC{S6yt(hz^mUR)FXapQ zr##1P2V*9X`L3;`+0Mi^4LWn^6kh*u}5KQYS63BK@NYsacwli8@tmdggw6>|aVo z_?#b+Q_%LqzLGFuQRn#mqfGk~D1FL8Cq$Z=o}}}`?NpdP@L@Gj^95?HppM=hNbCBNs4hV;8Bnhs9fu_NsaU`z z%%%CL3;*Tsiqapx$F<-2{| zAQDbS(X~L=M``FwLvi2vs2sxMUxWu_&WOXArtW>W6;tn)%G)JwT`0YDsc&~$GBz>2 z(dGL015xK^(tewoZ8s1o>TReKT~Vds7>CzKLt7es7Ov^-Vi+!u=_=TxRErvS-a$V( z(Z7p_Z)JM7oWhIugn$C2%=KNuzIfq;HtI z#rbft@Hlmoh!#z^vXbClY}Zh~eMXMj!^Orkd3z9I9APPa_ptdbzmc0#O=r3w)y(Dq zyx(N7sEjBpE5leHY(BZ&!*?mq@0>Vu&82WVR|ImyOvu z)mGb0Q+#c^Q)uP9W8!x3wM@DOmxcK}2Tf_G#Z~+N+~!k;gp*(NXDOeRe?}3vosY64 zJnpOa#4Q9}XEd?y$6Z67nLJ+lKi-W@DMcROhl(_!l^0H{-ha7Sx;RWo*8K$yUzL!w zYgH;QEcx|Tx>*~wW9C%lB_Brr2r`xft&r^(N{@GcFZd)Wr!laF9Rt7)Lq(ZO68>6> z=jD>1%Pu+)B3-mxQ{t*-XT)i8$s2ii?G1AR-n3|HL4zrWQ#cmkpLR0*ee4#los9iG zO~=hot{Vw!c1)!CvugduZf^^pz?x267UMqe7iy#%vC0-@2}I<{1STdgIA-x{+%tUXCU|WKrQ*+rS{w2T=rq; z-R!^6m@Zm+8uiJzJ*TF5A&%Gip$mTu*yy3Q5K35TW!ZN0@1Wt^Xa%n>Df~NYI(*JM z?_VrK8BI!-jT_{Y%N!OSE^}e^C$qHkqgnyu;V37#N{7HV@2>PHg@CV)P~<=v;@0rU z?gpMKjy%;16Bt z(GV)&t0e z;Ikyg!oH^;At)f|x!scVKuqn=EirU@X4{dPfcNlGjDCoA`;~US5QhSVr0_D`xd1%{ z63SbQhSv7LAtpwcd>BPja6~aP-dAX75#;C9hc~%=7dM@nWPcFDwiAvZrpo z#VwF$V#>*9WBVxaiJ{33)X}r25&8QL+>sp|cH`C3xVzPONg}*{9A{Zr>a%t8pHspj z^&I{=Ww?SQ3L$HPfQO){+ONG0${hSpF5h5O;_j+{0oXF(J?B^Le?m-0wcq4^m8Jz9 z&oU))S!@ZMTzwKkeeTg8EFJa*M4{tqC4NM&$YYIL^e=JdzEAc;z5-cgVQrJiQ$v^lc)xXwzweRV^8@R~5|~UWh_{ z#(yCx>L!u&WO}kQgs7|x`LcygM?EHtM%k4AVzKkB2r;mFw!^k+&f|Z+3JSSV@bnD) zOH66<{pQNS!1MFxfYrYhP76|Qvbs^~s_H`*en0cPtk+R@sTJe~+oO#6N1VLJWbse|U+caivka)E!QFN)jq88-Uiw_-yvv2fIZTaz4-Y~^zY~vrAig6(i)7k( zLsVS!*`VB6O{d_N^X%tWb3f!q2VB)r!a7}J*uI}{AyFb{X zEhv9pR%bDaQZsF?uMIp=;zir)FnkzN^E}fuzo0fRa6n;P!%I9jKdQ}et3+Kej+e&j9QvJ!lA^9vD93SEZ-Wcdce%4 zs+lEaJJu`2!^89EcYXbkw8aAcYyw_iGQ)xymgPsRQ5%xiFM^X{3%+7S8!?=dzKyL2 zi{`?($z*g@P9W!X;z~sL+b+7811}u3aAj_G7QDmom6v2;taJ*F{?p4e`FqfFn}Q^r z$T`L*KMc7QJHwEZX=)$IG1dt-Fsa5Tj;Lr@$herL?@B+FQA1G*;k&<3H9T*{L;Vg- zhQ$_7&iIdGKcGae zj4Rq+4io1L%yZh?*|rm?fY9?k9`i5ojg$gL2j#J^y-p59HeGMERjsf-oaLrJto3ME zC`UjdP1H|KZ3rg~8K=YBz;D1dYev~P?nbXofKF~p%ZYyzuwkCb$DM=~x;E(ZjPQfj zBT13h+nU-hKF*96g}Rt4 z!*Z@;zJ%$h-Zj3d$6R4q<)Vqe=~3VcaUhyyi!vX$iO2gU<2nk+U9l&Y68f!lSEzGX zqG4&Gc?5m{cTR(S)l4_CKO*Km%hNoTC{v-Tzm1N19^hrt821X2A*@m=<3Yf|siD!W zFjS4xN%NkToTVVi^2u!c-%b=j;^yXQqg=WE+2OU$V(!Au^?!NL2q&X7q{W1>c{PnM zK&12$;UxJ*^JAoJkp-2_6-0Q;u>_1g{a3#?7eNHbSbl~V{onPOYXia7?;ncHC8(XC z+<{8nB*a3lP7&==f0wz3>aI~)%)fi{J>jx>X{;$t*(0K?xy)DW{gEB{n**VaYw}o2 zYOczBKM5lC&xogWi7I2L%~9J zFHK8E#+NKi4DA ze@_7qMfZ2ZRS1N>qdM$N&d!cBQWib%=yZqu17qVIDIvbHM_PFdL~j;{R;~&FE6-wo zfBysYld0IrnqGK#@r3qD;I|0(9_E=M;zcVwm58;sGPYGSlY7QLL8aJXKRac*zh_R=8T5g}& z?7IA=M89l*dYaa&zK@E0_Qh$Uf_EsS*2Hax??kajQz{Z;d@+5Ja?ggeD@_NT-L-`P zXy{v+$9ihV;i~CkP>bX>{;+hXwnEJ4dpI~wY@NZs8Nz-Ec`jKK$J{@m>c2$t!5t9l zs$~}JGamFSI9d;NCq}b4vSxE)WbFxx3!PENsi*;#P07Us(SK082942j$2O&HXJ_ zo+mWK(&+5eDpBe2xC&m;%o@^tBnQ85y)2{Pd3}iTB`ZKw+()Nyidkm;Aavf>s|KGK zLd956)!c64ASm>!FtBDNI@eVHWjHn=jo9xZKSbXP6gdyMlS>piAb7Iz<%P;m&7puv z;O*v5VU$5YD1+RX=PQ0AtUW^A!84_suYUG}==XtcKk#L33E5e7Cn1C+*hIu_LBb75OAB-+=$Z2I$+(Ash{S zBxT~F9dzsEQ5%9DcEG7B#jNj2@-;pp9O3w@TGvlI=vp+o1Cz?FdgpHFER!t^d&Vj9 z6u@Q=6(*LY=Oz~Wr8GR$kmX92 zx8wRJf70f4HKj$&{eK=d@aMemH#XX-ANf<-rER4Xp@80l`C;Q|vB{$=fACNBWVBvw zeZ86296QdsJsM>YCV*x>Zzv7Etx_0mbTwv$MyG?`y9onNQ>NV$Djn6YiZ=uGTwj7GgVYC_u{omGhs_g zovm|#(&_^B4T0H3uRvq@Q{IQOy_=CXb`FEdm9d8?N!93dAAKj&KwYPR{rt20Ylc$= z(oZFo=+xKbkB_f`W=nzx)p*n(;QW?1bfK>MhMYduRAi@k_+_(#N+^U2oDV%ugarU| z9NTC;% zyl8E3WKECH{(Vngb8V^ZRcN``Q?pO^l&;v+e<>WDLf+qb)O_V~+PLj9Rb*b*ikz+U zOQa|Pvag{naN+Sby$^5F$`PAZn=-l^^|xHW7YAUw3i5yM0DEOSmlIC%>A(sAo6(P@ z_2;qUs73yUmk7QwMASLJvbd)LP0an5rb$iiuhi1=v$!Y4Hpa*lBT;zlh}? zgj$nEtk0*CzD2gddZiK> zx7LHGBF2(Z7sujgO^B3pdauY$v;$UT4Bwrblh7M1uVcms$Xb{sqS@WWx3@!MODX_RjddXD!WUAQA6@KE(`UUid6zl4{keB%D zYZgpT5hYnebj*#tQ8ez6;j+FItejNd+}pA!?rmH3L|$(68uuYoQA8S+Dkq;0I6)|nZJMr&eycGO!Ck)r;)gz7H?*C36S{F-Q@odg zmh0+?5E1#xriVyb7?240Sgf=?NCCi)U0yH%Zs;N}X53)B18yj48JV zP?M`|-iwuh`xnG9{9C00StBexxhlP5l8=cEsM(7h`A9LDmt;D9P7O;>ZcF;S#21lB zBr(|a`IS;YD(<-Qm6s>-IG$7HV$K10KL_;=sPsm`>?{R7XcXv3Bu75~SS&OqR#TW~ zeV=lJhOIynSINpvXdN!)-W@B%Be%1Lf#SO#`G*ncCKPpC@VH%Vldkd^ktHYS5gKP` zlH)Pq7CC%@a$ye-J5#rbZ&~iwhqHfHw8AFbWB7u+v&CBQ~tIE4>!IGu{(1@|WGMLh-nmzCJ>jj|3tByb-- zmx`zWKTvbjeV87y%QEC*p%0Vsv0R(-*cod@!VelwoPd>JXot_pzPc=It-GS_VaQJA z=B{t3{anY?`{BGTIsN9vkiE+{>xB}QBgH?&8QK2ghZHPSZnCaF%RW5>|Lj@Yy1FFRa2b(=3Fef`?r@%>J0V&Xk%V4{M-v7p$0vb8Q~e z)di2ISC57p14EQ2$bDp(Rn_du0|FTJOnh?9#>5YK)x09qw;1@??NTv!z1UR9$8tjrg_Mg8O5U(LzA z?>M0Wz_~l*eGs{Z?fa2x+Y^xUacLzID<1ChBlEi3ⅆ#1j`G#4McwLFs$*r6T*j( z=eTk%UbNT<4vxHW9lq-XZsvZOzYL$MZf&pIQ$Be_*sQ0iKZPC1 zlvBQD!-wT8QRG~V2P@)Jn+0A?i!Qn@rAX6b-z$P&+%GPo2ySkh4G~w1LYk&4eshoT ze(>3VAqsOO1bxm%HM%O{)t>sC!YB9~Bk2Vvw;Y^o=nDm|%m3QN}r1+>dWI=B6PI%p<1o&z!2zr8JuK! z^y9gtP%nWveHhV8zvG_+0q!>)k?$i-%9hgG+=urzG^W;j#}Uw#y9OH<=uO+Bxj&It z@|oGcjwPq)@{S34AXvRKcv818Q{;EkT7AVMajXft%xo*Fn&@(Jc=j(;q5>Z(g@T%8 zUMChZ`10Gnmu!P5?OZ&mdSAMJ^0;=us!6-XM7ePcMjYS#xHGs}Zg3oZf~Au&8FbXo zFUb`oz2IGL)^>P;;%|G`RPA@ye6tLOROk^Ab4?vo2tAy*EO-{%K}J`D`9OOW*aVsv zK~Bm~h+E=@F)I89DU=u9EF^Ihy&F1{GJf$Ix7Lf-4s^H}vGL}`-NqDZ`$EH6L9Ta@ zbS{|q#)t~^WkA;0<6lK3^Y5;9c6NGBOKyKlDeksKow4FPj{zMw;M=?|Q8!H`&W_GE zA73-GKtvR18yR|)qMjiO_)MliuFyd5l>=V}c>!(|*@wU}dpPrvw)$dP#qvYJk;Af& z@5BXnG5;ojge&w(goB^*TsK8z_~EZ;G7>3FU!7VuG&VgffRdjR2m<&$gt?`{ln8?MQ(_cR4IYU@?$7@Z453$6={KV_=+WuaBHfN!C?!3!IIpEk~e%eyh1YW8K( zTvi6`J|vg>v^^(3?ZzaWZJby z8aU&!#YpE?ni}idFe$p~zyjE?Pm7Aa&~>AltvKOw>}VRW?uTY|O&A!lxg#mLA-Z(i z+gf5CaJ2Oe*VGO%GcSI$g9v>TLhx%M6fgyH5OqGG8DEQCnjX5Q(py(#Ts5hmN)>C_ zF!jykVLoA@fNrYHzQ@=oOy9+dlgkKyZV-l0ex1OA-52Bt-l~NGvkm^IcLltNPzql8 z6)1*VsYFRo?I9=Gf4Y2^#5W8ZlvzaQ8$R#H)s|6K_`x-{-I6l zUpPUKoCL905^82A@Hp+%5@Uo~aa42Ve^pdSdb?7JVuPVL915!5>oVjrnXXC*+1c4g6nO`8Oqb`6{BvKXp5{6`9lgsxqvm(#=0~Q8^#eX2EYCh#^&&`VEdSG# zv0dF_;Qh^?1>!JLNE7smraPe1!Y$O1 z_C`L~Q7)DZ5Tdg3_g|PTw&uaQu1jm5LylJSoziqr(gl#uD_O5ynb2{mTv#=!^AI^K zvl?H~v*SZ%1wXy4@k5R6K-bp~KA8{WmLBS4mq7?|`Z(?=HSuNfvUfdjv+7&aF$E$A zFu7}3%XHC|-X&Q9xYWQ;nEzT11S70|;Q%-eXRD6Pkynxeuj=_bJ$eBIk%_Wj(8#bz z6+7pHA3sl`lb&m8lvIK`2FD?J&l!(KPc>Q4bvOnp^UyHy8#$JG+vox8EDA#;4_+$y znpmOF_E%xB?(M6LYd&$SN4H*qPtt01KOoBYZPKcm}4qAOpKaj}#4MVnOxd{VM&h@;d0 z@qGmymK=iVwL92RSJ`XuvYb_;0R9?WbuK+4Gu^QsBin_UGSL6#d_7Ijja~<}YA)P* zLiei%nDoclLqirXg%#8&3`)I+-T-fH9s@Ahw{l?@6!@r(B$c`6%M5lPw|YSh3u$wu zPlARyP=2OTtQz^*NcHaN3d9d^Y2Y@tZ*#c z351n#7-qd_0F*(}qlvHHg(W@d(SU{2(+#fekkDr#KAQb$51GZ*31UGtNB&hc5>}iM zVVb4jZFnWH9nX<+n)CD9V03QsqWb?k_yaUNds%sh{x!NTc9cGQ%6=A33Gg5P->D#P zp9U^jP)m(_Aj~RH7C=K0dSUE6`I;&1v3eD#sQiPO$x4S!H6#-N!4APvS45|?T5lM3 zsc`MU$nSC%*~C%)u8kCyM*|cZDkf$KT>nMUvhD32#t+;DNgaSlUTk-?Wwzgs%Ez3) ziF#J&a33R|X?iO}NlH7e1@zZAP@#4kk(HXvyMf#v350*c{dR4brdF<6PKQ+(ywT2YUn^AXECR>@# zvvkc5qYNIILhij@&lHjG`urK_RhxDl4Ep?Wb^0VVD3a&SecDr~|JDi}rfDF8ipi0OLQ;S{G>8ThW7 z-*fUG&$H1mt!01fT;gH*n~$g`y(D>&F>EyvMylPH?&>EyRAG`6%)HVY^_C4FcRlXb zxwcP39j_KYa{c6#mF%pcvNm6SIHETH`&|RI<(v=s)Ay_ zE?fq**%mDkWoBT7Ls4hEyv4@tL>2C664tCeAESjDRUB-gnJxsB14ED!pvLXh$GOCy z((6#_!0untD`qEjFgB8Bq9%#;KYL?1%Sq3{a4*Wu6(G9F4I>g-41~j;d7oGPMB;hj zCKLqJx&!szt$_$0-~E>TsaZXgddIf0EC`|C0?XayhiA#Y%0AaP5(MwKbqOwO|2ywU z={2ZYc4Q0`SV*8z?c5vEH376Iv^ObdX4wy6q272@9F!?Y~_Mxd^J=}w|AbDzW@$yo);+dPxxDr?1o9Jqwd+XOhvU+$YF-Z8s z=5MlV9=yo#xTI>+Y?e(*XAzq=3pIHAq=I5M7Yjzhkgf41cEBGt+9Q(!D8-<<2Cm)u}Lr89g z%P|^8xJ;D8EuvOvz@^gGLZsNcufe=QNPBkWX-E z^D<=f$)XMaBqWhv$eX<-xVt|<8NWX|I(iBv-yjp{XKlSPF|D;;-donN?DzR9OtXr3 z{~}5CY&6AsBFBe4P(Mp#Q(<^#>2XlfaWhbVSv>F<$NPDL;;|A>FC~|7hfP_a&0E$; zf`tjIUi_rS7MY?e{!V_GQQx zH`Cn-_94MNUJ{siqPv_HOP(Ssj%Kns8qX#v|)+AYWZh zzFH#D?Mw#W=xbwc?hj8&-saFY=Um6%menT0F6neCkJgvb*8~ zZ5}Y*ohX;n?of;j#FRWzYJ9D}Xq` z(upS3x1*Y7r+XWaVwQ#Jud?p*KXmyX{$c@=PK~jNsTvF3kLfpmQM$X?5uD7q6&jZL z1H3ns0b>bUCc>x(-?8YNzH?4wTk z*?B=5U@Jgv+Y6ywH;SuyEbKb85f6$~ez|@Q6o~xZRfl#r8^lIqs}SNC_)3Tx@c8vz zIpRBcUYV{Sgyx%UJ>>QwF*P8Jh4q29sGZFvVkR!f!>zaaj;&ZvgZMh0 zyX`Ux^a>3E^s*`A+dQ``&$<&H$%1TFt7@r?1#JJmWy`=SFpt7$QS!@a?2SBAK^2sx z%Tzuqj0u@nkB-zEn+GJ9sP_eM2UzK!d{-9xf_FdhXcx34mmuPecbavaCv{PfOZ7d4 z0KBssCwXjBTY^JZILBJ{hJQ!pB4Th|g}MF_Z%6GfETBkX`}rFy1pdhrnV{;kEa1)u z29zs1envt@F}|IZhXYjZeP@0oG??Q-GiuZru$F7sdYZ#2f_31ntWy`PI<5pfr#*63 zEBMv^J!6at3R{;h5X}*NrVB1_b0288``1z#9?z_wwgta2@>9{dsZW|Nu*Vh*_KX26 zf?aE$OMHZC-q0op3%r>g_{$8hFhRwAHv7TD^WH$2)d1WT+w*h#_hKiFkUW)g{W;C& z!o(wf&4XqWFYf|7cft~OtP9p!cl|VC`r%hL^T2bW-_=8#Gbyz17`A(~ehNi~B-{k9 zarNDh6(pTW=LM*2_-jbBEBPl~x5Y7@G}T=djgXd~+I;+>D-Yu1QN&|u?YLq!>XnFb z7-!5OAtm@ex`5~Fve6fI48FZpdZzTVq^XIP4JD#s-FIuq(HBLlYix|4jgq@aGHqV8 zk5vw{j+cqpmLFeJP+KPK&MRT_eib0BNB`O=){(pVeTo4@Do?=@7xpYxrzz1S@>b%JFxO%H8=lQ4y~w0e&U2)Yk^EySnSs!KO@`Q3Y?|fyyJ_g{#ru4cc6w@$ zX=e~$dsD&1iWHY}Kin|^htDwQ9F)U>P&e)|6 zOZ+?bZBq4F5yc$8BvD;z!ShFIHRb0iJv3sjt4x`XMc-roykTFQn?vbIgavd%FZZVS zT28x`B)wjViuPJno%#v>Ttq>PDlNOsKzCaW%chYQGOFy+mISN+a4o3Alw@~RSKcT7 z!k|N%J`=!67LoVKUO|wxtoo)7PtB{eKfW>mzRXm7_I@QMPUhr)X94K&l^m!F=Rf|m z(za$nS7dt@!6xB{d(EsCtvl>c&~mo%Z9OpNEmX%|GUP{di;mP&;admpK%-JnC1I8& zW)@@gZhnZs)m=d?O^Q?s_aEyO?<9xqb`w4xkIK?DBM>D%TjWSGaUw8Qqr2ryUnxhv&41VjDEKek*uH!m_KBT2fu_2zTYm51 zpTBKnQf`9Ygv+_-r!fYi_PSs&v`wEl ztIF5{4?y0%LTXKPo=O?5$?XXh7a9_5cw#d{J|`S{-lJ_}qQd3Ft;3X@Oy1O^&9>U} zIVP*^TJC!H`2Fp|hvKlg9MPQi9~iHVk~b)E~q$ONaLye+(r0vA__H^PPtW1ZZ~??3N+lFHfXWo+sCJK&48bdQ9$P#c%Yqq&t-B2mYl>0k*lX-wI7> zQjl-;wQU)RI6>MnTKR|YmQWe^A5@Bl0^Q)W^t0!z(&})u;?Pu;MLH`Vj0Apu{bu_k zAdqcN>YROo+&|oS@+^V}20r&s;v$a0fM5eC*|c9&Cnp>!{*+LF_$9SCQ)S-K6xVxE}y|VfkXBuw>z}2-JQ>N8UlASUi8w+n9~{NGewZ%GX30A ziy(drEEuQr+egZ-5&314UAyUKf zEUL{_!K?kWhrNZyh7gF#Bpt@O+&a6I7HA~D?tIj7J?o|BV4wdv?Y{K`?F{>*os(Vl zkb1XcO(B>II*wre&NZ5EY;5#~F#I1n;f;l$h{VuFl`Cebqk1r@Vv@@l;yt@*A&nxs zB~yhE)#5|enAI!P6agrrl=K^hz${ZD|e71{kPilxF>zLzr0ECYrtm+ zm4hLzQqG{gl<3&U(DBTr!B%x#M5E!iPu%c`;rD$l~KkaUqIXL;R3tNiiTh(J-N(|EP_I>B*7zyq#k z-)GyEw<-o9A5z$%T~^=31(=G=5O%f1q*U=LXEW7D6E|+?eVoc)++O7L3AsE!SkkTC z_!^w^O((_}Yho8OI3)%n>dmV+dNjJT)OXE&719-SPgI$l4=mOlS$fGAlQ6r4w6+BO zHr@eMn8T?e)CR{45lvP|?u-4T242pGa3D0~h#Qy_P;vB%-!Aj>G8^RRKIU|6{&ey8 zf6Hw|C&RqSNNsuG)}|>r)SE{cpUQG@1ItGIT?<}IJ|7G1YaNpVJsCeHj%!&~2?A?Y zJ@PDZ-))BrhL0HYPSwuzgK?IlyK_WkkXTjFn$3fJG*>1|fBm>>|Ew@`pC3OYMOY3! zA8`Su$cW7d5RFb93T&2qML&2IvMsc7%<)hKdlh)Qqtx!&G`l`pD4mrdFBxETHgOX5 zNv1`c(HS|Vw3F(kU)6xp;WXHzY5FVc6*&Ap^$}kQk01>91$}I5^Q}g3!hW}_p5Wwn zAsa#lS=vriHbBWqRXVLn0!Uw~Q%U>q2XFO6W_hpsI=7w`tR`=&+&8kcRXJCMVQKAs zjC`huM~jA{8vQU^76o6Xa5?Rw2&m%VZuU5xVY7jFI!)jiTq9jnV4D#dDcW75Nenb3 z4#JFi)R#+1j4RP&e2FIpg}$xh8Bptv9gYyex+-*mu8nIjJqer9q$!JD5d~X{)AELq zBnNk@b*8)nzOH?yI9yOMk+I@8r9?^6+?H9M^)LW|Zgha|{L46KA93HatH;po ze~IOFy1Qg6wwPV>J@wys?82eL3wa;(MRCAzjv_+59$85vbU9H4REU$fnw@~02zA^} z5j9J$h%Cwp(1z%#Wj#h;jMIx`gU4bN)6S$a096m&^&QpA-hI625%X&0D?7n8yj`#{L*)Tx; zf;zf6`CmJ@%)K!SZ&OtTiq;tiYj3n1dKTfnWR;fjkM-&g!TK}?N<9G$o$$r7IH*Sm zDyki5WVOVBS-!941K%!n|h%KBGl0TrJXAo#R zF8z!O#|5Y$m{VxG(UpMjcMwJHzubB;@flc_Wr;#Q0{@);iCK5r;d?=HBzF-df}S>h zxq`kYkB8L$FUz&y2&mSIPu?k|Dt=s=z2V=M8J~sx@9dI&h5-#Z0+Dz7Su)L2=fTbm z9xdV*IrCN$Gj{l|BOcrlPo{7;ULk^AH24F@kE1BhnVog*GIh|1zRCJB8bNRv-ab-n zyGt=BGe~RWE})PdV`n&`;UoL|ZK?}@NLiPg{%mv8t4715E7=Ya>n~K?PHmwaU2Zn4Pzlt_EQC;RbN)<54rEJ>kb6Q)$f?Ix}k z24{#Y0*D9Lb#=AjV~=-4vo$c3tk>j=#l&ZB?knj>`?!dL$y=)tl>z+hgyb~effge7 zc9OE1R!ig5YT69;--<6rtFQ`pnp|j2+B{9XQKk}V9Jr)Y$EhEL#=jH(vk9sXRUYB(m0tEd6<~zf=3J8=-?jd?zmb8r+5-?`E`s7dt}q?XVLbFpczYKWQH_9C7OB_7MLl zd=4ItFDZ{$KN!`&=7h@;J3MMJD8z)W;j-rThPu7ybW%-vjX24HQ)nfmsfTbP-coj$ zRMkG{#ux+2f00RtyX1XvmAxTdJU@1MpZx9sZB9f>z9aG>Hoz*joTl~dONsm|$xz563? zpcqQ)FyYwpP8o=Te6=*xf>lX8g?!1xekm3KbY7i)BU3>8HwS60*TqG2OUuiCJy#2e z!;_Y0%D#+((Hp>kM|6+$6crt`X-5zn`Cyib-#`2_s;$ipQ0xljS9H&2!)d(ED$)5u zmaJ!KYlVr8%%?3r71$?$VjW}Y@<+DtCf?e+Vw-A`UBIsM658WH}!Lenn$j$RAXp+R9y zCbRf*u!0PK@tJ+T>hmz+#c- zTgJ-ay^v0zQW6PNp5)!2{=>)CsCamLR$|X~f`fqdClcFGgBam#1z_z1{`o%Mhkz0? z0!>9+PS0~E3*@09gs%{;6k!WU)n@gTd+i$ntzl zYy(y9z(*-0Ev>H7#`bly?pO*oId8P1qHC%qZ!a&KucOmKvp;iW+#(5Vsr-jGak8?AnDT1n6v*&;zMYuDG@aYrr#`!m zVNl z`~JIhb^S^6Jl9LjA99;@CHi9!L{$&nTB*72oQS)jjb`TYQO9PLN=tL|vwok=7ecWS zK+r5YzlW}d=h1O0hxW_x-IW$U(m%DewLb-liyB&b>Tia8JC?4I6cOo}AC!BhxA>EG z#4%+Xy&2d@l_heww{-%bTw)DvHMS`AZRJZ8<}T=dau^QlV0eq@Pj|Xr_I~PE>P#za zxl#|-M4T){E=$S8=r7Eh%q$D~PGh3amm5kvM*Bue`70+#?d)~jNezUXqZH>*f*Paq zJn2D**cLw-WB0YXEXl_J%4ni`=f<6Zh6c7?!ck98&jph;b7+VCU5lp7ouZ2Ot0qs> zJ|%CV-9bxRp{KL!x(Nz}svmfTccUN=&P@kz*0(aoo_cD8M#7fb+}E8MBz+otI*ZDS z9Kdbc$3w}r-9J$5h?)l}pWY%F6VzeOjhgyLyUWARvmF(V$#L*XR={!6Sonh?I`9DM z$lx}Nnx*)=1Z4jWsp_mI)t`5xoj1voL=2GQBn=~PA+l^`0=oO56{iv!LG!?s{{Iou zpl@3E|Kuz>>#u+hXV9XQMVUX93`EB(`@f%M{{LbYp2p&i?Qqy*7DbKK$R5MPw!X|Z zKN%i=`uO;X7|LMm&_>oocdOPLg$=H#yMj~(_Gmgiz7yy1+C+_d8k>i`iXc5Z8yFNgCUa+iD35hlau1MgY#uZWg9QG-hD6<^S-vyewoQTIcORxf_o+v$2a*G=My!IcL^KM- z^v-AX5y>X7A~1A8x3L{RBp$d~)zzZ3E7~eghwktRJd`v7Q)Q#qb%dwnF=3KQPAF;l zR0dQx^WU~_;3*7U^I$JOztv~rGMOhLnIpkpdvx-}FI81UVTVIS8!|ad8rUCAB30fvIlJH_<<57D%i3^sU3yGb9xR~?L1*k}urDtqFDY?8o`YIvKNH1<@ zrkF$=^19Zfq^!I=bItHGPWQE;jJq!&hB2OF6oY2NwD$JOk-l+hKV&&yz6a%+czp|_ zmz-VCc>h&b#B*WPPU2|v471dv%^Wp|aOq7+JNm_H&l%02ApMH8%q)i0`Yjt^(}PPj zBN6S%Yj(C!oOg^ zUMn7=0QD#?z0QZOyti~W$oW8ZHJwtcd2%M0=Oe@Qy7X#!@gPlkaovr?{!8y+4h$PVF0vu@?nX9*8q}8JreR3=yhF7;9L;6@8sqHnMw5ESP9l4!;)_yr}SlpoRLUl{4f{@V2o9&2_{PLGO5lL;mVMXZ-B zgYVz;llDNksN>>kf4XzohZZNf$DpgLD?nu+{2C}N6?h;uGVi?D=sPlh3XdX_q?(1k zaH_&(G)!;1{FA1NNgOieq~FH2;pcGF1+8cozd4jY*$fXUw zsfj4E*a0PzL^EGumv{IskGIjxWz0DXaUB)>I}85L~ZGfjqjWGN-ZezD&TOuxf!#cQEF z-3^|?(?x|(6HI?{9b+2V#WgTEQ~&R zO2MCIU9O`#;2Rsalb#HvN4lP4y7~csB8W5_{Z^;Rxqg_yW6pLGPL>)`@=l_<{wKb9 z*t$FC-l_E;JS_&|D#G*G7$A-(C0u+oz>h9%b#@QUxw?>}lp>pj?HwS*68WWpkIsMY zwB{acP;=haqe9VsyM3>b9+5zGy@S&M^q>E%zaCIxB7|OU)ZscF`^F2p?ml!H@!Of( zZsxDndYzwUhT!j_|_* z@N<6tH%$A%LgNy@)y1l-+4EV_gfXLPME8`13SqfP1B#aJr}0DDS1WL9N(EAXI%lQd z`$x2KPfSx*)?&-3%|1XTX`P|%U@xR?HB0*PaV+!ki%B81id`Pn_^uGDTs4~?TRq_S zv1Q)w9_#DHx7~+gl(>#WeEnSO)kC4_#DdKm`0&+BgLe-*Q)b%%@_0oQbgFRGR{YXf z;v`RO-Q{X-C$+-I{}G9S4>E52IFs+LLIA(PVfZQIKL15*!`_ug|B7DBX|>85wG(aE z)j{?!eA7xR5CcKhJ{pkL0$aOkVOs0DBHk!nmEHE68U`R@ei^E79Qd@B*09O^LrXE8 zm=g+^a(grYk?@Zm*q^`RB4Ute-t1GN5d~9=2K-IvczhhslyDzPcdoPV(}R@^m;`xD zjZk~&VW|eqd|gR1jdQa^rFO}rzg&mx zO1%?yXB~~`b6Gz6sZ+1$iNj}E4qnCrTZWDjjc$GCUbI%oVDT%uLYC2>Dw6zEt%o4m5))$qJ= zcfWi0;*W=26ZD7x$hM8TCib-(QLIW8be#UVS3AiGk8bNb5z3_c+?D%={^O(KqVLw_99{gNk=6__&v@ zkmqu3#HWkctOp-t30FVpLY@UjsC_^3w)FtRLLU|EojErY?Txk%dm25FPan@hiBPe1 z@wH*y-hHZ%)9M4MbF1~kJ$c+1xJSjb(rxJp#az7Zegd`cM;*;nzQC3b#maG#kKIgl z9d~u8I#8g?YV~i3Q|Rii*A0Zlv5N~4o(}Pn*PBc>M9zh#EnZeFy6_|#XM3qe_dBBs zhuRuMoK*f#Nl1@2M0|Cz(Q_lGgiYO6w)W<43mo&{!E4Hv=xQ>W$!e*|*c9+)>iuPD zIb10IyTl-)tj{SUv#5v&c&*tJWmgTT^gNpH?^g!+z}7aR`b0`EnHY1cMt&9=5% zg_##~OCxHq`9RVSi#ZPiWWa|}cZH*95@W5INAWyL2GUOD-_ppCl^I?NGt8%#s~18K zD0~)HcJKu1M%5dA62Ee(#=fn`0Kl)I{(o~K?{8r6J59*p0b7vk@n#T3Ls3&9)D)03 zszm}_$oj-XGcN%K^N2dHR=ayBfbgI5IGiTdq;??GcXA$7$bk>XmpNAy*^CZQ3XvAT zrg;33ZJJghu|cg9saNY^C3Pn{qgGQySEydAblcgWze;I$^;ZWVzkT8c-(s_C+ucci zSxY~`u}#iOrm4fw3a$vOVX4}BXP$=l(1x@|PIdgX%+j3OG5`CX4*9)tTa#x6j{I(=i9 z!yaZ2%ep88USwIb$Weo_V0Oh*r*|Gxo-1K1bppJE{bwq``y^mIipPA*`=;u5(eut} z-sWR_;)24A+#}9s=Y-GK{EooM&dr(lf$M?Zk3x@EivCF&~ZrzC3D)YYWUSrgZePaNKEhnB>-_nTHJnN1m6u2q1t64oTZTZp@oh4O`O3&q5H zHh06i3-Q!3@-m~s{?1g=?D>&0*XtHA)Jxg*$xWbeqUaWX_K@3s$AP%_{7w>OV6fwnSe9F@mZgvRbcIN~S)(rmN!M+E|$-18Tf z+BezB;)$=(fbwp+AKs7`msSRbyPBPAEI723mz9C9bzJZ+=Qk6D2sq z*8dGd_{71q(dC`#53H|+tDRjoJdnHZd9JjbWVHN#yks+x=i& z_lFpbv2ugC{<@cFGcQwJg6kcP_iGzOjpzx91wULTGTHE)(et9duaJkGJh3mE7deY+`ZYW7 zHp`}se8i7Dn|GzvPl}bGgf=%fM<4@H6#LJ*59RH9J#}rx@xt{Oj2O_3ni^EnsmA`{ zrIXXilj`eqxjM??#S5#~QU&s`G|(QtgtntfhoEM7-c{iJ6TJ+eeJ@IneIkg^G*aB% z91#~LsF+ZOZva)&r_6XY?qp{tQOCB7I)aV6aFTyJM0UDXf5refE>%C>v{FW$gJ}So zErG*j$!z{e;?B=Y?$)EP{T>GbNq+O^yEEY+imv%tjT7300@L1Y>=%OeRDg8 zY5S7r^^(yF+YNLwvOoE*Lg-LfRIOjOq|?iZ79K_LBN$}N?|*PqQR^~SC5qEGWx$$3 zBV;{Vn_z6hPWGcPzj;`;b*qjTsK{Jjd$QJk))lE29fdWIVu9+UcLXilv>UG!e0&@) zyT}Z*Um2Id%pGhGTs9$Pd+VrQH(b5G)*ivZCl~P=J!5}9P^uoWxiJrT603o^PY5n7 z|92mMgj7 zQcgdK_d(m`L;%LbZ0f*+Z*~k##aisb@4%kAysMxdlwki?Omj|+NcT1@w2&dNgY%n<@%xY2j!-_n%qG9fMw z%ChSP-=o>D*ZxoxGzrYp&T~EWZFCc}Rm9Z!WPkZPO|!}MRgRWz*RRarXW%zSnNS*r z09y?M&6CgZ7OZ~esCP2b=>u%PCUsy5t}Y}q>xHmmec4G}K-M;jK5n(hUVL3{f4I5# zi+U&Fk4W>IL$gqNe^CNKuN7uz7DT9L`n-ufcLiKn_Y4hoxyQEdy!)Tz7bJKbxVx;utB03; z4s9!U2{h33LR~JGj^f=U$>)&?827s$rc0lOPkrAkGIUDg<_62{yD)NU!n*l)B)$`S z`O@+$>2%!`A;w%St@#gfyP01FoDHyVgRaHf6q)}l(boLl$9`6h z9k3Z11}*gE&-C7gE0V`1z;scCJF}bwuIw{8NAp(DWjBVgvT3o7UM#iPAg3Xen#Y>iZSay-&QVhQqZFoP8vj*S|Xp zT{5C~8V>$OEZGp<9m{B^q1XcZ@UTBZ(M-;LwJL zyJ(aFu`h8Gw0?-0-(kM=xsZ)7ZGS`ATQsIhp7@#N%a($yyIUECbg!F&%)1`vy1a|} zUGe1=9-Qey?pW?M&QcL}%vgDnt*>dIR?nT`p13$by&1>*z1lhv2B~S~a69ar z1@GY$JNHaqx2xgZGCk;@1XIYA8J74f^Cr)0h2a}6)|C)NnsQjDE#P_amMXXeMdXe2W zy}B%HdU_d^?w2OUe%kGqoHRqNXJn(StzW*$!y3f6zqb49e|@iERN`<{@V1Smzk^MY zi7!nl2S98~)=*#XHJno9o*C$N74vlnNj(%<&KIjaq)!|vhemvR@ibJ>+ls%2eWi55 zJ?4ATbE?6=l6hiV#KC7Sh?~%$tMyh_g=jihJc+2h0(NW&ahM{NYVhC-Ox)O#Bu6yG z$(XS3b=gVwjrg30M`oipXl9IBF(kwura2T-pG2;%8c<#YuntI#+Z^hEE_IN9#7D35 zB^}cbeJ`F2J$Sx;??A{}BTIlqI-dL>+75?@0$s}%tIVd@J?Nli^feMCmg zz5Xj#%4KgUmttqfV!XN}fVxS*mqzKAR&67?x>-oH_mkV5zpUH*?9NuPdwN9Rd7ifp zBPjDhQ{SsO2cS>SdBeJ+gA?fs>{r33jqI+fQm{7f)7%kWP`#5yRzTJ}7M01Iy3B6I z(8Q@oMvBq(95mMTM4SzBYru#(^Q-#T-`YF?*B7pZ9&f6$UDT1uY-{I>7C;>L6!Bj` z_K1n=n{_FVt#4JZ(xusgNt@ZhEf3p_C|lt6)ApTXny-~daTZd0!~z3QYOpm0$-CiF zT1W1?WqMPN>QzzVvLr=eTp=L?iI-BA3o90W-&kr&mEQV~n|^&YT}Z5G7iUR9WZ8a% z{vp-s$jiD+Dd&R;?i0g$r5K~|I%QT|oEVb029J8dVvdXb3?nb;+&Tvck|%ij`}-n;d|mDpCt zbRG$9H;euWZCNhDs@Wrw)p0gP%D8VM_Ly*OlI~{IW!C=4m7oGKq&g9 ziyVBIS&lbsUWJvG_@&h4Dn+d@BZ;Y}@6ZInPVmb7{H)KfIO7n0!d@slnHTT_vXY}M zPTkDK?8IHjCUS*sKKm&lF@WgrLPZYj!;u)cth^*$(sTKzO_0T{*!}bO_UM&iDU8In z<_-DIP_TBv_?`1b(AA%Gr$*aEx^)oVEq5peJL5Jnd%v4_aJGBQ8b9G}kf0B*Ehpj6 zHoF<@(=C8cYX0G zN~S-|p&!n#?Qw}{_Jy-@9gpCy@8vP#3Qk`LS&#h`vDqt?fL!MFTLqcMYc;d1iK~C^ zRI6R_xct8HxXv@X9yHO`^a{C3(&J`X&JBp;)ED~;>$qBTMnE@a_vNFlwSM^NE?mC!3i7?_OaOA>_#jj;uNhig}ynwlEo zax<>Yxfn6{9lTN2p_Fk=I=Z?`P2DLR+Oqdk2~oBr#CXP-uoi;hzG736K$R>p$Ez$5 z$;XL} ztShkzWQ}$mSIol+zu`N&<`OJ)R7<8acTa>_!&iZZYbwMNxmjXDs6;>Yl@nD z>5g-y8t*ySiqh=~*p{s!aHAcAGOn%I+vG8*F8nNyZ?Voj0UQ6F5;loM>G~()LX`ui z$-6XTQzQM20!i;X*1Yw=8)AGkPR#HZPb%x1t^Y(Nt^(u+om0`ikv!X3bm|u{(tJ@L zl7LQ79HrZ4e{U){JY%yE`z!Ub(oB9b$Z-e{S87%d`RD;B@4kk+Kqvno=M+OY0dF5<(pHPGku;M7a5u3=8 zqInY7an{)pbRv}HB;kk4>L#Ks{vACSP2$dkhW2+Xsvbe(c|aci;~q7*5nifxn+aJu zAJ26Y;^R>6{J|{6VB^uG3_b(S&6#?Ftg?&!@vh>a4aEB6Mbkt;^TDEg(%{R0-Y1y7 z;EfvZh48$A!s24%cKhKJ4wM@N%cpFwjX>DsH?H#Wg*jB;#DA-B>yJ^S^4AiFTex%J zO*-Ue;mY@2E*e1X9>`w&$Sjw~+g@7Uu98)oR14@(JJTW?uL_`4>Fl4hcVJi$B*RZKMqT?YF7Ifj<@vMJhUFP;cn<%Iq zbb9eq*O~9^wB}c2(shzzFqNC-eu<+M$&*D(%5w@5dwufqmXF(&@_{zM3n_~CUVKK; ztl&TOu%()PRCTmaiyyJ-F%$Q9;D^FB;s5L^=`a5(uD}n0d?X*Sg+B`BMaskb%t=Bp zNTihJbEzYKTJ*1>>*u}V(yvq#H~W+C;fh*E%1DvAEAKUl-53GN57UGsw1XPo-O{_z zFekH*QB?cKGSrR3+NrxjWZ)_Z9r=?5!tukKfbPQBf!V7f&AyNyvaGa|uK>NoU1(U# z+7R^nZPU!=@_ zkRS|;8~+zMGy`Z}V*|cIv(i5rN zv+%fu7RpA!&(f4qasbiu+>rn0v9N#g9nru2+!=AlLs^>$X*&N%H1x)AGlpi_20_4?iS+ZmY$2n$Y~+rt4GD)Fu7)7ve;yXgCFo7$E0K|Xp= z)79EE9G*@{w=BizY^Q}wJtYu;&aopbD3m=msMhJ7un3YOenKLJ%N#|Ghl{4<>UtK- zcb(<7(||l{DNjW`o9>3ch*6P+lqd?LiUXURt|A5m6qQoWC#5aU9d|~Lw=z8TV~4x> zpnQT|W_79KL8n55*;r|>>PkvxS!Ax>xZKDakb&4UDl)fZs8qw|L$07 zVB7qy;o}{Me%A2u{ERKNY$+IB&A7&)BI%O?SQ%N8#s@!KK-MpR`O1|ldw;aS&dxbW z{W^4QcJ_QP9n!vq!C=ttVV4B$xLbaVi5uaghqx{M0K@2{Pfs<2oDS_ZGCUN(bYBwD zmCHn!v%@!;ZJ*Z1isCZ49k(Pr+&2vNh2!r%ZM4{KV7O(|x9yX91Nv|-8LHITdyM`K z`NbA{rHfW55;{0oVA9F?2l+AwSQIcn#{w`YB~fIt*I}g3qUM=BK}gh$isya>HCSJ* zrhzc!CdUyH?y~+je_)6iAb9O=Jtq`?q}3E;9g?6A#jYU6^mNU1Uifr~Q0 zH(;|fWMTu})NYg2U^SBJi%%)hfA&gr0-hmcf3Y`qLBY%U$;!#17^)3)>^;@bdQ3L* zNaU8|Z~DXhp1Zl&U^b^DZg{6)s$qr!=N5#ZIGzQ$#OduBW2eFrYza>(pO05_WO>Arwqj&{=q~* z|Lq2t)Hs`$%3Me+1W;aI0axr9UXFzX(tBvtG1YD$mF&s!@^w+ zxU}_Ig`8i2oEERvL4hLM6Z{u>3Bap;5vl$0@4h!sG04se5w3xQ9$%J}cX?EzlUP8| zjsGKMN?p*}k0m}H+_!SSw>uF{H7A+pQ_ewfG&%pPVRCvW9P!g8|77t>m5iR~B}A+ZB?q;VC$vpSMp$BU`he z5d_~}8WC%M2v&AjE8azApWxgc@yeafs-7fY(2sap-`(G7C&*bg`x056w<|0iw^p<` zgeNpwgLmTDBJ58^6Vue}<~8tW0Sq|7i%c6E*snf4ah2nmq;|x{ZxnYv`8n%J3P9B{ zZ04%HB|Fg|Yqe(U$Nu^2=MdzX6!q+|6^z?&s0R(}M|Af_87)pdij{#H2Stm-Q=GT$ zkj0!#fE3Dk?jhPP894!z%|_Gb_22JPVwH&6kPX8{yM`^3nie3o7c{S6AdpF=nDaWf z9DAkRO#TW`I=@_VLjfl7G4R?@?%g)KE9)?1v0WBW7%{5KCCWpb#HpSkqWEV8O zyolL}B7+o$pDR*3MxLV{dGq_gK${(ZIOHl8)KosQcl?Y5cfaweyfCMRZ<*i2+`|pj z6h5Der~aSpV7#)Xq?ezL7=e za(y1cG@z+lsp_xC6$wpj zT%Er>?IA+Jb$e~AZj{n`X^WqX%$yFX1(S?=yE|GsAXl*gyx49RgNW}*=7@Cc(8w3+ zn_ns)-FS6nk=`q#-@XK6l`$?p{=8`Sc?9Y0b7P{wzQ$|IhzEuT56w3xNcf3N<$WJ( zEXX#G`l;wA{6wbxFPXhp#oB(%0P6E#a|IeTM#!|KUCCj^Nr&^zvKR%APgj?lpSj24 zXN$>=%?WCE6dI3FE`Jyk&p`Ry%zVHx6C?PU=zU(2@SbXGJI#{ed(6$t>n?Lr?@BVx z;SbksS*;~LF7TQO=>>HWYL5PbDs!}H+rGKnvi&J##oM|Ud4Tu>#vUaP&IJ{ubEqnKda|lv6?AzoHW!R)~I3Ct#B>xErU)adW(0l%}l3 ztUePeb>_IW6YhO-^s(6_ibr)jqfvbkOPs(!BIwB{v*0X}7P$_Geu@AD-i{3iu+Zp~ zG=#^F5;cXNNoa{Nv8g@gn{GB9`wo996)ItNeVxyb47xn+>e1l!L<5fe%}H~zaFqQ|C786Qj)2Wa(jK2x=~-rxHGh@b_mY+8=iICG=?TzeT*9X2*CwOaW)aYFqW3p?e_ zhAMA}fdu9P?^&6x{JPS;jS)uRS<#a{cl^)i^DK1 zzam&GMRu$AC}&_=1G*6~G$&*MWs=*jT|9O!4M z`DW`6{0(69m8_9wd!s*;CL|LOTQvB5kFC;Md3%yu@2wLPFuC;(ORE!BLB2gRWAkhW z{p&!;7ewrzF*wcww_&wyQ0yUHPl((&@3Lhr?vRqhDEUs0xS4Sy6`|s!$Y+e-zCb0X z{BDK#*xiAW^!|l&QN!tev6y|l$6q_1q~E3|qG4YAFpNsTL2q+(=WVC!D*nG*YkTGb z;8w2ry}1>9i$6NPBiEZHpso8`Jh9zLFvxAMH7f=Kcss@7pN?<6P6Yp*AE;^qvLFZi z`6ZIeW~NX6Y)m{> z^BnRe$|s?20{U~_n+wQ$J{y^(E(VJrEI9R3`O2RwrT_* zd8*-}EwHpeNS;#PJY^9Q8~^>O$&A!fUyZiQM*c7Viv@@vWbk^Vl&uMxTd8@xRd;ZI z6AYA@Rglri?3QS6dez?S9axPn`}0GH1^d-7KCKPHfzy-Kgi+?8?=bk}_861(w)Q#4=xedGhr%#cU>^W5 zY7IaWd{5#g_r-z^(*W!W=#)2V{=%_A?(3DOt{A38RmyA-W&5TGMA5>#yL=o3LUsrG z>pdQP?ZaO#QVU&k&0CWfxWfQE5%>}sBS+czwR0|!=u1bn(O=&9ZVR%yege-q-UI!efDqZdk!{*2 zvJ(>fw3B`d^G_#`*ZMdcfeZ3I`$H@{k9H;3NWK6s*dgm1Q`5yKzWuZS+e(uaX|eky zsb5)`QQqr;HdSTdafU}+Zgn-{024u$xv7E>+S;!!riQo%p(e^3!*4{?taUpl@1|a} zCZCqe?UQ;XDNfl|v@lF)i|8o^Cx#dgF~xK}2gtd&ANwe*uKCUBVw!=t&+deGN%s;3 z4U{T_`3K%k@`Yw8wQQudyKH36#ezPxW=FM&EG`*R_BTI!G+}|gpDd)v;6Ta^me(8y z4t+^$y(>()3G>-sV{mYA_KxlZ05MwmOyGB=Vp?nLyR`{(7~5~ZmU?YjL~xEt4ctLeqf0MGL9<%65m)GPHL;;3o(TGFJ#g7e=EHX#>n_R ze#Jg=sGSwLfQrINt?rPMAM$Sp15#c<)-?~(xQ2d9xR z0;-;~->)PeJnQ4=Wm7q;&1r@!N6=O_9-O86q};od1|NnTZHFh$dnC-T)T-m3!3&(-X-=ojS* z(r8{u%2s0Z)xUXg8PD>LUM6FG@~53?|FWulND}o8vv8V|@u6c?$TpDicEcq~K@gU)yyY zv+e2CB<0=9;fY-6kJYz&w}OH&h3jUH&wzsdc+4-X4x<-pmi|vjN-@-+`q+d&U)_28 zno6xziiKmM$w#ZEwOtA#l@X4LsU0Cng-QYKgmLXdO9L|)asO}u;(j2k_zvV%Mda4k zS9gS;O)(|nnkmHNwY+qiX`>-!JU_AV`!K)0c&>8~Bg<&)bhVm35!Wq~&Omeupv6E+ z37DS zl)7ZZK&AjM8LgG?OT}CvJGDHCZU_&`?-!W#Y{rH5=-DeBvukeA|f6?|@UW48&Bl zwM1{rDtTM6x>JFvMcz@qHiNjVEUkE5$c~#E{!Iz0{C#DZ^lR8xR@Vl?^^2El@wE}? zQVX@aUSM~$wL%{#P&lrB{{$m4Dv>xEuLAS&_iCfxCn`A|IyFA_E=TlmrmWAe9y1IM zaSiZ-6^Rk)rIIT_wV-2_c6Nlcs?sr8&qqEcxaV6$jyu_InT}1dU}lYSVlo2r!94Mc zW%q8=h+3((U};zurvLPo)W^URK`xipK(r_alWPartg3euuph7(+`y`o=%=iActlp; zB5OM$?(k%%ID|*2H@E`{Wp0EM(H2{RTehd7(W?nv7SpgTYMDBW#j{hoet*&MjBF1# zEK#>X7liqLAX;~xfwr|AcNd)6rlzyJ_n&=@_zdY|2t04#6+ar&$$(j|hNv6oJqyIY zdc#&LrcP&Ja`Od%r^Sl@ZKjI>(A|5SoHI9c7AI-em4%TZuk%4t3$NZ|F1}gY3G>^N z*^h!kKGa!saEx8>uL~g-lverpqCbVFG{_QopkpfZI%b24spAEmO4~^z1ziW=UPH=}HP2(Qip>YWA9^Bo6ySqC9ZlCv#@9!QxdRNu1S|w}N zGgU44ForfN7TWNyIlFeqb>!3zr&$8INQbEEM3P=+ZcuZkbs*n+`Za;6AtO;w@|JI; zyB{6+BpFnLGcpeYFsZP9TWC}+x$qzww~*a@;_Q+*lc&3k$gMzan~R((+|la(d)(NZ zB~N;hnh(I(vEkUx#{8*`BwWfuvlGJn5p9m?N%-ze3+3kJ5hY5Zzi`={W3)Q&EheUpfJWI?_VnkEUH-Uap)4}s1#m@>AZPk;w)&r_ z_NcPAV3|3~Y;0J`ulEQ7-Mgru%&~aM&GQzW{@<*1@elXriYV1z8HtH;o%~72^q)-X zp4$q4mL@y9Y?C_X11l!$m-AO0tz*vrDbI#1uxg`Xy`#oS>BYhX=Pb>O!{tP{ zGn3eqK0oOezX3Gb{p`I3USVpTtA%bpT@&iku3R2nAs0VU%7_f2%Sl2!@{VHPKm_Gm z_-TPkOFsFeQ{<&?oe!*33OV~7?X!Km`l6oZ$)wgA1CG$ivY_!swS8|Ouo%=#Y<&7Rmb6zk@|j= zM<%d7On_sxI-jmlwu7U_fnffYn8)Ft?eJ=7g$xdr68cyrAw@m9+iP+`e!<14_w?FS zp%+|w`@Mi9e2o^WJ-B3(^y0^MjC>GmRVzrgAiK1Vu7KDh46D=>d^$1HUeJ3lN&pQ5q z3FjGoL}}gB?h6I~#l(uLF`^wgWIQsbu8y!t!V2>_{hYqu?T&~g)~cXiS4Sz_g^k7j zvL$}`ZLzI`cOm9ornS_WSHlUC`FsIO|4x)mo2==+jsSni^8hC{2imJL01O7?9LYt6 zEef|`Dse_Q^XT#(hb@P%3AO_PYGvy^;aHWwiGH_hi?=y1BTXfKvuvzDnP+p}UyLp` zF(X%3YB9etsl!P;M(?Q(6sk56vt zYz`_ltoup`MdxQSLO(<%Re1#QXNEp;a_aFRAQlrM#D{!k=zf&EAe>u`IgHA*VzFrZ zs-ZOhnrFNp3$HgfiSAh*-z*TI!2}>m`LSRA8-pygNJGGmU`tarTmEv$j#8JFi`#=N zA@xTwwtUYER$P62z3+yWYbf*v_? z7@Bty3oqTrG?@BUIzSs^0QTfZ>4IZDjJ;I<-ZxC|poyMb4l|<~PR62|3HU&!+IdVg zbC)A=+#u^=v3d1#^!^W?@=o)*ABDy~+=Z*ohC3D2u=YQZwmu_SU0~FMS7kw{Dc}$@ zR?h`;L|&|%<9q@1Xlh_Y&Q+3`kST(AdNn@)QL1B2YTBM84jp*(8^2p|xp0-5CpwX@=ISG}s!N{9Q#D@Nkjd>>&MKO+CT_7 zAsIS@zSUKv#TUZ$Z%9z45sn}L8*V1urktiH(az4J$$W8S-eowRcJo-iWEOkeiwk}I z`+d_-o~^ZhLn!@cFMz@%qWR99a_=(@FhTE}HELC!q2_4GNTPMxwFp`n$p27C?NGls z0#Ve8g)8&khp_q5CXb>iGLops+5SeJ_T!sIo!2C}^4A~9q@&63%~PNz07DC8u(zO~ zxka|oU7>X)qaY_eZ#(ab(PcC8Q7kb6bM3;f_?##R7f&Uulxfi1RwViO`Qly{k)^aP z=|$?Dp_qS9k#;xYe>;Y?fb0*qTJq!rgm=7O*vU5w7p})X>fd4}D+J>x?Gp>u*aLYC zSp?Q*+6p9I$d4Qg-|zR1@?8#9mfQtWLCIH8XrUy?<&#UcUSlSF*Ti2OO`O&`D)F zqyJdiXUzkLqe-*P)qdin#<{LYA{9gq8B>f0$s8Mo()$+1>D z2OdZMX893eGEX2lEW&nM4KRxJedwZ(D?3}8K@z&AJ`=Z?otq%%Sk`%e`Wa#Q`(2Es zUFc62;cm{|+~^ob#EA%4YaFjvz(54Ow*Fm=CXpFLxh=JLaf6_?bYV^0DRFWHE5>o@ z`J>YW3h^06iLY24-VDZKcv28e(9ojKdnOSWPB0qAntc`i#<1jwcV>XW&persO@Th6 zqK!A{#@aB$52R+X-`u>ap!3ohl;{I3_7AHabxo^BdVV=kcA;M+-EQh|wfxNVaf_Hn zlX+Lmyj3q5Ek7Z9KAupo287|vrA`?jblN`@^sG@kt?N<^jrH*(dVHj?-#}@$#Iqf7 z*Y&Q4V{fGWi_IJ+T8m;#@omy!=8_W-X1O$rf;HM2yJ?@e)0iJw>4eQrFBm?f%PDt_ z4PUfbydtaZ2?yvPIVof-vWE{Y(qKCjXvI4U?^V42Z--Nq)ad6g10Sp2{2J?e+VT&5 z%#MRDB0g=tU#{8rU=H}5IshZe^CtsezkIeXp}D>3*P!*|3!9PK+iDNg+u!lfA~Wqj zD9YblG&1Uo8cagB`l~*aT9sa-Ks4aECeMtgNMbXQ#J`-B>$6G5E<;E(Tk-+ideC@~ zJOQ;raWt2~k}9&VHEQCl-q-cKY4TD|;fTb?a=T}%V>v?PTi`#+T`X!ndq5>X3E6BSV5Eg#0aX}<(@ z*z%Yvp~zepT*{MnhX+1dFbp;&iJ0pKYcR~-c7VA9vj-BwEN~+|CJwulKD!sh@mcR| z(?N=1n@}Yn%?^qR=aU;B>I6*DP1JK@AdKN;2*2w;D2`p*t1@a}ZcvQ~jS~QbL=eY( zY)j$l`pLrE^^2-wSD{Yxwipcu&K3aq375WYfSduyAb5X>tH#0+PcY;{=(_J5Ah*tK zb7Lj)hA*F12NJ$3zmjtz78un+1G1U=kyPWx{bL2xM(q)0(;a@)Q z$UD;|lso4iU>;Sue|?J5Jw+Qtc8zEN9Ebpf45udd--f(|SE2yA8)WQLB z%RJfpN{-yqOWVW)ntOjVh&y-No|}+?- z@T!+gPgNgbPXhFk4$g1NbBStNW*$R-)DJ12Ux{|cx+H}oR0*{??@x8lX);l*@cMmI z`+vj*;rge=EGgIF`FCz>>L~V3ul<|-?;l*&#?gnEgB#POCS4YKaJM2;}Vu~yETOBOUQa@_w;rV;E5(EF>f&}5j{l$Cpg zNbp!jM8)^|#2BS;_o$>TT!i=n-eFgyfyZvQ2)%sd8}Xx$7w$brQK{_J{p7<5GIwiY zhzN+>fnRpX8dA<2olJ9~Nv^nd+os562O7dfDFOg=^_Q^EFr|^89e;Y(33g)8M&I%xzEPbzz_+4IztV4l8dM~m){uC zQN6P%u1n)EB;k~jOqyGc!EpLN&NW_$tut{9Y0pUUZ=H+XaDr%U;VV$AW`UXZU9e)? z-mrOlGjM*NSEchFdg=AlItgxhdrxC^{H<9=>J{E)u&mS$<{YkhB~D{oy1pCzsv|0^ zvlqzcx1=6AbSzZ&tWZw8v65oED{Hg9qh(HI$M>KNhM^N=WFbdT;H#h*0KfzvFBwJz z0Y25qAih$54^2!n@yA_J4Y#_B0#-+8pOO!YXNoh^bZS+V=Yz0y< zVJ3EF&Kir!VBiGgj#65_Tu0L|J#M2L@m{~2{yeN{d>-x+{&yLc83}Ek#RUHe&>?Z=5*Ki*1DwEyGR<|JB>g#+?fHH}FKXi;Ak)o4V}iucFdvMh9m8MZKemERil{;>xqm+R5?}trL8=N_uiyYhhpqJwQ z$f%<4kJ~V2LnX$BCYYk6BfkXx$)U}G0i@KW|8qf&!>hd7SI$eboM!*-y0Ib@67785 z8MVdQpb`~so`qQ++{;&EGiHgj3lNA674~@;&%7RQvKJ&^!Gr3q<5RhQqT3r01LVp- zuWVmyLc<1bJUOfok!_kVAVmmgVfG9&))$h35UG$0w*bs0m;j5EbNKj#Hy-WBV9^ptLK$%~T}NIEOZeaayA$$)>y)TEbun#GFi8xh)h%B$kq5y0Tg6WjahmYN={9 zBhXu%x`shqx-eR(xx#oElDrr1$Lju09lGJVnBC>I7oTz<0}V+*F`Mzr=jc(=JVR|` zAb#C5sG~sK<_dNOUL@|?VUD80kx`=YPuT}yusDi@9YrTB2QXQrIbC`Dcy4{ z+>HjRIuQb?VXTy5yTmP=1~dDia#<#&{*9mrPfe2t9tlxl;22#U2*O#yY5xE(t=eOW z29k^$Q$UBu1#(>uIKyLrMjT6oz`rFXuXtBrM{Yk0WnYp3sXH!IM%z4kgJ-g@c~yki z0g{ihU;-eux-ydy*hNAa2M{CV$q?D@eQOD&P%vO(F*5Xoo{Cn<^)5+a{?P!H{4)$M z7Yh@LG&s$#Bm=Uw)+Gm3(Bmu+5`SK^h=NADzR3qTlCm{cY+#z@8eVy>>~Y1A8;i4aTE@v z+4Xu}4du$HJG6m;u)9ZhJ(>!{s`mYJzqmVd9}i!f%h_>bh9;mh<|d_U`{%z%B=8$2 zs(4{>5uH@vE*gm>tY8pAf*}(A$(>KTqnJij_v8K~H!Rz&O9F}0k`8`AERYCXz0k0o#;C>93xxiv?Xa84YKW+lr_2cPv(O7@y0N%)T9pR$&@B;k4 zHx!l;93Q=C1}X@KOOlzjh6xCKp!gv}K~V}KP*uU`hF&Cbu@b|K61Sm&t_e}J%!MBl zCk|k>QQtb3V8#T@BckA$1uN5KQVyy7*p%{wRS6o%<%Wt_>V?(&1Qi&te_PG!jB9Nv zkpx!{28%j+ymh}D|2wl%6+_~ql1%}84wRt*=*z#kPs0NgR)&pPF@9j|_5Ue#C5G)lb!jj^yd@0=pui1iNyAo=CxNE>E(RqYdv$rQ4d6*o z;D_g?W}~7BRGE)+{XAu&>+pKFDa?<)YA^&f9@JbTg9-!UQj) zTGfmFwZ^WQIEOt;n@@lZNewiT<^5^U%*A&#oJTidLVs7Hup4#YTV)GkJEmL$1WXrG zMA8q!c55LEi>&GeUc=$sa2CLcW9Kp##*f#{VA0!l;M9b{xaMSj^p3xTh5``c;6?(k zydso#0=5%_F{J=OdCIip2>z{pV=}1~kOo{~FmdVzkpOh2i3UJ%B3|-UkX#gt8NMu0 z_~*m?w4&S|C@lP8m(NpTdrhLRuk?rFnWs2#pK{3Lkq{6Gq(18~zQALYW{rG8vc$mK z1AKsx+?HZthx=tmC-Eh^DRO7-nC(hfat&&@nZlcQkC7s@hBt_M59<^8Oj6I32#|t7 zGm4X+s?v2Brzmn2ZeUqB5)`fE`u|T0P@FZw!8sT^j@$^~lT|Bo8+q3t-wKeJmJ=L} zklaZLhOEnI(LkS6eJyF|K`@4`;6v_v1gi7kLHz2ryZjj26rSe7U)3cIrUOqb+W-b} z2|yGf^DTnf>liIb_|f+;zN_{U|3+8W<|i0h|Ji>~+Ce=a1N=9T2`d`JbB?4TSCEaO z@EP3_BKpp%F^2m#Sn~~_kkkVuMhW7$v#B9S2Dts~q-Z^<`}C#CaAj>)uV2V404zrW z>AO(f_f9Q@Qn=!+g#;4@`+=hDTzV{>oe_t?3=i&~+jgiDR19dje`CGH>k5U#dod7U z2z+JOL~h&Z57pY$?eJL;RmwyM#iIM)J<2iTSj}V0{x)1z>GeI-R0eT{US_}X z*EvBCAr|nR%A6Her8oUm0JMSOXmNNhsw{v64@WeP~7Sm z&r%^qpsc9|E|^p~pFz6;7|t6V-VnPC==8l$UFt)r76w!@vkBzW)4&T=V%#f%tt{Rjd0={|r8Os;b`fSupBg(w2#nS}BWMabb6s&x6S7AnjXCy0J)U!II4 zIuHuKm_l!~INYxtf^C2Na3L_<^3XOA!;vz9E`lPc(Z1YBq*o>)J!cPl5i}XTYDzia zpA!CKHiQN*sCfXLB15B4+Y*LOYT7((jkG_&wB~tf)m-WIe7|VCSnAMmJ@^U!(f>cA zai+&spRO&kRl!_kL&J~I^>&X5uYcmY{2ZkmmT z5|)6FIvB5^mau*~pcbMHKIi4i@g_T%cs_%$!4Lc%G99mc9|(dbEWhLV2vG#b!D7iu za3YSIvaYMy+ME;X`COLyaP7RsKW?-&dNc=1ebOdDE79;{=HleMN$tucB=o*L6!mO` zBP~I@fS@10>XBejXjjZe-ss~^l8L^kS#H#M@7`z|IYhOaByRve-fZrh8X9Uwu2Vfa zqLOrKUoYDi?L`a>jr7(Xr)Q|cQvSV%s4KEYWA4&p55_``H|T426f|B2^pKL0j^Tee zOm)Iksc5>Nr#dKHC;GkCKabds>0`4=vdnEpza_qBEh?vU{M*?G`q~q@;koJOaJzkS zvvcAf6idcmxA(}-o$WA#!N|z?$9W?Ld+=9dIPuA83ajqg?81cvx0ag6ZJ_S%P^?ry zXj_|wg0anyQxlZ|T8mGIlSEuUgTw7so@R)vS$_?Y;=^*X?PCKFH09(*6mP8{)NCAR zUb{-*eR7eaJ*f%q^Gnjg9kJ8ia6D}naNAs+x$MU0vTLgO7N9&;Oow0(D6h!B(MJ~v zAT7!9|5ux$q)qMay~2i$hH^@Tz!ck@!^Xxs(huS~-b|W1* zAmD4whht#(5R_uAj@&Wptpkau0}k}!qN7pmUGl-hWKaX-`acD)zFT(@*k^PGb+A(j=w9ceJqSd=_oZEDMKB6Ehqpp6LNF#^_9@^H z9EzTllw^5T+VV+v(9x@tz|j{2!`&-XTLG?|0F9p0zu;({sIPYbOexjL)}Y|1P4` z;pcY9AxRC_CL^p^0TUymE(h=-*tr11%g3*qH~(?n6Wo)i(W(0L^38Z+k2w0vuJ;ac zw)3H15Z7PLA$?`M)Rc)#UMpW^rvruNi7q#tQ6F)w51U`@*#u*Itdt@)B~#Q%d&J0B z(>*~7e;&~UAw%94d9nj??LB^!ca3zliA}$*g(sMcsN-J;Y}(-;>c&4of&v0Ye$So= z4+jH;DrB2{&Jx5~a)-oP36455uU#+Qgx9Et=?Imm*e+&kN{wmt3aU^$EB3*lr`s#ln z006+ScRt0oUkKHGK!`3d%?>M^FSFmn9vpovmR%|}D;i9?B?@Tk8nQU%-6drh~gtv@G1g?T*HLM>~{0XxRdO zHyk~3qw(pK{ZS5g-ZLp2Mi#r$%bULlAW^WP?uMPrx7$JVdVcp8k0+;?=k<^IG$bx7 z!)BkMre-6cVLMVbElULj9HZcS`(vs5{&ay?(Cuwp_l+qk$9Lzt z@$#=?hKt3AOS|)76JIhaxrpg}zzu$mpXGr5<G~BOnzI-uA3!wWs%y4KW!uGRVUgGmm8`> z);10Kv=-Xmax^}M9tBqPc^G9}K6Cmkp!zYJ>8|=52h?qfypG>AUg9Tf8UpVN+hTxj zX|S_+$kJl>gnUnez-Mi6m!+Px>e=WRJJ^5yj=3tW;dfP)raok;^I{p5ZUFN$) zOZ_A2`)zSdw$^|JHhr@EDdfJv+hF_sF1_Np@ps+}yYSV0-pNeSnl-(<{(}Dse^JLM ziSV#oYk&KDZURCvd7PgBD6rsrXuVBdN8 zqD3Ro0>{rEoO|>!K97Q0$7^n)wGS67=?C=1SucvMZ?Fo*${&H(l2ghrkdL!`T3R=f zNS9*vdOIajf3x1EFr-BCIoH>!&oayuay!1NGF3%-=$5H#W4z(0jK7 zE@<>BeF)fU*;jm>SVbU^YpLJovB(DIdK_G#dfTfegK*q7b*9Y2-g?FyWSVSor!sjj z^IaN`BED{})YzQ$kA-VYTpn>t@T+0j}qluI0|eS5aT zuk@v_xy4KT6MJ@4)E7p(l5x^(=h9HPZgl>b0e*!BNF&afgb=ctHJ62N)Oa-FjZMVB zR3c@3cGcP(&8G)@{j03M*n||RpPih4?!A#DlCJcq?RfF;;FGrK$FD`L*SB`hE;C}1 zTxn&Ywu5!WT{VitvXqZM3v~L|d zqwf|Y53y&)537fGm0qVoI=cJ!+REb~7(1t}_K!tl<;*XY`3xTWNh5RBuf6lkboLdi zvnMii$~kK;=8HWruL8dznBVQ+eAEBV9R2_Nlw5C-(|+Mo;rHAK1noB*MGP%;yp}j) zm!J@8)VrSI1;L=k2!^7O-W>IbdfPwOd5SzuL#_{3SNb1r_4JvAKCC|tHSboCL|O}y zBD`p?w0qbq>sTy1<>siSb44LSzheetpf!kooaJkFZeFZu9hT-3)>}S%lwbbnd}3Jj zRh?;Z1okpz@;dr&>O_LiY}S_L1}H_JDYUeBc2DHoGdGlvd~c<{kT9eR*{Jb12!EVa z$GoTBv2QlIV+ZXgSfr*HIzJb-iFV>UI?j~_V?cSFtv#=tpdz4?5pgvvEUCxoDS`cM z&rd=JN&7-ko(KM`$MOxN9=YlzVcC`gAf8JHBG2W0$PlwZ+!kBPS#AD1P=$m4xSj%{ zoig*M@NLK-jN>={cXBA|?WgF{DXUWidEvx!{^$_s)N)EeCH|D=hr(W}D3GrkwC?jr zt1+@C2Ok#m7WT;6Gm-g4KtsfPwlE{@wiiMJeQ;QRlG8>ji@aE=t;|k%CrPSzR5nvr zKh0>gCGt2Y5pWx3_rJNU{?lgH_e$OrmEE17@YR4bAGmy(^AWQhGgqC_;y(9^*7uOz zK6g!MZ#IRs{%_4O$KOn{ZWG&MATZM*wzcFx@I89dc_z07lUvS8+dg#+CM8$10`g?( zj`!^WDwFn<(ECvaD%m8gZvo4vO@^saUV=&T&1sx;ujQ7{k_c7CT~$2aC(@3emA`bx zlUn_VlHgNUZ?rY`_*kAJcv7}WYu?*U{N8zeumG_D76H5&K zibiu4(W=uaR{<~o#6F}{T#}~y?uH!V5ei!wBu8*rbLJwJiomeHUl{v4t~A&(MVe|W zrE|Zv*|zTx?^})XhKjsg^;NU#*XD)!nu&$1x48O9Slpo6PEE2#r;Loa$@uPOO@58i zs(542((-%zi=F5nIG|pl^gGh)Hhvoh62u*3cdl;SUP%NvEv?UsA2%cFT}GHHD+mM~ zdh4gUbhHfENYBfSjRkxr?|z+}=&vtzJb!u_yh{XarmXsS*k9(>v486l=?2SkcY_aj zY!{z|rtikSW@n`3b6IVu=wtfajL@|Wn#@F3}HvC(0cdqC_=kSFP635*ww@Gtk+p)EEWB zI<*zqU&be@7aJ`*pUBSD76c~Yp4K52V^jv>6jB_@VNk`V0jmQjurMqATqky{E%H#j&o(KhMQY|DvS5Kxq7BT@>oij)_5gKS>>1ZfZRizYz2*g;uj^v8ATUDdwb@C>n>$|CQiloXDd%k(F=N4KTb(kzKkfb^_uh*?IT`i2?ART! z%5e~O@7?!OXs3?%KU?SPKQQm;LXC>90ZyT1A03%&G=6kHt^AOHAG;8AnjXAY&{NK~ z*1Ec3R(MUtMrLr@Wy?6VHCS{vJjeP>LDjA|Bu#?!1xp7b_-y6o~vmW`z6LQ}z92dTvbkoB~Xq;f+C3hmJpUWp+!;Ugv3g5|!f7**uo_ ziG8H3C0uWX41LPoqlfh57rzw1hiLtI9(8_REMojXO)LILTVa9}`-InN>)@WfF-Ksb!sL4FzhVPw>JU&mN*&h6Ksnk*KQYsH zUAL1nw#pEgR~)yxCr*tXUYtkDi%G<>;)UCr_^%(7%74CAt4m&(&C6%4 zY@^fmvO)DPUK_)=ZU~fAS@_xE|m#)T^`}qg}0>N+y*%lxli)NF&M)NnRp!&b( z-GZW>%x2*VPL-5k;npe3wM<|bfW2LHx&5Qzz5>x!f31CQ$;Bo9_l{wJ< zNJ_A}xD{|BYVu#@oByl4*L8w6g5?0z*LEg21Os&eL;zPHLwOVMgY~qJk3$TKD^3=2 z{*WnqZ$*wUJOm1r`tOTuw3T1C&8+u;9H#K&tv?*b@ljC)tB81}n+kUJGTH040fcXc zz%@gzx|QB6&t!_0Dy{5{z6U8NsYQ1lE?bPn@H$w{7x7_-tpNDtdfUuQbOfg6%N0R; zjpS;2&!#WJ_j}l?7b;5-fHc$%^cKwe*(SFLq?*R?e+&=|*ovr(KHsxOG>kCtcdI~%FHJMJ)q6qsKHBJ(r>a8)q4_Q?5EK7_o9sbwc znaY8G9ao#q)xV;uG%3bxabOh1+}6 zKZKNh`?g)XdDYYivqU@NJw;xgjzE(s>Jo3nL>>nGZNM#h52=_gyMM*aeV3%M9`1-? zhN&#@V&>X{&V%#UXaEe)20YwLH^!L}C4_KEygK`PQVx?b{V2&x{Ge|P8N22^D1Ti+@jq=Z_DEnCXeayj8Gp!q;Gqy&eCe0WHHmHvC9f-t1)Krb5o9&y_3*!AP5PPi4L3sfHNijH+zMV^||@yscx~>FNGoqh}v9k}q>L>3ni)bM(p$q>hW$ zpYQG@t&ce*xqg*My{0LL!qqe9Jur^aNqY_&1g>Quyx*6#$c{;%DxfM4PYw*odj&7E>v zNAgNt0#=oHo3jMYG5E`dCpgJzhcp6UOk4t}GKE?}A6X*)6>mBS;SibQT{e3+$cgkf zU-xf53iuzh&SKvjl(Fs-C?ZUr#Kv`ZyS+2m>t2vdM;mvg_F}40Ec>$>X76&cOz-IPSq`9xK8J)18UOb zrh-?`x4ja53}I11krqdr+A`ZEn}0MFdI5b5-(*?D+h;>04hv_%Sx^B(N*n^5-ls~JX>A6yjD2PlU7~$n(tDeOzF)v-z!^T{l)o#*LmYUSr!nO(Tq-qw)h&tW2SbT_gX zsP7vagA}uZdHSQ)1r`Q?12M0)Cj&Na+^)n96}d;1hApZ!f?0hSmI^7Ljz=fJK;Q6h zNuBj$m5e$w1W#HA71?U0Jw4tCO|elak{$NE_4x?KQ7s2j5`N z1UkHqXJ179j@0@gU=8HBS&1g90aD8~s5E_Qv48$&`s#-2n5#1_n|z{iF}bqQA|qRD zl&g!0>B}8J5C+BPfB`#yvbDhMFB@E)%82bm!+lUvX3Z%JUmngazomx4u&vPICf^>g zd->HKZ`LhyRgtc5QjfY+y{2A8lJM2s{l%?j)Bi3V2_cm~;zPNB%SJOI-{BDsCX8!- zT;J;7ZUIaQ5DwT3u3i+TzRWMWtLPSsG6QI1KlSI+Frz&Uau>vl54T%NMR^tl4J{jko%R7 z0+tC{3{gPBDgfrty;G;oPMVv@kna#49U(wE@OnSZ`LNP40Gh&;Ck}vN%}QZ{!~_d9 zJ(vUgZ3kIr@I3&JI;}u7xHlhGwxAA2R{$GsdVdLv=$VYjnv`R*oBa?wmlCLx>`-V+|Mazgz!(j2a2Gb0<-{9pgxHeea@x^ z69O<2fwLKN|wHj>GeQNerSY^9gUzBih+$R~3qZr6BEAtr<0MG>$ z5T@H*znPEr$EUxe{O0@CO$0?z^UG37OpQT)ZBOsVNd_k<_#QL9lm-@QdQLRjtjqsp zBp8u8fWb8^rw53|R?Y&>>=VgrRE7tjQT~dXK%{U8ILx<(9E-jEczy4a{FqiM!dxX1 zv6GdQgh{_pFMhUrj%tM^dq>28S`B}Pf-w;nKV93_Bx0XM+}cfmx6m~|MFV(%2v`7W z9#RdTqJ!Wcx^@+s?R1)bcM*W|CX+F^g%ohD-86Vnta=R$SP|x%8<+scNN5?87(5up z`4m1H-H1mxy7wxRrZItd@dmhxRehtEOMF^Ap2G#7Go2aQzR;ynCl$*=S?P~U2nF=x z-t#@53SOR@x>ZNRz=-D{A?ieQrTdIiU0oz)P*}0z`O&@$ zJiwAhYi8NiF+it6h3|WCy9@v~EEelUw3cwYv8DhDC;)|&6ky|)>D&#KGqX}5$?XAe)au+GgiPxh z5RVDl==!wv9=}VEtF84|$&}9Fa`pa(z@sCMB9ReZ@QX}9$l*q__C%z}0+*LH%jfy2 zRvwF{f{+A%n2oW#Oe+PKQlGkTJ7vs741 zjYcN47q~-$-xC^<-6ei*8~zYJgAJK%A-Hbrp;$iue|%9PrL$*Q$MhQrlmJuQY+=Xk zGhME>yF|i8V|+Hm#3IZfm)g>n4$JYHo_lh843n5amevXBlY}#DdzttP^kcaqVG0+yX2=IKNi)c}NG z42j*9At=~@mcBp_MItl+mR@wyU$7*q@(`#7ww7jF76V7@oJ7d$Y8ELqRU_D$0fu$ zObXPgF{e;yzD&2$5ivV$5WUs@uj)|%!uS(3JSjr3>?R1G7+4d)!vvuth%e3i*t6OOouW%~2- znds6EXlfQ}1TQ5>d7Xpa;&`E4rs)KU1`|uAzwo7(djSHvQ&ww#{yo((xLyQ)C_{7` zo~OsyRC*nznjlA%tDA>|qsnzB#>A*buhk5jP!(=XAd?T_}ozl#XyfBGYrD(hc;{06> zjqVZ?kruB_YS?gN= z%fqVWu8tCW@k*fXDAv>`TOOY_N{o#Pav(7E8_#dXys6pD9nFHkKygt)a4d3LFy?|h zu@Fp35Girf;xp#T?~B;z{bb;2+y4nd9DYcx;6LtPy-WeS=y$q>NKTFFPP46;p4aT~ zK#+MgB&0B}^U3v6`_%UP(ic@zez*O+%JlnieI<0yLsc{*foUXqVqCy;pMXtOp8sRd z)^__Sbim|hylveNtY~W+YyF0^huWM%u~YvfYt9-%3;7<7_`Hj?t?SqJqCA*nIkah+%)43sdfJB7Tt92B)NMn1XU{Sj ziSv7*^?-imGR)l2ABp?X@kINiir79luR-(i)f8eM`$-Z)fX>qDgl zK(R9U)~#YAF@v-#Xk*9;Aew>0Ctq9PrbWv5M06A!^ExpoJl+yi=y{KAVf@W*GkmNU zNx= za5Z)WBykBEa&Q2Kb_6dwakk$+Md+wOD0Uo-=-5dxBtS<6Xh7Yz?i<~4@SmS|cD<3I zi@NRiRi=5<2f^G)XFibyo-L|!c)0}gQ4v0=(Cp6SM!@dWAsC`q==nfrF%XDX&Cf|5 z|AY_Zw062|-5B?SKP3#}$^URd4I9yBqiKZD0EX&vO75ys$KyO5NEN?Pw)xM1ugP+x z4?XkI_h|K)!+tXqX81GUMxa~IT52;*fbZ_Yp|h#z+LH{5sA%7zTGwmh{j|+-ht}X< zD1?UwA}&Ec80277R8(|0B$r|eCiEPTjrKYSobcIY2Lk$$16*_vnNwsJ6k1!|j{joL zYZkwA`ycFprq(hzRb1WI_WS0xjRvTe^C4jSTu3XrgT_NC&$0Q{`%CQuniEt5Gr#nP zwq2?PRi!|4oaGR*v!4#s@!yL+-JMTlw3hwTAkc1od$Ko~GViTM;F7Cclx+pq7LECX z!7%Z<41L7^W}@`F5D|;RQj-~`VceIt)u3s*tu^@p3uOG|J5nctb&FyjfL*UEh>JLc5 zfKOi<5eRirt^MMC{ZW=IF+p9>XMR=w;1Z|7?Ujh95g8KUp_!wkRAhNpL^~Yw{T=wK z7U2EGH`Z;fn*hr3X~2`f!2sy=JD<T1N_FI~xE-JGmSx{I3&cR?+_?0yqWx7>RYPXQM;gaHe0-=tORK3Eogl?~CnEnW zf;mGTj317(fp5UhvWJ5kv7S$M9tH^5q;F7Es6a>z`av2B^_yP?T^`%bPs=r414Dn| z;ZUb$vv;Jv4INqYv6XTQM#2LEr3H2N0u6(WLD);mpg^jIeHuaY%Sm2`?}2FPiI8@v zq7F8jJnv8d2Jv9$6@iPAS@4d*lb7%&ho2}n(@$Q9=&e+Mzk;Gt!HM5VVgz?^&;JCX znUnSX{RVpqA&mx8pp^NG#_%sJQ$3clcwT;2i7x=Ag>yzF@D&mFwyDOC-W&`(o7R5A z^E`n6b~LhtL;RM5Dikvx%M=`CV!_g#5p@Wk*S+^!;9stY3>0`&`qTSM5tuQZ>=^r) zK|?D+QPn*Xrbw^k26h-gB2$n|KM6@uP;qPB@w3vVG-L5)Ang(kiA3S3oSSi zr4tk-0qwWRf(nZL2nPi@_olT|3T)bjsr=8q8p8f)U^}}0C&opj%6Gk8Prp~1fjMp! zKWkHF=LL$lyS&kf-@+fHtZK#}(rh~dn9vo$9hh^6N00VMliw5~v-34Ks=ar*JLDH? z0z5YHF3Tq|C%`X_D>tj(*FHQ5YYcjfKJao>OZqt4t(3G^LBfgOJU)*q0s*asZHweX zrqPzHQBYhoPJ*b7nu;ibl7+%2XwaewY??3$Q4ksC3WDS$>cg#EEgx@7cyXfhj=gsl zjY5rR>8S7iyN#cqeW;KX4p#ywjmcBhg$iWr!DLk8c))XY3WVtM1Xau=8B->OEOp zQzAXb2f?o66R`j9=nl3&J*?n=zaIdRA5QSvomZ&^b(k&B=}#3L5=@DFDM z9wk770U~93V-P|n-Pe4ey?mkG-`V9J@9DVLczuKRqJ}~dQk>o()v?I~eb54Rpn@$# zMPG&>93p0VG?}@6b{3D%oy0T3OhY8Is{Zxq_okZ0U{F^CQ7~WKwN+1t@%4SqLJ4K| zKm`*rRCm3mq`dLq0kfv5C3i_X8xKlRlNdhQNFXqQPd@gu!Tb`?Ba_I${3}KMk~H4X>W}&iVK3lvqe4O77jvz#ijv%{B%7q z_+rq%iEFoBP2#Q<(#L|oJ+Gz9S_&7;?O)vIL$1EJKE;Vk9CnegsKB(c&%90R z*jqEvK?Rc-@}p(oIWl!hL3x5w#0;#&VDSX$PUkV*51g&YwJLOVZl)3NT8iUNw|HtA0`r{Bul8d%YH zNm1wriadl23RYG@=Fxb?rRj)Ev!nZN%KAX&;bt%EjNw+1aNuj%hjxO9Dj z&w`op5b^~0Fw!yuU;*m}6oy17{C7nC44TBPRo^JB|5MBE5E=%)*in>pXI zOfo${9;BZQ{#5y|WH6iaLiOzjMt^3$=DwQV$_VeIFwxN*uh7hJBG-l)wLAP8?GY|1 zfa5i~F7-R2SSL-Vmo$n}CiC7?;pz>EN{tN~Vgt)FK+?*WbY+#rokBDbRH21L8dkmL zBZ0Gl1ex@7?y1o**30!gcS(W6-IwhX_T0a1(=7b@hRp4K;;15%O;-1x0(KONNGOg% z6d&Ybc!*9pwD0uWeKTeUIml8`Tg-eai`{CW_B+|obQVnfY*GbyC{ntl3-qMkNfxo-XdIsYM7Ht(tPd?jW@MEGM$~luCupoaHZBXZb?<@DhDW`a4-_2 z`?;#ZJ$)y3)HOl-_P>olEWjW|qgvl-VuY7|Ze+~Jzy}fA0e?OciTOX$uB|H`j z2cIsi7ldy-N5Wc`NA_mvLe&-3MNiG6 zk`&Ykgv#dkk5N7(c}$?BBnFY`y?Sta?rqn199S`sGe}a2g5fm7!(>v{n}dq7!1Ve4z5*F{;*HB8rsG2 zeqRG^C)?Ks4Kf;}Y#0zLVqJQL1{yb{dN$b|?xQngdMrA)B|T-#uBXgIpMOU0(9#!d zj^GWvv9kA5GaCq5EhcM`c28z=yC;t$UMFisH285AvhDRH^UPRm7_WnT&9I&Y8IK`$ z!(}U-%=6(69tU!zs~ez<#1qNwnZxrSWrlGA&VD*~g5(nHbl0)(O(J0wr^-8c7B!2R zk9zJrrErHIHtj|H=;qWfWZ8DaBA32(!)@YEZG9EG&X4Of7X8HV$_c^0d4xm(uvaBn z(-{8!J4q2JwXODARyR2wn=0B$QseEwcE*kN>k6t|yAm)h?!+OH1a?_-~nigw@P_@5-Xm_qk?q!h!$nPCLr_fVv5-mlyR% zIkER)n{B`gkz6TPYmQUI%c+zi^<$D~15NK~uxeX^t%j{JlL~yV^M3(X$MMuW=mGMv zp%~&~*#+5vRv}+H7C#=S+d|i4YS;~(iigI3WT)Ru|Qs(9lX`Uh9S;> z&5s*=b37KYKuPcIoEo<P`eLX`nHw zys_xg{1@{NleZjHtO1by_Tr-7&`f{?M2fD!xhFsd^_2s8c0dQ!I(p1cu%Nf%4xrdf zPEJm|z~f+92^L`I+H7xCJ=TZv`y*sw!?EHWf9Zj(+_Wb4=f(%_-3Uvmw8AdWTio`l zQX}gGXkFYLDAi%8kku?!G_*v9kaGv{G^tvor9pfW@XNxM!7qfTal%o4oSe&Hr z$cC=_#7xHeX2)b z_tV{h=*D;k^oEW>*PkSFd$`&(hJ7vO=oXt(j0=S4YNgnFH+uT>EB?1B0Er2_Cb&(w zlk3u~Ds1bjAy+S7#zqF*7j4FHf4LXuG*7zYl2W%l2rPT1z6c({$SpBs0G0f6fx`Dr z|CRH}ERKwX8l)@1Lgs4z9f&|9eV#}Nu=I$14Xc5Hn?hnkiPpAM!#Fw5r~*2s!yLHOCP{?N+%GuPIh9j=WzzB_CL!bIs2vN`aunQVN<~GJSntGAdLBTx_qPqCEzyI2-S{2&~RScWTmpfoKt75_(1w9M;j^pxY@DQUw^kD z7xnIHPdH0GVSv`z552jHD74z}QNqt7i@K=qh)sZaGPQ{7y(_|0J5o`wvDEr6;YTi< zK;GxNtinHShj9gvLIgt|UR~ObAA+e?@imJ>%3j;z{)v5>rTy0x)atse!ex$QOO`|X zA~hWakhDnquYNx;EqN8IX3{kU9Wsor3nw^KmXJcSPef_QbTH|7aG;UpZqMj3@%!*V zZ57KbY43omfP{Hd-T&7DOdDr5^bMqhnRc0QsI{2BXt@IIJyLZ z{@YFQ$AG*PAUeqvlO|fU;Y@)*BML#~GFkAH30bD|({x7%`B@NVJ9v*5dP3id!S@ax z;TrKbLcnOe1NGjticqn|z?1Wy4$XFx7%u3=N!9w#0l^8!I2M5 zxQ4~kV|79;&svF>;kB7E?FsR=%py|^o<*R8)={6Te^T_NX<{WQZNmr@jAHRwn*G4W zsG!x#UPHP|-*LxoIJYC6OA(Ru(p>$6d~0l51#k=>)%WpQsP~@dFwPlqntJoVN-O1J z_&xi@lpz;1`%rXb9u(3L&QSpmDWzW1lD9pRS1v8&6S{eyzG1m-Z+zSVhy8WHeKoi* z1)N`}cZRu{Y{fVQ#G%{X4vu*{uj1-szf8)`(WuFfUxsO$EBaFPG6`-o2~svp_*D{u zT{Xx~MIa048YiL`);QBjg0W$;90iH%3e)9sabv|XL)6(+mIQ-d`eQ@J8Khh%c*5?! zQWHZK-a|P9K7!sb!6j>-!Oc%;G<$EoAcUKRwsIVtr98_g9ss0u%6HgX_dCno*2Ici zpi^toJd9;@3ufO@&%wCo)}y;hX9*Zo=L~Ye>7U#-C{&z5{2y@1ydY;5Sd`$3!6{vq zKm_dZ`AOL!VdP`F7xY_nhpf{%1wde8;D1;516>_L0MaS?l}R zY_m|S+bN^G5G_hqAiOdHS}Ow_k^9F|;&G6yq*N8g`yk^%y`3Ycv*G{=iJ+Cc zyb1*@X#&1PkbcQFRF^-o5gPWK(~1C_HwPEOf(yR9eMT`o(8AieZx7Fa%eSTjkxjX` zMF7vY@clV0685yQ+=q9jLY^tcghV^iqm#YDxBVmAfWa~MTjc_%o5KIP;z>skfy_|o zY(|_*f@?|dt25s#i>TIOg)HuYLx8eZkxI4&W5OX>!L0iigX3hO4vMurQ|mwTn_~o! z0f%Eg5!4tt3Bnqmx1+%sFrmkb7=~_qQBiHPw4lAby}iApyr86{v7~;1gu)2!W(Qu5 z#pL_E!JKSmhyd4W00mAtdX(T~_Z>ccTwD&*Q6Vxv24WTo(%R8SLNB?iBpM_GI7I1P zj9~xU;Nx%TB+B?dSp6p@2z_C)zygJ0{FsLL`z0BMST2%jrTe2z5GBwd31f=HxM73gM}v3$ z$9}Gc<8(%NE)kicI5<&$cwrEIJ1m%!jFA0JWOVz@s+^pem!n^l-uC4h@1}XiCBl}4av0&jTNzKv3Em9VS=Uqgk00hASlh%1m`Om z>flGCLslm0(y)9eiz0!SdOKJ{8iW~S;wl9a^%~7P&y!#Q5DOZ19X)M@;gF&sOJzI3 zlOJ!svVPRCNH~FQ!#%6_w%iRo%_w%zJtpQuq6=dzr2c|7UjrqOA4Z~;5Q6BP5D3C@ zFX%tAX>ee!a5z3+zj`NoHd9n}ioGSj#jRfKYG$+?4_?w0vUg(d>4;2b zB4;Nm_U>q!n|hSEx`?q?SPP z!?0RljLiT1Fd>1gw8Mpq1X7BbSUBj#`Bc5M<8ZtlFik;ylS9q<=AU2TnyM zS%P7Tx{&D(Vc1Z5PLBKyZg$*sQD1;0qLlJi0#WfAp_u3j`xR54fkf>0@&u2IwX#{` z+Q5rHBH`wB$W)?Pk1x}5xXPej!=ry30wO;FWQ=9k26TSda%-`$uwn^NgCVcNCt>KS z2%cltGQjThIL9s$GC>jm)!3tPy`vx`G2q)Q43Q;lU|vJ8pS=vy8aX+6%NCo(+%vOu zSy($gNd9>RVrc*dgCioyMsazOZ$AiyW{?LZaj+qKrIa%U2>bUVBEMS!FSrM6qVu%w zQ^1`*D2n1nbTu)Cq2=RLRP`$%1S$9fEx{p8MIjvy{zSm6jRA&pfBQ(gU4au|T{AH) zSwJB}Nq}4lM>p#ln0p66D23vEWr}VvfTpEb_rsLK)%*2S%eH@ZUlyOmV+Poeg1|$o z7B-%HWVpi*`YFi*u&%$RqF_G=mO%!7Z%ZvbCbY`=q#I>|ls|b0K2N6~fAACMqd2WK zS|x|6Uv9KBZynVOL_u&N_K%3tO~WQ}AxKIGLIw_S*#bpbxO}DD@s|v9r{_R6yBT?P z;|KT{LJKEdzV%F5Pc3vcSztiS>#9HpC|@1}Uxi=f;{+VGD|#OOr*&U<=q7|hn9U8d z;V6Rxw39;+1G6LMHxN}T5Xim{QFQ|#X(ybLJU;Jr^4A^VloJ>ctFVvNm$%2DS&T{p zQR>|x7mot^c%(^;VC7VKdr*xn*O8eMU&A6Hx-R~GhfI{(>@Q^hkK5T_^iF69;g5u# zZc=C?f|7#C5!jhN1Rv6>L+V_;k2yLM$0LI)H0 zrx|ocr#5CFzlGG(IQN5&kIiuuJL|7g*n-h?9hn>IF{sl-8LSM_30`cuS=38>99 zR1S{G(gGa_tAxbTM={26I)?OL~^*_jfZ9vg6Ov$1Lo~>2{T|dS zhJo=-NZeWN;+328v8FNLbw9v9^p*$MYO>Xd7x4D(f@>=jjjHH6NNAv~NOd|5i4-MX z`|h;s=OD!pkkE^aBSjj;gr0@@11c^U(ggMG8HFG|#=L^7lYJ^8id>bEYY=y4PLe@0 z#=mQ?t^Awwm^nZOa;DP~77mUm&{!GQ7Myb&7P3ktD7KC4`q+_977Q8Q^BHGQBs4FZ zZTLAdEFxO0glhUS6VqDo0mMMs+I7d!=ss~!bG!0$PIARi5~U)=7#*hg1^E+AD3Fz- zFXn1e*9Z7;H|aT?`&9jrJN1lbVv=oW-W$l*ECiz{o=6k}8zsSQk)p6=aRn@Y`1}jp z>e^qGkQCnO;^*(K;@<>GWU5{lwLyP|5(hcQV8NE?i2k6=l0nh-Nefw`8EW({*6P-~ zv9w1OyF*4ordp5Y8^q)WZus#uzuyV?FQ=M4Z7^?IuXb4iCtzt#68mv(P`wf2eES7yy8Ym`U1SJsi8R0tI@ zaX9{)Zat<}k%_qdw*X$A1kDUFxMp9UNj1~9NqGIw0Uig2%OVyiYBox??9V2*cl-+j z4W}R+!3V@efY-X=BZQO43YVO6gVp*5P+B4X zb>-6lk6LjXo5P<~j`-~z2bhk{^|_ROP(g?bJ?aKz4sacP3u4KtswH#{JLVz%D$0et z+cd+D-I{Bq!Ha|KBjSR~2`L~$7g&=@Ary) zd2*6*w|Cd^h_8rIRd)U`lsPYsKqxF4{&?l%wUp-$T&${--lwYs>>W)cP+2O`%@JLTuyZq2YcPOvkUC?wKaUXxQcxSTC5p! zj#Sy11T)OTc|3b5I|n+3@lx5GpWc%3|C-abAi{>%`1t#p$KAi@8Cj7`)C2tXPLvJ@ zwN_+No25UG`reWgbvd`V3LH*Zr+GzKUVA@~Tj|5k&aJJ_-!3*go(=^*tILY{t8*w)<=)Pd!lKR#Z{ca`)j0pI4=gmGx2Mr}OqdJa$iNHyWw( zK!cT$^7%1P&G&{OR&nm_Z*+8@H~!HNgarl@wdO>A@ zvG+iu)Bf;E5=XTG>r*kd&f3pcr0>p8>D|tj+go?v?j3ZymKJz?{`p3nKs@m!SpHNr z&`{CYzPjEOY4HELm_2eQ7S+?+OpCqZ$l;x3c#vW2a@CKAA!1uKJ6vp0=<%Ag%$xRd z`MGBgPF|d<=E1SSYkwh`u(h(nW$`FeWyc&|fT_mrvBk02NLz{gALMCvAWOgN>1tcx z4H=a5YvjSGI@kIN?ZEqnL{RUyUDs?99+3^O6a~F?sQsJXExww6{!YihE=hr32)!~c zN=iyb&&>O-|1Q$yn@5bqKz6hIEiZ=gYvlwFAIJfoj!OXo9`$Wgbp^;99(gqmejvlG z-O4Zj>awyiFfesBxl2w$@7^=sn?aNK=An&BQs#*j*~5y6a53KZ`@LO!0Nn-gF&^)N zIKOa*&m-{p#YEw6h@O7VV33=Q##DfpyGtn^B|u-0C2>I%JS#s85zqZ=+h0f9*6Lw^ z@N>qv$>%CA@9WIza1HDq5s7LIg3ihlBsi;)|0U$l-GMe@B=yWMEzbfsm|zx3+lRF=L#D77y3HjR_+B>Sa#@ z38*k)Bq0iV`XDlfAFqx}g#POb3kT28k18ph|Mz{+8fdwD?QUAx(TVDMD*U<=f%ksO zy%hL*NR}33d9Yi~^+FF!jMRnCJ~;_cn-Gyw9pmY%;O{5X-ossItr-{0#f z_mDQdj&t(5NQ#bOi%z`C^Nf#F7M=fU#AmJLb|GJEf6Wo*I#^lFrfe}7QziZw0STlm zHO(+nah?Ozk2{QxAN*{_coFwokOEG!1Ck?6>}ZmrDt^>-KEHuTByH;Vf1nAu*xzB_ z`I|Gd4xR z4U1JzjFQe3{TX-|XiB$Onm$f|sT0__miCKuu>&(^s!bF)*v07v{!-I_y?tKxAaxkT zQs}bTZVHYD-X4LQm*b;@Hzp$pmo0k_oZ4u6CDbaf>gl{MgZnaD`m8TWw6E$C&33B` z)HI;B6aE5+LlEPL&*9X^Y|(95-+wlX{i>yK?-kGfM5UbYZFS&I--l0Gd$vL1GH%o7 zM}@Dph=^b0-C&J>-9gUqM0IDoM{$$STbjHQv+pw$>8lzqh{|-A)~%tf=J8=cQK|EQ zon9>CuA8Fo|5D6XSiL)siYk$^?KsfXHE~}<=;iroQ?p)-+2y?eT z^Y+!_b7OTDC~Rp&Yow6l?2?!AxVGMG%i^m)Q*-M9dP)n>Q>IGsXF>^Qi9v={5;8sO ze4PgvxGk!I=!2jvu_uVfZs2@6xOl$Q-0^rYmYjubzux2EAghkZ7vSlKQ=`++K3|^8 zb-D*<@Ar6Giq+i(?3YS^tg35qSjA|mD3#0G>e1!}QI$ac7JR^hXwU?xGaG16mYb4E zW7w=WdE4oOAp95`fIph{J7>HaN~M@7p)lx@FY{je!q=X8+OR0DO>R39u^1i-DP{e!WR;)yi-)o#D*ouHoQ z8wIS%?6)O@txMH|@wqaUlJ8yGua9d|>AG7jYg^rJdQx^ZO8b<2$IBg_eG@7rx-H%V zVd|x(dm*swC!UX)rC3$EJkFD11F!0K>#YoZ?p9VWi+Nq;d{{R8PDOga|tLe&Tv|`mBy;#ziusTIIemwZfQl2BS8?H=CQSN z`_|b}qrN@P%A{L;KI@hCRdNsHTP@4$NYW`k9MUc2od?{>3O6^TqD{pINB?lI+B_EL?` zK9xDzQKz?mLg`F^7%$!4ZYc&btOkp(J9yWB3)+7>bLKy#p{x|yY1%AXp{^??uM`78;IU%TuJ# zpJQv#Y5S6PxgL%{@Eo1T%NwxuJN2p?TgAP<{P)%j%2G?;hqwJ^tG-Aph7!q7|Bb(r z7YC13ct4_;q{db{&1?b(Dz>~TSRul>m)c!}`gRekTKSYGAdg? z!?L~8+v2}2cD-txwLqcz>cfLJ{JXZZG zSpnh7atdf_H9G3&op&|kLFlRxv!TJrWNWd>1{ptmnbGk`bjoEdwo`n?!wh=TiS^)gN+1>hlELtPr~|043N7)&hpkYd9$nrc_Ff zaF;D41pN4mNejM>@c!1deGkn7wR4c04&8_8a&a^%$lXVd5W+uix0zLi3V4BDzT}8~99-r_1X$ z+1qvCT_VNV#jUd=&Rg8qV0}Fd)}&SHvfK9${d#+loxd~SeNTpRIj(u1cymgJZRD1l zZgLfs>Zh^Em{l}5;!oWz{f^Iicl#`r{!rEIZf~_f%{4rEBpr6a)$BIk9@1CG%C}v- zr$7Q8}PKj>XP0G`nvcG zq@piX`U`}CFldZGf={+drq|}@DP)G0CXpzY13 z%gHl&y55%4-arW{;ZYk;sbqj189gl;wNWrry9_~zn|q^0Jl9fSEy*z=?){le1M=0j z*RqB{kLg&Frm)`wtw9|b>{8%Py<_XV1vBRigvhw7$$ zFv|$i@W+uXh5C~QtNAn;3B4bYDfv0*IXAcC?d|$q+HsAHTJ5~!mynT`jaPwS=V{#u z5lt;TX6Lw^y8eBFBrb3S5=g-4CVY!oi@T$$^jM&Y80p48BGJOf=Ns&5|2xd(hkasA z@}K*MhfRDwi!B89I%nw$vyyrKe#_m=9f!}mSY=;q=Z8V;!sxm)6E*+n)uzC{upi5s zMq{%tc!|Bx5-F%6R!=XhK-%h%FfL!m0Er1Hvdt{9-*o+&7nB$GFL%53J$Lu!br568&*c0(#&<3Rf@pNS(PlkOly1rDF06{D24_(v z!B|n#UZ?pIf4yUvq$V~uchuWJfka$^hUU!s{$kO)$j{22uMLE{Be`k^s31uhd?um+ z6^TS;4#dzIbRS-6tIw|l!a{-nUke~wU#M%wVc365Y^h$J+%!N27!zXU269y=-Grby zh3AA2LG{&=biF+pwD_tg<)Qw9v_i2s`(sEyD%EQGI^?EDxl2QxQag8VJ@Kdqx^Fs5 zp>e0*A&+OF=P=4B2f=_kpAq$zhKBj0*mZY5dsXCU{OtX!PcL#~N&%NcwU$klqRv*^>HPMTC{=55dW*9W zf2uDz*{7paVo!KnOPajb~DxTgwRc$6jk)!49VLaNkOlZ2`L2UK?xk~%?l}n z;8B#!SZ4%*=N{6|Yn=XyCVRfcnz>-pYmoh$`ji_q{(;(M@)X3}bw=krwJAWjDu3tP zeYMfNIu&X%cHD%;i*Ms#=Y4xKLV^#`QA#)h`NpNbzPgIDvocbV2bPCI(py&_r48B@ z=+*O%GygJq_;{=K^K)On8JJS;;W|GA^KwL3b&5%ZA93xqkdRUS;^S}F`Hq8sj@r+R z4HgG0KwdI~lA?CG3x6vuN8bRyIKyo#B0!wBecijixrThcoRu0JR@x28<-4KBum-UGSViW|$ z3VZZ7&g&4xMlyby#p>Bq5*`=d=WWVNZMMSUYmO6?Ga*=Ov6z*azTEOq3=Bn?q|2vw z&x#{F<-$^lpqd~MaM%6$C7G%qD4R7Hy05am-HfKM?A14!lsZ)h;nVVQU~vwmnLO!i z0N1E`vs4liqaD@ORaSgxnoL_nD(#jqoUP(M+2|gWVpB6l2rH~v8M)%HJS!huJXQ%^ zWVc~sOw7V&02tJUv%Hk6(6%9-9JXzTz+afSh-9l`L##!bJD?>2mqKKAe-{(Czf{%I zGAC3>X6@h@LAqRhOwOx<4nb5<06yP}oPJ1ZJq%8NhN-?Z(Dw~|TS`yYjK%ZdqGIQM zi-3B6BY~r|`wc_pVGtoH{kDYTA=~zHl<~Kb?))S-mWZ}sKQ}l0-tjA!k6~5oH8e65 z-^WO~hGv3fgtJC;cuvG}St2mBtxhMq0*{Q$$~fKC_u=iz@462<@obCL!J)Ov= zYa5B0`XX!{*vUOuilqdD7TgGK)f~#Q-%q}-Ta>;49h$&zIaTdJsw9a#^%iEd|3_*G zHSRF+7o$1_35?1&SmMSd^WPwDcKeIxq^aB@u~mi`?SL{9Rq(N>C{b|Qme~(9VQUKF zcNscgBXF2%;zxp#Y>D9Bf$4LV!2v;jqb~ZrT}l+>#abc02^q0`O6p4RtX{}I0P*+u z?iC1bK-oTX*O--_Y{~-XC9WUo<119z6*_8zbdObxtNFOt7nTD`BF1AQj>m@Y-Bb`h z0@ByZ17_-MMQOkb9(&WO)5^xKF#gVUC~I|BYf({=o~l}^Sp3o8ZjrEe06WfgEDg&f z1s|Yx`SkzxzTkY^caQpk4*K<{wZ4;b(ZrcWY3Ra3s?GCXIJVxfCoLLuFB3Yndj%Wh znGquxT1P*_U1|eXbt8vB7&J99H~dVeH9a0V5XKnP6Uh9U05y^yL6%5{A}PZa+!zKB zV^hgD3u??16vgeczNUVyqOS{?A&ArCvcI^uVPxhq(8A?zdzhy7zro1~V0D^&!nS{# z+g4p%ggQir`X#S(!S;h^&Yi9V#mFlDM>=9UN)obcYsM<5OQ3GXbF!b(RI=GWb4Gbc zCkwrP(7Cs2ihfE?E)FG*Zt-^_7-!jW)}IwJL4{ct;_QzxBBRs}VVxMkf zrLFM<1wW%c$WkleF z(&GrbJl0=JmehYZUjkZla_p+)a{A3g*|iu}4;R_zOQ$uj_gwJsmfc3j%a#rJ${p%? zGPF=+xU#z50>B21>sDQkrgB-QA?-6 z*>+n^P0jE8{uEx{mp?gR$<%&!1{?vU16l9grAWfV3S%7VweZG|)q5h}qqH!m5_ z$^zvgK-cf7bJCMlm2cZUWA zOz)?EV=1L+xJOJQBcV$)Oa?JY=56A|5w7K=H1&frjeu-0U2iOZ=MMd>Z=O87^V8O@J49$m3*JsC)8P~g=@w>oF4JpSTYTSa zx9UcgY0F^42n#iEXO6f<^BvYU6vVdFIMSAgIG*0Yr8XK=5i0fOgJ)cHhBRr#z9HKQ zwcZyb{%<}30y#8aW+Z;2p}4NBDfRym^-h76HbJ*=I1}5pZQGoQZQJ%l6Wiv*w(VrH zC$?>SpZ$K{IsbpLZ}!d8U0q$>RlRDhf2!$8f#d?&@7oASK@8tyY2bBiS} zXEewN4e#cPRepq37fya`We2BpFXOt6jR8anqUSe%(zA}I+}i4bL;wk5xo!wQWyk7y zSsxvBXZ`8F^vG`hHPhc<6=a$cWh&^q@KlLkZ~hMI_Bk{13v-q=8)BW%a95msS9NIj zar@JQM-*p$_aLm#ui_j7{>v0Bg8_ZEeAV3gyH9Ph&y_mr3;b}D+aGQobwq+fv8g%GSHIvv+bL!SK7@h(Liv@4iyZqafAke;;&s*p=5p1# z_}NZB-?$vs-XR%&SF4ftFP7eSkg<{s`rN(;?mWYnK2GuO)KM;1Xsxu)Ooo+#UYlM& z^gV0!SS=TQuW7UtR-Zqj9yH}hlNH2+7}i_A`h@aZ@BM>gdChZ zR_ir<#CifhuBz&$W9e&Tt>z|PF-D2dGc%Pdp3<&R-$_%S^`O)~Rii$PpeQsuSnN<&=@ ziI9(D_}6n%cn3X3RA}0K1tNQyQbz;FV0DfJnP>=X7QypP@UP-geHwZBOBq>l@gYQR zJxiowsMe~dGfcd+UDu1(UwFb^(S@m+QGXlVY=-Pr%Qv*JFspOC1dsqwHG0Q?3x`ap z4GnZu4S(vpz7w)D5ebi)P>MoXbD>t!(M2HQV;ya~YYkOid*;>GM9w4;keN&_82~Ia z$*njWWbVll6julF3x~5(`05?E$D^5;>MrNIYE3@I2S8OH)q0wk@PG!0VTZlHh2sBx zM@9F)?~GltQyKE;G|~?sBsyuW9Aa{|q}i{RF!qO$(5slp>LyU>u2G`Oa_tSq67#uv zEriNX+MM~Qr7GaL4g^C+U{p7&F_CWP1-TJVyo0m)O}9s7ovlTUhc;jn2)b;Ti&A6? z@M@}=FDH|fDfs%htS``Vr%BONQ|okmN48i@b|VW9SMYih_;dTKjf0-DAoDXJGQp3m zRdF&zDLEdKHdbelEM29wOWdyOsu1$KbQ~eIygVcS**&CUB4`O5GJRAl7?XWY43%=7 zwAKz&Tiny^r76+JWLc6i(7-C`t3W*dZ~!<0Hs{b)i|2GD4sirx!QRfY)>bk$=V|B& zVSzK3VbO!4iS%uO!?dV7EBnlLlf8CONt^;rj^6!dXCTpZOq-Ilqaoa5Xn*h|egp~d zxchPOwv)}E!#fraVrRZhN~+AtYf&$rC8p@l8h_YlRCQ$g#EIu0koieLzUcq8+XZ~f z1-`bqqrzN5h@A2C+{dkhY`zAlH9~6h99+4NC*rM*`JHq35mGE2PlqKTCSECXsCgZD zGEnE=A=a6cikdsn1AL6%2F|wiY7-hyOxl9~mT~_Se(|E2*fDPI?sS;G5&ho)3WFSB z=M`AGB$xQt$ji2cY&W5KveM?vZaof$a31#03kZWKt=!#QU1k|S35Xtx6>!=KvKWv9 zy@N)Ok15jCR9>_$KBOEQAm^yacfYA8<0#;^CM{HjlSFi{FlimAqa zB~@8p&ia2z?V>v1J9a!&tC-VZQZYoD>bmGBzW`=r57IGshWh`=Uvn7QR>dwV>$ZS1 z5m%okVQyiLDs)L6il1%3)A#MyC2m`LrI9uv;LT>>=WN^8(n}4(X;F7~=kdz@_nJVU zqJ`1`D-g&BIi|4UZf;&W@VbMto4GtkioazzaX}aSDS0y{4C0*#eNa*q55Rx?0jg$U+>z~Q66aHyC{PeWsqz51B$dmL($r(6&KXm+$( zd4{*IOel!Avx1vCYtfPy3u{wcqdyHy7lv=H@@LSeLQUc1smao_D zb`TfT1l@fe_bCm^?fEz8@tn0$Qj9c3JI!3Jg7XTOtM&coVraSbpS2}HFpH;I|6T2| zMGEvf#tinR-O<3;FCZfBYPa*y;Bq>I*kED+FW$kW9~&#KknYp}@K=lq~Id->)|Zs1ohneF|k`#m<3$N%i5B%6zaN$&-Zojn64;+c4(-eYDv@O&K;tIn|Ptim!cT7oR_-ZN0x zX-7n!x`^Ptjrdc7_VIvyuCLE|+cx@WTk!KJ@NMfE)9N#85k&isTANO)XqFjrsiKzn z5dU^spidR@D>I%z@b0m0Q-esE#ZEPoRK=7u$ApFTut!y?N&NC zk0OPN0*iTi0bQmYjZR;A=hRi&cIr#8>o)@;I6E{vgp(I=VAS3A>q~c=smXh2jp6Qf z06DNa1!SE$B8T&oYD#LDNp{o~Dy%`pJ8`l`^ZQY6`ZOzuZu5Py z6a4skqr>&IaT&8Zw93$HH!QQw;Q1HvM^(rMNdF4H>FdA>)PUPVOz}#A_9G4#d;4<> z8DlCIe+lmswKiFqm_*P&XbIz|S)v?+0n2rjr{a5h;CrCQCBMhsOCp_2y8VcO@RrwQ zJccA_l#W}`eX;Z`l(%CZ`q0SAGL#6yz-8e;!y&jARu+tA&*fVE=rks7KJk!v0+4hk z$h_-;D{+MUSd7|Ng8+rB_R2CJ&;dk$+n4LjziN1~@t$qyiThmoQE4!u3z4tO!utf0>KK8hU3PY@i93<@)&Trp3W zrzEFHcG!C~pYl=c`^q^p#5R(ds5MJRRZp=lzsz*JP@hvGCmzd?L9s35yI;3{-g-~% z1joi9P&KnflX%!oBsT=EX$lbNc5ax|e zoDpz&%3u{t7RKj=gjTlDSZwiYLjhtL&RNi77PHg5wR0r_L@Q0}|G0PeSJoE)H;jC; zAdX>09CTancS6J8?PkEXO^2IGr}wrRjXtbf&gOIgoz`zniJ$0xw=+=0Y?bX~n9S6R zu0aH28o9vv>b4GxH7X6UR-ijF7#dukzka{&>$MkPDTS;YGkYR_xJMA zT%YY5Vz1!ecBfr2u1#6lK8EUJ34In$I}C5OWd?z%Q5hwKjD0VS^3sMZH*@q@em7VAa9GFV%xWcMxMXao zkCX0?oXdLhngLg9)9E+|D$Ui^4|lTrZ_k(J6OjodJal;(&u1a6aT)Sr$tcwzB;h1o z)ocKJr!uI<|Lp{HOAhp<^Yc>u^?Q6u6cUUii3tQ$`HpY_Y+>Hoa=E+tr~H+E-nquF zOAWoUrvyQt#3Z>=>jl!byxe}7jVg^Mr2A>c3ZCPaT$sr)Lm{FdmI!7vd&dqXzUsue zqe2T7%@3_5$_8ta0qGA$VT;Zt`!`A7LIXtL71E(TT4Jhcw3~#u7kS0ueKoCDJAU`- z=Q#q9gq6V5mMF>8k^vLr2NOy95P)P#wYqzyK>4);A=WSxGVLlU#sY;#HJ1FeDJBY6 z4P~o1SSlmun`zGjhE@b6BH@`76wYA`i||v}=;RmjmLX~b6s-kSduP*rqV(6kjA{f` z;fz0o6SOTbG71GDWU6}F8Gqt0l&YhIr%i~wfj*dCloeW%K9D6L?ZPqXJ#(I-3(Edw zxgH~OdO~lB?A3)LA)a)?;qO-D6-5z`q(70wXVCq1;7wsvcU5Yo*N!p5= zSm>!tt4aFlawdtLjvLR5=99_sA0@@5tHGsVp}dyV z4E1p*P?$FKP#WXzhlZ76*sZ9R?2%=s*(r#v+lb|52?|xFK{WUNR_KMhuJuBOb=Y4Xx zi70J=RQy3luC0_o{F3gF`|C3sfW3Ojkq!nepDZ)fz;xMMjGxw%3q98B(e7-Gr(~m5 zTR*CpBVa$Ns4R5nB<&h4E(YHR&vpd;BE3BH@xtS={S|v%t<^cclY#q^B`5&v6cefP zc)346r`q4(By6#lF#@Zq`IjHdWjAA_8g7i$3d?EJYn?n^z@?;O1_7LStUz$L9ZzopQ*YcQUYFO;fbx zH$g&&qCMonUv`wE(EC0t7LbJ%_b-oyJrQc+t@oF$ZB>}e@Vc(Y^_uytJ+vd0vb_k8 z8T<*_CTRg-Bd*vl=x$Sn-Fsr|Z51}x1nmSlgF^Ix?ZDN8I$7R%^ov$=N+kW6!c09> zQjJmq_?PKNW4wHCnnS|vVHjcLdlVby5Nd0CW0;7dpjK_@Z*4PC+8sp{JDJ+3iEWOV zoDrrr8fDV~$Qj>$XmRvBTM%U)C+gOE@KT_A)PdAfpZ|uZ(r;_6$x^}yaU1?C{m{fh~d;J#0V@5STRgUq^~m!lo+Xn*=ae+7M#-U%+&?Gwl$ILwO*G% zds?z1#C~?dWeLjTH0J=_sI{!rH7OQuu}7_N*K3HH%*t!w&eG zO|QxDJKE(wCo%(8qHV65oygYk<*X`5mNyHKRzKWm(?wQj1b;%~DZ}*NE&*2F{2;H$ zG+3W$b&Jw+Avd11#kDk2&E&{P&A^;1>Z2NK?WW&5rCKw(MK{L6Um`0LSF8Sk7nV#k$6zbUD=GyECq5=0SW!D$ z0fUcPpY{fw6WM50&}wMvZUIUx$>}PFETmI|4Msc^9nBwKzSsWui^4=}P5eyuO`Osp zEfA=gn@Kl=4Y>T9Ijuft! z-y$psrvN#&Jnp8gF)oaSl{61A2=4o_hG++&$*Ow6p zsC-Uk{g*?jWsU!~DJd_0+}cb`|J~$pJDDhe35E$u>|C$RY|o7GC_gOu8+w$N*6`3ntDnv- zi=L9^T6PC`S{0`D7hbgznf!S-+waI0crqMvX3z7C9@7U=F{v8P=~R!bJ7m^5>sys{ zo4{aN-)KO3R1&K*tjPQFzL_6pRFIPB2JM()NPPVoTj+c3d%Pd_6r5C;ILfIyecM6M<_Sq6<*);{cV@tj14rSTw%m9g|)m(7M3Y3U`-u0_DqNVH`+zQH8n#jO%89a&jL+il#`J2 zQF7}21-&#FVYW6E?hT}n@F0L6Yo%US&#-jMI^xe)2ZXp25!{D_`oWJ?FMKyJ$WU7T zNa}O9oX$iPRFcT- z^M=>pIm=iVZ;PsHBKBOb37I+!WD0L$McGTWgyOW;t{FexsLqxcTNOy;nK zmq~4^|Ikff03-m`D zmAWotULnnT&7R<(weYeZ@GAzWlYU*|l9R!j8+yt+ds)ldy zSa?WKkegf5hnnJH`Nh|we6N?^5(4;*QT>+l2_C1ar zSeIZSCxZ##Q-pt(e0xYdj;XIFe>&9E`xtl+Z1~tg0~0NP%?JgUoVVtxI^>e7>!oS3 zz=mX>Tzp&x3i>v7sKZ#HWhHUD1tz63Sr?h}QYl-^6UZRdFg@|I=b1?qTb0?#3oD2i zpi@s%2Lp~q7)ndq{`~FHILPC7C9#kRr$whb)QsgA<6JHQ}tHw6>@f z3nR8pY#EIiZ!A!7eW0q@(BsR$qY#i%nAn9rKY)cK?8d?Qr=M~Wzr;Z}J86)WihGmV zLd@H-ye69t0?xM!MS_?@58=un7ghv){D5^-cI2^9(G*S~78HOml`*ehlUA=W8VL_h7 zAoBS@`RPPPT3j+sLxI_?jYEW&GG55OTc|kRWOLD%<9*VVzd`_#~gaOWnac?UxkXTyZ`%aZK8A#OEtoEl4pX)sgZes9!&(thxzJpnKB9 zD0HsYy9$T*#|SV}I7=D5wCCz_Uf?*1DAW-JGLga0XMux{r#xIj+Vytcn;Mkv zGY0{qR)E+(CHYTvQFf&hPvhGRQrjq;of|FkjNAhPckd4~omO1OvK3#noX?q{#+d~W zE0LZ3#GptXvg>2@Y!UC~jYead_KRVYiWCs5Uwoj!msTdJnD{fF@AFfy5OeEZ75Q>9 zMP-ur_eOD77eAEDnWg7vB8z$J!j4(mT^i8j6O=##TfP<33hgNr{}tptN?fE?EzLqn^Q@o^LEnyX}--6>Py#^u}L zJNjT<(yUe3q%dr0W+DXd7CyRML0CMcX~ffeluUc3c1AY+dI9Jdl2xMnQ4ANKDMenbJxrM3yV><*rk3lpLnHDb)Ps{BypSQ6(`7l$~N$$Vl=*lR1L=erp}< z?4ZU0q}g;2y7Ci?>W5S5-F^rCdF}PQT9AtfNmOI3)>!%Blm}N?ecGT+WH@PBb9Cq0 zIPzr{*oUHA(J#|1a9b!^L}0?=j(VDqu41C4^tx`BLjT6!6RKLQ=Pc!&dHd&G-ZxC* z-h8EU0`*7nnz>+TA*PC-}boRabSNX_!-D6iCsMd@N? zNCWf0n)4>@!hdR;%*3*Y%q8ytWy&K*YnnPw$vglAv#VvmCJ&)Ki1c`!s83o{%oxjh{Y^az1C8oinfw6OQx7b{2BZ&Dy zkRb_S-OXuxaW@2{{E1+w|9U{~Zz;=BXal zz+WgbSu=glP0k@{y##*sFo6AxJ-bDKXnhql7|Mp1z}JE)(7Uk%lXiC^pNdZ~&MKjg znLOEA&SI459IpC|U><71vvb-%+-_bstB2n{>ByZ%h=xn=}jyn~tLb^2g;mV@OOeSOX#4eq6Y9RTObm7DJ+s!7@IPT|yd(3O*wPA9P67{PB?)z#5y&!UgpT!(+ zK;_Rf-o^}XL0zxuh1C*2&N4H!kMVuza_iplBy6D)YJ*Xg3_zoi1rgDXf9iFev4d-0 ze%!Z04|85%Vl3NwUTy5=%ymu@s*pUdexUAfm6J7y#B@)c)GO6T4CzECUwPAHI;L#S zgOAbsj%2y5Lr&(iG9lR@47pk1y4aStBraEB3vPbSN&hr)x#RqQ7yrNAM2u{WW@NI` zN*RG%oeu;!x-t+nIq5%#VH;!&!G=_gLo`!RlTlJ@1+`F(*E08g*-{Rm1YtRW&(L~S zmJuc~F1QGiYhWDj-lK~i8|BQFC*7S)WTpbC8C#r3A--2D55BN{M!V+CkzvYNj-D&c zqt`DUlIbVT+Jgq4I19_yHE&cMfT?)Z? zNBd)bx#%A4Q%FR_U+p?t;DHJkZ~o~k8T{*8&H}pC?9#6eFC`b1afW~Tla=e`YH{O< zwaquH6bh2v5TM;33;Ja%LDvE35FfNfWXmwzOGrdqZ<{ zIhcEo=z9kC7IjL=bso9qXR_89Kb|exU$W|-+;;Y<0H7ET7jzvvfalM05L!WIPNDy_ zBsbaM=0Z_3N6RHqs)c>T#KVoWMhiKK=5bx3V%7;N#|38&%Xt_l;;Ek|f}eGN|(Lk&>wD6I2~CJ@A~t zDn988pgX_&KrgEd2fOhN-H$ZKheKLNLvQ9We>^Aq)*jJ^n}@*7opdK>R~ZEeXDMJKE@=ak<7aWa_HSj8@<~?rXDXea``!g+C z3i)&YI`H|c>v}A8iK4mVnAq)<0$LY^wcPYTc;Z|yLY1W*86D(;ZUG?SvCJhVx!6swVK@Q z854uuOd$=?TqXzif)yIlDy)`*j>xKx?A2hzv$y;t=|VzED}`|I52;EDm^F*B=pgw+ z;p~kB>b@w=Xs(~QV&XD;^^g^^JQ##tU*q&thH7L4i6haoL}4|G^#3Z!Cv8r!Gr*~A zzj_bEg}=hIy??TJM|1WhOq9d2|4vDb!{uH&`AxeqPVH3x4}1oo(ZO|5X|m8m@JWXl0hnji1vcUxSjM<8Lnav(>(RV zqayFk`@tnB`^h3NY=Z_X0ns4~l~ZC|{1*jjwKhUN1Se$W+|VlgN6|sIF?nr<-_Hss zv)0XYG3*xcR!H*Ft20s@eM)6u#WG zG~i-3MT;EqG^yQMN$}yh)L~urV;-6-YjhpJuXpM=0*YN8xcBF4P;n@Fz0L7^Q4|y; z1x{W%Jr2ZC;jt4Ndt?|10OgA;hbx>mhEyI462>xL;*Nkm@muq?^1wSfx(o!4^Op$i z#^rjOt=4XFT{Y$a{9hxjLLW|W35#E%EYlvrR90jx%|@)3t+<+opW33%+)HmK@qs=U zE38;Qpk6-SFCQ~l0pF_nSt#pP*))MxXmXS1bv+JeocXlzKP;v1hme5}TDCc)P~{>F(e7uuZbFI^-|Dk-LV8j9@n7^ z-FHfov|$K>L&;wHhLoWUiO48}0fNs@JUee5Lw5{)PfuZ?GNf{Z6UJ9tO}ohC8y4yT zheHr%?5ec_R!C?!?AJXz4|e-YP=8V6 zgpLmQzaP(eJHJ0&W)-c~r`YS|+ z&}v{vLBDaT{R(jjj6~@WzlUfXjDjOweZcMaXXZ0cLcz9_rRYD>l36T#8dkkat`>K>JrX)awJ^9tuCj3@K z?a+DHI;ZcNi;=4&eO@g5^{nT$XABb>jFF)))})qB7uD>1V4wE>!Ooue+(?DT=eB}< z-TS^Q46G0~6tT_RRHNBbN3DzUau2`M4TqwUR5mhCZF;&8Cvj5sv-Zn9(@0aWD*}njzo@d~tsrWw&CW(G~Gisui(6jWZK%`k%Cu!J_4NoJHOK zvT3u|L%%jMl=zZ~FcR)3!~VkYP6Eqj1efb2xO#qS#R^|4C?E*OlEF7b`35a6UgX-d%)$?^GX`W z?*v>Dk^WroxpY`MqIlrZEwwUN8{}UK7j~D1{f+nktUYJ z@zKjr$5IOm^?&uH1yf`3t7|U-s?yV#rE(HrSs->1$G+Ea-L>b+Isd0T-z%R7Y+?ST zN=yp$$W&-bZT5zcA7-}wQUdjO(6setQC2ckib2qtBB;Ma;RaVOptPzZIU%DMBJ4sZ zjV(ljUWrgK_nLUI{sER?CN1G>ji{E&{GdfpGVEz4EK+}1q2UcfvFKjt7=4@sPoh6!C|l zP&5;lk!HZcLQkRolnhb(h@TS_OgC{DG?pL@wE&>Vi8V$gh*#qRjMp*IDC|9;^me@s zF-}zQG9#(c!zgy(mPo!OOKHtchLE)zO_kdDKztiIfh`C-2Cx)B2>u*w7b41cD4@Q0 z6${Rzfj9wLUia)*^VIs?x+=ME`%=%LSiqo(k-?zApkn+$3Ih8L9+Q-8r#)f3e7@QC z-rW@taMzpfDQtV)YpLCe*^^eiLqVDlP9pK6J_&Oim%>GgMWOrIW0|*4xU$P}n`1fw zYwGj!#MIEIvq~wK@A0zoO!at}V?}PaJrUbZ`_ouem)~?FI5cQ-9=UiE33TM~VxDXR zWXWu#M?|^cG!VOE2XN6ck;s{3K=fe8_ZgOxNH0VsuywdEW zZR(pXPl<;f{(b9nKifg}n!c{%?Y?dMsVO4h&YHbT0wP0@Nl6tL{#sd4lRL2wUgzf) zd#odbj8e~%>0C39;Ixb;4GvH9ukZf27h^BD`|FYq8RwOqNlgg(gDU$w7*D-8M!g@= zUzy!o8iY_T%XR#Q*&}c;W~w@^9glIyVG*&Ig8jXO`frCJAVM-IU6l?2Fv2^q5*iMJ zK>??;ZES6iL6MNjl?iif{d(uvIY=B+hJHxd-;g4DQ8z&uJYamokaNr#+~u9MV{KPG z*q_gmtVAdh-%Vp)j*7x5>$-D_*fe1zY#Hwj1DwA;!q3+7w%hw6d%(4JNZ8?FQGY>z zi`gqRphux0f+5jJrFIC{E9Wa?*kB-TkxZ_@lH&`a*kCy)z+E0l`C;Cw4{K8U2hfuh zfr0-ZhZ)t{n@n=XM&$G;ns=F)Mjq;3kdnB>u`I~(&XO{T!tMFI%B04x-*hGj=Ak(Z zQrrnY=H)5}%gNPU`uyh!EN`8>%HW-`S!;E=%Hg0|YnC(BYkbIuV1QO(uz*Yl7r}uo zg#0olks1*RQHYEpgoqu)v&Y^qe5FIT`kR$N!1gws0$IS3Ks6+fRG__AkA^72q^(mA zHYakVD^tI9dvrhWg82`Al5yA+Tsho(nt=%+9z3_ub_52MOd=9xmzjvdh+bv0O8#ip zhi_~@!}EFrq__BW9l48%oD3FHB?4K`0*>Kc@xm@DEe($=D_&h&>tOMTSnzKSc=gQ4 zXn&*}4EeLc)EwcYXtP-NG04@Yr==Y{k%I;?YKB$t=&9uWD_^L4H%U~mbD6+SNJYnh zwdSh~l{{Pr>U_C6?|lzmv1Wo}6(sIOwAAMd627kMJqBHQx<5W#*F7%g|5V!e1l%rF z93%5#qds43+ybKK95)+uf61k2`Z=tXpo)KIzmGa3X3Sdto8$p4SH6*=t~3PS2O<~X zkGv$q;hj6zd|ul7hGG`Oi&Ixo@w!)|`W^A2E>ht&-)_6~XR3zUmWr_T@C_+UxAEp$ zx<$|5jjRQeR1KV0=6aZ=I&I$P5o{eD9Rdq%oouLRME9%k#tW>bncHoz<(3jOI|8@6}kzx+d?Z+0S zvfv|z&aLlthi2B(V|7JkWzBkv?efL8kZ(zcx@mVw(OudbEqOSyRM4ZXCg*g(t(E*c zFJ#2V^Gsi>-H#&RBk;Wtt;c`tD1s<};dF31lfAC@E()hic(dcS@O$7M4~YoV2pt`r zLnf>FOZ`BR_u{0v>+!UMY}smM`}m$Q@8Dq3e2=`xindDW z8F)!QJ1;0$N#C{nwAmIoSG_Id+TQScfX#!=W%x%}z42vbx}jfh1?Of%uHWfW#jo$V z(KNmumwwhu@SrtQdiO9n0D-`Z!|w6^!_-cs(5fo%x?AQhZ=Idy4=e>St!>&?ig}nI z60qES^}W%XpVkKiyp5xiP#X4KZ!u2YJ=E-8s1XKVPqNpSR6%aB=ZX z$4${{>PcZp7-3M#Ag!63o3ocI6A76mu+-^2%@4jJR+Q|c95_a2gxP9quQEO+OU2+Q z1f4I}T4#LjB`#rTfsEd}e$u3?zy10CH2rqTE*;m(6IRR3#l^OYS5aBE-gebDrbOg@ zup5z;T+8(3T}_Sz>s4Tw--g9aQuGb*2L~otvYqkt>+}=ZvEA&)+!g2d?D&+_v}NV& zv-|$aj`Fs^=-d9C;LqPybF+j^hJFFMT*zQ|Z!NZlS&w42wrutnu4gmkOay5&kVal2 zz5+&X*f?s|-o`{39Oie>pa1JjxJ1m*-aF$1kU2`$&61SEa{mQ=HV?1;VbUJCJfAb# z`u+b*Mxc!+D3j4;p!k5dJ8n^Z^lg1is0T z63G;Ubzs!}`O|Jb`rqDxzdysAI_`ZiQ3X(9cKzoVM}oWpU`FR+0$?5!f`;!=*@FMY zV7x7C&c;LV7HH1G44dU_X9)@aDQqxXgc>kCa<+*H^N>3R3QV zn$2BKQ+gIU-k1x#ofYiz$N|aKWF68o|#Z2_1X(+1uIhTJ9vg_dQ+`2!4F*8uBQCTSUIl?NvqADGhwI2I}hS{udk? z?7M<4$vR7XJg$ZQXK6XXg38*pe%Eb#&W#Z>`oC`mrSE|=haFqH1_1R;RimSyra{O= z|3gf!lDLbFho=0mVP_G@QIp@ky&jhPz46D>nc|^s+a9{^If3uf|1N8jT{ctd$&0j_ zlHF2jSeaz^bp@Cj1#uA-sZ%o%v6orT9c}V^Hme?jcJgBC?Ih+@lE&ko zdDE$q*Jb5Tm)-R$W&(ffYa8FGBShM+%H9B*proPp^TG4#&^ap%g5rjdt&;kW$C%jh6S2ua>DTU;5IC!gkwK#j3jN zpL-w6oPa7ow>d zU-=#F5hcgeCtyk)a%Q!^ODS)F_`n`$FvL&nw@}HbnF>M$AD!gQba`CaX<{BD5~yi@ z1EF)2D=O!UM>T$~vnzit54LF{;tJV_Tu;uNr{h!idU=mfnvT^5TyL7`R!JQu9O{^> zmT&20%i!}e_uQ?m!P&m2wZvfIr$I1}P=nd1#nI)0Y8cv!4+o~m%mKccr(=lc+$CzS zC-bH;LfE}MfQ2sO&8(D498MoM%$A5cM}xMc0f1FwjrJ`dkg#^oZAbI zrN^}!f-DPEy|hP4)Zj9;n=F~wefL$@$>y%LEshHQ^(}w9{Gy{g-%jgJRNI^j`Jmeb zGz#^k_KQ7MZoR-w^YqwPtHUU)xL87dF1v`9!+MvIyApLykXN`k3e@&EJy7;knGic} zB{Wx+DP?c9p$$Hv-}IL;>dp$513|UR`4I2>E${205-h0kLvX2xy;{0_uFIpf(EH(H zVcn0bHP&?f&GNOaMq~T6S|Uy#g)k1`&jV&d*xMi}d6C2J5S-NFNvU`|bwWH)d#aP% zHuf){birr0xZHy1TMo)m{Q&Lw_V=grXuVdbiqr8#N_i}EFQcmsPobB;$HDWj0!9zl z@nh=Z;qiEX%U!0{QW3PTqo1BMV$;N1L4Kd>lbq-}?`zA{b=KJlXS+U@ty1asas%J- zIes*g&7lp30;$mt(or3mqG!hSy*+#0l5K6ZCCgWKfCDuVWe8hvbu;O8lwHW_7+#6Q2R?GMyXVbv9ec`KCQRM z_Bqe^cq!mvOP5v=B5kSOcKJ#+nd(ZgzD@bMbBej=<+<-wQQkG1%I&AMoF0##41>2B zsEY$>6p8_Awch%E@WQ)T-oxAdu|BalFW|^C)m$C0)e0g2OQtgH^SkV{pcZNg(CPQx zukCJa9bD@)S52C9%-|_Z$Ca+@E!+b|-^#XtJ7?4Vc}n^87lDuBSJ&g7_j2Sl&i0fH z4sR8G_q_&bBK^Co%6SmcSl8rMqqVxM;#?v5n85wRHJyAF&uX{XfP6CbcD>)_TXCGA z%|gbR-PdY&0JM0_Ph2WgFcB!IA=7_l%lZ0}%0gfFBaV!HP75Q)Pgf1SF!A;eUeo(P zO}0a3%liA{q5ny?NX0^dOf*wbBWNKagCQ_92y8|-uLUOBL}OM+31qNfYQzXxAM{0%m%$CkYEX5rlXmAC1x!`Jv>rE$wx;=O}0K9ME0ntBG@Fl zKOTh5<9#1F7P7Z+4zRV=_ zTf%Dh!(txdT3FdpA|s{Xu7oR{_REx|78Wy-ee!ao<0C8e`vUe$#p_y4V4dKocUSuY zfxnQ2R*I$DxBx zN25cr8YZh*kD7=VrQ^}WZSMC{+?jSw-cw8ZL$+PICuMN!dDxH+=SQ&)jWsh@t;Y9D zw&5O4zqNNpc$w2>o4~kG9acUjB0IClJOI@7$$-18s}{ldD@XmthsXA2;whz*{w;lb zMd<{aWycM-)p{Wf2JqT%K@z`H?Wqw4*A{E7z?}%jWV@`zYWDCjz!JfJi_n;3C7?Y# zfv}RNdvL#dR9WlGj zfW3K*3M&x>Ij86u8u|@+3wy^3EG~6eDz4kIslz_ADOONGEw>j>i@DDlD(>btbLcJBj{4VSC1l8T#CQ!7<-g`J)3q; zj7A`Wvcb$(A~pz;8<2Lef%cu$XLBakZ4tC92;)OBnhnLeZ2B!Th;Ui<=!_VO@{%u3 z<20qrO>Tpt@EcmIX<1K2YZ}+OBeNB_67>vEB`69Uf}nzt@>OV99snjA5Ac53bGONP zIcsy<0$yhB6l>Gt$?SYc^7gp>5^YM=S~=dm5^#M9<9fK8n0!Izh%qSG3olT^qQ-}_ z##EudVrw^x?X319Rf!8$y!}SwX>PQen9Nu_nJ5qP{0JGB{`WQf3sdUpM^5M zIx=|+>ADZGAPemc16~gUeN5*Yk+(-i`eRXxSCvaWXD20F1&~>Bb zI_YlvT`sOW_W1)C4Kv>do0K{g-+6UE4%4{9xU6%b-FLzT>J9C^fA{!Y9&x0-2ua5b z)-n2+zxq11$`n;>{@k-+K#hvUVfA}^Os7Qqja5m*+z@Z)!;K;?0n^NQ>D%V`HLt7z znSi*&U&mY#rO&-D|E%e8-r#xs6viyn!>(eVKLS^MF)_ zthM=i%Xkvi7(Nx@OjFmuq)z+U8E+CQ1?{)&JHKPE0dC)$tKo;wkFVBTUknITR(5cD zkZ86~!KIN(aVmMBr1izks3)5Z>0CCb*_-B1xFHYO#@0}_qAzZO>zX`C%;a|AoE;qr zOUFc7vGxBU>no$;>auOCa0_lB1a}V*TneZV2<{;Shu|LEEx}!bySux)ySux))T`X? z@4ego&L75r0?t`y@3oiAH78yjal~wIZZ2ggOJvp;xbb>5|5*#F5DKD74%dVORtoG$ zs0Jmq>{cG)B6>~M*WRR6RL1Z7SrVgKv{-Ln5IMc-vn@%aW60n+N*# zAxe~)0OUd}%Fp(rva>K={!lBlsp2(R_Qtoa;_+j_he1tL``1LXQRVp12jyupN)_B= z=&*u^f3rJovC+H-Rh(?$bPUO7t0IaFDN9xyjIWYbulQ@F-j4|jcA6!{mdX z1guZ?LPhxCUW6;94)$XLN&{_c882r(<3?#CjSU9&c!h%^=(OZDYE zgyEywGlkKW_b7jf=)7`De~`!$+b)dq+0*(&m4rx$atUZ6!SL#X^}Tm-aM( z1z-Z{bxzc6MM#j2z`=yk1XwuR6XnSPbiz@sWqi?ElZTub#CK0VSPGK(7-kh-2I~t@ zdZC#Lr*=}ahYI;F0WYRZ4goTs?JI!G1MIMo$*0@t0}iX^i{9))9Kml%k2S4mB2W@YWI}f1pG2QZwteF@WRxO zWp>`r!a^p9C1UbgXE-RkmTs;x99;N+>)%+~DT5Cut0$z8eOfbQu+fNBUG{|Wyi)pG z?8L7v3oiJ`-$7*0T~i5|WH?RIvq3XSz2RvACr{iZ>c}9iU%m1uB4u#VbF5RSZ(p*U zkdJ*l5#Ad)Sxsh3o_w`jxa~MOia4&%4oz`=T&S_%DBgoowVr0ac^&XcUlWOl6c25- zUbK(LpZ1jHfENMdltVh`yE)n^7PF(!?Y_6`)1_D2-A}^&UG+v2ix>}s6M8V{Q<4+X7y00+YuF3bJcvAw|u{kBbQI&MPX2YZoF6`eYi~_0fSDt0QRgoQaYc^PYH>MOriKu*T@G5+Q{1 zlC6TUbbuZtn!srPV;ICoT5ZVYK^ll&oNPP6W(yk^3lpXsj7y0O<1Ic;Ryw;FKZW2$ z$PYj=pg5wJinLtEQZ*pibiv}q&Tz{m)#}Vn9jTU?D&Z;7eCMu?Ujbqy2fYeTE~7z#Dj(7iZHHLf zVOPc$>kZAERa8Wt43LBn44k|`YO1QM0wo?BsAHw-O$?F^x6-1zyfqWv23&eU7XT*# zFLDxjB);5sLpA{&iS{g711!w(ezZStQLg~LC>LJAp!Zi^bV(xg<6OwUo!ZTL{>Xif z>4V*fr=uWh!V`tFaPs;Tt7hK~>%T@Yuzmc?xoVfITR+8JI~~%>wo|{}#wz}Xz8@Ct zi>LsoAE|R#^unB^7KQ|n0WB0$^auEaq{ku!7uBSGeA=05P5vO;V1O9QAxfuf>g0dm-Om(GOJU$18VdMI7+oIl$ z;zap3yrF32!*io{c7xBx)IILcSmG`bTAud9oDVcE@o#>}kQwIV6ovyprdsPFQNGBN zBLxz+WUx_~8In)Dnfq>EeKExzWuvCm1+I)mP zx^@o_wS-qh#TD3XS0>sr)pm=O)RW@G!db~lNIbnLB&F?KIvs-QqTcVniTCtw_S^XI zRHgGTuNZ_SMT55_l;M2|>ix44@Xpd;7}s$hG_w3=XEI|A?_1~LCp7$jg)m?qeavAU z5zGj4jhoYz;2?M^d^Mq$>+?_gtQ}5)fCYR_0loYw+iK>S+{k%@m7AfDEEyd|!b?Y2 zWANTx=t6IZf5KO>bwYV1^QoJwVXa&anY9^*7)SZf1ON~nLI7FPLqmgiWHnyecVYd( z?Lg#rVEzJSlnAVZnkL7)RPwIR`BB*Xarw}C0*_7R!ae5**2H$13Smqz^^S_58?PoM zEoDWJTtV6g!^`d{LYEy_r=7DEVYlP@sKdp}!b6Q8vQ77yx^dZl(GAq+!|2xx?u zrVRVIzF%wHM{74fEa~~9i}bcsS2pXlH|!<;5UtG`;g|?K4Y$tjQ zKdu}##)cD5diI5-5HZ>LjptL?31`;Jj+a&r7Ha$$C3M|jqh>*_(pkhBVkjyutza4w zF9Dk=8~0$xRU*HQt;G32&6V9Ra{{GJ-OsO}R{6NXu&Mt$t=^3~?cf%1nJ*g}ymaaEPM>6kw0rs{}5wA1WF zk(t|*BE7-d`TePWli0Y$3b6ajHB|eLk`4Jo{#TW z&4=PodD7mNrO|)if12kn8;+47I7t%Zg(g1Y_;thfL=lkdCov|MWnHC#IC4 zc_psG4(y+9>kMdsrb3&q@C?JSz%hf1_K@JuS~`KMULb!bz*|5vCkLDW0OdWEh`fYo zRF^9}?hYjSc@B4vlng1H1{ZWg$S8;2t*d8UsSVkzA)tO;3DadTr+ zCDXf4$VOLQCC6ZC-2gWT;XXbf3QgEBKTRBM{-N z?MIAF?xZc9Bm`Y@ik~v=*>`@Pw|Zd``1=pLYUA#D2mP^(v*-)3SSkJKGBzqef~nyJ zn0XNUtU7xyV11rG2J4-2ty245jRfPELm=XOJDF_*;c`{4Th-^7eJHF)U)5(Gzpon| zU*BA(;VS`vGAP{b;0Cp1jF8jr;!DCJT|P`TY9q9e!x5%cYmyV-IhEU)PRP z=@GQIJN_kRUiB*kzmcpVQWy!-5B9aYhRN$UI z*f8@C+)j9F`<5r5^Bk2n2s%>xuY?SU zv68zBl??Wqn&ou8HK#3EaN;C$F`^Ya#q*p6fj#?@ut0%HrWOw`S9X?d`n&JzJTo57 zU&Do0Q97jK7i@@kcyb>oC?v#wYmKiiSDQU?0Z@@d;QX0waE(I;Kk@f#iXkzFrm;BN zcZ9spO%A9z$KHt*s*)Bg&p4dHPFrOD+RH`sQZ zn3X^pce&ozC~LDh+B?gqZnQA^6>vc3L#iY(PG3ccxHOfN_< z0XzH#*5mX6XnGl5*zw?m_N9JM1zS$gZO4;U+m(FvbFbn4J26qn$mD)7n*Pb%w#&gH?|ejbgPEiIQJE)nlKVwX?UDn2vWVaCe(9Bsxt@?$EAHnB8Gf%p{y9NX;2OodRqI2@^@_@BuJdJTdy`pj=O8U=bZg^7N}rE$7*S0}~0`j?}4>dbv(U z04FzzRc3{)WdHZ!@gUi)1RXAd`R)gNd`zhKrr>+{J689#@5!crOy)#q*NS4z0P0$DY zRaB0%;4tU)GRQ9fj%os8-0me{F@%l+5uNznl#PS{Q2?`Kxp)BJVQX&xyEl3-4f*L>jnT;< zo;XI48uQbPlH9+6T6xK3<9dUejN*Kr^Y9i6Zq)54#9dO5`>{308n^3>Cr`IK_cvbx zi$G?a7rWA~76cZIsrOKJW|_9Cv09!KFN5^D(57(NB(9qg?%lJcrYf15mb;BgDCd>X zMgN#0y~soykGN|o=`|C+l-p5R+e(?XaE_7)ubjZcwf8J85GNZ5`>KG1zkgkXss$yN zTYV=5hs#m+z2UOf1P6vc?e2NXBvjf9>B;d#y~Xqso3{IAfsqCg2CL@v;9vLFVzGPb zJMtUjIJifj$iiTFL(n_H%dqzwI;37Fu^mhjo>S>$zQK59SwafIfZV_6pU>;1^+j~e z5j;F&{L^&cATU^}czt;sO?+`}-u!cP1!3``@%L9Ov^s;fL5CLqyt_MIOmwILNgKgv zSxZSlh3<9^VqfpuPezyl=ENraUTt`?WZ4Q0gu+GIYtZU?LoVf0n%lEl=@C07rHL{U z_7;_ou`)|xk*NBI!|aERu_O^- z(jY948t5+q|69)P^s^2%3ng1Op`;|u5F89_1gZ^81yvv!OprI6NEW>61NUi@+1?|D zmd0~Xq|$RgG>7oU#{$7;$~>_20s`idzvIQ!hivN&KjULjNr9Op-z6@D*XMPA(lKDp z_IXlhmsZ5f28~y9v(N*7fIvN7XwyY!&QN(XIp<-DX$qfp+scy;M95kOrA2Qr7;HYS zLp(l?O7dH)_6>{yz&XA86mOlSr0lG2u zK_GHnn0UXJB>j-}qqNRP5wJ#GlF-Mtw*|Mw-gnD)H}Oa^4&HBw0AFgXn@^{*zn~mE2}9 zo)j|yuCh;)$a~})s(hk=f|p3sGAFg7L)}yB?F_70eCtbZkw^;*&7!{g+*|$bu*kko zX7~FaZld!9Uf07H|AhT1BU>kZK@WEXV#A4H`8%zjY96h+x({>`lK^lv>L?S%278>z zI9d31y`z`fHZD;wKrnzyXR>lREG#^+*|#Jhz4D+7kjQGV6pEkN@1v5%s>e0QHTR8+ z&Ve{YQnz!E{yWMKn3Z(X-u~Mc06JZ$5b5PLazrN&#h*3J4WN$S`ext-;F`zmyyb39 zO&9UGx3T3x*z<8eRrM*gsm(>_WU@|6J3%p)$Q%ygVq19O4E1u2;B6`z6ET4E+iuG& zH3^#ghnBFJE_t zqLsq9St1A};PA)-rP@Q7kSGG(o+f<$=&j!OBv%B6sLqc{ennDO4&eS9_(9P|N<(UN z?Zy?F9uW7x+rdL;tm8QjPV{zn4C4br`-g$~a%xE!8}d;JicCt z15=XoHjAY0_eRK;w8|oE2kWvlIEdp%@NisHmFu?z|2*3z(&rYaAdWpYAY#r%b`?rXYG^V}aO*;`DL3JHPf^#8B9#GEy~UB$!BeCK^L3BnE3-o&bh(B6M?XZ}%y9 z;H8Ta@wB=TVfWb=jz8dTtB;f@qC{Aq*!s&eC$JIp=0pj|s+cAuLejyE)`0$g9ELZz zgX6%xl5Yk=8tUw>AGE65>h@QZ6-Yl ztZ!cedKExGSeW!npFo0j;CKdeAkLQeG7C=GdB}^asx=c7?d^Jd?7Hras8w;2ofs;j8G7l)kAUx&B9@zfg-AgBc-F9_5 zRz$j8M=-Al0Kl}}7Av}AJWWOx@RkHH#~X6C(z9Wp ziF-c;VdK}NuLL1c%p+}9B^MCDTj-#!JUjnc;2YieAPO9RT_j&GA|~PBj)ux#qY%Su zb?zNsQbMuUf0HRA&V&W~m7~(8*7CjzgdLFidAr?NepNIlOv5(KW2t>MTmoaNRPVDR(Bmk6Ow!PNDsNk*v(_iJB1U-Ur9X*I$Fw_b_gVM&^7q${-R*1JupR^w zI~z%05WbZoeSPZI1>C}{thw#$&%>^-0f=nfuqNj>anMZtS^P4+)Ea_6K1_e4eLmZQ z8}bpy`Sv=!t0M8E7FjoJ$73y-YzQK||IcTe>hU)}SS!75J5}C!=o?UDqW4e&LXd%@ zbz@_2z)(n-s5u(a*J_G=4h8;)3!UpdoXnTrTIig)U5wcIOpDE|(&k)ZuHt?;Su zvnBi&mQfDgPxk48bi?{^DJ*M@*`vsh=GWdz$Vuu}NI+%d>yb`cj{yeX)3O9(k6$slTc_Um2aD@068S z#?^1`IdqNh>b+)j2@DKrt>bxl+Ib&X?WQgt5M=Genl}?xQVm;Z5*R&bLSHXO;t?1Q zzWt8y5*sVyQeX(P({VQ@Q9Y4h@h6h7eaHzm76r`tR-D0^JyZravMr&ulKSF0&3Soj z(!ht~XD@2k)Vu9P$%#+Xb z;U=+Tq3bWUmzhQl^->oS*$~-dMbEKoVvFd3jTnJl{(xbiiN+cibly&XxK_@L7gJJH z^ioRk`_3nr0y`ugTH8TJr_5YNv;3uMQ-Pj>WP}27(x`NYgFmzXwuO>}*5@Z>w9^+pGQsi^~7_#Mey zbqyRzc+eaIbM)ZVB4ARq>XR4thdCKBV$=!CpaWKu>TfX)bg zcMOexvFFw*o#Goi-*NxvCMhV;Le4v7zqv*dF^jScMXn3p84c;GW)Au7-}n93RYTr@ zwq_~&^%hj0|Cq59F{t(AF)i}i?oTP`v({|p05Fwi_ag~v>uPK2>XOd;@?=h@kUbmi zj@8xr(jMi4sQT2LMzO=lwlu`rQX&`Ha79^UNw{U$qQO9WM(Q8!+B zDZ2UD*FR^?!-=**2avBX*nj)teJoH=PU)3OX|-m-L&k0b+1Q#7mk7N$@`y7X!EtVL$*JJQH4(Fry= z?g`atUUffgR8gT$!XRC~k>pE3VAyGk0rVN6@NK;kn_ps2NIpz!kMF=KVCn8B@6vZj z+AIi#1);d4{4^za`Z`L}VUZ@u8uHvDUB3M^BxXsJ+=l6AzK+Hha!FwR+w(R+ktzir z44rSAF+z!i`6ocZk|n0tU21XAd^qnijJEm{b-{CMC2|G7I-DEJmQFP|^fc{YCRklp zaA6(vh4bqV%h`&)ih+k7qe0G79)U29eJ8A?L~2Wd6h%AhXO9oGVfq%14yv_okjSm8 zse>;|U2jeN66Iz~UBnYg_jXTh@zhTQZLHFjj_@|`Y>jr9EMYaP9`v0B_ny#5(nw@; zxmAVS7G$&IwoKHFdVln-TEFs!ocG{d_zs< zU*$-GYFf^3!ai>n@TZJF)JdwXk+s&XUr5fYzlC}V#TKZi07KtiXTfcd;KgOrqN8Sl znX}CjhY@}_X^+5ZlgBN^zYlY3gbpWS)Q5LoKC9%PhUIL6+jU~&XlTOPYS$Lrq2kPt ziO-}nb8Fuf@aw>1P!95wU*T8~(H7;fcO9~N74t~%VZ}`3WRvDhzwS=SuM~)Xn6WRG zyHZBHBgEVByWu%ln)1{NSk8Ujv=QgIAWzPC_$IuRwKYuo8cAU1XAa$=Sfl3~$-ev9 z{{$E0oyZhb_P>05d_dqr2=5+QzxjVTev5(hw-svk6dVZ;D zlLex`{b1VF_;H;m9te*3$!F2GW;dFMet3s!OCidL0ItujJ$9EP zBu&YCP7g4ns&dty0DsAZ}Hhxk2`!sTt~xXa&*L(&#H5var9X zS3=uFjMW>Dlb+R*_jQ8LYDy%qNb40Pg;(d)xDX?+^L=zJ;_0_n0r!QN&?ax}Es3IbT2Mb46+K(tAbbtI7W)y2z_KvGozc(5zd?saa+L@WH-JTsunzx^|n8#*i zWex3#P5y8}JF4u&JL?6&0|3}yDC;*|`O@&-0{{|UaQ&=73Lr3?$hbk*IPJ;+3>JLH zliW4^NG+B@4�iqf@3OYs0~D2)jbcOkJtt7>NfA>CiPnP~OI~QCGvz3yF8S5}(8# zGYHy^EiE|;ZU3^Yl+w8UbMu->krI4wzTUlXjMi0*Wc6xE>)!#?5f2N={+cJ1uV;w@8x&=*T90g3Gjop2%fa3>zV z2q6vIdPy-+ovtbH6Nx=lfDiHzZ`j`N?7fN-ogEH&ZAL3157W9RCMYk6>#VZP>x4*X zdQ@l*rTpH@>|GYagN}&Z5{22Dx2$d#t7UEt8lRAnQ6msEvfi{K{R#Qy4XoG6M*&Xe zb27Af)~vNblV3CLaI^M868F)aYTMJ<1#>;s5x5_*HC*NsoPi();CM|YhD=pJwP)`J~&IY{3=%X zlq!1${IzHs6Oj<*JyZJSAx%#P?L|G}$p@k?3uG<6Uh}Z`qj>B@bW9o&AwHKsb3@-} z=&!P+!3N(c$fo_NB^I-SIxGh0QM2_Kx+pca3P7zdMg~Li&tD$}u#ygnWiJisxIBVK zPQ$Fe`@)4u=#L3Ich|j3}z?4 z&8t1UflTwpBJx%lW2l&`-IV9^K7{LI9V#8D@$ixnA!W(xXwkmpvqp`h^aS=ek`^AB zG99usp~p-=T}W#gCo1^{p4#^h_sw=f>P@9N4R8GbBSr`!BJJhLWIFzCR^;_s%4$D0 z-k`NdTO}*2soh;qjRhgW0(}5kY=IX)a;y~=9+3xuRqcRn^uv$u9={Z57|q7_9neDV zgqN0ZtBg9u-*)vQ&FXQg%ZSzRX_QUjS&kU35AQj&Ltg=W2)*GYLEwa!DbShkw+6Sfhcdq?Of8>O`(kpj@` zLWV|}>I6;;WP~q;2_gm{zyVMuBVCu$y?ipg@ObFbCw^3Lej_WTUz4fwg4aPNOR_c& z=q{$B%=6^}GFmG2lhILtc$a2@U)v}10rS|Z>9OV5;yMV_IvjvN-*twGfc-DheB&64 zfb?#19eASQQeJu4T1PPMM2ISp6hf-qZ27Bwo7;At#Xw9ik64I=Op=Wq^gI*k+nm&Z>p7#3hs zj@0j?tReMS_eScp?Pyf!tl3p(Y!tWW<2PN#!w^FfGqc?D+=l`CA(Uk_IlaWbLPp={ z8eTjH4_+)>muK)*Apya`^zB)qdk{oQq2*DgB`~Qu}P+r}y;@{$rt| z=&jf%k2Im5H|Jm@Cg}ci1H0K%1l%7Ei~mN)>DzC@nRp(UQ|TKe)%@gCy?eAVOfvMZ zrO>HOM%ZCPY&5l7GFVzY%F zzGA_HlQd;T02PECVDodB#UOb9slG_X={(1~a3L4S`Gy!>wa&)oYKvODHJ0P@jf-mAi*vH;1aR!x2qm-D@Y7*tS*vc}R@*pemh*>NMZ8A!})3nonUxP=El z@7A4awz@4q73`1!G_uJ~s%fol9+&h82|uS|?8e@W&o0_JtTp~TiX_lZsIhn=5lQ8> z`$g{wIZeS3yuVUbgiTk3>dv_3)YR}Y8R<(!k>}UlwXQ6M5%a?Mw0Yb%P@o^e0ajeD z@1IZV(D&!e?!robV~`>MFk#Z6)AuIu*ICdfJB^FWwQl+RX9<{JTx`BrT|oIO75ex! z$6XIsdv!T>v}+>8$iV6R$28Z~jTinT^b>u$8DnpNk6W|hRP)0d`hlK3W|!Gd&C2Ht<{S`lb4Jiw@hOx6%FLp3X%(Iea=}bZPDEvvLna> zcx(JAPjz}$zTQxf@Fx)R9C+|XZ%k3qp>&g z@Ur|%z3bPT|G_xHw+9T0HkxdHEU2XvzVE=XS$e!oapdG8!2^k6>z=Q;H3OP@KN2<+ z3tS{=L)7f-u5NH>m#-T%((ZdG4}Q9=9`E^HiXefK-e~(T_9GXcJub$xwmmL3*u{Tk z$Xgxb<{}wTC<_3PBLV5M5c^DCg&})tfe9};tZN?E#?fA%{G#oyIXpUt-bl$ukiFmE zD=l+>2r?@*GyWUD1nZCV+JS5($+BgxXsLL$tRP#LDK9SPYTUS z4~3oj<5FBjKwy9*M{P~cmoNHSG1wyD3s0S&ApZaF4U=~s{63>2KE92LeMkzhNqy5y z)`i?bI1tX2gqONmwMS2X38`{i)wNYtTl5@jm=UBL3E89z^^?FR^9Ja*o1vlwQ29~3 zjYcE(srDZ z;hs@;5oZ^%>l*WXoqFG-5v4tBC>EkUT<%(ogBi?&yL0GHLOTi<)f>%x+U%h)Zb>u`jUK;Y-}_Wn>rffYvU zDmJDHKu~Tcx!c#eS~M&B2)%GhzE)J$);h5EF>UNRa&d7TdmnD*42ALT&i~{)F{%`a zz{@C>Z*_->Wb=pqw~^Xfx{JaYv!lbqc4;t_NTm68v1kN(wO__Enwq)d--tC7Jd zi6~fW$)m8K8a_6eS8r>g&&CC%JIs0pGsN_p45(b$h?|sSAfu7>KE~$5%-1`Nysum9 zg~22kjeGt!RtWRvitjsDwoXKEroBq&Y$R*lYm??hxDFcDG$KrUG$snhAR2oymoOOq zfn0^^5|#_V_?Erk!xn7_G4CH0hLBS2wpA~m?_6{r1`EkFie{Ekrop((B?P4UwyJ&n zwCKRj7-Sfi>kAY8KJD`xEBB@xU8A$n4)->9Q58Q$*4BI5#>%T6?fOQwxNyxh)WV+DQ#f9!U2Z4=jvWY3l4fUjsU?AS z@cl&BV_rDfFUnng0z+a-lA=NIeq;vL@G(GQk8`h9d)U2Zb~R-&HXA*6zA;d&2qZ@E?o&7*c_rgT#^va`3RZvmEq62lF z*IKT*pPij@|0=CFI4*QlwOp!|DX7Bat})1!NxImcrdKg(wZ0Cx8hiJU^h9zkR#W5A z^=+j=PfW@TX=+sY=YQMWu^N>8XPUp4X!4$8bPh^8pc(-tl9cWLTp%B7qQL{2j#4zB zEJDUvlE^_FWu;EGW2WP6t~52HrUN0tM%rr}{!Tq#C>4BiIGZ)RZZJYWo(o-n!Y*|; znv2*H)|jcO)tLJuhrw@q^h-q^h_ntv8Uar7FGoQR$)3#3R0;}Jkjx2*&%~|VHcI{T zV*csV+?o3wMAPoZF!W8++b>9>u`}-+(0FgFK5Dv5-yVA$EBJDiXv!+8a*U$Xp4dV2 zb`J&_5|}I?!P7a4glMdKRa(`HmitT&^w*=EhstE~P_RYUgWC+%hQ{UGYa1J-Ebjhf zA-s>vjaYxut@pS}CGYdB%?%zIizsV> z%_bHyG}!OqhCGX*D>`kKX;tX-%Zx3-No-VW2GzQ(|4`zS8R>)dnz09<>=;qOj1w< z*Ek^t0`q~WT^(e{k>`Bv>2ghDG(L&YdvS*=& zMhmK9aXR*^f`xcZ+XLph~TfOdQu4__C<$4!Op6)iL{FFoTa-BD3s%78mcAhWd1un`vhcLWIvkNI5E@90`e>CQ$(ZaUE0BLa5lR=|X26m*rw+GWI6y_;JsF zo+uYKH0-MHJrEb35e7-aiH(RD$ z@~%8I_%%Oip*-przk{N$XmZK#yg|U)WW9H)87}8CNeg@9r=uJZkQeG3x?KSP1>Rb; zB!Hzh(FRcSI|-v{+M)@*U-_dto-E$ncP9d+7m69?#m$egeJ*n14}^d`F;)U*4${ zR69=vmV)2?tWCGre{nkx&wn|~mXRAKaOxM#O4RnFW7#A+&3F_}bjSCmPYdqc8-3AaYk%VSjyx#5zmy#82`oNNPPi}2uSt&Lve zvw1PfN^GojirChR-gg*p$HTLCQf8#I6SzM5ePl(qec9bKp?x*3$2sYHiVuudVVGQn zB9NW>McjGKSL_ovoAMWG%x(LOx;{S#Fw`%sm8__+e@dpnrP5t#sh@qUp1lm+L2PE+XEh(GD^l_LO+uw8Sg%Uv8@QA=Wom=bFz zXy6m#u^M+RhMiv!0cQ)Wm> z{YUz|tC5qg_n0{}Po`IN%6Jj=)(@#Mh47d;dT;F~@HN;i)y|S3dLsN671fN~mR8f| zR0@E=iYg71E$P~5;>gJf`ZcMIWspL+tNvNbf1VvY#Akv}jf?Y-H%Fw-+mSL!wE~y@ zlt*(`Zfl#iJVHo8*Q1ge;hCWTApo%cKUZPbMWnx&apXt^rs)2r;SgyWCvVaSO#ctx zKpa|Dt=jMTBc?>Lr3elH`9kHB{Mzbj|Js_aou`_oBF!x3+;Z!N_{AKCg%xG`g|aAP zQSwyAgx*NIW<%2Jk`qRwrOWjBB4}YorSZN#7`Nm7fG^qbd#jikaJb=nH59e3p*kvT zT$DOH&tvLewU=4lUp_I|V=6Sf2qQIN|KAAxU#lf6!PcL-tk|_3N{a`%t!tw;Gu0&& zdf1K*4GlfhcS97K#m;yy_G>#&!9^dk<`ZfmU;SP8G#ES`rX*n^kfX7P(&URu^^HIu zi{8|r3?1)%0sYy>!b!ozhJ3B>)p92Jfhj}NQ+5yZGRjYsTKF?uv!DMz=OnSnMVGsg z7fJ^a`JPl|Ur`AURy8pdV*9Vc!p+(LnKfJ~1-61iy)N zkikUhD%YCY(#~Avl7#?QRA#32wq8dJ+cMphW?vWMg2>{_PhozB5hFcRoJc`~h_~Zy za%rnIwP+>hG3FoDktRKIzJ9cX3%LC}zWTbZ^@4DHpkZ{kyw+Lw-}j{3X&9;<4S5E~ zhVbtyCEsPr#NY;h1N1}Lcj>8hd+4;@?vs-AV&c^5{G$zQ@n22ZHY<5-qcm#BCCwExJ`G6of$MfAuo9(? zOQvPP#Ba^^K9jqQiZIE;qlO$P+YK11efJzu5j5kg2=Wp8;xFM1M^ugiJw~$6tXFjt zL#+A~0?FualnyyeDs&PDNa8UaOG=t<86ma<@cKoHgNG=6|K%NEF)-fWR@fh-qFL`9 z9$w(|l6VfwQ@&36TgfG{oAjlkp-2-_ysclPD1On1mDIOjT7uA07#rWw`uOM&Xu&Ge zR|o@Y%?KBnZz1qn5;Rm+=NYhN4L`3n+ZtP13sf>o1<7f4m4vz7px;+x{Z^irF+k$8 zNL0!Y89}pkp?6|YT~2D!EULpBt`G0)r;J4CeA+MM{wQ$2o~Eo41E?geaF>YehuvJX z(Qx)!6dx(XAyieY8_B2xV=i4{YLV204E!5udw@&lzZ>xTU^X%XvEC5ll zcwGpXIV6y#kanMM5^0U3tKjv_L@CxI6wy8qvHQe40^Gh-1pSt_(?372Hfps!nSs^v zFdWJaeY*>K)8n#RNkTbFz3`u>;l)J9lG)5X&R6x>{6-?@b(qX^OnM9NW@stjf=)6v zvB9$~)o!n~Re#Tt<`18{cv-4oI-0e;{Y}D!ei;9ugv3!Zw|3IQc5$vMd#YpCo4K8@LOUl9rfqiquE9riPg@qIPeW}*N^5MRQsJpVs0`8xqi9P1iSs_=Uu{6g_O`gwP zq)3S$k5(UUQXCOmze6RX^E^D9zG>82euWfiuOynhQJK$fVRb)f?Ct%fCFpW_k8F5&i9_{{5{t-&(3@Az3z3dC0M!EXh)WG^0=9s zKc*32#vQL&MBUh!YT`Y${`F9>(y-GF@jO`irF3fb95J&hx!{)Kb3?KYWb8|At0PCE z$uA}yfBt@D>7=pK$DM0TZmz|}%$|DZzdFi?=08i}NfM2YnIyVZ+w1AL_`BAR@0YExo8<8$wwrP>7A>>iV*m7r=T0G&Gyv zW;(+Cl%S6VjhC`{-l@h(8YvoDJ6w@y?==5?+GH|H`Q-+nyDpHISt3{2cjiv+T8AhJdv<%ks9BVq(2|p+zne@1qfex^sS>} zUPEpIq{Imk=uP%IGHs&z4ZO#Pg$%adkg;8OBTUA1FC7xl9a873yu*#$c}rDZd@Q)wTsr!>*7tDZienf%O0c&6nGL zvRJV{H;xkCoFCd!p2bp%@ur4O=J0kD8#V(IkBYN}Ti3ebB%L!^W~xB5i17}0cl^3c zc8CBaxyn!o^V|IALzg3-!Erx#wYYJdI-=x?U7{g8k&~dDPDGB##u&EQTYQS&p2rbI z=}|0WVQv#gTC2C!VvdlVeS@-*5y!?(y((if(oNZ%$>hOT5`EB4m(`4I$)aF-DHkn* z{Ko*Cw!n(l|FK*orK5I>W|EsK4GPPzBpdQ16H3d==dWwp_lqGFW!zDfP#wr&mDI8> z6fLjX@y9!%Rh`vgz69(HO7O#F?#b`fK&{CDeRsRm5K3P2As~+%|Msv=paKGKm-=cW z{Q;V4?ES5=FcOQ{;UCEW_s4EfrgDuabqAA z=`U_rHv0=-&OBH)g1=Q3C5?b&uy<_ie*?yL{{oC7NdE%H5sK%ZTr_3V7q3nh-6yhA zDJe54r#e30Mi_T`qnbwv<(bsz8K=)^bmRWWyxhf?IIu=b45Y0{Sq9hHUN3=>4%997 z)Rpb>YGk>YEAgt5Yt)#plj_5m8D42W0GC%FISqB2lptv^_MCfTl_m0NBWcn9rYao> zZ@>o}78}l6cXNnQxXW+JgYs0v7P#*VS>dn#=bZo({Y|g{qM4nFEbk-pYLm;cCx8OS zgD_-{VE1j#apv0}RM<-BW#dRk$yOgmm(_&9d$k3z7-?a{g#%H!+^$>Jx@FvLVC36& zeyTEAIT^(!R%IE9B9ipk-`He~FcT+f%_3cT_CL{f^Q+)x2<7$A7X=!jtE2kK%JNc1 zY&+4+xeC4dh`HvJst*Y-AcYwf-639F&jkzff}S}~3~zbXz{4}!zA0L02A;j2@BKLNeS|BX76x>{02AE?km&O z+fmj);$tilbv3ZjP&Fxw(?3S8S8?A)(EEYtmsNe_J50K8b)eg6(>ubd{krNyJNDT{ zB!cv%NgwBcg+$DMj9CnRqYCQWwq=Sg8RC;-?SmXvR? zlYfkYCmOn?&+VY{`OhZn~7`f^<}uRKsE-d1u!OisVCsIE-=l#@15zb{Up`oc{| z?1e5`iTpF#BUX73iE@**F$k1AY#vBUf&3l+h`F8JwQYkP%@^$yZpJQlcz6*A25;0r8ytq>Hrd!j0T9&Xe}kYJXo|u1 zr+^7Nf$TFrO6Vf*=e={%{qp+r+qvPa-j@3`xDE zgx?|)8Ps)@BXhoLt;?@s*DFNUfV%||~H;&d~j_msv9y{wr3W(^LBb8zUo zE&&wESLNjiuKlFW{FYkZco#9TJ|xbbBE3!oM9zRhnIa#*A{FLWjh>b!uqOdv1k+5tK4+G`do)hdA7iC^#=P-hi$x$V{BKei+9} z-j}>#pJx@vsI+o90zCt8*}&v{Ma9KpHCCt-uNubk)@!^P)LL4rA8G=#5vOkX*z;RrDg70x>V#A|D(mc{SpW z73D@CeVWNj9Y}w>(QxcrPnu$^Qdxm`b3y zCbOHV=1Thg%Gqdn(lb=JA^dIR-a0{>%4+Q$1Im)1jN{{sA4Az!H7((P(EJ-zz2F}P z2g03_`v}O|-RP~4{e`a9zHrs2(oSphq4h&rW5yD^SzOrA*u*K0Aad}zwQtY9Z{F||<>h>J43{pxX~A*R2kRaoj=tCP z=RSJU0E7{#<0HiRi5=cB$8eVC=cZGWAeu2sB8NekSX7(3zjz6dAjj$ zbaw>kgCWVyeObxFjMD2KFDfzElZZ;jxcaHzlr$T$kVWV;>|ro#?_45CDu#3zj-ARf z_!?7a`TsiL&vU+&(~92h6)>bOz?EI7{IGv0=y=}eu*ATlk=pR1cS@i0$|<0BZ`Rtk zkufC=NUc>9g`KCYPcBZ9n-HRu%aiH~pW6|

    zV`?pW*JV$-vQ*Tdv9#I%>4y;?_3 zV>jg<$0)QwJpHHZi)Rx%sg7gAKyE6OC0kwFwG3;Et%n}@gE1}3%?m1qK20=`!D=ao z56GpGs?!&za_14ntsh6S5neEhrzxVsWT%UDUiuFlp{`Jlix9^do6gUVTNQW+*X}hj zgycvO;t1+PecZp9gY$39IoDaIAov0Y9IfH94gYZ|_a!z?{j9EA-FM@n?8^87_OK2a z=x#oT==-9Cp79J#BL;m+f4jSG;zu@Z4LI)ncQ76tBqx-)?LWVeB2FhoNnj?UF(uo4 zt~KzAbeqIVq+dJU^I*+d8`l8crjS&N?Xt?x-P%EHV*1MkxbjuW?B;id_X{_Mt$uyf z?wqsF-)gD#>f0Z_R7f6E_v+VGuyh)vdL7A1arm6Ml>w(_x_D^;c^vH-kj$p`f4BgY zkfUS699VX?SkD@)S&hUeci5rgIsA3G*2sQ_SWo*LMYFqPll!YkFCpb!MacVUOPAQz zZun|`v;S858q9^XVikV9w=_TSqD)&#tD$B!hJVSy7a;#O!0Q}U%#6t=*PseE70A8Q zgzdMuHEi2ZSC*o)2+aZVwroH)o#SW$*@ip>85$hi%*;;Kejy9d7A8+6H z>!NPb&iWsDi}Uz?-Dp(ji)ig{=0z>iy*+G|wU1&y;$_PSUz@ZxdevdtHnYfprryd3 zrYd~?EFjZVJW%IUj3T?;rF%usO}Kd4H0KyFpdCrMa+>_y4KvN2j*OD9@v-MSGGzzO zGUky#?yGQ4Q=~98UngHMZ^>BXmH2z%yS*gb!DpW^{`;#rLD!tz072J)6t#L7W7ZSDe%0k9 zuZN#N@AG;2d-fCM1nve7x6kEE^XZ>nYfepsLmGWLqv8LEtrQ(6?bMqK`LnWqj7_e( zT@VoE7Ajm%ktdl<++biEv7OTVU6?#EneWnEqO@VG%be46(&P}WN6&az^afpNExtf; zgKvV`44jhEUb6KusdF*qTXqE;yjqh8N8YCxWm?oXr?NZe4<;PyQbTgD@Aa=5KWesjxfx}K zW^u={Y@=YCI`kN!c9L5uLCT+uFso@CT5%laUcw%ZI3}iC+cfDPt}d8bF$5=IWNba| zdghgOBvToPiK@QmFF1m&d~tXFEJ_W@M+VN4K9kX8D1tPTJ<&InwjA5jT=?LVOxMK0 z-ZPyNx%7|N@p6V*zt_aI&qL{9P5V+>l6=lA-Bk$wY@=MAL#`yCFCS|;9sTh(Ue{g-?$YBTNoEUUA17}Znqx3)O3UWu2DW|?H5c@-#R zdm_n}DQf_)!GdYGLKgq6)%v~iMmX;K4KT@RJ$%MD_he!F&fAGaF`L<`=2d5T(zwz- zvC+TNSLaIdmuw_4Bq70Cp8~K;^YvEgOj`d`2t8abR6q0tWlR(|c}~_(io8C4S?~tq z*Au;TEh>@t#R~cw4a_7bx^H);q*`fls)_^iH~QcIF)*BAox^z}wfqxJ()`z-T*>{y zhC#f)X%-7Z#j;$j+)XW~MI19jHmLyRE3=g#MOHjT@-9pZY|YFupap5}8RFWs&$S5CC+K&>nx+uNmWg{Rhiqk^t#d71r@dwf_*gOpUXJr~bH;x;z zftZLSoMCR+0!e;(pA6v`JN@$oF^v_yz#lhzNhh<%zY8$ObHy-;{NGi^{Nr-ySdrc0 zpHH#FA95^pi6tW`S|sK!aN{+>Fx|}?1!N&?Z5QuJ~ zQ3`TG6%?Q)P7of->ab7X=96M+m?X<!x^mAA6XgmQE$Z#7}L=(2ALn}pmbC$K0Q z(6yZ}w@4<*^58rlmcP_4xHa&W7a?`tsAL5%VM{Lp4J@w|c}V+-yO#Eq`_{aI0nYGL zSm<&JmBUsE6(|Zlzm^|4sEHi-z=#$8@2Dz#_q{=vxDxMG1dfgtEkBC^; z$V~&){(Y;gPLH_*C433U?Y0@pc*sw>P#RdQ$o`Ifh&9*T+><1~RE$OVRlzCqtf=bP z0`#m1U})9`fs1XXI$E_$-Z@(1Y)}Vaeq*j+7Lh|ef&9W%ErFs(S&Jmxw6rz%^oUP14 zz1N|6%^3QtQt_(YS+lmpx7$>F@0)*^gG+>2FfU3l+43jMDYq~*CLP{Rb76O6YPDg9 z`5#=aE?Jy;>pB5Y%rc>Vk)Vtz2{*}7?xA$UFM`N-MIcG#i6p~Fj0u(lo9fZB*- z^Ygn3&I?WE-yUIkc`GF@(LW!)TOv{f^YIxMVzQqqb>^_K;?b%Sbqi4|JzrG*>W0(T zlHe#%I1&wry9c_I&*?uvC$NpgM!Z9S8`isLdC+?fyq z8$gp9_f-U`x2SsU-M%-*r*=IXyYm<$kxvp$f#E;?{A*M>FPhJgM$Ast9eOFZy`M7c$J1Xw z6}-#sQaA!ZT8YEY?uOzSeBk?+>t7)wlg?O+%Fpu-P)VeYhjJg#2>|Be++LlTy0E47 zNRY~l4grgU!Vk|l&Q z3~GBm+z+zc261}T7S^2^UrkeK$$xrW z*+3P{RiHA>t4QH8Wg^g2 zyo`pDWQU0oNg@3su+Kej?@RKAHs696woH_4x(MymYZ77LyA9l$((>rs zAG8P)=J$6{`&NX#BX8pO&g10hk$)#1eTnjJ6l4g{)g^Bk^$VcQMUm?-V{OYtA2+MK zz$z8~K`&M1btVcoHnruO*r%GE#>>vdj;3;4)Z)Lm>HbtTk4xu0{|2OjI)Sm!GXnqI zx5^Q-IQZu`jSs<``CJYuRgd)cl+;?TCAxv;h4^)DmZbYmB4l5C`TEc#C+|$)L6ZZ> znW}!`@5M5Bk`hO&!4@SIq5Jcj$H7|X=*wR9FIxd(sWtjFBLQ4ElO=&*&DqmuWThdr zjqVZ{xFum>Oru!(X!OL=$y#(;v_R&Sv54-YMW(UVhP*sQ=d@jrHk~*sWktsmUuEo$ zIJ}QhcTqn|@oUvd(Y|ERh0!wg(ebcG-N`c-Hb@^o-7~x|~#==P^5CZn1{mKA3w`24v+0 zg_``?DCMv}Ld>G}U1Cv%jb_Fevx!)hE zmx9(C&N+17w1)ibPmQASdT3Fg@iR&${fCyx;@J!ZuRIGz^u>>Z-kkvSJ{VmJ(d?Pa z>zrEapFJtXx;tz>*{E^zFWzD?1?Qq!s%BnnMU`CM)}7X{e3iz!#8<>Z(=7}V$;ka! zN-{o~a5>7JlS>iZUX_7OLhW}D#FD$NMlopMox>(aN6Y+d=76kOeT+aWmrmB}m16*r z_8i*@Y9UI*k{&9%YUx{6rn59O;-`HSEObq2{tAPBcR_D(Uo@TApn96}?q<8aZ9W(= zS@Etwf7mu@*~^M&c&+B`y3=6Yg1|JS?xx~V1(q;|EfeGNYAGjcix^Tujs=J;lNWfB4!*55a*q(t?dvHo)8AZ8qt zaP#Rt5h1o*ExhxMOA3GP?U9pV1;2m8g`0^OtYxJ`fRreTgSJa$K)2fhY*X!KHjB)Y zMLbT#aVSnzTh7j|BBhENVicBarwZ$|Z+t_5@Ab;SxR7{sn;t{^o7E@tmoHhG9F}>U z);fNE#Uv(7gwNH0_p6%Me7zq^QIn~$wcO)%Wl58~Dzy8*{mz*UvcfS!ZX_XqKI-0w?qUC2Tc$G2R< zLX@Y&Z^%A)AMA#f6pEY(DZ(hD*hJft>wvfLX z2xK-GBk?Eg8#U(Wi?M8Bue)mI{pIG{x=ex?%TQm3oy7VKKKm$m5!Z)>-@i4MJDWUfPBQ1Y ze2(e{Jp{}X+*rMvDM+*qlKzxDKhWm0xL{Z?@)^&WU;j2(Y~=bgnDRw85@zT&DFT`O z(Tgc0BtSje9SdFIK0hCv2k0U?;xrk2*U3JqzpzoK1QuhJeHSgPR^*oW_ z^&h=XH@lf^?|vJ2Jl=##X{3P;1t0Dv+IywNK%Nvrjw>z-$5!{{I6Z$t-yl6wR;6yI z@a4`98YxdZ4s(8_9{h{iThb=MRDmLRQ=KQ-Xq*Wu=DKYBMWMa`J4ba;UKJSBCR8l#KIm2S?rVJm`#QiLb z7Y`x7elCDb?+2b$AmB7-fNQ4$wZNC-rABqN^gp4c)%)U(D=jNGKxJmVNT!d2iO+Ru z6Eg4_`~44mB|g&$B#JB*z557jZsg#3q|f_${70`PCNkqT)AQxao$>aq2zGVh+xo(9 zCFwXe-Vd!-uhk%;^MCT26s~_hK>%$mrLW>fftOZw-cLt9r7>y-AD4d+^B}5}%>{JF z2BK($?1$p1ngC76L%^-LA@@x#7}@2~OA5b-L4=&jF#r6L!|yzZ;95aQbuZ3oKpTA~ z`&CkpmKq~M9tG76;)_JYe`s2- zV%07_vK`sF$F%j)q2aBC_ASOncsE@^E7VU_xLU1$`GWys{9>bBDh@TUVx<}uFh{lr z)n4|N-Q2;-JG-udQKUJ6lv+PvrO(*`uPe}8nX{3lfR%>0)I!ylqjLq%`Q|`^{>A#8 z^{)su@4GqzLAB4-)zm`Zv67JIrsz{H0wH5NDndr>z`X(}>4F{3nIVcB0m9Jlm{pE} zFMz_egPfck7?De}R@WXr6hd-O_W}b=DgQVD4wiqiiLNPW7tadq^fh7qX2_pW6S>WmyiA8aoYDYh)ckC;yIExpWW~Ue z+9cYYYOvuu$eiOVs{vnUe4)Bsahn8$wd&zz)e_!2)t3N$DCaMIs4-KZBV8A&(*Vd{S=6|C!lSpPj)(AWq0$vo*cqqtEd19FnsQ7zuDlV z{F5bSvypVitUuLBBzqCIclGD>U66d?Gn-;8|m2|-xDc5}6PDb5~F{~d?7Cbt^1~?e?E1+|Me4a=$j`v!}|4GCEgA9g24PPY0yeFSNQXu=5CJLizEZ?6+NA|j`Y`<<)O9{C-C9?7 zJMnl6m53haW0nx~TOtkzAKO(axs;Y7<8kH2%-&`%wxUVh|HRh{>Hcy}rY3K#}U_hHAI>tf5pq=G%81jQLE#_LOuCdyAbrUC_06g$u?$w^r&>KwMvx z_nt7Y`doH$b(#AR#o7U+(Iurm^)=W)0H@^T;;Q9PJTb?adM=(R;CNbT0bft7Msu&- z7EBx%1ppR63S%v>NWJG$W_?GClk%O`I!(;HZ1E>Cgjdzaupg&jeC(RmyYDj-;&#(p ziy_umUJUkP1HT0HAb*l(6vi^Hh1k}>w`BlN#%>;}|8vNr`s$<|?x=OU(Wr;_cptXU zmqS0TACU8<@#R_yN4uK%LG9-@A}Za_f}}PxRd(achRrWgX+d+z_)=%>5p4PrsVSr3z*=FFnw*!;1?NfTW@?dET@S>pH)NaOpz%L zbf6$}MFAmoa&M$NFl@(H`!l)Mzq6z+#?)BvhSkBJ^7KA<4cjnG? z#*)ub1i{6xelIxQD8sFG_>a~Eyv={!177WL*!5wvJ#JYd0iTP$j=ZLp z^5ACL)6nE>t$iF|pi-Ng9N&?x&6SrGsN{T;=AL`|{O=^r@~Pzt z96AA3qif&&b~Oj7+HFSJ@kBlL)}s&gPVIkFhc!DdW)L!{IhECzbgYlxIb;Hcv&BhT z(n=NO>@;nUT>}FHgu6Lj4;ZOyKD(Jw-Clxe5UQWK~kE6I= zs0M;2`sXONja&To^7%miHrNe*OE%KZ#A?*E{}e2w3dsA$Vp+u%Ilmo862(dj|ch4l1ZO8q0+=*9SYi7h&fqfH`&{aQtxtexZgtnTL*};{8pjv_V zG|$(=<|=55g^zSL`Tz@FVo1PTYmITI$9RSSitM}6KoRecayZQ6t7LDp-J0Clp9*_* z2ruTI*(TX)VxbmpT|ER)<(10D|34D^P{NK*30ba z4Ds0FcaV9>Pr30T-r%PTF=_T$}Cs&cV^6+3_OZzM}bjur0b4!Ar??K^5d=LBK z3N#M(_LkM;dkYB$-6}A=;JxKT}CD55H37ZRhe;uY~;oXPT z8q$#T4aD^+`%GCyb(g&-QjLzV&{w(4mO`d--O@xtaO(O`H~5Uf2U0U>Li6@>HK0R% z7MipBx7@Q=A&bXu9vAjg_orW#(fxVR828j35Nn(GS6o#-8=o#U5~K8U1!F}xE&lfd zf*8GwD0s^&IoW|N2K&3)APoBW(Di`yzTR6aYODia-Zd`T>$1-2&fRPV{C|m%)hET7 zj^!s?FNe{`pGiI5s(T(eVkc(I4;>4?M^nSv9E{E7-U-ARSV{(r$FJSLuHYEz4wrg> zX!O4>{J@tr4<^YWB2TdE_~jgcI#~-q6%PLztkq4|{*x)>^l)mjI@NA9o?1f% z|Mj(dxvAb5{oT9qLXr?LOE{R+0>MNcyjxdRafp~!;WFptc=_?+{#GvZc_TFom{}VH z0xh>s&U?;F!B^bSPzEFwSiDau{pr!ohc}pfV6#v3zkl}X2D{6%YGw$!^P&LU%I(qA zg}FH%8}bl0sIf1=|LQthSdjQ?79)JLxcfj6@nydux}>qu0JJ;qBNm|$X9OA6?In%h z-cIOOq3}3GJKOpDeX0W5$uF%QC}X)&r+WplsGasL)`&m=%q8NMe9T8Fn|kZ1C~?xZ1|nJQrCZtP{DCg;Z=qkt#! zJP9oi%pRy3=P>>>0b9-@)>NzS$=x;bFe`OKe2j*7i-lh6&kvZ`QK_DZ7Y}6+tyy-Y zih4kwFwEA`fiCSNdj@2)H1i9ENorR5-my`FGmHCyYjRA#CX8R!{XD?}fKdWdPXMFR zg%a4dp7C0vn;4;lq>EN8wFHW^NZx0QV3FIFZ;;$eSwXy{8TzbG)yKw{2NgAffZXE~ zK(e^B>NvfdnIw0?-Y*$gvQt{bT~Zu!DlJm{dc~t7AB50nvRzj zWo~c7fd|WV)gdN5C}?PMH;Tr8y4=3Ir!cPFV}Q8x$b|;t7zOI7qFdewaV#d%Vy_>{3HU% z8n4(z9pBFO`3n3@`WDbD***720(G51>?A-wxmi36isTsRL|6nehUO#| z2~o6f9js)QmY0xgkrKk6Jng#SU4D&?$IDF-dT$JuQL%V2%-`0^U?l`G1_i}=si~<= zG1gtG(EY`a~b`-g`q=|o85p(jWWPd=|A6~g{Sf1;5|iSP%ya+cqgk>&sgLu=t6Cw%5Ya2 z`{sNQoJuppd-{YXgTKpRW+}Nl9fCM$%tRmCk9Ac+CcPz<(TFFP3@XFJAyo^XOcUI` zU|*X{u=~_kWpciqdSL5f!N~Zxh(bt2c*~b}DBxuV0 z?}qn4Oi#lDiBWz3=(uk69C^%FLw`6c)3x4@iMZ9Y*)l67c(ysNK2@_yR>=!bNCG2| zQ9`W8Eq-Xs$-;9!D5cT(%$_fMdbsoNs;BYGw6)*e;bLQ=l=9;VR(f3nG+F3BC>Fe3 z$*95_f(~OPngyIBnFhTj(V28)*6ARo+xdZh{w$CF-XrH2?iP0z1^XbH6rJ)5f zkQRcJd5LF|OAIPsfbDuCM8K~5`r3)WwKXmsn}V>6rRnr{qu(H>KXE>N z=Jul88W>cwD5t&){Z$-hq+@x%P?ZQtrlrW1yD?A*O^tVG^R?S+Or!g7oGA4dE zQ8rdxn-Qb$FWxaK8P@w)%vI15wuJRepP#hBUw6NLh8`0(9L#4qS#A?6-Bg@=n)jJV zPd7?w6AtE8-;iHdk+O1PSR@vTnzY*v$z4FX4S(4TjLLR5@*kaV@lUYzr+Rc+rBI?z z@fQ)TkdUMpD~|`aDseIExisY(i$^d0U^?v5hh3R}MWcbV-Uyr@oLH`8Ca$K2hT!$6 zO>R$E=H%k}L4;#+5F?$X#V61^efF6b(IZ^sMS0vB<;+I!jlQ!QIT?(y@9C1 zXMhL%s?XZYY|`oJX>8DsZ&3RQ9wk{Ei;_}g!XR5r0y>XBI_EcO4Gnp8%x(E}-&)na zy4dX^ucilt!~PLg(Uj;QQ1Cd@lfCUt^3Xj4O-?Q3&qn5YRtvWhh>fIxV*FQVF_3UhM zHl?kTL-nVJmg1?1=6>rT#FR|h7v_McOGnDiEn}lsRhnk4!^BSBcbvrrym%6g^1_y` zdMq9*jL1&ZOJ|jbjVgiV+=aPfORy2Th{aTfEDtGI=1WYjt(@ zymHmL8Gl73LO1fS0YLbJ#wv zALvr`sFcsToU3>es9dQyJ6XQF8>dYmAN&Iit>I^!2AYdA>}tQl*xO58kL+8cuSLQh{Snd8GL zzX2)dvZmuuP(gaBAO0ZHfAY6qCdPH2CQmr?Em6wD%T&y{VX7@MYT-_aifQ!n3M+_9WncSAe`rYJ5a!e3xsqi8zb^H{*z=+ z<6hyb9`FOy&5O-N1#P$Ih#N1C(#H-pno*7Yu;K8#GLn!lkg!p>n+GbtLw7kZD?`YD zmQ6RqbR1|CjYWy%p$+VEe&L(OZnCiGQ}SQet+TD}ofnb)r(k%yY|jw7(kMw5i-PPc zV&<4^ZtzNr3-MrVHupDoUj-HEkEHWjMgkmSAGMOB7(EELiAH!Zk2%JHoT%>U$t%qj z;>51+rh`d*lXjI1mpuzVn(wI4Lod-+h@Ep%-mt-!!*H>q}Plf=(0Ij{O0Iub;~% zDVnYpS80z7iX9$|a{81=8RsfDH52}%rIU}_n3#AqnCpB2JMI)~^33(U2sdo5Qm1`C z!at^q;K(Jzar?#~McqRc8YdkYKvKXWTis}}5`P<)p-(PbuC^&0!wTI=-8E8uubR*k z5^V@zQ{HY}t*m3ux{>~|qGiJJnmT6)2OO43!U7G|2sY- zAn#&)NKHBOvWRu*tbKe=X0_LDR(hnE58UoCDBdL4pB$x&B0J{iVd0xgYGSv21U%YC z)S=_I^TpgkRs;V){7^rBV~Ttbc3)z3%J81Pj7;sLzzPgnF>n$&{a#_!+|ba_xCQl? zFlhI5v2h|(?2^OkT>vxs@t(Elby8xE-w6g3 zUs+c1Q?kz~gx~()W;#_8{s({jaItPSQRQ@ozgafT@JzHT#?T&T%!hxAnVq!gPGJMX!s zR@A2v^$cIxI9xYW9#(d30$oWmSD7@qAIOBOn$?!WC^t^))3n*ODIm!_<~Aw%iY;cA z&XuCyZ|WYp0<5ALb;(UUdJDp8F!Mo8!H|i9RW@{9(#2k3P-I4JwGGUqp{S!*qb^m# z{EiB;WAobSeLW5l@p+U1UyUyu^sZGSQC5c~0mr5=8L^vY#F^UJrMz%WAD>l>Au_Ug zb`P7r6@^#M+&|U~#Vk+Tm7nF}zRuR~Qj?2D3&-(8daiiEljcf6P;pR_hZWG!$&JCy*7?HnFJ+#6Gmw~ zZs~>bPRF$)xZ+CtPO`lS&(b2_{HzB#d^udTQ2KqL=R1lFW~SRP8IqGf*75o z@?5DyP2#^j(CcPXy5FQ(`EI@MT5_JVDn{*hG<>)Q{|4g`={tu%9z#t4ZP1^A2&mX8 z5)dZ491y6nhjy*DmiHtH%%PCua1h9c5eNN3!@?WccFVmRD~Xl(Bb*XM2YGC)p|3t_ z2Go9m6vaj7rR3~<)(lJAwiwjUX6^v<#fq34Yz0WOipXDGT#!+>U#iiSj^Iv8`HTxJ zbRec{Fy60xce!?GUQJxPzV%VAN_DG$1|?Cxga76=cA)0EteBw7XGK*L1~0U zQxmVUg^V7D&0?}@z58+TC9Vi;Hh?e{G*HHQR6RH;bJq)+C|7S5Etcx z77a?ca)aISyxU~X8X$bjzB54MvbE#Tu57BzJifCS5O&eJQu44dGe~nfsgfnZ^^Mt3 zuoAJEUj%tb6>5CiU_F&@2)@yl-QZvPZn}EO)p4^6@zQovj5OAF9gPKN+-zp>@y$H6 zhpdAA<7-JO;O$!}G!Oi`U5^jYiB`)KvaYi?(+}H#eg;FVxE46o%@IfhYaQVS(gG4+ zj2w|Nob3$sm_skM)ee^HeXiS{VNZ)IXsC|N9g{q2U&voQV034@jjsS&v6qFxMG)Ve zcm*2fk@!3Dd5v&`p9!@Q35*n6`n4YivRY047gt{$7uB}@J#l}Wz+du>(h2Q0{-_1<`ai%l}o1x1fE-;U5vM_6Dw!>z4AsR#Cs=pCS+-J4pX7oGn z$Q5qNWDqky|!KjM@W$ml|P-K4k zUyf%f(A%qwmO99JQkBC5)cW^^;FmU9If8dM9+wYI03?rf{&jC5T`(!9exI) zJ*s3oF7$)F-8x!*O#?IV%2>D&&KfBNpRS&KcsalU(r;J5cg+|HASMDpI4Uz^h3;>? zS8skc?oe#pKCIp~rD#@qUXJ$yBURwfjBfK7I=*Iko|hD5oo^1ujvn6Lt;G70^E4kH zBj)6b=~On2DQKiSehnd zSvAg~FYtZ?n1+1r$4k(=1E}l5)5M18lKWHOxyRyU!$n+gR{JrS%Gw4efeErL*i^35 z1W)igOOc`S%TW8&C+c%#LqOtp8w8BY*o*Wg5#qENSsDeu7G41)_8EFfe5Bf|zWb{e zR6biZW#%J>ZP#eR!osD+){Jp91lW&P55GBlBkKM3x>iMjd1B>E@t;MD9zE&uq~b}k zj6D7$+E2VvK%1tPpX-AiO%2||h4#I^w!SmC!mtUDSjZnmD~gfyw~I|r+rOWk7*+cx zVnlS(qa(bg3jcPzQuh#m_A*t_d*q?2I>dTk+duha01Y*3lbEk`8RBhHf z<*o-%XkFQE0LnA-k*uStA3sG%Wc^c%&s!WWUof=aPh8-NUglWmeVcu}@FKklJHOh% zwcCvuQSy1bH;qYeHvgR$_R>pMBVoYvvAqZO6@D5FeHl`1=e^tXGz#6VAAQw&zpV9< zRJ26p5WXCuSyAcX zpqU!hth1)V^tB(h=qmj5t+4R#3R$$#qf+tE6Rk}6Yh8as4wC^ zeG-VLzrfGc8At(=hSJgvKwKJ1QOS|Q*O)5Uj(ag!5lN7<+t2P(lG(wBxnaPd2Q6hU z>pmm#q#U&m2p~-cbcF!YyxeRbtO7YovW+Tg5@dMuOTW`kdFPII^#`Q~S{fKQ8HR`j zXUrG`5=_4kWu!jfb2}lPX>n_Z>rA5SdI_`S9@i)Le!A7t?1bY$M|C$c`L9du>?p%@t}$6{BVi!%WbCu2fO#< zIAdJ$3zKQC!=?Ir)|mEN985s~qC@*cQZnOL5iB~xNuQ8)01H`mVEBVKK? zw+V^Gi^0IP_rBjp5OR#=PqHQQ$LPGlNxez8Z^jDXY}p(|HOkxfX2qQtHBf0pDkqL4 z1ZJ(xD$hy~b*8zp`XIf?ql91fP$FFEYj&bYZ0+cvj}ES;Uj$0}URcnRr@{T;*fFlg ztb*$UjIW$O7Vf3)AkmU~KB5wn#}(kSg*5ASymU3VcoMAwvHxsCM)q$D$yB;UUk%b= zX1mu$%#27fmN9=4lVB+Dw4rdYnnS&l)6z1p0a3z`1Y$==M`+(dPt{x7+Snc1MN7L& zn0f_ZRfj0LB(M+AkrI~F+7tG&5uOob>YQz=^OPIa$@ud{+*>UEX_<7GfG&^4RJEI~ zXz#+B-<7x0GXX87dEPN%DZ0f`E2@Ej!whMe-djr`k1x{&(~mMMn}?n4C4X8$`v-p9Sj z?LjK=q5F1?R$cpecT}$RTtDOk`>Rj}sW0Dd*VguXoriaS8=u(Kn=9JfoM;;DUEC{i zaTX+WCH43XH(M|5ylTpCH}{;oy;qQLpnWAQZS1QfwZEEZ~p9ri^vhcG?Y(z;oaA92w}|g?vMaiT-z#fj=q{ zsaK>(A#QLxFxKx83?OO?8hDHsXh2uh5laFC3mae9w||n*d-hH%cmB85q#>mKbF0`83k52ju|Otc(s zRsSYsgC}12`_<;J*kG^-rRzYP37H3x21?(+t5PowtI}wR)(1Z~)cL@t1be-_H+=?q zX%o45hKW}ivH{NETsUBCBwjU0)y!;bIL7qLE#7@{U!Crrcpm~%C!k$Rt z*>Ji3uyBBD`7gHvkgOl7;2uXj=zs@l-yNz$6}#MPQiMC^VY4DrZnxXs$o}o!Sm8Q z4tb6J1{W#aL?>=%${_ZgN=L{+UD=^uEq3ZAMDm1v^Noc zY&@&Dtw=r*!d{V$@8H1pVv}5cfQzd2hgPS(V7xt9ylQB=NGfd^VCk>x7EKPFl^EWY z7(OQtij7X^!R_07z`9iAB~=JPXI^>`nO3j+)bzViSG)C3kkCBrNQF zU7nN3Jhbf!(=sS>B>27EnT;sSwU%C6cW~Pp73NcU#?hNkNn>KjpFJHr=Q*p_$m7h!vQ;uu~N?#81xj7X0-2{fuSo9ngwc2umI-aNRpx*8!2QfRHR0O8LiQ=XUUMo{8&b;8UjO_u6ub; z96GU(``Hk8sc9acPdOI*vU1?y!xwMZCHTR%TCP&Q%0>?&YT>|iLRq|Vy%zdCNT=Tk zrPed9;N4+|{mo2i&rN@ThS>++cC_Q+v7mNI9p_V5ROqUvA@1G{RZuB(1?`JcP{}E( z&c#~rX4s->=SuIL?~G$l0IdDxF)SDE&R^hKYie73leX^j@xKQ$E{!}ko>@IPN9<{zw@O!r3*1LGy^e< z51A72MBd(B-VYCX&2}3vUVHSmIf?hNlwVhPaLRXob=Iij;1LhYwTngz_u`&Q*DCE| z-u645d(HtOIF9MyHZm2e@J ze&vaNBc%B=HS1}ISb?(hm7p8Xj;fomfS9jk_Le+HtJy1@|4BDSy{7Xow)SKY3f=lv zuG5mQ;(>^4w)=;D7t&((A&U<7lCbPU+M5Ei4{(Q%SKK>e4&lD*RZ`YCUCFpE6E(~I zQSQbvfzhmQ11X!sJpamI;xh==c~kTPW#}l%y!BK41LFFp@S&QBP7rUz-M+p3E;LFCOF zkNVe-&D#FPuR*jPOi)+u56&A_la|uSUvIf$p8^AVY z$2f{+Cl8v;9)6|eI4)GA$0~`stgZ?y>|}oys2;O86Mad&Y;GD;7UHU_X5j%P_JU(8 zW!~kg9pFDlOS9>9K`YLW$!M1AonTd?e)sD8hu;8}J{WQap7<*Yms;Y(^9?yx^a@KGka|#hdb+41>}s>8Q9x0|u^T%- z9gkJ6#(p$GNoag|kx~85WW0JPgli{fO)i1U$7Ug%1 ze5^dV@Jm34PKt^#i8&({Fi?%WK2xM@iSnCg0UCg*`h9W1Xj5{Y@xRWYSJSV*Jj{P< zBtyRzAQm~O&Aj`>jIZ)AhqE(r@*x6!56cTv4ihXa3=WB@*$gkPDr#$LYJx&j?X3il zqO3;v$zNf_{)f_kT`e?^f(F1FbGNXt&@GkFh+wJ^6lZa$EV-*KGZ(s8Y@2mEVry3Z zxs)h)5_rK|<=bMR^vayfcU8c?HMR!Um(==Pm z&)0p=fA}hs(=u4T_OG)#`>#U%*Zs}abGRqxQhtc}skdLv=rI?MkiV9R*3j7ed=Nu9 zxq3d6V}+G?@!DgtesV)NvjZlK;b<~x_?N!<_l1Vgi%KDZTwG#YR#Ve0&NtVTRy{Si ztSR_$J>#ahqK}YyE#G_S>QqC~*uooz<3(^UWz&^RjjE+9%l~cW=Cy-~S}c zP{1pYBnEo0SiS=}w|*Rvv9-+?hJUq^T)Viwpl^7qubBVaQ8YH~T-ENIR9P z55HGh9^-0cXZj5;`$|cOcDEC2=+}LFO)Sxc`3-MF@x?oMZ^hHNR(Pr;m0%?urHvi( zyi@dl4ClYhjnCBd(O*^0>-7A}KSaNcK>#YMPJDI+a~Yz5X6dK{OYhMJ=kMpA)Wc1) zJ8b{YCb&>PAE#_{6Wk?7iIJ20oy+u|D2{=Mb&T+<6VBEcr}YQdtG{gY|J;p0UJ3aAqk^L%)A)ngM_cM%52vex;a@ZlKR1jS>nQEug@ z(tM2;TT%YkS^t+Rq%N1uK`uh4Z2Q63xZ-^B*hAJU@#pKt3HS+(pGb#>CUTRH`4Uli zfd1E(WY6?o4>8Yw4wnC_y|8c}x9U=9l>j-i{Iv65`pkwy0!Ks46)n{J%ae#1_lJFhmpZxCRsYrlN7ue{y$y6sk4G7{JvGZyQn+;g8-(^g?$nlTJMw|4y$QHf2@ZXN=r$;9Y#$=E!Rw-7w z^Emj?vCU>MA>ZnF&sh4JS%KRsu>GC^3e z&jyZ164pQXYN$_49weUoTMF}8w)ZE%xTsfu&!67EmB=^5gK{Snys)dQ5oG$@Eznl)9-Gz?Q5cPF*=bxGNBEV@vo?Gq8zbY zF{aK4Mkr8TaHfz2*4&^&U=5g33ej35#Qf6iTS8`KhT$(qMgt=1H`bYGXij|p8ZyC! z;7@dYct0>~G|=SsIQ?{CrOHRg)rBP_ z@Nm4IpPWvpPPE2j6RV+ z)4az!zrA)7*Z*_CJS|n352;0@e)YR=h-XCA#L~->MLnGgSAHuefaoOiX%MJs!~}^; z(S77zr)H30dAduW+9q{W@du&%T#_K;#mN@q`y+oU7ni_Ef815_&0v&|q)C(xAjFK2 zE9eWCRLr4xrOqc#jh>hxffxazBf<;}mH?n1C+4gW9VGA^Bye)XFh6X3%jYRc>a@~` zMkapyPlcX%`v!dgtN)cwos-;{LqQW{d&%-<1_+aqtoOuFJ;4ulK8QH^Oss-u4cyrc z;I-DwKd-7)N_W_39Bby3l9-Q3I~#y*y77?R-Fo}MP}isPVL$du)D`+k3KNcg)7 zBj*RZ3$4oeohNRfJMsIMR;zRT1kEb#^RGsaCbL;0{(8E)=A&8sb^|5f;6No=$IXhj z_cPELP#Lf)dLqn7Lw#(-dzrLfB{3 z-HV0Z1{7aiq3h>h7@OyY*F&TXh3nzW@Qc?%yY(Z(Rkq75tQ(xUVqt1|Ci&BX*AYEY zA41=~N!a1SvwqwHjO44f)d%haiQF9HaDq6|5!Lr!6HCPsoS4$qK-ZL;ZJFT{au*Hk46Bse8#l6Us0L;ND+)^s)|w>ZO>(R1oEbYyG2)OA@un=S zQ=SFNcCqE_>(YP{wJ}Q7V-yk)<8RsPq~-4*5XkD|&awA##%lbgjfK9~m4=q@?t`cT zNd7xD=wa^x!2hJ(UOZ{G>F)~pXc~r2&XoSU1=%$AiO<&5QAud%;KnLRxc{;3&u6oJ!>2U}9!2NKc#F%D(rQxB3(N`;qfS#DV zeCB6vrp-{n)<(TpFm7?YW{hmvAO&GuY=gP8YZ{VO!F7l^8-<;(r9+=U~ znl{m2s}yh=_r4ND?b834KqE6ir!|y8Mt^>bbYnH4#dNL4e0nI^QJQ|79W2KkF=Ia` zbalDMYUtB1w;IJ-`R9Oj>0(VSPU^e+^~X%#mD&Q<9%P?|?;q!uTf#KrCxSiXEtP-I z|ARQr9;%4U_$6ZD0t_EA_;1qX*XDW|FpQ%y>a8x4hrjdINlY^n`m+PgZ1e48Ak=8` zSf34b1i3zX2P``BxE*UPf>%x;r-o`5rm2G!B(8c=eB+A7m z!CerO_D_k#$zQJ!rEuowApg)7I}HVhG`$r1PW|tJL~`LXj#iiOkgwpSI+g>?g0{A{ zmI@ek!0^8^{2=zN#VF?Wb>aH9YK*1S)}UiGW}G~Iy}-tgGOM&WRc*CIoaO#!GMwLC z{$XymkA#x&6fT4(gE%U@%pNM9xMbI~{as!PjSzh%tsol^sZXDwW>7h#*Z*iOdMYY} zO+=YI^mUw@^kvrwADn!8O0l&5lf>hs$A7@rlwO_c?5m z7vjsQchkx&v!T(zz$A)U7{yA7h(tZ1nz%&aknHjgf7{S+djpflS|YQ+&-yOg^!)Ax z{4n&|(Jlx%#-@$|bQ!vzLRia+}4ek6XQ>E-^P*jy`kdMVhdu;2UP2W_dYYNIIMX)XdA7|S z{`mR(-rTD* zUw?C0?2M-rjSfLH6|ZIgj&=+&C5-Nh`gZrenB_D7{VxQQ%B_Dj3c`RMUY4h;Y{=PY zl?E+t{5nct*}-&7xxQ(RiijqkG#O}zy#miG%!R}K;-AAxf9fj_Fzlc&?ZL>)m_IcQ|eeeO{(Gh)`RwmFS%Rj#T zBPNws8XT?@P7oN?rO^F9(b=iaN3Je)t8a31{etV0SSBLjLn;%kLa9!9T~Ck<**5nnekLww@vpWhd`)n~hIU=esRrV91TN)CBpbQ1*6}l|CD(?Vkj0?BEk(89K9<|^nx`Ik-L>n;L-^`kd zZ_AC`809%UHJhUx7zjH*G zWp#EiBebWK)$ zbvVUT_=!t_5b*x|`I=WK*AtBiO;n5hhaM^Bl1J=C+9ub-spYTcj{b)EZHL@>8jmn) zVo5LW(&Su5>tjRc^OC=#N@UU3Wj$x%n51Xu)@5WmK5PvX9GWb>DKhQ%)Xk-;%T8^r ze=9rbG%zj&;duC9x%zjkF$&Ts)$!2#UdDWlf^RbQ;xBRT_|J@0{tgJHN-A~t9aW$E z-*l*lC<$@zIH);$Y7hV?03zJ?tXIeRhF>4&uR>v9vF3Drp`|nBI-4n+&UWYuWZy#f zev6-ERe~37_-&`!iWIJtv$4yUy7YaA4wi}E)^0~!{4TbZ6UiQo?cYA14}p7)9oi6p z@`dIOB<*0Wv&z@P)KZYbI=m1?Qzzj4NdxyEG@O0UPrPr0H%`qQyS%)Noq|?TQGr$# z&=VyfVaXg?!7?8Ld;z>1Q%!dOG>W^BA8ZcgAX#=5)9qB(tX{ps`H@-`NJKpI=0T7-{oL_jVjMQ^r@qX7kXpE^Vdb0dLUYQe@U^w8p|x3iSPtdDeVXeV_tI84#j#_@yPKx<;{tfbJE6*Z zU%_8R^#A41x~FSV%WGW=3Sh43>cQ;nB|9~WY+{3WpOZFr@w{n}{nSHUQ0uZ;qkO4c z3+#%WEME?L*6Q~9yz8EXMdXtvsAS%|8*^SV?Siwd3n6xW=0i81GE!}s+2UBWu(!dh zw(U=i?j9~{{*YuWN42Zn*7f6Hyr7U&x=+8RnC@4rLmqx-XM#J*tx;2-ET%sdB8Vdd z*Rci>{bDIcT&O~t&-S(Sk(Rw#$nU^;_+;?iOajgT8W2Jx`>ZClt;YD^c|vJI8|KQ- zt8d}oeMaThEf^W(|xyhFM}+#!z&uGSLQ=1pW~7T*5z+Q zjPxFHZE@A?NCk` z9lPl{|MijmSiEYmc}}=et1mNJb%u&8c91gS1d#v5WMW2i`nknRF}{$Yrf7Y=!ui6{ zU!-y?&3v?L<;^YmE;FtP117)5e`^84liD$`DuQDN4+NEA_}Ezk6Kjyv3)S$5oe>z|EXJMYYk?xs?p1JlnNqhHUUt z(jpnM&!c1ShwlR0TM0^7-45THzTNWQ=9D6Hs%&GPvnJv|CotqAL2HGD`l+zbnQj`+W1TQTsvVX~{U#wxa+ zo|Zp--uhfBozq0j8uR7!);Oyy9%maZLyVnu9xQ^=;zW1aK-bm6*nH}0ky=RyHIul} z*1|0)l_Tt824YQM_hMH4@rX@VThTV%t!so;3;RqP%S>5M_S4F2d` z)FWttKL=jxF-wF?y0JwRM>>s5FV_;P4gY8r2U0K zf%K7W)Tpfc81%$|q7e^zzW-=m})P9M_U8wxlLMQzEi1O+72{zdza+9J!{3V*3 zh+OEn@#Ha002r{a>e>IBot@oR&W}hGb~y>`V-)lci%l7lREvJ-donpI_1RgZ{>C(f z6npDd_~z^~c+{Y(a(>G+L?ZFUq|b8ujflhIXp`+9xzKm!Fv2u>)w=s1e8!m<^P|2q zy+6UhS~?m@Z+mb5HGyLybz(Rg=?Lqbcy$Ak2_IVWN+pl$4Y- zP`P9~fq*K59D3eV4qB*(nzgAS7nSlg+CSIFoGDa@zC<%%N?_0yM`s!VnTKc^-WNiok%+hHC6M)>NTT< zlHf%@W48X~M!YSxWM{nn(<$1D-@sPQ$IVK}MwyMF=YAZT+uj;h}l`>VaAB>RZx=c$(d=Q>7=r>lML|F`EJ;u#c`c z!HU{!{swe3`20a23`A;zI9WvFk-R{9-SO)Nslmp_LmQ*j(=4Rzo8yPOfp|F;8u3Vg zFX<2wXubhG$pPA18E@Z(rDaYn@3xD2nKv>Xnz$j1-N?aUHKZZCtG{ zRWcGTGuoca-$@KT5Y|YHe9po|JrJ0k*fYCas4A*#EQyJ_NggMhV{zLnptbZ1!~*U+mocM_ z(=#*AT=cLkt^a%rs5?pH$+1rw-no0rC<4FT3=K6$(JQd#(P8HmV+Z=!0k6abn8SH6 zTZINP!4rS8KM+c+^|P%|@d18el+OJLSrX@D^=ShbGMZp9>US{e%W1bjDf-xxVzTa_ zA5`!dup+71{+OUx1;1Fdhrh`i_;BtV>HJys-5zlwqM*W?BjGU&+Ib&+Eq=2bTEgCF zgWAs+{y+sug2jlYhDJ2$JC~92GEIxJ{D9)=@pJ6Qgciz&M*-aD z`fo=JF&nqfx4#CV;S$SP#|l<#@==#4`a zswsB3DPP|GxoZE4Cq=jB!Cm{*nOj(x?1I0bV+wkWN|ujr)g?uPNsOTmsTYt)ef>eS+%9t&l&1HWrY%^1Vw#NNKjEm?1-cQJ|B)A!4X!cVZxZGNRB9(oe z?MJ8H7}nQjC-3^cdT_g~b>{Yj?krKPSU>Fa;QF1ygtQ{a_IpL)j@Bpsshqa#r>C_K zgC*NA_m#-5QO*q(KJY-WcM+EN5eY|AIzwN|KGJ-PY2a^VDTx7LOw6HtG~Aa$9;a)- zA1xIu>W)t2sN%S_${sMP^y2YGtdO7!((EIakp_GL!NkNj8ZCMIL3yp0E23VGBUnd!n&o`E%w#V{hJsn z;prds9y?KjRKAzv&!IlT1a-DF2;tNw2h<2`bg(g5?Es8&F`yAoD$)*w#lHP}DDss* zHX^@z6CT?u&O51K>X_I%PK-T}SU>B>fHttUiJ@*`VPTKTy2Cz{uyvsED2Y{zJXA$f z^L&aH*IxhXi{!9I!cnkQEV;1l9?~Gr@%-KTf<`*ja>qa|8^GgzZr`pmX=z2BZDiDD zy&Yl7I5;p+^daqh$k_<|RKYuH)J!q^NGlWLbr|q|+S{$tsIBMZW|arLFgYsnFmqr( zr&T$oxNu;gt1AG$cL4$YVyh6aHxto7DH;RHc<`mCvpIYg`ouxiesv%Fw0@Gt>9}+| zGlSR1bYj5cwUml-ajeti6)o1Z4z-_`3_`s+G!#QosUa+y|&-$N#rMBiJ%SyEngqb0%w&`!UswDSW<(8qM7`16e& zPTSrh%f(FCB3El`Qk*~uatBGUJaDc2jVRvb28I|NKJyc z1LI`11>J2)0z3T|OLQ_ymE&E^T)mJ8bK*0ENJhwME$pJg?{7{-&4)-qj7nLD<3D7~ z%*@j3DFNdH4nyNG=m^|807(>=@AKOK3^bss5OzPebaWgeL4VWz@!%d0JuiYDSr4oC zwm^%V=)2VCxS#npU2a0H&b@93sXmA_7JA`!gjVG)_%w4(eGu1zR(%@&cONtO&i-^U z(--{I4o6@zcmc@NUP5n$hlF|@VU#RI1L5pMdIiR_Jw#3-aildil7a_muQ>V!Q|S}c zL9f3h77*RARkS6i4Q&kTD#K@Cv}Yde5J;TtB(=P@{>O5w_4lZWKyT>vA?>taqx}LN zZMN9-)RcCg}t8^;o)43%~6=%m(OpE^z_==yhZ`6a)qyf zEjV7evU;wru6A}6-h{ltJlbwT>EZA9ZN4`hE%07PgcO+3zddP{A^UB!YGhU?Z2s76 z_12=J;~Ys6SLEsFgfyz6?HXL``pjkZy!RysW^z*lw_od7(2rgvIwtLvw$X`@8!Rer zPNFY&p9inI(PC^u?eBgJqG6GdatE#_hbr;#mT4}!yQs$odp_Tn!fjE}p&0OT~+42RCvo10eVkR!1 z!I#>)h&I4n$;n;zETN;1ibIo(F}8*e;(FYgy5kl<A&VIOTbN~~#R+~e9;<&r3F@-;8)#Ri1y}IHB z<)_BDI5Ya;Nd*iyDpO#}r+w4rBeKT$tx>bX(dIrg)$J|1eBS>q^!3s`0D*3i<>Rmr zrDHSeD`6Z&Xe*sJZpk-5!uzGEzVorAZsqpsd-8fVkHz)R{ZTicdalYx$J>#ryRG@6 zmzkb7l!7b5MNaySZDF1lJN&iK&`(1^mT9_FV^2>`OVRROWDvq85QmJovVQ_{B&&Xe zGr)m7va~V5bs$^Q|qdXr(VCu-Xp zPzXeEBV7HN_J(f&_7BBXGYOiGa9DH5Eao0yPF8P`>?PZ3$yF&W20wN4Ci+1hun#bFZc!v`WuF87xj-MdED zYhcN+YmRr9!qX2x*Y(}0WNb`(g?*>8EA|zmQYIO{kNW%3iB_B4Rd2uH28PL#8@~fw z=rz1lW7oP1=(hY59srI6PfabfdtS)&#$*u`m~&`5_12f9rxwL#-<}2yf0J@j$u&4^ z8{;}Y-+fNQrd{nHo~pd4`~**Lm@HKM&Sl!FG?`fm;JZHp>^Y}QGvb)X#L8N?aYiVE z)6-pn&5kN*VM(pA(l(p-{%a@}gneSF*$n1gG?GbTG(Ta68}o9BckSk6#a(t*na1v< zLg&Sgjh~WXqOHawf>8KxwNdc)G7nWo>zKZw;aO+iJZ%_Gy-Cf@Y^9*XzV-W)6vKBxN`!W7Nx9B+ut&1@V`?=@ zBA$U2BrKVOxcg>gpZz_3!65O-OM%08q1NS91~)4M3k$k~Nv;2%)w8zt^^{M#aCQFr zXt6hj^1CdZ+tES@MfUK^=fc1t-AKishe2pumT$t$Os^b+7myYXdw@b`wDlMzc(sx7 z`2lwXI5LWqz6${E%;ma4q1Ew%8lV=A+r2yD2!68Tpfbczk?tUk?5%ws(Q4}PP zUdR#lzOi3yD5Cr0eZFo+$nc?f<>8KY@WdXY0diUAHz^z2So5t`&a_}S%hBTGyf(?c zvYO3ibftq#NkdDt{V*W8@z1F5wW?Teof10+DlC)=m(80=-x+*p<;_flOCDsayz$oM|h7C{rv~_LY>8EiCR(Nlc4MVRE1vCJ04!XCs!{@Nu4b! zPKU8?5MlTUHmh7}KZx37ZfiEFN+!bZC_PDuqr-i|pzgK3;J*Mcluco|l;4YR*`L2> z+aGRPH8>m0zTx17XFtJ7VuhM73mV$eWVw%UA@HjJ*n{tGm}sEk*(}hVpAviG`3Qww zAt{hN-NnMs?>PKAN(z$JPgZjMa9CJaCQ$k-DleDY(NU2A0=!YIW4Des$0Aj3fOk)8 zjb=4{vTXHzsx2^gJ{sicm?}H*2e3%O^h{!!+Z}Dy(4GKnv0DYnQig~mCD~|ILEO%UqV6sL41*@GZSQgAi;03~ zg`=rw$7qXaxa3X`M@_4VkW3;0S?pz?nmv|+-wC*zsdS73PGFGDb9Ori3O|dlMBVdm zsS(X2Y5mS%gH+ZP{121a5K*_oRCqeY%W8@?K(ux<RG%;B8_K$8@SDmLzM7=k;} zq9919w%Sei6CuyIkVCOvRvX`wW*d>{VK>_;8z?H$kcVi44n$1vf^d7j+Zf)@EAgo2 zrWV}60LJG87{5pqY`@ezu8_>uazC&Lm~3klCLO%Uq zpb&C0G$nJ>)6i?)Q#GJ{q{rp(tEOMNtjmT=F$>R_TcsOnOj|6sPLpsd45ANq^*OvLB*_Sr9AXeBpJ*rM$?1!VjBRS+5uMR;PeG=vUMo0%ow(!Sl+4<@Zi}fEE zUI@(Dekr!IiFnE9!G@D~wXZVW1OSF?RD2h$EVEf~f9#5@O951{DRreI8~GPZUYTnu zK5{g#%l++A)!zfq%k|!;eGd3Jhwo|857evWYA-V;^0&GBFJ4IGjy1IQY+7(6U;PVj z{!p*Sri+wf&nJ%bns|7dj_16`XgW%jP=<;}t5Q-@WPXeTHNg0CwXL29 zy66U-NJ(gOMt;G zz%aPGOJHz!4;tJxxCD214IbRx0t9#W;4Zl1D+MBf9 zulME1?|RXX$SQhW$jU{1v#Yw}<(gizs_SlnF7c%xWMluJWr4B!*hM|ftUcu(`m7mz z-+G1`KFAq3Zp!20np*yu`9S(OO8WX}(Di@f((?4`z(FNL5hE!o&$wkFt6TBQzB;0A zn=w=PSGDzq`^(GpZ0es_k~2xU=Izvp92-j;9`Bd__}K$HQhu01m?R~Mi(dhqBZ2M0 z>%>+zRU7r_A{Rn905P7=b|}qUXTsiKJ7;GfeO0lYEfe!mXtfDiW&7B72z?E65Zc<= zA+E@ee`F$p)-yZXt@VmRccrUzO`3-L!;uO)w8@5mL*3sc6tD(GAg=-h#S8jg z=qs1%w`FmV-ISH)6MuMP9pf`$kETi6J{eihhh(|WoV#P;1-2=47yLEut$qjloYB#@ zoVW7h(?RlWK=zp#3@h6W-Ol%=F?_5xe+->eVBr3Cmzj~XsW59k+@{498cwo9Rcj3!yq7X zP{8(22U%21+TMQY9o^OKRN-fVqcCc;n!v(Bop!Wl)ZO`VO#`p9#^ri*zcXut)o#D{ z6h>WWu~K2V5=FpHC7UkGQK8L&JPVL5SS~BpubRx_(@lrN^W%KW5Ey&8x|}3XLT~!n~;>tL!oc*v`Pcse%8Ft#zmpAeIF?@cb4rMikF>3TvS~|MAZDo2~atnVT zTTs$W)T(5R1&=!jH&uCJO^8uwwj!~@Bj`RgpJydk&)9Ya{d@;rG-2e13M|-l%2M~lwqh$pAf@_q9}o; zza{coN*jwX0mxX(l;QypP<%~`R8pqk*eJlPH(R<$?P8-73V@*fg0ybM#^k6)<&v?lg)f2Hzw(eE}t&oXIm4#~_kl#R{#mh1y( zt=U;#OThYK*xphR3U;4#FFDKV)~wQ#<{l4#`>UIdUxfuw*hObV3#uIua_Odb` zTZuj(f*OzKmX^LlHKV9=z3U1K!J6;f2g7%>sI%^o5fL&nGH(y2WL1}Ej+u|J>)n1@ z3FTI^>-1NC=h)d(mPr!LK%_e;UGsbI?+RTgCWVU2fG^|{rn&?b!)Xwvoa%|b{diHS zQ*TsnKfOi}EBLQ}(!i&rg=VP$oMxqJ;U;yDhsqhbG(wA8;lSfvR&03~rVQOM>12M8 z>bDpto|jf*5_FTXUz`CJcub2S zKLuH?RuixCE;lonp#U4*fBxjnO%hwr+R>qT!FjTv zsi83e)xsj$eE0*+KqaF-LjkXweckS|vN8e=3nf`u)OzPC{fK@T$v%0;@sEx;h7SOw zFdL!o4)1-ksgUjQTUXa0#HeYu8V``$TiiD{LV^*YyrP1E*lzTs zm5MeWg|b2#gllDGXBTuhg+?TyhcLJ<imtKiS5Lxs{n z{!me;8mM4e1&iELjS<`x$=~8cSdrf${kpn@M8CtgOfFi~>ZMr+Y5(3^CQ71LcYo)z z`-9fUi-Uni3176l^C#A>!X7EqBwwq65*rH2yR6S&D>Qr|{S!^%cr;S>{kMYo%&lf) zv(n(6%)OxGQDc7o)?)XYD<~c*SQHRu(Hn?h-}Pul2U_UAlD=6fx=z)_a%a@@yN*Pi_#u*-}eTk z;zO7_E??DFwy8`LAsu9Amqg)`I++o#351Nd!@k&ey!Dk$;8qqnt4K^{OQi109&?3rMT|F77 z44cgYEsUXYEaVKdIzG805t#Cw(>?AN2KKL1#0ovXlv7ZtsVA-4GD6c6O#`1sS{q(l ztHkZ?oH7R1-X@#I_oa0H5_x4@ZhhWOpw%scw*q6MI93Sr-}@u8{-$0KFmgaAlgRJi zl_IEeoVBQ)ElXex?Kwr?gSIT<<_B7;TCkXHd0{_qP!hr5RspqSEUE5SvpM~CT&q%b z70k$nqkz19bSI8aPALgU4l!MGQl%XYU`%A%r6$Jm1=}osljCjiYt2dn(nv})gC28> z+R_2T!47_cqZt|h|0nD4>B0L#e1AM&URsLOvrws9US7VF^0CLqG-cItE=tnf?)`K+ z*nJh+-&)evRl?@@jj3!H_fe}BB49RiiMg`^`hX_5G__`fVsBq5fdA{dKld z>YqlIbVUH~67oqoqZl(@WkhMJAG8H+knStlEOdzH z+O5-8=r-kqiF2%?h4$HZH{Ye3ck@}QU=4~ALO)$n8;IcGn+nqRKr;~}? z^zcI$Gf`2P(cjjd33!I?cy6a!fKJa_PpA_q6ss#yWBU!|1Kpl1UaZWQe{!!dJPDJe z2v0Text8?%2bcEny{{O4E-W7F{*1qfyea@V4IkclyzWk=an;6I9$+XCKlSqE8$8Yt z08Re!2Fxm1!x8_d?N9fU@Y|ia@FS@#spmsw^n8BtAA8?>uj=lXS%<7hdQYg6sE(Z+ z%pc2w`%Qz_eyrH!)Ko>NCFx6HI6x{+3anA7gA*;mj>Ku~KhwG5cq%XLCiUx`9^lI9 z3RT~W$^p0 z#;lwDW@19ot1Op{Si`2lfCIr7yJg2EBqSIrldU&jbGh7yafOCZf)wm#D=`Y!tJ-%RPj_v1{zOB`GH95$N3#dA22TlI zmc6a?pd>CbQFWhZsL7Ikr;{>7LLR@`dJfkrB1m*g%Xne!aaDLVk>zX~v^n1enV^P!sjhAXMncRdvv->82L`@z zuXpVQG@!!|G80KV0%0(b>6;XZc?C`;YT=%ed^`39S8=0H+g37ClumOVWrZGZ*lT@o ziT-_u%-)dq+1Y<`gj%#-ZZ%iy)FawpwNzG~K`B#ScgvyFav62^4+Gyh9Pb6|n{A&P z3V+cVT4>wu{3p+!zi%9)vVNCe9BN)=-Tkl^ zgp6f2k>N2&;_m(QFNJ&aW&E#^ST|IOlh@`7dJXBkXG4gfkGKK07F-+(%EaKQA+u4T zmg=^;K|$(CeIUNZ?~Pj9=NT{aO+n|00Ju=0x9{xnffg|hG>{+A%mL~x;A^32DP z5hTW#Yp3IN_rlw~Su^ag%L9Wbl~)qRMKpy#I3!gMHMfj;I8q%{L_2?;)6C)a{=&YZ z(YZ6L_1o8Lz8FC^Mc{qhKJuGigXL&^U#JLLX~>_HSZ+U;&HoUNp1Dhy=7n#Q?Vnx( zzU%7VPE>AbZLc?yGHZ3*=x(Ny*p=PVV-?$q?_6{?TOIvGFnC>Xcdh$GD%j+7Hkj=5 zEZlP1H3Rh1wceRH)74!8aUD6>@1>}nxnO=`gfiPu2GiXKBDf*o#!4b7w*BG850o)k zFaQf!;8LNDOdDekkMD8wzhr0g$&QorbmL|7$*hNYBGNfdE_MX%!g0083%~Ev#xYI7 zOj-QzCpgt}3v)=19BI&Ap)UnyhS#K^K_;((&GqS#s(t6b(U3p;YqU*lF>%7_Bb!9aPnqN@6A2oPg znRFD}Zi6uE30$MUpQQS|{2;Y^e(`$0l@xy3%6&az-qb!zdRcwhH>4^0~&t0X#M z9R`U3I2xNeo}P%HtMlM;tB;;;q;KnTx&8n;^tuc!-n!;{%J`515C|b33@HSw7@T%d zAnt@KF_P?mM*panOy`nd9y%3oV;hQFI;vP2{Px|Qdc7_wnD;D*~IB(aj<4& z0V=sPW*l0b79>wO1-%`(1RsY`BcCDP7N=s!Pw4fYoqW&Obx*(fRz-f7Uofaq`G#+b zOds&cKgj|Fwn+q9#0Y$%kOuKLehd?V3+s_d3xMyuhCF|>+OlVyQojd_o3$)eL>rX za#f9&Y2{ZXqtn&y(LXWz$t+>i2a%Zh`BN$Hd9U&d1C9&p&9@w_X?aqg4(E%1_nKYq z@7s^i#op|?t!uUUb~N~&;5wilS~yce!RN_osQ+`T_@Nc*F@1ODynU)pITk&5-UtGFtnkI~8hZ+e6{p5b*-sbDgaM2mVbi6u_zUGoNA zIUb^>3f&0(Q!opLR>qdUxypjB;4W*COW_H~?xd3-_#Sr_26*QLb?lal-hZHe4IYbg zmeUq<2G}nz*OT|BDrf?#tr=RQ~2O(`g2_ zv?NhcR%;A@02Ea>>y$C|8N>1lw9+|0Qs)EZ zFyiDr-y#^TWnr+$dpTkKCP+WBQ~PBrrFxCk!=V(X2U`AUmf!2s z98I617mg-e>&+zdlz&42ASoH@s%l77;~GzMAc&5&GF3>;+G0i{86N@RlHxXHZoPll zBFG}!s31dwFO4k{7$_=(rTU|&9|%4R-{gWx%P^)PWyjRcOL`3mEDArnu_f;=h)7Jo zwrj~@Tj8{=_iOL0N%n`S7}3xwX4Zfp_mPVo_0e{Q!2(hkS++E1#7>*&Ew|f1o5+?-bs1ly zxI)&XgegWF5s8$@d>c{g8czEqFp!HD!R?eKPwtBSCev=yNisCdLQ2}%1sIhEZk5_+ z_`b9YmnW--9QM&~>i+g1*TepHWE8IopadHrLj&h;L@i}#iHK!sd_%(=t?P*E+&}f@ z`aJy>s@@UYXN&uIO0;4AKgq*?33k4N4x@rG$%dOb$8BwVW?fEuGM=f`Ud9ZD_uysT z6`I=yyVe&uN@)-a1mZ0^b~vH8c&520x46vFY`+QJgnFULE{TBC(sY(QUqJCS8iEMc z=p&=jCu@d67kR(aaB9Sxa)A!NgB*Q+0x%>=0ku6)2*GNwz(gA}&PTk-|`{5}1_RJkyUz21s=JL<9yA6+)=j zyA68-jrv01UFWr&Enh>TD{YYx#ArN3QsQ#ckt@Q6d2i5tF^yD`#7lAH%38wzx@<~4 zd$z~^t`A1;AB&OA32WdN29i~TL5RyxN|284tN_Tb+H~A#d5c*A;_4=iUgN3RRKFuW&uZw z@}7bRaV@^j6_zHDrU@n<{C#5h_OyUtVCAw(u9!^UzlQ(@T)xrfojmA zuFgWCg3QWy!F zMliD855N~PmmcR;M^{ql1_gj3k)kpUH`h4hyn^F9ZT{kJD^kv%7f!+lS|mu}1myL= zBg`byhn2wzXLm_v#F?~UqnVrs4nX0kefB9WXOHVp(k2OGIiaJ4%(?&FXMC~SN%10X zf1RG`Fg(zScof97A9Lrl2cGwhBs>Q?cO0y7eDNg}BbQdeffi*pyg@OHwEA;d0td^O z3}@7KMb}D0f3{A_EB4Ep-RR<^64eM}(&|KHL&&)L!A7uJNQK|L+}x)?>$m- zBi4>Fc$5#A4XQ$*3#{*l1(1P^9ZRH`lKndxmGcnjX-u-CJOypb=}{HA+3!p85+0E2~Cik|E7-vq%xuu{R!Vz;3|SrsCO#HPtsS` z;n+Szi(JA~d_5(NnN;JHlwUo%80QZOUnS)_Q_Pq{Qt%|FIGUFGjZ%?Cv7eQAFSXM^ z?}8!<@*qgp(+D1-b=EjsqfbZQW7cUSJNr~lHY~@k9vV~P=0?w7lE}-M$cI=0aJpce;rP(=c+_g`A?$$Nr zHc+D*rv|PH?QaMPT2LmjB%)-R1ztTF9ek)YzE^hr8ur8ni}00g)DVLAT3R(hJDBa>wxPJ=Xsd=9FyFp76^=VTmB9SJ2Rw>3lhH#H340ysZ> z@4ExLw}!%EjCC~6Ja^mwJd~Yn(cS*bky`}wwnK*D(i70mazt=b^NoE9*CoR-u5(get zm1V-0g1;SW{8mqiGRgb<{ensmSJ`l!M~eT|jM6nbly7UCJ&o;nJFSxm@2`cS%T0?k zkhNh&OCX=NdtjiLxA9k9V}32o+(=%XlI%hDNpahRd7jv&qD(~Yu1{f~=fwA4*+%C@ zWiW*DmA~j%<>mWTesST(5=d;pmg&1LQXW-wg^AOYS&7nmHe=8fv3VLjFcEI=af_&DdH{OU1h&RP?e# z^UKB5K0vO|mx9>*V6u%HS5KI(8S^mj-95?fZJ(*<#(is+nmcOMyY*~!{Gkpf{>AjW zOhI?z2WRGuzg^Mi%W{j~s9=hcPV8x@|EK*Cx(VlE)NOxn=6&v8g~Z>QiL4;QkBjS^ zqkH7hds`QJGG=g9ygCu0yiqnjE_N?;iXAkZ9$VuSgAL@-T+3H6%2G>UZ&Z6&>y`?g zY&*#rVyD$S9?r55jjFgrG5i?8u3C7v{L`6IJ3`Gs>cVz>h*8hWIs3BJXwB~y)LpHM zv+7eA%kvy7I)XPJAH#@Y*SW}K(d}0E&X;?{iZU@7O$PNiv~1r+YbBD*uZ*M9lz1Q_ z`;}z(upM&YVtqKVAvn6DW;0NN{XyauKWU0aj5;Z8DE>)`&>?@~Dhp+{QqC3g?8L{R z!H$1S{q6?@W7S+X{u@CtGFh=Dbp|2UH$j(m8oDTcg^xj}D~D;?vXYrc*Ev5+*Cu%c z&k$V%=Ee`f9)Af99wV>$!z-H39Vn|=IwRjN-6dfxF6Pr)8T-HPp^Bj?Pi$7eA)`i< ze<1n9F>}Sl>>8A5JinekOgyUZH5mbFu_`9|!EH@mSWd{mjYEVH{ccv2ki$sjoB(y3 zAf%&jKKCGn@z@@IDIYYh)t$j6$@7N_v9=Z5EwQd;_x(dX=W|3$JBt?hUHUx9QSuBZ3iO)J$nqSmcOhr#0(^7 z^Ip4noKn5I@{c>y$+=CeCga*g?7QYPWLef{`_^1bg$;pfr3W||Z644ie^O`Y5=KVY zKJ%qRBk-s*UKQ;m+QlMzupih)Q+p;_*U=iiTJ)WG&wYEB zf@{~W9z7N9#B&{YlJ?m(Y$uC{e?-LnS$+Qlq;KUPyFHvq5!Z z?w@`?>QlxbrwMpC$qa#ZhMsPi5Av=W$n+N0W8!__+lW5puL;gAE}BD-rdE-`RSEhC+Zvxmu+TNcO|8VX10nQ6q7QhNl3)15K_Avx)wNPov~;K z>Et^#jA-9sNnd+#2Hd5bRSd9?=IC?`Cl@N6`Ca!fYaf@Tp_hB3}SCH+|aWO+D_NaT6zVIhb%_M8s1%?-KvNQ%Bw1Lk@VO z58IR=k(0j#zt527mGSWm;?D-M`YSW&~Y2V}?*J!b8H5l>gjYm5hWw(q8P=^G@P8QOXAYj8(!WTdQXNVxe0%yl+_0vJgz*s;Gm`^A$N)r2Vt2ZlP-hb? zus=>v=)G!Nw50{LV5eWJS*FlPwpVK?jA>KD6o0zJ9B4umDa4|8&h~c0m|1748e(5a z0AGQ^h_t!fyj7A&Dib<=YjZdc=66XoEzjs_wDjhA@Q` z22v*3nK}vglLV8b<{%W*$3*wR14U=Gp-F)`dz+6 zdYDOXOHX8211q!VJVhSyP-~c)AALAcpx*X7-bo=lZ>5V}GB|7~gdD`DP4F7cH}L}l zFgz7DnHmum;#Mv9p)3}>Saj#6tUc^n5Vc#pl8>!LEeo-MiU5PLX_{TxWHd#72j3@F zG{dPhDnFS29Ni?}>Hm-ncH`H#oUoK!JfT$=HilzJ=vG+cb^odb55qGb(`t~F{X~Ji z;M@l1=vgE>Xi~%P2%4mk+C<=Mpl&_$-8pPhm%E^I4f#+@<|Xrgv*WnvlL?M`VJD}q zn}kiG^lLo~BIAS}s^qu+w!O@0_U^STW+NcK7ve5TVkK#uadeUFrb!h~`xB$PtXcoN z(NKN$$|JdG-Y!~t($dEpiVy;~p?#vn;00K1FLA)5@00aO8_8Aqpmsgl;bM>e3E zk;>#TVyJihJ5}pq)5>e^Bu};O(WTJr2I*qw#f+2Z>t*W)(oD;mO{7&x9^ajBU}}D$ zP8To>CYJ4&)`l~_*Oov?Y58$vE#z7LV>5e26sSUIG_vbBZLu0}QgU@rnQiV4-=0|E zXg1c0jmCZ0{-beXRbd~_uitVK?OG5zWc^;18h{x*d|`O@uj8cj3C8B$WtJRCBmDh}i6ai0$WsDVtjh>?aq!uYEHIV$7o=zcdzab_Q&}L_VgF&Ai z#eLt3Lq7x#M$<}j}g7f31(x4tGDgJN$$#P1LP zde|L>p-HyXyakT_x!H zB1PO)(Rn>Lt^AW0LAo1HDoEweIA>`<%%HkczSPn;1@&Xkc0Yhv_*V{-KOBrY_m2m; ztDeJS1ngNt>3L+R$h7`tuk%%G41Ogks`!Od%?91QKM>E~y>l%yV%*f;177WjSRCGS ztY$fqWd~ZWHAc_i&N{hm2iHhVEk6ciu}!40ip9=rUOX|x_}7eF&m4kDA0yId71ca% z8t<_dp^)zC#^X5@Ie+>@_N07QOj`@C)IOR;-yV6Mo|EmJB=Jzqy{g((zU_})CZ{5K*U80yP`B|^(-lWg4= zLE9M~0T6SVg8=}-SXLlVE8?GVXT}h3Fe5EZ@ogriKex_o*_2*0myex@5|;PuA{YE{ z8-=pFcaUfnQw0EsFl>e*-2rfk*Qs-Y$=NO{vv`l)PitQrvQy|4Q?hTem<7p?zLVNW z)`xTC$4AArHNCO~j=W6nI^ zs`N!hsHKoygL!Gk=}sJ2S4+|?|c zgv&c&2+*r$Xs(RZ`mGxWqv&WQi8gT}ThRG*sZ5|ALM}op+WnSnc(O9D!_O^uE=HE= z?b}^z@pc-%BZ2cg2W7l&`ntQir)gLo2qOOsO2@9Pl(JC?!Sud6Vit#9gMRyhngn!G z)Q7mN({muGF5HG3fyUw_g05Lb1&%?C|PWskp$jvUP9dhP%F^XIA2TF7)_27PNzggTIiul~qlwn6qF z@~8IAPrHX)pZ&6?HfL+$rN2=z(8ee*b0{C1mR(_B1fSorZjOZC@zNQz7@zYq3_5!{ zm9b(?7P8Bu=WfWOZJ1cx)0PHcU|edpH|Vn~Z+Dx=?gNVv1larTSZl^=`gJ&VGOjOK z`pRCC_FL14SQ5&ovq33-vxBj@E{o3QS=G2%-v=A^RlO#NR}~B4{NrXy{WfLHPORjA zVDIlxgF+_1pQoE=y&-q`3W9M4MJl4+UZ3za8mjgqkULGDtF;Ieguz;5K0}xB;q=8o zqMB@R4d{~ccshbEsYYvIx$exWg*jIDV@xvm8EK>an=+VO2W2qcZx}-PbX1o`p6$SIeY9GH2n+N0RAMFc~$9i@9$z?3TI* zWUW|A*NQo8jsIM3k4&Dw@bNA8kjo@pJ-%EYOlEVqxx_2-dz}7lzvvp}J`#hXp~q*VaoJ zc@)fCMG&~lKaV%k=NC&vdg~K<{@=2f6l1!>Ezqj9eY(l{sg<^)o$V@ZW;Lb3gN-x{h+1Zc7oO`gx9kcDS9w*+1%Xnni=CaVOw zI1moJGD#G~^6)d_WLj5vTtX=e*93_2i0{y8KS2`UBUNf#Ef0(m_z&WaA{k8+LDGDE ztZE6Wj;-*tMwATjlx^8b&qLwV`Y!^F2`hJs6$##XImQQxiJ@w5**$cwlzWdU0uqyZ z@~0@9zO0h_T0tUh`?G^Fgsi20>OD%DOa`ys7ea+c*m&_$!-D18#R)X(%@~wHV;#JU zw>P(@;Hpa~lC0J1#KRG40!bB!f$4bIj`RpO300jy<<8#Sz_5znx+Y*Rmd{}EmAlBn z`5nIBR{o6sd8PFLiJhCXv7rePl=Z$$Lj^4GoIc)S`g5Omk(g_MHioCxJbop0qArcXQ&L7shym|!m1v62`nW}Dx0~sEj{dwCqXGvP_n>GcE(iX=J+oKo{?deE zbyj1z9KH><@YNKwf3~|=-Hn?J=wnfLxVa=Z@O3zkFzE2kufo}t#r-MR+wdEaz9uXT z)B?|^at9?EwFx^MH1MCNvHWruZ?#?JTx3$eSaUl)L}uTMO<(kz?KGBPP;g55#>vUq z*c3FoKjEPyjrc8V%YNzWhBFs!C=|Ff`}aAB;qQjGJ#kKJBXgH zy}SA;pTE7WJm-nIwE4Wc`~25PSWP`K!4Ha4!|Zsq`28H-I=yvwuP24kPnIt_+$RK=ysV8> z$biT2I%7%Lu@)-*4E}X>?!#DL?@&ZckH3^=^duY!GcWa7@4Avdr0$S8v6l%0^57v! z(o8>-yi`}a{-vZU`b6*Tj07iU3FETkb#7}8CrV5Bi=gaQnNfgQT%x=0&oe0~ ze%LjK`qx61n>BU1JlTJT_{hM+^j8|#W^HwEqv56j6eLzwFML~g{&P{HWQnDeo~Hwi z7?clg+zOryUb1TxyKRd^TIi*%N3ZUi1fL1C^z^`ckh0AVOK(G1m(@=fjt0h1A#U{?nA7!O@=;G6GyDvbGoXop%EB;VEuQq(gj+abaYw zh%l8K|K&zOC#l&~Tbk$8B9j`hk~G}DM#KQ-J42 zvSBrjHAJls@VDE$NfBTjvM5i=-GGr3l@fahdO}G%FI@Ok7m|%VNwYJ5I{=SxT>(2PIef4^vBS9&C}Ma+i%_W@@cI9eu=Q_ zgxnBr`Z2ok>_wNQ?JZ^pX}|M(;vd8ZjhRK7qOU zf0dJ@5ypIum5r@cWwjHQ^@>rk8xK$0I%F7AED7RpU$JF*d6 z8|R%1CPP9b;8)N>%U{3OuTP19#840x+S}VHSicOxUqwYl34Wtob!HdmSaWXX2Bn!T z{i3_aIbBm2N?Akgo6MB}VaESl*Zg4~H>kX-uA`Ba%l~lmg$0|Pk}9wR4FlRaXm__C zo)8d|yof02?w;(d0V`pxY3S?w)EHN<1MW|EU!U(^14@xI%-p#+I8K=ur<5k-W1(3; z83qf21)aDMH+8$``^L|gvuUZ_O#A8(`4M_Le&@Q^-pX=SlRm0LSWeha{coADgMQD= ze$S-uJHqHn9n+U;_ey?PGNvJYcdxNRO!P~4>pLIT^|@Uq-0Vi{mTe&5!j|vQ2jqyc7z}^pr;3TWNwX@I0Ykt+?;NSqo!LY-V z0~|1plMLaw<#821Vqzf0fR*7yVY14)5R}Hh+^UNJV8dWojE)pQWd5f9NP*O~C*kB0 zn6$7g?3mN4$0qPd#o#14I76hscrZqoRgx(%00TVhhMO848~sVh3y47RDZ`X2I1X0n zCNGFAKqODXpDrd8IT|DI*aNF&_UL6}@B;5_?eS*1Z#6qnkKN5?k_+tzk}>ez`(L0c zJTd_bQve(Qu2-5RiS;j#hdN&wO^W&tyaP;+G<4vJLpn+|iyY-|P!D;Ge-NYx%z_*h z7S>-Z^npxd*tLEA?O!MAZI18TT5!mSD-+!awKQgYdcL60^|^;D5teRdZd0Sz|hJY9)smc z=%}Dn92yP|%pr&p|G$6IY9!KysXz;VN8=Dv!i3==ut+!|CMyOSJT(CYJ0VJ_C?*Wq z7pxv$q!gZb6QT?RoKn&G+!6JFOgh}0p*W>GGATvrQsqtslfX81Q5wsl%8Qd+=KR3Z)%iL6{W7>c$@e zYSaoB@PSfJH4G4-7)~NpxCk120yMWYePqZd48#nV7GXighoOcg+YX%N7+wlWyOO8tA=pjOmw_ii=S7s>cPx;1OY^z#Ajus+}B0gayUn zG3kl?g?w`LP5F)o$dJKf&LHrIr&6M4TrDcdO2=3Iu#<2^S@ezX_gFtZC?n30T5-&J zxc(mAuS#uFlXDk%OacB9MpXwPqD&aBpm>TNLj+8Tz)IITk|lYvq#r6oI*2HLWeFyf zs!_AgAW<2K^iLTUCXYH=(!+Y@nsTRy2R4qN+hL2|I7@_r44GKT33k*t z9C*zyJ0S!BNM*@EvjE4BCLJ{`7B=3yP!bfGNf@LDzPQ0yOiPp^uuSCxlGCgO0Fal% ztfaaw8bmdV6a+&BKoVzyB*0M)9i}&=JOt62a7u&CkPKUSm%T-^tfb4@dTLs zV^THeAXmfz2KX?}jPz1Ckj(-rIlANyv{dn^FWdgAXb#S5>B8aMNMKz4pEI=(>c!Bs zA2)k+dVa1^``3X74M0PSjZ)@SLCinUQ|6{+7)TbQ2b9%+#zbaBUJ{K99~;vA=@<`) z<6xW5UZ7H`{EpT7T;@eA?l&S$7j=wC>{@V#T*}EpAsUxeSVNYB=1*Bi&V6;UQ;NDf z@U5LUksYhxQ)a>I!@q&n@Qsa}ic?$a7!pU3n^9qd2r^;4Rfmqs)&qr9b4rJ8t&`U4 zi?{ve?$ivZpp)a|%bcL=`?SV~{_dw^MXEy-d+21?(>|11NsEvC@f&v9_*R9zg^l&E z(I!dtv4AkJL*{yXhmfJIuEw}HgaeF_!XsA3b^wW0Md|9@!ASC$o zwtgRNFedrra?`7bYjjdVz%E!mWQYU!FZ4C`rHc&KKPuA^rc~B+U|=Bm8S26?s|a=a z2!<6!GHlA|Yt&|XHS+7GT@DzDcv=zb?Cb=pU}O!2t2w4Xe1$N{;QrJ*H8FbOt~=KU zx&)M7b|De}XX*y3(XlZ*i}mU8S2;Sdt?=V`aJ5hV;hT@wrv=G6T@owCd_`9XkTpp3P_ZWHX90PQtxYWstGT}S=vX@iR`svh2zJem zF;G9Fdri$nCjBntA<7k<6~%_n`~%2UU!R_O`-JX(#eUfp3CZX8y}Qmui;7-5?ZAR_ z_;&DmOD%lXbyE_*bpX^-KHBJh%v@h}s$0<1JYPHZyt^p_b9@P&!3S}1lQPi_z0O{| z8F||T;eYTqpF~7zz^^AaR_)8*Fa86~Dv}2t+p{$#Lk555!Emp{G-{}8+KmoDBHHoH z>^9ptdAVTv{a1Y-`YvQAhl;AZW9&}H(vh=TI?K9>%W9b@*u4hWH@&WpFQ@PWn0he` z8ya%01l=Ow-k)Z;f52_NUQ%zmx38x%eGPy|lvl1@I_=oM%;y(Gmu)iVSooYlNW>ao z9HgRXExe-9-3_MiqfP5NP4Av{{AO$nLP5l!WZ6c>G?0Yg(Y@`$UB{M50 z5nj3pPk#}8fs=!{3f6yu-zSy-0%kVRr?HLa|G(`aEqIRJ=>3#pG0yqbI<~2(jkf7& zulF2)exN*?q?pNUPWH0j%eo&yhE`mjbi;2Ws+HjVmtpj7YQX|LW&^Gwz#rY%Y#!f+ zU_jSP)5(58ICeqcM+f?!@SpB(a{aE@@0AP|$4|(I#)d|oG8KiMo<@>Cn3O}KzMNz5 z;{Lvnp4r{h3r2WJ?&uS5&%*QR`JJv7Sz#frRQds2f8zlt+&s!a`9n`XFI>`kiJ^Dy57$0 zWfE;xOh~m<=I7@(c2QB!w(h2193K-BeTFe+O`tvdhS$~UdH+Ts@~NNa;6Dqx^Xo)Ne?diPg=62azM(~gk`rKTukPly&1gb^kNjV&^zBK}}v*Y!>f_P=Zzg%-KLY)h~x=b=v zRt-*@S;hSN^@Dnq@VY)L-XX93SwV4=BDt9?UUr|ysj#fpbMnrngkxvX&TZY7rH4aT zR!rXZEPrh5v}mC9ygZe<#9Y2Xx|n-$k%N2Q{99r0OINvp!Dr(1u#=d!km9DNj5Rb( z6$8f^`94>ua6!mxJKHQt(9_M@%Vp0sG($hY!5!7wwKQ0SPNDq}P0Tp6@Y02j+-PJ( zk~56Sp`!`#JLGwq8~F+ldQADf+qt`saqJlGv^f^$7gyaWc}gsd{Cw6JEE+D;R+Z&n z+4$Pt&A`7-o8DD-*fs0u;cZMcWHaVKY+Y7eUfH-hqq$b!dcs- z*)&P?cYLm94&q+IakZc1QEg?X2TNBW0BGjl5;2x%;{0zLCnqNbRosAv#@%z}1i0GS zSyon69!K`dP5cxm^j)}8nkYi+)xL3J|Am;8|8>(S~M*2~MwIItvU^hACK z^JiyKt$~XR*VZi=2ZR9gnup%!*;#q3{}Uvkuj8O*BoVjWmVlf_YS2Nf>BV3Ko*}x8 zkdCsp*PWT>47ZCibSNid+s{)h)8IXsMP5!e8-IGiStQL~IW3wn#8u{{cIVt| z<%PKn@7)NQB09pGuY4Eh?QS%BUxbA1JWgwrEL@sz?gutHgL@@x%`9xnOGZcRXx%Vv zoU}61brj%;AW=}24;DH{-yK=%1iP|g(9u*6Op5}%iE@?HgxRI2+1YJ&Z3~aH-uB=K zs~&guEzwZFnp#>)k8S?|4`kcX#Dv{bKG*yFFA&8WxySr&r}KU9-JR1P_*>B@bL(l1e4a5UGrOxw>H{oi!i3~*xTA(VWV-x zQ;|y!t7z$+wA(nq7P~iRX>FsA-e5f?CEyR4dV=zm&uhF0pU1J!GlZI#!N$Tq@H9A= z&J2u{^4_vcq!nvpqNcHTF#=M_F%YaAahC?A({hdimgW_pQo%#p`%# z*D#>}DdqtkEW~C2>}t5XFf+3<=lPuyx2<+f>!j7j$J7Chpewi1Vv$s>6-&UO270+W zMk?YZ)WP#yTU~Y|Gpog}leDPk<#TFqxgj+Ki(oEG(@#ZXZ@A^)5P4oq^pY` zVpFihr6DzMUqZ?7p3w3fwEJ)lC!B4L=fO1Pxi~+D&VopR3M56SO!vRPzu%~&`EPsW zpW#dSxNfqwVh26*`8*H-TvS%LKU=1XV=T7%9(NG*#P-eMx-qTB_-LHi|iV& zgx3R8lK{Rfgd6s(55sqs@y9k>SQ2iG$j|*p`K0&@ zeC+$TgIQfE7W(KEPbL=ju0pfmzO8L+3E#7`JJ2TIs&if4u>uSjRob)EGdddZ`k?dr z(mV3t?w7n}KmUV<=G(>U>h;7x#A$!>go>sn_s!Y%php*|u(}P{ButLY`?U%lf`P{YojaLwsNf9|V6F!I6+tWije$-H0gO}-j z4-|Wq_C5UsSqN}^z`SXIl%$QO!f5frZyN#9gjuDw>ZZYO_j8l^9vh-*IXoc2!*noH zUM@UGL{A`G+92)01>qevpbL4|9y5Nhoi*#KzP6c)Qi?tU1kLxHX!!Z2n{CcXb+x4f1zmwH33iskc<2m4xkQg8- zBKVu=bE|CAi;sut+mFDwe1BB+3ZZB-*2emDE=7++m&0<85-HyEFovus1}0+ClgrD? zHc%ES;*&oT0xFK-$VD7{X~n*AAU(Iw%hR!<+=^~Ge?nZ-`lSvWGz7RP?e97`e|n+O z&u3>}@V?qu{Z9lX@r@}#s{H2JZwFqHB#E3fNfAn2BX{K{VzQ>SP^`h&$$%~L?5O4; zab=VnY?_9p687cxHmwX}l@2gI!JyUdN{yzqsTu3@VAcULzMtGh5D=1-To3d-^?j4D zbkx=sevBxy91EL}bv~TTcs*Opa$j6P2nQYx)R}JHyXZg-F7^rKE;#KCmn61=f53E| zMmfP_%FFd5jG+@7{}G_)=H^Dc)%7r895LR#QfkZyzjjbu&c*Tb#>mDK`d^9M3ABS8 zv7cn%6GaJ_{xQEDb>WQO?Vv!~3IHhVrsnun=D{L;9rT5UNY0lOq1X+VNl2X#Acg;a zuOEKDe(G2y3ND|(+(Lrca9$kzrr$v_b)3k*m1QG%GPAHn?WjbOhyjJ@H;H7j;!%wo z9rMR2S*yD0_dP993~$@FIiWZL9o0sl;ieV0eNy|woYa$V{2xz`U7rR<2EX*3$qRRi z019whbmF+HD;n-?ZJ39L56Tz(ppQI6pQqrSYx9Bp7&8V<#6NMsUL(PLNyxF05wF_{ zmMEe%B0|p#Mps%4iFU$F2;ELfRv zNE31_?2(Aq^slgQ%`1qAw(@_%;r^JBOx=Z-3<2W_6Q7NGQeiEAVtGo7$F2Ce4M*ht za4NsJu(11y;ZtyuGSspD$7+H|AFUFk_0vd;k&KM}@-EHx+oQ7%j;mGOagQCVpA{@I zManOYP6lkX^VXY!c!y=Im|`MxYilmgt%V57%y+8&K&re0HZ>rwp?bXG!)r}7Cr&alx8*2)mJ+GG^>s9<;|I{x$JS6rd z@Pb+o4^*W7x?EVTRDJm`MimpKSsl_BWE1z7S=^tGU2j%&D10vE4VBkd@0v{qhwH0h zV=k?5|p8O+*5`PscZFpXnhFzKQ_q z!DGz-`8*Y4C$OCFGvf6*J^Q;beolViPc8%oUEi2v9MbcmqE9lAL1{BR(T0J3ybH7+ z`Frn~c$Nh}tqS<2+7W_HOhfzr*{lQb(Gc`eAO zHf zp9XBy!tgVM=eYF1PEAD#Qx<1Zkr$>$hg7KP+i|Cq@ zd<&A|6By_!{=E4m=9zw5WMAv7d@0&qYjOO%SIw$|@%L~^f`$BuwCy@zCREdGK<+~U zJWuu-T%Je)@d0+&L4G0A5)$S<%co&!4w^P#dcM~FDf~aguu)ITu}z&5A1+A+_UZvDE10>VEn`dud`Ca%Q0bohMqP4T_c?XHZuWS%6~-8%Yox>6I~ zozpLSAAlEu0msK{Mo3$88!P^mha;P*gc1Z*lZ$1RqFRm{*R?OK{QjJVyZpGRn-bPo5bfbNvJoMh9d+tY7O1 zRJf_ACSCHR4v#K-gH@*}-@W>v0_55-zrKV^>v>er%xYJ(FnKj^{u#oCh}W)2*pzeFhV)5>6Ngc$Km``0*Hu%wsIUA) z*x1EYoB%y*MM8ehyEnlu+2FYId+H3DxJ7yVGHD&KAIj9Q1YUZoQDG=PCT5cG^--ih zPx=*)`&^bgKS{fH#}GWWolA9TZym>_OB*NwgY?H!+#G8J>XH*#e;_ql%cKkg^=d^< zIlt4t>Y`UDzv&)`$D1#Q_#EhkU!ZFB7^w=JTJWneeW4RQr!sl>_uD9g;1(7;p26fi z%TuEDOKq*|AH-X4Z*TqoSP)Pm(j>!U;+Zl2?(OXzrP5)B?H}wV7q*-ra->fpB{4l& zeJMbZc`SdT}7~C~`D>+=3d3EOdQ2-zMdK zM+ude-+g_%$%(C@V^M&VEVx${v~-dXNF4TQdJj9b9iJDQpH=BVYG%<$QiD?W8>-Oh zuzgfiU{cL-Xb#-Bi?r;oG7N3CE^`EY9BfZOWG+Blc3V15ii_7jUUEO+QE6>YR-3-rmv@l^K8*^@Mu{PV01}fzpfh+y~qY#1j=ee1v&z;?a!Jz9A{&qAR zSzni;GO#kMr0!GzH+$YzdP)jdZ@HLrhKto+FitT~jkUV5wjKzN!vuKtdEe}UR?tTb ziwNxghr_0FnL*%&c2(8idupRwoLi{xe$53%<2HBbEFTpG&9vHf@;V^TKg=-xh$Khp zPNvL{=R+!iJUy>`*}oJbxpG@lko7k3n2Lu7*7E$fUtC$~z2s5%>7t6QJzH1z{rdo^ z;6`U_ZF_5Nxu2S5ddXTwmhNxpC`t}VWTpaclAn-$F^TG`O6#j$=f}I^ki#ClH(_8w zr^?<8h$?YX;d<(o8`u7>+IDPV4~z^k4I6Fk^|(&%Zx>FYJ$x*@X$cAT#m7hg`m;Ea zTf=ERk|-$#G3qFCD$&MgwUCR(_&E3(3ZN@TFU%zr1E|QP&~$x;1Y^UYy~+qd(C5E0 z<t_@I0BkHh$CU`O;0G_zr-`jRC_Fr;#aY!w$I-BFZ_fxD z2z9Kfo{sB9=yz8$QbP1T@Vr7sXQ<>O=04pGyu!vtF4`T=@tV2-Jv=xK^{}yC%1({g zT9s!dCo}p*8v&aeYxg3vwS7AU!!ols$&Fmvw4shO)9j{XOK!14-I;3$VpLQURg+XX zLY1e+KW_Zr{2@QgdRh4K!jl8r9-}-m@mg!zI4(!x#DTelm!q*DRKNjG+5Y~1AdK64 z6rlzJ$Bn295wwQ3`U8C%#D2PIOM82JHsd!UUa@Fj2Y1gW{mpgw{3tmYQ=^QkN?Uif z`sm?QQ%Eah@T@x!xXg&8lqW>Zd0aX7c~|9MS8l%Lt{M^%(*sUtFPx%D89_@hV_sFz zw~(2!_XG`N`;ERZoykq=dD~d%+h}ByM8}Crbz?BWqdlo$Q49qcY=tl#jc5U4Bo*pqGD2;C*z_iOSZ`{W@&NRR_k#KGt7~dPT^}P@TgqCYn-G zE-a^RkA7?|_4rYGNeiZDWmT=|#;PhltcL{q%~7?UXlG{PCjDu=t4BXSpXz+IStr5= zy}GF4)6&yo6B4>Kr~eGL^sx1Lef#Bue@4)G@5TSBV=uWqzQyc&SLI$fBrX(veh=$& zopgCYwbW9JQc_uY(dALs^{RrM&ced-{q@UEs*V4|woFw8r`@H?5Evc_y|r@B?el({ z^*!k<(XcN*(C7-apDg2=}^!E5knYJnPetq!z^+_ip#F; zl4A{ZTNt|c?i3VaKywv5#4keLRgp8i?E5Kc z5JC`U%-keU(J|S!BN4j2~B$WEb=b+^s7v|=8&N{BM+gN1G>sW3rTom4VMrsUR7 zxCTgj8jPdFW9hpOq{hm5)e2}wfyZC(ikG*!tHg<+7=O!O(2O1GMaoZE9pU>S1 zS;(U?LbzX7f>Sg9n}^AMgP$H?nj;U`gO9G~*rF8y=fdCN(Qzmzj}gB8+yEAYM*u)p zx6N20FFvfj96aZFqZ1GiSiX8c{IVaJ94Jy65fqLy;lWqa(1`8#vyxL)DJB0l>5;+d zW^oo#?B-@{Z{o>nFk$mNU-DrVfmRSzq}qpC=c#%BCGwoXZ(ztKG z*Wvw&#s6AXo0|a^hMA?Vt#9OGVk%VAgLfA7=*54pHEF4{FJboU_ zvEyL}7l#IBHK9Q3?xvHJG+f<2d@|nG_dMkQ`IkgUPZ#gb(p+Fy{@F;eX&<-3FVFst zvN5GyL&)b`hL;g}Z>?&_p}8dshwn*>_M-Oj{e}=(H(q{0V+n4>gEMXMGkDGKi~Lno z)T6qZ+S4iE6!W_?Jvo4}#xrJP{`=+VkKm`#9V1aUZyo(mS02MAZcSuWB%ZU2lXe_3 zo3X$6=xZWreJ1=4W}CB}lWn}~_7W!rE*73k`Nh$Dh|n{&4!W1PnziKr|Ko;1I~H-* zNH+e=dfB$?y%FHf#|nRCAQ21DReLo*#EMn7*%eo#=KnReA<;jAK2}&zYJKiU{<35t>K~G-bG3?mc z5t@4GFT50|=ss{-(Xf2<0ev2Tm8lB=K^TTy=F3QXe@$(i>=?UiJ36nrK1R5oVYqoe za4bKCV)`S4Xl9T^%1db1mFv%Eh$@3%OgQmFP`-#_Gr`7=yHeph7Z#v1i?G_yzLSi| z+EkTr#_w)zZnV-Hb1_xE$OgD-olw{j*aeshqqd>8+YtcHTmqOdkzi3dC?$K5p!+KC zkmM9&83lc|KU7s=phCztwH0?}o_DuR{k8IY<3xP>IR_;zEx3`pRT6mvTRWb*3k>%l71F7i|E9QYCaVWTb;?j421Nqv*3IMR9%g6vKe<#w-cd;gY} z;QbCBg3cqydE3TRR8*iP#b&~Tp!ljrFn*@P664q@jhY4+-hXYl^tqb8*^%j)vDEKtvdev7^$Lw`CVu=nuDnYn^i;%$PMx_s@L; z@nUbXPAG63 zt?%(GZ^I(8&vSFSu!=FQrMcQnGo}YgQa@_X*a*XI_z0vtt4rGpW(@0UYw?kX-tM5| zD3}?^W$9@bek*fZO`6g6D@%&U`!k{qg&qekbg0v(9mGNEkmIDNyR>A;H?sz& zCxaI8i)&WTd&BJ!eW@XuMF4uG@ZW*M{s<|dJnZa6&BT{cKFuS;;e*H1b=%;=+`P%Q z3QiWL$W(FqtX6;#mtD5c#%G(wh*U2LDV`BhjzY;^c`t?oFDuh_Jb8c33k47fyNi}U!KGN5VX0-&dRaA%>T-Vg ziVy^-WN|1cBDspl211a90)e>#i75%Ec1wpQh0P&QO=*fU<879urROfDv}j_CIPj5F z@D($1&E_)lu<$_KiHL>?<$C0BV}Z-8i90X6Ck=EsUNlZ=s9PzPuGMAVsM`b{M+}0?-iT=i3te_2mde&#Y00B z!X&L7Y-+rG8v6<>5RJPjDA7?2>>aIKIxTz(CQQPHa^V^H5`9fgObR*cpHqHC`*Vbq z7Afzod?f@xP<=lv7r`j5NsGSx=!*)54SIdKn3$R%q2BR~??$oTyx0NP zi*E$nf@0?y;4p3L1@-z(wX?7Zqv6}YMD!@ui=`MWvh~RUe_Ld8 z^w){PUy$~X7qz<;va6uI0SR{2?T0BQMhIZ(OJ?@4k-15a<5o9B+JwnqE+D&^nT5HQ zc{aRh)6C~IVipo9rJ6~!0&&Dnt#?T=kc`}YaK}$*pi0Wh++N?)Xm9N~YVhH*ZS&F2 z#2aD+_b#@)x%91Agd*D5#Hg^KpyPh<_))G&d|H%>0NW+sw7}RtxQ~yYcOV~O?Z0#G z0@H*1<%qyztowb|=BWIgM4?!;W`67iPRApB$-%%oxZa=%5(w-OX!8cv7!N4H%2PIN z!+ui#S0U82i6G!?9kVw5qNZ)S=VQbJ6=zHgL0@D&?chSRuipsRo7;yGjlJN)uCM?v zNS}6cB0C(y5p5ST1)4tE)X2!V`L1cQzRT?ZDTgqr*zrPQN@i-*>EtMdfD)}gS%exP9=dR^WRVpdRh5Yjui&x-H_wLDT^*z*y1-dD9~EO&Q{7s zO-4pUW^#W{Uu`Mq>Ta&Br7Kf0>)-m@`=w)VYU?a)tW}dS@FNFBq$JAjz%t05EOyRV zD;-b}?_o~YKn|isi^hh9a2?uJ8zgzS_hZ2$RHt=w_U(N~fbIAKA0>e=1KSVS8D0=F znFAK($hVmei3LyjJYE0m-U{gKY-wrPdsNe!kwNbcy}nsDKBtRV)xF~OwBF)SKXNiO zilf%{1LJ3?Z~%UoO!}TCIo9%t>)>LjwX$qkkXSf090kS#@~1$jEyx~x2>ub70VG9U zzk?H-n?-^49|6Fap+(e>pWU&(YMRQmyB&l>%lcNb>RE;B!2AucI{BB1LgQ;+m}k6jfNXS!nA|%R7d;Yq;B#l2dvu>(4IaFI^e$K{kw! z%Os6CKa7WcV1J>PmzhXRN&<_AX~rjIn6L+lQE20=&f*WQ+NQgnex2vn5|@yk5)pxg z95Dhws^#|Qr0z=0*1Pf`6Qhc7U^hQaBs^|-z52XmeP99L`wiJ5M0=TPm_C_*BBLM} ztfVp&JDRX_01yN_6-4T|6%ilMP~ir_OS4aSLo^QRQ9VvN8Vu5s7+H-h1*+>y}+@$Q&ePRCO>#Se~>hg31r{h zAr;L&FW^ybay!yWnTMG5`O?s#vn_Tc0iZQZOiWTE+Dr!pzHG4xie$$?#>N;C`z14A z|4mZ_t-x4>o5Rt#v=mh+H%ZuasHhUS9!5Nzs&=I$r?5;D>q&Cp0)VADOvGZtD93ur z#y8#wK`dltH8mx#qeP_JC`;NuM8cr6mr&_~HuJwglO_>IJNZd43=*+hne`fxj7eC=b1V)$%W>K z(qUs`W2km8eZt&Li--HRx&$dcA9s@LJ)+jh&=D=R0t zKY(OF-TRlv?}a$HA_Wb%{vylY=ZT#e^+DLUwbRr3N!qhWtCiYqCYYWo7p_u1#1cRj z!ic-O4NDvA&+RC0DPq#XY9`Fkonk@w_!Lukg?IevxRTb?_?ZSFoLoLDxM^pIBaVPDEa|V9&r-SXfAv|%PAnp zt2rz@zWpvLF#P}@c2jhlFh>A>i9IHK=k;XEC$&5|N6L>^1VHrd;Us^rKE$8vzAjby zmAer6C9rtsCcH*A0+MyQk&Sg7CF((hLZ85Hc4mW;9RF|M=^`>I%%SV?Sz z=C9L~T~C!oy9f7?jLSd^3N9C`v6rI&mbkpUuFGb6iOzSEHfUJl{C-MdrOS>-9XXqa z58k##p0LxhjWg@Ejn}QW-hX`zmD<;aE8Mo3?rv6+&JX3*+Sm6ob+$qx^j zMIkzGXZf9M&hQkIxjWORU>qLh0SxwEZz^21s9XYHN}~8DVG4OjppfDK13VOUo*$yF z_jPe|b3@Dakk9?B2k6p8=MwFYe|g>C2oH3}D3t5d#m{~(XllX3RcGFe-aoRRow#T4(utWAuTA@h|Q)lF6O6UkAr3No_Gg}f~K6I~+ zr4f7;#Yqgd8M<2IRhgsfS1s9F{X?O6#(eyYtu{@TjfUp)WeZRAcb^o#i-<;7{qT@Z zvm}gzE)y}$%`bzd=ouc7p&8rsz8^J~EF8h7SYc*niB$#SR`ED`3I0J5$**e;f$MrZ znTh=9DNM}cm#JrpueBj14W+eQZG7bC$hyk|ZFmL}bx4)7Xd6|j0rDAns0JgkuuMyu zQ1$SwR~;PrU$(~khlejKor`Pyqho&LH!d$?qol^*EmlQ^PoZL?MU(x3EMq!|?uRDH zp(A6dVD(u|u=U7~x35ppn;NkzxXY(V2=XJ7K(=Q=Q%g?!^%BJrRCz~bfmV-Y6=fF} zV~eyZqS#*r@?`nRXW+&p8n7Lm|Fx)3q{RZ3n8evTK6{e(BW4>%G8H^jRpa5`$ls@l z!lg>SeM4U8Tu;r2)2u04I!(DRe{MW9LD1i5-(X_qgDWocPtOtHfMpvH7Zag&L{(Ci zL7jCbKmhC@J>NYoeLi=65*;6>D5kzXoO|u=)jS(liDa-P!0G=$kV*KhFcgS3i~)%w zUjU__FZGoM&4(B|=SwvJAVt!XO2sa#h~}&AOpNXh$KsNAx-z`vFYq;lETDQ5i|sl} z*Gp4@C*SjVB*QX;y)w9mLQsBE5S@&ytII`H!u55+MW8a87C>Bq8YM=}BXXP?^=nK-5lfeh`S;^`3jc8RPXVN|T(O}dZ zcO^I^f$AqZ<0n=4KM#;5VbpV|M4W8N; z-nr-OI7Qv~d6m1fgLdS%oA)2r>e8JFmh;6b@7uEx5L&YUF9uTquxM5i2+bT}DHlw4 zhA6T1=S^?F6U4SYHHH}G;OIE@xkc&A9E40zU}-Ad)0jTh`LyDB92fhWBzo&9M2Xu>Q8yXoUnig_j-tm)_ z!RIvApZIVwy4Tl`@~(eJmL#2VCXi%?MgyztFI%z69h44vH5g7_$gqqor6g%KRdruJ z7J0Kkx#GY)`A0+Rm5yRa7zE|at6Zd{;CkLX`H7q#_n)PiPM%R~YpbUxDB`*#i7)`{ zCau`Ke&Rx+U9=qYzkh(}Ra*hSw6vBKRGVrq!qDsZ@P0wULrgcDH*vGhIYm78>C2|( z3HN>9H?_oFJ;R-hKzf@E*;P%QF63o{Y==F2|4S5WM7$GobQ@FBH?Pc%NH{Su>Hjw* zKrg@edmIEaiQfRU#RDr~P{O)TMvj8AHZ9-1!~Jmg@W|r#rV?QPg|+7LV6@tE zQhFF5{3WN8TBOsQo#qd~XK8$V8DSVl5}ATh2r^X0zfkTEwglAf0L<#*Li==bNJUj} z?sRUgc3&PDX8qScUm%S4jqrk|9Umh-MvBZe7_p%N$UwhbisD9Mc;xA5pPy(yS!!=p zR5B$6fYg52M}Mjsg@IQUZh6ycf|AEaM~nb9Fbo+387_``k6|@AL@YQsSo7QO3o2TI zEM0wK90*8~lr`Kb95r8Y5}6qSIX^IUrYBqzPg-ff4j~0V|BtJaSm{qd3>bLT{Z$!; zP>`ACW(?X85{G2ArLnBJyKOI2EN^7z^4itP$SA+!r28S4@AC~j`y_#TDEAdre+*0n z&7ZfPNGWFr$7_3eszBrzp?N*%$|^rebL{h2`T464B1Cg7f((pu%*oAs)0KFt8_~iC z5&u3Sg|~1@;(=X-Z`wB!>{!KMzr(_|-Wzk%1vPnVB7}IwnxUmpEQ= zbmX|@dYM1n%$m>M?6ae-gM=g@N_qzg1}Y{aqwHAhG9we*BB{bYhe#GW(D0$e zqzVNg07?!h`ch)B%jU;znI$6U(R_z+HXGXT$Dh{vIRNmz9J7j5?KUSy{cJhh`cTxw zF?3ULZB z&)=%iNl3v-e&H;-!PTgGS}V?Q7PDxzV3 zs61<-L-k6Fz9*g$U^^B??)9i4)nmQjL-P615{`^c9_!u@4!iUD4WgPB6}4|4{1yKC zIwpO&;i31pm;w9C(0F+IIn84hFtabjDqYjwP6#MjLL*OIuzATKD-TyVo&+RCseJZa zPC2faXxKFrF57VSTNw*$$LjjN$KN?9-GP%5g1G%{h{e%`)DBirxQ7H;?~6b~E}7n6 zo*#R<5h8}i>Z{;<@&yvsZ2pTlG8!0+K4|)Bms@cyB5Uk(Ss@_-$tal|S4THr3Ffu* zR=>V~gdjWzwgrnN{dPk@rVwxG>?90V5FEMiV`7i9PcxYa&5_XZ4|C)tqqDZRIe>vJ z{l82n@DnT%k`?fxa%@vYfuL%f>kJb7`Q8t z{my^xWeUO{YS`*f<_(a6R?JRjK{)t>#3+Nt> zTGtza@^Go5rQO2^JzlJf9i(VVs3#_j8Fed_2%yTy^bb*Nwk~YA-j9d2X|I=+(Ntzs zly9VJX*rPOO+zY#0|L#jp#cFJy4qt)cRL#aP-KqXe+0OI2!V#pI07|{FVZ}0uzI9h zWMGTZrwdT!`+c2e6wy5s3n@bO-O9^~hQ(s%yI0gJ#L>V?g|X;#%d$EQ*mt$#^C~0h z<7T(FrHQJMzciz@T3V% zsilO7F9}+WR9Kj)qRuFokOl5&j+P6SOm(x!#J3whnHO z5%WlAoCsG2Jp%);*Upg)Ph)52ml+M&Wrn#6A$+k4X;L*EE!L(~l5lVuE?9`pn0q;K z^*rgXG3@KeaADIls~%e={DMT&bGZpN9rY%!^yC2gK5_H(N5>=>l-1q#bb5hpUgWzj zS~dxy$#yHS&(pyI%^XvotQA}?le6Q)71AMTw!%|UwysN(w1$Ux=Qsdn8e zYM$%+}d%PG`N} zEOtD$?kcfBx~W>vYYZN{b1clxzA8#fkGd2VV##H4xZ4dif^BZIBx;oNtIalh&<{_O z786yGiMZX39v9LseYR`!yLSM-{c>zgrk9y3=M3Q#v%I|DDjOOKtUwkNZnYOn9|R4T z?YGF;jp~DOr*Kgjytrv%iw)N7DKhOJgHs1+w~PEQ{j{lm4`<#vt)wq*y1E87I$A<8D1RmQ^}kJD3=a=~t7K(u?7&FNtg57Chz$|w z#sxI~?rMJRMy(wMLv-3qJ?^<}-__&B)I{JRhZS2YBe{bgKiOVmw8U)ZI=@`Dcp4)0%KEuIlpJfYyG7vI5<3X+x?GV zsPePFGe;+-k_-VY<+n~-*C0)~L&Y%RJSMEaswzsU*_|yFm%rqL{?bU1av_HI-ab(v z%MD$uw}D%|s;Uc;hp)EOp~bWSl3KBNO$!UKL$Z|0^RiGQ(v6_q*~R*#m(>dzRMqnJTO@epdh9=2cm_>ITuCs z8}60XR~TG#!u+9PH4xoLeEmkCm6{gnzoOnY;p`ywUCSR+r}`It!w1-uwM>?-abdTH zT6!xncDQdyex{@WY%Sb5w{*ccNUsi+L*x*BzH!ThgaUj#D;pL1Qi{f&m4RD*alx=V=!??k6fRP|6acaPlw-LPm`4NYDVLGB#Sqm4O|TK;hdR5 zhWWYftDynTEQn*Uw$?+x^3_kD1NV*%Dc@O;!)V6RhiZz04FTin6DE)6%SxJ>X&NT% zrg-V;5+*k~p8}AAzNZ%9l#5`viX$!C*T~)36W!5<&x@*fe`4SsQ){P0Ru-Lm#f)NKTlcha5u^z@b=aAhGUdScf(;SK;r?dk zNow-2KrF@=jB*%}wK0MIJP3+vjDzM*E8zZ?35!kR84iPza`ythlFFZ~tEoeVN(b$C zkB_S&fSERjFUFS%dZv!0b)f2sa|bSILW;}smtpQPm>o>!eZ|vn9h~Py-yO~}>u8Gi zh?S6-&jRq>PXNpO{Covn0OsmWqB9kAi#Wg#Sh-25feek6u%!l@F036QTg~z zTuqt_PP8-(UQHF&_U;z7`K^U&7SQLhnObW6c4ucbp`3OOl^KNK;9TO_Saj~RqC!3H z%>3Q(Q}DyQA&aIf6I+1-GcA^i5&KTfvdkuQU5 z)s-&8@A>&zJNTp#45gt+k_9~=v*~uGj_jO`|ChLe23NH9LUL38j?(QHj672>d|`Y! zlVV>rS+z>DuCs2OXj}#WQ0Zt&F~G}O0Tf=Dtt0d%{EYgScivT_X`L>_vDU+`@G!!-JR#B zg`t7l&zgGgb43pa7jFjx*oxAi#E1QYf`RAfl@%-qfEry@dExopjS#HgPYHr~^}yrs zj@H&v5u88YA8aLpQTkhB3?CmJIvrZ<_1&Dk&EHoF9s*mQM?yKe2lZ-&8rE;QnKooY zL`Em`t#;MRjYG#~?0-|329z(geP*Z0az`p@sd%6L`}l3)W&LxY)z!k{Wc*tA6`4p| zqLP5CSvOj4lAbZd%=|fVX67XI@B6FE@^S;EJnVPPaL8!rw#c%fom+8aqB;UFbZw=@ z@$jK^XT#&Y-(SJ1Iasf1JaQ8J_Hr%-kh{G(Z_*e6k5hi@+=*s#&CmO9vf9PAYMPFV zDp(9)?utfwMuz+fHr)r))XSnk_i_4BSq)JjxX13WmyX(D@gyobG^rA@V zj7ZsuAGEQF1#7haLeXIC4GeccfZ-RZk4XHXuGjTTKsR<|L=+z4)b_#Zdl@^M5k!4s z;>#}4X8W4c_Ff4K*f&iCe8=`quKg1=aZ#>OI3B>nU+f~D!H$zm6F7; z8Hj`!Tcn{;$f))*H-gFczk_`Qzb7#i;G0r&Hq~|({u(SYiOS-7emSkmQ*o@j25orL zu5Xd=O>L(kkPOC}-2mfsk z9GF@e$^08TA}jEmN%6`rGp>NP*D{7e0P#gbp2dv$p^Lx$o@QxbXaF}iqC>Gntijr#sLnS07%j$S9hC(x>YTk4Ex8*I&zEN;c&bqD94722%e~QSTe?l&Vyv?fGKj_(5o3v>h>qpjV*D$y}7;J6YM+)#_BUBGR1~onJmaG%{xB^Z+_}{^*u*bZ62Mq z9$G9f*lnLAGzRvMgLyLS%(%1IsQGOdYa8m%H{W#MQ(S7nVrzptw*cSH?WKSFwKFpZ zL4($smLYEXE(dvdi%O9uA&@BkX<8DVp0~4O=X6+;3oM*1cQZ$7NC051$-7hbrPMo^ z2A{LvP(GNu*ZWqHM&wge0VX7{upw}F@G;irPRGi`Hjt5qW11I%C*3B0J@~)s1twh= zHQBVUX=GjU*OyPD!7E)~Um6*1$|fTjESuWj`c!=i&#g=!*RMyGi6!O0wwjK@ zyAP>fBqw{d-gfb_OstcAsdr$L$z{$-$wg>%4BqOk;HN>s0_qT+9#}6#|&@{fp3;0%zRE_wI)l zc(hoN5qNkZ7AcVtBQPu$9zlwcCi)MWI>^~ma(VwH>@Zb>o|>DTYwq!Aq#9b>m(9!f z>i|kOX0%7sYvg1F>rWjkGaEzWqC`_nOiT#)<&O2~gpn0GEcqt3MH)(O4vwWySL}p~ zlMgP_HtVmZX=%9ly$uC{fqQ5OW3K35C?-mGT}+LQ_oH-P!Fk7npowyK)&x%D^t4Lb zT32Em2D_{SjM4Ok_M@yC5un5VMU1Er)Fv)HhL?(4)?`C@u=jQ+&TeaD1ta9V{&70_ z*OK4!yr?qn_^6eC?f$7w-llWT;Tk-$$YENeTSgf1j!k8Tn>6R(BqDCRaeQ&|c1%Ne zh^t|+$PdSA`seEe1i)Z^dRkds{r#kjzfh~N?9?V%CGF1&q^qX#K;-$V4ag^V1d=3r zgy*^TmBqL}Tj2~BQt?9(69iLpCA(gcy3NDlFr&h?M|P}vu5`K zI%{s}k$wk@3FEWoS!EsV*ORg;+>$NzW_|)(uP38iFv_u}#(Q-E*FY^f%*W|;H~gF# z11j1G#1zx)c3LNWUU4|J5Kxb8Qc9u5cO!XvdYn@ zVL=yO2|G}fey0=aHDOn7sOW=G(B=Qz)Uo}y!~5K9xILUwRaS-&I4yW!=z6=+bSI6| z1Ld6!oyZMYCun~iOZ1ZeKLN)JIP``ktEu>S{qcQ2f8#eUiTT;-xBva!*jP+L$wD%5 zF@8C8EOz_LmkNPOYu>K725rOcGcUSO6(6@-koh4FHr<_j-Cy)*Zik?d6ET)wSiW z&aY->XEn>gp)nCHIeYSCNfFM*$M4>~>ndW5oIZW}(4j-?OUcKN`o@L_9Hj^&05m*2 zYFXB~Ei(=B{S6BaIV1V`YJDpL0`3)Lzy1@Q82t3(_i|XK&=MXA zGq*nZ2mph35)`mdAcPRWUg0Jml)?V%d|fXPgaSdZ?`US~Rhl)L7o}%Rv<#SnB^Ze3 z@e`y*;hjVv^JNVZ03_%D(SffP@EKo2=)&OAE z2uv0Ng9a7=l8})!3Jy|L6yh7W-pz^*Uu9rb*7Kj)z86@(ofkHbV5wtKRxjZPC{5h<>*{T2!!&FqTBLKZq%xDbX{xX3; zehaNe5CmM`($vy$@y$Q{K9kBSj3^=MTN;`gFTVMQ-)B>41(K!poP1Dhs`q-VD##~e znW6<|000KzNkl88 zJN)wXpDr&iUcYfG$e(1up+l55-~t595CvhbYP^*B5Zrda1{`=kcVLo?0zfcALu*50 z!^JoM@ZYkjtiq^9%{8^RVN0~jg+$PB_*lk=xL;D&-hpk_uF1cpNP z3Mqpq#7Tc36z0x5z~t~*27n0IeqI*@w4OsH5P+>hzC#R_0dfAz&ykn%ivU;?VD-Tohb{rczczcn&6oD(G)WthSs zfIp8`OR2ho{^M^KN|1);8@EXJY zfi(yW%c6*qS%-CMUDz%2&QxA&!88C zE%J{+0FbF4Mqm+w3@NIC{K@|B=RF6Ln8M%lf^pe5B8Lsu41&S($0aVDZSaM~WFmk* z=HCjw=INx2`B4O*DeuIdF~nll0y#82BL*u#0=B>?RcO3=;ctk@MxQah#$jV|`1d#Y zjelE1M$pU*`I3tQAn$vXzfS;=`RY{#2{iJagT4usg(S|`EAsmb9k_IvMG48iMa;{2 zV$5pD2;s7g5K;~c{CNn4i5zCn$g3n8Tb{Aafi#E@#*Ae^$N)gX&#nLrA@uSTCok2D z7x$op`gWd1nqRvKGzc~XflU=H0#lQUnc1X*MehUzzCvaT$P!+P#=tJ4Kmvkb5=Mn& z$-)#d4~8h*0h26}^Z!Cbvjd(QjL574)3blO5{6FUx`4ww2A(H-9#4E|=Fva{2K5{Z z8k*#wCMS;>g+s8=h@c^Wiw4FZ&#z$R8TXlul~iJAR`3ia`0KJT43vU-`unrtbpat9 zLtx1e`hqd#dKoJKOn4#G10vLfG;;lgf zNaKwdJ|O5-0I-5QYL2o{rFO9o~oFxdaU!juq^%^o&zLA(JdN89Na`-k;h6L_o7YUU6QqP3X$JfFu1pNAqt0seRx&Wr+uNuXo8S{glbaDp#CcfAN;8S=X^ zB24%@0D1B#%qt9_5mEay^=S?vB0-2^V}yduMvq{!pZdqp>JV=GXpaB@njsRgjvqNN z5gSQm91sy;5s~8fkwcT?BbjuL!PJ6%4goVa>p^H$zy}L%OqmhAH6k#x0>ltTLHHaY zVu<4TuqMjWIv-aNL4?+2h?a3HHrX-+fcC&zcsXaLib$T=#{dkW)#b%yWXO}djocx8 zUi_0Hq~yB82PR`fnY6=T5ujyR$B!I{jSr_XIR=Y7&-8D?79%Hkchvr8gVXI@NO z2nPQJ1`H73S)TR0Q6N;M(BGm+{wy1eV)Db!G(dib!ru?5lLCo_gfZU}J)->FBZI(_ zmDX^(v-^$Le{t-{;lU?QnUz8HZ{oipJjruO@FXselOmw^T4)E1Sh7LpK@4UK!Fep? zbkBo(h*A19EyPcmC4ymZ=D+#100@M`hYl7M6{XhJWpVUkV^h|4TE-q}=F0sxJc zr?dDroS=ndAuR!nFC8$0(c^p{i)Z_9cy1&mPqybkLi74>$iEyC-~q{`UKD}y9+u^p z2>}SCw8V*{M<&NdGHHh~!XPlO^+VXwYDU1&D0$rk37H82gR0J0Ad3Q%;Ji)1-$$dw z6&g>O&2!JF%8a;SLp%*M@3bfA+3)I Y0RxdB-g!FDPyhe`07*qoM6N<$f}9M)8UO$Q literal 0 HcmV?d00001 diff --git a/docs/image/ClickHouse/gluten-debug-idea-config.png b/docs/image/ClickHouse/gluten-debug-idea-config.png new file mode 100644 index 0000000000000000000000000000000000000000..112b0d3fbdae02375fa44f4d0f3403ca0f4b6e75 GIT binary patch literal 226892 zcmY&=WmH^SllDP_Cb$Q8cL~-3g1b8ecL?qtf(3VX0t6?GHVp*V;10pHA$ahQd*_>Z zXMXKoz0Rs#Rl9cSb54}1vJ55~2^s(Zm~ygG>HvV83jpvPDDW@;B<20~ep%q$)MX@q znsKrN0H6fqq~2?K=NvZTQH5BF`TZA>e`)zpwwUjoIJdo7M^$CH;w2d?p#-_uaI*0HQSy@d&xX7c$Pp4S z>N^h*hWzsS%ciJmy?`Q662$WVIr?Ye!EaJOwU>Z?v**o{qXIxuj|PSR!!mCcQ_+?0 z%TpIkUEd$C`2NEKk5-eh%0EI8s2^}Qh5qyNzlpx-{foT3nYBea)6X=hAL6yq^BI_#no6>B zI$2qa_{dm#cmJw*a(~9J=s)cJOEq>R<V1r0nNHKl00aeQT++;&qO zFV+CN(i~K2~9e@ zKdnH^MlmP`8L#q}VdLZD(q`>RU-mQpw{5&oMqJQ5hYB)ikHa^l_=g`>737_^Cxm!T1CSDWtRV#;W{7v%u(s`uf^epFACBhd2S&ZlWuehjKBws-niyt zj=(c?EAW+Xq{vd8Q!m8ao-{v&{q*IOHNT17?H8xO))w-3+_>BFi)YL=2g4%=(SQ!) z)6q;Gje5X9Uq9t_UHOaIbJFlch&MnCX2V8Rrojbov=mwI+WN_os zu>n*0Gg5L7_c$XF8n|!uY~1hw+|T*CnldiCV1^(1|0wgqXe0Y8jrc7)|9S23;y&@T z*U+ouu!DBHrK{w0dC_Een`YjAiN-o4517M+kA z#E6H690r5IoYDtJM$l!<;@3o9yp+$JoX+;5_~}xD-v8Wa#Ykl>PPmC!3Am#f_FjjwibP*=I8YpC|reMCY3=>~6ay3DE}=c1V%%T8kc&za%{u>`pZ zo^rpeyMm;6r}ByyTgY;zsxa>xxDS=K&CZ6*tFZSeb;f*suy4qx5m3!pk!O6oXgXqW zj_02(e>b(nVq`3z9@Brat5Na%(TZQlKEv*6a#oIJt_$(v%3}#txjfbVAim#UOxiQR z6+^tfWaChiwf@+EBCM2Lq@|C`?XXGA!L*9JQ7X&flYFX}mKC0-wR(i>b>I9i$=yli zPV<8|uOlR=q|L%j8F>YX-~s9^E*=#oK3-nhaPO(dnHS@g+b7mC5Yf7i`S(U-oxWDl zN4fPBcC{mqvK~4)w)N}tb|@cRr*~Pd*@GU4iO|ZWFT0OH+(h*Rp=V87TgAap@`2!TuBEH_ zA(~51G!}+O)c7A40iY|}`M(&q8*w}7&31!@J0ch*0U87uv%K<}J3j^znG-F2e39PW+k7 z9x-^HG+t)vX^A7#e8R5GyPvzE=Bye%ig9+hv8?LFU{%thU{`pwjR;CQU%7v{fhVCK zXeKzzP5G`>^*@;Xu$!$!z2~JTA6q!8SUK9}-ekB{h_TDLgGP9k_iN@Y{N(aj3}_3; zF!s?zLjg@{{Av2fhg~SToUTFCxUT5sr~XDN{@k=ZygrZj&l2{zo+|0(6I~YY9*qc3 zmV-|JaPU2J2|paD!1NYeZnIr0rO!&d^i>~Do`L%#SS^BtjZ9$>GyNzgMr%doJaVu@%HQt`udt4JsCqG|DAk|6LRi>pSY4X&g_k^C2?h?xSzUo(SFJ zk@-=zbn&gWBxkP4!%QX-bx&OgXY5=cM!s>$BLNq+*?IUJdm3x%gC42OL$0*gVxrO8 z+>bfx0?W#6I1z5S3<^`IBxlK!WTAgrQC~n8mpVD*;=Y!mESqsP$b1D(H*^) z+8tUV(Gr6w9h0H$3dMC@1 za>ny0mYZw^WTa*4EN#7=cD-AkYkBpiLWZ_0Eo8t`A-O-<#qcGK%~C6o$;zIa7~l3= z+eYre9l|zm{#uXJ8NmZ{M>zb_8Gym-$s;W#Z9SiNX&ih;u`7rlS3?nr4d9hBG5U;t zEoP?hx}#5zUZiR<0kve2z_O_6OV^yWYTU^W`9Z(O@&=2p5SrE4eoUoSZik|FqS^Rx916g)l>W7UqA@8w#nqpm7FK3myt(OSnvuOyrc}!i zd2vdu;$5?pH0FTggFTO}MVN}as(|uiG0~}{^<%8JP`%DXn6`v^1F^6SZkkH&Tv>dK z8XOiSH*RPs7d#^mO&mF2NJG2W9g?Md`|Z9~$;i=pTL*jXNRYs=xN?8zx0hsF<=i+5 z1*4b`-rQsr8zuPd&^WfXMV4U+5S&2iVcu{x(>XR2e7Xgu20oVT{GREBa(OC(zmm${ zU(BdkX3_RDi4NLO(Gvg4gtd@;!vF(#^&h`T<@^?L*8RYJXyJ~-A|eBXe;S)szj|$; z0|!hCzMH<3c{e^yye#0mPih(<9vdl`Bt^RCYUM&-o+RzC-T{5V)EO9PjVHy z##Szv;Q*2Ni_8u|u3~ln;HWH(Zw9+!=oWD@s}muBOvaB&wpziJj{^FgsNH1y#E#39 zNKtU@A${o20pvIbRL*l|LbhEpFEiyueavlaY#PA-y4WAVoW4O>{g4bzv--wl9u$D! zJCJGtXX{&Z*)IL)mGh3_WCD*bU>XTK6Pv6m5UBuh=^eRL=v2d4l+@=Ev_dd5^a zg3KvcV}|3sZz_pGo6C@hdX5!50FMqVTr>X=_q3oewJLC*UE{<-f=5poRbT985BD2$ zV_D^P-J8Qi7a8JeFq7g{CtEchpX+U^?=6lTQCx zvHb0`-jt{rtRdXZ$dNV6ZxP+;d(zuQObTT|@w28TM z?$~R?cK0DgG+;}i1cVW-N8kMPNvh&9FTMr2n-$a4qJk)sUL+%DunQIfj}wm)dr_d5 zL0{p6iU@zPHz9x#rdz75qfacSh-&-e#}CZc5&hRv@7x^oDV7^)X1s4jrv`+h2(Nz* znQDL?GVI2-DhFYF$%bE|Gt?`)g8da zsNPZqoeVCYZEd4qj|?l%uBp2zHO_&*yu^oLl!;=G?$nR6tZO0qf2sqKLjc%EiGCC?#x)Wjwy-hp<@WGRNvqRJRph6J9 zOhre<0w7Z(e=fm_B7Tp}+$UYxvtiiHNb9pZnzOOtqN3CO*`la!5aK&VU)6l_G*(4G z3hHGgmQ92MkP(=pN)l25RE*df?BQNi*B|j&=nT|D2;pS%OahL4#6?7OCxW{nj4MuO z;|3gWFs>;viM;rdhvwo3T#KiN+iDoB-h1VVgFYU0 z`uWOR2d`c4 z#oUS6)Rg*BJc!VIFRHz)c6J^P#Dj2Ozt-Q^{*7}I`@BW*1iseqTZ~Rd1;?izBTyMQ z!=s^+H45$-DsQ-q^$w5Enl2v-Yj1V|%uMSgek~$k)oP4hMnfmBjWW!mxtbwN3xsS0 zh>?KPTARKWdYqAgzE2v;pv@}{Nei4%WB;?hwzm7XMxwodP4^${9CMn8$fQK#G9rI@|2S#oyD&!^1(v zLSY4|QYINd<>UI~-c|+E(bRAC{YE@Tps_XmmXs-gk!r=o;1DB%jD*aCOI<|0hfBrA z$H|F>ff?M$>UcY0EKW~f$i645 z?OxXpJK?1$DGf*tcxYHe*m#UYWXEqlOLQ@YAN3KMe6MPspPdcsmEhX?Lj==4&Q#)A zekuM)zV4FtbH;kBSG*eX>Q{U2-~7}}^k{ZsgG+?3E^T!^5p-vce|WbV^dOyu1_Df{ zoRgDxZP7yD@at5AQ3m6z*%3Gy&B$!q5wGQP5dox7Y(xdwVQcqo%^{EU*e+7Gu=Y0PUA@{VqzIY}a?Zo12?2q3^Abv4%w0Kts|pSBM}~3)HTN`_y@3XQ&&teX1letsGzlbtogz(ZPLb^&p-1$@eNkPy!3!qBZ^->62?tP?c1|$y zl`cE#eo{cWyjuZ+BgJY3aH2BgeT~*KGJ)-Km0E#My2a%=750}+$g0JYv6eFxGZ}7` zXM8rXU8bC^)?=B^*Q6Yew_D=jO17agMTH;~aJYK3Y!TH0HF#n@xSNt5H3A;X0b3R^ zCYTll041QvGb&I4Ep$ZzC{m`Vs$ehXIvqHicY!IKu3rMXWM!Dkr_>@NI4*w78ymf2 zIa2nR?E|w@^^SG~kHDI}4DT~3TZr&NN+Uxi=ZN71_vUfr-XZ@XkQ<_<^~S@3L#b2E zjFcU3Nzx9HrcJ_zF9C?cC5~YZ@7y+B$U0&X{@2PO980*GJKTA-iArjR0fnvV}BeMAM!-1&lshJQ%aTEvBw!}b# zj4x5~gTrDY6F!-kKOP8<28=M#YemWTV63vEbDXYJ|%h@3aVM%FH8^`EhP#PU{RC}A|`tkEZB+{jJ^pLEs%p> zNPNViEkvX=?UqHM5BX34$kxB4DYONyh@{*Z-IxU24C}tA-KB$ciFiQK2Zws3{Gg}L z5&%CCmHXF;AQZ=eiPjVWi?gERBciDpbrhmaYl&2fr!?l?caoHp!;1+hpbC!Y6AQrm zj#ctITmtY^z(d0T=(?op5Ltj`B-%yo?T~JIpm(yoi?Z&x{ME}4rcU)*!x9}f-J7D@ z?G(kBn9$uupk;sAPo-+=dtS1)_&X>$ zyF|otp$s(B_dmUi%~?Dj{&c$t<_Ob(dT3zKqK^}ALvS4A$oSaN0bSG&5%B6=;gq^0 zIRJouY^{tj1$=fWwZcp)KfuAXz%r#~L_`k`#*C6kL{GuHHjs1GVx-mV0+ZmS!cbWj z;3=cV$+$uv=aSc6$ssqmermv{!z)VpvuOPan|MJsz#av!PN?De=jnS1IxMQ!v}%HD zEvE2|ih=JVV?W%w*`2O;dQ|Dl+73wFJzRvLVIWULW}2nQ0#Y`p+Sh1mG~b_zf)O(H zLULGnoQRPNU0Cr&$mdst8WnAQRy&_ULb?loO_$IA({vx(1|JPwypgg1Bdk|;c6NeJ zj^DQ5kO{fU*I9-bA<8mQF4TSe88dN;X7QJLvXFG1g+)0pQV-3rCU?1+u+}IqcgGw@ ziOuoGXtKMBZ0GS0M z`qA<6!{r4+SG;fkzAh3sKB1Gop!0PI^uby~67rcGxxqsPBhNz40{N#4wfo7|ViW$kIm#+F18_>ZBQ^#@99f&Tha`%cp<&af)wUc~6K55fzJo@k zF&J{!E^MbtyLx!&WyaNc5*HmE9no(Emve9U>}uV5v$69a zDQ8a=mG#zW+#gDgXpwN6ld>}Q_IN93m=<)Cx*hIUPddh8S34%B>rm$wo9Cx_2-*E{ zBf=+_RL4dAI$t@@wD`n#@0Qr}?knr-^$Oq9Mn?|M~Vl#w}lf-b}p7B{t-2^S>t?sE2X)gFsI{GyOp{^|aKIUnCmNJY1p7*g%WU$4ZU z-WxfNE!MJUMd~fKzNEuzIx^z+EkUPWy*lRwh{RkkSpanuOG`@w`4P;ntuJqktJ)Gt z%qd6a=cfa9FNex@wGqNe4w&Kfi(lp#yjitxcZXap72QW7pQ)KVTSaR&zuI%;ii>)U zfB)KN$>%Xv<`~^hHafy$CM4IfTqVELL?dia_`0*jb!T|}`Obt| zZQ7;v`D9PL3VJrX*oco~JC?b>Je9rl+WO;Vd3opGrjs=#iDho| zV@+IP&*O#G075u_pWnZK*w2f3KR=#4g-CvhDa@LDQ5%Tc%tc1H$CQxRA8FZ<>R(_p=MHKCIu6jUnD4Zkt|R`}X+nH5|7`CO_m z_3(;RpyTgDjRRDG1f7A-dZ-6#K0h8htAMZ=2%21-{j~dcqbOGy-kOb0} z=EW29-;Y`5yZMZjJtD7w#y>tde0<;}?t5utuJMYxdb!T({u!)oH{86-70Z>|56Oe)oJ++AtejbaIgq(U8uZXgBYPJ#zuFm9y3TXy6-A;lD+%8 zSUQs0(^a&Ybe>d1yr{Qo^wwHA|9eG6B5Uf#YMZ6PS;Xwi=u0+>K{-#z?e-n2%I$7X zQBP>j_tBFYu!hVV0e%5JhryEfT@jes*RbVu7$45$@3Bgj0hPRHJ@`S$$H&9mW$;*5 zBJNi~m7yrAxX?&2Y^BXA4vviLHXEpSMYBjmH_Lt9NDrXHP} z#|P8p>-Upy+U~=?yv~!IUcv>4BkFzJKFM{L5r_OEuK7RW{$Lc2u;R8;)z&5eT2_$( z&S2ZDu5C|L>hpwbJ*!vTl8Jw44z9|~y!c`J@wREc@Clj846qD49;0^@r-*^vP|noz z26Frw?y;%Aa>XX>FTvna?p{gZvq$5^<^r$an)oljKRv0Z&?{dZYn7cVJ0#^9AqjeI zST$RI(6ucsNjB8G?<99e5|XP^V-!y_Zqmd!Gk>vzN|p z)ZuoN7B`+o>ZPC93vSp<+U%KTP`s@7=NddK;x9RJTeu8BByk5JibvHBhG!YwXx|%IF zJC8w6tI$_r-NA3}e1(^?4Ce6D)>}0Qm1spWdFEhy9L2Xzy2BjP(Lwp{OTobrzT0Zr_4_AMV6ZjWe>c`rnp9o^S8P z`-_IbTN;eP-Pb=YKPxG*HJ{8ltam~hA(q|er5DzFf#+uwic!O`%x0I57Ik~0_2_9V zez3MH=gz04^87&k`58>gG>-B3FOvCW;t|nd-BeN12fLw4*>m}|g3I&5b%hoe7qy)i z+? zq)jEU5RNx{xU(5Ou5{|GN2okkbrvobefPQ89YFSFj(f{%Bs4VnNOdx_3^}@O8?IGe zYq$K%Ex{Em5(~^BDADz*(aEz4W}f`s88z-)x$+D*+K>;~-{ zsm<%6muJrQ_V%P%1_Jo0+ZI;6U)mb2wyk!yw8#XfBy~Tq~0ic5GpjCNR)hq zvBAu|{BTaX{tT-P-13|};&6a^!iKHAjv(YgtT^t7x@oC8EVV}8WedtVa)4m2B)9ui zdjZ7~A9R8*uinMK{a$CtBalu2y}e_B-Zcj~lDej+I{JKEju;_iKFoRt=>*BlQx@3Y z&ovZnhVl$J-nKrZHV6I5hdy~Vhd7@s2bP`>hb()DIZLzRqy$H+2~|RDlU<3CRn2)y zu@E@pF*9sPj~1h7ux5V!QssqX$ulr9S+?oJWp-pj11Gszhf83i)-*IsyJ zTWj@S?blhq>n;?3+BJ5RghxjAY1JIh+)#k-}Y%GKsd(t?rvOXZf|!7a@`X*Y@aR;)i}NSTbx*6 zDNeQwO>-G4LMomc4SBwsU%D#`ce(qs7skNq>V?^lyT7b$46gNGy^Ws6d|Sw8pYL}u zHvfEHF1}uIB)c1_(G?lFJgjhhdw6ib#Jt}2TF;$qb6o!$Eaf^F=Hm za;gt1_~w4`)V%gs8}yhS5(0;A1ZK)tw|gn}gcEGVt2)<03T+UBxjXVQeRzQ25Q**r zU!1cmhagw=GB|mvUL;uwb!4DaBBR`TK~5S0$%WTGiRhT|6ekmQg9_BaK6jsqT3OM5 z%%+GRBG~@xM-%L_vWu!-GcUW<5BCq!%NkD#&lKCu(fp#Kk3S`M* z?d#|F>b?zM((R@zdMAc(Jln`;){GK_O}1}q`|Z>V(}x@0+0n%7qCa&!vi(Q3`D5iR zIstoS<-z$q4>~%(v7pzkv{tFvv9RaEY0&jWU(D`$W#xm(^HC5>$p9S%&r8=b9p*Oc z$*-Ut8^MNRu#X5Y)i-2_87{~j9`T6Eh$vT3l&ET=l8)F6pT17jwslj-%bfvTh5O)vGv+DVPz?bbFz zI)(1_mHX^!E#_40Z`JMc^>txuWn%+LEP#N+_~C^K75X+JMuOK*N6ta>gj$5`hPrKk zuJ%m)KV5%ci6K_O6|KO->HIda>5F3R8!h28E3#P5cBkS-)1ArT*&e*Ez8^yJ2=hvyX`uf-~AtcEl; zYJ?`(bk%v#auOLsn@u0H4u=)e*>jqI87;3-3})&Xwp5f>Fd+kgWLQA;R}ZRDQk00m z>&F1M-CYl95u3=D;2T$?jM_sT)`d^DFBT^)5YO&6!J{~6&?!Sy;v=wj>5+K7qZ>3H>%WN~;oWF*J@!oII0kdEfbob9>QT+fXHrQG&i?;q%p zj#xPIg}uN%@blH+&f^?n1NrFeL%Sv6_APpj9D&;AVz;#wy{@^@MIUl#Jnfi?Ahe=$ zs)1GN4efe`S2r_rbNUV|w#N@=Z=lVkO)V7_aLFOF z!M~Cd>py91^fj8SbOiEY@JL98**{%Ze?TX6$DCmxl}9F4oLgL9*Dx`$ub(82ez+VE z*W(Ndq@9sO9CNIORXYOVFTG%xa>UQVlamvoznlAN`zJ6M#Xiiet7KcG{QQv5U>X1T z)UL9MxF@8u`?q-DpQ#m}3txlc&wbfd?D=9|hZcApI1r4J*x0VJuSXm?qK+dI5SjR= z)rsfjq6ehzPI77+XIq$!{TD91LGn24TduM+m9$K*t6tFS*Ox&)`}-^$;?F^k>nzTz z9iOfaXHW9r%m^G%(-I3+RdkyBoT8la#j8q8C3kk*_9k-qx;>h!#W`u3n@U;=3#s74 zBf|o6zj}C?Ct%}do*XY8p`%UYKNhu}+hw?_H_k1ZSR23tAZKSUXXje@;__mjv2&pp zpO91g{l?c{B64nS?s|?NY;EUVhZs9pSqo(e7)xHQi zS60+hIf9?w`3|@HxGZh1Bj@o$slmx?fZagCM=Ru?M>m9cPKskX&-La(&yD$e9EJ}4 zr%vqpCO)?-i__(JJPEROq#dn;SfprI?jF)Z&V?+cs6UE#Xp0hSYS@S17wEz-a%7Y< zH^v56H0w52bjquoH022bBnzLf8}r%6O|TA6>wl%@FmVYA?w@&gkn5}{N>O8j;DcRQ zvr-4oTfTaWkYgaj*-Ai+Jw10j1MXkHeMc50MQ?~zJeO>a#vsbN%G~y8vB$up*__;b zG2-iu^Gx9m!G435Uxh#2^-XRfei1Bm+Y0wRoF9s85xjZ{PA_Hm2rX@rX-pT+Abqh! zQruoka>1(vycQ$!8-vs1iG1PvKl#rqss`4+5FsIH0^Ht7|2P*mK%0L}_!yv@{_E6LS&1xd=Eke4C%e`rG0O9fw4VK#y64A4wdkJ#sD4?sl)C4H5bTREanzhLXa164;!tj4?$*uy&GA~R-yVDW zGqSpLgi7IRa6p1`;Wyi?Z8tw`R_HumN+i;kBCnIV>Q&u^6-|>4Pamz>(n1?vGp6); z^q1x?0f$k2v4y0uu`$`IarfKq{+T?bA#}c+PUFRp{9?(E(+!4V&>qU*LpysRkIMwu zmyxOH-p$Ytv_;Cf?RWk592nrW>Q*&W({Cw@?-&v3E%oVnn>?%a>!-F{v&)!th4z|a z30nOZTZzS!0kw~&dSfusRGoJQ25<$nRmC+mC!gon zYoK1mFQ#wcM;O*nOw3G)%OoJ2aDZ*Zh6xmaGSr*E?hbmQBU_&^g*d3Abx~C(U3!~X zo9EZ3f32w+-1Ly1&a76o`M~T9WEcQ|=;n0I{#&)dOLHj#^Jx9~liJR#y)_Fv4rlkb z5)ZSS9j*sPwL+eLQDHCG3xk1T?3#g46HG#db{*&Qf5*l8L~{*FY%O^`noh4$qf^u5 z*Iod1JVD)|lK3Y24!Wopb6MLGi_P=sZo$}+Wfvv+r%gVs)CuD z{XpDo;|rn`Bse#@2C*2L%GR3K;mC4pJl|Z}L3gzp`zj;Ik>CTV88Q>-Vr(adFHmK4 zAlmk9?@dS=Cc}`ytTidjg_`#iVnXeOE9!r}ZsV5UkSjl8BvV7uNi%z*niw;<^EQ zyQwjIFGrB6#q5d6%?J;YO!Dh%u=E4tjpm8+AhQcp)N~`elcA)rnNUi)Gm@nb@7y56 zpJ3#dw^rce7l5Q2x^^apMn*^d&ptyqAb*PBH9_npIT$))xpM|TV6`0=#i?FS#t`?y z;!-VUY3`&sw|IFwxsm}wJuk2Bp=`yQzSy}r%$Mb(Uhv-upa6^$V7ozHM?#qKDIcWhkQJ-HQ^z9i$e+t!TZ4_1{W zh#KvR#mK)YCCm%*IaL71B)w_ zNjpW*Lw)sLd(B~3g?6#KfF4gkvOI=Vw2lz&cDabO#3@+ z(nq2|Xnt^%Mtg|_FCGmVf`b~y^joZBCt*KKZdoRk9I@G%1v7(dIFJJ)eK#NhGH9dK zW`sLt`0a_xL-L{YrDzwZ@o`(O5j93QU8}VEGr)@P(oXCmy*8SDb9(x%;+@}}X80<{ z$?Ab(?;ykbU{~+(XrdJLDIa%4K%07Qb(I*Y7L`53f#Qa0bEVa7A<3t;@!V=W+WnfQ zT*sWEs4=CC=5d+ek*4qcP~~Rt&~870a0*gGv~+}`?OQ=_@28&S(hu2dkKOlL>Ek&< ze%-w0UtW`U@C9kxp7nbL{as6C(NWHAjRwL+&gSwle70n#ZQ|diCVMIBM}?XAt3g~m zZ72IrAOAq?b=Y$&TP*hIlFK~y)*cOR?BvnrPuDuDTWcyH#ac|36Z9d}fHEnE;4|OC zYl?9xIa@7xy-#z|n{(~wkm<5$K79EO&b?)DV4t;bMUb9lD8(!aPi zKyDSZ?~o<1;@v$jbsab-7pm7&l7P?2`st-4U@Ub-Stoc})#JZwZ~$kNg*x93ZRhWj zT9>6v7x^g{?#d-%SsAh`q4_k+w?cs>r6+l)bJJ>-h|h5yn^5`0YwD%W z6|mGRMnU4oh|I~?4QVOr(nAyVs)A{CPY(y>=of^}Y;3nf>kr3Ex6!x^%Dv=Damu^J zq%Xr`2APz4mGJ?cU`7CBn#t+B*6DX~k~>utN_cZ(*Reh`w}fIkVblQTwRM|OENw;t zuD_LX$CC+98T!rB3IIe7OO5@(u{9kXE6r;QCMIsDy@@p6wyQW6H+dC!o`%ZvclJ0I zw6*V&p|`g`E*MmD1gRFs5k-~5(#)4Cl2j-ww)?o&zfZ<1V|ZzuCvM4*9Fo9FhA{BZ z7|!AKz`Fs*vXg3zKN-hXBO||nVllv%C!j9nKXRori8L~#cG>C;zW$7iiQa-nTo%^W zej?;n*Db5W#A3^D=1*_hMUgqZMe<%^M#!&U;l2W>|9WgH@@_t9(G(SAJr=JhRQV~J*-xfNg;+^}G0?{sT#Kb^)# zM=IdZ;j*L6OD36u#tKhJ$q)9(jD}x)ikp)Xv_f*V!YzQ0GKF)=`WY%=L8++w`}{8w zHH$QVad)R~561Q`ruG^D^x4En0W~SI)h~l6k}-i1gb6Xn4+{&i#02dSC!LMThoikX zO(W=-sVrV(o!9BLRO z{8FLd*8d&NIVKg&<@%!l3ZduWc`ft0n|CM1sFlr3h~_;=mDYUh>g)65OsDm7+FQ~& z!xk4RGdZTxfjy5h1WhNWxPcFZ<9RPt>ep6lM4C_HGz_!4<_x@eXfj*7J$>jW>$O%^ zQakEa?sXh#k@2bdsV;>nN?)(8*xt5-(@xe}u7_Krb$bzgW#5ww?4KToOb@=9PKDYA znIdbNVle_5!4k+7-#UM=!BI0LOs<*f;}~{gNYSP_$zot$BIE9^d|)|QbH=AM>d`Zr z^oGYEu{;*SBjYwd)8lEVynDCxTSFG?%oo}#z36*mq$r*;Tc|E@u@C$&2v4*6g`N+$ zKW=E-9|9OF8`bEYUlPA|)L(J_!};HV;Sc&*Q(mw#0E5^&3@MlYt! z{mJ@B2d7cl66KML23oLV?B;$ah^5erz~g%FTf4{>3WR8EP zWkJX=blOyZD7O29p?iPb(^J#DRMJA+Hr<+iy36< zNID@{9*yVZH9LETqpZr~4`mflo670`GOfzsqhXj@(=aeV2P7hI*8C<+-AoHkpYN}n z?`YD}{_?$a;sGgXWpt*l;K7^I)mA?*y`hLyj8M2&uU;iu9;7u?Rvuz`^Kf%(rUWB~ zN~ofRBr@8X*Cqv0D1R;}k}Mo16#%EDev;thpGm~(j2*2c^@13mzB&zpHa0$Xg@oBP zogY%EqLhG)I}9`Pp?|BJp+Uvx@*4K()LhQjtQJUv1Xa3*<9WhCR>lSG<+b3Nk1u0J z%N+v)gC&TN4>8kr*?2d8X zKN`o!zQCK)7T2fmSE99G)n7l!6(Y`fMPP-5EX1+F9AHx4vAT9HCMHa>DAUbC2J6EQ zG4+<4ogqQ(ZS{qnqIf|s@d+IbZQP_28Nf(^+6+Z}B^8#m8;VYiAsiKV-%qfJGG&xD zy+_PJ{jN*vf3%xBl|*e>xf|iTL0!vHb$9(38yj1yaNh2BM|*pIZbiqc!8|@Ztlwgx zXX5;ZdLl=N$0;+H{$2lhAqV5tlg6$-~C;$kstrNy`wYw>klHv&r3?A*!At@)0*djMjE$qZ!Il% z*s5V`IeJcTKmk#%H{D=b>i!|AA=ux;&#k=ayZ5Mrtv$r!LV0MHT3J`=>(?Gmf3fvy zVXxGgMdyFv?3g?b5&g<(Q$y%)vRk=|bLG(bLXps28~wjj)_AyM=>9eX=quZd+D~uy zthcRRTBRS%qCVd$?2UF#PEX^0)qCkCX9~1eRyFQzrXDSOIee8K1JA|n9PzW~3EYo} zzXhj+lt_dk} zuYoeHqbyR=P1ev)Clvcy4ykF*6>Y|Ek!#ma_ZYcx3M9;;N+MKd;Q;E5Utf>r^Ej5x z>a~^^c3kfz!@94><0_A44Tcpo&Ku|R)jTeYJ*}?KEn}A3+u6>?^F#$cW|k$z_pLD! zr#rZ7;R?-_1c}*?cQhnnvMFJ97iJ6`Eo-WWM+*xtE72{tBWu6tz;L~$Bz9oytCEsV zji&RmF?oz$#q3od1(Y{AL@y&8Mi%wmY~%OW%3(-$j3P@Qv@kb(Adhd_4D`7G`PF@D zctb;jy%t}Z1$14UlgKO_P`6ToeU(??uxf@ESYzq{5S#^P^IR_|YcTl9>F-;rx$VWSf(5Kb21HM&XJ z4}xZz(X323x1Sq{3U6KWWd(gjLuYPS7a4<%*CsFj8w-&0S1~NP=p6#{hF*ii> z(FIl3#6(f^d+?ZPUwlr56+N;cT_6M|w0f`UH9s9Mlk` zMyZBhro6BVjpdk>OqOxu$o}sQJxVVw)`HsiMJfXUdmxy8&|f&!;eh|ZxC#xQt1lEG z=;G33HeP?Mi@eMMYOF=JsK>lJNS3iFLvI1@3TIX$R$~sw=RtZxtYm zLH1h;l`KRgGBN6Coxe7y-%lOcGcX`w&w-MCtte7DLXS~++4uSB@#Px`RUiGl z>)dpx>fp^bBz}?#unolyA{fC2;+xRx@JM=Dx3FDz>RoQGJSoHj{qJ{+ejK5*M(EOd zmpiEzy&vxI-l$snzIuNhG!ZSsZrnMyGAScZpFo=gk7UC{kHiSV8SD^JSby43`F~Vh z1yG#LvRy1eLU4Cq+}$O(ySux)2MBJ#-Q5Z91Shz=FAl*S-v0O9d+WWet=g)s-LJlx zucxP{`<&D0sDY#@8(m!s9=I%BLp!t@a!g^*pKJ9y^z~IeJ4zT(VZ`&7>43nok<1NF zY(DSX$Bhggz?3ACZYZD^d{REpT&G!W?Jg^mOrObEU0w4y;p>hL?oUkjYWuT3w{U+@ zzt;zS2r|b&PoA6{|AcXRe!}IougFG-(tnV(TnJr4n_8=tqNc0(i++4*ZNu$#o5~GQ z4%0gvM_7uDjwMmT8T#nj)YOg+n9j-ypqW`)@V_3(m*fMDkFofsueqO%jf}*bcQ$sN zPg?}opPrast8TU)Tz!UzmlgX<9hL0Ty=)S*!3J4Em5P} zy}Y{Ud9gbfo(|ls>n+Tssx0&EZZ_UNpTO^*Pklm=^PImVL5GV~Wo2btK09OYZ!a>OScF`T+m$z|`aTB> z4ZFdN-H*Hdp;453DT%Wi9V{Ikt6$&m;vP;i(Axm$BahW56$eqEjrKLedK5f-TN|rn zt?BjEp;3EeU?1j+{?~mGL+BJspeuytC^$b=vAf!Iii4v?k=b&#dXDt*0T$LsB^XXV zeCpz4*`hTN!cc^9uoQwIhY5tXM=D7DN}f3RGe%GxBlCt9BCvhbewxkp-d0sV2G zD(hHC)Fu^+bj~{xLXFLXpcleMB?O`e(j?V-Q|@iPkz9ABPO0V_E~Az|{`$(Bz{tu% z%0k0>jkjbv#PE=XhkZiSxZHI49ju}7n1eCi?6B4C@}Lt#e;(&|DwRp#_fYKzMni_M zBnW2>E{1Pks8z0&$P#q=9xt3T`a18YebXN7zFFKXCF=Be;;&S&3)rjP;4Q2V!tmJY z@|{TXWpFtgDV6w27=A+?4^SlxFI4STGLj@TT4uV-Nj@!YR!|ukylCw4x;jXRRvSBC z>+0Ccr$-WaBFMyB>$u!NbMU*pb}(3Jy~yEBsjWj{DHQ!IfH$HtQc(pDA3CQ<(%K8e zUtRC!@3>y+OQu!dY;0y{A6d|4k@Y)|DQj1f+U#qQ*<_p4xTu=k61)BS^+d^# zcZBnX-l6~7tHP9Ct7XD!+hNxbH9?%1hAL z=b&W`vw1gN z(QarvJL-)v8HSi=XWQst8gt=(Ry_L$` zF&lNmitXirzl|MD^4Nhhz;YvOVh<686pR$`1f^R5R+{bBjx5c0N!TgY9%8tUV_Y&y zEf}@oCl{t(FFQN}1lYTtrw9ovJF3W!ZBLJp7_+@Doq3jY@xie50={56_6@hCL_u%M z151OckLTe-)|8J&hxY;(@4b$-+yINMZlzzK{voRFH>wuK3?Ju@ScA9grQdf5@7J{- z*20x<2aGvBmx>Q>4-T6R*C{s^lW$W2YlKc49v>iu(iZm1$K#vfSik$+_qdO)`g;Et z1%KDG5u+U@NCZfWvMlUB`b#&Sd_6LnW}cp{`E3U$@+zdCA{Mv;5zrEVbTCDUi_8Eo zXM#9#vjvFb`Lf31u2mmm8p_ zfVRfrR(Hy&zK|3Vn1%%a>80~L?Qv79B*}t7p<>CummYv!OP*Ke z>)M0OR|KFIF5ao-%!)`JV^fn>sPqz3K}Y=xhH9#s)KtzxLD32U;9GHgTD%Tp(t*LqJ08GftVv%#Ob)iaXqVN-;b<%ZZXlA5t9Den*&`u=sG3*(v zb&#kAn63|u`QismtpnuQ6-&XTmO08}I8uvTX< zXQH>im8X>?1A7}%2)XtBvmv>~NIS1h+awJ$9SJ;qaoh55{Tc8BAXJb8P2{-T9?hIa z6R|K4PRXikf%RXOLZi}UKTT3pF9w{RoS6SP;<`R2p&q!EoIKE^Abrpe3#Nk!0a9oI zyzCX1f31~x1FUeM;t#wbSV;?05}*`48#}@~|HKR30GyLP#V<04#l&M8t80?VKn#>5 zK*Se{gC8PEc%FQO6j{8GRyh5a?L-~zLj>@>?o(G=pX$Pup>dFq&hjAufPKn&xVR?o zj|w?$>q|GgQH%}Cp0Pnm6niDLLFRCzuyov_Oq?bW(c)s1NMr*}MP>~E)imy!{6Gv! z5lOM2{Vyb_6k#(X6L=W53C;Fi7##Codc7(+>}B&nyt zLQCc?TJFa(<_3V)n{>;VgK-;>Pfzl;Z3yG#+sKqTXNOqQi@YVoNc(Gib`v=EMoK>; ze20Kp7&U@3W@DiNkV*R#i`79B*qu91s2NE zNNOPmW%GV5hfXwU0MJ54xuzBLyk7jYn9T8fTF^@f+s*Ra6LRGK6oDHxtxT*)X)_$V7wH6*;!(=yy3rfOLPF!kFP`WOeu;rnRq@VFXO$;ogr>C#DqXji2QhU~8~ zO;=FG@t#YN{l-!xVDy*0*K{ty}0|&oWF)~n(#WOgAYmxwA5Mn2U4dsl3 zK}%IQn(N8ds``&xjuO^A2OvZWgJW{&t9hgb3Y>NsV7&=;?1VHTZ`&SUD0{G-zUb5# zAR9Dv7lc+Ct-Yi#Bi>h4PzTNFhKTBSK0hj!Ev4Jb3n}DOgi7q*sE(=x;`+nlvVK@F zpw#KCuTDwg-dfzM0p&reSdkBjy0-07?C>ITe2;FG?aZL@$6gZYiW&dDJO=qax0hj8O^~n20Stvn-C#< zyf%6X(ssyPxTCI1Yd0xI?4hmG`1+p}MG_8t;;VMD;r!|OjDkRjMP+W06WM-cfY6EY zIG|2Cnk!@H?g&Yw9&&zRf!$^i|0@({O=1v=R=zBGMhuuRtYORrp;-31@p0GUxZ@|b z@nxNT|MFw!gCoO*QwHdmt&6WG6788XS*5L8OW$e>RgYxdz08U-IE)n#BdCnp!y=4= zaRKYHBM-+6KdOO9T8JXL#|#BzrPjO#2;~Dr@D+U2zKE$M$AktYW~ll|9A;_F@vQ0y zE_-|`5G&3I)MUr-@%+g`N<|1^%N={DiOpfQ$EfvL*#0@`d`SjWw!Ahhrf2ge~gM|_Ea}|X&7!ghuUMXG;zA6-TX@L_iIyhn` z%~*U>yD~x|^CoLd3pEvW86K;%e6N)1^Hp#}RF_^aBgls@hmm^JV(Fp#p$lHG7V;BS zKX#HFVuCMHTN`e)x>sbY!Jgzu#KC}ZM&qfCTqrbJ4`o!7q<(Q7?3p=~eP7ExGXTDJ zL8SjSI2wk$csS7({SpW(%6->tg zv9e#U=CQa3b{RRN&QLw@Kg&m9v9Ym_@bE>9I9Ld!I)3&C8(RfvsZ}02L})1*>6Du` z?k2G((kAup>A{Sii=m@Ee1M1Ab2ye>*ufxuL!>;gk(g#>c~z$Tfn~k>%7m@TJUF{+ zw1CO2_D-tnq+Vb@Iz*#(rh&fDVwzpv`QvA*BdVI(juZM}LpH|X(BVel;84&%ul#{n zr-C>N@r+Y<(Z?S>9}Z2%r**6xE_Y!ng}daiX2tnHl$5yQNFUWXN@g#GN{mUWo;S0B}6Qj{U3C zx#q)|r)M98DRZazbCNy#we00j(0I8onGG`4ZwfW+sJiktr}wI6l~Vc8UOR-tVq(Aa z9bbPss$&`cx9n_!s0&T^qme@kZ&n%mkJWx06&3ee9E>j7bqZufvp+nl8z!HW+RS*~8`}?d>}prx?*A}Y zzQ3f7K<0G{oYxwn_$42$@mcq3>10u_d;zV=xO1=4XYYBlbB}r)ZNp}^5~uiI&Qh1Q zNnMl*eJAHg2qEmt&e2OR`_M};te=HSqsWyGu z9k?v<*%9NnmxEZT1i4e2=_!Rl zX~U0RKCXY~1l1d3bdKf>?>(_7gSx$w8KX`Oez#tr>Wa0krf<^{yS2^K-b8W+1{;Xl z4b>VQ;G4XXd3C}`%^)mbstZ$nxOdA_+TR7hqRGzIEUC}`YaKIY(0WI_{b);w0qfYg zn_2V8l}b7dDbL7~R{Zj&CFcASevb4NManxE>-;*gBrow7_5|d!CLAME_XWUwm4s8rMnsVyP9tI z&M1HZ)Ej~sQCtUniF9rF{EKC3@uZ5R13UCaaOwv!1ht=qPhvAc3+(d8mtASnFivuN zDU|J_pEtnn$^bRe#%Zac+tp43n(4i>`tPYJD6}3^RJGpu{MGuWK=W0zB?$;u?U?K5 zN-Z4HM1RGU=Tb!*K}Ri3qrM-vRE?wWF;lnk26~dE1|n9P)ZOU78;-*b#u`_-UDB#2 z#vx^f3x*%>OcT8S#L^sQ&w0yR^Va>E!Rk3pH>_6mu8nVS+|9m16)~v}EIgn5O0e@< zz0;w;+Ez0fr}HZpd9sGWVhlXHLK5TM5^{jK%;we?ABk6MYK+-nqh@rgjrL?44x9Zldt6oN z!i-%>U>$liW!_#*X$dMXev}VKkAcu4XB!o_Jw6M7`eZNNgGyRxoG)!9Q$q6-R>V{a zCZl;ECDaHi^K&|Waq{wkrqKjS!uXv^rTqV%KJjXpiR@#X18KdZA=A$Aitn`+C}g{3 z)ew3EF*n>1l7v|{`Ce!oZ`_37U8)%4;d&6?p;}L-;v#&O-@HXHL?-&Lz>JX$@}6Km zusgZ=Jc5;t^pHXQXLo1@#kfXq2`{zs5k5x|QF6ZTI2oq6p}|zm3{+&EK)17MUcs3X zao80*T3hn*n4lW846_=gN5N6D5|R}w88soivb0VH8<3v=o-xhS+u9#W^jHr$3|EmGnDck zYlDeVq1uY0M=Hr}wqojZ738cDW$jhK!L)3rr%^h=(C_2$N@}FJ*h}EuEaHhppj$VX zj1-F{cJIeuw`v-!#oOCYwO5NYcBsKK?9`=bQ-T!^w?#|-R5eUg?>@iNM`imKMNQ?l2V@g1+$9Q+e(2{I)PksACOVk^InO1JLG8%ugQ-HCc#?b=;0EQnd+ z7^l%Nt*p@`%RCP`&#nuhRXcdFsBt?-|7J-JI@G|qE_RNlc-XQ;UG!y$GTg~kZfNgt z-k`aEeUC~*yHQJfU+nzs-GuP=uWcHfr}*b+1OAvw{ec(1_^{f*Fa+{o_s5a2cGZ6_ z^HXPec02Rotv7h{)R#B2!BWFftI4d4wfg5y|MS;>t}wqFwsK1GI~Man)s`3~?^axo zGvZ<4;s5{WHHb;dr)vy2&)zt%_CvdBS!f1kHB^?dbRAUV&R-KB5hfE#S@^#2w=y?r zPY7h!HRIQmYw_GScw!$iCDvcP;J;8bi2KPWz{! z61NA}UmQ#YCv!A6Za`|uUUfL>s(#~`$POObq-AB~AfYiD@)Y7A5O%`L=&?tYu_|_( zkjodP_+p{nXy7y-Ml`aYGSYu1;C~G$##q;<3m6Gz4>f~%z;dC0lg90^6U=>nV=nhG$mp>T-A@;|N@t6%1c3ea)M+VZ zIw5ZVW z(Cvn&LDznl5~`A+!|?S$RVo6>_jbo2;AWf-5)iZ+E+W;0K$3T~7q=PkK1h&d-|aE? zW^<|K{jy~5cUs-@=?94+C05-O)gf5U?Eemb-Eb(T_VBiKW@cvV6~qY^PJY-5=viZQ zrASp)2AuSdCi9**v>UUx@s631D#fSs`?$N9eBWm&mCwe;oj3qyBj8=?w_T)55M^x5 zNOJ^RTXIE`lEU{hB}?BuAjN8BaC`LqjN|9>lZG0G&j=n^`4lFM9A>oDE6LGc)#gIv zw>vXQEf+MAF7k{(v+_s{E>^sb{9SIPuRP*~k=ZPRVM>`G;`8Riqy8+c=f}gK)HF>3 zW2HcSuL<6B{_j9MYrfO&|F#1cHTDxE!2Cln^STKx_!J^Ag}lFjvq}jLI!N5e6wwe} zto`A~$D+9>`Ikmn=(VM_Mu$tC%u5}Q!-kH_*1g!DrBi2HJs;qTih5H`{&9{PtqOWq z%fm$K=DG#U9Yj@c!%CaQ6mN$ISbU*0v&8phbQn`sYT_bm(am~9FV!$9Mr{>bE!gbx z!8=uvLp%nPuYVQVjh*4lz4&;G!I&m4fOXMF``d10T`YJAaBqnOp0z%v^Axkum{S=I zx{aR*yrgLzL63)mj}LZ^s8on;mITVz}Yy8t5&K^3>Kj3U);)GSkNR?O{> z+mI=q_Jf&oTYF%__rGgC?zxX=M|$Ha{&rUdD=q8Y+n{f`XFhu^wcRW8i?i$V$YzrP zx9@9d;i&q3NbD)%tP93O{L#_p|JeXv&Z(iU9wW&~^7W3qmj#O33H=chI0Keh+yp zNIPEheNLVBekKSkX9k&8GI3a~c?rjA>v!7(ZRYWjW9zQ6<E>O!vyQCS-O~KAeTB-ZtI@DupGd0d}ZPsjVwL~OcH?1e+*AVh| zTXWaH#b6kx{RWT8{W=D%`q}bJ1bGvevab=kF}00wuG_sL0N^f83}wDO`Jw6DhgakxFbI^&bl;evNUnQZluOe zx9YdOaYy#GgUB-87L(if@mrnp0 z)6t}o$D*piDUvKcuaoJ7DGmDcHs6C2q^~*01Y-^G-|Y?BU+)KHNZ}!%Gd}+I?qKP! zO@(8lh=u^nPK$WVDcmRs zx~GI!p2tg3*@{k>C{kVo(h~C1Sx&9y;C+S2m_N~r@6CQ4p1e8g{H_3;{`r>^JEZ-C zac!=uV%OJq^5^yzOVH=5BSULFk-drluc(=+X_12|Q|eTCJSuj`{r#h9!;|m*-1i?? zbXiM_v;8z=-~EQ;{O%WPI3=8kC(ph2k`vyeXfL!^XE>6B*L(jWqv##++^vE4yY2-K zxlEoyGw-zVFS!Cs8*_q1O84Mw%lzC>=#9DAj~9j<-n+{4+A8f1bMT(Bc59U`3wlj= z;eR7TDK>u4L{%oy1oY<q<;W2_5W`f^f%b7?;OsN5-fI{*=bj1!b_wD~AzWQ7H_sk%$33OFqWjK1S6DKkPO7fiL!4r~-U z=@dby7IrN%I}sqSb)zI`(=SgC4`NnK)UZK9Xsc{WFA8?&py~X)%MF>A30!0;a3!_3 zo#5Q8CSEL%Gw2*Xc0x07c^yM8KiQAr`R640&m+lUWc+3S8#Ic~iPIQ7RuFM=vtngp zVvx(>CN5xLK}Lef;Z=Xw_rmfte(ViVPMb1Nh6`j<2n@ymS8h<5Q0m3S$-qI~AvL|B zC_IJ`Bv3{_oUDkRSu%PxNept*Svw*I6cfohMke`fM_M*qIgx@`;SSPVEW|gjU>pAj z{8wVdQ@R((30m2dQy9rp?8{p$LEyyN% zL=!!vP^b{%10$`rveV=mmK^I6I2gsIvLN*>o9c!(T0)m>emG8Ti4M?>Z0c*Vb; zdrmk(SUg|;6Tswb5&h|eK^T6aKkjfvyfMLw2u~)5^(lB=krWsoPsrj=8qZdZE}2aU z2au*pYe-4KK`S#E79p?ySU*G)3XE4`3RisOfq@;vgl4hFR8WL0SNRelBvL_)g%Lql zkIKFFNrO@pc5I+vUqF9-2{$Pe7T*2qZ|RFn@rcmqmras*XX1g;OAJWjMIl&4(g#80 z1T?e=0A*g7N($I4p~CXobQxqzd&2uM$YIfC6$?c&%usvz=d86tFi5K~P_Si=7n|=b za(r(iB#3l7yeAB|4jlNOB$3l@^bVUIv)mPUC50(7MwpnCx*D{BBg#TzR;ZZu=-|WL z`$6u96mr4onHRnHC1#B#T|7n>QyeXZ=XX`Y;)Wd4rm$;hnE_}fW~-f*i>iIe2slOI z_|oUK%htE_ADsByb%J>1iNd@5IwihsEkyObux(W*V_>)dapeU)%)r+Pu}}t~LE6mh zE55c1xp!6HzLYx!ug`-Xb&r>o=}4;CyhiYk6$VmV(B(7uWX|UP7aLk~2t`{4{X}UWNwHlT0|hxA%*8HcK16 z<6yeP&WPbwzUUAAtv0GmjTUqfCA!9X32HIwo|ai&GuuO`OYBva5+$mzYA0PRm6~rk;Ynfa}pT zp2NlXLA6>SO)UfbLxoE|HW*rhLgFfJt!Rc(Azv~;K4v$!x>T# z4cKZ;l9%`H#5MC${|NTMDsR0_db~{HueR+aSFr#uWZk6eX@29JrJfvDh!sq72-X{9 zxA6mjFBt6q4y>j$qNN6xWFoy6WM5#fFkwp50WrCVKKz-Hv`hPY%JDr z+>q!HP`IDwO5E@_K88kwdf*X^F1uSW=#rG{0@DX9b5Ml8p}>j&Ag%$nSsp+Mco}lo zXmMy8HmZRX5s1%;Mh$>GTe(1D3g|FIwkY6gi$g9l!mPZmNo#uqDHH59Y?2|Hatt;% zsdAB~SEpvsPx7{^roi=bJ`>$IAWrc!ugdse|kQHGmWW;x|Fj=q3x!pWU z>v}x-thU9JEROAjvl}hmXIgSkE_u4%LhatZr-PLm7D2A^BX<~V4#d&nY$I*m-%mlk-HvbTG^`{R8mOvFYNLxCYmB*G(x@F z^<(T>>9wT4ov&P@lKQb@H0d!g7qg1V(ZjA)!^a^IL!gvOZN;a7Li(`Ns_ReW`QlAlEwdQT-_|XZky4xiLRbXVK7=)IOU6cJ?7fKBaqU_fBs?hv^zjfh)uww67&5^Fc4#c~PW zZYP%X&+%{%RQE)Y_+3jYiMjQW8sPUj zDzWA7@X3ln?pTjhS?|!>g z=~IcjyNh|dK?->ML&SeK`Tnr=wC(6;GRMiW(XrqBZQ%K7hRmV!S^DEI_Kz4O!Pk+* z&_=vdIfj}W8!|Go?Y|k-qt?q6pN|nUISsn!@A?}B36rNQ+UD2D45M8AOnyUUUg*3w z=^!56OvQG-k2!o67l$!&IZgU}x#fB@!5JrIZCm%F%%B%VR#fGNlVkIt=^G~$m*@6g zHn0BZ0T{nEuL;k^l*w6j+Tr?sIAzz^<}2og<>QT`9*&2L%VEFi3(j1yn+ghEAIC0v z+tNsZn5$R*;U6qVvPYDPCFE{38OHlAYEoobrQopMdUZc&U?X{RWVqG$ zTS9-c{s-53tNj8$Z3^e2lY+vOLcnWdQ2q{{vgp6vOx)qakHi{{D+tz{%kzO|q{)ma z5u2Qxq?SmhIlb}=+q_x-YMN$`(r-Hb8O$R#ny{Fp$PmaCk-9ltC4AQj>Ji?P zSmmzvF}|S-ZucqRG}a+PzQ|BwN=k`VcJ^g=r%HoYnqn_;ECBI@(Dx`|()(pD;N=~b z-EOn<>S~zC*L}_9cNnkvG&j%fg4v z0@s@nOVBnIQNZnJpX3DS&W7+Al;Q$5eZH#x_v@#_I6|IyVb)M5b$G4k{|Yw*##*d8)hGj{ihV0lyG=-Cn8VK0O4yQ`wz( zzFtWz#eO+kZIp;kXARh-B076H>gvj}roIm#>?^seBzheJ1-u`sm?ZlDof!tPrEIC} zQ_Hroe4M|05M@{hygwG&guOhRucRvhYo@2rmR~d`amKdd;8BZ$d-U$}~az%{6V1_t@PjD`vskNu) z%GAl+%kJr$6FgoEclpaB+sxTTj%t_Hnk9qFV}XEU<;#e<$=Xk(MXH7Q4<5=w-?TosOMJVM9+3i?RE9=)#KiQ2r83^y9cIY0Ks{ z@TostFYn9G@0(%f3@ij1gYNH-mGr$C^kNILWN6R6@&7kOijWr6Ej`j}lrGj4MaEyc zRVc!ak#9=+Qt#sbS%0%gxV`fyk4Q&V3PX6J??8&exHHJ~K4*ysVZ4x%Bhy z?l_QnT2Rbdl9`1i5p8XGoh^eiKR;;42*?XF-Ndh#TTxm2agqB`tTd`ZK-_g0>y##L zid{g{W=mOHW@8O0WZX9t{O$L1dr7>jXWnOI2&8DxwR#C#dQM|%Tn<;`c5`ijXOYiU zH75c8tCK(}y_=ic9pcv8O^<0kx5^X^_=n)LNXUh4%oF0z&(CUK zlJaH8zSLA!$}A1CPdV%9n$uK2;ii|C|R*ilQu0O(m4u}L{q{?Zw3~wC zF`34gWvk+bTC$M1qbvZ4Zt)=+^F&cZeWQfP4e`jZAAW8@jo9k92w8ATbP$0*U ziYDx5;nt*yM5DhO_W zMOI_UM@^fz+}DRqpJ|k>9e~2LaizZgtWoeeHA=4( z8bVT}Uc~WfUaAmm1v05W>~%WNw)5xOYzJthjpL`H5hX+>=;s&^bqytDO!P*0SonZq zM1IdtWC{X)YjQ3;B^(z#zVkGP&QH5)bTH7&)@1M~OcAF6&Tl>5cPCh5FM>ZudkgBz zivtY7wM?g7TX(vjqx*)pk zE2#!P%AfQO#<5jD?^Y~#7es%9N&MUx+$6pm9(fL~PbjxqmJ>qL^NNRurimeryj2Vk zn`4L${%NZ})tEU8ZZ!>7dI~z}#*JXhr(#J;cpXTNun{1HS9hp71g~w$^_xZEQPYE` zda%rt`yYeb7^-f5i;LI$X6hS|D_w`nm7>8%8>`+ar$Y`$VPj$A-MieW52NdY?ooL? zgR*%WWksk3o2pQ%&nJev`>l?Hp!-QHI~80Dx!jXKAlszNlauzh8lRG>jJee`dudkT zCekPvsjF&~3}fF9fkDYFv%59&d+&$K(EgenQImkM`xB@0dv|K zhnfNCp_Tt}?3l%Je`~C6@RgZvx zt!Wdasvg&8$tj_LoVhN3zrjj2c6L`+SsPnh*8+ zUVcARW1k7Q9~*0UIBczS|EZXW`=yY}UEJj4E`eubIy%qEF_kf+j+Iwi3%(*wO385zroQ-(=2q*w$nh+60*?xYh3ORi20XzNR97wf()Agc`eS>-& z0k7NHV+0Osi32z$ygnFb)I0J%9>kJh;f8h_Pi5$Gy)e~y800GkO=NE{H@)vceI`=@ zw+d^JIh1$xFbL-w3~IA^A~G#6p*z9m!xeh}=6#p#1?MO^@Tyt>| zdUI&M?zwfcV55U~sz4y<{kpFOhpYQ^IOUc{7= zh~}nw0oxZ~+w$0I2a6i!aC!O8*hNjc8XTRR@VlLu>K=AnwlDkDC>}l^jHl@HcwDbP zAtaP%a(cKA&krel#UHWCTQcg-{>A2TS@-__jz*<`>3yA}|NFe2m}d_OqoMtK{~&R(+XIiNWJs_qhz?ZnVhZ_dEN$Gc`==aF?WVu&bg4$K$@-4IWgR+`+x=`H0nLKP ze6l-We5fOp5v+H1TD;ZGl{KmZ?~+3DWM-gK2afC*5u^7}v}fk1;;?EO-a$I{{2n0L zt?lqn__I#fm?F#pjAo z_&X4R)@U)v-t&}_307@h;$$N?q%f}bJ)}g9vos1prS!$4372iXJ(ao8?F1Lo)gJyy z)ZbNPf~KAy_Y`8YxLEy$s$Mu9^y|+~?OS7G4_u^&r*aG&=jg#R;_=YHdpgI)k@5sC zh~ThV2;R)2fkQ0aX4|Kg#!_$q{&VyoG}^FX$M3&c@U4J<*mD|O1@5k$t24Wr!r{uO z#b?X2nTP(~uhYe0hVKL({_mqsWNuXuYEL~tNorL!anpHJu?3a{R!6j5siAKTuuU~j z4)50rSii<<^*#MRN{7j$)#fKqwZNU#V>An9|Kx<|db+_St-DcmYwNnmluze1vg707 z5RQq30K^ zo$(o22{*pqU31-mIuLb)hRflkl&uAu7xzFsQ)i9n3G}u>_4N|0^c9^!hZF4f-U)~n z4`o5I#1twDLR~hahWS2kSp*%Wn4DImsS^*YX#|hxWEQEUR9B1~kDO3}d0J&HytBVW zhN0T?uB%W{VWg~f_ygg&n|Ssc2T)U_JeG`WFd3}jXnlE$y) zF1Mtv=CM7yssQs_{#1!Tnnd8BTINW8i|Iv|A+8^M$VS$|X4q80hEcoip@?nGZ*qf| zAXDO)<1;%U+v@Cj87SrTxF9V2BvVoVs;ICPOLhp+FCm8D-7;P38qC#Qi=iQgR$?*? z=?|k;Q6(LXZf;rRzn_q!lF#sTz8SiVXW!Vyznt}6sdmpS zyGpI9C>uk7O?A`fvV9F}gRQt+Aa}|=SFq*d+#?pBKf@z8_pDo1daGct78g*0d^ws3 zj7F}HoC#MGy5dZGfg}m+CB7LzQ&@L9T+$>(AO({}NJvEj0zMu-j&pwuIYZ7~Gcu%R zlWApCidS?WJG_q@xXmAeRrR+_KmQQ<4;Dc1?e*7B-q6PiGAM7bsbGmKZ8(akk!~nn zIh_$Y(HSDcV#>FP4mM-9KmbE}0ii|UK4;I*iQf>*9W7n={{|`CcfaXFi&A(@J@Pnl zUy-o^gkVJN3m*HlIxz=FF(aGY#tfOEzOPNCv}#VlmQ=l~rsJJdl%e0ji|>nFDPW4h zW-X?kNs*f973PH_7?skr4|EUml49tO!l-a6Q&+@`oatMHjx>(CtPGhuTBCZ5ojbh& zgjQPn*WZ}T9#mnq!wawJW(lE}B!0Zo>Eh(IAy4K!l22znf0F`8&T;z)4gH90ffEwPO< zYw%(1>AO(&dO<60t?yzmuJE83@{*+;o3MG&^|NfnD;EQyYT?=VTEN}deXFB72`mtw zv+T%??~O9~;WbRZUR699)fI7dIysmfxa*_0*Lym;%j4g(&?*t(^F02T*Uw+gC6M2` zp((9VZtqr4FRz>g{J6dc6Q|!aLfQFdBIvHO@V&lQp8h-9yn3xE`?chP$Jli{3!Y3c zZtSF}D=aL`A>d`L;(1nvQBF}^+PV6vujxknG8?0)A=QrOTODydWiH%j5Mw6F^0UgN z0qhN;xJqE71MF&v+n+S{acTdT+JLL8mzMO6z;oOg_Wa?tTl@)g-*cv)Kc1f z2NFj(r%R%Pr8?8+X6J!ze^ykeKC?d2mAaG|Jw>uB2;NVF`#v}P4^!V5U02j~9ovnQ zHnwdvww*LKo5n_?8{4*RJB=IL&W-JSx6iBZ*BRrEGsZb*-M!adm}^dSHC4*L)To?B zxWy`EA0K@zvTR3;?x!=bd^!ZU%E9+yxKL%a)ozC51lRxYS|Ex}yegObcoy2gUQ#6x z7jv(Oyx2*ml~<_OU$YeALOPs7q~k1-#DMf*)>CiEq)W>86M!0AYVhyV|_bA4`k zId%L%4>kUH`>(KrqJe`!kBqwTMY&uzsMLgtS3XNvzR+!|VNa4&?_ABl8JA5U&e~A> z-}x!%^0ui`zf=Lo1tO2C2W?i6T|QQF-nAVZ9A>H9AZRg8V3L71eTg&9vfcU=Hn5>< zp&?FW18Dg01N2||9_}AH-xs^(AFI2ZRyN!=J&pjXB_Q64&*!)TkJR?!!=#F(Lh8@W zO)djZryQ(#=tNEIqAah6&Mq%WQ6HantFe|GZ_~cS`LijjA0;8-l%avWwC5nK*L1eP z{fU~a>V~SK(BjG>Ye|D$?NPqEMqoc&0A?X6l zzUg;9HD`ID1i75Fx3_mj`CdolxMlHAMbfKSWsvR!$YxGzS7m1Z@7 zO!P4aWO$B~5W9!7AZd~UI!%_f)VJC48txN!Iy_eINye-<*`9n(Z+iKyIR0?!-s)0Z zi253bk4H=z7lwoufJs5C1Bf^CI=+q^X)s^({2Y9{TI_sO?9_IMKk_iu)lU{zE*u1l z2m9_1qO%jTFSO5fLg7;5_Ua)z-Mm;sH$?mHf7I~DxZ!jj(4w$$+x2?Zb$=pH$7!qE zV82&1ciH`M*x`bRe|p=-ecXCAgJ)CKOD9pgk5~d?$-bNX&3F2%EX>=0EId@D+rY%A zPvo22%ZlhLs*qPLNd^dEMiS+{wO3AeoyN|^W$}wA<_hQY)J&E}I84T>=p~Ying8z` zTdn8)Sjy*V4WGS%kkgI3jc!VCECLy2RIbFRtj3wD9DdH?(Lh zWohQQLAOP19LD+wBG@CTzMY@kgeai{J3slc<+vC#%$qdnu?;*9l0qlwOdSRtxK{)d zTRx^-#>Xk`2FGDHRb@38V)kCBufR;I0DG_BX&5pS#l`5GTRMREOJ1N}$MVck&g*c4 zm)rTWCuqZl(I%yxa%O;pyFT9#nVyV3J1#p{gs=tNj<;2#D5Yr(p>>S|{%NKmP76t5 z5DGm!93LMCkV&S{S8|PhY_*ZFDih{>KFQ_TCJho}#EF#QhV0aQ><~R(Y_(jb+1YT} z>hyR%)7A8VP%=tGPW*9dyX2JD9$5m>pilbmFZ*}diwWbB|C0e4D#gP$U`V2_)pdad0=^KsOPHeS(d@B94n@-|Bh(nA)XkEa?dU(03eFfW*h zQXq$~=&eZp#bHQ0{YaP3hJ}J3?ohqQQMEWC0kqy)X<^+L3y>7&8lED)W6xAUa|dq=|s5V+r$o4TTAjLhSEl5#4&y z=sR4!+=|(2H5&!pl+lBS$HZs_$`=OfSTR~Wm10`u=!yNMi?})8Mn{O62v?uaW{%v5 zxNY}Z1?@JyKIe&|bAHUv-duj1P7g5CL%AxO2rlkkUDZYfzMk7vMQp9PT=9^; ztbM+;h>{hk6omVRWIJ;cMn8i<@QJ~I(WTQGxI(30BZ_Y1VKaYv@sUS|h#miTrNm%0 zJS}A`YrXVlloT6+A;2yFYL_~6*7e(Tu#6rL7BOmN#uv{+pb7Y{)rtlP4C4+D1d$gB zG}+5L`3O{CN`zjlcYEA)xebfrMo0JDN*J}eZDnW4V$+*45#kY%@EE?pk(P-CRkd>_xbe%TRj=)J8Ke%|! z@&Qd-7bTe0&fr=@>&tZq;sv%k?{08KkomkmUg&gMejiKi+xZa56GjL@K|*5GX{^l? zRL3U}bh+>6qf*4X-UQvpb6X`(uyk;G3ew$BO>-vIRk)mHGOPtHorGRF;Kp?Bhpr4( z+r>7QpI)x#IG(V2(su*sqO)sWPq=#4haKIXQ4D_Mc<^6r&HuZ_-05@4ww@MoQa2Ls z%T*Yov6B#|>8DO1C-4M3q}41Pz45Q!)^{plozuC zXA8;&X|op8p-~+rRPrNZ5vqW|R1y-w#9(mwhQSu2S~pUAjmc7#2aRZ=((uW+{w0f+ znfrufaS*||l_F>&mD_{*Fkqrc`Bq7v5PXXukH#|5M3y<(rTJ!58 zw%meV-o#7F``DZDk4`_QE1On_ZlvA&;J2B^in2ul-$-^Zi)|GhS< zL2cJ2d!)?jFipl_x40Z7J0UuJ206HxSQe8H4X;Q2&hHQxggSe+(NDYoQP2dTJaPGlmsZh zmR9Wj-*a;Unf~fKVEf4T1za0LNP)p=mBokjmeX>?5!&EXtN*a|%x?13R`<{7Y5~YLlzXxDLrrIY&JUt468?KIV6d}gH_EvfOP?V@kk?{7%uoP%wmB6?cThm|J$(e`rzZBX`5-UO0!*l#KfIRshn7P@ron$FfF1?3p_!-^XkU>(eHJV2|&VNAnwpqYcSz zmsMx&STa*zfP-7aMrNcZ9My7uW2qV}2p(&#%6I4#yJAsx}`3%y4DBx?~TiVrmCtbp} z!?GPf^YuTG!hf9QHPmfzn=jeWfG+=hxaX~Vy_b@9{8@H-s~;(ydR7`d3#Lj&k8!cL z7L72A8I{8g>~HtsQ+^uh;^fiFuf(dM|7Nnnw6iyQcbfJ6W`weLk z0^SoERnJE|_Nm_dT7VKk+S?;GH*Hq3){zM|1I~u276TS2v_+o4uS*K1PIA^xQ7Q4% z0=b_@osT<0vY^kpcHikH{%7Pp$Gw?Q=T!Cl{BmM(u^jsiacjf0Am+5`H;O5QTWEp?Sx3To#Eb&V>OG%dcx?6iusTipAJe6K}z(~lQLS>d_)0go@!L6nn} zMByIoj^>8Op?fUGt~L^SRF)T?*8NdQzJl&V<8#VQ$KVVXYRn`hg0&Ru6R?_g;+K6mL%vt%m6nNo( zFjZU0^He4wtbes!2L0kn(cp+t|3NxDIsYM*;AbK-Ap>Q?g11qxaYj0!G$%!u5o7c4 zy@EEid_#i+Y8L;z@?j#9b85B}4>&0ErC810v!k^hHQm3Auh{043Gh%JiO(nF8TA> z>%uGy)7q+M2~st6e4!FzxDS>>kT`U2a-_Hn)lengSKR_v0W|fZ@R5O4HdGfT<$asn z^1yXxtR|fVk1=tVTY7ZJg-uEdED2jfa1vGxkNmE=TTQcQA7n5wsV~dKP{dLvkuMVz zRlJ1J(~m7;_K=0XvRwtu;VhOVm{7PxnPF;!7}pi zNUJ1;0Ym}VuQ+oV$|I}VW(o>Yxd2l6xvE()_;2Fl1cs4qLb&4FIvs3P3`PC)Kx)cy zTnnt(MpC;SZBj~7WTasLRI@@{_%gNDg2Mez9M%*Zu6c{t(@P>cCmA$0ti>w(sgdG$ zPa5Otc+k`@35r1C4X=IU1cHB_zEm42mv~|^en&qb8%mg#gVL65Ht9T78~ zE_D>DuzYNLRG(Bn@9q7F6Mg0~W{cj&J{)f&G2^HnO0GZbPvX9D+W0pFJQ{qgU?MUC zB~dY&Axw2fbZM&7tJQ0uh`#KXNR})}+--Xz0Wm>()_1b@zYi*@vL7X)O z)nWe$ZE~U|J5h1KzDG%T@U3Gs)z~B_A-h<>wudYmv>Ri2nEZHrMX&O=1Ji;Q6QdU^ ziiB?wY_h#=Pl`gmhhgW%Shoef9uNDF?dF*-V9u&s>~sS@9ZPNX z+KK41R}Cf$qZP2AYmJuI?bFbZm}(Wory^5o1QhP_edA4WNHUKpk7Uxcl~c6%UP@O>S;k2~$`Ofr|FKP)sXYq3$YmAlT*lBWq2Q!Aknsw+wgd9RFqe?JEHyI|MbJ zyas6cKS|s(eUNWK3?h zIZ@T$>YNG)54?Ah=N-VGxt@Xc6jkcZ&dyK-j?k&jf1Ocf&nhlmr}hidm$v1nBnq-+ z6xN>_hhyzXA^FLT(Rea-TV1yDLa*PC?1a4+=;KhszQbpOrA0#eBo?oXfliJAI)wBPB3-(yIGA z3=^~1YuUfwk(%+2aRDe7QQ7*<|h?}(wsIC+sUu!=>vmoQm!BqrzW zXXGu5S)Lw!ihq7gEPDmUQ=u$xI#{wHC{5dC#ENRdT}>@ySE+E45rPnClo`8^L%f- zsmU1BAj`ACA=TE7JJh`AtT)^c70Ygw&edYJB5Ue|g@SCahrqQ!*h*EemWPgy7TxD7 z$IKT1b@U$WlUF_s_DX_9H*r_Zuvr<~B8Z$=krkE5a`Ruz7F~l9v#cKzHrw1tZM~+u z)WS5Egd9#%!jB9G78ASG?d*c>Sl3X6ID=q`!AQ1T-|Vq*Gpu@<)xN-~wEL18Wp#-a ztCZ}1<6}GjvAqTh!u@Ot%$Wlk%AP`no~EJ{FPgWW!iqW__r?|o+{AT0uZJ$2QW#6h zK);lKB|zmoXb7IUZosi$zo5oz>S$)zx&y()^twj)IrW~NW+#E?dw)wwL@`mk_U$%X zD=){wO3aFG+7}Rk^`_${_f{tJoK^D0W)pJC{fmRblN?rb?8A$wF3NGl8o%YL*88p%12WtYRkaH?toP!zYvw&%^l_b;Bj-c>&Klt~Ta zN8CUAyW>5Q=e^dilERTa(;6R%sRt3&QfY?fBUPHGE2eHyz#O)5#Hb zI|Np^^WcK)m96b@(Vz9v7r+3-`6f{Bad&^~i0SS28SVABD#-0J^MY6@A0dIfvO5DA z`m*LZo!v=Y=B@3ce1l%ZJd7Z+Pqx}k36kM+NvUzoJh}MMUH<`MvVAzaUu=%t4L?1zf86Jp70P@ymaY0yQJsy7_!86tfj?8h4?Mn7@hBY1XuEY|?&LoAtJx zg@{oC<>93!L9s?*h}qE|)J27f3o}|i1i@cSYxNPws3WBK}I zZvvKgot_0uu}EN+ae1BH4KNcy5Y8Hzrn*R~i!8x0w$4Kk$3=V9)|)=gzXj_^5_f)h zO?NIP@So^gpP$vM3xIQIDz$z}ge3I!5Ezbp@-rY*S(lJ?&#}BT;IC5o=JZcdPM4Z0 zu30rH9{Jrh6?lK;9C7ec9k8I~r9Vr(wZlpxrtIW{NF;f`a^DA(;q%ar1kTnw-!|`^ z-0bhdb`!2MDv}wryNsoQNxnozN@08*lE#0Rk0*(I{AR`|5baSO&EC}r_jGiZ!sQ!% z?Z%^QG#44|N)@lYxGomXwX;TxaIx_d5sjwZw~f4=O@A)LhnOLz=p_*cA++ipf`o$y zq2j1)k$TBY`Zm0}$n@U&tnTf z-Q?}7z7pUK@=}fdDG{N%49$?3a+_ht1u9a8t)!b`!sGk{16{T2W)pr1J6!m|>yO_Q zfWA!mnW@R~%O65sj}nSV-ORpDLpS0T$xNrMe$%x`D0e8xNU&-cqhAt^9LGxf0ygtL zHhT+ArogoWaiNg>jYS)sZp$zDxTXw_5k`Ty*5e0ke4I5Wt!``*a+uJ2%^fy|gR;oc zddpYm@8c=^2#{piGXVkB0Dim`6}*w@b@z*2HleqZ<_n7SH@-~PG8 zBw|5-Ezio#p!6w3Sm5+f8vVHGNian=NB+IH<9Z-a+_K?-L+IN`#bjTFSZ zBNm@q2~mzesP5W4McPtEiJ($HnFMCue`AIyURY70NF|B{h_BnP(rb|L9M6^@DWYV( ze!DaH80uj}UH3b8tBDs3+DWayo|wwwTV1=*@A0)EMcSFr{0#}HEMCa^E2bnJxL*%} zCThO-!XePLK`c$bZjsO1_PdP}<7FMNfd)FZsSvaA$(@gJdh1zzEBWalp87y58idy|sbrX77HuIoV-ihKFN)<0Bd{va?Ge zKW)D;JDZ4&0#_FI&E`*iV3J8~KXqiha$umEK!k0iH%;RK^&M+=dH4%i!8PoNBC?Zc zuSeXxC4BzMa!3hN_UjB6D$Kx46rG!xEoHP|U5sb$0VvS#Kkk@5e$CBgkere{aF}X= z_cOl{ect!DJ)%I==B8EPm&1EhT}9>*y0Lg1t^8B_3x6i{@etSe%NZy&s(Z{ zwzmuM-tSCiaN)XQPlt*%4WE4(kPwrOsi&+vj4G$p@MRxTsn==$-xBvz`1SD{kcGk~ z#J{1-e4*_+mqow18V}k&BznI;5@*?L$lD3C$S!fV-eRO|%AsjTEA<;{{_r7X4veZw z{XQN-z5aSEybr?O>Fj{8ItlrhH9O~Da$Gt^Mms35?_i{1<@#;T&o3J@sr6vSrXvI1 zNGiq@A^5VCY9>aw``>*yl_PQQTP_~XHX2PIFBqa+ zyhh7(&Af4gu2UF9;T zW7G5KVrsIY`@TOFQvJ&Oa>HA%X;#7kNIbk_rbR|~Qmt0fxl6j-lIq6JgQh)E>C!h;2gLzO2O zSx|bPvV>_>VniI%)qF5ojKI;JY2!lbZcx+G=a(m|N^6%RV5kHRLy{XPoR=|EV=&uc)ltFgn-%|Ulgwv>dJYrnZ_qBI_hzf9y!IGv!{NIC;v zEQwZ#Fnuam2s0_BOl8)XZZ}k4}Y0J2GiK!+- zX-rvWv+@r;9ZSiKGww2Gt08td?fLcNE+d ze^VdP%j>D~lo5s1$pD>^*Jy>Y=q$=4j8MwObwz%iXk`un200`lkYEyqam(-$47`71 z4y`1584au>aO2^JF4u87@H*t<80eNStva`oVbTd<@}A{WeoXvkV*kML!RID;GJ+5by(&ys5rlOJ(D0 zi#XA?TVCh!uF2#g#ZJU$UZ1CI8N9BSyno%>+`Rcsv7jOKXrWCGt|`XTRqV22A#VSK z4-t$Y9@K>`RM3+oAp0koBS>kJ&+(u=WOcqx3`b{mps?#*AdEnulBfk~{ki;xizJ5c z#8co{r&nq-@k*UgUlKP1`053qfQQ7eIWE~Ix#Flz9#G-_Cb3dnBux-34`s42xF3;# zM5xFJ_4Vw$-UkxhVzF8$s+8L&Ps!u8wiIbY_N1K)aeP*0A1d1 zv0uC1-RY7|2ZjgFJSt{CZ%=SQk7NqIts_oL_{qJBh1*|@;v{$;s!@nd~qA!(}6 zCjr|%neE3wUx>V7m6y!0j6^K(yxZqM0%YVhHyGKqOH_HKcCuijiq{gO0@?5?SJCLr_dr!%Ub(m}7IgZlK7fD=aX_s+JQ= zRmboJ2GK^d#@l*htdAKB*Qv%$ZyVlQ7Dh%Xn%a(r;S_QCX??byS=NQ62z<~OcH{*l zdc8Em@(Z|mob|DT)AM*2ZKn&Gup%z6tao?ZTV$}|MVOtPt+!sdnX?drhZdux%-Nv; znyN$_f9z{afBevOyU<4ZtE8?I*XdJYYHoGk`{?uGbHxU>d%kE?YBifbuc8XIDknxl!>h31$w~n7 z^u??=(wup_TqlQT@_Y?$zs!KfFzOMDqX7s#n4gJ!dy+@$LQmHf5sMO=*WW8Fn`7~C&hkZPMAC2+7+WHvzqy@Obp}eA1=~sb*mP_ZM{m@d2+TIo!2`B95WlWKTh_}Q1up*L* zvOY&a*B8Ng*rUpH=DY19(m8Iu@R*!-U(45G5PY(>HyOgtfdHUHq}}aVWnv}R^Sa`b zUnX1ScJb&=+ZM}{Kj=NpR3Ipva9Z0x&Cc$9%Qrn@7!3QfPVM7(ED9PK@#Cs*L76^! z;A*{SYJjh$Y196k4old$sL^O6{CrYYQr7e4EsM?#G5I!Fh6xJrt-yGR9!sC zq*mv}Q9$z9=*_w{H$K+x&%2kRh>WG!u|Gervl_Tw^5u$I8XN~^i=u3N>I%rns1&zw zkulFwTI0g7e4#G;H|qQ^&sOV=j+!S63QS5Ae{^he5~}FOsqBm1m?YFOi(%#o3NI?+ z#Ph4tk_`+{DQw{)QtQGVSU$1=Z`&$YE!{2emH{?)KH@g`)$2^DioDGZS`Bu08 zm?2RUqwNEk`NAvAGOcSzpm4R-a~@>J(2J(S!-KP{_dG8zV-lxzsVHvNKkT!`8?3iQ80`drPGE>-%d4S4};Xk{W&FhtTIiPZ@1uLRANt zHP^nc9mBmpxd_YA?X$O_UsVTR+$JrA-lV&xUVjBBJw`>jYY>AJaO0{mSgkiM%61qx zWeq8;D5xsA{YDqWo$oBlX@UFW@xs+*y=7mUm3L(6ca75{AE+T749_M21{!G{H*WxD z>lip`JAjBCFV@@OFImE^F7!6^{&T>L03>Z|xbQB(F#8}q0!F`g!F-8yfBrV7B=?*g zNx=m3Xr7!XuVX|yxkAnF^?VULt{E95k;lwZ56&@o)gD@4@Ocqb*WDXI?s9R{RgqF% zFHvb5hcbia#EbOKwbhZDj4H(=@mrjC@$Nvk3pC*Rz|K6T(LlmMn`T1Ah(IC%nYhP& z57gYet{jp2oU@_A%vQ>Zci_0@93n>CzFBC_NU;YmTi##g9rnEVddqGYX2~nz6B!w- zNk8I1__;h+k3JnwMR|JxaLNXgqTvp^U#jVb4L2Ffe!e=TDpB;p2OC822cd#Kl7Jg= zhrtB&yTBH&H3wIW@Kd%#*wEokzsGl0Gl=qtzi6sIkYe&2vY%4JjM>#*`LQhD4>I1X zKm?;<+EL<^yY0k`Zx{Dm0t_sMpbi2ind&#Xo`;tVE1EiWamIW(+h64`i?2A~gQco6 z!8(4==AY+_eAzmn*X7J@Pfa6tKYG#@GLNOY+U5Qz(cB^vW62;_B&+>1oEXi+rb3;= zeIHjtp-5Irl^#X?MOl!Nk|k+IObJikPDQ5Y`9mBryi9*PD(NiaGD}ewkZ3YLk{G?v z^UG#?agt9$M8hNwxeR=tM|t{MeGgH&^6=A;F0VU}C1=|26GVhH6CnvXMkk#t^W0v% zNl?{dzy5S0E7Ws;S3Gwl*u6C~0-=Fsi(#u1R?#yeXWrrgY$iSm%s*SDZGtN`}*#kHpNV31J90`tzW%( z?OU2Z_KOVGJ3sEy8jdbpT;ntlT!aEZfzU|wDnM2pq%9sNZ2=-u%+5SHJM;d0f85yr zuQVY9RphrYvr6&5CjH%*a_lLHRI zdbQlabSZ)dC%__D|1gmZkP~Cal2U*{Ax*1f!{>KnL_q9AONo<1X|Q6`kZ8?=qq)nV zO_BmOz(@+Q&g8CJ06XPLA+<`hj!sOcNhF47+5zbE;+C|^KEgb**}ZdP3d@Iu3;;S_YMLT? z?n1RlnNO|4TsT8gYbn}XDZQEJa;pd!PXG9SjYu^0S6DDBnLcn3CRS4u&f- zb80%NUhoBUu_W+1OQ}YPV5%_$F>B*_Dp8oKQCJ%AlvpD;IEaEo2($q(w8D6n@EzUq zny=ac%J&sH;3hkh?R~cW`{M9wj?mO>33=_a**59|pXwt^87o4XIwlU{0Ne+UMf~*C zLwRo)k5lm`PS4w_rOS=(Z%X=M@Q!&(!qhcyFF8{+IxffC-F~;1A1`;XMBluCqJeUz zCc>^`M>@i*3kwUGGY*|k=h*>%aKpQ4aat8h7_oiEzrLD9rgdf#0?he&>_sogktmW> z@^E3kgT#f-i&pI#Aig+y&IASveU7?aG&X{v=gC#FZ6#0FmY*p(u9rpH- z`8_XvUeAtlMStbfpueT-`8YzX&?MrmeXaR;*~;eeIbB}69D?uDL0^Dot` zpZ|EuD<9ePczxaa{OAux1kHtj1`a7no2-iEZSAu`Ei;sny`h4~8m@%Yu6zTWjJs4$ zcZ${>;;O0!gBy%6jE5qwT{JqR0R`($R8T) zcbk$*>gYsCl75mm=CF11gB*v2Kc->D3E=iRam~gB90m*vgvCUwm)OI7X6?k_fJ#KQ zs*k{IxKEAm&sh;-sb5G}Gt=a7CDoqfR=Xju3Xzc3{FQO04klq#{=1*IO4F*obU(7{ zWYct_X%(ws(e)R1d~}wJsl_@Q4x>W#o^+t)m}!yq$jESD?)-e`_Fh09D^?>_@nnmH#UkMA+V6$@-@qg}*e(8b~H<2q-5?;turC(jW0A*Jd zIMHW*J84xGidpnYVq*4Nb~ZLW_q#g+uMcN0SHBy`f=PSJW%iTX{ z=$Ff=EEnWhsn+2e985H}H%Z3Bn*>Z;b79ih0Asq`g1<9+M9gtQL89t5cG$RT z--w>~`->b1`Ej+{)_kW;%BKsb4B8XzEcH8lRs9q#5Uy{A5DRh(Riec&9;2$(I=w(a zX_SnkN9XzGd+W=tVOUAupTFB%JvUS1pi+ImQ|OlWYnI`#s-7%HZLuDqklV%jV>BU3 zlEa3*;`4l}G#*w!&iKOnn&10gc6Ieu_Y16CR488@WOudxLr^c8$#{ukwqDOEj*!Q* zkhk}R+|fG(T!Egfdmg0WRSf={YWgYFwJBs;BXo7GgvIa08c>N8mP;8mVVNS= zrfF5q`5gTCO;FVyb!n5S_0uM%UxuT7*kaTo`elN55LG!R^!#=@xreX4fYp40ga)&o zcRx?;NzWB3p(8v@YRJ;`4=qK;N)n00uIgnERZ*Ugm)lqG2TROk(5%Ya=t2dkeAOEO z$$)qo738z=sPFfVI`oSt#BKL|4q1Lu$oIsencwyB_nzC@B{(q_B#Sizu`RFHOH9|} zhoZsqoF{N9DVn7K14H*FQl|i4Lk5;uCOdF+wV_18%Ia=A;2jp_8#+NGvdhB*vQKu1 zpO0bG@QB36!{x`vRo**ai-6Pq-<{>tZQ(5{A|Gmu9A1|fSihSBNH4dZht-Oj_njy* z$(*$Yr<~#MN`&pcB|rRa8QqS1`-UR0b>g}&K#uO?anESIwYJ*z&hK%RV2F6WypF0O zch~Trv-{?dsrz4@ERVm{4sQ!&T66j4j|Uci@{<3{+-{>s)Q*eZ)<~^~v-h+pP2chl zb?(Kvl*dXsAM;cMbGw>whyD(1Dg4ZKf z(4ng-v^6sJc%ghFQ!G z*abrFj{Gc@PGg1j4?R35g;K*tGQyEDz2Uh+JF6&f$Qh%o8!uhr5B7Cw$CT$ z5&X+vIYObU?-_+T6@#>u!n+1~CF|dv_Eo!hJ8J)FBdsQvuV0bou8T^{FzBi#rUm-?4i?jc2 zN9euelO5Y>`>AFN7^@9F}q{)-Onbk8YTdb*}^o zDrFasu?>K+$zNOAe^11{n=wrv{cw~EwpkZ1Ek`ShM^n$og7nv+UXTe$Fe&G!utigh zREt+iQ%k5T#k`WVkNQ`4^5B;Juayz^HuDa&a&fux*gKYouetJYgFM-PvCwf2E5j#Z z{;`u@CCIjv!=}soV*DYes0Cd*=-Pe}F6gM#PutjR|SmCYLFjxFlcyNiWl z1M*kS44Z1Ixy54{PAQvg&Z-xGT8MKrFE?SryrARlG<*&!_?6#>!6z59RR;5dRDg|H z(=)^NH}i3fy!<)&iQFbd$@9siZ>2QK&|;BgE~N~KrN*oeE)DZuxEMakQee>T^)-jM z^<|uUh2>vPbILL5PU#&Q%igX9TmIu={sEu%RB!)ur#EPvvt-k=!&_AJ-}1(M?vDFl zYG5hu!wK?>Oht6=(aGmWTdAKsBk`cki0W`nWXK%?O4)!>O;j6#f2}6YSV$8(CvC*L ztRS{3H?%U;t72b;>A$128#Dz}Ohek!r7^K#hzUChFO~DP^qa^@l-3&V6#oy z7C#R2-}D7RhsRT^cO=qs{=T(|xMfgsF~8f8u`4Pjuvr>T$jPK(kvN*8CyZ`0G+Pv! zMbh)=F7~mvNjopg4Aw4541M0n*SN2?EAuag<*G@OwV<P4O1+C68EXT*j zhHO3T9MVqZKaq))q-JsMk>#h5bIk~fm!fOUR?#xv&Ik# z;^UcZWUXidWLQ!XjVaL6uangS;{$1TQLO5x)YJzyXO2)Vg=#YnwWw9hAk!xrhE(BP zPtzSW3PK7|02TR#5Kt6MKKsXrun(6HQbD+b5+(&KzM-^A2iiaV)PIN{ z*s~4bQp*$Nd(_I3SQ7!5Xkn~r7TF%=uhCWJ!w?4F^Ocl-IjEecVU<>TQJsQ7aJa-~ z*_Pr}hP)KB)H9;AWGIphQ^RxDWzN!L?a7p2P0r-r?b) zO4*&+;@Cgu6U`JP)b7{SmTIol3$+zu1)r_mvO9KOanRV}JHo~#)HcaEd1Wf%N3Jft z9@5~SXAD+g%&x&fob$(4>rqY7fl8E~)_0H!XI4hW7H&L+0Uglc2~*BOQql>e(!csL z&p1?(j1FZCUZE3{D6LHzp*FGEexD(e1p+6|w)J>F8^HRs_oghvtj)m!Qy&}*52p~@Chw(Hf1_dvGmQ_c^X0`V~AjkVyH-4C); zM1kcFh7FX0p&3(X-N4nAuzb%gTiuQX{Gl*pIXPg6n7fv8P_zh2?3HZ4VEWvh?*HJE zMg`LY8pti=pAuPHQ^Y^DHn9?l!e7nO_&ytSv9q(^t!ZwN=3BVhehng#=wsKHqkxpNBKyUnn=)Mu#OPOVHlFQyTDDIWF8SlDnh=9Q zaQeTv5l8f95=4*d=HkZ3l>g6{xSzS3wr}pku+%{n?y0DxdzRM}u}b$P&?)ogXs za`r_j+86Edn#{4+8AVBNKcCF+ba`d3)Vb{Rn-9+XGpw^Qo*ucTbKF0lyhw71 z10njQ6?WlSp3{V$uNC=fvM(;J*#_l&>2V2VJc(hRzZB#o+R)Z9QT-Ya`7UU0O}%KshZoGKD{ISdv>ZmWsyvfVTNei6+i7l_sc}^huXuW+&3%GblO-%}JrL7Cht;8z%l-5T5qTahx zI+W(Y`z@O~e06m-5!qT_GzPN-$OEb|V5RS2oyq;%ssqoJGwv z_#YQHcT)$xB;Rh?p-7~#G|z2lEz&QuzA8i*xfdc)kUZQV_dKmAa?v|{?C&*P_N-ZT zs*UfOCroL@ru~dA{)Q&)ZlbIyk^eI#HOn= zV%pLcQ~Cq}tQ+(hG)P}O@BzOFQw-kE9e%OO49d`#!;~`zVxoD`LZK_r@LQVBvx@kG z{~k;rj&33=H;>&BkcZQ8WQ#Q9*w$YcEa#jYivgTs;xgAYt38 zIyHc(i5B`{w@C~|A`^uL;MDS4LC(N0p{YQ|c2tWMfXHk>hByeam`r#1J-XT(LZCLu zpu)V9w@x>HPU^G+;L0!}q>Zd%%Ih-m_{hL5i2I~(kp3k`8Nz>M*!bhxC7J%8KQ_6c zJ0P5BPZ|1;Bd+0&aVKH~BWh;B=jm*JKK{jog);mpTADC0!fysigu{l{l;Csb*2;IwUdp^;Q|`i;nn~^I zw2Nq6$o8GMeN)V^`odPj$@C(>=Ql39DhA3^gkUg;ML1{p zKN$jJGrNydaz1wZm;8MGu9C++o^N#2*!GxmyWF_nN8!uQA%x2yI5o_vbV7D#-`w0x zO?{&gJNfWgt8ca-r;t_q{Vs4cZD8y5w+K81`=Ca@YrjYEQ(n@hB7irq)$g+#9G_>d z>K93V;tiG1yOn$H`#hiqOA0M|Hn1~(H?VQPaJ2WS?8{bTQxmZ+g4eNghu_T(i|Nyx zpD!LKDU@Y!qcB)sZhi+^Jd`lKOPYb2mQHF;Ts%MbU%!+7p~GYeh&;aa9G{o1b^-_e z_MbCQamncqTSBkf^J%J0@?C2U^;^rTjqCz6n5m_urLC?7D8f=d=osm#|E#5DXR>qu z)%AU_m9#(&0Q7>$GOZ#~E;jtOGa8PtGO%&NkmB*cTrbu=uI{@=r7xr`>zfZBk`Rwp z*!#htY8?z#GR&ZnZfZ)Ve{{{=TvO!)!LBN1@%wK7nORzNc!}|QJ~HGM=}+q-d+dUw z=y_@J`wb?f zVxBsyexI+XAkrrFrS(q`*DEaSnnEToG%?!2M|~IUBPROTUdedPW~t;xQw1}V5EB;z z&*;*Y&-FT2R#p~_@1b26L4VK_8!QI^@xCM8IKyY{6d4K=ibGA=^oYOVkeY@)3TaP%3)s(fW>#BQZ zC@VzE$4=d!(EOR4Z1;Ivo0^i+v%KUze}B04f1dAuSZY(3rI7eAnax#dsqN=DJx%0y z(eD4MAE(6Rf1F-bw9@obf!qaNm?|Jb=^_!9pKK0XV4@13KBS{wclA(IxNRJ>)qQ1f zud_Eu6=s4k2lcrg49AfbzZFmV6Cq}hWs$KZE66)Dr6EIkm&|x>%yGoc4<}WG1f1`<1-vnj{AV$V#A0&C9D@zaJJUwT7FeW%ohw~CE; zZ1J%_F5Kxt7Xuj)fF49E8=dmy8pGY^aBj1)B^qOAPmccrargP>=kenS%RVY;4-zjc zAz9`%Im4@G)|cJ55kSIh^`Tz%;_M6Di^*70su-8Pb$O$!ZA`~0q9_11P%(y z!4DsNDR{vIdiGWdc6W`|pO37pu_Ly7`WgK%@6)U`*w^qyl%SiKgtnCw7O!tT#e&bv zksr$?p4w-=)pb>?VlZ=6IHTqojJ4vk0+<6tOaLZA6yJvubYlIBpGF&H|A`!M^ipfh zvi<1Ewfq@ue=_#1@g{2X{kcUDaA0O;Q0umC{eGkP4yvI0{EQeHiG#?V!rKaAEZ-k& zEVI@`nac2zRGnpwL_%6w)$U7f=cE9MVE2Rz!3^&?ArQB>w|jeeVM}4Tkn?@1=47ON zwi+0CYUAza^0|9qe1EGSXD>hdfr=Ckc58NX(QQ%jGCs17`^{(owFwx0&{seohdG8`(mG*evwqM&VZ#CZ@Fr@i-nqA&+q(&?1VWUoVCvGt z$JXTu6!#$s_&=FXPwS%X@0b4Xg2(6Vn=PK+7E{Fm*g|v9#OMGLFaokvIHs0Z-#!dN z&Pc>%E>Iu6Q+V5zf^ju2!@Y8+f1S6Pl4)L0pzZYYxp|e|9=@*u{`QK;UTv(#qEB2q z(MOKb`#dI9_sK96d=5Q?C4~gr|99Tn{8*im<7rrHpnM9umE)p&ga_^Wmd9t&{HDqb z`$pif`Iv$vy)VW>i#9yJwetJSq0WumL|jQcvdAJdtmkDXFN?SC{n*`~jYK$8GL*(K zOCdaf7JO4MCocWvK2FsB3%^5%FnUlJ2{knWMp{yX1#iHm^hf5YJts(N05aP(oAp3m zmWJI=%pZrP(ss?|<>g2tPDCFNafH%K?3a^I!!CZkUp!7tf%E@_hnE0v`A_)VpYX@X zgmSgwTc0Lg?bAyri;Id|8oK^`GSi?UrEn2AS#LLu7~8{EMG8V*z~_yhi*Gt z{~!%Y6H5Jkx$^YG{)|Se$;A=frTFhg?H`XHB*6eO5IRfMbRQTiVf#Z?7IbB3QTFxS zyK<>sb!^_trFCsxOz?S_3heuU#*7Ff3vW-tAB7{Rud<^~OOnR|m++Bb8RE0F+U_<> z5+j-sEyuA@{_LNAZoAgvXzn-7==W7y)3fU3_nr_8(6H4#bDeXiu;xKbA}&7t&b93QX#b*6Fb&y={F$9PE1p80IKPL8!z|a zl>B=pIg~hlgXrm;At~!9K;Qb0<$Ziw=hMtwMwRS??+;S$&Ue8|znz3b*A6K4y~yU^?zYk9wlZQW`Fy5p=Dr3+~4f?ymRUxKPQo8 zsmc4Hb2P2q&s3^~+%h|Mt;Xlni8UTG;s9BNdV-Djw=iX|g#!zE?!cG^=>kGs1$ALx zE+siRLJfzWhBzhwgH#3uU^3pJ$%Ft;(SYo+YeZGZkSVyjv0_Q68Ar_ag$;;7UT-== zD`y0gD3y>9C?rL%-K8DLBVP28MV%?F!t9`U91@MW3Omh09SW83bxHBKo(K*O_9wXh z005-nBx0bty1VO48=w$R9jiwZAnmc=yOJPLVE?Ak%tSGc1a-gcFg98zpEHd7myL95 zc<#>2pT%W{onI2K5rsZcLC;F4COt^peb_6R@S8lP9@%Dv;>OHW5< zY+Zm=PcXbTXPmE6cZUxy@8e!_rc&Ib$TlAWP zR|hgs@UMIKvGpXD+JVrB|8E|VAfZz^ACZDRUKP`JP>8(XUBPByQS8^vnN1kZ8%hbK}2{hq~hBD|ZJpOE?xB`$&rTdx)`)jwr>V+009nKu#E zleI#L@gHf8>9D0Iw9Lf4(F;IOzPEEo&*2qJfJvb<*8vf-`0vs~+&(BMQWRDcHW)Uz z4~tkeu#W{;xO55uNQa9>EBVSQ?ia18kEK}JKw=F3ipYHs~U5UJOe-?pv=-%2}no|>{eOYQjiI0#VjOL3#Z zpa^q#-{4?@&7LU_skDQwgI7V~tEJTUZ;zoTYik8_*Z_lohG;eU&+3|t_xvYEggAIe zVU^lOX0vQrcA6b=m8`9%cooV}EQ>D}IKw%z*-mR!MN^)Jje}VLmszFA1m^|?MKL5X z0+m(cp(K}x1t2XJJ)Ivd1O|me;(DnKefN)YbK{h>m@USmb3oK)Cr3vVd2@4fTCXPk zIGxw0;eRt3ekK0RxzUKRCaJ zL6dCp1EDeb3(W|n7%UTNDLJi{Eg?R|{Cct2(XTXm^j9#C-sS*&{R!aK^QzwL^Se)B zYVmh}S*w`HLw0V}_~mX2fwW^CKq|-67tYrdlxc2a!{~9HNk!J5W|NxCItie?idOGo z*1H6&#$n?`Q<<_*ZwM)~!tsdWg0VhM_81@c*z5sc%}S`!(x}(}f)Uo}IbrFD>E3QM zk1<;Uiz;q7L)HC!;P@B_4n@a2BTjK^baqiem6NH?5Gz~lL?#|VW|jd!@|4K}0R>gM z)q3wM)`D*j39Iw!r|Z6m17)UB^s8EM1mFx>O>+jF^l3q+rM0WAADkOhw5yA^HA7il zbL44pB>ldyefS*bOB>79coJMDE-tQ6-wV7TrX({XCCqOhatI~xs9u2VJf+KpUf)WDTpx&*? ziRm^?Za0TN3oe-*xG#mik$M@X9WUcS=CP8SWeMYxs%YD^pKH>FR}!srl+^L^(6!K z8A%v*>-Y#z>mxy;+q{jp?QSUny{Bl}%HdEz$V%HYjiGfUa`@WJ+}u)2uEUTSR)F%u zs~;H+4RvLC`E~Hv@m-0u33f$vVDDSz#8+LyQ$zBXFpE>%7_VwrVqx*$xou&|ukH7# zyZnR`DE)lQ+tsC=?d=WWq;u076a_c`JuG}|rkpb;%4(hFKzjes=)T~-z@CtKbGj%5 zczSwY>HW6Fg1LGpBu%;zE>9@S_wUXg?NyYF_sZ%^Ai_kFM&d8|fX-J~Av0&$U9`J@ zDT$raE2oO`@(XMzLRxe*xdsv%VUz#HmgDD_ap}dK7q`}V+3m6aWzfvcY%mr;vxB+$ zMOBfUjdO>N|1#^OfJ(56QbuVN@=dA088kxhAv6F<5Nxu++2yUc_`PHE}m#yW+8L}CwS-W7hlG%;p)tp7=K z_nZ7z%jvtUP;u{gsZh!9KAd*z7&IQcNAv9VrxVmG^KTQbph6{5bd^vjksNs3hgD+3-OGrov16Z=kD%r&!AzLB?!8@KlO)y31 zij!BL-=jWwDZJb&%G+kqWlEV%oZ^>$p|AF^FQps-!@+-O8+;)y=4`BSS)6IW4hgTm z-{z$J`Jpd?B7E$lI##ZfrJtXpWdv@vjg?g!)DhLx4*&Y&a6GBW3>3f0QSr63KsA?R z=^wA}(>Q*6l-nn42{}~xpNpI^G;<-A-@2Vl4|P6lJWUVLuzgkjx<1WX0t6|OCIWhF z*4o{x3z(W4OKolV0#2|+^h!&hxqN&_X1s184W^BaO^186Ftu6<_{F zSF^=-8x<0(k4wr=#S? zP7NhBB{^&Z;c`vv8}==l<}|C_**{V~J9~`DaJ>sQ8U6XaQDNlCaB)fyV=gGOMST)y zx;sF^BT?jN?)PkhQaJtdja7Y#jxdaiXemRB8ACu zJyrVb%<->n9%&eErQlo$dSvN`{zeCrr9CsD$j4wT8yPh)1v=~zB0C%p$uV050K5&N)J?!$aU1Jqh2PPLf$5;$sEPSlLQ1T4}=2{W5J*tvmh}FO0j?>OofDe z7Ad7^7p2tS1LJu3(FOrn00IIr2+Vh4GG>yzf^ZYe7~4thJL_3< z1`w>wB&!l`k*5FwrN$=(fB~ro2I%2I#)BUnOR=pm$~#PCn8-3hgtnBZ5ZC~z8deTW zV1SX?CuNZYoWwb}xZq&0K~9EH)FKTFFuh;uK&3>SHY6x~Mu?MqE(e*pQ#*tIy}B=GtalfRqM!S- zj_EUhyXRDMHo|6i>ap}2qxCPUsliYXdVRONNk^1RCqrb0ix3e};Bo;78DKa&2Zw`1 zYK&;G4WJtm4p!J48So-8`K!J z2T6#jK1nD2RrWAI z&~6~TxSFn9->7-y+r{Q zl_Uw+@aV8G!4Lz`ha6Wpf^u{hrv{CD{jc0I98mpV*j)LM096(U2`fyU3F`u)2oqC4 zghA;2j}>ZGVy#ylCN5+pFjwWbo`vS*O7{v*#EG(ukf*p|fXxWPFlG`vDguFW3XfcJ zy@(lnBBDtJYUR_TpCcu-Ozz)bu#Y+B^ft@B4`Kaqen-cR%FI3lGnwCb&mXn=pawBp zYVt&4r5ucFzI?fOxXmo-HQ%-WaD5vx6GFQA}rDzwP}Y>j*I82WH=Jt5}^UJbiYE$ z4UI;@==m2#U!X%Vv3{M_*gcm_2*b$&0gUdT4GVqXBv-uzf_6^pn5ij0eU?@O3UR&w zUhyar7|(g`wLvJVBwc3ch@Ul_CQ4myoUo`12A+A|hm|H7$Xy$6-k|WC5`m@v0E~&V zhXxjwlclsuC#+PF6<3zZjx&>se*o0`e7#D{aEN<&Q+ulQz(bh8BQ?T6xKLj@T=5uS zE?x<$iQ(P+h|b&vVrS<>2aAc-6=(ko_xR=Sqlb19cgYzRircU(=*#HQX@9!SQEmRi z0_0uR_`l^>5p1^6=1NO)>gCP=%OESsqW1^4`;Yldolx8-nse2qSz;D|u}Ddm z zBN|hA_O{Fr{G6TO`!shKjZ18(&yxaRvtIeO^;+B?ef1}0%m?^@$@s@ub2=vdc5F(g z3*Q|SLH_ZDo_BJ2P^oDW^uK7t7{Ep#4bNLq3*T~UhYifJoUxHz2z_t*L^xXz5g{Ej z#IE1O7}~XgjE*?=V`Z&L*IAiz*3Fpl>2LkT%vee-c>D@*W6~ z>i@9DkT0F+V0q7E5H;}@pTj85Bj5tD;OAJ|c7?_AxJw?z^xiDt|Ku0EWB#@(^_Xjf zHWVVXiR0db9`~gsgebCCWqX{1)c;CeR>C6ac~tl-bjC=Ha~+#n3;fDDmxQh~JzKqcAbiscW&zST6Gkee8P!0wtK^I62c}I_ z?``|;JwNudqVwhLubCh-Nt|yb$@{}`%-J$0=CYI)cAGyxWv9IbRhF9Xk_(NS`S-d9 zpc@}e)<4}+qYuv7Ae&N!~$5Y_5~Hw88>5pK)Hjr7@(mJ zzHTk)|C$#})K0?x9~J<*k%Oc-@<`Wl<|Da^+Pi>iV>i1}vRvEvtnW~b`|N7QQ!AqK z^1@A@HDpqU6SAj{7zoGsm&1d>lDZ0UsD52TK80_i3N)SI%vZhC%;kzP{BVjtVp%9P z{L0+z`)w;|F{%F8uu3~?z6go;Hsf5gyz(F9b~v(%88(6*C&9VwGyl0)z2WZi@1wr( z;ebmm(V&r?D>9nE(L8+65Vj(w2KZ;#$H?iO3Iud(4SJngI$an_`1uGGr+{dlz)$>U zq(~+Z%wc@a3f%@SdV%!R%G!P~3=Y=dN8-`t>&MIwCva>UZye0VSC4+<;iEfd#L%UHiW%?bl`;!n%--w2>S7EmEjEH?>hgYn zqn#K^Lc|DkjI4>Qw&1en{5?z+G8HK!dmV)+e6d!`TU2cn6ZbX-?hPQzq>uuJII|Qx zZk!^hd}bCmlfbT@YpKvd#HMP@o0)m}y%R#HLnm3zLXWG&NBPD~^QU22%sJL28QG#! z8qiNA?bUoXJ~p;-CIp+`Hgj!uk>&ou@HEJ?YMq_ZWx-rRTCkU&lQbVvPOG5KHSuq1 z;eW+h2Z=|2=g=(OCnV<}#(e+Iq2J#IGI(#!-C^N-ky*@w2c6TT#-GAtf0>}QF?}+S z1a1wKSqb>F2Y%J)ymK2Hq=l88-Jv9UHM!oe941R6!vXh8#Yz3ue1AHJ?ZOZ-0E_@C zpu(%UuCby92;+c$5v!lg8{7*ka|hSz08&gOVl_2MMf3pClu+@#?)i~rpD!GT1E>IB2q~A-#wzb_wU9GP0npJK=d&0IrZ$;k^EE6t zXFbZt$EoVwPLSmu&KDNCEF6n^sBj2r7b+6oIDixZgX1E6Bm75+?~K@xOsJG~Q7*+u z(c^;-9R&cLY5+=bPxb8ir$c1`gM_})hH^Y4A(1Wn#7G))=HP#k?~daCUYan3a!WTs zxP#XpJ`6El4L=wbJtheb6|#k9m!zb90H)2b1ZApituC8V1uWQC-=6EXl*dnTfR9$g zInFqNIH25`5^^jy&7+XPLJpGYqvU{U} zcPh0>lkPRNCjb;AY$*gF(;~%AhRx_dby^Ef5(>1D@O{dW8@7{?W)!X{UTXbwmB9Y} zSB_+FMLKq}W;>jz_ZelVr%&L(%P@NXpFt#2ihpZ_q6u?pRDCNw^x8@6fmAq`hA;(DQ62 zGE@~+YOuCX7#F_J05OtSpvvNN{Yz_MFN&w+=x|J6tSlAx{l8|W_M2TE(U-CAFk`J#j>+&6_Q2n}z|$BOJ(xjUSKQd>(qelc4=^_kTq| zR09ME_QBYK-SfzYbB&bu+&2w%5U>Zx>?;Q_FbNI1{mY5=Zcwm$xA%q`tUGQ|su>mX zHZl$lQm%0!;JNeDSU}C{3;Gx!gxbtG@Ao*MPZC1**L{|j399VU_}$!0);l&F`rkzY z0n@7AoaOGRr6J(8sAPs!O^wV}v%3dkpk;< zw};{BKUE(DHVYq+n$EKSukX{9x<~nHI=zU643=S$W&sn^2_s+^0K;2xaes2V{h9#+ zVW`$^Qv(ecFL##ePS%(uFjn z=GpIqv7hAMrJ|!kjSYo&WgR65+cG@^Z<7W*8G-Bw>w17Q?|0wYWv za_)a~QWCki(oWRdKL1?2$EF{XLWO%tsa8|h*n|a$twG6~U3IVVhvOsFc50gqnZ&6g zjC2eW-)e871YfIKE2ETDbLKZYDREtFSGNBA`6J-4B~-SGi|_>1c=O!eDGZ2+e)@A# z`8+ntGNalT7;&u0px5n~QD{7LM`(XWtsNhSi5#t4*5dU#xK37mpAh5EvvvG@H8!hi zm#>!=oUkVZ33L{Q0Yd1q(p&6|6U*UALPz3-w9OE~=(6S#YK-v}fQX&wj|Otzhocqk zBI*WAQx%&Yu5oW+rBwMy^D)z5O6xJkd?hAQ(KDRjkU~{efPg?D4edD=Y_9a@8#j=2 zeE}RGl;^8v{AId%mJo{+Oewl}gKwbWkvxE0C^jk4)E$i{_|O1`%(^=lJuqKAL~hy- z222Mg3r<41x}oCyBwvX6Bs$*1Hu?z&AnWgmSord1{S)Gi$13@qba^F0h`N=CQjX$z zCx8xjw=<>UA7Wze#oQq)!n;J1|4 zmyPw6vQZhJm7`bu;+~m8P4$*#C+CKyE2=CL$`rQDPrG^qqcTI*QB(l@QLTX42(K+R zS0}ZI`=_MDCeBTw3^t%pAOpch{Q5JSyzCVh26n9)6&xV%V{SN|n_O;@(!Wx50qrbA z7dW2a!2nFK5C#$nVD8Dci*}E4IT{dPG7Cn<%AqER-hz-)&>z^0#}hydKO}LLLrVGu z2(sECJU2Zm^wG^M=ru}I)Vb@Go^sT#PdAo;WWM&4KhwuW^X6(q-?aL3*{MKS`E&6z zKRfSpg9m9DvhnfJiYVG=J7<&@mOxo3pqeM-oVqX}IZR*$Rt|znZeg0;t%}B0K!_z4 zVE@4K|2DA(>!F@3w>z4>j`8sM&l#l_#mUFyl=!G0S6cn%JDZ+~i<=D16`XC$l@!u) z@_R4Yl(g`_A;>_GSio!mLg5PickS-#4{_WIIsJ>u5)#B18dX{{CUcB3xL)Vg(;uBl8!6P(Y7R01OkW^X

    USE6fHv5EB!mtD))dKN7coz6o-+WfP8ZkQY%4j5E1(EJ<% z0f5pjt-flh+6%~!;HFW2sp2}qj%L#cLcqWU_kfN=z#`*K=4p~)7Uz+-Z151V-e{p( z9*`LBJ@zMBSDe5ALPT*sni7$~p9R1CZrtN~KTs<8m?GXoO$4(>&ySDUduo{o%}#G| zf!r(Vbk=y`KT(-qO~a+FEm;Ex!@}EI;Nd^isEI&=4X^^yO|+x>Ir4tXykO! zlwUqCXT!t8n))fQJ)<9^B1*dL!*PkI{cakP8@YQ^kZMa>kOBR25V3}5Ys3O-D2yg0 zd9R||Bll%nsimOd;MG0m^@y_@m3J2H3-_9e@8QWS&%`MqT6&}~K$gSWPD5RD9R?li z&9OX#Iq1y6ZnLYizVOXZhc+NsNGo41kOIJE*mQV)_Hs0~+iT#ko}TpKeh)1=`dhW- zm{!L@mZ-P2{W{miL(kq#2*oRf!BL+0_n!n-ZH|X7a_}$|2E-vB|1$qM?!+wE_+}g{ zL^Y9-lvU5`Z|h zw-h($ci{9!*=Xc53Vmep$p?h~L0j%;eZkbDTO31w@7+_`&y$KFfraH3iyM`z&BetY zY&p5W&51JWiF@m`rNnG`o%#(QUwx-|PHfa>wt0PCe+ukKDwp~x_mgoNx?%mqY`nD8 zk`ABJyA6K0r}SH$_ig_QSvOuuN%*oVJih7R^cfUGI9l(Aj>KQsDh#sy}O$9Xip|9MY_Gx*y-2M#b~P6q~??g*(`<)xGxM zKPMz~Y&*0x*1jEuvL+C-k|ZES$?V=bzwT$*S8)Qh8WIP+u5|3XJ1OL$Aa;L(G%$w6 zxEz)I;c}bl^M>xD>};omLtfGTLQ+7zob@v^PT=U1u#O3>p3ZOLM)Za;(bS^0KGU*ikKTnA@&GJ-K#n zTv!jfHJx8)lFGvOAGRckwjY9L^@+?Eq@B&o&G$ZkG%G0jf%0sp%e%ScFiG~EIE&o0 z+8){}bBz2fJm5{pBvxR4+*DCF)6vgiD)IY(0u(YnGCA_C{bl@8u$i4bJpg$Lgd7b> zFXGMLO(>{p7kGN;tNMQWc~W1gBG`Ib(C4ncD5JQ%%sw5O=(m!4M@1oF95{*JK3X>e z8ap~NGc&WO4-enAdoxld+1Q)oZ$6zfj4ih@uyaR*a`3QtwC~8N_UXbB01|f`;bE~* zi`}gi4CilU3=F~l!cbN<=qA8F(I`CE1AdDQK|c!B-P+5I`D;o_Qg~*dj4ZrCm&fV; z@~9cS?Zi{%`-R{eBfaJH@bJ`WB&$5iJ<2DLjDwW;pzk^w6(&jSr_U8rjaIvV-2E@^ zlCZk%Ei52uZWmjqA*=x4PstNpbH`{?->7!9UQ0@b=WZIl$-OiV4 zogE`}b}zDy#y&IXvln=q>nR)lxoc31@ppL`O`@fjuS7PQy{m{{Z%-~L#ut8HUxk7q zE7_(RO;_(a3!!11@FSG%cv#7GZ$A0? zwkRlkpMJTM*M0RVk<7kB03Q!8+*M>K*;WVZy3M1Jhn&AJKpDapBs15Q+I}6fN44E~ih^Gv?LVAq{8(9)jrL%$K8RQ|B!Gj{iOcr-Gx= ze8f| zJvc8^uXVXed&{zeeSH_KL_^Y#S2Hmfsmx)l0}$E2_tmyT{LP`pM$v3)`@=%|wzj@| z6W-x0qv1d*^a#5GdJ5F97HwQ^CtA+euN!ZOz2K6HV{3pAXdgtWgtNHxs|eqenZ0cm zdKp1;a}6&qnlk8B?&r&R+0o!DRoPfDriU+Z9wh^@w_N1e8qQw8MvkLJR`AMTpYu*4 zsC(R`|9z{etxz~VhuzzjKyZi?-T7QyAyhT%sPn+>l)V3uaTV5x*wl@`M;KLd|$_Kfq-R{y22^O zhH+%E=%kDG*mcID(*nQ3z=E+KdN*G^#**njAyF)gG2wC(F#FwBcYndb${ zH!x&{dofcnZda8JFo2&jc=9gQzAraQdKcO&9v3I@!SUe?#TMe`-yQ> zDej`L>gVXOCJ#>=z^p0Agapc75@FD9`+LiISE*A!SZLTIsOEqERydYa0TJwGVq`Sm zix)Senwrtl_j?Nxlmh+s^7l0>lI@GA?QOruXV7c=!kl;iSJg8YJ)k1GcG|GpKNHv0 zeD1rpzdsxnSj^093I~{qKHX*h_Qg8m9_$jHo$I5Yb%VXNmNjlx=|v$sk?Qc8;b z>&-e8bJ_Ny0-j!9U!R@LS4uqqIgp~qBSt@efQL_|+e@^tzxV3=Qe9r(EEx;@4TWgO z4YM59?ZXM@0Jo<8JHUY)ON*VpyI1qHnnLm!V~0(3$ERmNbZ`-1V%sH1E>T!=3O4Z4 z|LvcD?TG`%9@{yn8appKSWii*#(FZl4S$iy&8{uWAi zAET3_c|3`4tzNhNCR;nLte6JSAo#xNHB?!;vf>4s<8N?r!@1Y1WH2(5WS{u4yfzR@ z^2f4jg(H6+L$tQG`{pxs=*lw)@00xC*ZsaQPg{hsoZPwFGwlwOFUsHfYPd~Dwb9tl zRh+lu^vj;h`?)AzGUhGCzoFcoOT!|r&0_iZLrl{~CC&R&@+`F1-SDYBsa*3zP(sSs zBsgS3r-{Jx`GcMO{QNo`U~?;$e07GLc9V*h;{`UQN~6w!VqsECiJ26-xz{_>`hMG4 zu0hBDHl!aD6O-@2#7x2xnJ7MXN5u1zQ&skIj7RGZ6G4T9cny_|T-EYEhVH(HIh-@n zYPTAdny?a6zySgVBxyg%a3NhG=`qrlRknI|y#_9pK7a5|tDWTZ*w3hs8yQKGRYgnO zQ;?JUR#hU}EA>rN)BV}25LLp%C>8tW5gu!mib=q+a0eBgG#DU+eVU~^z`IbysLe9X z9}dy1)MjTnlAL&b>LsPWE21SdOR?Ki=X5(bnnTdZutjsAXXla2? zbfG}~^QMYQ$)8=X0|);I(N>^v61Q`~rCHtvqve^2433xEYJY@K5vq08m7NQn^Ol0I zI-6o8rd6BR2nuhv-#Qd|q9SEvqTKX4wWoLSH(GtKr}HIWcHh};iu#Y2_YyHwf!4TD z$(^2CQ=Q(o<_eh?7Cm9Vl+Y~@3t@2piE~#-GWVy{Ew1%XKAMwliwgu?PHN5vjqZ5l zk$be7@X$B)$B#7D&BUCD>uRAtyt>$(%5^&4_XFC%s#>WSqce=wBD>^5(?lnl&Z-U` z94<^_>IaoTGCD?^=o~FBr9o0YCd7~PL?#IEEKJV6wx`fV3h);#9)`cW#KqmwPJAFq zU{mVuPF^zR%NN4;>Ly~XS}R*7Cdcg)X>vZ^`eq^?rzJt2xp_`HKhjzjW zm#=N@Z$er;Hyyu&_6Nhr=rfP*&=CR5&U{bLjQZs)fYjC|=<&n?^FskssE!?nwpvpz z9a^`aen%y`jrqxUf-im2AQGN>MuZuD_aUqC_Wu4va0x1e+-dZ>KY>5_w&x!zf-yr_ zuF?GoS?(L1p!DxX)3KsHd!kv_&Z;H^0MuOjGxGPYhn)tcZ7G9jPm|SDy<CW^f(|v?Z|l%x z>YfLLg9Tv1#bG7grq^k@ap8ET%+CrGLgxy1?IjZe8jr7M7g|#r<>pMK%N`#c*=edv z{|up&Kxhd1y%Q9_KA$p)_VCnfYRms}$2&9>=Jo7_ld0H^{U1^?>_@jJU!ds>w9+)= zjt*|Q(w)0UiVY?yjjI(cCu}Vz#0t~j1w+JcM#g_P#L`wC50j(+??F%krztH%JTVqzeUeUhjt z1(J#Z0(yEONi}?;xXBVc7s5Vquc8Vv^)GtZ7Qa8;FXmF=zuMPf6gw7<=y(zH(~iK_RGAS4A222|fJw+4J6^hm7y<*3<3W(G6%zwG+G_w#luFtYSlABN^qb1dZ#OKo zFnGYO_A*yhx`hhT(hTx7fhQ2LP|V&rY2s8cphDalQ*K0N7mDTB3b#!7`qoiorIM=_ z9dH7WV2Xo7*XOyHS65GhwqE}=Rj}hG>(?-i5c>~{5e9(L@Nf{QkwA4O=wM`#8l~S} z0_6CC3&;2vZPv>>b3F$JIE7OLf=|5}Z?)Uk{}`yJqz|G}7&%1)dk%77CP1Zum~wcM z=pihk`u?|t-(Nr?2=cjPLK0I{B4ETU3NQ?iiyVPDEOhnM&HtshR(*`sZoQ(iWvRS9 zp}wxk7i!DASFE_%@|98uhYClSR5pV#sDj$XyJLgM{{l55*V$YF9wspQMvb_z6bC>U zWPQFSAjnncw7diDt%W2(Q3LEToFiV{FEvAf&5+axZLgd02qhKj`-5hGnCy7LXeO%(dn4P#Q9=+i-#}iH_6h=jX)TJYO_E#C- zW4>^EP?2N;{{A+I=bTuan^{!y2vqsS*n+cw{h^uxP`-i5qB*kw2&& zvXB%?7K_hOvD$9FI5Qv{iyVXeVUW`0=fjitS>sLD?VBPe`$ng_Z|C{8Xo6))cXwgo zZ6(obOQqr1DgI&`(H#spAx;VW2@sT~zpOAwh25^J;I4eOu-|sP$nc9**!~R9#UuXu z`ud!c4Gsrj;Z)Fk+{+(Kh;cJoK;8-mSLt3}KB^eXEzOgaE0vAwj)ZLk|?!N&ag2g_GBm|M}2Ip5!-1a~9ZI7s@;j?H~o}3>p=x&^> z1s#=*xi*(!w@bo5aa28STjO)`pHxR)H!Gw_$}Hbs1XfqiwmB(nVS{<_u}DP;(eP4D zR3(2+tvxfGhH-QJ#mBv44*$-dZ>>?O)oKNYc-$hOE@-$-?i(laqoE+a?viVs|&+x1_$Umz8d~5dH zEem}6Gqz|?Ke6VtHM~teQuVZ|?yH*xKoyx;21ld1^I1TWyO*y3hL#LAi`dV*njFW(hDs9as(BfT$HZd#PNC*huFYt7Zs591 zgO+?+R95huE;Y9&+(hGiGeb6Vetr3MAn8bbrGtH6C#q>WD(Z(p2_7OgnonpSV7nh4 zhoHph!{cM;)^8D>%wYH*6&l95J=?C7#a<0nMW_RylCPhq5ZTS<^3RltB&J}DKlQQ& z8|Qa@hv(@#NZK&vbw;vR)30n-x2-casd3*;cVDz!S>u@t66&4q+JN>8rEqUuVXnrBI}M_3O#&$FyqKyBKCfxTnZ@{I{6* zM}H&^xL6E?w{7?C({myKtCH*O=nA`_-)rUl=R&UY&91rI8pgTAHsTu`I z`hlDix%iOox#$=@BoHg_b4fZ(@z@^C`1;&8|J$d%YYN`(5vtGYLrpmy44_EJ?Jk%aQjx!3?~!&B{*xsb{4+ zT+@dNYmc!&G;J@~euUhRIO*LQTs(Z-149X;^}UhdiCP)2sBrJF!9}yG%^Js7A^q>B z_NdkIvaRK@=+Df9K>T^ilo7DYR!vHvW(Ql#~x-BMwI^lumJymCNUrn$|Sx$BL9y`jJBdpTEZ+d(w~EIkX(krgqhQ~{qlWM zz}CvH#bH|;M9l3w%_#Uf5L*4dvM@vSKPIvNqyW(4=Qpo&6QMGy==U_c zB{Ys)KtevHtY=RPrN%IYaidSC~&|--D{NE18Y7AK-Q)RKy zkr9DXz*++#cw#X>M?*_PKWOtRXVd+0CUk$_%z!xaY*OIWe*MRPJu-_RDOvgFJw^d8 z-zOJaRYeVrJACiRzG*iHeBX-e1I0+_1R6P5{hY>~v#IWj#GTJ1dz{CI89LE7jAi^`e{P zqZIk*Pb4%Nz>*^Y*DZf%Ok~3Mh zj$4)d%Ifp2mli+c60nANudlE-{5F1sW1W(p>9oFo>o}2@o#i)>7-iJ!^2-0hZvJ%} z8MlpOD|#$BIh|)|(Kq*HF8I$`TwHBUZZh4dg}tS63B6u}N#DNwcEmQmW_1kQM-m3T zIvX1kE^NeHsm)Gnd*Qpm!b54BkV(CY7LC~#s7i1s{A!ng=jaaC-PyW#PDQw|pC?q; zC&OLY{WMK4`9oG`wDx6{Zh4C7%buYB=G7K)V4sk=?Q18|+m_Pu8z)psRpNEm`~G~j z&iFa6Kn-Zsv*>*Gdtl)MEUx7twRHBbezr}=CZh6!9J(wN)GfhYV}Pwoh_=9>__fId zAx}p_A-!Os^6K(DQ+usP56jo+Rd*`4RKJ8aWaQZu?m(T6rxuI!;c>~Rra>w}THUs8Qn z|+U5yr)SunsE7>|4U-6dZ;^D4OB%>~~p>gt|OTCI9bESI&3D6H%h z6coZlatp^aV`CjBEKDF83*8mrxKW=NJ>S0%t!U#hWb%(;bGw-k0U%VfVE@a#OLl9G zQwD~Ygb+B`RG7%2%xr}jURPsl9f#=)xT*S~1x;M5086XlDAHVI?-VSABL_(+lO3`#=0pZ*#Sco97 z@9TB*^%QPyTaLYEsEHL(m@Sa@Wjb&guxy?WkDZ?qnB`v{E$KIAXO zOqAZo7{uJ<6MK$=CpOdSsUb>KI3Y%3irvI%AInFW6hS0ks_|nJR8_y}O&*()LpkT^ zTNI#TFa>#WKmQu>Y|go%j`eP>@z4rqHI)7^JH0`U6M)d$FvA9vQ*CnR8y{Uex@~#! z8~3q&&7OJx)p?Ixb=)!|zIe1jNy-w4fE_F)O(u>W2udYaPBej&tuq}lG|PUqD$<+p z6g-Lp&7_>1d|HaHH^!>FW>#w(b8B184L}9OIXRHOyQ2K35EPLZF$z{A zqk=`IUz0-5;P{8>;9g6D47O8T7f znFK^C*Hy*N*2bdF=RFXtExbNVPsFRJtCtlAe@Yo3_()c(%T*I{kk18>691UT7H`Iu z3vrT z;uHwMiaQhw5L^qzo#GUC4N%sSxz4v{-{PSVf%F0Z#=FG`C zd+(p@)$dy zhW+M@SVd@vleLl!o-eznrxm@Ft|}Zlo{n-K^?10N%(o7;JvW#CP+4bc*1%q(s{J0; zVCU+z8Ks`Ju0OxGv{kQ3C5Cn3;$c$M1(2xk_f5)tQ0EHCGj9uGc=ij5>8T z^$)3Z5Ks8GCQMQ-4M8wv+rQE5EK71c*jp%cTR^3>_9){l=1(JeJElYrM7w_m8(*)TvyQDoR&x zV&*!phF+ekNHb=YydMWv;44qkTuA-sfD@K6GQd`}9jn2^d}xE4+m&KwjkaP_`g;d@ zHwiX5sVj>B0ti4}j*J@33_TGa2@@jp36x~9RoU7~jswvJ6Qm4#>_e2Kc2IBYObP~zj zH+{yi7au=(SllNqjZgY*vpG}!L11ZTgdY@2GI4;d(Wc~yqE?iMv6cR3rjsrj> ze9-oA9Ku^LEZguV8{PKnfj_xzy09{22{JA`u&}sO^zkJ6?CN!KVHzuPjE?@`Ic) z5h{YeaW|r6M6!Qy|GVg-mFYE=$nHPJ?}@`xKavwU*&stk4c9vk9fGSXuIHPUE}+Zd ziQiI+yosxrBf0)3Y=Icfmoz&oGY1`6ey$=Q@YVftOPpwPn~9{C*PZQ|@7d|fwF{Ne^rvlqiNy4bJ6gXt zcppEkHpds-C3JyNBemXi{yU6z>)Iy?3uEJ*$du~NYg1U0-g$bl zbr2pY4M=vNs~+6Ycmu!hS>5EdoqcoLoRUR(JnNcX@^V5|->}@`xGPM{Tp`T(K@gbe zf7cyyTICNdCZG&An?_-GwtH%gNi_?3#mrZww>CACZPR*k?v9b&CbCI9hU4CJz(S3V zum>7%$ro}>g(KK9i@M*~v+kh#1KcH|HeXm))D52&mta#<&R4@NV~T|=H$?l}r3QFG zQ`xVwLan7{R@i0f=Oaodd s`I5I@{1r-mqCy z4)=eu)w>}C1kh@wdAB~`@rw6DbO)NPBh{(04VUH{OWJoE&K?bnhGs5uvk=mSJ<{VVKVVpanC;R72hVY* z8 z?A==3HUzX|$-3QGdq&VN$SwL%BOlw%>y2bF_BGn5A`~SS52S#|rm_Tue|OPtG&Olv z9v>HbRkm2Df645dL@SI7+)%(y_lt3C{a-3Pz9$G6L$as%5RUtCFSl{Af!OG%YWzqd z&k%p}yQ(EOt!g8Y#%E8Gdm@+gS`Gj~ zClI6=R*Jj2aiyQgH5NB|UM#seu7uw^DMLG<=k(Olk-Mor z1o9gD+ME7BI9y`{qF`aD(BL7*M}v_E@RKI`w{M|9WDKJ7owU+)bB|0ikgDV9^0K;^ zcU`@!r0X;$>R0%0NyW@AWI$H{xNG_gOXg2Rm!V(%D76-rB|hG*ER%hp8mi|JTzNz< z3y+e!;S6p>0D~|F-g|K_X}FuM{6?5G1yRcuYCN0&H>$VxFMidfx}&2bM-JxdeSdkt z%?@{b{Lk5wi<7Po*C!DJI?_GIh1Bllc;bxcc1K7|SuDjY2v?VBRz zZPi8}xq&5Pxi1e?Py02KSFGIX`(XEh4HXJ2ndRgFwW9Ch+p=7w*|yg9-c{v;1k#pg zBwIPE<4cpy49P~#UPoK%pSQy6Ek?+Ga|G81*GvB?X*!!1nHPG`@_YRA)6>@=0K{gp z-T&%P{hOxWQ5(#o?)iHCFM7PDni@fzRAy!-ft>AD=m?$dsnqW&waMqq!zCq3iEv{M z9`c+d^pID*OO2v@e4-}Sfrxecu-F9Sqjqzx>k=X2XZyD^XKa%~%`r$ZLpmOwI?2hY z1ij9W{VqaZ=#c3W-^LD06V5IwEiHXOMeMn@>eg~#0=XWy0AWCU6)Ofrn1D~I&g^$_ zY%E*Z@g@4FZcQJAtk76WikU|V;bw3D#g!mI2IRsVrG<}AO*`JNe1t(ZIEiD6@cWgR z!JFn0=bD~6L!-W8>C*=y{^$R>M`WjMvavyKEsdDFA|NZ^9CqutF^_`w}r{b&z zQr4VRBTL4Xdm;ufO&}E^HL7uT`&~(n+Do^n*@Z0viWx=5X=qSS;#M2%2+TjtS5NWr zfcx&3k@MRJ(@Iz?kV1HbzcVBr@!P{a&T>$2KQHd$&<*_k0wN`Rk1Nf`k(g>@YgPK? zY;-U7oprPOXK$Mz06Gf_&oETRa&5JC<%r*obpHgur(gNDf<-$8DKpeK0qhdYu7Y71aL0?m% zM|qo;MVyKer--=SZ=>lo*@;~*^NW(aEa-pII(Dr)0(K$R|D=tB;CBpW z1S<{+DD?$33((2;hAq3AezbBM(XZs!)g)fJRL9o;Nc!Y@-;Iynljq;Msu5h_#$Mv) zE(-m;xceMNUcH*0o$Ur5#Fnx6+^XGq@GsWqD_BnQZa&m)!V#RoF)6dOTT(wJMNNl& ziF<{eP4<52>M*~=-l&t{8z%~lOd<-DgG<{UDr%*5C;mVC^IxrS2=(8-qqWTR^xMli zX3^ZUD~ApqLBYF)kP5_`Z`E)CK|%kGZaSvPs>;-^xzf41xki*51G045E@J-ehhXvk zcA!=-5a;DwC+F=513pT}>&GQ?>WsY>)AYQ=p(4#7?Hbrx&Wg&*Y1)~dbT>TtlenEp zf`WAkcGCI)Gs`GesG;xkt->D_JSzfm$<*9jTmUo(iKYOI{s(&{*}s#x9|asgIWT6m z)X&kk98X?odb}V__(?yV82uJO4Vw0sTYM!<@0sgqlSXy52EREPI3OLEb~oa(j={mn z67$ns@xd~qDz+6f0%8sV!`JQ;m3mn6$Bx4J(tT^>~PP`W+WQO!5=7NHJ;<_P92mwI4Gzc(zUj< z#B4{d^cn#9kw?vrHwZfoNi+1^@Ir=%lLR16=J>4)-sxWGh>-zCp!Z;6Vj>JhV{<(Z zLiSX&_mKf&^ylZ1n3&B4Cp{5_^!EVjfQa^XP(Vcfb3XFY5aQg&UqLB%f!x_o1AmwK zbJwVbad)c+FRVC(KbRm=ds+f_J?vy;f#wm*9NY}@I>m-?>Zek)i%77tnm4xug# z^mPw}X`Py&0UImGFf{p#gPrx!@oJ5ojF;C!-1GBGtB%LXc?q$i3DPFSVXp1vzH_ch z&+SrmhJW?(I*cldE5^*n))tYPZ7sQfN=1pWi6K{k;&D^1p|J}~l zw{6NWMB(0&{oM!OTeO|9uF0!geGdM3k%Grwhaxe~bmXkrVgLF+;86*+=s*ufrr;$>?l`%D5^6S&LCgBd-KgG6t zDu=~VWJ8R((P2xct1CuMj5g;T&k0wy3Yb_ySpkK+h|#&qp0lv95WC;^HFNKy9-Gn# ztJ!GehzZKe{W=h+xJ~aZ&nME+wAnR$IekAlq9WGP@O*A9!h~=|FANsZ;8jw>`M5EH zg^=;ec1Mr!^7B%ZRn11+TwYpZcPV1y;1nI6FUR2qnfGO26S0vKAdffP^Rx$qN zX=?oHu%qVyH&=A?^*xkC1b~wxLd7NYBI<>*t^33AU zC5#Pb^gAAld9V1%8)P+kX|?ZA5Z^^uoT$%HWmAsV^+KH3vo=08SP~IA1sR)uo%K^@%0xa5U?4nk0D5qb(@R$* zJ$x~$=a7+PP@Wm8}+rLKDkMGS7#M`C9mKgaNIluzWWK6_wA$iG$L}aQ59_MPZ_j zm1P5Umo5$+BQ5f+Pw__6%Zmplix-IX`E=&Sy+JAZwVmhQ(hgdjf$$3x0R3;u7hNqw zFAg^r0y(U9Wfox>`j~lr1(&UfrxZ@4l@<6qw#bWiR*%#BAZ7*rv-Av#S9s*JRRqN; z^`1W+-413)wx*_ty!euT)UpgIj^skR6+_7#=BPyKVajR9UupH4zAV=HP3?@a?OPC} z6Nl3jo!(5thK+t6{;L$4Nw?S610i^V<5VoZ7bpBByTzlvo|!DA6%}i$tvf0FYt~0(oM@WA_ros6Ap-QL8+HW9oqNXPHG@o{iLdzz zi+|a1XJj0NPD%CG9Ros2F=4jvH;E8)%SN$k_=acoK=%C#Ve!cl{zp8|fIH!LCUglb z)Lu;o7J(}6Jac+#w5X}(Uc&GCw9VXO!>><;#yuZW6TMJB5CoblDk+`#`Zg8NVB}l} zmg*i6FGiK!dM|7JY;+~pLB)4|qu3MLM?XL=d_P?Ce4YAh^!nFC%bEY#NhY$!o)|n> z7PxS)B;>9^nUNlAw_Km^i})vMRgqXEj!RyAU>O)d#PUtwgl?!xITb%SYiO%3PcFYh z-xg^!zqq}OewE2Rh{vK2cN!v8C)~pB>2s?HF-h+RnL&e%oyM;UOWXX;);E6t6Trs7 z7HxDpI%Q~_&UxV|!}`ksuy&Zu8X>2V|GP^}^kuc(%haY1ShYC$>P=emX{E5hhiPtp z?jwwC5x7g++8!wt1$1ctd&9 zsyrx|riUTdpKE-^KA#F8u3|kP|2h7?;KUgG28}EBW<Npz~@q@;KM;$GyEh|tu{8xz*)w(IIPjGoWavQW^X5{g74&{3#K zj6n$kHwPgI8Uro5y^x$7u(G=PEcf{!*H;;bN1{H)?Z^i0Y{l@wu53782nFP6C!vFV3GkN1$sQMJRVgET^^Zj9g0c>KE2{v(7Weo7@x%o2egqXmdZmFG#HnT2 z(bVosrG3AzYIy`_lhv6E`ekfxzH|82i(>n7yMKY(a<}{ArlHglqCK}}6{BL2oxRN_ zqBMs~wfq1#SZfNl0Gs-q_$9_CjgKb=5ODxPsd;&L$MIQ`c|=4W4oZku{qNS3gmP^T zvOBX86_ON<9otAu5?yZft-Z0V$l#KQ`o%?Eb=8?E6|Ff6lAOr0ta5aMAM+=zi#TIc z^ICrU+m=^0f)?O}=x+^LEFx+eRRZ7nXV=<&PYxM!eVy0W^F@RZqjU+a-!ddm02A<6 z%@DnL(|)@d#TcC$xwbN_;3&{)qa^$5aNE(Sz>VT|vh_vAzua)R=@QaA@idZ5!)*jlhSF;FTKPpsP zO@8@eb$Uu(Spok-zc)B^bIMvRQ5N^wwSM);G#ce+Oo zuXegI-OSw`J(12+>oYKT5jVsKW0m1J3Fp z1lrnu?Y9w`XvL$C005V++)^*Olt8k)N2Kv6Sm-vkwp-q{-qoyHl3!`kqR4h{}_dEIVb>VtpqCCLKJ@W(6lny?k#m9q$Gt5fD=KOZMeov^Y( zqXWyB)HSC8;5^4pv0uhN%WJz+N~bJU#-umJ5pbvb{&@ald%ShTKutcPOn!QNJWXBQ za^P+lkdl&eQZ33DZDwZS=xsv;5FiMGsy(k+*xA{Mi~mw!@92jXG~`#6RJUCpHh4~! zvn^?8bm*af_AISFvJbPS?+6M3h5Z-h6(a@{MBJ>wr~qKE+Gm_(k!KA}-y(03-EBD( zz+7KU0O_#M9;>L3fXFGghdp!R{y%M~iwQTP$#Ai_JcdlODX(R#30O}^deHrBq4{~X zAJQ2pYW~u@%f0d5gF6aEia>fJk^r?oLO>4uOIDuAO&pzq;h<=7>GOIs`v^E0_r`aLAp+l!50&h2}B zeSOFH5p)u20*+(}k|($b@{>qd+;KHyb*tu5 z!I;uftInr(XYWHIbS6@U@c`j>4qZn#7$U_Vo$cKVDl3{0E$BBc>Lz5P>jTVnL_4YQ zENM*;px*mI+Z-|#2~`kczU{xlZ7o-H(;F?)ZiIz`TfW<@l=Ga7UAXI#Ol*f@ z&#!FH*$rBI7)e#aXRB&T25o%Pvq+A4l%}9!NNmwTN>Fsg-cJM)2=~&P5-T2?CIQ~e z-~dQ*LcJBKq1Y?|mSFGn#AH{`7nT16H)Q2KSdUvevzpT(jA_s{;0czi@$1~|TiCdMb#L*9TzdgcBKysXs z$kbIsU`ctqxIjQs_g8OI`w!OpA0j{^oIcYBO)4&t8s>13&?2N0wOwykWB!?rVh zGynz}^1uMfHZ}`hZP~N8`V+6zH_TUmzul}c_>k6_IbUm?JqAT{S46xXWf6b~+EO>5L!@vvtMrIAbFHB6eo^?zbx zAf$nG3FsJ)`29V~_3npWdKaxOP29gc#pw}<2kfoAjM)2`7QPU_+>4P0Otsdt|GIR1 z+S_x0`Ag@`O=EUm^)bIfJWw10+NW%>J2@ecZsF58!%N|ziB({{5`Q)Sj8UBy4;ZlD zJT*VDn58QQkj%S1-ltl7AVp?wFnd^$w)j)@lP;Jh2FLaB*l0-r9f%W<5uuI1MSKg{ ztMqy%n@bEo;s4pJ`wsIxD;cc$H1ksD0Xv0SP_uzXuC#QI*hhsRfaVkUV!$<{I@mX`8yGEnnU%>Sw?Opz{0HZc$hn0Z*uJ7c77U;(C+;Ne_9*Qtcd z3UG?v*5y8u06s;FDFXo+Z=argI{(JMUN^T2tj}~4 zXcz<5GKb5YDYB48DnQ@=9&%|mIdY6E{+unlpZnINmxpx0AZ75E!k5*t=c}ABMF0x7 z^Tm0FUYb}VL#fi&v;_Qbluzq$qcQ;y02@HcA+u;VklfD`-$)Te|1lB&>2Azl9w2K) zdf)kYAttmHH2iGO4LDaVK#+4_%azE0*BzrmO&`uy0e~X09Wa0=*5Zr%<<}e7!DLt0 z#lwyN;M9_H2Y9z7sZ+$`ylGgu78?*Lk^Tej+jd<`%py_zv7oZ0s^XwL;@#gLyI=;? zwlV+9KqUXbtgjdW0bte(k2CllS1dk~4z+0|fAun4kX;LsE-OtG&6TT|;ZfCv#RPj; z^)ZIYfzv01uC$g8#k6L<B$L;tQ4RD-3$3ogLdbgsJdzHiFck3WGa%Vgux7e_%31le5CT zxoVuIic@Uoq>BUb(fcrTKxB4bzWDmXVx%;+)=K|;BgT>G>yshTX0&!^yZ<#XCFc83 zBq-?PQ8eJYhQjlC&uNe6TUkO^hW+Upl?Z{<~gBxSN)Ua3MW;^(7 zvADVmF^xA|^f!lqwnTRfIln9!P7~QHCoo=d)y_kF9TYsdIRE$f`TZ16+r13w&31?{ z4(rML%x}=**i8h=7qg6DDM79R)9o+g=o-KHv8>;41)1?IPkTPxJ*A||7WdrQeAJ9X zPlVucN$AN0K(OWVD3$T$=s#~DqUo{IYT3&Cng&+yDsb1BV>#9mA zqAT3vQy9U(hTTRV^Bsq`kiyf!R|s$S1u`_0Kr+TNJ-q&YU<%nNqDIB;W2yy%IZSqk z_X?rX8_0jS?~C_2M_%1c z93z_Hg1;UBg^)(?lh9hf894Iq3E4?P^kfYUKv(~K@SnYk(kjy2%qhbA$hXWkuQiln zRiafCD|OqdY}Qsg0-vqGrx++hUe;ZZ)n$D%knvsgR?cQ52?pE~LgnrQ|Il8-w zzz-n&y2|#j_Ob*sFwJnXsPc=VRe$4xg`s_%?|tJ>`Gf_%9!7;wpfuYC;|MX+&Hr$* zd+2SnTLQkbh0fzqIa({P!HwBB&5TX z>Bog(c;QJARmq9HObw`7-g``D?w6(*)Pez1ktaCDE}<-XRnhAn-|z<96^D!GZVz9usxP+oZc?K2;5U=S!vwHz3->p+qYci|mu@ zdb^V#GRDPcn+vFyZr*q3@H&*p+gOymnU-dKa?4dUfbgFn990iht@%Ab+#;i8dpLQT zZGJ0g9c4n!NIfq(qb~NK-!{y$8{m$LnhV1^s_EX_^@4?DlNew;jb=5-;P!9lD&1Koc-HtHS0%2`K6w;A2AoB=w^Savz&mRzn+f{q#^))#aa z%-i}Z>_jJ=|BJLc0i|8dSUGHW2TlC`WtLgJCu#AYJCensB!OO2Q!=>s)QAE1uG z8MI%$D}*#g+7k3}N*>4oN(B70c?;l(|M&Np9rhe0Iit}wczq_OPdyMGT;e0Nh|O$* z?o@PrUBTY@v$K5jcbp3EstxweTaD8SL2s6LMfwtYR11&^<=^g-v|+R3Rk)9b?ouEagtj@U+-3_x4`hs|qeav}9*M(j%RNNl z4r}?oOqUB~6W^N1GGq~%sP6q_0{r3iWp1Su9+4wPw8GPU$`FWj)u^HKT(u)dR=>s5 z%hVJvf+kSR}w|KOOC4+3`DAc`wvkYsyDo3eD)&T z`c3t+@#M1t5AspfRcf*)LT!kAlCcT$vf$$56%-&=zDItQf`BZTG2@48CIbHpSvXvF;K2(0W?$4&aLMQBtRNl;#oP^W6Q8q^!mj6Uc*;{SX=G#XHq} zeMiU4simQ#;r6qiXVc)sLs#$2yMG-(Z$%(7Xb}wEyuixka>{$`azP7{n);5+7?Dsk zV0N;XX69Q}?rbFG)?t1~d2_l9pEAXOBxdPE%W}nGkRbtOm5CC~k<$7#MlxmJlfAEa zKZN@RLu6sbfA2I~DD|Sf98r}i?&rFyx)wUAWFB&jjx_LCdm3&`7K*B%-db(8R zoYCLoek1v7_|LUxc}9S!~q$+ z-Uwj4QOcPfN1hoOawd-g@fVXiQ&Zbu`%3CWPCS|FB>#vdeQ(36UVvn_5n(E;S><@j zq|;_HHr0kuU50S{2}66neMsV-GNJBB_F^T7A9M{Iqz-}fl8ki3nE6kSWtisr5Q`yW zbm-L2#EAG>j9(XnirAy}2m|Y{!z$e?Ikhw)0~t=E2hycHGaZ>x!umUb$#s;N6faIM=LM+pU3>n-!PU|a)S!$ zB`eK)#Bp`|4xyU|Da>=a#YQA3iqwSQut0;U7;9Jnz^;iUo&u%ryJ#R(De;dfne0Xw z7?4h)im|J4%+SyfpO>+Z9I8*|-p(+txJ%fd{sx;>$*U?M7Y_*~FS!f78G<9{EhiIC z^Nu1a~dKOh|{9RMT^x|ApU)#&lr zjQoT!0Gjem6%c?ymQSe^5omHl$cUa-|FqDd?qD%z;1(Vfp)fFYvBxX)ML=zm+1`$go-{z%#9jDDWq(#RuTwo= zU*U>t_=YPxDT4u4$XheZqON3ZicTXyHx5*xMo!KOmPVB)7`8;By=-Ph z^fGaeBH#gS4zh)^%xNWb(ISzcM-T+yI}Qj_hlD9oCzj#^lZYgp_ir%Fg3w8w2(rME ziNQc&AnzX)r7JCYv!G_POzPNhdEN2rYLj3{sl+?3K>a&l#T1hyqTO=733VVgZaM&x z_px67RXbRag(}*NhYV+z*jH|cDwrUEj@JqrtYe??4pk7oI_c^Buyu2~;Y~qLv)FeJ&a*!^BXGzR-dW3#!yD* z(9DtFou?ILk}IEr5x}97h$ZZz>`?c$6_WP8Bz0~yQbw?7*|{#u0bM;Hu4)chwgEFs z>qg8qB`#cS<>M$qsNmecF%+tcHsx`K>3p#}Qll*)W#+G1%9{IU3j_5D4u|jS{wZ0v z?f+)7mY(Fu`m`r`%iMrco4$KOY&uO~JkXphmx#gSRwV_02N2>TY+g7@kh=_Wncg-- z3IgwlGLe?&eN?1oq!l;KN-*CWbKDd`l2tzRj+6WQUV<*Mj*pvqK*Nl1JCS5QhjNsX zm!^zF16`Si(KwG_*9L_;!3jY979LJmz&nPoK@wOF&!<#Gv!L;3h6pJe z|9}sa`G)NugF`B1&B#d6pT}T146M)7NhGs8WV$z+R zzh5Eqv>IDXRhaawb{iR&~qq%Xu zosvqW%tHzA6gI@e>^0?-)Q-1{(LQzn0D!s0Awhvwpx3e2EXYPKqWW$luMc{3qI*T! z#d3_HU8;M-+wTGu3%8MT_TCoQ>cuq*C4NRjGe#>_x43nGq+C*ghyyuol{oRpSSG) z3h$)dWBr}=Gyn(yVM^eT#-F&Q(U3=hd=$qip0{g!OWG(pM33j6wF1RdscqNP%18;0>~3|>4wZ>0Ka?<=ZiYwKKFGLzYL zs&4?vELWO~SKGt|zwW#*y@$|%{jc^?Wv0zxe)rV~s=Pw>$p62`4uvctHr&AjR6t(s zB!WoH*QYRx`t=v}*yYFOPD5OXJZV&~bf0v}HH*TOQ392HISh4?EV!31gfW7~$=V?W z%z+f#zoO`w?oXs46cO@(0j)nPPjjRoz1rQzzTvP#R6{gBr~Jz6_qLNC3Jpj&n=0xmFzIyG^G zlf{dbP_VyvW=|(s%(kSTNG-`{eyXk)31Xa z{abwPN8e8hGhrE6QEh9Yty(|eC7CZC_Q=Wqqv?N@l*j09g|&ppwlJD2erEAuD;C4( zY#AmO3k7EU%0QSJSkr?zbHx4n76@ujjHv7#0{7GBSgjb5NyX$)Ok-DOzZ$JDt5KbH zzkF;|9okTPuMzbG*QN^(r`~;w7X1EIC1fO~$`b}0zjicyt?z|!1dqtE(C}f;t%xcr z(fJV?#&_iSKF#_N0Wb}{*TMyJzzq9@1bxf2Z)2WX8Sg4kH~a{`i1ZBRRF2;LgTtbf z#Ub)q4}>gwc8}<)#NTmM{gE{6S65~J5KzsBoQP7uruCv&|J-=($4)%;xn|V2R2C+C z2xfFonk!7y{*<*difJ#RZYY5f2pyU`t^eu+4Ov;hX*6MGgD=EFr)6&~b9?_sme~I5 zM!U55heFuX4m*71=8AmMa!fVl3= zv3H*YhEqT84)t!5juHO?kvi1PWYDpVeBB7_)Ecrg^{QK+NU1b>x!wIAiPbW+cUH6t z39yB0^pC*yX3ne4(@5z15233c{5>iBA02W0NtY7&RCh=*h}`Q4jVdc*ZgZQc1h zfU=uUa`%yy@OP6W*Z8j^FDzID+Zz8@js4$(^`9N61I72?+FL<;Q{Y`lyggcS&OFk@ z07G7M-Y%=(^1PS3S@#<{3C)mh0!}M6g`)AS0=WTAo~b6MfAVR7;`(jqHWCvZs7N=( zoF{W+%!BFeNq0ZH!crb{cpCJ1o)ey z#>?=Gs()WX<49WZ8PtEU7_7Diod|ngrV9WLoZZAkwwytqK!ti#B~3*Z$B)$%cu2&H ztp6MT|2~BO9exuBRf~|JtCzh$wI$p>l;O$|>SVi_e|)7o&T?9hzTQm1)>PhmG+@KH z_gN`Zc(M4Aw2jXK%02Kw`riQeUWC?d$(WEyPq&Q&&uF${vXxRf7i7CI|x&BsKE_2 z;~JyKJhYb|-W@6#;rm z2SkH|+T(2E@B%p#p}<U#Cs4{jjY;a!f7@ix*xE(w96)RyYkukevDbcn3=OOe$Y z1Z(N=xx=U%jC2~r%75O!4IEKOPRyV)S8S*fXe7yPsDj(m6*9SJ!dRZOhMD= zEvRf3DZbmN`^?_@UJNkY$B4m*w35*l{BriahZ;uh)y%FYNSi>!J&UZxrQY`m{_Q>T z|2s0g#ztY6qPgnmNo5;phJ z?;ZpHe9)U0_faMJtk;O>OY5{XnUH2MlXLL_APdUp5jwbn=@g`Io}S<(6~xO^X0y<{ z6$&$_j3(eOG+J4AvXu!s!&1=IMw5O8q*J#0PKb~F{XF8ic{s7VfR(IC&gPkuz=oW> zs(P*{!ql4TAupNMM^{1^Z5E38Zc@P_y_zdA;|-9SAmOKvWhX-#QMn8>RM_`b{Cj3B zMuQiv1?T^F#=tgiTd%xj`vK^rNCX^-y-0D=^m+L2(edOZ9^%im+c*I9NOFtpWx04P zC}khi+Y2~IF54E7w2n-euh8+e#l;Q%{TY>Hl42T*x|{F|Y%)}tu95?DP*_+DNipw9 z<@y?GBO^<4=68{}D^A{lh=_I`o(z+F`kkce8{^tPi3yh+KYq-&T?|ml8>xTN$HVrbR8wE(#&F6ZaZQnlE}XRI)I!UQV3};D zxS!I?LPO%(zProsp01&Hr9&%nJoxyi0;8hK>l1nV*2hD%+s*=)vJx+?BQM zU-puOYHfcLZ`or`85>$8w~q{|lnWQy;2MDTWc)ue_nIfmJ=x;{xh`*qnBRMdHW(mZ zAbpA)sw4hip8DTp^?w;_HK~#JYV=M35{)c7>D@15|8epty#SWXC$ zIwn3wpzZ1;&XIz={CN-(C?PydM!FASe1!p!1InUO0{~zEl0eNJLKWijJWaS%Qc@!H zeFacDjdmbN$CbS!~z61|Mx=FgGCVNNyna!5V7g?n%?eM8uejO1iBEjx4Ru(uZ;QEw6z`a z(<#nc)3?aS(f+LsA~5A-z9^0#klI$98I$iGYJpeI(%2qMA@b-3cs`<3Yu2rt8 z_R5C|ZPWMay_a2&tp*V9;$+E#&7f5_-j(j)c=-F>otIVxrpZW-M$oVuPlLp79ZXSd zEV?Kva;$#iR^z6#u z(}4(aJm%UP`+&mQLF7OXVnhgjUk7xCu|37>U| z!NXY#xKL0TMzE|Tzz*!HVV+O7r|2zLjtLJ3X@^o~PCdcUC9sMl-1)?)4T7r&6ISDQg7rQkk$w=lyPN6?n4LnOmTZjg8G5Y)4X4 z)<9@rQ=Lm?gn?&>u1^t#nY489@Ns|4LwO$rii?%a7jd4i@irW3`eVw)+bQ1Mp`RPb zf*`zmXZrn@VN6x53KsaLjOja8FQEn&3|8S^Y8Y2Hw|19K zll?+~Mv=>l7)1~BfkOGTuIv{3-S>y1qrD;tNBA!X(W)!!M@xBML zp1rF)3Os~%KR>RGHh1v(g4uyEY!^<(cbbvuwA{I?tu^KjBPy!MWM)HtE`Kj02M4Jp z7IuDLI}=A^e{--h6p+PX&3|Wvx%0D}xOjz9_Ba`-dk!y@NSL7MxF-y=2uof*mh|jg zKwA&3wY5f3u}#Jq8sb%f4E)d=DJ%>J5#XCd#=s=w4|fp-bkIJ`$2IVOJLZw!v)WLC z$B$Q@mcb>)4Mt$VKy<*o%+9`ai!Fr)GxbjW$Q`65mh)MfO~XfckB35sw7tf^OZ77s7dUOQS1{aPBD>5)T!_F^Lj2jUKpQeylR@_lQss)~_f zLoGi%K6*L~!89)>v$e0pIl1pWyuas$SGyzs%*>2?eK>e{v8k-2S?l_XO+cW%y@4Eo zpO?q!a=S14oQe?x;z2$p@8EF8X6fqrtFseaQ6spqz8wA?6aoU_OI3EcQR45cJvXG=0eL0 zF#zSmqhdXu3O@C;>-UActTi{M-QFsc%3}FGZ{4}irhPo+YU${Cm&340O7{N||{0>gsxb)z?vtb!=`+!sao^z_8xtVA{eJ*23F z9ruQzUK%t$D50g^O;@g7-sXzpfp8)Hw34&~19|)$9PN#bP=53#5Af9T@{fMv{aOm- z;LQTejg?is0QS0E*);(_a{r6<=a462n+k`{CxfG3z5&+rjYs$Q_q+E%9KV~**XQ`?u_YL?PmV->poqNUa-4yt>h z`d!zcU&Tu)*=3bgQLd>EE7^s{!_R+ubVlCkeKysTCOl(z!LFT5B&R}U@8GfEQ3zFD zXbA}k&D}lj==lY>xtnUQT&?`i!@U_3o;Is0ERY^^Y|oj$e8GXCURx&Rb}R3!NqxU( z{V*~iNUMq1^fz>UeZ4!V!(?!7FeiaueFU0=^4B!n46kxM~B{E>u+=os2YvOa8aWcChV zl&QXvkc9!sM{m=N977%0pLn=Qrkb=g(wC0MuiMjCAjZ@h{nmBhXX!o1y`!tq)6aZ? zUcSzMOB4%zR`{UfB_Xo`6mvd&b!w-t2#R_1Umg~G1<-MCAWDRXJXU~PFBdwbC^8eYP^lM#Z-_EK$K z9$T8*3r902{K@k6Xa|a~R-(+5S`quV_A+5>EAK&yt!R2`^KdX`#iQeK&tmUtD{V$# zui)hfNp6&%pVR*u!k6DPA!`?O zKf0X$+mrPoy4YaIs|=2lghhPr&){nLyIE1wQL2Hcr~J$El;FVc_=6}zB?&8L3|d@t z9Mk8|r)OvSO`Z?^XN|Y#7R1E5!S)9phlEHbff4w}-io*SP$yJVFMDsdP?SE|IUdBX zP~P5HG|X{@iDcQL?x|YhMzHVgh+#?v$djD2xXWQ1iCYDwEk~v2ow2<5j%h)U!lgQc z3_l)A$SmyV$UtbQ#y2Q{pq0O|g9CWAh|yuyTdwof)zoyvYGH}(K)?BBY&KfVQJFx> zI3OV4!4bR_g81b|zuo&bW~eq43M1QGRCh2A^%gDWe~f?WeV4qmzL@}Y(%H&oaf02Y5YhkxI9$oG47jw&`U=HBOw?*PAo2W1g%J{}I3Nsw%(~iiE zM2)OF3wJMisx1kTMeDMJVAPDwXq?NMJX^fO$s42Hzz+&S%r=gRT&PxuMY>H+=Iv4lZ?0n zvg#^FthUdaXlrj^>>gqaPhAm5u!)W}&mHEyGbc_VSg%HLdWs3=joU47cF8#YGV#VqJrsIX4+{ear>07s5A&Dh+dh;6Vr@;I z5?NGa5lDxIZeWm^~P8PfC_d?g!q%oIt-3q9;X;Pz|Doa*_&!I{I12 zy{AeJ_pB@@!8(vGa?>vuw_qDn{X$Z*FvH{s4N}>7YTvkKlpNW+V)nnZD$=2LE4Meb zv9LG3T%J4OLn_KD@Z6)vMifclx%ge|G9DyeR?_%1T*P+mQl|pI5YfcKvT4w{4P%Xx z#*^qfXM#b<>~uV;YFvj~9q)fkqE#{}i0v@B!=#W_R#x)yT=8oRv$VXf>h-GaPeBtoot}RJj9l#Wqt-$~dgM?s zHF7?uPU$nHGW&Vj7#SJ0%14YjFd`|!_xd3BLvlr7P^VsB1#e(Br(f6muKl3eFrw5b z0fHtU22H*k&(+N^gT&D5x379xHJCKJ-O(SKh@W4(Xl3tSNJ!OK3|0w zY#!oQV-)G|6|N6`7$#K?iBC5}ijJHtrSon*21@5N9N30I|Ky_AG7`QO~DOsmz8QPY#py-4mN7Bp#n;dEYt zzA|Y&MyFhlOwJ0LWPXvn`b2|Pj476*FiK*fOx7B%F4khh{X07>^ zs6%JQ36{sBbFB9dv!__6W zot>SMN5v#{R6L12*>=KA@#EBhV<@o()D!?Ew+j35WBtw3(aPA#9`co=5cDD1T%3yv zZaZ{H*_ni(#TzA5^v@}5f!1dRANV+QAg-xOETfaB>mig#$wTN#G}DHagj|55(bP{- zXBBB_WzFC7O++VNmP0`lPCYd>bsR0T#d~L7^X?q6+<)l4x8dYdfhlZxwa&vqz-ylf zk8|$|9f-3O=(j4Cv$7;me0a|K!w-rt?u36$jIWl7 zq5KpscyP!nXUE51svK->ZPz?cvc~4FN8$)Q+#v`-s_~K^$T`GZ&sUo-Yo}g2W*FuI z3uKG(8+sB#@iFL9N8~Bl#AfLk_UO>WW=Z`S1H&i#$6xnG2tz2a4q)8Veq33+dSa?D zQ)x2bmxzY`y>B=vC@`hmJrH{HT7mpKN<02jD z5bo>bWZRvTfc-05gu`>p=SSozi={z8$3}>PXz!e^7Vr)}k?<&bvUrHK-!qA0?&#>K zD#i|AEm1@wqfu#kC8f*SY`s^iho6HKkY90E5`{S>UoaToQ92;V*8t<~=r}0>MG^6G zbaeDokC-q8x>dK-)xmd(;E(p%Q_5ogLQjM?J{UqTTY-dynT1E=2P4kS~bX3p5dmfa%eAs zPkGMG`yPjiFbZU-u+9sxsQHJYglFO))<%8of_3?T6@^<0h#K8N{42f=JDHKwa{T)0 zub{dc_%ZyNcKj&h=;&_U$HCQ=UD|tedU|^KQ8JLB7gzxd42MGEq4Mcg%sC?>!$#&y zhyG;|?Bq6LKrn9BqoIP6sBVG6u(t<%0LHF&a?CF*xYTR9$)-Zlx1Ic)AGg*TN*G0h z0K-Hg&t8=ID0|WIgs5c(5QtnJVo-Ro{+uoxEs&OKhHYjgqat|nX*Y#2LEQK4#34KT zjF`0fY8?s8{Dnzr>~lY@M2TNcQ%xfnQUzVj+88QM2tz@>`(5suUCKp`AY@ISCt*1D zI_6vb>dOuO-~uTOGL&AM)*hSYY#?3G0zE!T`sT;}uO;e-v1`BF!YylwR=;xleK85Q z({SjmYJ({E8X;W}et5a|0iQ~q31OK__2V~Fi~Flh;kTsrvn)Dy$0$v_a^B7z;+}+j z92o~CA@P@e*P-=qSV#f>Od{|x{BmabfpjJQ()zor!~5%dj6Lur zn(pq62$Wr!WT5!!)Eib3n9a>!r`ym zTBQ_WFcNRm%XeRXw&6&Z&lNaJ9L}cI)z^}iGFZI{gSWPff$CZnijUq()M=om*u3)J z4eibCbzcPqd3pKUel_U`eC9c5*-_;S^y^OB5!2bBE2Rl6e!E2wFH{x$45Wf8k(VCB zL(F=c{q|0t4*x5rO0%<;({uo2^|2pAQHQC>Th-`XaRN($`S zupYNJEX_rsQa(sPbHPf<@u#4#RIrHp-vdG4bIT1zaZG?~t82K99v)5}E5Iz*JBZH` zep=Si?<(^5!MUh{(dvvv0tZms5|6TuQ{lsQ*@#UNp|w@g7Jlx@H%|wjfrR_`!bFYO ziaT1fUknh85h$_l={Z9(W*h=+&;({?ImtQS0bZs(`cF>TWBF8WBnCPtEaJW&KV}?5 zT(O12TARwIXJd!+(dNa4@*E-s5Ff?crv)1OLAcKe+^M>Y#b6BED|oH5fX4PBc<04WCnOD zNB)E|H%#Bm^yJQNnaHp1L+NZYs0gBx`#I)uYQ8O>JVOJ_ghZpg{`i7+9 zf5n&mqUzzK2o3?8K8XPsQa#oaiS=Ie9m;4Q3o=d^pK$pxX7^<1Ly-%-#k&7N``$p5 zZmSR6xzHT!#}uCb^#vGsb%ZY-JUUW?ht7@9&q<-u9Yd4rU#Jh2}h3eDQonMP%|Vp`pi*hth>p`{ROYwW?*%1tBYf@KNd2 zE!Ul(_rZQ|Z}03VReA~zEuF})irdj;Qo?1eHPFM|V?w{hxw`YMd(Da#W_AB@IRjTy zlBU}JJf*cIYQ3klwe@MV`5LLnPgpUFcP~@zAb(K6zlBv$kRf4Zef^#C>A8+qi-yMy z(j3mdc;(GoCGWA2U-6sz1a!tVQ&M4ST7Uyxr4wL>hE|u@oNSv#RmzLVGR#?fIeEGH z+Scp*LjZS8Sd;njFu>7+SEW}RSvJ#BZM!ZHE>up{0yxCWs1Yk$&d$r*@$9NEjuIN| zf)VIq|Ca0fx?Pl&B@m;B7vVCuwY7D2c2;1G_hV~wGG3^k9Y=d*F)cHbgp_#YpIY}u zsVtj^BKlE1A?;m5gVdMTaS!s#ZNcZBBPvLS5YgJFp<|s7KfvuSPn(;Y>mgx^uKxb7 zr(Inu;I=zYD`SgBK>16_qiT;T%HLm}=c_#6w!X;eEq`eJ-V7Z>(a@nwyR5yvqTfAG zy!dyIN*k?r4dKWt8aJaYVGP`Ku2}*;1m!3YDPl1BY4MAq`1?gD%AKJhd9$<3%LBuO zs5J!)G6L`13T_}o4qsw3LsBA}NvN5JM~{K@jHIOkHwkKrThZkcnG|5<_15Ao)u_;K zFQM(1gkV6;SD4*v1qC}r)2~~vA{%A?HBF>sz#Uwpz!p<@x^cFuZ(aaTQ6$Lvw3rc*Z2}X|>q}+SEXg<5_wjFk*ISUF2ak zRF2RCBhzJ0dPFL$oR^dHa`&jElqO{-4q(6fSU3L#Pc$Ad+SObqCMM=%Cg8KRofZ`2 zbJszjdBR7+*Di`&h@ z1D**lll!1t1z+|0JWsUkdz}tbz-6DN>8G9$&3?&#QD9XvW+Nx3MDP3kl>Ko<8qzwJJf;Kc)2CjE(I*ENuM!Yl)5qFD4HU zOPj79ANP-sTjVsjy$#ub2XV!~BH4$-kqIGE;*aglUasFNVcbB|>wm9MqI}w#xG(ze zK?#>JQmAO+wf5U71pDj^anaGyv3Kf>pyuZ0>F(GC`VaO20dHHY+fagtD!VfW2M4(6 z+_Q%eMNn?}wOm|pqG%W9_O^4 z?Tf?3vUM`)Tcd!dt&lRhNQ9pmhLM?QnCLOF3gL`3(4)H2WttsB7SFOR#!%9P0)HyB zZ!g2afczs_o6xP1zkMFGn3I!(hfPh#{m_lMWu2Bn76TLp#c+}G$i6;mY|@Wp*_oN2 zNlz90uIKt8ioh3GTIFA#oR5V?kf}q1N$DuA)(vF!u1h|d#9&}R17|Wbev$+PII2@S z?@iC|^HSvVvk4iV9yPn2q_iCWeV*>`msCDYZ+n%j)pD)YS>9-W8(V7+NmRb~e2~#P zgc_k+b&xCOtgQk{^qBaz)#Y#wPydQ~cuZt8g5#t14=ft*IQI`y+V-F|T9Oh{>5Q&E z5|#tyD7jxeEb2~}x@2;wUCGd8Qk9?>E>Xgb6Z}qZAA__#g+oLPzuU+?xw-NDNTwGu!fy$P^*1u{c zj{}lYEvV4>+WO++&zy`h-fcK2G+gEpsps3y_2nPXcE|Jf`SLzNm%((->#=f+F`6VPk0C%sJ=Lf!qU_w}P?j>eK(P9Ypp)!Su8HW~w*Vi0#Tf)f57%FP) z>1vx>Bc;~>-Nv}y^yi2q@wE3Zt7tud?*0-12?+@Z2zT)Mn!l;@c(q?y}G4%eR{6I!Z56R@^|19Kt@91a8ho7aLhxP6~;$=1r3M`al zJM~#BL-WF+^qGG{fiNNCnkst>gz4?i7 zqP^vLC@=>u6&=XfFocX4>se#uQWR*N!-=FQFl@=}qBiO~-H4>-9f!pf~UhTVwEt+I^q@M<6G`#uoBd<>dN3?S+2UvWSE=!Nd1@4b(^L8mVf%L&UbGFV@)g$p^UYg+4v{ zqX2G)$`K;s-o;9^H-%AYH&##x!y^VVAX3nWf08yCQl6W1SUIr4M5IP2f0fO&pB@fG z#d=^9qcUR=p$`UBH(DATGgoNIuDCZkKkBxji8CUOImz;D zGt)h$DkQ=im*A5&LeR=(=*2nUutWVPx?5I~+%bg{*%?Kaz(JBh!om%Y>VaLc)-)%odB8T+@4mC&#@EeFmVY_ldsKR5SlD^PN(O2E- zr5A({#DHt%d@nyO>4Z6!tU2TJqA$oJ!UsqoW7se>EFz z6F7Sd0~6b{OTySo9*2c+$Xw)B;h zxHeHj8CW>|bAx13N*C8;`Xk&|$ehX^;R|S(05vcY#pI4Z zvnYiA#F7Fj9R94f_}7rw+~y2rN6xey{H%jJVzL*SkXe*Frn_u$m*G&i^vrfHW_VDx z>ym`s>>Cfcv?vU^uaDoYR#%hWS?E29rJF1Cy;Nh)j`mOfJSRWRQ+%KGp0?q)@(X3B z%+mGWk`@Jj{um^s8i}wV;i$K|u5y~@U~cyPWOXV&xIgcZG8pcm zU%0R9jH>{Dj{;q%T!DuTY;h9GD8IQ>?Wg;lPG$_r?@YoftJx=6){(!v8b%JS@O^kF z3F1|zb0WNL*CoQailmJy@CFf7S%2b^`QM#Fw4T9>;fIKOO-rQP0jbBgiP!hdOakK5 zklWWqJQpiPIxZG^4lkMpX^Hpg45-QB9%so-l`T%b&r3!+jg;wO4+vk@KL^iB6s3(F z9M|cL;3tMioKR=HE2?>bjQ0%`;}tC_e5PG|9|O|Q8SFaEv+Ij z+yzocySLje1ktL%RCD3B)xPBhU(74<=tEG-d*~DA5p>})va}M7zD>lS|mv zC8|cl@+GXZTA9Yn?NPT_8~i3UhspVAr8?wAhNC3+@8%AziuMk(-8p4Z4R)(ZQxS*r zG;5H)qrxf|htmtCf9u@ob4_-qK@z5>XdOb@Cs}jDKQo%=DYWpJESkyQdwVK;Q&h~` zm740YVd{!rNi1Ie)&tcAwvlzU?{usPXm$7(vAB7@G{T2f?rT6eR`bcGdw<93G=Eu~ zoPXGs+xGb39Z(IJIO{$CCPAxwU5D#nAFk;ws!J^6qOMT8hhW2CwG$h5g_B;yxphyaC1bXwA#_cwz&Y$KQQoiU7)wzJSW4xgCWhonOlNkUzi7^*STt zqj$3TKy=OqIkc!V`1d!`p0}>;G`lO~fyVXJ72uEm*0b?*8T8RG{^juZga~^+UpM2mflA)oI(iI?yc88^|n2Yr~nh`EMWSTjKRk`X;ak zE93tsY3TDd-v1O2Q*NI8c;z;tVh^rVNNMKHAxf{^cH*}N?a+MDO3>Y8O(Mi?diybI zd1(BeLv;uQ!-B^=(9;ObN{zfB+kZzc1H;q3nSX^9dcVV`&HL5+%!|6MUXkuc$0@(Z z?kyzpebaZbj-W-eQ<^!YFs zpjozrJsIJ9`seC@&U*-UxyuxrKJ@z?j@{>n>kW5;|9!?1O*py?mD`;E`8UnC&K36P z+y7Q<|4G}doo z@3quNrz*JE{IT00sKRYD8nE+7yqGX|Q1IX=0$RA)S>azb3A+8w%-CxzoJk+GN=Rw& z_RXy7uQ0>+UbZ-096iAxHsDtfDJ)I^A_d`sWx#RPm#9gdE1=lYpvV5dL8$8uaB3Vfr|#V zN&#m5F$X*MUqv{sO$vF&h&z&_|Dz9=Ya+l1gTKAu2K}z-hTU9{)IPZ~;ZKB5vsofl zoE-|g9G-Ij?6??Uq1WB4er)|u-PPc~`gH}=jQDVswpUGE`~Iy=B3|x>6kY$RuBf$rq3e67oRB}TOP(QEy4U?)hN>hxP_pQcb?6yx zIlLls)MkEsCT?x+J8fJAv|mXr+DV*mP*l55;Vec7%o}W|P_(q6WKMxp&9g4)n95w! z&GOdnvpm@8-vP!EghT}DN${`L_#YNm1}pA-t$bPC|9so@8VE|cG^Lzc!N2U0$mJUcuR`*(2!2h98+(6$SHcwXiqx!TO!)=mE4Fi--xso%dyLoGF(<7N(@w|3EAd3?g!>nt$wd?57zae<@P*IY;rA(cz1TdfU7iKUYZ+5 zEXLGXe@r^*ig7GyjU<%#Y3n@3a2C%+aQulhOs{5F6Pn+PQ)JeSv%IY=E2shk$3Ym9 z@JfI7bsP5C=7njh6iv9TsLp&$Mx0+{JJb1RkRS4|=G`S#EY6r)6|?#FaYxCsBP^~5 z?K>*2FMr{7dTL;o7p=3C+YkmE)aPKppL!I? z3f&I1n&wp$F%8_YG-RJ%uK)bh>K?`Z(!k{4I-Q!|uYUZ7oa=u!_uk4f;m1jHz%b1Q zlS7u{^fCpqKr5rk7@FcL%7plcI&W#?!JzSSapFw^r!jAvKU!z{BR>=7quz@6uGDgQ zXJ=Q|&dG8`fqY-X+J}zw&%rdrKG~f45V0|rIQXivlRBAq9Weq=C-zqNDifCtDgmbB zN!y+Vp=5WgrHq=UV{HK18x772Mj;8>4d>&tGipe~s%uWwyXDb=Y@YF-e^NGBU-w%T zr$dO$v#s#lAZZv1U-nIz=YGUd>c?a}X@KAPipy@NE@tDdCkIgo2lHGIVqF>~)w6qM0+suU6aM_8Z-kjjLXD$-Yms zh7GkJ#oyW%SvC+KT@ScB|9%ppah+z?X?TkIwWUGLTV3d7#)ED5xQ|Ln{Dvxjk~UDA zs=sdden6|QcIRx!@^`I65r*zwn|r&y;i2!ncXsVwU!2@oHM!J!)6L&(@haO*&2PjL zeCKYem}ADVohJ|3h!2;Yr#v_jfv{Vt^VSpgZZb=V&(UA`xzyCu5^2rT|7UU1>He9C z2}k7&pS|cozwzNtrl+%wTlbP-_u%3B0_TckLVW7}>Eke+dI0!3nUu$2P*l1jt+92X z#@3p=ILW7uu~1QuR>AEl+AXmvZ2asO@aLa99PuNB+w1(}XV7cbp4p8Edjq)Q;i>Qz zhQdz?>ST_tbKp{(fiCZ*V~ynGZt`~4U;1Nzydh}ccYel8=%1o}9C%puwhw6Uxz8v+ zZ0=8rJ=xlOjb-ck(y&jc4lN&G3iknEm37mcg8SyuyENpf_zh$j=Ks7Ve9;&2YtNUan15+p~4o5>3vjV^X$H)TdiL={_oJ{%U zsYDePPW&zE5(tYat42$PE0^Tf`T)MBuYMx*%5!b}ToNl(6PLDJ2kWc8^B_d08ms6M zAgTFvW@^qdJ`OdNmF`4hnNq~vL-}5b8L`TmQ9i;{iWx0cuo_Teb4GQ}oLG7oHPB@T zdpD)=vXQ;7MpnFci7RRT&iuIKt8f0J87pX^xb(J3Cu2S?rlIk-#-DB!fMU4dn;R%W zrlPtXhv*5JXeM$;egsHNrL*Ls=T4=1`!dKet80?9)VRdX_Tcvb=q&R4uu*f}pVD+H zI8iQ({M-2X=j6Af4)d#%ymhkll>}_eXeM1unGf#(R)&|iOPsH<$5Ox~T63uvE?o$L zu?geg&5iHV5OILcoUXZ+y^}+twW;mH4&_|7{UsZU6~y||tUu+^RcA4v`UT*Y_{{d! zoW$18iHEg$&$HAlQ45sl9o`1bB#MNBN?alc3hrs(h2G@)0*4Xxnf1pEr^g=Y}1dLMH01Z15 zZlvLZVzS-=JzOB`^hXTRw;pb6*@%=3&7d;#<<(-UXw(r#Mz1*|6-L>${Q}Hc3axyx=u36aeONg|8*GROYu;1~7N6m?5UBD=5C^p*Zfi3VeK_ zwfk-SxG~4x9<>@%o6t^eH(G1kHnq{TZgyIf&R>(Nsv=VHUFnZO>YC`RXdZM#hxjkz zGTmq_y`8(zP!UDd(wc*mG7B3D4Gmoc>Fb#-t`8d<85PxA1uC356Z|w>hoG~S5s#w% z$Fk0>47MG^TcZA=baOwPPgaW6bw*#XhhuXhEJ2$67CvzM1a*}DKqw%vjn|+8H7TwR zb?WMzRmQ&gZPQ@FALpvpZXu85J%!RsyfOmV6BA`e-nRNdG#CxL`s&_4*Vi-t2bFCk z^V>XWu=VF9NAR_^tkefq(?)Ehg5u6rb7CNCrDK9dGPRr|@G4;WU`zaCbyRb0<8Lf3 z{?wV-70g+yvBs}xRB$B+V8`pjOa5L=ky5e+ZD(>U%dhSnm?L(}j3s9}aSvnK-tRI- zQ;J>_l1*)oOmEW2Nt%z$Rl#`wndcf9^7`D&kpY(cd_lR%jR9W#QNwBR#EbIJuwJ0|F zo(g-S!0~7q1-4Tik}}pqiS!!+IzAK<#7Y%uH<06M?qaDT3bkG8t^yENhNPNxZ>+u` zAWH671k^?t2vfN$^taBDPQ38%@dt{00GjlPfvvB=L4)Eo-HH5gi7$z)_c=3zjF8@}0|&p74U2h`Zl#5++fsSMtQ?`mV+ z$s(e{A%6Xi^d_h-JuU1js$+Sxi@oo1KqkcAeOQq1fUf8{8gLIy`-vH zL8U5=OWRZ*4MAiZlxElcNVfdUAL*tl>Ir zYw!@p7YE&1=QRvLVM5rR=gE5Q_o@-TQ(0YsU-lTTieKbePP0{gtT?e$APMF4f+G#B zylhNSDS5*)$U8!L=j&>iin4gciapy1byzgr_F@&5Hq$DATgFjr`GOR1zv2H3 zsglAx=b@w?Z`1ee!Jb?0<&DWXUST!-m@6C(`k5E#`24OA0cgp{X6gTCE1jGx3K%B& z9&j@R?tDAfg~k!xFzN+Ldhum5F#DInEW~mn>R7)eV{yGX4((GhlW$_8!TLeIaFtfoq zIeq{lx`e--^#eQryHR5`UFAPevO&1mlZ&ddFiZ(``py7KwYS@Z$HnIvv1bng*&k2u zFbgy76UOv3(rslMhh|DN2yREtpIg!Q8a3DIBsp~ZjWD@{LU&Mcv7zMd+*eY*&5M3>#D&e(A%26;qN8DYF$ z(7qkN+^x5BahxS1_d22tU?H-5y=cG*M_#Ok*1e77yv9MpK7OWU#SP56y5}`4NraG^ zyDUyJoo4y-eo#@THpL%E&ho`9@{H#fgzU=)ha&f)p_?EnOTpmAmp$(f>I#%xwZ&}z zD*sZfiHH0(S+t5(LPrQJBh)?{H+1Q_EtT=ZLN-L@VV3x+$KTFoCkcSdxZ9Klyq5n( zFK+&KlTXpTxB5+=(cA@)mg&K&8DS?#idnGBqvg;Z^DS(+-WFXgm)#aeeCh^kg1`H{ z$R(!#RZ`qW^X*0V3ksa_&)91_a}1u_E*@2j#2{{j;BDG~i=n&q4juuwjrYmH#ial* z??2f(xOrZj7~zSIxxEO~_`NpP&!-hL)u|gD4F?OI<9{^8HpdSD$-a%k(>A|9Y4JnM zO;q8`YL_ zfifvR1r`|v21PxHmjaI*WtR>B!ZM?w#3RnCOuVC0Yz4gxd3C)iL#EVsVU~SEg_p-;iGRMdA%a{VL8ty6uF%*E1hZ)nxYTQ>h25W zoae-%cq$8X6-h#`uVI=@7|9S!!ME+1iGWMg13mE|z*kS*v13qS+WiYdl{h~fg^>i7 zN-bD@V|Q>;=xs(wS{)zdnx>*MH?HGtMd?d}ov421&v5YvZ&G1t%VLuAJcs$+SU9w{z{JIk`ZH*}M^|sA&f04(= zUI+b%s;20;o&Q5uT)=Zd_lV}}W62v+Wd3XfGjQ*@^E=8~iT8k1&GUBWYlt{d`F@A| z?S2>}C-ihD6q6qgD)JBMd?Xz2k9!*>AwcajNeIuCUOthkWJ)d=m@#}AGhApOR1KkY z7TKW9d4Bzv{Ca*M1cgARq^A-tW0_pgwIDVE`@tj6@eD*L=gztvK+xCpvoid4n-1wl zW6MTKei+wkUEF>iCV$zK5_;ak)Go^1gC_h8?L}O^+i!R2V!Y=9)vt>rN-G~K9Zv!o zMU9^bM)zr#4CY#K-^n-m9gOw7w5K&w17mNy3G_R>>SdtJGHAQ9i<#O#MoA*K3&3v> zhnI^m*!r?|IN)3pvrkR|$p8V&kcNjmf=pY|bVnNgt>AME6f!T)zn4LINjblZ$c(qRvZ3CeoM_5SPY`RJ}0JITE`Cse%B9gym^z zVd4z}c0O2M|FR$goRjMRQU1mjkVv3?N2(!*Kq%f9|6AIwjVEB^15gPZZ;GYqhE90 zRt!@xAXE%E_=us5A*rtmLN7a=Z%>_X%!Xrzt?p+NuTO9LZ?ACy*K_Si{+MSB_`+LA zNEnC)s$QLq_?SMB7WC+NTEc|pfK;J8g~ECvY9W_s z_k=%TuoHPPpNU}2I)%}OTECaS!|P0*Z#NExP5(elHMg{+Ekl48{&=VO2FENMZ@WGY zlN^dCqbnBiGGr*Ds09>(717F@gu1G}?a~Tqr} z;$>;pc-MFYOxw3TLCBk4X*aE3PK-fqmnCaV zHT)|p9$ikzHGNe~Ac2>oJvp53Yg|%X?E7XtXAo|-#(QG-OAHWo@{;kBt}Iwa0uV0o zi5rDX(6_0!8B~nlW%lIbGt2xLn7T9;X;ZU@CrnxqsNPi;TV+JY{gAGDLW3M?>323{ z_}bS=FOFn|9s(!C>bSkAvS1NmdO+krCSPvgC*6sT5bODE^{ysN8OnaVtR0{#eADjX zz|oBN8%xB@)a{n}b$bFxC6YHq_#KXFDt&RZyqpB^s=$HkCL1A^0(BfmhG+Qh&SwW` zKrm!A4!@(pV=244K%^7q|NS0jip6cnc9M@sno!Z)`aLvy`tA1ZDL|}=m}}#H2&#IG zZ(AiJBO{Sa_g`r>STY8BJSS;ts;!qFVNJ%quD8MZPYS0}7eSy7nzl7P9>iiteM{KkxugDaYiY^wVRv z4WNey3H&l4QjXMKJPx+;1^X?@73Gx0Yo;Mx9k=}6V-ux^Cg!Jyci7GMnGvSmrGccO z1-MC+dXTh4rm>p|JfVj07)q->9gK(<<-G+rBm|MbRazP|yMiB7Ff2U0c%tSO77|}? zH>Za8{iUC$RvMhhVCVuF=-^|iLRi{gzeCxN|e*@9Hdd*y-!Ucbi z1!mq5_Plxgckj%}S@0d;4WD$r8@=}XrxRT5)KZkY4@SEG8IGSy5e7kZ3@>^=%HFKY@g0BUUGDf|af24*HdP!KRGPSxAceY)E zG>9Hl94GI9ib}WK!PC&f)n2RNr0USON_lO)9!!M_P+)w(D+9gjLsrK`$LIT* zpF63hDcF$P(pvxsWy5C%>!J^X3@Z zA>y&}eG?-bA&2z7RcrTVN95^=Kh{WDcX9n%F3(XmfGs|oJD-y8%bUhL29+t zE+PFb1s?b>dzEPYFe(Tyh|Q zRyF_2rb#I zEp1{~M@uU_s0YSkx=%I+UIa@=%YqQO0qNLrkl+wgd|W5)B}}n|C<0{Z?rzx(6fowI z-OZ!m3uhj#*<`Cd3W7pI{`%o4u=G$``h!oil>|77o9A{`T|LQ8Q^PpYeN@{lWB9F1 z1;-#E4lz1}X5Bv>OpfHh(aff(r;Urp(AJ9g?*!e2m*;oN0tl)lr~6)QBNsl2iz|z2 zt0~i*@-yhNV?fO&NM4L-4`xdtSy&&Fi(we1oQVGcc|KU$SFC`D`LbUfK_9qfTAXEP zMGMg=SA=4ah|}SMIbd4}dAMQtiVz2|9fZERP6HVvT}XD~FUg4hsgnf(CR$uU2fMif zcxiy`l$KGYH9<>JW4{&tT8G`NgsW^|3~Y>*3X-FFR~-C(XwVc;DZ8xPb=mX}UcRnv zA;PCc=t9sCWYTeg)CHC_?}a{=#v#+iF__WMbUMG_XqWpfpv`S3K?Y}uy5M6Ti5e`( zZn?5Jl%8ZzqaW+u0#fSXUd@~lbFsKSB5R9h(n2IZ@&x~`p`X}ULdePYby?8zBK&b7 z#Dn3ADBax>M%pB*0<1wKNSpx}reu3@KlBoe1P*eFM8;a=QA#`U2Qi>1Sp*n$ct$%d zZ$<<2F5bz270WWynXr|gzDRQPN+J-;P(45coH`Z2~$7gFmh$U_1;wpBjvo2gQat#qIZjL1NKy2tz|k zi~Uw8HCy$EmO}l2I!sz|rs+%+mB*y%FOm`nh+<;Utht0_lZ7qEL{c3p#5}PtK5oj3 z2u#}z$8~^ZVD1e?HH)#$++KdmWVV4y)J=?!5$#69`G{K4S z(3hyRG02E4sbv9S#(J$Vc37VETRfHK9MbuM;7z7DJaTe!peu(ak1bHJqJ0+qgA@(2 ze2FL?R=nYfUM`YjI1%#T>q{sN4VsZ=HijiKf9~&mQ$rff){knP$W911>}@53?SWwz z2>$Eh;D@EwfLxfnxQ||%oLVR{lFX7R4gPZBAt(W?x&b2xIs9nbmd#)e$P-)W9)KL7 zNPuhc@&p7);PNlT>zUqH$12j&n4jLK!9c}I`$1|&C;^4|Vk)b|3?~OWa|Zh@cCzpJ zYR|r2n_)_Jskmi3LX+rl*?xm9Nxc>TklH8f+j4Im*(!~JThvV4rmv&6#`{>@VYJ^Q z2VNfKfx%$pKbB6_Lc$1LHthUK#dm91lVljVa)+T(sPmwFNda+;cR{(hy>Gbd(#T$$z5kJTaQB91DD9ph+tL=I{cRz%PyM-N44(Cw}=RIV>2WDd4SA9Bs(Ib`a z&ABO#Kn7JWd;*SMwY9YI@F)*?^%#00UO>Yj=FRElD=dO}|I_3Y`FRoy;XS8j+;}Zl zW(FmbL4k=(fu-5~p|Pb!T|bZO){+uZ6kI%3y*7u#oA|(GU3n*4*VB#O|M*1D)gnrobP$FrK7T_q0-2&5)*L^TeFVt3o>UsuD@fNcY&T*O+rhj`a+k z$h7VBuKdCo7epaT-tXn<+0lLvxa2RA#@cqi@@iVY9tGFX8BrHeX&|CYi&Jie{oM=y zCP7l*BnSrJdvYe+JK8HI6A50u@XQvWqNGGM9&Zcud+I&$I`QQf;mCx(!m_q5UAqaL z6MZ^ZBG<`CFT|jRr^E3PLf$u%{c?$^FfSkR-igW_zlX=i0;~Z% zq%iIqGFF$&_bj84QnZgK?T_s~VyT9%9)|Gsp^2;H6p_cBnmF_b^2pQKS(PjcZCf4@ z4uR~9&4%!QyE)#nU2=7Jxu7TS%D>whmC1y!+wSMCQS(f%{Fzq@w+EyERbl0K@B?w?wlzpKL%%eaPj?K zTbi4*fP?&Brm$$Js{_0w4VeN~yPrO}{CoP+I>m)S0j&DP=a=S%KNT7ptkQYdIUJs! z9`kR%7qpgu-|X0wi>@tbX~yM*fzi-BfzC8pgg*mbc+C~3EfDeJqJG5(oV*c&GJ+xb zQ>Q|owYFdPD|Fs|fP+I4R1Yb2wB@t;S4!FW1%xghd+(vpyv?r~m8v^7z$LQ*94hg& zXj8?PHI5^47la-|C3*_{WM{OEqj=+ecT`a&x6T-vSx|+E7JNEgq|>6Uz>(4J?h#mn z(her&INLJ&s*!ZD;+ZrX&7@IDNKE8&vwGhGhTTV#R}l(C#fo@{R^|T5dAc#klqV+k zR5$wofML4l7k7?<9xh_u^}(9&Rqa;-C`MnlZYC4$?N_A}AP06)2?bhn)AgaCzOeII zmG?&#th?JyvcmklqR!V>Sq@x1u-bI5T;@T*5_Fv5qk)S!hgCK4N-)Td$78*Q%o)W= z`@KBZ5Z|tShvUkT+wf8r7d4t_0b$WK88cLdLH+ zgR>tjflSr8rH4zBKf>@5JG!dl4oCRF$54OCRXL0A1?F+K3F<5R_*?0 zfk?9W`W?%9JC~KrELq;SZ}YeQ4o+Us~X zf}R&QWzYY}=%sC{KyhKnw=>7bi$9iU3T`)9v*>UMXipSS5?MB0_9445}ujzZ0R3B9GR)4f?s|3dp?tW@Rxz*U05heFG{;l`=fQxFmfHB zC0kzydMhOr0^B=OjlBFMu=zw+raT+A3m$YEv%^?NCsv!C=_$hToG{84<~%CTKLyaU zk^%Wt9}Yggo5meYL<1%Pzw(~w z6>`Y%{WY_yP6wjrvPC>R8tS>3ezvoT`dqi_v~KXBlqkN#h=L_{wJz%DD6~I3wVoUu z-dq=&83eNeeLqh-IDRgCnfK#KV=L}YcfPX3_^DOD=4N~go7h+h|6N~PL`6qa=fTLS zMOW^Wb!=?p?UH|QzO0>gKVH6+`ezpYzH@#>u7nYrx*OWIo_fAc@GCg{5zeM7C*n=%~6KTOHR4yj-e8M zy&vDl4~1=0oZ@^ZV?)fZgP{T)mV_YaF&;#Vm6$N36^ASo;%a9me6{`A4u z?M_w+(X)qVo-I+xhRP?~^F2CRZ&yI*b90TNL+&-=c|8n+nV$aSy3@#sB3Jz&JYwW4sxN18$ClhUb&To;}&DAhk0DN6kdN-_l@(H>nNy%1M=t!Mp@JNtxBMK_K-RtgR6&eJPfn(?}gHST^clJVsv(0l*O zlY@21itH3^; zIU}garQ~5T=0i8=AwZe)qeB+RLu)7w^t>B+J^Y4 zLs`3mi~4&)Pk+1HZ!iD-`Yz@7v@;>%aq>8{;2Kk2*uqS<+J1g1x3zehSmgUuChB`N z{+ynzlLQIxGc+)GPDB$BX!tf|Dj$w+$`(iGUgz$ncf$A7+lzL7(EVL1_UUlf-uvbW zjeyez^rP?xAO7;g&Ipy5kLo7~tqBy6GN z^xrrgDBY5kWzcK;FkX|vKaAs}k_B>AnLU)VMnLon4EYyK!$f?gzcM-e*G5URtG%n* zN3Df{f%IYSf9e{m9+!(3lar52)BRuT^->I8pUz(#>hh&)>MdCpmt>OdNx2{y`namG zRkv@hy*A7+SOC;_eD5yy@`_?%sglekwCE10f8)CJ$dX>0`rUMU#grdvRO+?O`&^Dp zbb!MPzOHH*__&(gElf+{I7hbIQmz>Gq1}7BI#6#Wkbo#k${=_dI@;{N3U{gj^9Fi@ zIZaTUYhS+QmV3KB(_LWnwY+XEQLxyY=3~X%Z~`_uRC-||Oz+bL`q3Scwa=1=%u%tg zNOC_WgxFys+P;RoMP%l=)# z)k@cR*S{vu6fj+vzQzfyxPewG3M8QTd)ctgRqa8_u6a?m$X#AjQ`7hF;_03WYi#V% z{=1)rI=IC?K^dp^&Kl(X$dif_9%_}!juLy zbV%9pe}OgB3N9<@WBnKy82Yr8ZFbudAh+gWqocD#(4Ey0kjK{|2U8Q75coeV zfIOWH9;Z@^?b|=Yf0pUr_{c8Lx)y$XI|re1sZw(ng#o2NY9VS9f{~ejh@S4g$!iD=TC6HafNoP`%Wj0Lrd$ zch}XH%X%byU+-7#YBK=*F_nsHaM^&9lT&OS2%0xP*B%}uM$$qRBx87f&n2gLUkynW zQsIOy5F23(KDx3xY&}k{Y6!^gv68nEiGa8 z+tj}9oB#JkaKxJDLc-hI90L#4DfVYA2j}f>-6^VdHhON_eMO_bY^zWKwwUQ zpNEIPRb5Nv=sl~JmX;9=z50jH^ALRwZmny5J)0sKi+?Ii4n(+XQ8nHNPy|eT<(`>i z$-caIzYEDhf|;3Jmd)AAzPc7^5>`=FTXWqGflHVd(YreM50^+;mY!qMCQo6bf;E>9 z(%hoLNYct8BXhjZ!m~cyFA(%8zLmrir@;CSiYPywg0R4~l%l_RQB8}N>t0VM#cS$rfmmPS@`!|zFL%HSO zzSY*QOnq+L{p7WDTRC23?_=_9?BMcgv;O6@%4V#uW>s-9pZlpU4IX(~DvDuK74o&e zmN}X)ghqmHTSUZ2IxD=Z2`R?O{<2^^+2Z>b=aCF~_p9dGqL!AHj;Ebhh`7PCv$IUy zw(Bbcq1W6RECvz(D>Nn0GSc?dzT;`Q=TLSnF?(t%;p(bElX*&%g9Q(4&682l2D#tF zdBWuHpR05;Xm@!^oahKEl9V5mb@9NASd`Bq`RmhE+xm90QWfV)2SXw%DA2&?33X9- zS5T*ayr*wIS6(QADc``~VW z_R(_wxu4izo=@+u043&~lq7u^JXQ+7utd3i>fZ1<>6_VZBhSpc#;jv3w1$vS$;4EZ zjmr+l3NKIVhtuB6h=>Rv%RP{I_wR-bG@n}2(J;@;ak|6mdw!H+z$!2Iv>9D#F(EnE z^S7eh7qGe7_--_Jb&jh!Ob*}6frFnhl)vEl=X0~kyQ^L&RRZiE(1XDFhZ~1~oCea2 z_woe77qtawS_Q?$X%Urb9%cHSZjR>GXQy9)2mg3U%+iNHv~kJ}GM93;u`Tg4{X=Yj z9PRMwQydrSZPz4!gE*4OL) z-i_j3tN-F!B|U}0&bH?vmb3G+igRsL6cE=YuU^n_{!&s-b*~>I#y$pVU)FZj)!|eN zh=#%v*ik6tF6*sM%783KTbs9uiQLQ%a2Exb8szpQjm-2ku3brEWyeM(X~SpgD+XFh z_kQ^j=z=vhHnCsu0+cnV4%pU*oB(?9SFH; z`rifMtBvs>n+m{LL#}?8qzvCTWS3iMTUqT541{FM?hiFd+xTqc=-eMQ`|j{R(uB#PnT zWsj$LL-vnbRj~F9w|Pbxl(v%P`lH_L@7|Z}larrp_Ku$mCeQEou4<>xbC8heotHX= zIXTS&c^ay(SUd04yq3}cJ_0BL8YbBbWb`o{`X}Ywh;dDzNcp`-c2-uTo_D#s8!HSu z@0lwIw>wuo8*H7fRKO+K&1`}ufem;OPbq%4DZ@XeG%YnuKAD-Bxqklm%NF!Q6M7^1 zy!&ih#8=4o;nlY2+oTWKqv#~VmR!S%wF_CFKYs=&YWY7?6)r5Z=gT+S2$MtQln}J#lyIrH;LFv1l-9ou(Xx-Sv z#P#*H!CwKtE7_60-!5ygXMmyxR8lIQaQ6?Eh-VsG2ek(`0sy(e7dAZxsyYe?wt7$8L zD<}rx;rTHKV^$rB0VYR6rR&ae2jQ|1Is`oV0Y=T$OKf@%y|yw{C83$7`MT z=}w+A2D;_uCX%=ae-A#k`D?CXK@^ktox2EtnE5IGv<_$JQ+Z8|k(nb4l^jTbuvJl9 zMa98=b6g|mV}XsaCmsEW`r7<&UOpCd5QHfFO2h|WA=PFhZ9l!Gh3o{jrh*|$1EuNi z$o`46(m{?sHiHFiERztipW@QwN2QBz|- zNoFKO)#-O4FO;3Bd76@}@mpO@g#1uE4jh+$TYA~QXUY7*D_R{ws<7O7t;cNFx5IlT zbbDE^_T|f?v^$9KyB1vr4(R1WBR6BjPd{-4Flv9^wKD%c-hgh(e!24yDga^LkBl&fq_GO z+tMp~GR$CQ+TSGRF>EufDA~QVlbeiD^q0Wt4lY3}P&+X;)|6Uu#k%owHx8+q__!)~ z&+%2y$@TN3pvR8vQqS4!Z49T%#DID-5#CBmc4|)AHd&U)3gJEtX zefiSQo{5;>UQA^+_fm9f+r%7oRGCp57B&1tP8Ls;a6w zN0e`lxe+wIkM=_zf^6f>12d&?8V)S+IJs}?Yrx2BZ~q;;10&bhuVfguv$M-i)q4Sg zk%kAM-FcZ{IMWKKt*w_;R8mk<5q6bX?~5(z;B>T-5*;t}_ecDh>%znzv}7QR>I2qp zug{SjDIs7qFN_WeLUg7>Wau)gp`)&+x56ugWVjHGhC{{Bs-&E+{u!%VvWuf$V{ag! z*2&-;6G{^i@@0C1&BEo61g%!Sm6xhZxWu2Q6~P(_4w1(AT}I3DGD=bkny!JJojTOi z%*^qzA3!nePK5}97g^RWRP%?mq`0`OB%NY^&Ts!u#O*CAslZ8QMvg0c77)&`nqVhf0{)it9zYS6^fJy9i|Y0OX?zRP?eU!oq6$e@iP$ z$SBf4Fbr8~ey;hGwS{o-Wp+n5dDa+93R4{D;|k0_s-`mL3hrRx;Q+(yWuKqhpnv>Q zLT|oX_|`o~{#Y@&RuJ!iaxT_|3pXp^5#g;U9#j)oS$0N1A@>E}Z2s-eBK2k+1- zfB>lP1&Frf9z~}Wk8~RQ*=|xE0kv`#kdWZIiMeUyO{zznR1G!1Rv88oDcIQ94*1JG zm$ICsiOLC>O?hL5T32jT3i!1JIK#5S%tUrIIdu&o#N)_JQ{K? z!Q((C9H{gYB*AB2LBA2_NrT|?6v;^x19n^8e*4&N|DDFh%5gy_X=V8^BUK+xj$3F2 zJP&Z}`?}-HE*!LW-RLAeA}2Ox!t|_f_$QneXp*nDagOiavb8d`wjXWZ7HsF`;ep29 zeT5ji&CbtPC!Ha|zpZP#>Sg&kUv?kq;B91UXhbU~T zIxM#gvhi9?k^ifysR{Qo*i2Lbm}Y@^SDKUxg71re@p;N>Y9{wwT=IL;Et=lrq6tS? z)sRqXoaf<(pCW-kg;tdn6={Vf z>q4D=^fNQN_Fw=1985V4R=(s=R>kGs*RcTti=3Ep^QDCm?61AOJ{Y}RX#}M1{7qq@ zVn%*zwF@RitpmjFB4Dk8p&WFwMx*7QG6kLz zMmD$-Fu-YaALdYjgutFPv(ut29tfoP+Duug>Cmx0j-HoyWKUaL8#z4lC3RGDeML$v zmAjzceGW!maY;tfrm4)dj0vYq)#IPov5D+UVV}B=7E|d&Bs5bnSpkXKtABD2evP1K z(fFL)^^eHWK~&NEA)~|NBV$7gx@*N{48NDPD5&u;*m04$#K>v%1?JLsdtyae6LB+1 z#MKnTe-yJ774tnmeb+*ye?98yGMfAIE0+s9MnXBHhrQCV|Mbv_?@M$?RzO!d%G+;Tz*@W`_@>iIW^UdE?E?DdK(cS7|Mtw$obVbVkZA z+dx}Ej0Y6Qh#wC2hbCnRwjPiR24UgVW@*#la}|KLWcSy6FS)-%zoEn-*69QyFon0X zm-BwQhl2z3Ks4V*1}-%vc!ObMU@R$`>~?+=@L89c)N40Jr6EWJZ0ycJg&97(_Qf;^ zpM-`0oh6i$G@yhZqWXT^`} zyLmRQf8NNpw4}K5-!LAUoBbV1Qwe;$w{dLv7`;izq<90y>|QZh&XN5>mn@Q9G>){a zo&L@&a)xf#Reskx_Krws1l?glJ@)OFDT=a?Z@^zk1e)%L@gN+|cl3}yB+#Nrip74! z>c#B}HjQ@>S~99-D8rMhe*Ky#`SlLAaEBBGoluD`k1O#s%nlt%7Z)e!TR||-)tkdT zmoH#Cyahjr(W=>xC@zkLQh{P|Fu4@MC<)AIL}v+Jve#K?&a#49CXzWx21@kH_l4f$ z<#u_+m3Gd^qg56! zZQdk+-puKBEHUqW^NJ+!_otgL0Y84=$H+<#y-oA%VJiT$gwZE@cB~YNblWx3Zz>s) zg(c*pOMQ>B3W`F+0g^i&W4N5>5-!3<=>mat7z7~>f-6|u7{!Mnz)k=uO?Ij%|Cu5| z8UDSx94~e44`xhwB94^Uf+fIQMdM1tG4TrU>%$V^1GFtx=<5j{e1s~kXa1;=KfLxU zFF+V%$kgbjVan1q?u#TxAV)*8OzXg4L{; zqhh){go$D+yI}|M*74XM*F@Nz z6+L;Z&zr64BJnl(J4|LblxmF@A74q^jj8ff+Jh;?GfM@$unBPWKx52pAGOyI1|^F?{pLC{J+E@<#vDi7#v)H+RXT- z%c<1u0V#CEr{?}_t2VwO>j-X1kAS4AjOA_@Kyc@KPbIR800$+Tnvs^3ktS@od#$~l zN|PpwobIo^C<-+MXQoDTg;wS5z-@hKD%xt(68H>kZQYVI&AnHnrhW)S}ttOnIPuRQCIb-{(Xnt2r#+yW1++ zPOF}GQ1x776dph5P|kQMONX8hdJ!_Y`Nr?T>hq=L+`{zpncu_QlN5kx@X=5%#aF=I z#T~ql-MTH_%sz<;f+Cdp5(bBuMjaREXNeB>hn>$}CJg159H3{Sht<%I$dHzh@-zz99Ub;4<{!g@O= zRR?*vxM%R@nwZZrRbBs57MnET)7fqM^UqEsHl*bNhH4HwcGEnGKXHFdEvaX9+qaVY7mXoFl{ZyF&BMu_N0=rL6w{rK|z@$O9u9-rRcOj!d8*O2Dt#ncKa4? zQ&R}Z6k)i8*oXF9%U{aZ272&RV`Hb%l2)@idp@%0z?s0_Ka0!P)bF^zQd4vB-o!jD zb?zCa3~~_y0OJyz1plBx(tr>rAKTl*i#`|<+fRqZ#hz{=>Zo+`0qc4L$ zeY$u{P;>eFiH61-$qfYrDF`2Q?0l_KgUKZR-b{N&Ke?+ZIJEGnO(HHVUA7{J<0T{* zJfjgLtRY@Vc(VRV{@{UJ$``|M@1aGsFZCM=?j9OKY|&=VoVvV=Ry=-=?ZC15b z=}DQg9Gl@1#(uU|o_Jx3rEF7oi^F5~okx7rcjw=w;Cc_;cm%1O6`-)*E^ur|#$rL8nAz46?l!0(Fbp5^adEtCO?K}^fi{=susCUr zzCRa}n6!m5nQPyi%SGoi!r~S3K$@nGj!L^U-)?%L*&ok$_X&6qCtwHGu;azrv9K19 zE<*}JC6cKMOs0xwh18PvpvmRGf@f8J9Frfr_!5D<8wk_~;y1q2 zU1bZvJ3{?3D4PomGBmvys&|Ib!Z{Wx#I4Xb?19i5+y^Jg!&D^c5j45&B z-}u0b1j=|m3{XvdeQn3%<4SgiL5J`6_6k?m%i6G+HFJB!-0ZB#%{mH80B7ak%~UNm zEC>t6{lOexvwfoI=eLXz8MrJ&g>f{atZdEa{OER?epP5n7QGCZYWvmYdF%q>V7PzM zVAq*490mV#^8ztzR#ip0sV?WQjjcM)B`GM0Sm{`$;=9I%mQ$eT#W~^Kdmx4$KHr@@ zTNmBjJ^UL!6Y?kU4}UX}nvs*Eu=cg|Kvi^^|1v@tXIBa;Dr(@`>B>g|7=(?DO^A`u zgmfP6Lr&OFi(%?Y_y594=$ShWqeT`-m6wHwyK^U&q@qYZX`7$e!s1cTSArhIHbC)8 zilGE0PZT#TEB)BJxdzCp8ZwLJ?jis9v259&jT?{lWy6w2U+pL>&6_TOvu;KsNy2S* za|jF}E=H_F!sCrqnuxV$nwjf#Ff(?V#v z1${{Nlad)U56(P3G4{UFHOHpYW`#CL<2PtUL>(ndq0%L-A1{+9f7DbuL}XS}~eVw;4`0 zmo)KAXp);DC_I0O%Rb|+-h6_OPV#H2XZi8u7#*hk8)^=f$5KScNIWll|Gwz_2O636 zyJR9d*A?)N#8z$EvV|k8Ag{Bb7S7?1x_Hm@5a9Iqz+I8wkS>2NF zLLgyAefY}(F>nG_-&D`x%949nm_IE#+c%76#)wyO>l?seGgzt=5}r;g$gpm>C1d z=!<#(v15jdS>1M*j(hLjq8x7q5E4a06n~#fJuyNO5R>zI2H5CTF_;4I!ez**#p6CN z#LP~=kF*iI9Ak}rCA_q}ir+OQE^AWCh}y7T+8?}t)@c8Bl3EERuFxtyVXNapz;Mbi z3Q=|m{hrj_ttzE`Fa&9qJ+7>xIsBlvkCZn3?MczqR-#(KUOP*XFc;+2i2)iFvsQJ? zpb`0#d48ZVu|&B@|E8TdhE!mp!?R|jHdMIX<0S;naMZ>` zE+4mtP$KFi&#B41JA{zWu67}A3C>}-yjPteiI2elM;Kx8cor*#$C1s5ScvvoTUBT1 z)ZPX!pmQT{I4oL|@0(kMTrrb>gNjKcczH>u1xK+>ASI3ciIrYrVq$; zCz5_A4mtU2nzPsru+!s`pw!fMATj*J`tsu|= zRORNL+3K({tJp5yQS`?vu`adiHAKs=0i<{HfyT3B5 zY$zoGgneMga0*bosiD{TRinYKDiyz1L;JUH#;%U?uV)I?F_8FC2m-?5RcBdWzlf_Q ze)ACo2{&aKpAc&D(a5R>&{^2Hsa$kFB{@3F&Uqmkcv)NcG(j#6_Mv z4^v^TV@-0oswBX&MN(4Cf0}qIL+H1xc9Rs5ei~A4LhyidH$(pJ?rTB5{%?ZF?5(*_ zqDTyJGzYk0@W>NOl(vKWKaEy zl8Zm<2#YIl6-P#cVY`6yJhSHy+d4YzjNBMZ+DTBtIrzDG_yp4hE?ZhzvJP%;rYB4c z3sdl~H5Jpq%MBHR!|h5+SI;)txH%M-hmDi)?g0k8bJ zZ|1&MDsSK5_Ns2a_pj4RmOd$nP^wY=du`wBja%LJfQ(EO`%`9Qn1`1ayBA)iY94cS z4qsCkG}7<=4@gciycU~Z-ib-{fhG?J8!J=5^+!u#slB0Q{Ph>;_#|-K$ENg;j}um) z!oIk?y1Yb)1u*AjEbpwVd#QjJ6Hfmc8a$6!j5lPpD(Y!?ZQe0)>)YEa#Ku8@vd3ae z$Sh|OIoW|h=c=B8+AJ^ICuvEsPj+@ZT$J_#Ps%E_RUIwTG9Mc-v}mZv{~meWjxID* zzMQGApEGFI)>HbmG))VaRs{bkfv_P$B>eervy+pF?9H0z4&E~|BFGBa3Iv(%J0BaL z&n@iRDr<5ax?XLgI<4c6z>rfj!3oRQA0woP^GT;SyMyeKJj;*+KnUF81&VQvO8v!E z-94Wv+P4rmEHQu(hV0>JrDas!&`x|Wjbuvd{r}=<>E|W33bH%Otdbf3Vml*LjjeXVUcCq<<1qDk|ZnOIqn^Dmg`z^O)O>!7`PEj{5^wWBamgiY*Y}L9SA<*5bVU~VsOb5=(W$*(zc1MO z=jX-e9Pwa@OB;g!%b$FmpwI$E*)X&63KB)CKD0)j)@6RY3}ki zcso42q@0;pL(86FL`Depc0omja}N(6o|dO%?dKRHuR#<**EOKn>d^4|@^4X&Uwt#B zi2H`|yXTKIa53CI)J0VvVfw^G!>P$ljIDH0!ZS26fY;k(`N&1sUbh^lyS*Z*!-%R{ zC==YcE>Ac8obn>|RXMTd!E1!37b#-#!}XTw+e8FXnDW4;qlUbQ13i)n91vYSTQYbK zq@CufsBW@!a&q$V;U>gH{n9;Z1ffBpp&8;>ds}f>tC~oG96f~uWyC+_b9zsLdH*E< z2FZ0ib`rRrka&lel`7r03FT4OIG1UQmNIbSqmb*koBD%)If73LK`VGKQ7ZQy#dylw zi02(1J2LV}M2J}4`f-f5S<1-Auy;J_>VHnR1n7&3B;G~RuOy)IbA}T%qJVO@prmkQ zy5C#0)Ms3SgF_N=u+hEoopegjp`oXfUW*kRLxO`ea;f$_Kv0e4%4V+W;drjoa6)ao z%zVPj07*7bT~*oha=3qgSoww55MzITznYp_R84-~#Cw()1zEFJJu(zJGiD-Jb1!Hk z9^0#}Ay@M7uu;g|$nTaUO_?;zS~iSy2$Ti^2|#h0A)X)lsAZ(jMnboauj%OM=<3Q% z^A^j*$jHdaSZHMXkEL+m*RMqjwjfiJLbHz;O*;$>EI@C{vZl_fP~><+h|2U8wXV7+ zZILRU7|Bp{{pD&A3R79eF01|z{!FVf8BlP+>lKYEb*~Y=FcbqRiJ*pyv>rIV5>31! z5rpC&+CyvR_vs@VL$Icg z(>ABx0+6Mv@<5mJSxV!Y?|p`S2VPHlTtq@(U|V@Ome@4E(Sjy`I2K!*lhPG*PH|kd+<#Z;n&Xv zq|+wdnqsU~#aK6ABp4KH`IoD7+w1CUt}adqXe8pWWM{>^o>v+Y{Vdu$FxwVvfv5E1ay=aPkb+B-uK&R>1!*-?FuyMcSS}-l$MpL z6s`X0YZACD|MpVD?NwtlGwa)*?-SUmqw2EO<~3i2_M#m=IMd4RQGo)vTz#B5E7>zLgy1W6!CudCOGH+hmetn z_A{Eyv~k4o#AlL`xBQ-MYp&(*Y^eDD#h)6Ve!9}A2O1FdrKYy#ZYbp?T-|`BH_5G4 zm==qLJKp20SDeelVrkr9aBN{&^ZsvdY>~NYi~xuo$)lzHJ7kSu5x{udhMH`02J(L( ziKSX)8fw4WXZ5AOxapN@Ket(;2ntAtY|*57?le2|H|t~^?UMQMkW@dUxMTO7yov3azQshDR${4rTVli;$l>m)tago~A;IGL&C6?7j=6%~|A*UNe1Hkpb@ea;|D? zYn}8%s;FPR^bc2YdS2`t{=V2V;bqbFGja0lXlwfG;e7N34F{giXnaD9vBO%CE3W2{EG2C{%EHt3mpd&tLZ+U@{V58F-;DD^e5eI0Z(vP>flo!wv#^8q?)khn0 zv9~u(JxrCzq_B*Q%csmxG7gWu3MC}!B|ZIq?}7r#2jh0pStKiDS)UvqXQxw~&;HgR zvRLuvGJ(G;E!I+IAe66i;v+NA#=$jIO5ml0ur+O`M8b#@o^xtIkr{KKU(pot*GhGEba)8A#9xA$f+cdc72wj7-W?byDM2o8Dghlv631 z>$mV|WPTM|^5~;*lFeUK{nzq%||@QTYM-IZqW6r>42BJpGSOJ z{0ms%>hi=!JaSt6$%@D6pEUA0=94Y>v!T6(o=W7-WQERGn>DZRF88bHiyHVcaupWhjOS4~*cUdChdBfU^p&tmU6S8H{;8~_0I?JcEe+oZM0quEXm z=aJfi(rKDGevM~tV=~ip6}msxFZQO{dhy0=R~}_FD(vkKC@7-F^_#qRE)ZYjY)dAb zs_nVBFE6jWhNGEg8S^IRsIQJy)ss6qtAN3@)?+8q5uGfyoOGKVDp7gn0b$#M;`zrp;mr90cL zKR#sVssTwhi^oJ+$v2Eolvg2uh*h_5z{cL{c>hR6w)t}~Al>eC;Ia;tx=jWFZ59qWw88|xDa3~6s0Jj^tdG1<25s7xZY}w6lP5wGN zI?nNO99L3ES~xCa{R6t2fU)uCy?H%VL{V2Aw9`XG`1l z^5XQ{shRJNCSE=dT=phq$Y zJ-!HB)P=PRvwcTg<~DZGGj+UjlfW?{sh_B`&$0#QUo+^_TGsQNmuhso3l|QKf(z6k zeS{D;tZv=4*o*U0Iv?y~?>V zIj#J~TXQvnHb2*!PiG{N5ma9gfOZw#Vz63(qg!pe1tk5!7(^H@= zi+_+s5YE1%7pD8Tw7z zU{;aXGZas#1!7Vg0Outx@zqY7rOJhJXyT+Yr#dv-FL4iCncy;0r|0_fB_8UjfWnTa zD1JBF?dDm%41PIyqQ;iGz3|3Y2@y zuj(PhbH3Z!b;M!}3a4~~#*XY0BT?4Uao*@8!FJ)!!A|KOm5Q04oqRh+`AU8#gm2KP zum*a$t?1^J*rBfE&~245jyTdliK_<{%}CYIe4}O&4X~Zs0UxJeL+|dVYzcT z6U1yY>mWwZ)%y0M-GFiRo8d=o`Ck5&I?Kdk+RCpNLsYBrvu$lUo0ZGc^OgeMe5;}` z;9AzNxX4hV=m=L1LY{i=-!~!bxIGG+k5Gtw`vv4|%r;+|ROZLUND1RQIu%Bd4^xho zO(!sImKuF_!|LWMyjK@~;~5$C9;PaDCFo3K%Yi-b`B?yE%NEDS zCORQF}jDNtw!ZK*{s`;EYQ;L&ejK62h;Jn_8IPO7XB$Pvj=+t#`152#2W zm*@Cy*0d^s@a;rt2I7adw6rF&$(4Znl1q-0CXLmpDTjm3E>1UUY%Q;U5ch_Qzw}(kn2r&1ILiDv2F!EF_YY7Jnq@^7O*9eC1 zG?*}$%VWd8(^)h8_;Y9oiC&$lp?GWMq7se@|9OErY;Jz`jNi%Vd5&H>p$zxzyvc1- z$laa02H;G`$H!MzR#wsR=CROM7e=Q>+kQ!Hr8WzWu>R-cwmVJglW*U&X&D)*f|4{~ zcmtrK@aQ7{X%F-=cvoHRnox!VZqM#b=yBH4FSXjyk?V*FZ`TduiW*~hNQ1-nF2a@$ z2+UPL7EDg^M`!*5PKVs=d3U;;i0)7tl!A}dT-S71BOL5vz5cdVt>Ue7IuMm!T2v0x zEmZ^-AYzuNcAEt|he+>@C6E0Mj>2zIdMkrsDXmCMoJdh>1>CO*WOgV!7w)Yrepr@P zE7xv%IX>o0#l^dlaHri-q^6=WJzPQb4h+%M)I=uaZ8UubVvbu!2;GjM_*B&>h+kR`WY#A6Apr)tR0=k}PNQ`f*v6{8>o%sF%3*1G)@_o@L0-h?<32+8L5L}Ez z-iwFvYTHt%fjC*OLVfw4x0`0I>X@AqT1#zrObf@DEMUtFu-Z0(LPSTmtkmrO{99mR zHf#i|qjj*zFXvLDkvWIhdcn*>U8FK6MTm8Cfk(@Aaq~Mp&w6?0sX|M$g4c)Ajqk!F{-9 z+H%MPtMh}i)5Qi`y#-OxM>RFta&w0O`VOPT;?1{mu-*n7a}8nh+jg}dr1Cq3fG1@U z$yyk&`uimm+^c>d5DN0wuV2)kKZoBxrlb_J7LMb%U?hX534A(QuWHs?^)+{`MJgAL ze^XV(4N?lQGn(5YUmaMWlgXh`F7UcHVn(6XG)hJrgfO6A>UnXWtDrGUwHKAywe z!}OT!0v@Fn%B6?m8TJPcr=j-sQzy#aJWTdH-^I0oUWvp=V~*!5z`oVJ-M!c`F9fK$ z6iGO*+R%s+#46J)^`>DvOULQHigK%^%0s!?RdA#h1mf!N-<3vPY0b@F($gHNz80Q5 zk3V;DQ!H?vq&YlpQD`0B9%E8`-*i5f`fz>Kc3`J-I~H3w@o@ctnN6qh0$hT${c%pq zHW;pO8zG-@ zJ2oX{=l#(`Rz-e&L;d)V2=R$}%^tV3gSWiSHxE^9o-V$iXZ{eeT(yeT(iSyV<`e8j zXzO`JSdvKhD_F%7abs&6%a#|F$MLC%Sy!9+x_cPd*o`f!!v=PCc0m7(bdTm~JjB7- zk;E$ly~IJ%@4Hpc0UCc|_<)NGMXRT$r>h^7ybq6ImwIm_zN8MNj2VU?zK;;X4;A2K zd6l%7$4^EqZ4xN@hK#i@tDXY8``E}lB)8j{t;>^>F8UxvlQ8dm#I8N_zz!cnh`R`z z99>Y5ezKtZ%Y2>-$RL-oZ@p?D7PMqr)A9)t0wa||I$>sR9_A~gp9mX;s^TsLz9EVq z@I&V=JUW{LVwV8{Cy}x~6o(x<_w^?R_x&=?uaBn{OR1@(7-7d7opM%%#u!jW=-uy5 zV9)eNYuG7){)W`Tc}WCrKFAvNRWC1QhtLs0ZCo)y0Dtu2ne=s*_Wf9QzmAyjxjwEh zDENgwR|QA1G4avXYa-EqW_;Dtq>Xo5zT;WJGp5luTFguV^K`Y%xv%y&k?*}==FsfU zmlH?{RaZ(q^ni9&9|igVIO0hpuekaaT*A2>WaZ5q7)Req+v!Xnkf678rnqGC7w;&w zjxe`=VeCd`^qIV7Y#XLp-BEXXv}*d_-JxbSU7zk&dn6(I4H$_x6V%t?d!k+?tKO1N zbqGa;=!C5-kMeH(7s~7*NzGVBeLiC6s*E#TeWFi#?(t52yJ=aUtJ8e{YR{FC+O2A| zmH+q}t8)jDJ7i29fOX!+JZ;-ihVh)yL_?1P1}k^Evcn$GJg{A6PnU%YyAq`^Kfj20 zr3?lQzj%?@btq|lbzJ|w1FO}8Ajah1eo8sP-u0H?ZX$V@d6}7)UcrW+TDVs4M`;|= zDFsO~weES(UY1LA&Y#20CbX`ghyaPtA!+@~P;s9%5|6M6ZJ22HQE2N7dVm!3*u2Um zL>AR&N`>EoUEgA{`QqCETvO9EyZIH9v~V|2QlvC)8fIk0^fw zKEbJjtEW{z`1b=BQAa}Dv++3@1k6OVLF3$tWEF{&@IkeIM5~kzO{lHLTP*pP9Ve`k24qU}%<2j-@`?~_Z*W&dH&4VTa zQsJYjZi#$6!Z-mviZD?7gc_;SL=SB(`A<$*_9} z9B+;LBC)Vl{P({!sX?ph^?uPWCBV&T?Yln3zFmGmOkI<5fv)5m$i47mH8?LX8z~Ha zdoh@z6yz)bp z-pdviP6vCCI(6ttoxW;n0zBdtD7wpN0sHxEvU}&r)&r8f)`cnbRVhnXijMVB>80l5 zCxT1Itw-wVVT}IJx^O0DC^iUUYO`LKLLVFD7MUOy!3~|?FM(_}*VLq0y z0ZB;3>2>NG!N@o_HxS@KZK9fo*f|Ax@y;D(I@Y$q0my7%56I~}K!w>UF)RE(N@=dVF z+!jMqqTu`|r`80jLhA`}=&!6A^@!b3UWtH@OKW4403r#R-%n2qEbIc2n$>OwT9)GC zo>XK(#K6!}m4FV`OKCEY$F>LRl?tnm?2y9}QwXh&8#Gw(eloj9{T;z0t?!(xUslSQ zcC8@vIezJ?$eV2Qqa}q*T=Y~!Y(HsdcvGVEVWz}%e6O_2&m$!WO4$@P^DPV*40-A# zZpXbuJ>AMEN00T&hMq_(8f&wG;2m9pmJw*xu(;E<Jg*ET@u>{+T1@SyJPwz8VlAU z&4KSz1kxJIH_|M>4dSLWiI9zr9L_@C>HMHq=uw*~MpM)CcaDP+M@yhjT3?%+nw#9F zp&~pSmnTOroh4fFx^)8DliitelvKny1FXV_v$RkmS!FJT%>-!b+=jaA>%;G*d8&+a zI{bp((Le<+pf2>6+-6cPUopctRa*3(IrJ;C*_#MG^NXSQz8IO;QF-tQq6Y25nasUG zN7rRoeglLqBru})_7XyMyn%eSGlzPO-kEA$mt2~!{9G%EBG&qJzV^NJV!rWBFKn~J zo1loNS95LMiu}zoX^6+`KtM(-?jjFdfkdNs8YY|~yg^2~7r+Zah z%Ox_=g1ks1WWPt149yQO-|6m4t|#K{PBLuSa&VG!_MgzQv>*4Wq%(|MHa`HuS-gYq zeNU8vxJKj3h`gUVqZRn_W<}(kx#J{VVcQpbPBIl?I86AlNA|yhuwt@XL?EPOX zgg`ud)TYZ)%&|CJ4j0OaWI@rCCB2B)QsKmo;WI4-EVULK6DC_>0nQQ3ZJj=NRuMt( zLc|GvR8vt4b6goUuj$j}*Q-ZQKHFruXx4ifYiJ|`c+qiMy=z{MX2W7Aa87aX^MfBI z=1qO=X};`TV_GH0Jg-Uo48NLrP828=YfPK;oL``i9<$gtDEstNi_#!u;6gbXa58Dq zTODX@^2Cu^AVEVfNticW5q*jt!-GN7F{VOjxMUR_=M@Q7KI}`}DCP6gjHa#fQrhm;-a%-~^AUAwLt+()DzE-i zAB`H6J-j^Ho+-52)(&3D-##fYt#PXicv7nJOeC%tO5Hujl@K%NpUdm#*JNp|9v~TQMW%jj7gvs4 z_As3DH;cS>wra%O^OEWOVH1;~0yx}vC(6KuK%k!upQ2?-z1ht^1y>HG#c-s|T!oJ! zWkDM*sMJD?>u!YbcDXR-VTnv1;%x($uj8b&LpJ?GKYtjd$5qdCYLvC2*?JgXY17x& z;_WJ$EkmX*6pZ9Agas3YhXaf1p(eAo;%>|duLN-Y%_BHvme#=7&(ecpKtyLFW|qPz z{Wn{*R&b_PUosPPnKHt-0gqc2)>x^k?_Q{U&QHq&;zf{Z{-DU2amiPvRO6(BTWCR4 zT?{d~#IH;J&qWkG{O?FGmyyKI5K+EaJ!R&$I(ciqA#ay3n2hg%lJT=C3H@Xs6|6$w z-FOq3w-9(1x{eVDeX<1R2pwA)J7e$El27?qivo#ki5hKLTmZK_ zK`2XlQQb*A{#Hw%AL3h17EEH=YNe?kY`I|<<+A7`@fAQb+)~F>^M#L4>-_m(a z0?u=)!wD}ADm7d)^iP=ykH@4gea5&53wRl$t~v6@M^JS2=O&!(ggQDt5)%06vCD|? z&Mle8xsYLp!BilA^@}>g%>>SUSoGhP$4tU~ea%x@4T9 zbK@so7d^wpexmTvd>E~TGlOqSa?T>A`Ip6b)t(lFJSemLHaC`D81zi4$)lxlxX+K? z(A5#_Q<39=yvCuwkQc9*GOAB1Tz6Op+K=+YuMAe}Vn&9zf=%R`?}c5|Ua4DEiG=6@ zQNbuq;-%Imq&Y%AgF+cBSpB*Lg_7V76&FX+UZpljZbt1Bq6tlLP;B3D2fmw*nBDJo ziOE5r%y_uaWLDXWR8WDfV8hGbNOe>0yo{paGk9T+D>o90eC+W3fxCI?EEw=}Kq zXDR{6s!sq)JP8pfH}xNda15rsr4?j9Z3ylzaD;&a%{TabXM}#wKZBKc4bBQFh>FE| zO~Z=aLG_sT0}^U%8Hm~U;(Jp8w?P4Xi?ZcjwAJW(xP z<+11)>z&tN9&dxwzz|KS@Q1m$LD%u_h=1nGF*Hejb$lnkQZt)N2s5$t!oK9~9?H`c6Y($&kf30uY3stE!>ok~tyO#W_{ZH52#uX7EfG1TN)kg0jq9N%ZphHyU|NVK! z7ehK1i1_$;+}=?U=*}-IOrg4ezSHGBI|Ja$FYnU#>(_66L;$7xLf!BXS!ux$RnY*yMnq@mjhtI{} z&jY&|<_+xWQ)vD-m~K)-W(8Q{FLp5C{%?NwgAAkp5P}^dz8RnRUpp60Xi+iN;W)c& ze@22gm6!fk&$x5V>*#>dU8Upnt~AskI%WwYL`^%ovDZzZGb>Aqy7GvfBSgkQf_xXk?=mZ7gUKtIfCE zz4_J}aE$|}Duk_C&xbCXt!+YS)@R60^Y0>Tkx5oBY2_Ix*XN% zL?D>7O;|AAWo_V0A$$56asf9gRqGk3D=j6^I*Fm?Gb+^S%$UGOMWhK9B8-c^cI3O? z!_1|%4$02&GE9+|Y)vu-N;3G0h_WP7w<2_Hq(!d3-6P!k%kiJ^atkG=D+rH*^#0>#-#4bQ%%KI!P6m&%U1vM!! z9{V#;7oSmQ1|I3$RrQUb)J!{SCl0XL*x0@qCWRm_qW5R7qW{@Vuc27c;T2NiNJQYC zanDq44Z1Jd3Yrt@JD!L1=uwM< z`=FVc+Ro+va(7B==!OyCz*(W!;(pyPNr7I+`Fl~x^`y6GxOtpjtKM3@&1k7t$So!p zOf0nBj1|w8=&je3G!om|{p#2{nlF^cCkn?665??=({3>A>do;2YN@HYZudt}jHr#K zaZmq11gl2gP8O#FGa%!f1C77WmZ+V3-p1#&rgdengoE`|P8n9Oyx#KYgUT5PhMEmolreMCN-+bcjHkW*D*=eEMQEKldrTdYdOhPx+f z^x8e1Em1vLr~(}I{_OH^zb=%|=y0Ows5}_pf3tVpJ`K$D!8PcS*@C3|@5M(@CBm9N zvzK}N2!500ey`Sa&xf5)f4vnqVD8fhOg$iy7-vV+s3Sf;^E zD-zUqZ3-^3_4c*_X#!4X$WB{WYBh34pf!>Cb#~!*tKDu&L&tq?pzp(N84r7poS=rr zBHr@6ZoA!^g7tFH`|sziimJR-5}Y^VdW{Rm)}Vt8B2tW_1F+>~Sz7UYGA$1a8*DPF zQ|oD}q35~1ONR+%XEK_yyguo4GwrB*7RTqQ>ur5sUt=nl`PCV$eB|b7uraqOZu6=6 z^!9^>lq2HS861q5=%3w{?1*)`*wj{0_IMQ^Mk15cpns{L_ni3^1egh0^33g_wbfeO z^X8e;{^ASn+w-5kwdXhHDR81N;loUPhow`E41ne8W&_-vtc`0iB{eoXK0jb}ErSV=o7uhyj^$&AlcBjHKUr+;zDX=`h9I<7zWx(_CLTdsKWXu%rw zC@O5Zy@tZVp9e?xjHH4cg`A+LlI*Up1BiNUFT06}JC^-}HL z-ri<*_TU9^*TRMe&18=_9m(@B_&^IXg(nR_h(RB=7n&}RS+2Ki5cXpZa8)U=blpvGR>b?y;I^x-EU2(>fs?g*1ed>+zkzq(v zK#j2Rix3t+u(2&KE5^?Y2pMLlompF3oL%eA@}%ecfi;$TdnK*s>)bbP-j;4%$KKGQ zBCl&O)WW0crRCF9#Q$ymcdf_aXmX9Ix3W7U14nH2F6vBMLWbyvy@@pD*Wq}&yE=7O z&yiK0c;PsHiSIU>asPC|+xczuG8c-C*;hz$W*MT-C+I7zI$VKp&#=L-wekAq6dnRO zTbs?G`-X}}#Y+`HrWW`~ud?;jZ{D=G?oj3^-Ys&r?{+Qdd+ zRA)P%rO%%(ki`*_iv-}doKa?)L)_fm0fpqa@ExAQIVR8Ex)kIfW0m?%PE~;2KR%5w zE7#jB)tU>#?{cjyD2w9z?iz;-`axh|VZj(Fw_BB>O0=fN-Otr4eHI4mX*!OZv!jnI z8ADUm^3Z5MXQ(G--uE9@w7)$a`%TG^D1lS!oiH)eag5^Hc|MUTMmak6AX`Rv7Di!t`WQ2Vyf6MY|i~*#85z{@_J3CMv^YW=D>amqxsb zZLa=lh*~W7=u`i{RvP-0-XR+Y2jMd4GP)q+%|E+c7iA|~pvz!yV@O89-w&;;0)K#n z6kurX2!)P}21nK#FO&Rq=6TM(_L<-P#E##gl`GZMhNgISn9VG`cb0F)JX2MOa_hK1vN@^otKFzq}|T#Y8?N6~*wsz2*}g8cmtURB4v38>tHf+ZiOJ=%`Eyk5>4Ma__20|g(R@$7Tq>W~uM;@f+2pb%h5$U(3iY2@g>f-t`wLZ;`fs?^x?y#Sz z$IL3sj8TKYZ3Bd;H)3-XQI@lTWToQ(VpJ8@7L_$==`6nt-_f_sGj_PWjc#P+CTqJv zLm`6&IF$c~<`t@o$CmbJfUCi1OeX4lKw5_6UWLHG!0AF+TWj+OvD}bF(_UxyZKWf@ zzsQyX8fmVoNc2&YoRU&qO_d49uM@$;Eeo`@dqu$0fQ006izNF&tvs0%@YKccb%tYF zSpizM?O1Jgw_I1Kt#`M3dwFzkGU%%uW|5VO{XiXXcy?lB%@J?ii6t$2kyRVw=y%xZ2r_>nAKM?no%=J|Ki@Zyp9>6>MA;nYKpFW#eS$NG*?Jy}ttul{x^PRKm^&bG6N@u9 zYb_xY@wS>i7qB05cmQFaJ)Wz8c`Pgr~tm@zwcns;;ol)Et>Fku9CAoSf3ou5X(cV!PI z+#WYr(O(1Mz%d*hO}a%m^^;aBh;W4dI3W-N6WR}QL(=Cy_nx{&8!t^iVg`Q~j2VOd zW3Y~vj;~a3$akUwk=G9`LH`+rQSbqf#{#%3r9MXR#D60kp|iMX7A9Xqs=V*l;AjQ@ zv7xc4Ilzhd9!1g5Kwgv(dHUF5-}Rl4QD`{`DO$hci0H4+A^McRhs9}xz{o-7?IVN; ztOOeb#!4L4-wkClwI8lCz0YGOzP5Q3_I19$1|tU6u5uCxF+m8Q4@8z>$(Lq;OIn7{L)!r@>@_f(8t-<6j9V$c;K7PgBZ11b6uO zg5F-|Y4Lc6nHBu(voLw*-BOe&AoZmI!_$aLKB-J;>Xz6-GrRDtQ0{XxG1*8MgR?sy zQ<+d$9KY?d(gf#5X08jHuYcw*?gZ23<~vMI*us9yZa;k*!^VD6NQ2Gyj5vt@j1+tW ztUJK+G4o8T`*jFL4dn0D*~!`2_Sy^ez39=Xo&~^sKydt3(Lj|Jt5t7sdU0U~@D9urddINL zx`@gf{K|w5j5B2FhKNJzhr{v%Om`^{KG1L(Hf?TSqvMHY@Z|__kDheCoP!?uX*h_5 z?Rcrn9KJqm1O_mH1KPor4$JW7#e;K@?r|Yd@kC+$3}9)Z0)CQ5Mj$-Wi-&TM^lAn| zD#03{LI5O~S!0A1Ab3ugS$il%5n!#T=Vt7a@KcFLU{}cmYJ_)il|oxJ{BDU^)d;6s8Hy`pBZh=e{uN85S7o5V8<`yHm$d>$8WzPcJIdBdTvw?^``7g7M9V znH6UHSTV;hiAnVm?w!x&550v6OySrbn>BDCBWIb8Zd@i-98w&>`)BB&AGQ&^h{z~} z)t%@{wXEqVvL;g{hgUYDhU2X2vzz*mF3P&EqO_QfHMCX-hGKUB+~)Bh;+ zl%}6Ndxp#dzT&wLMGcIWiZ;4l?Zl-*GQM{8qW?A_xBnU|YawW5cpe_^%)Z;$Z@r^g zO_DLpH{uYm?w}w%Ak#Zk{g~2aKeXM+Nds+o7X|TR)F#vJOwuFoy$gtUO(yRk!Xaee z3jWVlLW!z+wL4uCd?x9GB!65p0pwEoG=j%^f}d>BKcDtCDMr%1e^iQs*9DRvM*PIm z5Sw(@f9eCd)GKQ8O}M0HQ>|Xaw@8rnQXO>JP6#}4I`2fE%uRn6b-j}aS?X^7P=c=2 z&mA%|{r_Q{`avJ-iZ~C97KlIeL#ujtp3g8Z35+hDM$}sPQ%g8ZL1IGA@^E#U`sudl z>Oh|PR=pq2F;cCVi$;lIWZPl!ELHHBxuNLC+noO_1^7{qQAe52Y#&)p$*MyOOy@6f z+?&$^|AMMK^=&~DWZK)ao?y*?$+sYJS&y9@jNX44H)4ELoD3ezdhGwG!R{J=l0Mqp zp^U%O;AUS=b@i3xkN-9=GM(tJ+FL~&HJRvW-k#@7R*B?sFZ4;i7 z(x>|8zaZxw(G~oEu>b+lNH~dSIse2ypax-mnB4uBuj*Eg97|;0GymT=Lq7jamOS^U zRXEP|U*hfGCc$&ntsF$EGz59u`_C%_ZVWNRwEVx3vF?ps)nI8}{c~V}{TkXLx&I+9 zc`DTy&ZjV)|H#gmw<&qjv;TTAJ#=9Cm#>5G{?V5t_~p-F{<9%Vp?dzM3#XVNLb!ZB z|J#x7>HdSY#F6y2q|Z(7LL~Uf7kB>O-alw?Q|>ZZ zf+o<$6(JyEZ`|L(*cqb#Y${je;IhbL{Aa>Uu}plTz2IqzOUN+^nFm42H0)FIVu{vq zQYr=C$}XV*qe=M@$KBgh%bHtX+GECvB#W?KhNxu)<#n?;ITlxjA7mvTC1ob^#c1~5 zJWN_^@=H8w;@n#e!=2u-E?Xj6F!_IzZTS02T#%w_;jazIfAb=WQCI&cR>LwF^w+!) zhPSAFPg0Ny1(BraNJ)N!Wd%Xm*B_dkY7P)niTD*>Gb#~=!ou9b{h;a}7Gc;kYDND$ zwiF)8*lqq+V4=7yfjXMeCA^5eT+OgEPaU}*8By>N81h0Mam%7Dal|LZ%sc8Pr?6>E zZ4#FB@Vj?ezDnsZ5*ConHK*M*!6g}*Ksx$Ms#yeOLMcDVC56=Cr2<|7gK^rorkGlp z=iirXf>Uz&m*SlNWnIqqK|Og~K**o~4z*s_!26Pp&;MEmA+!F`n~gr?l6p`jB`nYr zkj8)&aRommsnex4!15hS$I9g&ydL!{^P?G3v(=~srTWM@kpY=iVcCT z)T1D4mDLRmA`?C=4F|Gk7%LsyxfTGswb{vi%1K2Y%j(u#H5VaQA0IVq>x+KQ!hSCV zB08KZ8ZvX0Dzh~f1SwTz0P|f#@-c;8qN(xk5S$=N5G^GMlme4qRVI0&>CB+BKYrxm z$9{R~4o0aW0Ufcf1?g*B20GbD1BVbDvtT6qdg(9!t{V5_Xo!-%+ z@hND~3CKp8-g=aVB6X2*#FZ{7Bbro~ZO+5USBYDNuvf9z@eXUKG#^s=-a;)aFn9{e z<}_~o<$hbS!>3Uec${(;+U^$XY8LnP7eO49&FgG)_E_(b-e7KjUx*i zz~XcGk*E@C=L&(To43^Sj>}Xq_9@2uBA~zEg?8fC#gPdVVw!-e0+gBzD3BU|hfKom zPl}9;T+6UA;n+b%hk~w$wYNf~qtZ>$BSz&+WWB_R34Kot16xI6Qr(R`y|}nGy$l$M zqWOhg8lU;vW$?~FG)XQ@KnvvS)bWlRPa$4JpfL z5&Z@L9O#O;5~PYIqF*w{JePEm+|A+0#b~-*Z%voTz%csZ^<|&GqO_FPV=Ilh*7ba? z?pQT=q0EH-k7>VkVM1i2`~J zOt_1T8zEP^v;@m`Dk>_<@6Ki`IxNhs;o_h?u1`6yudcp6ZR;gffu95%A>Xu96?u1e zw^ml(e6#y_mf=H*u)lY@RuAZ_hN0kocmxYmkLSwBsN||SWF{88vHl)NgwgKTdyvHi z1(e6JXdhkz`wN~nyGWifCtu;V{xZal0ZMb$3OHO5^y##5Jl+oWdg8}S_`ARk&>7uo zdODR{8Vml_H@72c*4FH|w{nhm*m*+~;J9%FLjBkxI@vr)NIMO%rJ*>F%8cEMfl_UX z!8`YISojPHes>r$BbHqrLNl zw{&1@>ZR;e41j|bqybV=>0;rG4J$Gs5GK8sik)holIOP4=I{gSeW;U&g&$J}3WtN} z(5~em^`Ulul^LfdFrDW}^}b+f8*{q^mF(>qYOIBRFsbv>Px2ISE|^tkGgPG?gq$73H&rWzbWcTbP*L=^tk# zlSj?ZxH7lobhPk3e`V|E&`1LPTL}^9;9)X%zFdvgVcQ-$^cNH3%G%UJ-z7lCrmMFo zm80~ztF)-&`ISg3xO=(YN~h&uw`Sn0rNub|HTCe2GaKtbbJ)LUX+Wm}v1NO-H8v6Y zMRU3WOObmP^X8EOJ=UOp+Tz_`gCs55j)tPv`BsdV)%dFSXnCoU)p)p8^hYA{y)&4Y z&o&jXyofVr1WQN|k&tjOguXY}Xg9b|_lKf~X=dYQFUl}lGgWPIwY%Ee;z#zf%G$K$ls?kH%a zu4t>EP%KlYWcX<=K-C(nNHK-^RA8)M%e&o2H?=_FNspMoC7&O0tc72~`JMK`R z1Qz!qQtPc=FB*KG-aKtxIcB}?4yjVcSdA)7w?BU5F2_q@2JC@_6=0?Ud5&HIQLV-S z-*5C{LqOSEIL`c6d0Bq`E>U}jYga2b5@!meoA9gb?fsJ?wzZkfDVQw2U(4k4yc>-V?y4Q3~6@4j1bw%U5jeL7DqTrD0Ro^69G3~Z*DE`J~pNX^rOM3And z1~*s}Ikp`UMEVyu@wPBQ;yWd&)0jSrtV!Nu-|UH8V5o=#7f z(iDbf%wzFNI>OA{vIFRiwe;5Ok5-D5*=1M3%3WVj%lYeLly9?p;UtOc`a*@M?5Nne5l-^Y7h>pOQg|J$9burMJ{dFJGMks$7x!h4qnZ=Va+`S-R9t%qgz zJ`btBuf8509&@&f)qwf&it)^fCQrsfqo?-yI~iJB3f(KwbJ!gzE1ST3=!NGVv|;b4K2Nn@6kf&-QD=Du~z%_ zDv{5FZO75s;YzFV^X4hERtgnJ1EU{$M)K}KI ze#Nm`?%hPrJ#}^YP*dp?0;nDfN0bS=_n*oK-P9I4 z!D+Sm*uD!^#|3TudZhr8#!2L}g3az*xJ%&&8DxsI=xvzoBaG^j^e zZF3?=J0oEI3r`mVSj&!(1gnq?kp=qXaAdm*#dPSNlG)tw&%^%^{1^CJqRz^IkzsS? z9s@PB1O_c_pWETqM9R-P&3RXHvq3K_IXT_7cqzO=LQOE1((}IFmpgqmRKedQt#w{^ zLTwXm)|^V7l;81OXfA7=tE#ZlRgxyMbq^1@JD<}zn#zK%;XrfK(>>pkev$<&IJwl% ze^c>Ppyvzpy|2k^zqyjJbu+tuQkgH;e%ac&q!!BF%I3rEwabEJDgwtnV3w%qw7Z?W zBs0SPda&{CDW+gmTC;rI?K!rvEh#UbE5+Vy?cM^burhd_!MebCV0t7OlrQMsm*4Yl zy1}ZFAZ!(P}6ON$yll-*0B9N?|dcs;`!n zk<;;Jjp^|l>)PTPpOK8Mw_Y5LG4mF7w0R5aUir;Ac8`e8W1~GWJ3Bkg&!)~|o}*Km zr|&rlZX9$;1d7pz*VO}Ebi79K?*UhHwC_zE%M1(~fu*Dh%r%*rbI|25ax1zlDjf+Y z6P=|R+&XPNZGkO*{!^JIhR>{Fni!>>M0RuKi?& zfq~D03H@S!JsR&X)xPvXI11~m*5_M(K)4o$>5E0UI$tTrIb-m-za4Y`?v7sY={qJT zM;(h0IKveTxazqxk`?0MG_DE~y`@^|@W@aEG735#oDZy)uoTIT;`r1qog>ZF0E+MQ z+FB8fkpztp?`p}dORc0^myNuy(E=GKXGde6nNWGNnsZ<>J9n72FoxT)aab5a_ol*X z8?4Ftg*H0kx8D`B7A6>oHLy6qW0G&`yiHkI`s5pNbtgL!hUk}N*_fZ6NqHxJAAH5 zDc1@YBe1Hfs?hMztMICx`K?Y`8yK4lGc_%4YiH<-C_LakLbNqptUUP2~_;m%xRkv>wcBN>;Ca0`gSfrdsU}xFq~L+dU}mtyFHf;EG@5TM7>ZI z{(HRq{rxU3E>0v7kd>=K9VOBewu^NI-(tWNecQnjqlTPJff|W?3%cV3 zse%iys3<@ zbpX9!z}VZh@#~UQq{aS7;4RgbFF!9wN^%-GU zgiYW~L4RlW>%>_zK7LAyPWQv6j6;h)6r>|mlLgtmTGhAts$DQ+%}oToKZ~0W22F;^mLy zQhpBQ9kVS2;3fhS)k%f@8yh-NnPt3qi>-tiER07`K3Qk0Ul#`u{iwfBijOsA_`W3{)8MJqjMsOAx<#HRN!bm$p z(U2b)7^pE6D*)H&;oQA+L_$m~fgql=a_%Lx`%7Z%bN5uDtiTNY2_V_N2 z8LAWxLm7+?p-PB!_ni1-q|a|{J& zM`G-rwkIpYV=5y)vw;u`URK?x{@CS~YxpF`NLP1!Z0eJ_pezyW*RyOiYIFdce67`@ zu*jL7qM{rr88I<22`Mol2?W@>BKTIInKEpm4^!?$}cWPQCwMAL@tR2NfcE2G`QYDkciX+#%OQeP(fIRC=SfdJA# zS|;lP@!;^=#bf8uj3guXg)6IYx=P||y zkjIXh_)QP0D#*sl`n%jJWz|^apD%sH#qlXZt|Acz6WMl}kqDDi^^mKrcP)KZyD$Mx zG-e4F5#R;2vU~20rs#O`jI($NAQVSNMY(cub-zB^jiB_JWCSkX#w7y7dnC1!PBUt{ zU&ciJpC32VZb?LDen&aKH%~Uv6xp+~`H|7OLhQL(esnuIl>sYZrne?Q4Je=skVf7!NPc77wSOkdT4ekAn7gSGCz_#M8T2<96S@1w+9DeEhYi zjh{49tr;0x&lg)tG!bYB^(x=)_Ih_XoVGe1Myol~GT$E8e3pu@N{dn!2h1`5}SI09>L+f*UDevkc z7=k#JOOcsn|t4o}D|1jBimHx#ca^JU zufi@6(%bv`YX&Pi@b%RMK&~2UV9mcJ>(W*2F|`>o@aub^=-c61m_s&TQCb1`UU3*r zgiinn2*wXX@QKUm^g8c(1v>~I+@*mDQ z_{xs0t<1`sn}*6I=((tQ>O)v$WozneZa(f=%`7ELM!14YKbhVAqt<#+M`!WOmL1=U zOvuaY?p9c%?CRk2K7L2Lt*yt|=1E4bfM0W+lXK<6dd7T+4tnRCr|p-nveu%8_A~K0 ze(nR0#gmN}m-S)ix7@euli*wB0eXNKZr9Lvw%p6!Dq^-JN5XH}$D~A8@nd5B^$uSh zZ2d#gz~4DupOcU!!E-z)R3aQjwpNCUnz#&S_rpD9{aYBZu`sO!Xt{CUfaI1;(zVzi z6m9Hx5eRh=kfrLa)h_onDF4boc&&$(GN;%3Gajuysh>0!O>(2xP!(Cfx1lX}&s%n> zDQ``+)BV*^M%A3noZGn)6|f>3Y6L|=xeerhb%CMfaLKximk+w_U-(Mjo8)> z#ZyyfVW|G{$B<05aknFX2+BFRY(%JduSXTmac9>y1;DjC7V!A@SN~HKu0tYBbTlYP%sjXVq)}Uji7Uw<21Ev!9IovXiye)l z0C7GM9m4<+B=RYGVQxJP*?K7fnV2jy099aNFX{mKhy_rOX9rZ%jX+J*^M{QMpod{9 zHKTT2F}2z0c_2Oa^$0r~`vYL~=M_LcP~)p3tsgB?{Sj6L6h-8DcLRh>sVYFI31PIf zyuI~CZfgSGT9aqTl)nxzV`ZcRf_RYta=1vNW$&?)#U+AF0{X9pD|=|v=-~kGE0GjU zks+^4FkYWUk8qg?f|HN~eyyxzW4S;_(L6qr@z`T02T-L0dbZ>LnD>-lCa&fmi~8R@1W6*J zODEc&0R4*%d3gTcUDyqZ&NZC)XxkGuo5_h@-ZowC0pNbn5m&# zmsarNm$V^>b$tO0}h9*Jo*DTl& z*mS^26pUD)1_iwsQ?54FGY?=f+rv8$2CtRZetnm{wz-l0b5AF63xp68?i-}zT4>!0 z>4`mQ8#zdnGh1>j9=RtH3zsUBzN3-=Mocg4bLn&$d=R!a7X?Q=5XBc3fr`Gua9PAx z8%ED68_qZ)9IG2q6$OSMA_TebF*3cEjaZfz?v&C(-vvsR%s?&%-xJh33JBuVIdHrDPj5`(S$5kg*>WRdKLeVQmd`9JrK5C5h(B?@lb z%FrH~26bB?8cfz((d_a4O>k%Zpr6@TE(7ZEE--KyYlBilkl(%*EfmGF@sA zqk#sH?LOet&?_$(kd%4&?k+Xkg3<`v6a74M)=I^rq@~i^y z>Q4h%<5_#hK0ajC&>3}ntenU{;5!m3ng%sbAgu_DkpQU4KcoENulu7%Uh7UO3>Y~W zh-|@qQjE(Bm9`+ezW@^4$3s#)eeJ1_!9Y8~^R_%+s~q7P&bhG_Y7d%fgi*Q2y2o?(Q?WANVxzkAp%2+){e`pj~$>qj??)9Arz zLrq9&s*yzRQss?auhP`Gwbm(@n#YS?z+&!=)&Cw3U!Twbplnd7xJTIiW?`e@KM~=p z^o5`me89b()!&D0_>WiIf=u|+&At>uX0!`*Jpq&s;Z3VL>i;=%qL_W}KEAk{vzj0p zCx`!nfPjMgcVzMm_j1ubSVG(J{oUS{NRwK+lUOsJm*)-N-atXZH@LH%VO5JVV}@}b zpMR|3BY_ie)P&CTqyOhY0>H~6KI74{n8!xj;H25HrGM7}of3iU`}xwuIM>f8M`X&1 zR=m7U{ME1k&)X|Dj9VePN`|Em5!eL5Q`xcf6wkm1QQQBIj9(+!!#awkvIcAeg#Vd1 zoUW&W_PJhNf+7P7fFSkhrY0iWRSap2SYO}`+xK8f7Z&U(vs)%(~-^p5A?ss6xn~#HMdHFOg7#Vv(H~W=5MYi zF*hHunavbb5RI5n0Mjj!<(St92LHd36DraF9>M=Vv;RGEL>s$v{;_#~O3JJ-yEh16 zDu&EJR*0vutIqUXnCu}yP0x=>sI??#YDa}F_J6nWJ2jJa7J{-5bW z&C5>azxnRdnd%8HyZ$b-`X$J~yKPn~e3JuD*%SnXl+=Zk*)!-~4jhbLY5sTMq_2Q^ zFaISP-u46z#IgGsGF1VI!wE5-V}5VeIpVbcZk_igc_(-A3^Oe28TI#nKffYda zqDG(%z{-%Y((*@5&Hmw&W%0e~Ua>06^+-n?4iaC9u+$^qt;`}Un393Tu`Fo+?&2n0 z5Adjg&U&``nWc{^NCeRrEdidq!Q7G(#p%z2Y16)|dF1D`N|CmC6c{WuVgIW9&z<+W z*gstSnOiAe89tBRF$W9+nO_hnW7(&;u*Q(OOj+-ac8Z(fOf3y8eNmMM-X6ZVUZyiM z*~BfZ`|Ew~uTgNVZ#(Kg^XUp-o+GQMdTH8qDwA1QTebbjc_c!li){Mm8Z^;&rgHMd zV)?hzh)7c=Y2!N2uPq$PN-aj`!*ORVZ=_n@G9#nGAgk^;0+ZB)7M@t;3alr*YCYqW zi(RvyIM{U`wX0QMxIbUcJm|>}Kt#ibqZKr!J^r=IkaOk^mzv5W`==#ihFm5))3=cE zH;VjAT1wsE8!KT@>RO9aKs|TrIHwv6;2g&Zs4FGFq;tGFYe1rM-i`94ZW814Y_w@_~;*zg&rc$D}x`8NAMN5gNA{Y*b`m% zF#|^n^PhAS@PtqCI^Ij8x}>^H%p_b~lE#&om3An$ZeBVgn4yLPAT^VA;W!ZE54LQ&PKpoqGfKU+hFa!p3!Y z;<1<>_!2%CC}H^dToQWuIDZmX=)C>-O=WKFORE7(RfX1G0RM;q$DT~#)OfgQM!R5@@KT`bGy_t3#PqYxsbnVFFt-?gCCoB4ShYA)Q>@4EPSuYliC27 zt>+S__h7VlKO(*t1dJKDbZa*&n5`Ni5UF6H{b*g15{LpcPNWM;DNpjU-q}%Fq*}>r3 zad@l|Bl}}E@>C->A!!yHw1rc);zjLd08SB)v46Yb*qZW z9T@vtlLXTwNg0gi9p;|dA5rZ{-xs!*)#kK3~7dZUgMt!=vlxLr7 zS&;2~#fbN=hbKIbSCO>xm zdmR8+5LF%|sY#GSv}Tks<~jo?PN8TLtVpcCb(t+9H@Ig^B+Xxc7A zJw}u1{F*ECfhC{fU!y;BuD5K1M1bviH^L6*W$BF1GoX-K#pg?DXk6#0xXvF*dEY8Q*z;-vg#=i72S zbJ{-d@n(vh6;I_$?Vs?n72yN|`ci~N8PZB_yn+Q3i0wwpJqIQ%1yireY7LU*1h&nFLBk$Z|Y30RroNa2U>13b2CQ z;u)5y!55w^JHcC4WRM)yO@hE)5IYdfLYn3&#}vc zj28l|E;-_RNeSPTkNF;!)-R~JJPWTmH-$m7DL2(76qxvhr?MR6_-L*a6JopaCnfSe(`E) zX#)ooKwi3za1+X;{`3HZVEQp3At4?{6j{<) z1W{R@b#;1|AyfGoqQsMJ$%w?~-oOU(*)MPB z=qo$4`RAi%bAonu4Sa)f=xz+udrSt_t(2lOIvfL6Z5G z^}h=Ten4xf-FGt9N(e5K!ySxQ$T(agrm(c^LE@Y<89AxiT`W$%Ka5u<&6;Hhs9;bj zZA)vE6*kz}Kf-FfZfe|>%F(7HnftKOUFVeA>dZ>)FuQcOYE;@iZPh7{=N80977ud}iJaZdE%oiISiy(N9Fo}n8NMYUCLJk4o z06Hiwt^6b=2Y<{2I^xCK0@>s1@;g5AGx)pNe*nNFi<{&#v9MG%k~^7tO&b%ZYLrch zV_E890>bkE>7S6W-wF1wvEnjyn~zwjne;9XvonogvBnA|a$y2c;*rKhM)bZfiB8I6 zGf~S`KR#yj6YogajpvdffK2NRp^ObZC~-&%Ni*@XBHp`s0lg~T(I%OJ)B=F%ULzoK zcmm^90w&;5x-X-x?eRF~3zWl{Rsb)*mlh~XRbBhzBNM-ei@hvNxvcQlgeu2!S$Zr( z^?E8hv-Nvf<3cQ+^u5nEMT=%|)WGJUslF5VodPO#;MG~uNL{3T(zKo4p-iK% z^!?6ATGcL27aZ7iK8%F?$sKP!7|4IqlyIHRX(^hZ$AK(WJzGCJ?{#p$9L`{RjT1Q(`bSW#fnOtt5WPc4$5gh>cCzS`>%X+C!&*1T6dR5Mlj$fIWM}vV4}4llLrxpke*wicT%LGt za>oy3USFUHJrHqzzLI+%LdnF$LPZq=-8YVgUvAGsZ|4}$MnT?$Dz^R`dK^ma)U^G5 z%O}@8&mfK6;kD1T4E%x1QC-|iTv11XqS7f%zfayaKW|>QMwAtG@h<*#Jm-7W_dUwr zZI{MMPvyem1SCmPsOfg$jjbPz8Qr@gPXh|u%11l06g5oLTibiv683L^{f zffoKG;rcVuy4}s>g8(=|8-LP2pzG+Z#9lXQHp*ZF#op)V7xRh=7V`Q^#58@?EG%-6 zS5dlfB8Q5QL6yocdV>+M;J@qEo%E!|lK)!QGfYUsq0EhwANHbV+QFVV17_>PH$K{* z#G`KijB$N#YPW>_4yX?n=V^Vb99-e>|)}#Q6LR2dbC>C9q-UH@;-A5Q$mMjC`(S zCV%|1qhb_rk{7-U*C_w|D4tL5Hm{v_U}=RlLznC8@~4TX`Kf@6sBpAO&sbNG5zPR^ z01pJG5@d!~35qujYEagik#oxv5ugv1kOf#{M0cSc1K`nhY`mqUAEjYvQDrQ(m7ywT zbpZ^IhkeRAHhNmpkF-!X#cq#;?24QF3rp_MjH7FJ?4FpzU`~g^FetmOLm%XtR9POlFfA^D%0` zOj&61=I0MuNd0#OC|&L?V37jlAvVWqnJ{*wuNtJOBb1)5(H)sc1GIU0V+0mVteenL(?n9LURDNjf_3zyuY zO*pTj9PSARF!u_d=49m5nN5c3HIlZn+5D`<>DAXi!I9Dt3iCYle_w{c>hbUYgK!%E}}K4!kUi?6SXtt}%pg3xErha@UvC>cZgOtO4fWSY^63LwF7&)yuy{se^-qOPjuPcsNfJ-2nxK zqRp{QcN2$hL<>+Nm7nkR;n5@Uzy8?z@OuCUoix5UrOsMj6+l-gu$TVT=%X%KY zvVyXnMcKOJUEh3i00uc%l5h(~%)XhH#PlThQEST-srG2~k{BwGo*nH|S0Ays)1Z0lPPHl{#~td40{@XJ$6_XRX9G^Ov{2!64EeLDL<)=`}Q4g zJy>|JhRQ4$lD;wb{k1TEs6zCiVb`zL2}i?L3#1a^72snkcz7lRR|faYNz`S9QigT5 zz-m)+Hf%gi=BTD-V-fw5IaS0DkuGwL>vkIHy zMCmy>-Y;x|eDaUfmZ>koiD@zE%jsF;*ZZkgqu0?Y%HEci{YTDn-TKNBRhT1Vn|0xh zCjFszTLSZ64*GuX(_%<|gLHwig9zyvi=pu0{8I8o3$%B)Sx)xHrz9Y9Q*~=XN*F0k zC8deU(L`OkTaV_PkM1rmK7oByoMC4w7{VkXA04w%h@8?UM4_bsQ`%gnM)lE?oMvf8 zX|d)BlFInN%Iqa5V!5+A9xFK0{>5{;KzLvGf}>Jx@v>*5(MH3;;xIMKZR&X(s%OP> zpoNV#7VLGE>TuTi@V8T;WQs4xu+WQwR-g8b<-xnyui zEbN3lL-SvlPjWSDCq`MB^@5E7A5@#CxOEYD8W*J9+!|HJ5U;ton6~n;1om`!oeR%Q z#l?&VRG1h>wl&EC^bQ&>?pn}Uj6XcI)^t7Y@dZIQ#0RSF54C8fY&fdX6*!;m?bKSLns5jc?G$;GglzDlAE(^APg)03NO0GOyPn?-c1Uy+OeAl_uBvz!!8>;`eWv!FMy zW@31tT3tUrtgfCBbx>e$tjc&ZxZ|=Wji>_&l(LlhF$ly)4ns7{x%%LN9j~4@@459t z2!Oo9Yt%wfLrw)y_tw&;p@e8X?20~f9K(~!a=>L+C4=l(=>bTeu$kaMfHEX32gGJciAfd2P5zSy6v&w{Vu7gvRC_~ta4XhK*NJOS&6!35}|Fb?)VB|te z>%h#+@O*vd|K)Zd0gAWIU0@e!~2uhnreR5j(o__ye6ufG56KV}QlN z>Xg~UFb#?+?=mcaRscq|7B0Ryi3$ZkGd(um6Dmp3J3Kn+igJ2(sM&>FS96KSNkuP- zu+@m+M+yVIGw210MkNq@1!^m*aeO)JxT^k@A>?}bXZ%>)#ic$fSv~a`70#wJm7Tpn zf&;NKp%f05N;fDwEqkAQVPlSIhGGVk2Wl?@(JZ$xe0yg}os3esO7j&-ET}h@YM&3` zI}m^@?W_j?q~k(sKH}XPbimq|*nhr`yS_0;r$*0?i?!Sh+q7F13$qqG99#9FIRVHBf|sp?jOZvJxZR-XE*x!LS?8_; z~Kn1TAo+R?2cvNDsHdAyNXr zb@>K?aF$72joi@0Ab%q?NCA-DwC{X-mZ$+A7@a>*&#N0{qd z8}I~Gm{p;vk-_F8pLxm8N!m_ogd;*g>|C`B@qsDgCO6#h1NOxE{kGryzE#3Hgn$V5 z0q+o`wb5oj{}BV}@AskzltDtw6LI5dA_f+HVUx9VK3xVQDT4Cr2lF8^-g(_r47`{e zqI4UW>kDfo}dccejrB}-0V0h>T$f> z7v0`meR;#T`TQ3OOewZh1X{w#B57)O6@2g7msgmZzn?xe6vXI!xLpdkFObOuh{%Zn z-Y-U4g07{<+!*bV>m0~#g*0Gb2uvPPdoqn5fL-KIL%#5Zg+jjf?37Q66F>-7vY57r zyVD76UH-hDcs>Dqdbs_>`f;cUBrDvQ-z*P)@K$DQ1f(S$a zAdBZi`P+szg>S0vPCtl#adUL(yY_;4XSX~#Iw^dAZVhXKiHiK~>~|0x&!!cW2`r z+zTS_ffUMd`yw!jslLWAXtN}UIw@jD__-RwNkKjdpyCn6AcoO>q8KtiZ!b&$ zjGUb>lssq6687$Rn8nkOT$(>!H`LEb#1!Nd@_M`; zxeco_@<@-p?|iz|!aFFY9^fRP>JN??>UKK%%M%mQ$=0c<-^kA)k~uRoH+QxDy3v=P zcs>4M&+v!)g`%0+&Ii~6)DLRVPBuX%4{AEeIJCQLA&>k9ux@<8>z?0FL3#Dxzq9CN z!hBS6J}xYZ$manJ+wVx0+Yj)1z3#)D*bLjugEXL$e%lq$zPpFt-LFT_`B@THQY*0- zMrBVo0dGsYS-;ba!CWGP7L!UF-J33t*N1v+-Nyx-IyW2WR18~P4xyAU z#M$ZhzrqT}b=8erw^MX}n;Gms@4ncvl&IupUkKm|D7x(KI=xN3d5;Yy7mi%ixyD$X zu6MvIHm6}W028;#trR63yp`N7419ktJHKTV1RVZ7<&v@=iJjH)SSqF+8lO_)%dry) zcyQEIyk-*xdwMyax1Ys9)UkpotC&L2f{Hx-o4dPnQ$(Zrv0BBUEmhX~vUU}tvwSqZ zZxdu-un%q`+)&e#VVKccH&{g?K;>K8#M;+>(u?S!7KclEvsMU%uLl~)r#`r z-?CkD1->*^73c?w4UoX|Q@uDOaW$Z17l%WskL*erm6W6J(rtasulrUSf+aeUP{aU8 z5+7e*$wX}IoP^iBK79{YUcJZ2^hyX@ez!hJuCf6a})O< z!=N-G(Ny-~-(#(9)h>f~H%)C;mJkD@U!&t2&lSddCDb%O=myyg?7KVVpqx{9$ROzv=V5EEhMGRS2OLn8E6-s4lN}lhM>X z9zO30{Ya#Vl5EaNjS9!Qk;=Ee^eg9EjE7M*y14*USIiQ z9H_QbYi0E!((8Qrrw$rtc0B!h!bzHwiOWXJU8|Rb0pa2~t!=2LWu@YO{8WI3May8?1KkR=y@V}R^v#W3!?7CUh zpAmXGX$eTkBZyuN?F9d|BRm``C}G{SH8YvlN=r*g9iXF*i9_Sg6hn@d*Kw=l+SzoiblELc*E{edU(7^)*eU2|v5LKObNA}mYk7Uz^2UuvMt@&jeu-^Y zNJ2*VW7e&>=_If$euiO0OU-CiRU-lSC@VMTqtE4aXLavy5yl7Gz{3jv{`Q?Il}o150cpRysOq5v)@A^^JwaZJZ1YUprkj zsDw)yXhW0b^|N_a zH`mwf0*YI*%Uer)Tqg)y^aBvk4-wxt35C@6gaDF^)`x&MvNsM^8dn#WOWUiJ_B%%~ zf8>bjAE<0(P0eb;hlBK74>Er5Q^zgbJTq62B#8sRYez+Uzj=Ez9;KP4n%b|f$HL&PZi*IOEg+#Cop!=iX_g?2ag2mBOhPR zjcn6dPhi$_N6p)i6Mix`MO)#I?d&whuKVYf5&oa;{>`GSjPjEzq8+YvWf$v}BVxG6 zOq!~ys+zSg^AU64qY6s?FU05)_LItfUU%|O{fHL>E^>il)y0y#+oKF3Ea2oid&R z+w<`uaL?$etJeoO`S~{5`J3EsX0J^gOfqN6tK0YIY^1EYVDA**5xl&-I8P*FO3V1Ed&sOvv`uU8Zz4=lZ68V ze0OK9^Inx5jXhC)^;3+_l*?`IETR6uI%tgNiWnyN$`e>DGt6`a^YkF3Hwe73n;z94(0&c1l( z-UO{-3BM}MxTe(s?mt%;xX93<$8au6ngT)gM{A?%XWFL;ZU!5_Bmz&zVipwvL zBg{_gem!O{oORx9cE3DcjbuEmXDQyz-(Ph5p5Zn&RR67lwv$i1-LN3dtZxj%N`CGtHv`Mg zS4|$0>-9WKj8qK#T!+osJn=99tfxoBu9w^PpRHvM{5CZwRT%`tPF;%#+RwfNP^t9R zAv%;ulr66-!)w;fA4^2IAowEZSm-FiDjRg3R^CKxLERZ;KjGl4$==8_Z(wc)| z0YotZpUs$Q<8pMr>LEDo3|8D<*~g&q%fW5uO!9JWcb!vzaZIO#WA{r&kK`@zBW^sy z;qN^=DAmA5eP>?H-uox*lHQi;SZk-=h^V;HQQ3B%%OKMuT@`3EO@~jNMQk=t3ZByk z-;*o008u_A9v=6nzoTqMAGi3pjT-&Nh@Cnvpj0GH9LRAflyj_ zB-EMI*V`$vpG$}xw2?zVmYtB{b7LNbES^wcvXql~d6K4|i5ND>HOp7Dgv$i?Qyh3& zSyfd|?<4}A73MY2Fnky!c6wgJ-@1j`JL6_{dNMl?XPE3mYsO9i>`sA%(b?~2hc$4U zfKV|TB{31X0sps_$tfj&@Ff^oNJN-zrnId3Wo0sQEG_ft@iD)kU}$gI6KQN@V~D~m$q_6@685>J%!a#lgF zKhrwhPaB2cO@(CW&`Dhv>nlm>36wJK?t9gt*?g=YB`oaf1g8G@RJ<>L7Kd8otPUq(EcEzII))Utgi0Rt3wMz1xcIy;_TwzT#xbq*w+f24o?l$hNyFpi%OTB3FRx$> zIGVQ04R{F4sXhgJk{z#J%|cm+p*udO&iA}mK~~skaJM}eL+n}2&CnB}S0ia<=-{X2 z{pspKx+0+SYOpup>9V31UdHit@qz!Yg|16OK4!uTm#{2jWlm$qO1Y6<#@9h>kGQ7$ z@+ZD=^tdd7I6D?@Jte}}->T`-vNCX4_4Dk{;?3TKzM!fwbdtwY!7Gy`Ou*uzCY(YP z&F%}VL>LB{kwN;{=;(4wg^8uxpVbL41sxVF3`(s9Y*oVd){9E5+ZK$@ThXrPops8G zD+ZE_1uZJ#>}oRLFBfQjm4v<%gx08?Lhl1`zS z-l&zI)7=Eh1O>fiD7VRMA-m1%``b1e8k)Vt#)EOI+r$C0jYO(e!8H2Me`f1?-uRq2p6=o%|g^O(enofc+JOQv?@`i3a?dhn>WCyw?5ptErd@<5}esw)E@%dTZ;7 zD_txV*lNQJWdu&f)Yn(}aj+q#Btvfng(edHS3l({gVg+Uq?vp~!}Y_Tlwn`S<#GYUR5pVn$kN8P*L)D{@P@ zX1LOJzM8Ezcf{;3?MD?WGOE)7G&G7zN){Fe0@+JKq+ITE2o^qrbhJTPze&D)`ND89 z!^hUdsURnP77kv7wXw0;IJxtibQ%CG@ZjCNbJv;5B_k!O{rr5M+wFMt7nXvvzQL-5 zj!5NsICeRaL<(BTpUi!GK4{Ltl2+5#*Vi-hxHv8wx%dT@a7R1Etd&HIBR@I5g8h~T0VNo^zUE{8T{XD8IOOdePkM~C7%{_1 z!k@~uIXg}ammREly7Oq<1Vf=77IG%4=7e~E;H~M6bEULpmAgGG_v;R&kn_8myNWB% zpIfKDfIMb0`OPit+kDFP9#jtddOkwzV_<9DKByKS-1f7=nPS#m)TL@SH4{M_YDr%& zO0E1YUH1oH-r=3jTqv{%AjgbXUV}26mOGzZ#^ATs%S`O8dol$V*k<%lFl8|#$|FgX z5lIEy*f~Ue+j#rBVZMKTv-QWu z`@=`ycP3bkRM!u|kF$mE5r|;fC^WZb!OXwAVPQ*t&wJWsX_{_u0|z*xRT)m=!JayD zqhn*XozsU&?;0+@azS3y10n5|Rf(ICQIgB5n%*bVNBhNM%-=5V=B~&d#F^ zKm}!^?~t+ZhFCss?g#m9Z~CwF1vNJDVA&Y72iK-nivsrdXI?Mq^R*TZHuC3}7q?G) zMfP%Np;RPy676b2E*ray}OA6couJAokM4U$_@~FJ?H>XGl2+NPD;1 z?qx<~nl@d61}q;Rjzmt0!{~;@dK^foaDmk5eDiBg7x!jZF=95Cy3*FfkMcO_51heR zne3sA!713}K$L*YfH+CYj~l=Dpoo?@X5^=46-D76jE4+zqMT9~+5?l#6wz|fw@w9+&ykTKH%Qf+3Y_kF7GL#vaYKnS7 zaW8E3E|oR*aHQ2)H|4N1V@{mP>I%!nuFEO;SDrRWoK1W5f`WpDu9gUj<%|;rt-PQX zKnzHXf^#8Rm_5OQLAtnJu=~>0#iy@W#>LfDjt|e*`J*YQ3qn>?U%z>^-rG84lBh)O zMP7#yH7>oaRl7TSl#sB4HT?UFmgL#Vx9Q&{zNW7Kw3QqS8?2lU2XudRpr_G)%!gOW5gDKkJ+y3y?Cd)8{}et%ZxbAP(DzkJ<@ zuh;SRyp;1%^Fo~?%SY(d(bZNiwJD_JXk#!tr5M~Tk5eW7lGm-oK;d$qaTNt_|k2(Ar5{+_3MMPd10cgK(3X&avxa;|uZ_!DW zCA4-m5+RK7=E^cx`rH;gym+=H=ac5a#W_JM025XhTwWLZ3-YGgxfodr5n~-tjqR$L zCPw*EMm5Hk{qCn?AS`T6Hw{380di6bSP~aU!Y1Qd-bH0tl6e`_=gNzVD@(ktXbKG7r!F;t&FuOrz)X*W^Dp$Q(lpG-Z8%P%9x} z*UBgb*Ze%l0aa#?5uYCW?-8p(hdTP!c82}$QNiMX5xtwkp=NK-?|6UKF35-Eq z^A@<*VA&Y!cO9JQ+E~=U$%$O+cC)((5%fM=x%jG*62019*HmXwpbmtlt&}nJv6(X^ z!txQ5T&f#C59#Bg$`Np#EZME7t0NU?mn2a-8L7LpM(w0}F4PSsnbEe2*lJ}+wzDaD z-yYm3tG3fXsya$9PM{SU!G4FWHC`2=^K!$>FRn`UhF#GI_2b{5ZE#AV*^nX zHkNyS}V}JW*6;BL&{1y+SrPuHgO zTeXZ`)n8#J3kK~Cx%VgWjcQ26LgOFyAp|c_%x^W(XDQw?8dSgvl`S~cg59eLKNcm3 zH*84qDJ&@o2F>+8YNGlH+33=J@J75jObisEWp z#ySo6-195@@f4$}#>aT@m{3fVK+Xiaz4gaUh}#9)8un&dv|Wk{$L+lfv)DmH+1B=(e3sP4(ScDx*XwjKuJy=_^Dwfsst$X2?)cXzzakb> zmG!RU?DUWBuKL59`hnu$p@f%c@-IteXTR#nJt5AhEMbCrAl`!gcl_#tL#c)Q+v_Mh zoU@q9pC0%4ln6yeygpi_LV1>5dlyps0lyG`HM+jLi%h8J4RtDa#yd} z1J7wow0b>WmqS}6Xcv*|FaHK|>otMMP(SCczaC7Lq=t8;JjUV_iJ7w@Jn zE>DM31w{hBK8FE-PW=oDmH=ycCoCx|TD}Q?Kchq>N@_Dwj+e))?{L4%U>5er$3}N0 zFyrDueJ5L)H3K*c+Yvw%s5Zd&xJY=IL*1GKk%RX#YPTeaQ(`w8w@ehb004lu?tHL6 zlZp=iIYvTBY;xiScNaF?JT`S+BUV?7*uT@XJ(;oR$@dh=@=Ze5(pS$ulx>)CB5mVX@Ns*BMh;@hUxd z%%qLkyXRF^*%%+%D(i@b$!DZz?3n8e5xWW`es1MOAU%jn25O04u%8CzCQ3d%vPNN&R80n5UPpm<1m zC+EE(&G+SFJlH=YmBy}ks@Pv7K>;G5bWu-^rnU!g(86XI89fmLBMJz(;c*i6(B-lo z2*|rN;89Sj=i%q#;(z-|teeX3Z2s6gBy62lXxy`U@(+l+)IAb9m~8vwDN+(&AvRZr z3R-WbcB|Wn1SeNrO@)(d_G2qygEhX%CPsGr_RjU?Vl{=^AkdpOM;HYbP+^7dchl@= z7dr+f0R$E#q>ZP?;~Kq646e`pZB^kp$rbyBD2O2htTkT#{fmPy8yxG^x&A0Dk`Vp3 z7lRE`Tfp-%ba1DR5v$jL1lJ$schx&*yi23AVD)RISPiD6j8XtnvuJ z%T?rX<6=z`1NHxEF)@HMC4|2mVb1;R1RPID}PD5Pwn@73uDbElhO0 zFLOf$gR4oCVfl*FGW0^Znh5);^V-snbJb%;1rGG(W%+Y*vX0EHj`(~}%>>W9d3c;} z#$%7Bmkpj`Z9OcC7EYp=e<|_|ak%is_V}}^vZhFyAiVYJr)`X+kgfeE?XK_aBh zozZQa6Xc5LraDPlMI6{Ss2GeI%p zW}7UxJe!`&(dsi7N&#al{f#k*h-boLTfTDYpxGd)xPw2Y-Zbi%6-)lX2In^HloBf*##3B zMqaZ>4r#swyj}g(pvr<}p!d!S1(-N!{7g8`B*FM9B-L<+2^RCGSWruWv=%lzF5Z^P zj;LdZ22-$$j`IuMeMu9g%wTYPr`d8gKB^YRyM4}o|9unr(^uaWPXaV%OIvXZ@xAP9 z<}y;A7$nB9WLp0sHHV=MMBFXvWz*~V7&tN85b&Gs2l(ICRx&Gy)uI3-P~#Ir%Mk#8 z(&Q{q>OYNQa^#$c+t!9opP#TM4aJWq8?WO!--K%mWgOVZ!{DO}C9Iz=^9q6o^>nwH zukc2YZX`$?sPCtyryV_WOUmn)O@qh(S~_xX0D39+ml_V*pW*mpM|Laanf#6(Ub^p= zTbsvono@}Y9kEx*AOtRqxPU$hpaGrHQao}@NnxT#=emY*wj@4xN)JW7uv_9<+r*qv znauc{5oNmc0U4c#v=jpW&|+M4Cw)YoGY@^wVgTO@Plsz#Z5T*+QrP=$vX{JViyjSD z%3I6ZAzA_*5BRB+rW~t2U6L%R;nys`*p-8)kqa#c*W0?+MoyNs->o|X-Uevfgwh0x zIfi+h6`bK9Z62rbdK-Y36QWtnD1=j5JNJ!@@My6eGfTdc#fiaonvqK_S-7F$!uR5> z_2>;3cs`b-p$t(dGt3y^GVY+{I_=GBst+#8o#;d5aD1VvWjN`GB{ zQ8qLgm?MXRp~t&>bpe1+?i0d&o4+$dNrfOcl1DHyGNKWqE)z z^0O*)ERKBXS?O6RJ>w7t>HOw|1j@=&W{7amn9BH`cGC@65`*OKc5-s^V{0&$(T))F zK`?~Dr`Oz>)=EnOBn$6xrn5S+JKE}`^G1z;%zp5Cg@yx7WaD7~A^c-=G8^ zkORRd+l1ikygU%kBI~i>GTd3%b-M0HEsRS}f*c8_)Rlk$WJ>4&37`g*IK zeS|;UPZI$`bwdmgEiU?T{^T@N9P7OAVJcdE&inmE!iiN+2XLv?>EPwrlVgad_)gF7 zic2+Vjv5Y*uDqOaw(|#5$+k$-bxlhmKnc^jNQsEBlZQG!nmpl%;NcO_sLygZ^K`7F z0(D&;i$zKf8Z&a|%T_RE!iz4{+#67TKt1&->FjhFi@bAwSnG6{8c*9~bl?uh4H&@X z?a)RHXKyejUBXt(-)BBtS@Q(5p?!C4kzn_8LL){9d#VYt zL5h5mbM_A3pD{uEf3`fr*7sJvdp$HZH#cjiiRe9CF?V$&G|0%#%#M8u>;prip07?@ z^h~D!W!x z*)}VhcKB$lGw6nGmU70xjltK(WP!+uu${Vg-mHIyXcx1{ zLr^{WvaLBIhG@}7b_C3uT@i!!0gDU8r*CLc>JFXiJ9F+|b)!B=e!3o5J5kiKozO?n zUZZ=W7yfmgN)-E!uO3MW%*MB#*wKO$~j^?Ix{0iKt;@LQR zW&VsO0Q^K*NCG86t6yUF>lup-f%JN_T|`7!YR`uaVN+wlPB$$rg1*HNFht3Y?WxKu z2!SvL8h0V(99Jl~dnuL(l2YaHL)6}!MOE*5?~Ct}79IaE)hxaAUHEy~nm*@L>$!kB zZ?bMMSvK5iCci+_#;{QfIZZ0-s6O?+Y^3F+y$)I}6eeDHA9R3pSFxK}!toc*`Oy7; z`MfP3OtlUZJcc=l&?m_uAqhOa*%3oVE(E4t8oFOc^1L2wb-VW@{PPE#K2SwlB$UXt zQ;(wNzQ!Zcq(j-m(wRA_0|FC8j~sTzX>SpI~a91dvm{ zZ^{e2m;G$}nJ%I~eQv&XbG!8ECxnsq(fFSzY9j1Ufs4`QaQn)}yU#ky1VKC*R>nxX zoXJT^u=rN=-Ot;2ap<|wY$U-?3UKg)HXTC>0a-XRwIuj0H_*$`(4oi*S#(8-tNf^O z==WpV3pjo3*svfrIXSkBhAC||O#1_X@TovyTw>nSP-`~iq~9BBgaqm>ir9y?s|&|V zQi$}O9bHGcAzsim-A@CK9DG;17yu^)Hw9+24*8U_#g^2{8N z6!MSZ!xu}vaTu5&F_uv}sa+cB}H!tZ5sQpc)l3*LqeMN5NDW9B* z0ZSRIb$Iv)v{4CF!HlX5m=XbX7g;{Dico+RkH-(9DG0x8!0={m%;-o4_5(hLk4eYQ zm4B&A427r3;loQy&p6nHAFkh|~xM46H@B|s1W<~d`Ci_m^rXgw<|v=n_V zsWwM+&`1>V*!7_8$Kk4B3cgVhrq8N#Jfvl2v?#5BpuMfwYJ%Qx`=C}i!I5RVRC8az zEq1Fqwjut4?#KJe035V0AwB2LCO52w^KBn!E`*>=Qd{V*PSUiw5|^B$o@HGT0s|(Cid;u zSylcPSMm7b!VA5xso`6n<)!pdk0~klWL;GC)>{y1{vwrC3XjW8ICuHvSJqqlon%+d z6*X4b7{Y$`K#C^E@K}%bw-U!~7Gg$G^RDV7>6tz*zvy}yzn-0uzv(#OLHqA!S6{E% z%6db{%@7NDbgk}ayXth?NYBc8+8wDWOtkCt2uH^aEkP6i!#?rl8%3t^z-r^bnjjVn zLhyo>_sdlPZCi^yP(0BZniVq{_3$TA5^H*qKw@Zj(UkN&8C zw108;hlszpL{{NliGFwJXD?+XDbRukG-SuSGz|F$r2ShwRCH8sA#l`AzL0}XdFp6N zAt#~exAbnErJu<5vpIXnNu`ZQ@P4+Z;9w8a=Z#OT**&}PvGXw9zOTZ!xczS*98$_D zDt`v|z+?2PE-;Hp=;10!kzKp<9f!ve#Eqv3_zkb~M}7YM`Qm*O2+$ZUD1e_uo}Hh4 z-JfBGF4~LwI;XeZCM+WQ148p^7UJvy{nqH&1C)P4n!c?$fuq;FC47cL-$B_`1ht=FON+y$ME5c)G<1sT1`Yt<6EIXqi2zXv z8A4M6^O$)7--C+>>aYJ@_{#OTbXxQHKnu zuygcxP{&YAN&dIjG>mq@eg{ERe(=dbfDK2iWwp7Ewg!B*91Sf7p$-15WP7hnA)ISZ3U(?f^^k*=<^(|E(tEfUqL zPC@yhC=OqBN4^qQ%QN;4UzXg@6~ri&d4JF(n1xB}0n&XQbW8?heMZEb3HxPY3PJM{ zSKFnhCJ&(#00(+@;8}^IMac5FCS1{OozXV%WeeHhQK4yW``#W1pQ_D70H7flHkV0C zt~feshLb1fmi8NszHCSfmaNnmi|WNQMml7FUpRP+-@ex9wzuG5C&qe`U^dii{hQnI zW!}$;Vk$+b@IBrW)>xjB1n5q%p{r$EUekSFjNkSGH#A%*sRZi%<>8Dp7xAf$&G)T- zGSZ;D+dmy^MgSp4FZz^`Ij-}p42VozR6}xb+wOk4i6%f{^qrZw=?jU36m>J>3`0e! z_qF>2c$gFk zk~1798TSwlXQe!95(RN-Xh~j79fdTEG8M7|@L6^zd){Z*_`Mr5m^c!-NOp7*+N?$? zdMm{{U&>xd5=>SOnBy)0nK&jvm9rzg8PA?7N``##8a;xc)U+N%UBy^p@_d|{Di%wT ztu^awURRxoB~$iA?!UPJz1-quDaV2I;}a$AjAWCtc{m_OD62iPbNX=wR+#Keyrz?t z4{s0%*h*CTFU{0vXPxgWB*lSp%J{oqKnf@=UhAsCDEy+M;cOKeE2)plBD_n1GG?(f zQ(M=V*iblY%$dPhRY_W%KoXswYpl9DW5bt)c0$pN)%v44uhj(d=s?JYE76G@V7|#| z2{N28rf-p02(%t_W+`g(O4--&oc=$1WZ{rwh~wC4g|^&rN(%GjN=XqwiTwo;brPr3 zX}#wSgQ!u_(G;@9kL=h8W&Bc936RA*@~@9}La2;YRieS<`ztG}t1hRPMG-j!Cw!j! zcXl7|IUkzK!emQLRUXj0O=6#qcAG5HKkp^&tb`GmAf%O-QLH`u>1zKI)KZWN0O)&J z8ClWi-H%L3K`AOS#H~`kWH#&vFGIGPjcL3jwasvFansQy8~I;GEe}l?$2SO#!O6+V zmnx2oad$lxT1PJ@BOhL2*6Cvk0mN`vQgI#W|-*ROLAmPF#&k}K`3)3TWre(rH=lL4h1jjsHGUr7dv{sKU_K$?1Vy&tc) zqhaIY$X&>G;P@O`L4SiJQuVb})!SEj2|hAt?ZGxHTt! zWhv}Hm;@9R5D-t2d--Qu*m`SK-|V4od}_;KhQp1fxJMm|M+*xw6Cd0VcaGJAUNs^BC*w_UbOYhJjybsOi z_+NaCkpePtGNQI+2p+gV6ag(w(yrGGW9U%1l^vL^0(^WOb#;oH{Br@$2CQvqNBs7` zz5J|hF8fNFVtd0acTamS-{ztiNKDzjzz#{;+S%CJRUL4{?Ma))fhlQtkI{U)1vl@k zZ7M6etLx6WnOyHKMZ!v2=3;~0hl;r-H0EZ@Qi{iggO$x*#8%Dv#}u5M@57rk4zsB&eJiAS z*BC|uQ@?*tE;ADnd~RcHZPGX`BPQAA6>z-Y)-@ZFC~?7E~=0+e}lJ=|v;E$(){d`Eq}KevLh zU<@MnS8}0a?nrE%x+L_YO0vJ}Nf6h@et$R)_)}zBM?|v8>#1+F_M0$la(x<}4PQ$G zy;K^auJ$?eW^3!kDuWw1WZJ;&IE#9DSS?NNqMW)ams)(}$-G-kT9Momy<=@-c5?E- z#yLRgfl|e>iDFkdHWuQ0SE0wlXgwzhk5mkcp*oQCz5;qB_?RrOA_nj&8?7zWLZ?sp_f z+^-cOB5nX{>{HVaLmMs@BN*ALrI^u7oru}59{PSPPTuu%5snh}3aILTdVensMvC&_Nw zvpAVPZ>iii&e(y;FtgKp?1m7`03z`PX=6NY#Io7kH(K+*77cGFw+d`fif}&m>Z>df z6Dtb;^&alIr}4TvYRu*gZRr;)8SWmv4ksK~XNU_c!$Ri>KDt}GLKMmksJB2t-Y2*@+Qd;W`>LYOu@i$0EUM~(;S`IUYCy+EO-gB@D7w#*4M}5rkt){7k3aGM{_3@ zSLMitQWCH}zw_tr>Qgs0Gg#baxxWo3utI*_SX>_`!*Fm;C915nvap!>a&aus)m*&0 zJ*J4nLH4`8wYa#rEq!_EZg+8qx&|eiB`6b3lp|V5(2wtaRb6Xp*;U_&yEs5uUE4-8 znKD&S;khB_jfP0?-$S43VDQJ2h1b*j4?1w!$$C9R#gHVAtnklhRb>qVxKD5XcXN>B z!@~-Gh4%31&}CmR|I7NzmiSJLxRLY-QOv>RR^OQHo!(kYQ&*a~`HHey{ze+Ewx*_w zLPP4gzn!<^eO80`zkhU134k7CKMNnH3VksP&AGUO5X*`RTb&nc6Qhh4hz|M_%E}cE zC3V^;!sKn!8zB->oHb9#ArwfMEU`8rWe@;RNI#l=Oei^chMO+_thdfr+wEhga|aQR z{ACY*!J8;a{Y{tDCsUFWBcFM6QWu1U$b6ic^!U5Zj2(5b>Fvd6N`fj0W*-WG8Ev8| zi-Izzs@jUe-r87K+u_=Au{&C@l58?2o-ChTR!Q@q`}%f4$o9|D60R*;mftI~O;J}h zf~2WFnY^~Kz{h|zUD9N(ty0GG^eSON9(gf-A_N;HE6LM^kRWUHM=+598rp46tql>% zgy%QUx2`KQ<-i_}v$Uh(8=sJ>(fwKPeqlY2vGwuu>ZHH+bV-w)z2`e2xRd|t@n;iA z;3+Yn64a>pIXQis?0TreMN*t~CJO5-Dn`Ut@KSJ!6P7ZKk%y&9vPxzm#9ziu>s#vU zl{Ti&xzL3qdLpA5QC`aF#TOR1O7@e-GkLuX?Tu)gHfbv^9fkBYg&Ath?i<~RLaU{y z&C*Rdnr;u5y!}3gD}(ATGsVqxW9&{fB@oum$#f~Exo1!tzqZ~8q4W+Lv!PM#!@W!( zZE1!>W^z|X7voSuz*egZo-GY-{3cD1su$<{kN2GeCB$hI{HOsR!ojwt1nre72-!%K zD%ZRHU@}2C@UYd|)7tuVv&DTUGi?l^ zn6}y_m58Cn#&*4g-S_rzsyMxeSkS`|3r84&n({eucAE%uLW;;R5$X<*A-+vM9)C`i%j^FXh2eGlV>L8H)oZI>Dn3;`gAtx--%3am+gKr zzNSqywa2tg3hS+YuKF>;td|1%=_Y~^EiEl$3IAVUCE4|BM?UP((-adJJW$2K7K)#)PHFs^x7|IznkXgezZ9wm~bXw z{r8?=<&oI1$=G$HY|JV}$b>*TK5tk%@(sJRSN7D<0Dy41X?0m%UU|8t1AW9U)$@M+ zQdpQ-$+N4e?)FT!Ku(=xITga+qdAG#kwR0d5-%@#V@Ek_mv$4opBs~{Qb{gb%9`r} z=WO8rE}*8mRzVbX&Z4JVBG6u5NgPnnl8_R=k`Z-LCA$iAHKP3d6~${s;)a;lwX&{w zd;4z$O~X+p?w1UFJ_j}U2}9@k3W)sSXI-53uH$_Vi&ee%9W;T_{*oGN|K^E~-h>{{lu$8Mt4a@B4?9X%o~^QkEYs>?~+rUbmwN+%a_0Gw!~jXrXDr{_RStZ$x2&KGvOy$um5;@LJ=|;|Ft)0$Xl5aX1=nIi=&}DD7N^sc)C3tz@8e(ILfzENQtSG&3c9GNtnz(-VjM9=8`vMu=3i@0TUidx5m?KJ3?V~O46y)y0kGCibDJ4^pZE8q zsbJ5m8Cnk3cs~b;@{p1nN4+e^xIR=-Izb=;j3|vM%zA2UkCz@Tew&4Rm*(!vlamKX zk+Q$^vLDXpBv7(ssJ>o1Ki`%m2p{GtggWa@R@lTVWYY!q^~z8*`Mx|=5x*XWl*rw0 zykAFT8iNVY!}RIXF#FqHu;Qk3nPW0GV(|x>$+SRg=4JBz4-q)rV)u-5(TolGymp_i_^zPc3PS45MJ)9dM5_f z95=lVRS7jJNI~9cuk+U*R-HbjnDOB)eJvo7V21lVa+s47A!T`%qQ7q@TGfccH=k2el^RJ_+Fv=9KsBX;)CzR@=DnB_{yrZ6%l3Ks zJBcCxe^>zMHl_eCA4&+NnxePOPK^5bBXn<5a&cp$s$peb`6xzH;`C_IAjH4$eoN&T zd;B-w`&LzTr<%f~lcQy#NF5@q%Hyx;}tB|usVk7!)6VDjGK zvXzC(6oj8@7CM1`V*(S&SYtc3WzkU zejjKgYueU^5}2_Ry=z{5(Yfr2C9(&S00nC*t?1*V9TnNMM(z-iML=JOpLPBt-%r zrjPY$6vI7!sQo7rmUg+bNhz?JP_&nz8*cu>c zmkCp#z(-FH_`3-$xfM52=k4G`12;W7ytmGm<&W1EBGzUJp#>L!n!38%){fx!dMJRT zZ8hV^n&2xrdPP;$%Vi%LHzo$@Fmrbc8^<+E#@PFPbNAVWfbZ?*`+66YxCW%SbK&>E zJf(o!>UyX@IKRJqJixQ}vjZ7|V07r8YJ!F<=wml!QeK+6FNhIv1yLyt2OYjfRirGa zt5e$3_CGI|d+AK9=GH)0=0iM=&D|Ot7bUAOEX5DG5vRBl_=JI2qDbU`tn;%Y{x1wG zKcf76pRQkD`3wL_6Nz@uBa&n^Kj;T~wnPwtXp+cl-QV}s+z#s2*OjAf#wYkPQZwn9 z>B}n0I(Qww$4d(&&KJbxL5Og8ygVUO^-vc4e(&CTPHVfjl@!@>#YT0wiIdaQ@%`}w z1F+o_GaCyt8-noh-xk%YH!?%3xn}29hif7*9MB8Uf<>8@7FK3f+=LO?8(TY?+5#X` ziEmzg#TNGvbmZg$W!38K>1h8hGUE2GjJ#atxouinwzRP$CjYO^JI{%1UUkgNrKV2I zn78-oy}cwCTNO5R)ML8tNDPc`1>YYgW$vAcPl&%I4QsmDZg4SL$;ZlpHa5%4<#%WtI3P%CpgIr+Ho zCOHEx)`@*9DzAAjy2!_7`?j_uI6UplzZr$~?u5wQyGRdaW~Zm6B1`A~AtIH4MJ3}o zL`huWoKx{}y$h!s0-_`9yB`MFgkfcg%ON{2E!|eV6vRcrBnix{ED)BOFRrh5T|YG* z9cc>JXa|YR&dexl>ZUL1cKhx%3?Fd^@Q{)OASE&H4PElrEAg)lzQD<)wXRNPt(;Uy zgOG7kY>7$Xgb?=`%Sx?E%U)lPKMN`4i;+kg@(~HTUv9z?{{YvkC*ufqKDL{?2Pha( zK*AN;ySnP^_k@x_SEkM`B3Y`LNO+-dkr?(zi;&lYzj1hSdLHwmo9kI$V;9q>bFI^- zq;ROXDOr7?34~{2vgc(~*#`UL6f z>sXkIg@`97=ZF6RC`p+a8{bwnub&`c$Occi}-$(}FNxn=C+27#%=ybkCEZ9~ou#&|)9=;RCNjtj{ zDzoS0q-CaN&M0(n$cqh0e0w-w-y)lt5wvGhXzJ1s)#KOkSSR=uId5AtO9~=^P@(_g zz=7fQ@yoBHIyxbS+rtHj#1}_|d`D=DOS8+*+iG+t4{)CCUY7LIiu?2G+c|ojw&_$m z1OU}>LMWa{2)V*pSy_wg9uWq@t^`TZ&1P?3^SXCINy))HP7eJkD!4u<^lu6(AkURahCXtBYM)QPsun>2xqG5%j5KY;NvEy}E;Q(dBfJRr~q}75CSi zL>Xo5qotbzAO5VZGzFflz0C~`?M~mb{V=SbgVC-(Yez>-zkLeQ;cxt*jv_>dtQnC0 z(^JuhgcZHXNcXvK&*@;?=i^0?1J}m7Qm5UMn?Uo{lfUPO(GVbqk8lCyBJ91(KL9B& z4Z^mT+C;9X{D>@ptbmHfZ*R@LxTdPh21(SfpNN^qMnhmcRo}QV?(S|M=}sUyT#arg z4=;a|K0LbH1&YbokB_VG&_07fSs7_*_Q6i{n?l0`b){?IhN005}BXKP5Dh>uhD z#rR>WYuz?O!_dmc#YcO0CV|o))bzXleeUe#BI^QiVq#Q89i0OJ?mxS03@lvfcRZS% zeQg3qL9?K9S^~eq0*fdnLC3$S+P;lq{DmC)0QN>~j2sX~HzXto*>_JW>1b5QFzEFyC{d_Pw zswL{V#zCByo9-m^Km4UBXn2WuNw3>pi5+|&DKvY>1(;n1TK>yk@%WZ$=)9r)sJ$eU zJ(Rhm1tK~GAcYxZ!GCurC9rTY1Q#L(nnz;_?hYl7UoUuBY0)LvbT>VQoq^F5NSqZV zZs~~{uq=MmAx8Ly&!0(#E-%Qu* zM(^eM|HD%&uIvPGhgInI(30Q%KKP*aUlI`atRMwt3S*!%U~&i+aOcz|3nI5~Pq#FN zmH-;TX+(VsSXI?T0y9x%sn}l!HA@=U@bUN{(JEJ2J> z6kp&$vqZV>-y$2+CtY_O6a{fRdG_1IJmIW){R64^xkY@%@1de2l(# zjwDHOkJ1YGu!AG_*n~7Hw}rD=ue)IzdphtDnHFoou>D}6BxnH5rI*-0o!~zM81%=% zdAYi&pF$-{t7S0!zd;1*ddM+a${k3kFjA#Gnvb9UN@7v}IuBS6$Ug7>m}4H5SvE)q z$dX}7LWBHO-~h0w0C+cBQ7h7~^q)}|kCZE4xlvKkm5Yk;(f}lr5Ex{DM9?&6SY>;6 zcTM{XvG1GSs7wJr*ifOig!Nf*c7za0>tgZb}5Qe8?Q06Ca)&J%@bU zOS305#bLE|`SyB!r^Q8tXgc0GmmkK)f}j(@vEta&@yKicFw(>cjU|E-L2}XQNdOD~ zs9o$@bbEio9+`kuv5m0x@tdrd_xpwJ<=;a>eP_~rDoAK#{?q0Ea^4^RR0-;4=43Us zsQVAN9mc=|DKV5)S?b;OR?{_&Fc=Sqm2&x&gBu;D;0AO3Y;{2hydjQs(^< z-4LdfO$UHQzAuOFws4L>#bV+Dl$FX<+Jb?F3ZMEqo()0pazd_&*h=*Y&0%-|0K&Op zSfH?@0=r^+t}(@g9dAb-r#dYY((jZ5+TJ<3woiajVLKPmVZZ0YGx{7qP7E!KT%6Ms zk?CRDc_SE4?8n+#avf-&PJW$_+X}s|%AiksVTGu!LI*5~62ePvu|Nz;hvAldF(IOe zCxxS)KQf@>NY^)_Yc+-mX@;6c-Az`iSazTsP{H*+XK_p#S%ExW<2M)-f=+?!fg0yA z-Tq(`ZU=Knb(3{^#el@}mn@0NFQO8wE*B5}LZTwta?JYVCJ6u!hZ!t@_4Mo> z1V6TMmaON6#>5EuiirIlf2!OLt8>+)joJl>D{3f$d*A8)`M0$^w}XkbLvRNScUpw$e#pZ?*kTl~|g{Xk(3!>bx< zip{WWYXeQd0759GSFXq&hQFSLC7Xtbh-zQT0Q`SOQNXm_jl?b_03pUdEPM#l3#E^0 z4`v9=fK|=c&P{D6Q%3_I^nm5YQsu;s+fon?`uLE8$>0 zz1rT7)crcO(EuSG6Ribs$YG_u(L@SK9aLzjw*-P4Yqwkif%?u(b6 z(GCMo6N#B2?> z#_{HK7(V&1`5`)5jfUfGVxNt4^EYKnDJzNcCE88x?SC&Lv-p~or3F|}WxO}+Ey86M z;h6OKrtWVXgc5%)sahH%ktj6{lNIkw1GQ9L0OY-HENy}5KLh~ z-Iwoft5k~4__?Dlt6h`kQ5POQP^F8g1q9Xq`t>VTXyy&HjhE3>P1;ZAEA3gbRUc}y zenO%lr^e+UKz$(hfqQNzH?~7gSWoyi;ggDHFc5L|1W1EHOW?dnD+!kuC|E!ARXvB{ zF(@d_{Mams#6R?pqQX3H|AD9W%e#azQ_2X`GvesV{PgjaJ6qT<1Kg}mepAkdPkPCv zq)_?Z0bDd2Rl>XEcb1}<;ID}wOz#k?aI@>DtS3r)U;ub@VcCI&AZuh#25d)~J_<#e z7hWa|p;eJWI6&7YElgr4DBg{y?eS5{WYF&;XZ z!0s3;^#5>BrGo^0WG4wIg!484D*<2BF}}dUE#ET+A0YDOp!tchw?Sj(xvPy%`)+O#YIV3M?Oif(rZQdvxiUm6`~8AMGVoC2>v!FzrJZ!WG+ z2mlzvEPs}+SPuGQi%o%d{V8@+cfp)QFTJFA9wLvdm2Wj{jMvfoeqNxd%#W=bRa6cev*q5Dm*Rg!B+JCSZ-SNPfWxXh>6 zw2>UkVEDh{4v(fpI~3P#a)OiVmF@_s=dRSGC8=29_tJq@G1uA!oWd@sqGBSMH}lc= z67J)p`+$z8(6$kP65&Lz>J~FDgOUZ?XwiYNIj4=*FT6Z82tE2nOLJ4cv8S--IhI#uWoM@cKcS_G zzt(uVTDx!Oio_oGw+m}y>;^!RXe84Pk|p!fgziJqG89iJeG#Mx`#|%$cq1H4+Vv7! z9W4ifp)qsat`lj>MdLvHqOEGu{5FN>Y3CL<#SqH7488kP;#vp>o4dhm{|)Y@uZvKy z;b#5z_m@U*>3jmwxKFg>w3AA3!>B5&DH}&BM!#WJ)G^j zYYUFq@ev!{4W>S6vrTont!0gw!qw;wo34c{R5R^|czJsHI41r$F=0NoGM(Sl*3tAI zDidzm&*S)(nX%p0vB{eTOE3R&Smv2gVKfQXNdy4M5al0S>AmjhVJP1Itz=W>*HKK0 zn16KJTuyYUw%PqA_PZGQ8o8^(;c2ZUzh%*^#y$>DAc>y6*{ViMep;m1_p)c`O3zT% zN`BzvuTZ31lx3pEiB{UQ#7Uro(z`XTT2nn@85_m0vTg#U?)ta7CQQ|OC`_NZ`}_O< z$JJYhwY79{qnuNrSaG-F?(U~0xJz&;?q1y5;u0wCP#hAR;t(kADH2?Y7k3MG)As$o z=iWO{{>ZblGkec0nKf&EYb{?7Yswp+Cf+F5#A7+~s)RB{8}%8u?3sgse?Ps@>7raz zsItHR;?k84TS#grx01Qer+@H88ak-`Drsr#U0kvj1@F3`@u30WtDeL!d>#aSlk~4+ zd%vgZtw#Uzet2KansKtliEsN`Q_#u85MIltswhlcm}$)~Ha6-^k;)-1Eblyta|D}< z3FSD%ZR1Fo(j2QmJAxJ%>s<)$*IqKxFkW{ds3kYvX@F&m+04YntESRk^5BOVlnn-i zC{Ucq2j<7R$`&RJ=~#5n&@=O^@PEHr^s6ezx0(2A{g&-eVyj%qY;@Vj0hLg}E;vp* z=xrMgFn(qD>~r##yOLm>tN3%*v9dC-v8DQdTX!=CMK95LUt6T3Yusi3geJZD=r})H z+P|`jw`x)_AlJvoU|o+BWx(NDY%J~c`OH_#c)kov0FzB?(K|&Y`RlI=#3Bu2Wvoj& zFB1lf8+{87skzPLXPNj4Bc8(I7Hw8f?;O9~ElS<$TYK~4L#s>mJ#|Yg>--j{!}A4e z>bW+iy#z2=p^uW-_R2zlW7fI%G*y0*!m6xJr5h&m9Yi{w_vgl?yM!;{M532vIf8{l z--C>hRIo5H+`sdRt214&msPdjR#e=C@HJIBnJgZJ+<9S?*I2lZsYyLtEH#pv1|Uu+ zEOKO$rnpCKeM#}hQ_Epi)(kc}caC@MIGO@gLxRW~r_QZmH|swovpjIR7_$Y;4>SIw zTP^~h!d2-G)lbtQfthnW)L_IE6=!YIyHT%~SM#sZt}`?zeU$Xg$QZ_(k7jdy-N!33 zI;EPA=W6lGRWThRmF8|=oKkv3(0JJPc@e!h&?@wquf|qYJWo^qF+!kQ&AAe>K}wv53XNeA=dDOYR2yA zGcRZm;G1MX^4w}4wgGFi)eArEG;-f5nNZYU%)jwLQip15@8`WaZ9(goj z|I$z$pNqgiL1}u9UxzJZ2WKD)RIDsIf;hgw+Dnlgt=aodL7a$p=~vFsM||wL(kho) z6E|vMn>XiQ0aw=^FmP)B+OI^-F(J8BTaJ(l4 z=PbWy6XUw4FwSR>G@-NOcLASnxgc0M=Hv=l^~dRpwF2xXJv8Q8?ulACFtnN9$o zG=2z2=B!L0rC={emse6ud9p}`XqB((WmA4_Aw4vvKKp5Kl`7GoZb?ywvD_(QV)Ey& zoUiU-krdWItcsorGtFs-?>FqL-Uhipl$9NHB>vIKsUU*>Gp3YW@(wsXmu%YujR-Fw z6peXlGrf=;^^hoNF2L_wP=*!tu8Abqvk}MnhAf9$P)K^+-k0$kG6~MJ28a$(5vXZ_ zs0{3Q0NZO}PXDj@j6smGg|gP*dQERf!I;W%$b%pO?LwXI%BMysuNS^{26gdDr)PV? zsNqh2Py*hGtmGp1WN>wW_EsmA1%29X{iKq_DGW8T>v)?`npRTY10X(02yP=5K^p zI^3&ba@yeW7WCI_Gpo^dzut{JW&2DRmw0SIys{2i>rE}!=Mz4)DR?OJ$VojNA>tDG z9N3J4!oABP^=^2?KIux&u2+&lZ(qIpE_glTt7~+-0C3wd$b!Dghm)*sMn9i3UMsq+ z^7BjpzLL6S$^JiVY1D+7y%XZ`l~$<@d3CQ#!JR>J$0ZGWA-!CX#v?njS~MD-ogyoT z7TGH>IGswsow0@x>|CwkX~sQir=* z<-JE8pqIs}#k1nKUInaW{g?Q!IiN>xUr3REj>mFT{#ZOA6u7fdO*?_K->~Mxf>OE} zTs5@nJL$4HLlunn`9PE`F6nr$1>O=Y<`o|>znGDIXCIqj%VY#<(XsEa?dSAhdNI%FK)eJce}A0V|F3`h3mkKj~4$o>ThTc zhxmno9_D2P4K^hIghUS7SPb^ImAfDGUGy0Rc>y?j=x4?vzVKLf%Zr)C)E461$f(#t z$`Xb`+ulYMwZ?3;9&hd^C7hHSLggHLm^e?sbyX9e`)ty`s>~g=>wq(JrVYYM+4{`U zjmQH6_~Dpf^@Z*s<0X~*LjCV#`fp2Baz0RJu?w;!cnkGm1+tTG{NrWdUqEX4LmAgCN3nk?>)^jK2d$|l^jszkzv)N97X z^Vi?DC_7?Cb=0$7`Npc0&qksPfZPIgpQE6-?iavKP4VN&$Jp&MH2o=pspZF5#mfvD zX1Vw^wtJHV9jPN6de9q;E$;BAx>813x#J;^`<> z=u+Bm6vJ0oG{gox2jIN&p0cNWvVF|AMFy{|kRlmRo0Pu;-r~-q~o`yeP~fR9+;^(^A4GuL4a49F&2K2QM1?VbycQfhn+nqW^@l0J)4#6 z!z=H&0|UTL(mjg#T;QLb8qgydEUOLqn_+m`^VM;*uLBQ~WS*W`11{EG%I!RQv=| zm|taQ2>hg#7j-aM+@2{vu3x~_Q@+)Y2F*&{PrraK-~a=Tl#}?(u(hL&*XG*a6q^wv zkg7W-MYBIu^X_QjmC=0iT3mgW>Zl}hY@*-_yIh8vCPc@OL%<6ds4OEOjYbYDZAE1g z4fN4(xs?6lbpHTXe@%DYM{=H5b}AU8;QZ>+zdpfDxKXTFa7&)9-d**_U}$|wvL1wS zn>5Ba{a&l@mYhEqzl?5&WFWYb9<45ABU?}~9`v9}v0*e#;%Rv7#~nvnZq|QZk@pI` zPh!xg8c=ySCr?7NR&v9tKxV;yuE|l};kvwYJ^^27T`Qd_+_UNC6P~Gcu{u|1 zf^yN->UMemt42jZNlHc2d=84ieb6^L?%U@uLX3UNh^G)qZ0t^#?^vmP=V;oevi@@K z(l)fg(b)Nm30F~m(-wI?B^1|XLMVqhe8ZFDvvar!7e$1{%3nQI-^PT zEY6n1$63Q_?~az*4&UWPv)-0#gw&YJKOH8~^dO!DKZ#Z}{60NVP9% zrwARKdbh%4GKOTTJD$N#+51c@GkZwCqOww=s}qurq569-THt!bGu#NgXKI{4VjX?5 z{xmv#V0l8+r6*l#JiY@ZDH96}eSad8uR&J;kD-4Yvig;$_Xg=0uU@+Cj8@oQa4 z6<1BKQa}CRU?2~f?D!f2^bIh%=-Vw1nXq-aPdG}RzL!or$BNtBv>BL_XXS(ot*X!) zhwnR(@@KZLqU)>-EzME2;~sv|DIE4^y|mBrqS%_1EM7PPA=#0 z(qfEYJ@yyq#A84c{HLI#q6@;WBk%&`)vhv)@G`H$QV5z-P^knE_+T_}lI<6DdX}o9 zprCB@{u&b^`XOo>=nOD)GO$eO?8 z?T;=X*`pYEMqWv3Mu!pultxJ1fq$P!;j+TBQj8@IlucK&U&vpZa1TMay(_ID?*UN^ zDfFqS$AEbAq77y{u5^9?W_cG(>KnfN@E`X4V%xs7>E@%gP_zsy~TH!GE)!}-IMiyfgnGNno6{)c=z}zTXF4JH{ z?+#m8lBwh9Fu|#<)w*u8bp#Y6@GEu`4E6_Q%T{OY1O-5PayoGm`x%gxSKrPCs(Fhv zd=<#PL;Z!O`nlchme&OfQJ8Pp(lx1-FmYVWw1^5V5t3yRKW4InGzZ#7x+>34zT zL*`AV>vB-AX2!xVOzI`5fJm=X@kYm&87;f(oTytp<)q5>I0v6x1-daJ|L4QL3uVKN zGl|xQ1gUs=z(mJ>NKqx5FLy-j&=D8M?ekSigkm zr^kENDhR7bo-bpt9Fyf^Zt! zv~u3hU6pY-LpM9$7VWS|%TAmdNwm^sXM`ItE_-WQYG$9Id{pQoIm9)eZeq0$J*~vq z3&eJ$-+AuvUBlpw;C2-IqeML?kCMbbKEluE3n}?!iXSMrfr`e;fDJTbVCI-w)SwFl zunF=Lk$>8wa@=flyInjWIZ8+$RY>)tyV-_tx)BhW6rSSHgTMs6N&ej@z-$}936 z*3~gHd#SAVj?OI_NV%%t`mG2amZ2uUNwmAIH1Ls6N3m|}orNF@(F!Yx!pFwxnE)oz(i&G-mSt~BOrHk&t8rvwu4*2!)6mpOMNtO&@O za10!;ziC1=IM3;P>sRGw^ehhKDXrj;#wFKx0A=^vpE~}E{Ru9D-&LO3BT0$!G3!Oe zl{1OZBirMlxe$u89Kkcp*!ac%Iy7ahlI?847m$F(1lvMl)GI&aj-;HV({J0Fd^7$! zMC)^W?*9AuA76@1x`9o5t)OXzU_ zKc`Mp*ry*KlJ&yklJt|LC;xb8!257Q?@P>bs3Bmeh4SJ>rjrLP2@*VA`Oq(<` z3{MfTxNjRTub!u$fNN(NmUTKl$zOQdErlQTdpr=k6<(sOoZweZ?z#3Rqm??0Pp5GV zfon}3@s%hIrwuP&AYs`5x2dDfAr}{vbfYCprBFBSemv11jV|@Pay(lQoQeTm*kHY=3Nc2A0 z{U_BCk=fE2b1mMg`(-O67QaQS8m$Ft~y|g-&tr~ewQeqr0TUM~Pr%clQ4)7YD zx~P8lXMR4i#JDaIaKY&WFaA7u@N%Ymx>V;azEsaB?c~?c1X&jOYItXy+9Ymt<5oXV zb=L49dB~E8T@P7=8Zd2(ukzZ_$|c>NOY<@_^xyp7(|~N3HGUF64gz!u3FAKV1m-nq z;Xl1ICX&DRBuv$A#J5yb&vt$9rpx80oyNHx zJ!T;6@`%iu%wU%gD?^Wzji~(Y%|qGbQXRO!r)^!&P$VjZm+@AgLTEK>4lgS!3z=G- zw%Ei87mI(_(@090v6!ka*-yv^gZs~5(kU=8G5LjrN*~buC6HG@xb^W$j~x|_&G}by zK>M%JSd}1SPTGcBy?Zjl>*m!~w6{RIw{&Yg@0h^J(Nt#Euyf|2Nmu?x_oji;?_jKP z(D68Qyw`v8%2wge16=FN>Oc2s;-Mh_3Fy7t+874=F*N9;jxD2#md8A zC|GJ?ji=VHsX1i{^ex?jx5!aW!aaE9sFO*~*TO@)Q8q!xAj^8AaX`pZ91m_K-djw>NxuI= z6u_JNr4HX-FeDG86A}2sM<;(Yg(ZE^Gkdhr$#B-ihaPyI<#TvB4PBPsiHDL53_1Ec z?r(GKJ7F*>IMh0)Bmku}|0J%18BW*6CjlF;N9>e%~6? z-x*9phUIm$or1Dunfm!P=D%e5TaL9V$w`TxZk6hclf|Zw8ecC7EgAD0n2)OhhS2n6 z4!s{w;@>r-ZB}DZOP*~FZ7@aht3R=@Exy75u@p>mPc2_B&CkwK5@`5RCN*Fs_hIbB z_(=plWEO2?p(KNNVu(h8@FS=RTR)ci36B_{)p0&Ch8RYUCuMz84gkbC85XXWAxkEN z2|GTsiXP8GYc>2c4_${U>*z#&NIp$>d_~H|_rN4|L+4#~7iRiE3saG_Q7qY-z;d8A z4ZN0mSbp+%o1+`nRBqFd8%wxP0)t>=%Oc#e$y{`v8HMx!9T6zd@tCjomczB+>>4t zXr@!?^@AwBHOnhon?VPNXNBb{nd0UX$f&7fUT6*$qSXDS@NCR^PXX_HoKxD;H#^8o z#mHt|M5nm+_@}UGoR}zwpPG)uDN%zYgbuB{Xgs=ra3GL@DQzs^JWGvQ!mp^hS`A)~ zub-YF<21Lj2RQYLTPK!tXSOn)eb^St74bW8bun#HQ?{%?@Y-i=vL+h}h*;U*u?^rU zO#{}--t!*yILuV|T3&t@sW0Z9_Wji*cHGX#&3>>+Wzw;oAOIm6&7r&fjvw*5{rXDm zSr49fY~)Y!KjQP|^>4MtJ&$8;hG;MlGL^XCF+z0B7^Lmzbb)!s46j3+zi61@F>KUJTc+Rt zjODG}U}KSs>SJcP)p;3@`2tKsi$^3-VQCakK@$;4afWrPOody%{Hh{6J+k7VXDEj$j#}*O_WQq9664G*y=z_?jYJL!W0@oZ-S5Do_S4+; zaP?O+PvM3_Wg~6(Wul0r$43hb%x7WNv%^>9}&8Q!omh>frnB({fWVCpKUKYY^xVMhU$ujzb zW?mHeHEAjGm^=w(;5v(IH&m%|UJrgr3HOwFn$@yb_%UQx-8#Hx2Lys(uR&a^01j;Y z!U9cR)gKg+R4M-loI3rjpM zULz+B5bA(thwGJR3`pfxVcHqA3?{6%;lIq=_eIvYODb&x%p;+_8}%cj%$;ssQN3;l^p}tawgpJqHoZFdCsa%}9 z^HvW8dbL@|XP6m&E}a+H8oVQR{Muh8J~_7DZE<6<(OFZF>;7n&5>*W?!=K9MS8(T( z>{W2P%XJ)f z3{})nqMJ~Kzt2OfKxJh;*M%n%Ts~GP7r&hXw-i~dvpO%qf>1fy+lx*G*9KZop_Hq= z78tvk%6qhfvCo+DP3UIk>dfKJ^xV#Xy$w+)_U8kBgYCFo7?KwsltT zhX|P-iBBD>=3VztKe{&Gjm(-n9&+Q_&2ZxG%|VM}6Pu0G!HHjJXz_)w>J>F1%M68| zRI)X$<|C5~JNzzFG;u$ z#ullkU6`cLRv-QVHd@wLyyWAr879#1v5Eg}VPKwoNsGlp_$7p@Zo~ENstJ~Tq_yF^ zbTt>9g59W+cU0%SW;82y>?pP3tAq4_ljt%ejb-LNUEImVQ~F(ZUgTcBB!l_nQc1%)|WcMuO7b!(?}fa>b~uNeWhBYT$|&b^DP z4}sq-?oKxbVzRZ|sRDcswbUd8>KSxfy|#XQm-t=fNL^umsB*FHaV>&6b5=B_eKO8_ zWu-HiI*PkRaw{{_UFxyNhT;;Lb)O4sN9j&#fx$9zG-DiJiKuZ*za~8cXuhN7$!R<{ zBN>d#N1h;W3<39(FyH*qlFoA4DMNbF|8N>&KZiB~$>+_ru>R=H^7qzNL1gJS)x-Rd zz!fcI9H-@~e|4~LO*xCdXW5&*FGG%V?d29@bYiPP9zO$M2rNpN#Pw3={gf0orqR{h zE0OM_ewF>ag>Tl!xs;`(Rc|9m`XSIC`yL7?0N-67OV#+JlJ4Ap zk>k&apoA>Jj$od%l$tkx7U9Zsn}Wkf?PJLSXg& z%KLHm_-So21)>-ZP?%;816dzFI+ugE^s>*$iEKViD?{i69-MM}92p>rUX3Slt@WtI-+T>=Yjy#+ZKSA`EajZUz zO*MNj)qa>rWVn+=sKeu(ffh?*M|#b+Ly3XZhgF(uW^A0l1*Pi{W1w*rS%IP=;KYnp z^K$+L3G|LtsGNd$v5cYD%(=YDba3qrAoH zum?Nfo5x*g9+f&=Y0>*}9v19An|PaNEV`4Vq&%N$hctC=KdeUhulToh>F2ZC^^cKY zHON=EpLXBI5RZ%P8UGTy?UKOQ<}xWjRTV(iNMY|-E`{YtRD$g(;ysWZBw#AeR8Voe6E+ENX(Vp`C)}FwaSC|xMe373BM6VRjlY{ z?fcG_Hh#U;VmJ3K67qV#5BggKu}%+^biPEhd17n8yPC zVq@Ur{Pz*v2(=57mb=GlcH{P6g9&shV`K5+XM?48l1rWE58^^OqW;#~6)sYBX6N}y z)uXn~`qw`xQLo!O9CmVPZ8{I2A6Cy$KQ9+D^7~fQ_BY-6$~4$5#70o-ci8M4OG-6& zmbDz}q8>fnUZ-*xPha0@1wTnWF2M1$R0k;=GE8F2Y#|dE1K9Yu1KF6Gzq(ZCZ?YZi zhMkQ9SBdYu+n-k27~JbCZwCUHw!6kTO%uIQ>!TsXYu`tZ#Sfv)k~XC(4#wvv(*BY@ zch`ATCn;PV9p|ve3mb1v{hL1GJRAPtqsjklcHA_kyo*ngq;5~|!<4Fx#gv^>rx|FZ zO-9QL)aZxbu`BPXmvj4Ut>2!@iJ!tu*^U|11_iMeAKwq1z?9$c%j60Yd%AHS79#RSF#Nd!PDo zC>l*=mvx<^Ml;xDHmXoA&;PM)fLafFz41`mI?k4%&u3`UunB#H59ImoLApt4m@Gal zTx3{cHK!XG+6V~=dFDx4_0lozv*C*MBh)O;;4~ZTo&LMqV_C%GIi;GzEybT42BNl% zOplQ@&~t+y+vKQ(5lH>f+S*ESdFK!Fq7ow-tT|5NGY%};EE8r851_^KjLzK^Uv9dE z>T`&Am-SvRdCV4)i*1jYs!uD=T;5fE;JLLV6k@HQ9VL_17ob;uXJ3VB%;uXow3RGB z#N1NJXESE&?ypLzbt$p^`4}0}vgq}6U?*}NJti%ob2@Z$;9jvzafNjo-fOXQ_VY{9 z&?c{bVUO=>i+g4I$X5D2G9G4od;4grl%lg&HE9Q#NOJ|X^USph$*H7MS@lbcp6`Z; zdAs9arzjYNw#8JSjMAp@Y(b}mdegwYEHx*#6Ml2M^)4-k)=|pV;Tq1Lz&UPXBRrHCJlDZ??fXV&41$R>T|zmQM#qM=Qv* zytYC%e~;U6bT-n&XFWHfKPody11cm5t2-{bgDLb{KTL%SAd8TxT^~H&jCo&)g7>yo zSD+7iHcc?j@(pfd@^-zSj0!Q&$t2xEDPa6#b}mmxO?gM+eiy`xs|0zufnMaeq^I|j zC&BkuTOCTrdYxt?*ue;!yhFe9oI@9^ow1mw(=4@A&XxB3jRC1UWU4~yj)xh}xOJM) zNFM+_pweSIaDNfzO?kUHySUmuJoWT5qncIZQ>|Z_FFh@UQ46R;j{lv>G(giUySZza zzLvT8iW~YO7J^sJCE_T)cHBdq6ET7t;EKj^p^ z6t|(~xyprn?0%^zHnC4;4bLXlYpf{I-na3}KN-gcIgg3C+V%J|+LKP_7-)5sI8>G5 zFjRuR?|qo(d0ks3NAOY?&5wTCpbkewQ~x^&W7RWD!m|*!vSdQ@LTlTT=mC9lwI14c zVCcvSv_mX7_#)k1H2~^Y`3tw62heLSpPZui?_r~RbFrS{o#cY&{RoVLh_EH#_RPg3 zz-nkxY}RG&tT69!-llD3>2W*`?{;*nnZZ#H7^aetWj$thSLh<)dEZ-W>J&Uhj6m=$ zhEk9((OI7in%ya=si`TQUkpGI2VsxFRPFA4OwyK^gdLw}XZhx){I}0sAF=c9W?q#F zSN3kXZ4v2#Ah-v z*DCr+U&QF{6jR<3VsHWO;C(FVdN{oGZX$195ShOXrY(JmBL=9dF)N1XauK8V$&4Hc zdmTh#ISgy_T80{}W3s`Vb5cB)joZAga;SOxF|n~^Xrt6hRdaU5AF>7ny~R%lqvO4U zzjSU#S*qq9i3VL&m@$3~L;OuQdn7uHT(NT_0L!w2M>YAVu%R|2k1ej~f2fprU;1vV#f7nv2y!@O>rSCU%Ub zFs;mS)6i$DOh$>@XRds`C=4p|H2d9h?D7Ajm3H4<4_n`AmlL7g5Du`Kb8*Up9SGCE z_I-Uj3RN?>$Wflcq}?bJrVk;88Ml0ptf0flNGgSW$i0Pc7|uvMo`m>c9kdt_&=9(Z zVkQX79_!V-Kbb6kmp|RwJT_+KeR%(M=l<)4aF4Wff+!am{Y1K}!gyZi$zo3R!%ckW zQ{_(X6TI`OK#!i07Yu-6(K|jSiPFXtk+Y?*!=|@SW|R(KYqq zt9I6MxO;ZZF10$Fv$fvc;$-3`|8W2Q`_r#_3vwYz16GkzHL4-u=}F#)pU2q#K_}`g zm{?TeJ|{z$h8XWk$Fs+*>SkZQ^h3a~36U1ikpb`Z`=GC#fO$B1Q{dxnn&vRfc05(_ z1<%pNOyk{=f!C2ybzQxsiDAATD}vu?CY$hD9A}ujRC~~$wY$G`w65*Qw0^rAS*i1E zYvkB)@^JyD|FIHl39Y*KM{=-wt}x@*wu;1>u^6{Jb&2 zhqs0lWDA{7T}fGum_+SWM@Y|4^YOTh$K!@jrxj&uDpz3h(;%{Ns5tEK;GoEHWYpTG zE3+{wc>Pzc4Yg@g&?0`0%{WqIS2oEY%WSWA(%6l8T~^vRh^rnSf0o8qu70_`*D4(s zehqURr?|gY;&Zx+4?gM1KkBLOyk9gM7O))^NGuhEr1MHPU$YSx>+ z;q6h?s!xN}$t6;gu~Uh=Tf-4d;Z+)@rmF>Q$62`+jHdFa70ah^&1otW7mX7SwB`D(*Z%HG!o%L+uAQ;8*XS&{71@;u)UR*Igmo(> z%mK*)oC*6{{X32h4*MO~?1#sPbQ_$xQzpzrN>EmEn)-v>JL;o3*PyR8NTuSRIv#vl zJUhpQY1ERPHcFS(*n%2d9b_i!aW%OGQ7*6y9U+qdT50O!4Yl#Q-i8sIKAvJj>+Kek zjva%g(uOJxny$M09=l!L#t-xj1uI*Y{9Et3t(@$QJDr`L`Y?*ukTv@<3v; z$Y;gB`C3HiQi!^5&e@n!Tn5HRMTYe*)_at`Hr1c4^&6QPh^WRo@_Wju^5|&u4!l5J zz3*>ra4-!%Oqn$aw%DJMo2}SWuU2pN+MOHX158g&)&)HvL=Ni#<(=1~Bh{UfddV|a z&6qYOfrp^AfVVjn+);yw%l$>0rq@P$!4D0(P~iY&7LPTfr>yD@#PoI?9HL9uOAuZa z;*Hi6BK zQBtajLmQrqj+MBet_L>?`AAiw9dqDEVXOJBcOim@)HEf7l92HXi-j~oyl}6c?cfW- z-|N`Y#r2`}GG0RR965I*oq;XqH*N(L;7)(o%DaihrPC9n(@06u6gs`S7`$RLE5F;t zbZ2Oo-9Sk$C^ch1MkPKQygi)au<48J%iwBs8@?~-1%3P;AtrSa&Q-my*Wor*SiIhV zA0hR0@xu&!!7%%#VOPH|8I}UVk+48l-OkKv+vdkl;8nTir4j=JSe0|Rwhz6fP9#>E z-8__(rsf~dizK6t?;ekgsU_|}F32sv?ft^#krlt(bDAm6 zk2B%RzSf*p1sE~Jglhdw*^roRYZCswU=S$@w}^@sq$iD%1@maWV`DmFssfN*B>4bmweV-tsizLk4+wy^SPjw_L8f?M-8hTmve<# zzgZ@@nft@7?T%f+8M5KlPdMm6`M1Uh!w_;dSe@kO zkn*|RoPe!9y|>BN8~!$$-sHN5P$4fB5?DMJ^Ex2myy=$xI{wEZO7EWLVgJbyFX2lk zgFNK=5~)E-LE)Ie{@g$f2Ja}hHU3ElQ<-Y*vdW=vYG*93$ub*vk5hO2nN5(GVciA_fQ93 zPliEfO@o!#{%s@MHv|m{vFgyvNzk4NOIsfJn&a|HT46>(7--K}qygap=CzGJvI>zY zpc)cfYXLI;Os7vL1{&@mtq?qNL%$y-mfBGn6oj-J31%~80IRx6f`bBv_{adyY2ilG zbZxkem!pHWv5aMGwAm#vv>reJoA)Y1BU{uGWEdyGnq($5!@m$=lDY#8BbOs=N(mha zcjmWz!kgRUI}5pww@)*94)uN=M8Q5LK|evkH~YS~F&sfFWFz`EA#v#Bne&p@YQbkc zO{O=OwS^JqCG$4;(+QMj?%F+6$zLx&=fdBQCf^wp*1ok2jT)|X27T$-vl(QY6P#yD9Zv^&EKe#?9)18S({svYxI5rFf0g@ z0nT=UA9rN&+by`ye+PQG;rvVMxO3)ZW>HT_YaXGUuCFW$52t*+iNjd<7Sjl#K^ZJ= zRPtQntA-~Wc3VtT{U5-K+F3`j3ZX-^xPQIuZ|cD*U^0Jgu%edh(ZY$$t2_NmHW7X1tUi#pmBM;=66J#a-5&5%aGa1DW(6K`5=56%eIJHf0GcRW*IQ{x5#A&&+o{4n`|=5VGl)5opgYC7MX z^>rRKHu6qK4B1a5L*)lc%nL(<;I9ey`!9r9$AW@n?a~L8&xhUkG9TI-y6LP0$GiKK zsC+jQX125a!uU$n+Ha1ZysXN;W;1wI5V1$!;`H;BH$nKhk0>;n71!YkZG!IcdZkCB z%Jgx5>@(JUg`0R8p+}sC1&Tn>@IUCa^RLO8!b5^p#06P9IO|6#uW2>%xZw6Pi? zhS1A^`*ko=gYRURE7`ZZw&OzN>&DAAHTJ*821>uP*Vb1lY|W9+J$;S|`jC7|p$$)` z&l#7$ogSV!VHkQy3<~-{f7GLbKWS0=-rbi;=wP#AElWD6KF{S$QG}&=kKBspEOtck zbqd8L(q9of;2f>gJgq^D_N8Ptrqr&zkbL@mzY;D(JcV>7+>8--xPM=hQ`ERxmk!W9 z{ww@NR(cQ;wTo;`T-ZzA#|d>MvFcB5>w=%;=8JoW2d zI!|rAK8Ch61C|Rqpq(%or;fpuTPTEj%5;m74bt(bNsp4J^Pbc93C{Scl(}`N&sLp#kl{1YbSvd-M4Q zz!wcY&_|H^)ry{jX?ZSIaiW)#F<94_kaaLRlipiLdw*gf`rqF)`KTcMO*2Qr|LWv~ zySM`M8~^lCLB2?Vlt<-e;Bt~O62!wphDW&~uNFZe!n^R zjc`i`p-{mMr{D=T+Y^~>vVHvAcC43Lb6;c>QFjxl{$HOU-ePJF#zWMjlKBD{}Bhg`l7;eVz`A zkNB-bP|^*$ME|PykBc$izIgvebfchX#awVB>bkKQ>#ays#h!h8x4S>%z;F~y`hzm zkboZRK3O;TvkUk0#JR;tjR2+8QXW+)YoBON}+k4&LFy8qJ;JRL(!ojt-E` zVfp=n&JO7!-!=cvISRL1tnY@C(`(One|?Wy|B}nI2oq_djh9N$uu>uO?3QEWx%!(W zkNOak{2iWh(7|wq%;8&jFzy6IE0Y#h691I>>!S-mAyT=L3i}U`h_$!#3J?$a7V+a| zqwar1?>ZchbdlBgjQGy8emlJ_V!a`G^PBlu-u8Kfq(nyQ3-I@7S(`h+OTqAX((u5) zz$b+mN3-A4Ui?O*MSl3vH0fkXx+MTQseq3rgVZ?I? zV5q(q_N^@|q*#4)%0T9P(m= z?fNp?_ZpNIv&l-TtMn`lRl+@a4vv?=+|pfh_E+*yltp@vYOFMi_3N#X?7~WFZYD6? zZBMooL~eJzr*{v3YC!+B*%Jw-#!djZbA(r@+GJ|Rxr?twGcrf17>CJeQTH&tE^RqhvSA4pEAY@%l{cyl37 z-u6uiRAh%Xt4$DDoevR@kcYp=af$Y+B=MMkIi_oxmU{~S@Nk!f;H;rj*3bdVBM_bo zHrSA&eYn|(&&n!U4ZO}dH5C(6J6LRT(OzF;z(QI_Xjf5qx_dl)Y;N;t5Dj?SQS1KG zy~`7v`BrhqPxWapt+R!j`@bU!+P%HKg~wrx7hWlt{|n_j2;C5>*GQ z6IEnXk_A^}?=oBBu1(j(gEzwldW#yCI2-sFpb#}pg2#cGb1WtXPtT|}6!}4G>$l!3frdr-KgkKh;;w$L zY)$N+?BnI5Qqm(S3PkuaDbCQtJ_mkGj-DOQ*%$LD+27JNamed@e?B932&@3m<5|-d z74_T5CI7Z@V&f2`|5FnvO=3MgJ+!<|3*T-Ow%#1ci0kL8lkhY(%jE9(BIoybch`5n zvPrRclEgCPkH3(rqBa-S?)6ZDJk~l&VzT~bNwvJ>)^I&t{E3ZWgB8CP2MssxQ1p6j zT*7p@eftEN2G$>AxSwCe)NiLl#yurgx_GVN3VRRx>E=#)SFApN zZ1w*AdmaC*<$tX-fFiI?FSHN3oYAULr4UmvMOz72LgM(WUIRc1I z0Q;x) z_`aha2HFV4iXzKUvuf`}3++@c)1cPva>(x$0!ay)t0B`{4VbKS$-8{+Ql}YYhE18G?K*BU-v9R^?iRbjLmjhr zB5ag2Ql{-clTXD5%M^g9GtVVi9nX<%qW?8Y8h#xf8n?e+LndDbrt8V8oc!ph=N!P#r=6?2U_#+Z*0!VVe?ZfOPM=!h{zPye<_ zmQFEKHw#wl!<=GW0)1|Ne{p``idl9P)yTR>Hl-o{hE;#s5K9N$T19(ZIQ8kxpY0O@ z0s$8WA?QDxvBQaMsxv|>2ao&?#h}@z|MuITFZ_G_lPvu2<;D&4pRNA?9{>N3)y0!J z7G;S4hAsd5(ux-e(O=@qzc(7_bl<~qL;ZIh|K3~i>i82F=5yRMD^!;RNq@)0km4_L zI*)!iHb?1^K{Ua798S8Nsr36qzoXEasozRRp=$EGZ<)o|krUtZ<%TZ55EI7<0|_*V zP@gZgFZW~lno2K7CmE`EjjAAQ?WS>4CVXs+^kqEvlb z)kIyq|6)prkS}$v#o9F34^7Qsz1hlL%Y=Vd1&g9Oml9VL8Z0G4>p7ntJ-<9122Vdu z187yd~HzJYpM!nty#b9|fN0f<8%KDBuB=bQ z8U{J7(bFIzP0x;TwhpqiZY=zLaBrAh)6-{*Yp@|9whTQx<5ciD%CT3gzFspF2_#< zoqWRM>pQgZ68qbqK2j)u1;)jhJws?k&40F1Eu@|L`XsDOz zU!VEEy8!CsaQJ3LqkmJG$COAOc zZ3|rNd|c5Njknsxk*xsPqm!dE4YXsm0N#Ep(?07akW$@sV9>8)`b7SB)!yifYqSae zJ3=rue<3&xox@!R(_(yNyxIP^ynt{i?vC)J>D>CDqUd|Ma9(#OK6 z{XwGV@US&ErNG5co5fu+8j6nQg-H~MsbQ$~Bc)s> z6cAN~haXy;N4wL~sL_Z8iw_XuqR-_{#sC)b$-|UVlmjEcJ&Ow4<@3HB3h(|=@{oY5 zg(J=>U767&_;MZl?u*teF#>(LhS1v5tcJR{sOcWz9e`e zT~>!aNAke(8nfo6@#oj0*7-O>jfCI#Y3>5X2QZjLkm%a6iijT(V%5ZPwT$CoYZNaJl_FdXr_~eEZq-)?1Sxb^pZZc1hEAzOp0?N*q!I9TD>f#K!?u zNu1<&MWPao>+Uz50%ZKkkWzYukM5b20u=^3+|d{Y9=n7rU~8CgW%nDXjzP@oI_vkf z?7Uo%A+VkayX~qsvT`M77Ji{23k80xHRx-|i|sNcH}`C;HPN~6!b-2-9}_IkA$|xI z`_(tPym9e`?)mYPn9N)%6LT<~ev39J`gSk(#|fH_E`X&Fz6+<<=eo`4G-P>;0@vLz z@Z7G(-joi!bC~)nm(L{T62HG(P;;Fqih1%Ub=U&gDMfc+gvVCHQGF8cuR!f z0tJB~Dl7`}qq+$oq{<2v77@k*<*^E|i-z)4Liv7FWley{gmh`lE?3V&mtx7hZYbj` zEp1?O9pAKVp`*@3H@YoP14`$)Y?OP~DpRXIyqf&ps(17rZqud}qX+=Dl4qB4kO|gm zqU%1|rj!N*AXNyHwl0{qfWt;4EluNV;9K18=T%i+$GXF!=5LHdgIqrLfWZ& z%Ynj=VY0kD6YkUMGE9p)^ReOD7e@Q%VDqxo_iJ@UqF%cw;c)(c?$3+x*Y;4yc8+U< zmht6?ax>tLy+KIXSW-?hu55bg6?)l{wE5jdP%&nF6x7$}W1ROkmLniYH5x_E4#5zi zQv`#yI>gDxx&x!_u)JI>SCSqv76mm+5*;Ixk}o9yI-C@iaB%1s6a@T7@u^k~5sccI zNGgcVSfh{I_l_ZT0eUG>pM?DxXa=P-As~)!Od-*8Dd|5Vk%&W%GDsqu+pp>qs)~id zKp-<{;?;>jnoHHg@I?DeR3>S~a)%kxRKp;{%0NS*b&iS`GQvsn`UKg{q9UdWLq%#3 z=iE^*)ZKuEs^b169?2~ANj56;1}|>4R=&x^C{iHE3)8#6cO=M>3hNADa(r-Aa9Gsn z_%&iPY|f2rTo-uCLwA%xFSqQ;@}gu9*3DAXv}cPa|KZgv_IgBHk?W~}AiW%@Ti!8( zl0{ERH#$5+&QuYp3CC@_7Mn~&EJbZKU(x?_)D)~}s9rtd;O~|c2Qc0g5HNm)Af#3C@A#KFTrSpC%hAiI_&SFoHB&wr>DPk zV|MYnUF~&80ZDHIy_n<0_F%sP|EKQ_^6hT--3rhjzf~R%bUwe~o>5ik z{bDxu^Bw+oDbmPvH;?&~_Yrks z*D8v0p&`^Z<9V(u|8b~eP=(s%ba7Ep%^KrVP%yxQ0DsvXyNOii%Xr-0ielDpf!Lvm zbtnF|U34m-u7|^9JT))ad~pJ<;gKLmcRku340aKmQN)6UrP|s$VX{Dj>DrWta(chg zP(i1O6BRWx4~yUiZtoNc$$0t2Y#--YppZfJPSEL~N`mf@aIZZzC%V$Fzv%>i3ZlWMPToNG3j?>eum9rz=-!mx@E+MSe@75j(tZB< zbPWb%N);>4sH|#kAQ*vph6Y^~L}DZ-2Bm$Ut9aoowI^O&lByQBO7z5~wn-bWrlzLT zVAJaGNJWqCFU1mL$k=hh5k@4*bn{yP^wCiL#ntX&kcBTX4okv zOcl?BjkN4?L`byfgape}j42kIWR|~0+~z_Xxa6-Ftnd1=-~QZe5Si!KA1UxKGufKh z#EcR#e!2qQ>cvam_qN zT9|}}zioXS1MlzeKTAYC@AV4Z7qmY*NWHW^_Sh5 zCy(@iPaQFnTM}pj_j~D4Ywa^a{1}llJzI!D{BEU(Gocde?zdfaI33SPJ4 zuM_h^N$}cyyvpoV4F+JMu8jpPrn?P zTd$O-CPffv@rn%%u<%9TvkT;jgc?1+cv`E5cBpzhO(m~iqoSBfRiY3H0;%3|=sG7< zcC4!5-L(VobRu~;6u<(CJYA|QscO>hN>Ba{WK!yF4G|a|S=3P&qg6#Ew?@ zPEk-ISzs=eLywb^0w|7_Y8W%dZ*Ea(4Xvk}#v%wr5_+9KyV+aWnFkCWXtuY3nH-Oo z)O0?^X-?1FP4tasc;9@JlN%K!0Lu7IDFqhYZJ%vK8y#2N9Wll&&Spst%>8yMvu2Dj zq)C&Hj6joDPyoS@8oQf4rf`C?J0X&$2Ybw|Tx|r4I(O|Bp)=#!~woJdGt zejiX|R+1(6r-+r~2QIG71mjLJfyj?yGfPu5YfFSkpN>Sb+S|Royw>K0OSKwoy5U|U zQCZ=WCu*JLBqVe=u$>9b9F5Jbj8>?soR61+m$b=0%kYd1?3r6xX|uyPaK#}8<{Te# zqpZHi9!VLlt|O~5U2O`Ut9S#K>x>kKb-uazIj=4@3&62&@`DV8*+}E1<>banwQBO4 zn$&6C>4*s4y;G7!LolwZD+k%ypuvh$#WlI#+K|OypfCo|zhH_&P`pJM)zwvkkpB#P z8z0}k))2GoxJ>SFPOKZld1l~j`;)GTghHHV62TO6m{T&Kkt z0{Y8^y|aadBFT1?*2$1RQy&Og+TN{D^x9N!l$QeuE54zo;?wDk7v{3roPohO)?l zEBXG7kauxi**7qr3I$HprJjrr9@~5 zv0=t?W8>oDq;(%t)?x~V&WoWd{;l_bKO|#Up>|PkSaQph29g*yX7Ot)H5^=Laz-jX zA>jmrALpzBzw@t4kVagQt^$9Rb^qAdO~TspJfy7L=*YONWuK&n&1L@e`?66D!TXsy zOiEL#;0JfiYztGS>v^Hq&ob`QvLkD)W~T*->o1v)y11yKhxENO>FR!+mq8vKHWEs4- zg%xGPrpkdLz;Fk4Q(pNjfLvXZQx+{HFDuI{^X-AdSE|0g&e|OG?)`iDf|`hk>r`%` z+`9C1Id%spC#TLa)Hhei*#OJcdb#a#>^QZ?fp-4q2cnE5LgG2$F;kMva2%G9!n(3A zp36-Odv4FX8DJdx)qtcf^{idSW;1S{50y@ z?58wf206Jlx3o5i5f!nPmhmQlJeA7|CAiy}wBbDK`rNgpq1O2|5|J(ekL_Ow5Ja=% z4|g^xK^)kkVk&N)D#~#ReZ4q(1K2czKy@%+ >gsGVer(PGtZnM z;Bc}Eg{`ZC;_oBNX2Qpd9y&K0Og1@jHc@DZPMde+tLbriA`l}5Pik)jJr%1JwW7&O`GJEU`wVI=~(KSIJz`IMKGgo<^E zii+kYhb9N0mgJXLQCUU2xn`?@-lW0Df5!)I<-tT6!zsanm_Q#CTdEfOV2)9%Y$=#l z*QMk_0OS7kM~b;%e1bX+Ap(_?I?oSD6VWVm2)Vg@#^sC;ZicH(bv!U4vHibK9}aNL zp=2Vg=+ZR)v}2}Nl7xkqw@+Kd)xS{sJjC^L3Ktd2x8)sEF8bSdsPnUdKm*!tXQmYd z?>#BAN9f3qfSZbWButMKMu}>3JBrEaoM01T$nP)G7YnUglkBP+9KBB_3D;l57#CRLKz>bllFn-(&fUMWK%RBh)G&d{7DKfGVFo z0msVPGAshEcWVox`euRL9sdwiLQOsWJ<}bc@gn!nMKLjQhM8q1{6d68KlBhx{h4Y; zg<4%|2vXS^=^2=qsH7rwXmSvM?5bvfl&AbXT?DhzlVHizx@n#X z*tod3d?2&|S;Gs9<>chxG3)DEQ}|9^15x^>uYX+$JT{?+SIO={nlCy0&C^{KAX#kH z_;Md>Lz!7OhXfOAH?0>#;IR8U4@#QYQwmEHqBx0mUCw)l>#TT%?s|^T7WL{)lTiSK zN$k5+SavK;OxmD1#q#=Jbg)g;`2KeA*|3~gxX+*I3-DE>F~A7~sH4*U0VI+QjZ`74 zo_5&H^N{9LWp&t49Vz)v;{vZnM`Q60W^1zKcSmo$6_tk7ASR$_PGS>VWV+}|ex9Ht z7P>X5vZ^Q$e}*(4JsD%C&$}ygS;OCN|DponW6YN}#>R|^(2iJZD8T|vmOR*$Vr%V5 zf{EB41SiQyu|Xq%PT=i&n{M|(d%y%y+b1+xYQ*ekOSWBMaTb-fhKa!Qc2R-%nP-qm z914bp+I)h2xs$VVeO;aOoc5wdxktkIqTg^zBpl?5qQ>&tq&ila(XnLKWlar~)$+}i zM{#{sDAn(hoESslu~B(;2T{?jhx&wKAiYjX5<{1(Ta{nS@=A*o;HNH}i|OFUiS=VH zRMl;DiE6m|hcz_1J|=H9&Tr-Yu?IUvq~+1Kg{vZEP&ToGwq2DXA2HBFdwxDn(Ov`9 zpMWoGS5l3V#JDU>B50Da@(FQsH#i;MEY{V0`SNAoSekiYdYAky3KD%%iULawUoBir zkWO!}lRwwooU@~&s+!tIo`i|}MJ}f%u=T1wuQ%5))8~6^m58}eNyZtMrGy= z^)>PIAH>r62S>gwW2i%j0CAwA57E*@o5hw}W!}eF!$7|is8&k?1_-$3!PCO5@}`=c z94hU+cx$OKzM7M6Y<&(+TB;x%6p2y@%u9BX(dcwFT;4d@(NW5Jn{`$o?9}&E-j2Dnw?y`2F^i zW3gdF>AHQ1i>e4LY{jWLP#SJ*2MM8xo9O{Yeuu#@FKRDu=E<$-NlO+e4T%cV3!k`_ zmXb0AOMsc|@gsD=q2HoaI2G&CA=GM3|5eeehV<{t`~09L(DXT>2!4XFFR4`+1i}SgMR0S93(OL;7-k#^Y90=qbWto5EDWi!ZP~F!p zw}xo{IeZZ#L&!DIODF*Eb3VDc5(gbFEiJhm zFX9m&6XWAU%4B>U(a6u%{F-e~r4N1+D9nMm26mc0I=VXUBsIOB=tF~p7)`Ocz0#PU ztF5cUk$9(!)av3!Wofyzq;q@p?E@~sJEHI40!;{*xs-<>qh^=W6LagDfp|8I&`-d1 zrld!%gIV{EYswjfAX#csne(sOZR${ut@H7T@PDQ%H2127EDW09#IJ{AiZ&~4MrTRR zj#;PvHqEzvP~YaB1y?vWK!u`b+k4lNn5w(DJ!E4|Gag&>-5!PaKLo=|)He5o$)7pR z=du2`9BX2d*%R&_fnDT(emFmAKII)^dNIYZmH+-9n2UB&VD6JLUq{62Xl`67lqRpU z`A`n+o-Ag?FFyfe?q}pjFf!zqd~EeF)SvORv9Kx2#Lbeh00#9x)Ym5(yu_D$H_wl) zi_DDgTG$?$vv&V<5=q}bxO>#OnKfSS<#jYk&RY=#2cP>R#{9YtBp|Epz{92{jBKj+ zU61mYF5S25gV}@zs2=x$BIIXci8){04f)Y4EsKj`kjd=pc%d{)M7WkG!EHrgO3s2- z{8EuvL&)Q3yJL}UtvE5RafsL1Oc zKu2H2M#{>aF}$x5 zdWiWLx(;3g2r^S5d0B`gmgE4e^5+kPxFX*p(afr)5k4g zD8q;h2t+3*7hSYcAC9)hgOWuTTjCZVc5chG*3;AAV`uM$Ayw2+i675@-~W-dCl%n+ zMHd%E;v#|lrWlfv2ay_=_n0&r;ZbO?k%l&xmLBetwWttCfq?!pxz>6a1ziflZ>WM? z&QRDFh;Z(WO+DRZ%%+z@7c~3yrR(AB!6>!md z&KCU}`|3X>DSpRU*#4-}%!fctSICcTuyv{Dt}6u*MaeDIAEPn89ihSg+1a^ioyr9h zU>_y&fe-^|ab&`6GA>`2r80!@`7H#U-lA7!iT=mhUa zRm%%KKD16X0ZoX^E>1~F1~af`#{0}IeO+z@om$!MYc@FZbLAuL5b3_D#`qI=0LXwx zvoTvTAe_2HUbp!V+qR>jg~DgUE_!)+lU`bm1k#C$LVX34)RFSC$R+Z^i^jC?42`0L z&|}4fcZ6IkQh$!bbQeTwXvp<_X@5SZ_k8x3{cIne=ayv+QLF4x_2|FU)Fk`J82U*c;qVRvNt|SOY2{AaS8cKERApU_+sNEX^I$JMq%k z!3=)$p2w~_KU&J5$<7M^@?IVjk?|$oy!Se)8c_B4b!F@2RNyVC^fni^2D*q8fuUBt z68QUI+~mq6M08AwcYaxcGcd}qe)a=X2xC3e>rbmXQN*W9?wP|?Jc&|XYwWfvJ{>T5 z1IpMiAU`;*9+88t;8O!-z}Tx>?=eyUjXEPKLP1`>2~FIHG&66=^K32e#!Oyt%IWAt z=Tt_WODiSM2NMQWCJxrz&yw(fbY7g^r z3Pq|kmnOvhO$Ae5w&N7M?XD(O*N7r|U5;1Us;rOOTb>BYI7@|j2nisPLMV~;9WT1a zAJN|PY1wU0tnxvInsYoX))DErSH~=a4{M=1mQDqsGcWH;r_3!3gm_qcOZ_0VXk2OU zekil6OcN~6^W2YISt*kzjW23v0q1Bo!4u&i#v<&BO51wt{;l(C>*M+ym3=GV{AI1y zSWpm^oSf^Yrh=!n{Bv~2J|Y50bC5j?bf-|V7fkOvp~_u?Op_PqIMU8&0<}J$kHXj+4RhZ@G;GS)cqRkk8yh-kYRJPnlWaiaR=@@eEquyOw z;{72y*z=R59?G|<=$#tvzO;R7xz+Avm{RoE`pXkPfO1zeYT{Wdj7-E%SYDt%Sz7;1NS{cdn^s$Qv_CV+ z$1qJgFPpQ@psm%E1r~tBaXCIVX9RMV`XOK(l9<6NeGDd9_{A>@Nh+=!BVNj=hn|Z7 zfh?hk5{*|tyJ9E|2hMW9U51J@4F1fs-51RpYwZig4hjf`$rRQWWoda?T~Ll7K`vU1 zYlMK=kaVU1p$_AcqLOY%szbp-;C~cXR*KdP{(#z%AHwvhm=_xk;scc85*e8ikw&A^ zATB)wEE41i;RZ?W0Oo{GY$;;O21_Ya;Knt|D28100CY+#4}!2xDFt8bd$|b&I`oe$ zAG^uaA+eJT$;C!K8gne16u&-Ed+R10d0t*mTNEQo6u%U_ZWiD%{cL}-cq6u4 z9i;3<95!Gh7(<$a^ycSUn6Sf)*@`ANeUp>lMWDy6RDpC@j4=Q>Y;cFYDrR$V&|-AT zH$;qVnajGgEVPmdrJpM3%QV18odJWJJzKv8kQ#$zRNQ4QvjqqBG`!dKdi|g_C5RUJ zSox!+sJU$yam&ME^BsA4_!qc^5@)b|(A}MF(^6C4y-$V+fbtKOLLk-FNm)vn4-_R` z9*c@ZiBORJ1R@|>s^y7GBG@}MsByN|F_YlHHl?A?2nX66DJxb|Qg6FP7#e{6AyqyS zWi2gLAqa#|v{VhI!X&M)*`L|i*m&~57X$DoRG6KJLzg5i|0Hx22>?)J1ul!1S6jHU zs0iFL9=PJ@VP#BVnq#?Je$2}~Kfj(K(O6!7vUyBU?&(aKW`bMmzB_lZlsCJwGPAaV zjR>#Br!qbL4BahFE9dtS4VK$_PZMjh!A6&vq&U!AySYyQ+Ry}#?R;>5Gp3R1$0Lky zC?W+k0q^oW=bTPL{K)U(?9I)#Hd(5gZ-@1E6kQ|9R+e`s9Z$MbSw1w!XM0IG0^E83 zGPDiDznJ}>j~1GC#?ts?yl7bd5-2>V@8}!_=5NUWWD8tsj--}~4AoBq za@s*2{LyclYHwOrmBI7ybn%)v{_$3N$5Jxis19jPEO+SB+vALQV}-9e{L2@rt7{&Y z^V>KjXKoN26cCU*FZCP>zm2#$E*kA}gdu6jd?-QD1Qs$qhUR4U-6E}%$&~w)pFZ1n ztF|t>Sr2(}H;7gUY`>m+znsRt*EoGtcs+lm(}RbgQ#aIHp+*+R^f893g<^b{df!_( zaoys(SYsoiV+E~<;95d;{Bk4+g204Phai+>iJ9mPL`Ky*=*;Ty)3aA8W)z^N3Ze}3 zG9I53vj?zq-N&JVMyCiF>adR^RygCi`4@E33N4PNVbFW11YD|W;x_9ogDD({euf5n z6^3#YILKs6V33SpM(eUmVzRW-0#LZvf9Uh79~2-@BzO_?d;w&(tz3Ajyw0FW%jgo` z(ebhG-{o2G{6l3XCwu%he?MPVRBih??jEK)<5o|xXnJX$eYez?v8BcndKb;7W@<_r z4syH&KP^d|b1L&U$h6ZTiqw#iy$VU^`4S=t*0!~>hVgORV52)ypT4N$swm?cAn!ag z*o4zi&Y(z;0m@;*qy6NJw7QQBxwZ;Jd`AavVS_!J_Ooy!-Fz@5NOn@1VLh;x;$Ra2K0}Yj3EG3gv|WMk9P2ETVA23=`Qedq{XlVFY?~1eO{- z#T2?Wj9Ch4YHeYe3xziRuva76ljhPUb@lbLjO9RT7!oo)T59l6ris=RCRBKO-7*i> zNAoEfpxoL`bl~lQb1;3e@0v|w`J;Y(JfoiLrpQpmhYMT3v-ROp&*36P0wv|Em4sfh zUK#IM6ge>wNqTe`<4$#5D|I?aJ-TwgFz?S3iW9vxx712)I0ftNljRXi zaaG&KnmAgk!-wXH@>pe>z8j?KWH#gUhF?fOKknP^^Q1q;I}Ce?MxS{|BRg-d0>gp4U>lZ(4vaD)?2>a5&$vPW~~La43FNY%@%J6D_ngZKxttaFegI6_if7!_Vs}B za(rvQOb?;Tp`f{YY~`r78S?gL(*O7^gO=@M++Q1kSmrEkgMOqUSb7J<$&fX1FDUMw z6+TkTMn22dq%Y2lV+n<^U!7R&mpxWzWM)GxBX=SA*_Eo~!ay!R`$<*(aLANKA@OU! zo|I4%H8PGc^V+w(K7%U@rqS5@>o$%ZcEz9ef#nTTWwJg5;iY*cFVU*;!+Mn#C94k1 zumqab-6JGm#{~g-Kl)j>Av#j0&qO9+jz>{W{np!LvnivLoV* z!5k2{MzE8^$eH@zSWAMF{io~+;WeGpJCu22PMf?V{9uQc(93LwcChHfFxruAZ`0Mj zUbzdMh$CTFl?)(pcP$aX7miH!Ay=$$%c5xi8lLUdFrQ?lmxxcuZjmx7GMqJOZ_!f+ zUCwDHEPD`%9``cjIGzgJ%&uRu;gdBpV%&P}viefT|+-Y_$2b6?jo^xv`YZ5#g)vXypu#C^gP&=V*@LJYux+`++t1YNTp z_I#>q)P7!Ld7RB$ds4{hwe?;H(_hmEcD>}4xs$GL#Qo|Fb{A~teS-0zUG%x)VsNdW zP`#D94YAZ2er31$qIEkpc0OouRym^Z4}A^@rJF}GyN zesf2tCR2NE5_wtE=qc_rW)kQlPF$P_`X=->#egr|LuaQeKkuQq!_xkqae1l+`}1bV zIs*WpGBPp}aF;XVRA@RGAL8Zhoj$?^Jq(>UKeg$pvtv!g*O{~-3=42-2-54n);ssL zH_eV^>9XFJB!~Z-4;m7*MUsLU!~_Hm!w;X8I->tumLP-Mt_mK6#D=EUphK*e0VT`& z(8M~dUR0YQ>P-AhE%VYY-WGvcjx^czsegH}n*3IC&g_4i6bpm#SA?$E^&SGfaiC9$ z&=ZQwGZQ%i8J#oAbe|kT(=}#V)RlHW#|ySS?%%h4s@U^}U7dbZ6OWi@9vx!Wl@_>{ z4r08%Io@k|abnb;w0s|P%RXn{dJo0bF1bE@4IWx+>j|X`AH!*21|KjtXs#y>-)>Yq zKIP4+Egs?>@1}x#|J=(z-R<{{{tNK*%?Cdz2EB{Uv)}#hGp|{wmE2|pb@#Oy8}R6V=P=-C>V38u2D9O16keP+j_;kC z95k#AE4bJuq+TjlH0QZ9NjziWx$FEki0Y*;Ya~ms4pP2ei+BIqOMkdi!67gpmL!;j z&~8qzKg_>>AN<#gw{N!QfecR&eBz@$JETmLt@?eawn6$^kwO~J{Pg`qwFM1A#m!OliTg{Gz$r_iu zb)ocL(fIx-xk`M>ij{fAu(DoO&s$~-A9l;sS`OG2g3hizuNy1P%Iacn{w*#j?NR?G zxng6szS86trJtxqf0r`+rzaMV_Xy<(lh1Hx$lpHA{L7!QJf$1`H=2Yu!`v`of~Yp5 zDj4@Yv+tliehV5(t!q`uQj>I)Cmys>M#s1F(lvKKdG^~&A!ssr_ z)vxg5iPF!B=y{IIRG55gSed7 z_uzNd8XKD~4vi2Kf6X5i{iD?Zi4MZwWyQdV;5TPrA!d}%-W|`U?xA}@_vnr+u7H*+ zL1caw^}FkoEo{8NGDBxap7WEVDgQTuHvezvAkBMgDAlNy#BU?P#S2KAL{xM&i+zX1 z0DHa8(4~R=5QpcdX2UH99#%Eu7uQd8VxElL)F$&nCTarZ5pC5tR))ktyjqX=tc%!$-%*nfIjQ=pVrG^tI!q)ErgR ziGV;olN7c#K(6%NoY7c{(Z*MMQUlbI^7twg6hw1Gafq6`xz(5!<+DZ-dVs|DnV#=?GMIQ_ z!sm%<*YtmL&qL~eKz&>GyOa<~;AAq~(<8=}=-@;Kx(o=?8gVlhMY)&3(C4 zVrzklyUnbv9Md?#4bX|qu?5^t^in!T%2mE|BsOW!IvCWxm{d>VWIXyEQbIlVW5)Fyzq0~1N@LhmnFmU1cO+kM!M~ojj3hJP&_Tl2NymLgB3QNUX@A0P+el(NM3O&#Gi+L#<<(h2qWU?{ z{>+w{JVr-H5`ilqK_!JEVDF=zjr)bv&f=~$|D zdo-9MOkJ?uqmKj~qb1@Z64kxL;LKz7M3J8&1p?u?*w2o8+WL)8t;^da2*jXf8Nau= zh+P5@XMPT=rZ|uPx7m15|GPIVd$BFbjv^NA$%D3nNSbqRGCYdg+FmkV+6_$D-kmn! z_*Ov)J5jK_C-!_CZFpO4BR3pV4()wYli>Z-?@cH}gDMl9V5Ot0xKwrA0jhZ6D#r5J zQ524$T!gIgF;(}I6s18BR|aE%ho~Tfp?oOJkM}h6dE|OFNU&%zAt>k(88^_S_IyY{ zo)W{WZOhy?9!zzNi>9$*xkj{7QtD7>5S4k7Y_LgM9WfJ-G_dL+Gzqbq)np$GvoIip z6|+knEdIijG7B&gY#a>YBHc!sCp`=_MyZ5uI}H`AkXBX8KEies^uACKc8VV@fqbC_ zK%6~>gVx0z3m}6umjMYQ8d1T)Wcs#1u)^YmkZ@><`Zx;H`;jB0NGS=6B4oiX`&q&H z`-kO{g)c+|&-(o^9mJkMkNsg-P^*DL3L&0509D%?_D5*}fd(SQ2^ql9#vp|HqiLWb zT_m-}+6_gh)D5PJGekfwYHb7&pail=(qQ)Rc5ro)2ASfPFAV9|^P#GXi5XWhE`A-& zLW50$>5eH4u_m+6sAyh}eSdq7QM)5h1R;w4W5*i_+Ymv|i+645>;81X=RhL9`#W+z zZq6C8kM?YWUYkd$tZk1=d+-JbWmYMNpgBftNSM4D)QznVaU%9Uw?IsF*?=ltiFBww(_$t zB}wj&KkaHYGZU@-g!$>y=~|N$sQ*Uq`1lx>?{-jvSpVQ8C-|_>-wW4w$%j*4}=GD{o+f6fc-{4y-q5Hj@LwthjYAa!y=ndnYGer#bYL71aJDd)tF?>wJ@W zQf+Pz$1=q0#uo4S95?LdeS?-#23<=ZpTOzqM9oPfksnhSNPDXMfoh~4m2LIBX|k>7 z&GV;MD8eD-<)ZyfXk37#q^}>a>o@@*Q$stFKXw!n&VQ5|lo;O?zW=9Ri6$p+c0Tp| z5B|`yq=32)_z&d;1rpUN#k4u399k66fUfXErMH2# z_3;(JHcwthMMYh#l!Zk77eaTtVEr5Pe(m|iruvz4@nm+03lo!V*30!)T+6G^>Fp7z zG;#=?NA>Vu#W8vNl}1p8?ZphY@)-krX4-!Wf>cP-X|T~}#9)FqSd86wgN}}R7U!9H zee3js-~3_Ozg{OY_g>;_>pcTWHlaq%%mFL9-M(eE0FVJR2T4d(Rt^s*_w~(T0pl>Q z-S);b^V41Xb7JCDQ9y5?4nV1jh2u5BGaJ$asB5SU~FiX(Z}(d(`_S zIK2%r_<+l1Nn29B-Q%I19!Z|!f(NbE*_q1hi*1|RU?`WsIS?iYyzJ$#I+>;6{g`US^{8mJ{6_wLQhFNr$%5E|auNJ+9!%_I2Wv$cM;n+K z1jUhraIrTjN*zV)`TDE*MAqx-S4PWAiTB=dvkr-+LA#)Ukk@a=1}v+ig9Ep7u$kBW z?JH+`WvFBgL}gVCB`cQqR=kFP*4fs&_pfcN^~a5_=ESwuh3apQlUq^a-%;0FR#Owz zB%;zj+z;em?v8zz-HjqVb~_I=^S+;wxmVnJlkRabCV{us_-!;ym9l#2ldJWYEeD14 zmWzFfidB4s;Os?N5r3Ggg}s52iYKnP(K_pei@TyPgGCF}8iK>2DZrxP`Ea)u#oYed zH=$~6eeIw?WMN+D@ClQeNZ_l-dOq^*TA18&a1^22<~=c(T=05Er9p`Q_V*~r*W8IN z(M*oAhMJxIa#lg`>B>JG7D+D8>0?k7P=?9>OY?fW>*k58{cibt=C-q^VdmEZ=hqO! zEJXY+3_wishW~!=B#O!9_KjQsxzo8Tl(B$Y?crVMmy>dvrMg=Q=9i+oSs9my6I_xd zUM}O^UuFiU&QPFLz2{|v7f->JyBQUQ)%(3J9qtOPm}2AlOwGz@i6}yTwIj1jV0u2E zxAQ!Yx3^5YXnfV1oRqFdAr|tw*d|XM%W!vh>=X3Zo(N~=e~e;&*7GQ@Y?7w`|6=aI z|6=aWp3`Y85y$tuEd2cZTwJ)JVv)o`@?yKp&VU1Rm`<1p@q(PzE|mne@P zR%;>Hinh16jnOcI{Ze2)vNh8lOTSwewlg&=fO~u-3<(u`VD>)0te?yGT~@)Q%NUlG zm#4F~zz7qwd~>|)ehox;1_$ccE$>syK<4}&=C2#V-uMtK+?=c&cGSVerG>u?)2L#8 zrs0E^xyj=;50>yEPn*u~sj&t(2KE-#j?3P^f6wjK*O`=_{xdy&4qas{hY#S&HPnYF zxWQn2G@BY697G(2jPUs(CYG`x{3%B*_~saVG3gHctTTEe@UvyMAEun_>@vqu!Ez(A zC_oFvrbl!SfELPvKo0z@ztj&PI2O|Xa`|VzE6UAc70BzgyHO?&UCUC`TdyxJuFW|- z%74&qHCunVZ8B&-UyR2tUiWhM+BJA?bntu}Xu$(!a_%dt zji!4(GP?lQvIo=_926>-e6JV1*J){@mp<54npRd;vVfrS*_JvaqVQ z(+xn#&+83-6p^1nMT;seb3d8bEMN0YnnL%H!!9mfa@m+LBTjQaed6OOwnzW0IcYxH z3H`?4ey?>nvB?_7rr-3CV4hI0*4r?1oO1#=PQX8&vf)_k{gU6Nreww28O_v%s8 z-k*bCDvnJo|L@E|`|luz;Py>MiN^eRvF^)qf*jzmUJ)haMuv^#`EtKor8!czCmbG@ zfs5QFEna<@HR^rI?JYVn!Y?3jKV+-WPnFM8nV`oRnFX69MyoivdpZpx`In--6egBD zrB4uq+k0eh=sk$WZKqj@YZyIeHJW$%O?*;OF&fpJ3ZR-}>(n_#IDH0r{6F^IGpebs z3m4_pSNRkX0RaU8MT!Cf5<+h(NCzn)bfov*yQp*l0qIRTgl6a^0aS`~DWQ`{3qpX< zL(kda8|REO#`$;dz5nk113SsyYp%8CT5Hy4mZqjNN4_u4OMq)jWWIkvVUm8m*j$s6 z?7hlTq$Qh^zV5G@|J-(Pkj_qCU?itcs<#_Ep^ay8ayC#9XH7n=wCpoZeFxQgan&U6 zwsrVhGt>lX@?+&ro=IgrP0d$3U(d7U!(zoJqSZ)2+J6-%ww(WE8(HLhe!~U%zCJun zqsZzZ01%m2mbLU(PMrB>SQ^C1*b4#~B;@#?L}&!J%Rb^{e$mEhAmdlw*w~0yAg*hi z0`SKB1%%hHxrA(%XyHsqi{Z2KT$gsgy(B6OD-*NZJ>CzQ!cP8+e?x%`3GAK!D6wR= zQ%oFq)h8(a-Rigly7Ba%JVgN@=b{A$TR)Wv#P)LK)={(P3s{wx+jHUf4+;AzuOBco z*Xu``Gk+R=?42w4?qnvz_KA?Bq%xVF-R6F57wdFA5993;ZGDa+vPsm0j{M74uO1Kl z7#v%DQ`*u77iOPdEmxe04u9Q^Fou{&%e}Z`V53!raIbKEA}z3xJNw(#2xO1Lt^Y7L zZ*p;yiOR21kgKn+AF$(P&X+XdkN(}cE7Y4P81BY_S@Qr znR^GP`jI0UizM;q3i4 zW>ZJJ+VPlFp{h1+LLq+Rv{r`L-=E5#E9TSu<`bYycTY}lT+0{d!DS_-cfH%j5ye)a z?S9@n0Td#YhI;_M@pM2yL;GQO9M*5I?fl3h8{i>}S)Kv|!eF(N+VzbrZE6=y)TZ** zC{KIPid;`9VmwOm;q*enr{QLxZ&3U*#6=q=5Amo(X13Lb;Ltw55@bfTv@7w(w{IyP zC}+L4rZZoqghv-T$m!1MQ-fGz#8>mrrm8~@`U&*&->lA~fr1^wfONTTCHxTq$#C>ngwSj~bHy~9xLV`4 zn5ndwd=YQQ_(7FNui?9Mg_5qCJE6-31FkkUt7TPNp$SGNS|%^~hcr^^zPwDg46BdA zXXr04wS=E6E80;QP=lgkxLRNg2Ddi24QN?U=Kgd{e|HQ#k8Lcsf9S^jjfr9YH^#@C zH?v$hGVzH`pak_naa$m^Ix@0Z@4c7@R_NM}L@QdEV>lR-er1zlL{E; zi#vtRhN2;*#C==Nqz!(@_3IJgJvGMw*b{@Yd{;yYv-GS;TC7Xc8V}uYW4laJuUaxE zIlW<)MDYGE{+vMn*~{1G*FTaH|1Gk!wojn4tUWHg03E+i-rAPk@z)?6P+67dJ_I0i zTvxsVs6$l-VYgJ| zvH)c%*>zKbPMg}peq9HlXXFbl(CS$2q`H+8J5UY4%46_U-EMPJzsGY~@4$cc7|)+K z&btjK7Gz7ufh>-bxHJ?Kca8(SJ(qn>r4b?h-5YwIHGBI`Dyj2`w-+iJ?TNKt?BY2QXk}pF$oGwo-L5$rE08`u2?o1FZenf+>U98RzCrY?6(?g%bim+^ac$61Vg{G#GzALy40) z`&00!+{Lk?5#BLw_4>8+@p6(HPWY3xd#;a_Gt$cKl`)bwmx=N0JmQo^zqU^y{i;H1 zv{dl;_^9`+lj)jtkaG4wZpbT6)-Jb4DUdzQFZeAnW8c1nxYr9dJpqKxfK>;kiCAkv z1$9b3{hH>5yzFs5sq9VB^nN`(e$SBC$}|EbWbz1E!n9bOv*LHtRk52vtZI&2 zE}_xG*4@sConnVo(~u^gqa`Xdrln#1cF5giK^uU-{@i?3MNS4~*bBR#1;f7I=}8}blAa`PsoE!`rEJc^pBi* zAQh9Ij0sCAamudu9SQ-M88;`~&-h?GRgkyi@D>u_vd^QuGwegOE@3O%S@FUbk8I!$?&8B*gPvjUnP35|OkA^#X}b!zA$(l{Kj z(Ta%?d>r6|Uz0p;&Y*Q$?=(aZd`<&-nI@_N{(Q&gYz;9aCN%kL;@O9k?~p&O)q6(2 zn%Ewaq!vNZIC+KIjWg7bjy2OS`ORVnFAdl$ba-pJc|QJjrGCBjN_R@2^Zs_Ih754b zpLApEaUuh`W5tO$X|KiP)eunPpzvUNtN(d-^HbOB2r&g_n%~K(dYm`ZlJU_gY*W&rf`azunV;A(eLD@5FoKM{Pal zX9hD34Tkpt~WQAdC*)x!EjYm&rNo!nt{pw~` z>hf{Un%IH)v@soA@2B3Fxk*>%zb0BV4T*eZ&17q{*KbDH?SpCVN?~Q$>8@p?ls@ z6Izg4T1<=4!|vl#8Q7v46QWeH@wzsvf$n%F#H7{#IYap#Y>2&1PtM)IhJI3|J3U^A*@3$` zH1uvaeR;p#hcb$wFAw(DcMVp%qE2Kx^-YrK_JopCe>1eIjJ`_#)hR!7r(C7#SNcHi zRp<9a;N?L*ItRK|JN5yLwki zV1?3W?xD^uEj>kn-h0jK2Lj5xA8d1$}z1-@UR}T3b7os{3`%1d=_~PBgth!qOow+ka5!xx}83JJd-6 zH{7x!4GKARe>E)rXk}{?4b$x`(|el`srcxIGWIYdjiKHWa}C5_B-Wl%v6HI$uXjSY zR+{@-VqUcVS9p3^8k8$V804YenI)fZp7O~I;gD1p+4RG^`6FPD`H(TD%C9dhf6cUZ(AF@ev&r?!Yn{(!+rBQTtS4-W{fOOJ1H_;|o z4w(E{HYq!nb?>DVa!a7E?n0EN)5Vph(ehif(PUm%nk?o<+tqM>_4OL+WZy(}@deE~ zHG+$*d~F4?VYA=7<%KghCH<5k@=k#Zh&~on)bWpmohcQyK?$|!*C<1AdRlEaB?&A_ zjH2>Lo-sZRH6EC@U!=62N}v4FA=w9&9@X8uM@PqRLV@K0;9zTXhrv2V+&QojEobWHXez(Ks`HJUm*9ShH-<{OD#TEk!?B}bj8HAM!^DCus zu(6>B5(izm8gd^fc|Q@uAhqUwe?Uj9QQ6OHWp#?%hCw23nM;X!5Tl>KhCIn&LSp7OD+4RT6_ zWW;b2=}Fjd(1-3)WKC>FEk-U{WB*FYx0$OUdx!aTMwOnQx~`zS$Gi=wi_@Ybqo5GT z0b1i$_{~)f#gk~IH;Kk62j%wGRTVwE1GeTSpf;^ONhU2oo2L^ zrBgLEdANV>{M74aW5TNop@sT?Q7S6Y8CuFYODxf_7*0l|WMp$M@kq96y@uX0lQ%kk z^=0w;U{QibwfI8zi}xfTQ6q2FXaBs@=PE;Fu2PVVt zHt+U8(L-TD_lxf@+{*Quw(Cm&(UqVx|HNk1xptj<$VUKZyTj`Hi9eYDPL&q&1qp65 zpBXQSg|2fH!^X?gig>DYJdy(yt=od9KYqWf0?$$viK%Ugc7@@p(}ct~)9F5Byx8}M zn;3$0c?aLfC{tYEOC4T+mP_O*+EGogfhY-A(NRP7t)7Yp_e5`Muuy7vdC@9B)-^c; zin|bls*arTB~4TV13!DBXNF^v4M;G*saKj=$o_D9n&EyM(}2?a#50YlhXLqWvQo8W z5}`&F^JLa&b$M^?Y}e+Psh&@gG2QwG)+*_#g={ZW}` zsiP!()OQpwfF|GCTX~tKE<#^~-7LF5KAl~_wh_p4FjU)yf8fw)oM`|3KHAAAF8P%6 z$%duTJ;wv}?_mWHwJh8y=i%Qz>4Z9&ovmaMgSn5sKeC0AM;XJcditw9Iq4!H1)2L# zyhwnnrDDK9zAKB3@J}s1R&ll7xjOb*Gxcq5G(YOD)P$~J5u5kW`76#1ajlb$!fFz& z`e8;?WcHsxGHW=MVT~RC23LlA7~?gft(-J!zYfU0rchE*nQlEI z-IDQ90qd{yc@gS2qoDymUh(q7RO-`(D;=ngPt%`B_X4e-asSB1YGymG4)@-^EMib) z4v_f9;>qA8X!Cynjxf50h`Irs3;*l)gBCQGP_F|oHvqjBtPz&peEB8r%_S5*@c%dq z(Dvo^^$=*`zyH=p0A0DfN~M4{E-%^5N0<1!z>Pf>(AMSkCIINm<@IicB;;ORm#^%A z45f=^Zcu7JKd(`^B#=SLw;A|FN6|{}7U_3Jv8BKZjf^>U7wF=3Ae&;(xsvF2NfgLZ zJ|um!Qk?Ff@|1+<|8AyB(uhp<8Z8}lw%#vYmiV<>eRHZj>g6Z!_Y4XA7v-EbOPfbp zC2X)>aMa|>-W&!xYSVl1B6Q&35fZ@c*Y54%dt@DurI#047OR(#;^1fiI9E1oS6PY{ z3@Tw`>(%mHeePBd6L)E$Y3DyvDc&)cg+BjrBby!rrhX2BsBs_M^a`{L1l~qwPY#Un zyQdtzlR7Rzy&uFPZg}~uK3LF61)`eB$iSh%n>)T_Mc?fw!0e|J-#BLc!Ar6y%&AUn zWJYIvnHc9`mYD3g`Jr*Q{<2!aS8LL2lnVUJ;wuYp6;wf6>y`Of-M$=R>qrurFKTit z7)0#oOlXqT5%lvhjP;J6fkYV%SJy!Kc1bB+0iq-D z?UJ@}yDdDEgZ3>_zasUa+FqN04s9%l03esXO*ql3>gf81H0XcM5`gJXJX1$RLrdg2 z86EC9*ygWWaEWn=LFqTecpM)~IE{(BDNsD8yYb4i8Pi~rxe6_qVV<mBV*WM6gFT0YJdmjCf{)BJt(Ya&8~ zuNC=}d3lgB{&u!xu!VV?XVvu|6&^x*bfgD1JCgQ!CU0VY~|n+*7Q!C{gAS`Q(jMQ*UsQrl$gcA>bgMCqUq{#H^qNyOJjllHV^SVLW% zoTiYtxOgy-AqRhc>*x3H-O*zz{zGWpr=finE6W z54IfOQuwpSW?YL)_Ub1am`@QsnIFqe=y>pYwSiW)BRSbFh{JAg)?#R-e!CT_rr9Z0 zYQCh=C1eS3jVBcDnJtA=GEJa8PU}4wgT7!6v2oe>z(p-DrU_oddUrp=*{qzZBni`? zg(9TPz%BY3!+(zc~JCHGn_T8&g zG}@bUAZ}<}SgukGKM?;GGI*5=-Mo)(u6&la640vAc5#9OT!pkA3FU?Wz5uqqf3=|y z0G7xJI6k0?%c1jV#CQ!G`^;@IVm%K$+v_*O9xKFI7NJ$SrpZ1c-AHR{{0eFA%}meN zp@s$qIl(7u2?+_yiWU}2v>KAnMMaPLO)qLZ4IpZ{Wk5#zd;lK#__2kJKe>+AQuhH5 zw0ipZq}*oMv(mJ2eW!<&xY|zanqHuW|6Pg?X}38YdgJJ6*})E^wax>Dun)K9(DQz~ zRU9fT0cWSfGJ7#VE*nWy2VG^FtUB4U zz?>Wp6Z0GdxWaBUuhhF&)$WZ~=LYb}2nL=mI`yb|A`{)dG8Elfs>@Zy~YKz2|yBK#MtRMj=z?j7UY((vnXjejfR_ zX8$7TT+IVpZH$B!6ujyzKmLxcvQW5cGd zO%UI3OOLhav*}=m1B8Q|wM{8ngYl4)JqefA;ENyfyJ$@tb8a*P_Ket2$ zHp(1h4=9}fGKyw=5lnfB-0kfmEmGI8$CFpz&1m=K0oYHufxEq2Ild%(LSMiy)_zdw z5(O9ldp`;aL8Sb4m&=EVf+dt|-Nmi=HXIIf6c=y0hR502-E6=3bsjHWgHi!h*rL(7 zIhZ{+Iw~r%ET*!m$jR9w=wx}7yt4=HA;uaUT!Xu4#eyfouENvOlYT`J^5Nl?PEN5v zeQ@IXgo_JO2FIvc_(Flva7GTID)h*l8~(9)E6$L?Wv0&VqJA=JcrxO6A>pSkW9xNc zv!(eCHZFK6A%*2$49M+`>7+noeRC1oi$#WAS~xXM-kh@No675r0l6RF6Gomicm|z! z-f!B&4z{vWU7qB8|Bx1$6TloAQU~L}!rB?jy(nyKBmiD2z_iMPG-TN<^X-q@_x6cX zve0F^?Q=Mo9)hb0NZCSP)O1=@BQLBKde-!k+8KrejS(208e`ZeMjo4G;h_#Ta%fe1ds4->GF@R*z+bG+_gvXN;T($Q_R2J`5e z4Bow813w;3)-CiZ&aZ0nt5Qzf?vfVdS9kjH!(S@sJdCdd9eY_dpz01uirWKDA>2d+ zkliXQobr^p8m0g`SaA_cz4=ohb@&z~W!y^8Y1T0k$r_*uc3JP7yk_=xrFFtyNI*a! zF^U0wlsqinawTTn$a5_JILIbI!2|wp6+!`gpXEWHRnN@oKu~pKh-x_aKQU!SG^6Q} z-89j~a-Qk|4&t|fRf6)e=5;NDCs2cEcl=nXK4E9E=&v;YVqe8xjY;A|#F@t2or#SI zCx2bD^Pb7&e1F?+6&Wq!kuY-{?)mXjJgf=?I^hbw$IKjbIK5h=Ck}-~7Q@)kF%(RO zE=@a!ia3K=?6YQ@-u&i+;r8Q&4n=P~d4#ZYi%c$6Tu|^sx})yy394X>X1e3CY>kD* z$QFU%f#ZtHFU8JIZTJvNY`~J$v?n?e!gN2-X8hDBy4FBviukR+s*+f z8p0a%B}z@`O%$Riyw>|?z9GHo-iwBI5u5MP2ULM)3nlbcnK_WY`sh(6N&9vE1KqXkV#fCGOk6eE?XV5)I}MY;VZg-@lRU^|(r( z?wSTOlvJj0Mf6_!`6<=K#nNNAz8TjI5A^|+jV?TT&-?CdG1!5{-&I9}ux1moSN=ea1+6CEmXrpYIO{ zlBC^tJSon>e-`?su9LkJ-2?JeVxLu}=DX3-DeYG4b!Bpz<9Q(EjLh>R@P~qQt{iyIxNL1Fws>|LN77xclYF#{dMI` z<48Tdxs-x00^RF!o$S*vs1EO%)Wq+BEagPm3<^Z^=v1`f`$$_y8qUWSA(vZLeGG{R zHlqrExyl;8zK zM*>J<@jHIXJ=cH!zy#qJRcsPGJbHWLCTreRAIr~DW!39p6V>_V&M&WJF;LgS=G|4E zeS#jUnc9h>q}!GHyM(cR7uDpmM4)jeP!2fqnap4Cx2>5~}V3wo~1 z^PqxpS=B(em7OjGXp`&>EuiGRD>sE4B|BOf*6)O=3VV2S7iEY-vYUjBSBB}3Uox_% z04wFbCRI6Cz0llK;EvmS(J{b6N1A>Nd&Rwqb4!)&7=h)Zctqx00$y zYN+ZZumioz@BwoqlWqlG0k;B7|C^HCb_X~n(&%*R?FpZNEXIEDWhJ!^1_fXV*Qz|7 z&un_Ui8AMG#awDCgG;$mWCA>279|gVlDhk@{3@XHeia@NP4I;7Vw@qhd6!Dld*fyz zptF88s+KwV{6Av>{MfwR%5U>4J6H=R4aE5?kqBa-AU&VL-mC|CKY<=a_i;6OFFD!!R)z_oS8tR1WhwaYYUxOjR zsd52DKIQ5>##d@D1F{~9k>($-Fj z)Y}P3Rfh~3AIGnR1@z)Pf*+7*<0AuTd#emp*PkvFGXO9H03SfM$GlWY!4tvcg}f1d z8%c;y@lk<~?oqAKAD&ef(hxQEeM}7i{$|Z~dR8Ednk(z|<*Hh}O@`CkwyHpbgM;TE zUeR51FJ@!2XIoU2>(tQX5-F)ylRF_Nw-(KTq4sBHXALtM<&hb+%fo$p5#ixL856`V z?ow%Dpd@yXkp+q&KyFwSV(UB3ZjU;1K=k9gRn?4TT%DWFTcF6q%%Df-MlUb4J4^qb zpG@!#o8cU0Mn)}KYSW`y-j}K`krekX=UMdwNskUAE?~|HOEPKv%u4oy7wgp*>j;BZ zw}mT8-btVJ2BSXNm~n1R^$yjvbvxiV&A8c=?4c~{QITm(%*?%AN!P-xD*fm{+V^u= z3^N91CuVpT)e7tDr?YO~x|LQswgWj!97WhojMFBBhCZUUbF$<1#ElQcGyz4!FTOw0 z>nyJ1MiIbIBF{ z7fa64HfKRb773D(l$`UG-ji+r*p{e_EPRcpsHR-a7Na)h4$SoX0E40BfG&W`DtVA# zQ6cw#a1$IBX+YOT=WMM*XJ1vu8gAXCoL*0nIi)wX7^Ere6w0eD?s3{PTcsi?1rl+HmY&o35p{dhk<$ z{odoNAC_CE)>?#pc*72Z3j^K^JQVeSaa(J#tV4nWPqdnp3VyaWUwTS^=Jf?uvZzVk}|5oiB_juRulWUS6OROeQAbG zMRyt9WCnKlf6_^2TC)F;tUwtaW)9=VbH+iJ8&%B`s~G7kzzVaO2ZTFVmsuh&Kah=O z19oVAdtcupcPi3P$y^tJUcXJT&-EExxm@aGe}0f`LvnJeK^urN25K;aZ;AJf$+uul zYKj302?NqY9oIH{o}uo!b*OHpZC9beR2)qTf<)-@C$4(hIG@r2Tr+e6`@AO{{-^qTz=V zflu$R+Z}llxM1SXVdbKwL(xjPCrx+%B(LKE#An-KIix6;2?xHjtIGLjxEJ5pDY_M( z)+BmkO;tV%hHiCK;*{`z8=_$KS#L3VIZK!W^~>OjO2m;F(&OK#?)$HF<4bY0*vGPX=MNK#PHeYX?d z=!4%_TkExdvOdmcLzAh^#fFZ0jXfQ{ee%7Q+j0L3gm2yXUi49slh-vV*jY1EpAdFc_=UP(nHZP-QHOM8 zi2-g^#kXz`(Z{0GS_R_ck4G-$Wj4~3#v+tW*e37H)vUj^L+PPzJ*BYg&nF=%o+ihVO2xgeq(bxISC7)xxoSbbB7I}<_%Wjj)ul&h)Yl0rr5~D zgjmR7QjbZH(`-*fK+e(k_!=AOkmGtMHb9lI%Zpb5N z?)LxJ)~%QOA2Mq$ZI{qzZA-;pr~M9Mr^Az2vs~{V*n8l!#(V?D^thRte+f)O8y26js(KTj^5F zx8t_h^3;oqdFElsy@Al5xvdYuT^&*q)+82?eGeQ7$vc)6@e&n%|togYabeipa| zwK9}oQGFm${IhAA$t+*4B{i^+@O=IL76CJTF=QwKnhDO|TTYa4zT@#4e#!o-u_Qit z3})yy6S|ijd}QAW>V08rxOb8>8NpX=JR~?0O!1S!MOUGqKbpXEAgO~T2L9A z2#A&e_l0hym+QtksrLi!13$05oBpN$XlCriptsGKrZ`;@Q-L_swoQCCWndcBqLGF3 zPIQ;E^JkCgDfuuVHPirt0E1!orj1-&Tr@O>;RQV*t(&n`Ia?emDUsSB3N5qUmvWrh zk`6QsCTSecxLVCixsxwdR9AJ9MCjlLcHP58qweY))s=Wy&_xti!#6j9>EgWFHf@$?*uI%~I1cDS;_Te7O3!V0t5;$>pvM zw~5-g98ZC)4LY5{jZOXO1c}qhuz>G|w?sE#Fbu6E?6vWsfI{~ftW`Lh(tY&KJYu7G z3D3G0r1@v8ix!!l(R31?5=Oz0L1AEkeTF=Z^V^%j%D@VCM@kI<6c0@}0Cor~KOH@Rxi~7Xqq$Q$KyHiUxQ_JNO0{P%l48IU9fMA}0f%z! zF(1sWa$6pH&A*oiVEg*}l6g|*k(N98IE4uE;M0=^KsRM@o0pn@k=EwZwT>#+MI-)( zx1qPr$#37jwVKL@BT0*=|owQ(OCX_6- z1R~iD&C)WIEAlcvGK#Ur99dFJX|Hk!-9U&^vnoE&D+Ef*i&fGl)Ww!5nlQ<+f3C<7 zxnV(>7%TWP<|+`Ffrz#qQu-xvVjN) zQB0?N20gWD$`Zt}D9`HEr8YKA+Jgm#65SD&RFt{y_oXftb~@gRn?mbpa`hK%_-SII zbe%hOQhuhFLAlttIoYsRXqC9Pb*`E|=}KZ}+~$fT)Ow(x|4WYMXiNa{@5VE zo^RW1?ckPS+LylNN5+1>PGEPsk-8gtMeJKN6D$)b=O8E-Cm&@IVnIvsCeEn}R^eWDU72xAPiP{i+5TpqG z_xGO_#!FxHpBU1M%5&2pVg7)2nd%@wri_R#h@%G@VYrlMn5%hzyxUumUp`C4H+|Cx z664#N;`;`s-Kp>e0!ZFbcv)STI+4l7K2KPOJ=TWDWa}C2U$?1L+41BXNEJ(rZkw|r zhP=={ONu6W#EtF&(is^8ca-9|oTa&dw(s8JVQCRrCp27e38mG0U#Weix?drnjeF*p zO%+gldb`R-VO7*B)}FczMhMMfy7LrF#I6Jgg`$ZGEG)}Ef}@N?T1c@jW+J`K$7;9I7&1YIdy-k zc?a(d>6ixVUz=t}`6UHgwj}88gM&R0&NaTN)^7-eYc&n&3aMf1UCR+SCQT~3_5|nC zda9jI#9z=t4`TE6NHf524F?b)!1FDR3Al4jCTS4@MbX@aPhq91I%;y2fxQCMPsd-X zm3nbcI6)e{(H`pj>jqvcPy_poxxtJ+vFeq?!F|b|ZO-wp$+^yp%P(>FdBmUA8MV0^ z>Bk!CXKGh!h`2c>9prc6c1)L{i7Oh-fs!lr0mUYA<3sZ?eai&z8W6?7C#!6eUWsb} zH+{9<4teqqnf55ZxW;d~M%IH=TTMxRg8^kkWST8+Xp}e*WY{MG)}(dAX3M3Tv!_nU zw*}C$IJmupl}3s0DM=DIe(y}-Ug}8gq*I+Ds_ParB@K0bfsL?(E)I~)-j%}A#rA;c zU!acJt)IUuTKxOV)AS!Q%4-4v{~aB&T1iDpU=;f`eRWkeVKF-+ zU0pHdCM$orAde0-JtMv4$3vCXOkX-1ZZ;SVmu5_=&I94O0K75faV=}eew*#9FXR4E z-Hb4~r~P8Jibkk>I|FoeIxJ;!jzgh%rXMUerm6~Ve!qLOt%s%f#6-j2 z@M~uGQZTeOG%6L0cW+}0XiLP&uU{X~@mildFgoxtDE*MsVXldnSADPd3Y?o*0b*71 zZ&}&C+1S=3Y?>z_Q$V+$cwJQoMHi5Z^@+7gc0;YEU8fOtp8F<#59f#9l+rKL7}A8) z`i{o4v&PIHC3k1Up*S>3YP5!vhcw!Hd7!+^Oia`5#BcEy9xu8cLQqhhc=~o>h{b=G zGy^4|$`5E+<$d__zT_GOeU`YFy-=p63)XWneW}c$AM>jhk{>TkwTf znJ@bq@|%2ba~mCMr^>EkVXzVNd?MY0zMP(diifG};)^mj1TapWjZCE%06C_CLP={# z?0W&dX7}Tf;rPJA={)mo!^7ZdD=vclDE@e$ofJLNAqJ@C9lT(>*R z01xd#CaZv2`)QUMrPURQj*RrwFX3vKqeE8DpoWt6y+~+mU6m+CH|uEEuM-;-5)$HV z`#UCORXyzVD{2jL{2vtvGzn|abyAW35o~3aKL(2zO%k{TO3#Yo;XT%6ECl@C=io^r zkBzA)RZfQRg^#FPF(0U*828Ow;s$1_10Hg|noA|mmdGdbs9Cm}Cet3amWet3P&ItG zC3@gj!Km_w!pBau`4A?{eFt0{g}a{d(d=IEsbM^eLf?K9qQ$79SX)QWOnUXgZk7@bQz3V}aTxBu`ZPj>HZ`&ybkvf!zT)+?)G74I+ z$9{)4!MCPtJ*@h3dUS0KF%ic1rkFAT*r4Be0EQB>PfYRAIz^5ww!MYZ$z`34!eDR^ zF(Blt+XZUUgoTNvr5K62X7(V{XGrC*d{>r!g}Hhc5NN@<4!Ht4aMS83gt77IQYUoC z$?Y-%jt{FrUhe}oEtp(PmM^91Rq7k;x7l^063gIWiQi@A0^g$92fBA^huh!aBgPRv zc~LOEys9_y6`~-hF|WGZm6)q=Js~j#In|pM^weO$ZMA1@y=d;m77AreOXfsf$I2TO zhAO)6m?YW@daya4(9OWMJnn^?p3Y1H=GTy&Wo?Us>2)pmqJ@AL+k%u22K6S@IVWUL ztr5My6KXMp;J=%Cjm@O~joO;LxffiF`bHNyy@EZQFf2`cX(9QQCX+&qy*XO%Dm8x! zbcbzinN##&s0ZkwO_j}yJ|EW6 z-?|+0h);KwZOwzlzztL1t!8nV zd~-Mj^54+TUL?1)L==aWL3BNm7P9lgEa)`@2j=5)FFpCfV8!T1kYFn`{V?+=nmP=F3FM99{U>b$VAx7K>{RQYDpO_}7$`+jcl!kL*>qp`cvM4V~J z?liTFzbvZLugqhSEw$x*XvGH&N0jkxY)T)W;Hp`r-Bkxxex4*d=wceJkLM@al*C#G?KNXM>>mEbPG4cAT-xrr&80KM-2wruim>7?cTp)@)^BSljq#zbvVJ z7G7CU@R-Lg>1}Tf3c%rYa;WH!zr7U_0z}BPNk51%x$5S{1Ff1ucpoDU8yxQv4zp&? zXHzAlWwx}P$|s$SS~s>XV|2JoF%6r07QmVTiOh#LC`qSD-rm7YR^-m2FOK1W`_n>u z@E+I;K8~6!nM8Zw0s>51oiEOc%0-wZ1I42Q*%P%(Ud#iEqj0scbYoGlV=1vY)s* z)md+zc=e|a**TTA9KO7fNdXN1pP)>Uv=$6vf2PVu2K5l>d{A?Mn`o$OIP~_}3R*(F zBEM}dv^CuKw0ykEP+x!^62@@*a03cCZ=8!1v(qCJXZhw2UV<5Yy%(J=HoEa~Rjn zHpn(QIZ4>sVtzdO&G`V<({>))esLVa!b0m(j70^`BK!i**2_w6du7ol4-ILz2X8f>{o3B7s6L)K zcQ`q1@dW)eWAWJf<49%VzquGsrZ_Z;G^@WU4;!$~9;= zJ6Hl7@bSd$y;>V`HnmimmeF(m`AthNVGwPMi$UNLv(Mafalr zozaxDfB-TT9!~EbY$q%cs*C~#vr@E6TGb5c=L0=ld?vSlzb&rao3_dAkGqX-^SyX% zs7oLmZu$`;Bhx!i4}sJ;AN+OL&#N|SbK#UY-D;ECJpgSiXU(A98L~FBmq??;fywv5 z1c~^02R-)lDDz?xkw%yu7`CD%gIiW}I86P3YHmO_UAl&Z_|+T2j(V+DW8Fvz%-_q2 zF2eW$ac9zd{JG513*6W#&IL!;{lD;2JQChqU)I`#1)$REg!tPQ#JMuG8f_O2Y))Iq zBNXJaXsZ{I&0#UlER28W4;QPKJo{Tv9aOrHw9Ib|VWVGy^>}82 zh;27MoL0aG9c`g=K<5dCgBQfZtuwK>^_iRK5iHXV4tk&IlvR;bMo&PM!U%~&%JanpyJ+wM(BOq8?Y>I6XzEL8+E}kBT zCK}E+x?r4dt{=^Ld=gwcKDG@Rew8`y$-6gedfsBz!zVExvh@qyI>{eA;U{KsM#V+{ zBa=zJftiVMD>Exc#j3CjZYC_=Tx(X#Q3Is%N4qN8p~mYxERPzTn9NtarRkDPjI(mn zm9?Jaad!mYM}RZ7;WMhipjq&5)!t?=Oo zPJXVq-7XF3hH2JPwYIN8_iujuncNe<(e8in1E{VpXSwquS%kRVI2^`~|TB*CZo^Zex#KRp{(C`o*v3gg~s&5Vp1xAc@w5D4%Q z8B|p``)6zp;qiBZ!)Xi%c-udVrU?l4-=9WGcyBl+&c2HN#IZKMzBsT^_xzt%p{`BX z&V--2rp>N|At!2mBc-`nAfU$8t48QI+*BwT<{ z7|fz4F4*mSH@2LptmW(ISW#7lZPwl-L=b6iUmt!j?L#=}8WyjfqWOE@6gD^hd&1w` z*7k|KFub=bf$o00rFC)fVgq_~UuXF2Z%X%TEqmNVIP48XFnz8zsQ>pb*_j5XF7?r@ zC!B))dm|&?+wQp-=tZ9Ih*X0&9J6viv5I^CRiES%;WPfE?JGoMXE&Y984}L(GOuq; zfiT~C;bt=lF}xzRM6obP>BJRJNl8#s8|DzSUX61CjnPUlH3%r3H&kA{`8_&U=h1Vt z%{=iV>>Br_H7e)UB4Aqo<3D%ysPXED=L4#iNu>c){sE^E28Y6pEE4;t#Na&ua)C8u zPCgO-q(Ct;Hlf3uH|h6-iqbhFZDZzrwYbBWhFCh)`2ou(?9j+l zXF_(6#%7ZbZl?CM!DXgJif(n#3(`m)0qV|obW}`mtJ(gf^+Ve2=GACTUO9U&a1MMk z`t9$%G@9&B0B%b=5C%xO5rVB{YGhkVVz<0;-QWli+v^?&4x6C_9zEk{v)Pa)SI4c|M`>n(6kPr-RZo^rL;ER%^W3l*!^( zyLFRZ6%+u770BR9hsncd$$8NML9a5nbEa!o$D2I>xGT$&4{ZZlj_)EPKhe0s=I?8T zbKBdCKq1E=M}5aoQ@`mqvcGl#QlVmDlt{dK!o3=y@|KwsS5e95;+A%q&kE}AXQYXH zZZ}82T`A6hWqb~Cugm>3pM(fAue(L0;dBj~ z>9e(oX)S3B#;y1Ln3&n_QjuaaS8O)^)LU7$dbi6kic7FzC*nN>)h62^bN{|J=g$<5 zijTZMd6%mzD=Tld;MN9g*agqeE`UjYXmE6ZMh0#EL2qU@g8!yLObySwIs}q5zTLPC z2WYz;35X_vC4L6;^z67Bc88s&eWv}YLlef2)WXAfl`}n07S743SfwT1cY*8*=Evs+ z2_r8!Udv_e#6HiNaoZB@vkP{pv2$qHVtwFWb|4FxFUD?ySp)Yqjnxdt#-4pI;!x@- zDPFC=sobzBYnCi-u{+NcIsI8UfPFsh@fH(KCjRH$n(LYmSWOn-l%%E>KiEH z)oE@S>?ZLNVP8Y|KteTPXmWj5n~6nC9}O(tt}S65s?bm;;DOOQ|l zOM>*GRB4IQLkB4W0i+mea9xUoj*${6S!r321OaK%b`g+-k_DtG2?{}~AVdfd3^#Cp z_iwoGxA)ULznL@7%sg}E97hG|j9C;0dFUx}JZ7jROBe7%ue@8cO;6H^UmWL78Yr53 zrh8s5?51v(dHSBRl?M7AZS=jH zbY*h77{t)h(2uhs|M2^n)8U?8O7DnWUlPinRI(M!R_mi94JJ7p1b{kN!#|TVK_w_= z-bL7C6O+7#8D$WNSp6-#y6Z9v?+g|qC&o?hrKGg`f(gzXUeRRxl#Coy5%Ua(-H<5J zid!(5Slih5f+^>LHy{gKA76pkxAXmR0GlVp&3ABYc`UQLK6rM0zJ9Tc(&DAyd z!zxS#6_g2o;m*Z(K}m*v;WU;*}SoagQ8IF@z)Qx>`JFY-@ku{pgnQ@(GT9> zY~3f{$XoNuk$wAm9-spCG+mW~e}W`6DM{nxmtv#6m#J_c$AZsbQq*0>X7{5+j%Ve& z&4Gc*1FSu`<%Ui{^?*A|@|!uTJiiea6jbC{;%ejnmr?lUSsCWpYvtYlG&FrtEi*v= zi^o79kzHpi2y&PFeTq{bo7rvKBQL8HnqQ{+#s;dDS$>)8>cH1K&B&`+bBu&6R6(Hr zC*y6w^GS~stTxg&68K5#O&2d;tVL3$bp^HKIv-50x%sZ#1mulxK<=fUJPG{4^L!tt zLf%~w@kwvXknyQ=+*)6Qw%Gn}nK!?!28-iy4#jEdY3K-g!g;xp-oo@ma{Io0%k=Nj zypxF~-IJ{`0Z+xgSdL<4papH9CVNs^Z2{)g<8B7?mABQbl2wz;M3XtRm5EAJ6N2sL zShqwYYU=A_mSH&VK#o6*k95VCRqV?1@8^3oH4xxE%V&~M*|s+BKX(Q~FV>J>AD)fB zJPd{XG#EPPg0qkdsi4}lg?8NxrF|bB6OhA;U^r|j*0a_gCfgGpAY349$FGGvJfi9{ zZGdnA;>$vQ0}ENlrM)*}G9eNTS~f7))*Q>);T-E4Uk1T#OY)vRBLR3f>k{)2#Qpe< zXB2bREDes<(i8&nHAH*?r@bnG)w^x3thB{4J}-=ub#ChOYt(@iUv>mePX!&P=(>;9 zn115s8#8K+8|XfcFgJF;k3{Yrige*eqhzeRjIf#OFcBhGo^qi*eArm2|g1%@ku(XQ`s&(-r(3-1YYr+6eD zEUn$IB20{t2IG$zNZ%ma17&GoIC59a3XzfJJ(fyLK!T*ZmtA2<`(}y>p1s5F;P@Ju z9S8=7;5TVsXcKcWJ<)&X3b}fSN=X%3#w&wW=$d`dZjtqGlS+jxlb!n*%|znF_KbdG-L0VmWJX#joJkda-u{)StgDZ{eTN|DaKC3|>%qiEf3z!d*$+sGN z6sg@o@TuoXlu^p)FlCevzDW$I;xZf1&5YZ-F;rzWHsoZOQ?)dVe)A>Z!}#_ROeH$Y zB)&t%Gw83x)#mAkDjXS9;yMbWpU#=cp!AvCF{b34;e(z1`(ejxJKT}G)=`IVa@Yq) z?M)n&SWTV$YzptiOBwfM_Li)if@{}lF^jwym#*+#@@?O5JCma^WR;jb7OJw8LUPuj z#x9MwJS$Ax8BmQ<3~y#I9yMSouXSd#TT2na^)`#kl#+r%tKGgY{&Q{7Nat)T{} z#gCn$H}$pjRnFpP1JKAG-lb@d?%aMv!xSzWxklf?V&vXKO9gbB!G@AqqhL{C>gS?ukdV-YASRINhzs z=p}5X6tbz29GKerI667KnY?~X)4r;8-zz9cvB28752^?coRhwCadW0&k=7Nzp}s39 z48*&X07|VEhc+KMnD~~s8QK~lZYh8bNWbKb7`#EC|LUDOpPHZBgfCR=19Gx0I@51_ z7(frbE9m2GWuhDr@Ah-eMONfu@Q#2&!7KSGx@3T~>p|uR_SDOJ{&>||0Dy2N&~qd# z@qkAH^);ubtC((SY=DT|}dmP4WfD|1t5|K|460=;`fW>F|f%K72mL2Bs}AunDl$z@(C7 z>zh!H9`ql=CLpDV;|M+lAfZoQIAqZAM^idRu<1&Jx+{&jRQJ1pM&vwL6yVc{@FA;Y zL%R7i$N@!j%wh`DepACM+Q0=JUS5!@?9Hz>yiwVtfbsU1u7dNy0WNAw-NN|I=Y0Lj z2BhDb_(oCUqr)JK|L2Vk0pQ*CdGwN=0^jV09Dt%}oUX3a(0_rq!|!X6sVK+Vh@2~1 zR}wYZTkj-V-McmSJ-2noDV&dW`1F6_2y(K@FGR!lyFr-XV zfMtzaFBSl7rI>|Yb11SaAjq2g^!%(EuQ_z#ReL}X08HMJcxvH{ z@E4XgHi!r)lERg|6ldKU^pOKBMrw35LVi#PtAfZ~_;;1KKJYv(6K_?=NQNo{qq&s) z7q-4o?e5&+^Yp0wnJc?g4{ON*8|E1ZXlo6CzO>~N2lkT*HLZJx%ZtxxijRg%8X_e(Qz-ceF6xCa1l)CCx{Pb!2AE=bHXZ2V!1$_h-`ldk~TZS3GiTZgKXHYMGi45}TkP;aO3 z1`b(w%sH6MlH<-5#?=;yYL>O`wfIzk$nxVx26ui`nC5F+srE?&1-~7w=F}0Vl8?&m z^71}0X>2d;vS0Sl7)SrKeDPHP7_bQ`ARhNdT;|Ke@4MV_EKJj)CP%tfMfQxCdRbMN zp3|*qVpTJ;fD!8?jyMRr-jAO=_tEv{a7H85y$fH6dQ>KsJ-Q)#42wQo%DC{_J;`Ca zLTF1MAVBP3B7$1jwC)Ce6%F*$(nj|;k+bZOA=6hBrN&&>PYFN2n?FWz=*IKh^Qovd zkI+GMT6#IeQ~1a2K0AA&1qKHqTnN_e>$z>|xz&w{7 zRha0p9e-2jls1x3k$WOx;MX>W1NX>hHO1T&wM>3`Ujsz+de!@4$CxOzFE5g!X42Q` zVUo$S+30c0lw0@6UsXg-QI3v5R2lC1KFZIoxOFivQV5bEUwn3U)t~u6c-WVkR;HV? zfY1%=v_&^rweAC!)1p(Fs^;eyR*b70N~~!Z;i$8yi<;WybODRd^i~kbhg`c>*9f7H zw=Oy&lv~dnEW68aEq{j;UR{3hM_h2uz@`!5pxd(h@dAXWJ4Eh9K7(-Q6nNc$6X1PZ zFXUk&OrQ?U2v*A5HnPottLDZ&VYhE9*tqAEL-RW?Z#ajN5@R%bFB@;>@z3^A-XEpu zp^{jRp(+(s;|FevnDuDyc-pgf2VmnyM0#v%(^j8eRd_8@uaB!IM)C4&Iph{DYctf4 zti+mT$3HW{vq&<7v6!{tEAd+9>C~<(^L_8)xvXHT(UJO3PM7V84$IqP(P^hdu!I_3 ztldexo~p4Wbd%G__cd;u2!ZIS8KwIx@&f%;!h6Mu77DJ4KpH% z?n%BBPu;h*FF#Y`YHV|CCF(1obFU>_de8;@Rodafex!UdXjxB{ zx1{rdxa+a=cA8wHT$)tJiEv~Oy|&=D^+%8ODPg)^lr19gzvjIbPh&YMj)ioR=@zk( z6?%pu$mi~hF4GmqvQq++;qmmKD9^(2k~VV+VW`xZ%`dFDA;ua}8a-6jtZ$}yv9;>M zJKq&jgWX~FB3s2QPPLGO1Kmppr@8*L9r4TmP5l4IAIkBMQAv5l#$~rphu^{!YC<&f G`r|(j?Wkd-mAK&@7=kn>S%qZfgL)VwY&OGJ z(uF^H*3;wYBV2rsCb_xS5+!>)xwyGk)ezX7_Z%3=Q{-tOf$AAs;N>UY)ljPJFvttw zb@E%!3PRvtxYN%FFX7-2K76$YPWTk??+0Cadyl6!t99g#QAtS!L`Pn@4lFGd7bBpf zz`>PACo*a&E9(ltLQM}Bn=W_gQw5@jr4}!;{B91R186EEcDR&w_nXlBq`MSm5 zp%`J(AFox%0sXC!5wH@!E;ghJQG;-qgq@mM_O+n@|!-m88W3Yi~E0k@U0*odKunbzNS z^}7Ri_d8!>$7|o)IY^r%i**(FxT?58CQqlegS?TvLVDr3Xdv-;KH4*k$ybK8x5y+5>(eFm;Nv2^s z;Ds#$vj(1YWBXqvx>K!9HqGvF72-0Zy$W&L zGvc?lZk$xSGSpAKyB`f|A4RsSOTu{d9BWyEKF4A|Z`WmCT?B!~CV;Gi2ELw;fMoW< zoO$H+Q1r#55NkLUk2M!w^DS-4#~-b|r?M_p_wK-~T2$3#HlN&?Iq*Y`~Z%UWCky1b^f zO!sVJ0jR>RE_nUbbMH{j2_k;tW>eeQ&yz-xBkR>VE>V)+3Oz6IMDO(E?#;ATTO{VM z>^~P2tQuM>E+^*dT{Xh^OF#%I3`cX5C|4;j1`+YY!8+#vpo zG2m04VvrL7W%UhN$K%0R6k7*HXCCN1HmmT}^vh3mJ-{2l9~Phf)grnj45?$6mJIH8 zG;o=9y^oICdm;!iFfs103WP_%w9H9_q@<)IB=QJsX+sbQL?7&yJ>T@Pqoc!dsw5;T z3KbEtLc0|BL*3y*g(f5T2OmjPo#$jx{S{BoW|QZ6>UDzJb2v~O9eMh9+64OS;^Me` z-hm_*83mcResTn|7N0xYQd?9~K@%<43+&s+g?&8zMf6LSFRXB+mr-%mkM`klI>R%b z*2Dw`BWsuHgAYw#rCcG7*pOt!iWE`?ShzHfPAf^5s&ikyh`g5o+6g@iivlfHB*>XB zV{&rxa%X4bY3(@}h>D6t5iqgP8gcvLlU8aa?(l)P+fDaCk_K3tDrw-(j}(zi z1_LvRlG@CS`R@Ma(AM=vfIcorx~j4g1Z}-L+4s%RhtxffQGA2dS3*&{j6N?AyT_?0 zChj3X5@-3TceURHJwh(RiRttAA53Z7c?ITZBH=xseQkwTPw5h*14WdSl`*iXguM=K zbw?niGpls1<=4c-#4R4D)TtXBoZQgD7GpM@j*iYL6R?zw9HiM}JbH2t3T=CX9l=0H zPcm{y$ZdDU9U34KJl;@_{p6>P+0JKYl%yt)acYqdPEG@p*3%LaL3cheu`5P3+AS{g zQ{c9HGvAo1vT}Pp!{ywFCv(C2XjtNb(*9q>zQ5l+BD|u}!7rWIc3~W#{|?lp?Ttkw zgBB|_PtTx%^;Yk7N8{Bv5FB~M)uGVU8%jz_K|xV4_d<&ssH{7(fQQWF$*I?whBJgO zbz@?B+Mw03yRG{hce;?z+_rs*Zi6GF(M2+nNb5)A?SvI4naRhG--?QUwTWYc=@}Vs zcR&VAHsWwci7DIKJOi zsnB|uP3yBE-)0{oaivgNaXmfA%-2SLQ>_eNJM`go$J3J&1hTjjFKDJ985vn;XD3Do zB?}99`TovB7x%K@>WX&M9|0rG_4h^`^qzH%C!r#hyp^ zVCx}>J;i2VL(Y68BPuEk+TaunInew;54U8?sS==qHCu59S%qfp?d+U^47F-2Pdptf z`KYMi(OZ}F9d-3o866gDDtv}3@$h(HL)=Yn`>JAeE32#ge0&?DbMUW0ZPC9KL%+*X=k%_ooF(?I zvZcISi#dso{tX{)FQOzLtVfB1DKa+xR_yz?nHfChq~fgAIx4s;^i=W=twbV0*!fkI zKJ+alr#-vHV0z!42L^-5QjZHn4vmbk^o@`{jFJj)gCabygXCt-GwRnT zBW~LU>-o4dNpqJP4(0@Hk>0)mxy^Khs1{5{O4KW}@t7bZrOk`Vbo4METlGr23$+ou zHq{ywe^tJ6ON*Edr)nN-i&QVpShjy+u=LQ9;dUKoU%!ySzj(e`XCUGqinelkT~ut1 zQ-lNOt|JFQx@2ozX2ayBPmpHb48Egj;n#3EKUY)Z)@8oDMY)tb@4_9s_s|HM-`P(>j{b9F8yJ^c>Y@6N>D z!rnXQ-J6B$16P#GezLZktOut>ZY5}&Ut2>%O6Ph4m?^&Bge&b%9|F1e-A{j&lDvYH z3B_zO0y`}d3{--Kk$2v&ym??`f0$seuoxC)xeuB6Ww9QAz5k%k(>ia%IF%8#Y-p$h z^$12&HZW*=^S}{zGH*u6>v?9KnhznVn}jJFL_S&SHQqt+TbTKwNE;d&CZ8$5d;hYy zxcIcI|IlrJK~9N9EpL9m(eZpIQ7#P=Q!R=_u&mMJEeFRLWPg4nLl`=lm!hJyT5;!k z*0$-xKhHBY)VlT#<}PvK-VF7=t$2uh`7lwbAbev|wmk`f>^t*eD}`p=uhC$Hm?--e z6ck8GNbqG09~>N9*SLBs&)%YBQLeZB4v0>C*bPoYtbaI<8@;_hYxA{VdWK?t23a$4 z(et^JiOf?-fhMD%z&UZAKT|)1f>;S&PW1QB*MUg=K1tY@n^;P0$ZTA0X*`MO;dZ@Y z%VM!_g0!@>0K@0=I#`(V?~fZYTbj!sU7Sm( z#%5UV#1nWHh-4TcT~;;nzYb3-9YsF4xSZdTjBoh;9LP0qZhrMN^HctI;uKC)FD5?S ze3Lo&17)h4WgiGmlXhGen0ZiQ{#vhTW*1-;^UgvCjvul)@Q^PLCT(e1vP|K}BfRxn zTuVZ6utN(%6m)crmo_Bd zCnGMet!?G5ITIL$dxuNlb~x9)db~Ke<>ON7c^%`_<}rBfOO_|X#`^KZU4UI{?P{XZ zp;l@ZnrNVBoKH#V1xZC1qPKF;E7KjPwu_jzJu_j^S)OV4`i(epk=t)aTCgS zRqjE&E(s2x20u*YOVoF>P)X9=y!3;FySYIN&g;J>uM?KeJ)^RCS{*uJJ3FbHr(U;i zGlxSD#4kRHi(%s=F{?w2DrkklZ@9STtcY8^4oNy$fw{$^wQ*(e6aW4~JV+*Dqdlp_ z!VH}KDme@Ox**#UrBjyQ^(0npaKjA`?qdutVa+^o5?EJ%hQ0S;Z5c$`%=dQfa-j2& z6>e}$r(U1u{^R@boB>rGb4Ms%0KQUhk zYbz;rO>1D>ugCvkM4#~M;py}rXzxFvUxzLdv~lz9cv`-xwB}gJ&HTncz8UUEboE#z zrPF$f=jlLMA2J-=QIF=;{A|F=H&ZQM@9gr@<57QHA!6xoeRI_ntm1Ub9;K-A_sb=v zM>#E)#J%^HG8yzkDH{6e#Yd^~rjN&})pGET0KH{JXc zE(e>~Na~W-k9au9j7&rM26a95Y@)v4R8kYlxU*5cL@}vCWsuh;`9_|Eo3Ib{?~bzV`~{yMVRmM5%!+-3 zZsr?W0lpSzC`y`+`K@kIgF8!kRBs2f`nJsROt zb-+HptmN4GuJMk$4RY^{BqxCJ#Hp>U=&j4@%QjV^qT*6pi!r$eCSV{-L#+F`>`f4d zlZir)_^lBTD0ytPe6i7$N{oII|K;tqAi+4_Q0|=%B*8#$$NqXDGA8?dnmLTemw9sY zGkE~Mul>$Z#Qkx&8H565%Dl~I4b?VeU}|k+?~0b?a%Y*PxH=R52P*JEYFbMk%L*s_ zCz6(I!D!`aDI#I1@$YYn=Pp z+!HhPW%-Ex&SYc>gIeizp8%>O_U+z^5Z=}^+3+psFrQOUuqo6X-Z{`})^l62huTT)X z(om`8(FCT=4-Rpj!wdIAYtjk<;mgy5%rPeK{(Y8C(Xjj*&%5&AM9RK^E=#v1Ux4en zS24*32%rAM5&7Wt<#6Uz-+(%`O_s)*??vPj#qVHL>h| zw}u)#P{yMW2P%~dCmuYPlXyMy^DkcLssNg+J9hb>#9kx-sP)e&#r|OV|8S72!Cz=& zmYq#jAq}6zuR_?Y4+5)mbC~!<>;pMmj0pr&{?um%saGV)nWRM!oLgp zTk4!=a6ffGY%j+ebcYA8C6!y@EQF@^J7@btp@y zu(muvag3An-Z=CAs4Zjkc>!ofJ2&!m+*gk-+sQgLgL?a>k&v)mpef}e_I*B!$!wJA zjk-OL1GhHWhWvuIlBs==YXXA{TILC}m8E)ozjv`YENALRxXz)w8s^dq^(mf0mPHMG z-11W7=C1U$VwWb!w5vai@9jQycH@)O#9&KHogq7Y!+;TnQ??Jx9VE&l&t+AU zgG{~I>_(m5hl}e1NoHZ+<#30oQCK;oKw1HLMU0aA!;8839uz}Ok#QCtpF~4EkssT^ z+dvn@ky$$LXgIOpg*X+Ryf64d6+f`&zEOFh>M+2ky-Uh}S=Of-5^MTQj)^)Etw+cx zCl`HL*C^FN5g7UsGPZk*eOCWR>I$vE6sgfy;_ zHjl$K34FpdNzC-}{6w5DtT`rL^4iM&R#N`D;bp)&d>-Gu7wFX@6!Y$=<7J6wOKXS@ zp5A$dQ+pZ`7f~O&qJCYc*$y{|Mx7TBEfY@}rnS+Wp0mxxL z@Uj&0l&=j~%viiF`d~XrZ&rgO#9z_*I$s1~AI!0kQnWpRTVg_Yud5nDX40uR|MH#I zRdM|2j+?utwc8G;kUyVl4Sse8zK!I#kZ0SU-6Z*d*KhuSPuTyioc)ERm^N}Nr(qTV zQutIiU?kyQqC8tL-FEb(xI&uh@$`?e-9#TY9eLkwa+7r}J32I^Pfvc=2*UWV8k;00 zuj#jb{aF8;wE zO$k&1uhemEwUf{(l0xhCb;hq1+I-%{(%3c#cPO{Zy!thuUX0zoS6+5!*PT~XhMs62 zw|MP#*gHikpjl{)u*~RjnRWRA{BR#OXPBtCg6;LtDKT>SushRvtmH>e>C-ty?gE$` zgA{hft);MzzH82y4HV!Ga?7TCR8ZPtVWMj{i`f^w4L%~CBCfbUvz(rGz zTwLufxG~pnfmH3L;0yQb950mqmMR zHo^dF@k!M`tlY9d&#S6%7PATe$`GPtZyhWckQ1(gv43uN*+q>&p?3T}s2G|t<#BCD z2cJg>Wq1pqm9#KigGN74kK$xEXhbtmDlm=fkckwwnxf7^)>Tk^zv2ERfvRMDCu%)J zXTff4e@L;AlKZu~2(5#xZhgV!iz(A5aG$oH8wFG`i%M2K(s%0g^&sH_`H#;E+V3L{sl2~|!tq~=C^ zwPr2M5l1tSKXP;)qNcQxBG7O#;L1_6WCaCP94)s*c)ch#@g| zlXVLo+ab%UV1)!UBTL&V=Gl|9%OoU8iiP>J^M`u23QgBOfxgf#a*<%k{_CgP7tmYR zn{fiml^6swr+Hp$kXP>PmAqAK{c@`c3GT=^WSgjcX9Et7HRETPJw3bINSRIT{mLKvFnf0NzGPrUhOv@A}gx(Jl51phh*cqSf!MvYMq|xJ57#`5bzkc z*r$EYHpN6W#S20UC1A+>6B_xN-O62lHCC4z%0oR5PETLSLC>boM}Fz}Jbrq=JOwsXMOa8u zTy?nth=AXgf-}kkwW~XnRaFsivVk7vnNhbb%6ybReg4oAx zb6qre-?W>7>wswmGimau{`>L!tS}dWR!1RUFRA`atT|K5wyB37Zc7bJ@*>_auNq?B zLrN>_gFl8Qp=#)`Z8RBSDxd!|0&+DVXvlrf2s3~#M8bHZ zfeXPou{*pGJUaLQnC5VQz_M@u1k1Y9Rf8m(9&7gee|CWNx9IXAtLmjv)!e!u0`J1j zEJd>I^|Y~e$PCH`pvvEH zDmHcSOdQdWyllj7rwGjBQwrH>!;*!k*+@z-rDE{swJW~3xC|~kzgHxp`pR&(-Ks`` zo#o>L`YDM(quNs2Y@y|?%4$1aqpDZ$uCZ%W7=4DU6E+cJh`|}$M8tIrHm>pFWSLpKN&hk@Mz@t+yzzq=-hrh-W*DEb@ zY*-j4?KM4e6O@#(+a15y+VOx^R`TlYtLadA}5 z$=`hrH-AS=kEox`^I}wtpLB|lW6>C$59{%^&VeH*BR05~yOR!}X>6)4$MbL14V9Pk zhr9-Y@7?n!nZi}OU*ebfx@m8&2j^kM+j)Zr&wAh5+PxnOx@%(By9AR6o12KPB;ra4 zhuG6Q4n+?2cP7vX+mhz{cLYvtg6z`@4M$Q2w8HhCUB2efYbwR3Cfo6^q$zbdHj+jm z>N6u%Pp=nvC(g`5Ki?L0*}aKgx-L-%^l@*X`b>OB&_ z<$Qz;9q&^=0NoXHGM|*0{swgRU39!Iu?*Gv?qI-L(D+z1MERdP<;AF{Uj9KXGm|KX zp89rd+lZ>9!`w?{$3!Vu88M%S`QG9oiR46RE|1^r${RKZS7*2L9f5$Y+sQEjp(xQP z^sqombTx)CoN%9}O^zuRww)LK&fLW}#)uD_F)49SlLu?g&z*QOiT4f& zH>f27$0M|ZFcX#Qg@l>ZaRyKMyuDMmi3Q)w2PpSfaaq2|YN88A(Q9y_0sYvCJ7uC! z;12w^uFJ%6{NZ_=-M(4y;FppQjO5Psx;+(K=bBlhq8H~AYA*S9w@XRCBwcE1kmdOJ zMR}xjgna!P?5jCI<8^+7>G_5#$>UlAmt@b>L#qZ(Cyo7TX#1Za5B)cRaSTQ#4$W;bU>o~0Pbq?;|TU%8t@Vmxv|95YX6U)jK}P>-;E5t#Pp^CyUd*$8(trFiMy=d6TM=t9#w8`(>*J@e+TZIemds zNe@g+xp*vg(`ymQsvD=*^q|Gc*?l$6y3N}+T#*8FLVbDTDfk@;k^|eB{0XRhH4ZUt6otSXnO6IdJZT|x739#)I+&fFr_^bdxv^xOs`*% z`abB*jQAZ%wV#og*Pr4+&Ri<5B4akX|x&*a<+lU-_6A#C@}OD%$H zFvtNJsW6xSL#{r(Y(~brGngSNlTYb~hJ=+xCXIs|6ZO-wDrlrt2c`+|ybtkv+pIZm%OMPd}>BxpgWg~{8HJQR% zrCHt~AB%$UgGf}Ta+|BHR*MoN88UJA1Rz>?`F9guU=rRHIu@px2J)}rTiV5ux~-#| z;J{HRpM!37MXvpgGJ5YW9?9Ogh3y`PIoq@Xps(QIscGsqYPuMwU_Pe!484UCq}Tmg zl3R|l$pDNAOatG=uk*7&L-|GP00YDBD{e`!Py06FiWPZ$y7p^DV0 zp&nhdZ?QbeBk7Y1%uNDGlQw&nOw~uDUgY~@PMJuiS%YdK7>`c>T}#Ro0{{{LlK^wv zY#1c+wR3&Nk*2f-J3Tp}iBX$lyeOrLJX}^*HOSteG2MK0Y$84Vb;CzP8DepG@q*kQ znQNqlj{pmit;Yeb^RF1owwxA=ccg`b^7hCW7F#w^K|}HKPF|?lz)WcFKLQ53|E~Z; zhpEjU*+J|YKR)+P*2aX`65od`8C&T6A*S47Apz_5+i(6!Sn)}v>J0v~hI%)UP#HBn z3dU@I<+=wejfqaMP}9sRHIdf6f^Vj<_k}_)Da%=c2OONH$rFNQ#;Db;dKVZBjkv+t zw*SCqq0Prpew8{P{Qqw(K(O}z5*CnR+u64}ykJ)QlO7`&B>XV(WN~itI9^xb{J1i< z9Du9hxb%+erhKJq{RzFJhtll5Pcw@UcoW8)?MZ{#dTLRR$-;0qB|V*|r~UhDzpfSV9H5vr3jia(Wx}n4-&( zSWO5%OQ)X(9&u`GjkAyDuA71WE`C!@Bu3r_IJP?LuCj7;7~?xXVe0K}M|k~d>3+_6 zBfzWmw1X`EhghTLt}(`)>G3wvbS}xGwFdadR?67;Y|`u zQks75*`&?cIR_ zIoO|kp23oW8bxPXzlRNw3J6n^@!C^PY^Q7w>b>rN8?=^Gben*~*UM2bkp;Mix|j-S$&oFo*zvzqhM(brH`IkT*7+?b z11x2D-v??-P&Nh*9?4q|JfEjiv&t^Lv<&W`k;Xw2HQkli`mY732H71G*6M|;IoQ86)%;~Y zq`UT>LZ9pExkM2ocCT+QZos%`_(h*-21BHz)V8ITy;uGZ>4J0anF1}OdP2$PK5@$a zMmB*g-~IHL;$!EZ55~-juGe%K=;$PtFPPp;Gb9Dv3`QKnm$Je(-+5LL0@;O__!+Fl zJ=43UeD5Z&6zZU-{RQL=LGw3C0+$_l=|41z;@CX_7djF!GE8(s{}c z)0D3!<4Y)3ZRPnl7;yTOaX=iNrldcTYkJYDQ#u0>PlB`o z0i8MNt&=aN2oW=G68|~D$B}P;WCHST1GPrJB<63dne@RUDdPbjQT)kz85sD#;;cBQ z+E=$q?IG8urds`>A0y|BTJG+$t$pI$G?f+gYqd)e%z|C^-58kFTq5))H-6OmdcP^! zP{>l%(G;QGPDOoSQ}DZj4m-025VlsEh#s@6@jz|2xvv7Sa5kR<8i&>!2_BYw3+N56 z^XhpO_MK~_*k1J&^`YWF_oytPc)I`Q#1w!XuoQvw37uJX*Z|WKV0qu?y23%}_U}Jo z6d_fTzgByxFqNt)!q+r(+ZQ#pQ3N8fPCr=Wm5z_Ck{xr~|J~33B`4$j4|-Xz;YSbL z|0$09bO$fGG}TDa=xg@zy?2tdSjPMUDX2Y~9FR_dKoPFHhOuQw{NA~F<8CbW$GG=r z+~okz<*tep*N=b~G?Kx|?~2vJsxk&#P5%p^qyCGroXTAW^Ba$C_qw`c^f_yaq<7^^ zqp$|BUZ4|FMIdB3-+kPR{@vB}Rcr7aFiYGi>{l=`aEt8%1ss2}Uv)rC3=KWM90z9! zigvy)v_54wDe066%YBuQZp=Qr^<$88TREw=2`V7KO8J4QPbRK07}EST-X$#z(}vR= zT#NcPfJd)=XJJyNL(r9*g;q8uxeSHS+r7Z$Cwrd;?i!2Am^>~OLPB?*3E+MmPM-7V z?@I!Az>53l?B4ATx7Tv7@7z9vdB5GGz(DZtHd_5;*}(;k&aa7zkTuANOKV}LAPXO> z;~*@-VPJ9HE_!XZ2^r2Nld~4i>l>t_?UN2O^0boAXk1(+;!mtOcS{p@7^s|HE78v+ zp>f@6N+aRk-D@?J3+utmtYqhBp|KXX!ZI&!YdLlZ?!Wsb47i}5w9@dF|4hpEF)2>L%2>|OBYgyI4BX!ThX}`1TN7d;`JP-ul1ZcCd}0u}|pU|6YHy_Oe_@~8+ylL|Yo?PS{8X>ywt7Qg+2 zLrl}6F24ZtgMoGt$rgKz+xTI9+~=9`l%?LgjaVLHKavM{n0%Pr%Jf>s-%g$8`A>?c zO__G}<&l?7PdVR@gASw0RgYr*Pbw#^#98@EtlM-P3X68PsXXX;*xE`M8MM#`9dv5y z+Jv1;%{wZK%7&f%gtksus^{{L>EbTCx`R|QU?Dy>@;0uuj#mdsG}c}|0cV~RzAbKY ziIw+`!l!dWmcXJghW{9#;-8Cr<7V1;U?Au_+!xfv#1c*tWzRmQ;JAn z1pRi?=_Yr-Y?)OCOIt-&Pb`-&2EhcyPsG&v-mO1X;P?rh>WizC^D;H=WEIwwk_0@IVqt=8kfF}fJaue)q% z!892bnzA<0RRuD;(2>dU*#VOtiI*yO>aX*k!M$4&=b@7B*P`BTa@|_XXdk8Abr384 z5bAY8z02;*abvKPeiCj_5+3m_4R(vZh0jJ9o_Kte=BqHp`+IUO+Z4g_@OKsI9V0hO z;fDLbjE`IR_{{8F-ub0JU3TLeh7JILo0Nu7JLq>)?^~f`_=m(J4m6>+ZBiDyhNJp*(p;vIkUsITu76FH4?}@aTR% zW$oCuZwNQG-AQ5Z;W4BO+&vvPGHn+CclH*fEN;u&yLx#d?-S^Jmd2N_v&brr`rEs5 z!wbvzs;AT<%OP|@mtLRI!3=8V{Y%;6=xC8Mc(^u+Kw#K9`!b-0dA|<=!QDYRG`5i?w z0~9O7J_Pd&CN;NkVHS4~YJ+c`S6^>bggdMc=P?a13#0)kqQbeTHR-)m8^g7VFE<#O z`nH>j3q-!`F-45;^M<}s+r&;KJ=Y4c*>3EhmAPBLpd)?P$(IJ2+IXY+#^Vxq-bg2sxcT??$@$cjl?IlbZ?EHdyS@7qEKgu4&r5qD5)cqfVkj1N&qJOA z(Zr9FyBQO+7XXF3&C97~w7zR~A)nhVPi;zjA%XNyQqjj=zMoOx^eCvlawScmtJWO& zT)2^cC^Lr^l|sAHRYwV7+VG%A(6vJ3E##SDbBjh4(Q$kd&Ymv1n1pkwnj|xH>omM& zeN30vyH80wwZ_8$5o%6q(Ks<_lx%;OnJCFnFJgENr4fd7yk5B4ZD*JYF+LLNEH#E6 zah0<8g5ZIo+v0HLodNFyY~#Wc{WIMFT4AS&Z{8vm(w(~RJ_LWO1MnD#sAp59Q1yNq zsO_W#l}3mF4)}E4{!ATcB2UNu%8I7T5`B<-OpE?&kI#RkKL3s_G#_yvk1_)j;E53% zg@AhD-uSM;)u&0#JdDSEMXde6)tnmAQ%=*+fOKBH?(nV^AlkqCiO`(zOCL$QWUy~Y zE0F+yKcJNI3{Gts%OP$ZEU`}%P12jY2^Nj_7d(BJVrA5@IS;8H2=&p)CkCu zub|iEHD%||61FiKOT`E5ve!X`{C3o%zWhU!qD7dUP1D0c{1BUUTKH})(=@f_m!^5| z&?)i#Gy$sl2z+1PZv+Jg)=Q(kRFo>Fa~J)gopSjFQH6b{LX=X>D56IKW&4( zO#j9AMd9>AGX!)7l?lU*U&iKMMAgxpvzgm?Dl2EV3CcZ{J3o7OzgPrU><_OqaJGzx ze^R+U>e^eEHowWslwt@~F=VTFKb*;2>R0Hbpd#*Ez`uM!i-2rd%Z)=1n zwWP5syT6Tj@GC!bPH67EYl;_tW6kUBnxKNbHB^pi&YGwTf2od202CRj)>$&X$A64C z?DmH)aYT)*{+`$x)sN!#lp>m*J6_s=kW}elWi>ghe%>z=CM_gPU3hJ{;q~Ak`6{bC zQIaM~I9*^$KI}_(mk|46%YRgS90{OJ5tC4DBIVBeozCwo)cx+FKvx{u>yFg4WfcA} zNYN#U@YaZ!?AU+QCsRi8U#f*55Ih)ab9sKMWh^6Vd}-09?RPI>^s9D#CmJ@_Ke}tx za+TD!m`Ja35?Xujop#AWlO~zDKdb^?Sp(Ki=Bs7|8Z5HGqc1AKpj&&r6qhI-O$-4C zmaLIpWuj058Qa9;N=9)7{Nn$q^=a+Jy2E2nW1`6qM#U~PgNZ1~Kb(0VrnqP!YZt$t zpI=3}xtyOi(Y0vyE`ThAJ7!I5uYyFG}*j)6}x6bUd~`LLyfVMVg-F*?!L- zmo5?&5M`+0o}R$HrSzM&%P(r3-dPXq6HkP_^kH$A7-VE?3~IbjAB%OX%&7C^4e5qQ zyy%tG&JatlT>ZP&u1J9O30L>IQ5tAt7|EJgQ_l1(@hRhZ)*;6}-YqpXbJ&4t&6g`8*f)til07s-Vjw2-412yp2;5cw6I z_GsNiujej>FyfY5dugI{tz0VOuY$wou?+CO_vywkA))`yL|0K5=w@Nqm1WG%Y=Ap> zyTEsW3>t^6T&f>s^h^;3k4$U*ehJLyz`x3-<_6%Df0X}^7+9zcfMt_OOK^dw^Dm|O zpJej?P}2d;y#CV&Dss&4Gg$))ydeBal%|D)=^3D8eu67cMkgVU(F`gfi82rJaTmzs zpzN|>zbg;<+pf(9{^9C|%!_5x4{SBsp46)xu+%vQ9H~4W` zpu0{&lNuC^UV>xzHVnRZbeVGe}~%&&&L=RU#-$iYQBYsTk1WKd*oDj zS-n-W`=8UH0e>QViv}N94;s|~E%Rc16x+jLRR(4-@yxO$jqu^R@O21^POXPmP)&>9 zud4hm87}W~y3Ad1S3q9yz@!lP!1xL|#=S-8%P7a4zO%(fQ=2|M!LLh{-T5LG-Fm^f zmc0Ll{_1{<4xLGfHnjTruQr>Q?p|a?!?NRK;92NsS#(O(pI3D@UNm7R6*Xq+Q)C@5 zNyngdrF3onu~}M&iLF zGAF3@)MD``JhuB<3i)Q7uZl9}SDWNBUBJ-N)Q5t{jvCdS1w7$+Mgdt$EEvn@0aJ8F znX{fJq@9EI@#f#Q=Po7NU>nLgp#GlAU$H&Rwtu+szH8tt?{kQ)jOzUba-oP-_0vBz z0(JQFMV|XQih|-sUESd;Q)hkI{?E_g**i-y4FE@ZfRbUM4NRGO^Sw^&kq-RKFqT9i z<#Vm+uQOS_xtXHl8R0$&R(dW?iMZl9xUp<044>WjL(#NF=lCa|_$Q~e>qDGZ)@^Mw z*9*7+C%vlBzSZZE8z1|PCS`B0;Mb;0y=u?%MN~a_oOktoxZd}!*Eu3NVGmv{DTp&&3(E+ z5xf~4aXgK+imY>Yf0V@U{C|21G9Qk^6;P<#F)Leu{T6xjo~n$Q!f&VNA0F-$;E8RD zZP*OcxCw!$ytLM3e5L@w2uJG{;qux$u|1pB(xODUece%yxvt+RX12-EgCiA}0bd@l_Tj z)2>Znqw%dryO&qrc@sob$k*n%-)-stu@QiC?%Ot$Ks>PM^wm%ltVFEVS6QEzxWsbPCMjKd6}aw~C9v4_nb1$ry<7T+IqcZEo-v)oj!XE4P4U$G3PA0* znu@@WxrC{?ePebK=?}YK?lj#uxV;a>$pI|iBR)1E=tJ*fi?Fy073`^j*X5Uk{LgkI z$V@{rPtegr-n^;(_Eh3UL=;|5N(v=S$^u^jb=`(Mq_OUxRK?}~bD@`Ss!Ck{XbRpC z+o)I{TEvye2SugV1XScR@5VXqz7sZO8=M`qfR2A3NgRzf9gR2Bzxg-OjaVC!yg0F{(q7U+CP43 z;6D$P-$0R~z9Nv$XV^sYz3x0C2zPB-g}(qQ@*9Nb>>X;-d5b2FUIom|LNPp)VtK0a zq4v_T^<{6mC&%m$myM16~7|ROfEm5zOiXeFY^EN^3`EcZhgCM-72BdA}!tBtspHR-64%bce7PO8l}0O?1){oOtULSRsRDaA}DGjJ!J+F0Cs?|k5Ma^w)qKUc(nYvaNMx@Y8B z0vC*}{f$%F4Y6_bR!}LSd-%k9D+Rw&t8)b=XkXiYTQaB`e2mryPZSIU za4V3=HE$WX`k~W#0xu8KDGK;!JojcA*$L%A8u<Om{JBEhH5)7C@ZyH|$1y?DYwd*k2PG9GRuPcO1X*;Pmd zO^ryU4g&|0JL_wlONupi9*fK;;+^@#ewdh9;MmtSI(P5g!3Wj6e2YxqqxW-14=v`d zFB+`Q)au*#Q>asdM=7X6#rx~t9gB_J3ZMO4L6L%azIi@!elfkP)v2nmzH_dcwiARn zbQV7-2h3WcEI00;|1{DX#}KL7lf@;hz*0RbDV1qmDNj@{#mE>r4az>M@oQk=W>u2k zs-&M)8crnY4q%d%7T25j;!SQtyy;;F&!nLv)N^Q2f! zlZt|=L=+?*e<^zsTy{yD$FNo1(JVV<>iNw$H5F%^%apVVB;tq{6LXhpmxy1$oT}T% z8t{KPkK{#qPCwDo9&^5auY z!Da^x6YppXRa15oy7o0+M{3t;w_a6W*Z3oS`M7;bRP)YyWnDKPPN8)tDm<{cN!=-*@n3)m5TW|9(GdgffN|acKWN z7}G3E8skF;IIiQ|FN{XlJ$KAu8zGv+m_3+adOe1x&YagO|&o7mflq{uhyAeAz zi>K&b1r2}4+xbD4{;$0{9<;~=I0-3#Jd$rwF8h-46xfw=O*UpgeO5Boi(h$R--2}E zy?FI(at+}->@ccFV{QcaZ!C+K?FnO>#~W9CJ&3Kw!U-0SybERhAaYxN`=&&i6C zGbTwi2neO%WS^UtO1lDNH`C9pvi{qvkxyN z_MBkTq!(rHc2+8#_ed!(8>-^c>J{HBD&g!qmx_VRPwUo<5lydgCQ1}2qU(1+YS{Ez zM|H{6qc>q+R6mhw1Cc=UR}W)cpwGrXX|s_XWKdY1oDRh|~7h zOY^@8NwTYrn-LXPOPjh|dfaLoqiXY#>$c-mPSQ=Huy&HrhB&NaZEe?LTP`l?$9olR zFCOg3706nRL6%?>qLaU=mPGhtc3*Xqs7~!ly?pkepe(_yZ_^qT@}|xdvs&dryL`(K zRK7oFn|TAnaJ5gx|8*}b@IU5d)q|6Fm9~M|pAw)5#sC?HSg!Qlt3dyw7NgAypvjuC z)n1h(*{Yy?-+V~+AytE@=~JT*;vh=#`(Ov>)d>1#-+}^{5?>EB@=ecockk8b=?QaS zdd3iG2xRqqN#G^+&#}J~pdXmkK79=Dv~Pi|Z(yxAg8NH6>vanv407HQeu4|C`|3*? zCZ#4Z6nuv!;1{nCVn3Hk%*)BDuzS&>Rrf^a0mQkfHgkHeqH%0NGIkX!&)2;({O7*) zwM!7}HK#i=aO-6WetF?qrsqp_eY)-rAV9A4hB{Ta-)&vEtjHb{Vo!-Fmn2CK!Jx@0yfX^V-qeIOm% zqWP#HytVLWBQ^8(yY*lNXP1lbh#VSs=h<|lrr^YPHVgNR)QvDRkJA>1%>tI$VVVv8 zjx1z+UIIa`hyESToxm-LtX)~B@FosFhBfVNlplM}l@SfH=CMd{fgDXee6V&Y>bB>` zX+Ux*mAHPhXddyWE`RTsjc;aLWHq+{F#5alLxp9pHcNYJ zen9*_`0#}lK##e^0~A}pi`3XX2)*)@j6W(vp8c#0`Q@SLQ70+&dzGuw5RTtoM!O_y zmIGT8`8}0?)PIv%baxwfO(x{AcSWwLTAw1|NQ&o_EJP$I&v^iSLq@Nt&XDy=v#T{tU4`1_IK^ zS5o$w8xAMUA45fla8DQyN~#6~_DQnb|k73GFQ=q^4&krI>;J zEu)Ldlz_A1?rGd1h~K=0kd#ZaSBel zZ((2p5nc)?IgZvno^Pb`}Y>>a?iMIN7W0BrQy4^8wj&W~Pm zNk1QcN~ga*Tt71vNx8~p&~NzXz3)`*9V@mHkDsg=ag_=2j_yzVY+nw61<*aaufJIG zBPm=n2IX{J)OPWRPw3MT+f2vCoB+>$(hMy_jU)mZS}SAdNh^^~^DZR`q3~tTBP8%1 zUk$Lkv6u<1{n8hp&k&Vks!<{piMiV32k+lm@^lBb%s!)C~&17d9fzi}Yb!BL>)Gz0amKRI2tR2@5QrzCMc`7trh<8zJx zLgnm~)Td92AJb5!P;$~yr)FfF$pYq=D*uo^^EqlIKHis9-}(V&VfUbrw*u#Qc%86w zNz#0$v=VG%VT=B;%Ex6YO}}r?%=}QrC$5X)G|y8F*Z_lk@---~2i_6Y{iC+|S9bn) zy#3};hYQD54tb5;ub!+k7ptUOFS)aIdwIRw1{bOYsPX?K;|^z>1W`oVhavdQ)5Dux zHJp7qa zMxMy3>^6_-oea&0za{3kPJqqi;QWNeJusX&Y-O0`G`Nn$l8zfk2u+((nD-9ttgKM^ zOQyuZnfG7af7?>)psIL*fGo|IeH(fE*73q%KBsFpo%}ms?ds15<$Us$lM}Kx+`tr7 z#%bVNB!sDdwY(G!Lolle0SWDLefDpG7cu9H@E|?Ft8Y1=6D}%(2;WvpCm zgb`1q7ae!`AJ|l=tH?0+LS`W_(02lh%xB%c=WXI*cR6U&)rgSj`J18a>E-hsA;;gESCq9nPcSYL%)Ar~bd<`f( zcp;gKVc`)-6=8C*QvecUv-Nk)Ty9g}2lM<67^*GX0m0s(Pmsf`SvB zDXYNJ0XO$;(*D)iuvOxy*+yOf$HKGKI7W3k$Bu;bSNApb9offfkk1yqB|NX+t;eDD zxs~FscPpTh51PA=OFrBPNJ-Jk$=Zr{nrRkib}O_VbKISJ(Fiki^j60tT`d3P{wAN| zdpR$L3k1x9W3Sq{Zdxft16z@TMz5L$As7T%eGN_dc++kLdg!|~yzw#KQ&`=7(;(AW zsk8rZqC&XL^0t%77a#1rU)6&2;N|<6o;HjKWANs5k-zYR*Z}8Df!KiZ6vkZ=Eb#T|ZoO%ZkS`f{EwW3J^3*p0r}xpWYdmje^|`k#4xPY6ELr*iwf zhrh?wP6n~_A0N`a{pp^7cH>}miy&hL@6TZ-424nph8 zMkP~-N+C|2Zy45c#BviRqqVjR)MwE97_V>n1nRv&ZfaWIPr7dl9>p$VLBrZW0ty(n z!9OyA6j-xYk7B!by>2%>1TG={Oh(=;fKgw&&jwu2zCy0W7bhyp9{`!WN*jRZ&50TJ@sI+?f!VVFl2Iv&J_^ zvzC6|0_sZ!x__cmyAp317477>wQu{S)d$lzNg;p5G5 zZJG)?n#zi}tu%8qe_4HJic0iM7SlVbCaS)fJU3RG(Zsid{9 zeyR0?efyAhAM2)sUtzpufN7ul8x}Z1dQX{@&?&dE_`E8xZws!j3D}JJ#4GOZ%hr^> zs^s~_^GCCDlM4ZDqui>m>2IifDni# zx7|GH>zZ!DiwW#D(l}iH=rExe!tcn8@z`dTKJ?0zMm59vto@$5u8WJm>k&kT7t4K5 z{PF{DevNWj5jMlD6x?P7K}py4{km$t9&PrM--VR;yYKjzYx3T_I9YRT9@ER=7Xc3} znE3N;yvTCj+9S)a$6*ulq;~%7Cn7Vm*hi$`bS$s1takiu2{&{4xsb?6m?^9Vg(p63 z3tj4R%7K0+`p1mM=E)RIo{R4#gHH7m0c}`8@Up{IIi(cG@_ZD?;3OF)ZrGCtw9wa| z9Ds%l?&^0o{|}C@3v{HIzA7SPx0tDS_(Ngf&_0Oi{{QXeY|E}N>7AQ=-DG{-_psBv zU+RRV39x1|#~A3mGvi7(iaK zET~k5|6<9c)MfH^*czTPTclX*(}02u!pQE`I_QR6c=+c|+KQ@6RBhOOc44=Gf6#IM zCqE(UxLXF1vIeTnd)h7*z5ez$_ENwNa6kUR$8Q+L1D~Gb9+Ls^=l7yVi&qUA{*dco zxypL`yKvJ#04;!F4{X6dDjT?;KBPA;(%x1r20*k9Yce-WciDkXQFWD{54e2x)%r6a z7!rZI7UKahaHAlV-j&~VBxrkg5l3ebZ#sXqpD-4UqR4xy5IR7pnc9yz#neyy}8>1_a^?Xm~@*+Za!`F-*$UI=g4G4hWWyem`?m1P>i3u+k4XRu~~Ihp$&d3er--O*{#{?KwMlfIj^`hM$8<`C?<|LPKG zKD1z`V(hp*yW95wpJm$pP{}KR|$wLL?K^5-6@g!ks$!Agt^HEuNA^n-sk9 zx9+jizEQ+0famF7gG28sJjaUKY~PHrmCk(eZGQ}3#(Mt0Lqns+c7zIceynOYuE0bR zBY|8{SJ(28PRa1pjK^h!uIqA`W0o)|jz$W)7CkcV4FLgg5kyj)U;A}zH-V7`MnKQMa#?>6`6U`jJIm}Mv?dx@~Zac zlRjYqPzYbtg*i<%lT=duGgH2p@DZo0eC7z65?B1bt-3w)(uh>hRK~7iR{jW(USuop zGqWd0Y~UIE9U^?cySwo=e^gwxecnbmftA5>w^rzN#Gsi;QgxSoKAnuoZe<6x1TnxK zHksoI#(08U6$jdJ-}8+W&cY<@N6NcVZs3r8Z@dTMFF;jkGqq`UNpU3HODzQ=nfpzi z!g^WNkb7}f6^DjX23sfjM>o!BiTQWD3@;q^e0!!!x-?<&sK-ij8vO}=klvT#pw@HG zMxS{b#MqPt^(u8h=`?vJQ6?!~I2*0@3qSl`yb#65!w4DSt`;Gp_)R$J{lg*Fy-vPWq+)We+NuitL8))r$qS z8iDt}_Ts4r`=Ws32SDM}t=YJ%e@*gFOS}#%J&3MsV|S-}?XZduAi`M+|6_}6*Jf6G zH}pon_Li1lTdvA^rWz2Y%uXPqlaP=I*w2O$M)(knO4g&OBjP_>Z?O|ucZ2Cda1s~` zxq#TkSQJyFl9EQ)agJ;1KZ8M9f_#T>k5*JL^O(~>gSY0;x$(^>z*@-CmgU8IPP@D5 z7kw--gNe*QtMA0-#uij@$OKua+|7!_RVzR%v26Y>Wp}o>!U;J@3`mQeZ$TnvisRZi z)U^Z0g5K!rbZUW>nFuCBtR-Y`^krX6cXiF#SZ6n_KYGws9$JxTX-z1#+Cbh=sR!?b zr$2j5g9PQzNz*t}_ntTaq1k~u1bc2^ z^=yg5K8g#{^c5;;*zW1p-m81+5roYszb>)KMX%!_vW_0+J2Bb}EFsztEfSfrQzN2d z6u3QkQTS!X$tK#TeMWg0#B_an3Htc!&5N^gv_u69Cw=al&g6h})mh$N zC&DRD%|sCYJbde8qx)tRyu@j16#Tx!uOkjWe>dQ33@WE>b=5kl?`ggplA1*=CF(3F zwedn1w&fRNxs#D3VBG_)1edvLF4pw8T+;Nz{ezuiaQ+>Bcja)jSfU1|GKN*4401T( zx6em{CJMS7O5zbysN?RQ_i;brgkJk%wgdfnDOhG3qlog(STUhIW zWuu;KP&H{yuFDk}O&5Jq0zWR;uz`j4?a?~Ln&s>A9B+;IfH%Z}I#lld_xzJ4T*xg* z3DES{S*1Az3UsV?7j1dS$gHw=zSS+*1BrIO>;swUE_u0x9E+erun3#m&*Y0c4pDlx zl=&pvdd*!LFEH@5*ykEgH7!cdj>;hN@+|7yb9K9jN@!QMFVCvYIlJ=M7^v% za&9uOb(`Xa)WjXlyac*_d|^cSP^NTN+lzpsiq?3dygupmZ11sT}HxI{cL2FhIot&q(b6^K*nd1;I6vGT z4Z|SKyyF^VH8RKDRx8+HofF(wBG44izeJ!8vww>qNaLyY*G0$b>wt@%F1l{(cZ{{Y zi?u(898ZUA!Q-YEd_>${hv z;>7vy)QPjs(xZ?rebWviTD&B7@)wu^(!AYQ#n$|X(HO2+!lBf6fdH>s_~DW%DrvN9 zR^U9ew54UDxDRjqo}K?K;gUW%2-4;vbj(Mr-i(WMujt0L=rr2y9KNppXSIXvl3hsQ ztUNtZjoJvbyTaBQ2b(9SclBWKNn)+oLayTCWo1JJ%qUnpFxq>Gtdf6K)FXMU<~=T7t|1w2jYd+vcH zM9}k(>*H~P=m6%|?)tX_w5qoQNF8aJ`E8HrhTQmFT5A3YS4eMP!IkeoqG)3pXOL26CV7qFML(uq^u91|Br`EKOV{|g)~^CteO#<{ z9nV!L%D&@Q!uYoNU2H870*+8ridY%dAGtd(<_x8l4tPIEmXlhLt{*Y6Xzv|rO;VY$ zUI97wd(6z&JmC~4AQ*&in4O6a^bFOjEQ4ZzeJHEMW#Y}5BSCBFP+?DJgz~C4*1Y(fvZ(QcAiNk1SE^>JNeA?_=L+F!5J_zj-~^I}8Bk3su*` z-LU`|uG-UQ5I)}X3p~zF5J@<_!j(a!LHm)>2McRvIYK+ zk>*Z}d`mSiq3E(~9~QGzZEod$j#+8*aTf9tkQVCb^gQw7gwna|q`Ax-${RdE(#kl{ z08D8K!Wvo^sTA(L!tCl#fk)0OSIwe*BkUe$)&Dz#|Cz||qWURh7rNkm!WQ=tSIpEy ziv(r8T|O3HwRSq0YHB<@2{h1?v{X+;*EVa1)Dts=I;N5l@T>0-{}o;+vY0?-mtuh4 z3kEt#+#v(zaZT7B;b40I6k0-$JnRQr7#c_(4~B);ATwmiJ`&2yE!jR$V>9mF40*dc zsQ6u(kO0qOEP@vuPXIb95;7&QLkPI>+K;4ZMW>1i;Fu-<3pVrG=3CZQez18xUi8my zzaACdgo#EqZ!?l7=&ZHBH(yEqbo^Qd_W2Ks|G89>9*Qzjs6PO2(W$>5KqlyB2Xs2` zv4pb@tD69Z{FVIC1dmo?dn6>oC>k1pgia3^NlAo?o$(|5%fvB7l4LnwvyUG^D~m=HBUsdEV&&UTW5PT_ES0XVF*3UF~bQ>I;f z{Sg7@G&8EXZ>B7LxmXqWpL*%Ou1T=Xz`_9#`r$wUcWsl>6wuvloVKzP-{> zE(OrT=I9B#Od*)$xKs3FgCyuComej;96L@GLA+NBj;q@8(=%Flaw^wiUV)w*0OF7J zh3uL1buLW(S!+Zfdbt`wZ7kOPpT>{NXlVoSOG}>KN#D3=E&SkH&ZP$=POOxnCe~9UHj&$BG)VZGh0uMweDc7!A`ChkO_s0WZ zzQ?jUohXyBJyNmOC>>8%74ELJU#=)jO=5s1P!)7ufbu;b&;K8O5q5lx2X$^LE-nrT z2-w)z$X7`IfSUZy(CcWSE;KY0(cab7XEJw!+!M~lbA(Mb%GRjfiE!o~Dg z5~uE_Ct!xi5BXPjWhPojz4>HVy`&Ol%btfn9M;sDa{sLqMMKV)@hzH?Yd=;RlcRV( zXby}+B64xaeL!CFE<@D&d`=BlODl<(kdSH6uKOTUmg$8mIKZR}3=92AtqM-V+WzZX zDMe?8gA}4fpN+;uW8bf!CL5qwja@~}MI}Dy~LX*(y`FY;|V=RgbDUGFUeRt;YhN zxIbi&H&&?t_QrqIyr!BBMItzV)$8~c#N+nR-SDz+v$r>YyLUYSpIjV(M5_y?&I8Z` z852WM zjkg$1Vv9cX{@K#fQe4bTiuEC?DnBy(=}zn7-chOd_yYAd9 z3Kb*(A0MA+t$jS`;`ao>38e@U#-Y_hqvu7zP*~W+`qSpOSB9;V7~MWr<0}PUSmeWK z>`;9;?GbLE zBGlYb+enofX58kpmwUaMrpSm1sC4nZkXYnJM_my-pV!aw!2u7((LJBOdhTUprX(X1 z{JhnTn3Mb-&5r>5P)SS6!bEwGSuOB8!>awJ-S*q{6XPbI^_$_q#6&{eiPN>;XDT?x&8~y$VBQ8E$JJU-2`#xI>X}rP=#J-gnGq`+n}ueG znU>z(fYWm7pEvuKM;DnD6r^QdDpQF1ojvS=DkD!bg}uMel(Djf zRg#gCy?)pl_z|A`P9F^oI3rqVvzjS9y$u=c`O`F2xbl1xe?C;4GZ~lL;r+NDoWvgt zR8++%z{7Yhqv2dAK3v{$6jt+-w~*HI*wP((b(dIuySW1h;jAKp!G0_kB*K4UcY{wPV@Nq2(YoS3GB*tscdSxeYm|SN)T2Mu2Jv?jQG=% z%5|Q>=8r#0{oV+!3oWWL?zk$noH<#aupc=kMRO#9@*&z7r95_~d}3 zCNR97hp)k?S-mr~ItK(mDXFN~J#t_99@fWfLn88YDlyY@`i+uOITX~@d%>3VMlgNr zH&_!Lh{VGw;=J)UR;^^ME0R3(ZXtfN+n3L~4+%{%4a*{40LU0)n#S5STEA$iF9N+; z|JEY7IBY{wW>|9`spCf$u0O=8S7}wnBk-`W*iqGJb+zzCMf6dVpOpX04hDLe?xa<3 zZbj>Tl&+i5cU3fp)pq;Y<2P9pPmP6+;Hx_&W6!%qvrSQjKKk~vn`6vm_V(MmI)Ac2 zdFGV-#Ay|wr)l`ON;~uk3)n4J87L0gD954aaI{}|IPJb4hk9*1>Nk+E8eb$<2$OSe z{YkAse^;*G$ic~Tv#|JW)8KBc58D{JA38CyK9i{{1=*g87InRy-0gT?o(6NEEvokr z5D*N0t+QEfhS!%A6(uOihsJLB=VbfKvcBL&mrx7<&VfEE(+j&z^GTG3@9IZEoiTn* z3^wM7xm>_ATE2_k-pD#SJ{Mz?Q$qKf=KglLxqs{WCp#eROxn5 znR+;Q?Tl@)0qVD2kXVhaORyPlKkGP0B}-+X+E{0OQAE(M422f~b4#GEgKwyFCvNU!-KR@JBSnejxn1|c-t5V&Y!;SpNS?`Kq3B1Cx_Dl!;&K$IzKXa%0 z9ni;lhU=%%=QF5L)1+5Hetu!gVn{W5m{?A_2_bUOKm&Gp3kx=3c$iG#%a?a{M1&|3 zKsx>JyuM!Nb9)+D%$y%kWs=_M)!J-Nt{m$t#$9-YA+UnBw#*!zEbY4;(dmEXi4RTd zFno~n!_8SQCJEc8+Mg8_b^hZ->)$pV%=-0uH<)|@ zbF_5ncspGCZzYhzUOB- zR3@mb=!|JZgjF0Ru5ti*rV49vD}(m^y$ZjhE8_=K^U8kPZbL!2)HuDpN!eIN{Cv`n z2A@6|=;}5)KIgtVSxn1A%l?(X$h616wS)_NsmYv~k|K|Sp#W4|3v~J|-?ym|I%PR_ zw9>l$aO={-TVdbfTyF{X;6mpnCFZiDsr48P`{wRgR#GnSOiNXydjhf0-s;)u={)A8 zm!t&;h-)*yC6De41O_9&B#alHTYOy6cU;iM_YV|)&(G=~@JG}2{F>6|{E?*H8hiDF z%L5IbTopAKgiI(G#+0h+T2$t8HiR06iNPQvgGS6m?%!F7$IM#9%F4Rb;dt`&4UVoz zJw+8k{7vQZ+SuyIk4kv;&wD>iBoFS0AJdRKRSAYUKNTA z%K6I58zw(2j}dXKKbrtkl9q2QYcty&D#*zRd}6YkN-90WA|WP=J3KgQb}Ad5oheJ8 zRa$KPd^Gv&doffE8J@A9dswD?s65&z;#=`^2kF3uGrmcpI5#YQ+409^MPXJL#-ss8 zAAA3u`1Tf$I&IFJNBa{fYA0QT8aKL^BH&f_ab{*?(!XJkB~jYhjAWBc3muu)5^C1x)V78r<2j#QKVlcUoVCVy>+ms*d%K?T5v)wq<@XIhn(2tAan zGRv}Brv^}QCnswAZO%Jh)n$m@tltx8udTNU+5{a_-IzH=!)ksn%(qAoX3xcw&_U?D zyqndF2-h4$N?=4OsU+;5vU`|W-x*dqO(f1NmVcf|CRavObbB;I^T0XZFaU3Y`Zt_6 z%`096d!y>5H<}>G^Ul>OyVY%Lll2AW>TUQzR+i^uS%XH=%|VCv(L-SKPC+DU`#niA zyOGc4HKmu=L8}I5vqBLO02HF@wqG~9%D+5WX=XcxxY_R@Si`aXb|*m|9XTn#+@Qtt z%xkwf5mCCP`Wi03`Nn?9Og>`mtCy}dciVKDfKSCX0b)D%I}KRWppIb_e5-`6a-S1> zsm*rWPCG@^*ujqd?_Fxr1E=PVpc|_}rN95(_XWfT<;~(^Yimnw(-jC5bmyfM^+e__ z9|iiHkE!^b6UfL6UL7auXm@eMBw6U{N?J8D!_a>5reRQs1xo3mjoI}N&r9)H9_YV} zS^GU2Dj9Y})JrWDg8Y$~_*-~b0yZ*wRY1aN9t4;m&nZi&!JMj@)!=6DxX@^Kwg)HY zQzztrp@)I<7a>yMIaMxVjeMfImsQmDfk$~IGyyQ(B%&}v59-M1=>E-#{o?ba_(#sK z>C5D-81fv4Exdkbu+Ru6T7Cf@3waQ+mgPGm@0%Zn8ObW}YdD%#U;H4L!wO|2uzFG~ zM#a~;wy==hCMim}ub4#_pq3C7@CKCL?r)3CU;_J2*ZTGh-m)vYN+7u64Crebn-4IPIPjR zn5n6&6EhimMLC(kZISxWe0Oq!65+`8F%nD?q`WU|Qe06%{&x>yE>Fmx7N-7Ld2aT{ z3Mx)5wR&=MQ;;2#8$W+~t}pvfPM$vMlInS}Gg_#eAz$>T)PiZu^RTHm{W-NxLHl^& zI&lX6q{>LhC2?9p^l>Z;#Uvp9_d9jfo1>1K;Vsqv6Qlbw-&fu{%%+$Yf7TW^tNb_4 zi$PKaMYt)7-$NyvvA`(-PG6VnYCq;6z`q{La1yq=oeR{lEP*eZZtm)XalPSKRY;Hk z`pu#cU^w}HNsIU0C&wj^!>WGx@glV3h!+H*CP0yZndFDOeED6wT(8AsSaL+(X=`#c zb6kH+sQy4)FtO36O#(JN9QNs*ff5XV?jx&{xsJ?BI_j9N05Pi62yRw7ANb*XpzP_+ zZF~~G>0HrQJTxvc0<&DxdHpo^+Vl^??_xK8A&VQpdX=%Uuf>=%8CeWm`Q)GhUHl1a))=ce8Hy*L+YLC z^ZlA#c4<)TwgUNmNJu<|gO!!6o;Fu1D-)Amt0O@k4tG zEyqq4uT|}s=iLtAzldySEA-FyygS0_?QzhNK9*9frVD1Wipj~%59Nm=9vVksV`Wvz zny6S{Q+!Z=I5)NY+1Ac3LZUZqrUEA<++p=NB(d3VbKy>={&?jXvGiStrAI?~AW)P9 zK$tGEBg*yLEFOd2RQ;vxX`OM)-5zObdv&B8`sZqFKNeKi>ZWymml|@)<&&=dr%`2q2`SxHzy2tfUlqp~;m>K|x_>WySA# zZqCfpQpAzx7K61o8@zUX4vA^dbeM)t1rS1OR!sPh4hyT59aZR2>lj) zZxi1&^ucdS_RC%_qHOZ4j$^YF?f6`C)Kf*HZRff;!*9J!W zT|Ew_RBL=I1c^sFxS5EyRiL~dni{Xk_%Q9f?&bZ8Nr5qYz;*vyNf&2l*vS6(yAz)i zc2bOAD3Jcbr`6{t(bqZ^%xoAPE{9VI7(EQRGSP2+kI%j#9L4%5@J#l^^L z>ra`pt6oqiAvN_5@Ilm@t8K_j^`IBRCy!4JG@E8Utl-Z{>?{XU_B#*DZ^|xwVez>?D ze#u8m)aC?VicHgqrWD0f(H&JvORKkUDAU!E*-gaM!H+vH8_|l2W~T_dbiMA| zuNjo)$3H9a9xElLe{Kd_A|FV#{L^S3$hs}Z(%XDnT)^P$#>SgahD_a74^DqL9PYm2 zX=!SjQ(SE0OeC{A69cF5{Vq|L#~E2k-I3Q#9h^RKZi zmr6}+@-t*tTA1exYTgBl$*KbgM_>>(4Q<+3)h&nQSgr`kyh_p7bSxB&k+C9VWZeKH zNT-Fa>aj=@GPe}97!ZQtFNFuR&_!9ql~QF@1tV&wQdzP}rO=wroNT2K+DQ;D$oE?2{4q6hmd{-?aIDXHy zD}le0DGY!n%i6$}g6xbLZqHxzoQ+lWl}Qc>EiDHnJqM7D%#4B26PUQ_yDsKYeBfU3 z1PaYoQXB))O%apkiD7K~4H@erD1ecg6WG0wLt7*yB=rx}rSBOWVdRJ>#1EuSVq%hD z1rP@l!Ayb(slk|bTu2mkB7y1g7%zHJOh4 zp!jUurh`$=snX0_sUQF3Eq4^gY6HbIG^U;&?~!~iCu9m$vf86ZQ<2=sQPKE=0Ra>l z9|iVba#GRT1u&-g$3NhVWi~zKdEbgF(DS9UH@tQxA|UVKh^;>d26a_hqYcn=)bvUR zGRo1S@V)cz^dB`E5;wsubPos6po?)$Fw?^-^EA$pR8>(D$HYio?>1=NgJ8LHbBFs% z{_@u(lB&u(AMqQT-%@mc&&`ViRg%VCW&#NdTgqX2ifMy`_5*tc0T^R5lwp(icr3_N z%xteed{5@i91|13prU|Mf(21CFqmR;{Qa}-B-BIZsL&JxHh-eTdi&CqJ>!2h2@u2e zms*Mxl_iuF3xUZO=9`58k;GB#)Hr-0ZB{_*FLOns(qdFENw?s?_Gkdg(S!r|OFhqLZp&+zucHQQmG<3jt z%kKCR4}z(LvU#WOxSe>@=E`rCNw{vP%0mXlqW*H#u zz_3&C;%dWHA&w?&eh)*ymslWNMM*C9K-m1mAa9yIQCR+|lPeZhqPt@-^HAcy_pVKc z3c8gs<>5E1({_-_P3|(pn4jWVfUzse-4@4nbThsw`R{rp(*qQ8eZty%+}1>)LIIQI z$lcg{liD_U`;%N?A^oBgd3*lS$HwqOoR6oPTwi!{bO&ZlY3|4?$4`POvkuaLA>%}$ zlIHirH(TlaZJM;H(oz9)WVmDR*k9UpjzW{8oxgGFBz#rT84C+@W9mO})($@g!;)3wIvNxS*r;R2q@#oBlT%4c+lLem}yTJ8BAUrcrc9%G=eG zkp7$&?Ze-OyE$CH%QtyT$>Iw^vS@gi4{s*ZA5Qsxj=z{9%PW*?&xs& zl(FtDar1|lJ?TiefoP1rZ0Uf|0Ye=@3Cj5@!Gc_uds!3{&~~Y!TMg~q?jI~)cfr_p zp;Q6sCWU?`VG)d9V_GglqaUs(XMsjEQwFP-y^s@sF_bu-U`fg9z9E(mg$kPgk89}> zzqy;Pzu(5?nsQERT+nES``kKxY8yY9c19pZ#+6QL9Hb-aJ_c7`*X{nqcw%{^nRPO~ zz|$!jR5OANsJ*C3AatwV?1C;WJ|y%$ax)CDCR&D+E0I8F_Q;6bq=8dvSI&ZewYa7> zmqaWw{zm+O5 z7c}U){@4^xFHDbe7FpTh=l}TG zVt;&bkIvh!P~g}&U-)q;1zyExkU(T&L0Z!v%4mG@0*3qpY-VPrq7u#B?6YcA?CwAI zSon`^Xq1x26SUX{R#sLXE19_61BTr!)Y%RvRiFib1QfdG3_K>oe3IJshDV-_a~_Ba z!cMAr*=cX0ET?_G?T)DjVHOS(a4t_-jc|_%3X?dIw1l~*pNNb-sGaleM-Gy3B6AC; zx%ey`@UMQ4AKZ#OD2tDePd}Rf^UpFd_ZF1BeE&A}2_1xIqK5bZu$RtVHP^91%U#dx zsY8!pM0H2``KrNh_Ufi-%BoP9uj1wBI@#-=Wu%C6~LkhAHWu zv#$ABEGPp~Ge-qPj=rSJjFuxSKTkLapAfl2?c4sDURgcS4y>q7CxZ{mA%Tiq{CtAT zSkHHXZZ1#rYG_`By1}_txKH8J08BzI$Yc@PiOG8(4wSeRa$3tC zf9p@Fbtjp2`}V!!how?&yLg$8)l5O)?weAs%zz ziJMQCui^GTi_?F|J3_oERo_2`lp%k2``nPn{#-jMsK#XMWc=0d{RLK&G4|5q7adv~ zzxGzq>}uX!i^YG9zui2=g=8=!_2A$Dkwr9E+qCO~qE?c%FV~L0P@PHUG&JbU!%33P zS4R`!{f+rz?Hmam>eEa&s2CzpppD3b41Xn5am$hVk&?2Lve^3HK^nJUb&Q8nh5KiT zF)o&rSi-!@%#{yBpo0SSQ{OJopOFG*c6L^ULF`lrq~>pm6n@ZYO6^vvYpoUroF+=V zDE72SHht%ypX#>zWyXgJN!3toFlUDk`MOB2kUVY$X_n+wK3_D^Ds=wm(9n6r8|Sf! z0dh`vTFZzdKYKFnAWN5(SzqZ_o2AFUL`!RR^z;PWssGhM<99nHjGipgL0#GIJ~&xv-l)A$^ognYN|U$s-q>-)(6Ws;=x)2#nmG)=hJWj6c2A?J3gf3&4A;p3glmMg=N@i9TrIh^k|Ae=A&7UXwbQUdT2Vlh#iyiXBI#pnAbG^nJ z5AyUw|17WMvd82u=2JN0o)vcgS3a$%Sd1!0KHtJ|I36oBooEDsQz5cGqjn!_PfvkP z8X6ithb1d(Yq`D+zt~^Kg)okbT0yV}T_HA2;V-?HzrC3Y^K{w~+OGNktL+MGP41&~ zORU;4Iz7&keEpwt$|j3f3Vz7hh;u2%sc*iN^9-dQskRUFMg;9PyP7)Hq27% z=e6GC{})nkT{^X{+o_5$+f{y!b<}uc191N&vFRf?FFc%YZG{_WH=zU!W9b5Bs~5Z8 z4F1rE2`KP%_373)J*W$OdcTU0LPEB;%r8&ZyGh;||MC&?wSL;`rSzq3%21`gt@VQ<@m6w-qE$r*-^E)dE+T7f9C!y_>SITXx z{`vj;YXE|f7g)tYV$9!RqRr3JVsF0zDOd>YilRheNBZ;uouivp$1yQ5a40;E5ql7V zQ$jT_aExoWcL8<|C9-tf-I)DQ9l(!>j70j0kSCRtFEo2j>7TUEE-dUvNx+jkmrm^b z;(;_71#yVcR(W4L?0VOAt2}K#52-O)96HwX@a%eXVQRA))4S{ah zP2WZ7uwc(mdm{v^kKEVWJ1gdZJYywR)vX0y>?$Ni)6Yw#l4G{Khz-`5*N-M6XDj<@ z=nfed@AjbiG5V7El;M0tu}_y9(O)N3n2-tXMIRckPaoh?lWZS4>UWHMpPBSdQWCU& zxIkRSaARPnhi3Wz>#Q0(QP(3<^~^N551pFa3^#+?zRxCgoK5K z<@)7I=zrL4L&N3nSE2^``uFz@tG6J(U}gGvnJ9|Xl$5#VFZb=f_wS6Fd{)yMG1IHO z_VSbc4LXdX+#R33e8}Y!5qUZvX74!tg&Eb!$IA;xe27+2+<_itS;nM14HZvLP9o@f z^?XRLSuTOVah%@xXhP~^*lH+`bDt>?2t>u9_*q_#;~uL3gUmBXm8+{O7*{a1Il02K*$1Fl>JJ1y z6yliP@p2T+WTAz^sCmR6c?v260RlQ0O)YJV7;|Ov0`?3^!z7poFbx|7P2b5`jI|qsF}Qe zFC;X#u)rLbY8@s26Cm9;pUh3jijkMCNG%2<4Gxw9!T)HQMwKgJqGDkDDlPr{=W8MN z%HSiknYee^AdCu&-2Y>}z@@{W7xx$1uFJo#_bBKwULpv+p#7&98q*-y{E0%wjG#ak z=>t@K!Lj^*O-Ma1|9JEwm-k|ck~4JRz14W8AuzZ%%86HFFhU#64iRGiqc@b?Xp^8c zYCn{>lZrjPZoh7ei;JtXorPavr;GS5-2N)CxIub&_p_{QxdobKLQ4PH#AHiZSxX>V zSQJcr{W|~N7pqg}tshD!Dy45}LQa!*KrKe?^>6R29vi)ptZPomkuRVvGfo7tM<%t)eBfA@azC}gYAvQMlHxUt^yURUK?ebf*&K78Ow)HS^M{RP@$=XoqY{PhSJY-)G8(Cb3^Mi`Gr!q|5|wVj1Q#@ivk%;%OpOmSgGuo3h2 z$SuKEQfnT$A%0sOkMrw=b2+-~$bieZaD(5rddSfdHFf~CHnYH0#WP&g`%VTx02cc? z6-ew1aix|(4n_!S=U0XqqAr)JLs(!#W<*C}$+G2$*26Dc;^uKHptf39bJ$6Hu@nB&Vc~`7=GE zsh!YJw=7GK531*X65d6938a3hs-{-u+&j5`tbdeY7~s@(7fCJ=vhG2um%ucre`Nn= zSpruAhMki*oE#N_C6z0D+JO2AIFr{-95l5!~rNH+Gf+4W-Tr$;fxS-HEn&o+pemr$}6ea z9mn1Eb*gV*n{58CUTrk_4L6x=EE>xcp;_kc$ZmML38^>y(3vZi&A7caKOa&Mx6=Q4 z>b?8n9MW3QE0IQ@?P8b8?CiD9i}A#S1Vu$fLK2dbxvI^jCKtKI=8m0M9@`ldWlOWFbgmdg=xSte{+i_>@(`p`!VVH;rLND>^N=v3O{1Co62A z-SI48r!_Gx$n@G8!XbW_SiD)Qrc+pjUMLBY1>Hi{i$lqFhmv7f#ymyna&SipR z7mh@a2Xwb?O_Tm@9ZZ#HM<8D<Rux*jq&&scj|{Us4iwjd>`D5~T9A2JT~8RR3EhU-XCS)feRKYP<87 z&ta^DlZ;fc@n`wJpg)Wm3~gy<)@U~%LOOovCC;T$tMmM4MnV!v z9HHm*nxxZh9UQ-lv{k#G_EvUwtt)N>*MoJtzZVvoEk=tw9v3?zReD3Q*q_JJCtnJz zjxL#|j*mXy<;xeq9jaGJNr{5I{8wxS z(Pg1T8b>1LeC}ZH8MU)&uCBAzk;gVVmgu*IwcMACxrk*6Q)f^l%&O6PfJg+KG- zU_i%_3KAh5b7(3rN+;*<%WAv8eV`M zKO&FEI@W`cRPL7V4cy$F|4|xl+ai>PhIRi@8X_mC7w2!+Lg?B))zx12D?UBeP*GI% zo*Q94X+1=_Q6S2Pdjr_v5s~lBf$?(Vxo03vSPBX;@GGfe3#FIGYGwwq-{aAB zbb6C~NK>12H{4!dK|$f$U9*bM!^-pwIjDe3q6L2F1Abg6j#H1*bX4iDqLFb}wjw}L zx9BQw^R6$~GgA9x&Ux+To+uq$ZGgf|rA$k8REy zOoMP4=h_iNjeMm^D07F}Eb_1xq~Fj&B~HXTS7*B$6?yP$-KoFR2_ok~QICuo#`1LD zvYNKv{s1Sh=mk*8O7`GmREifj)R@`*6t%8&R*Xr_?|&(=zi&(OXFBBLkD|Yl?)_!! z>uQ~<7*xe&Wn~pbbU%;Fqw4h=QgrjRmYnz-?dIdg#>SfWF`Tvc7qne{SjF@F#l#z1 zi<@f>;9HuSxU%DX+G`=fk(eaBaoJ@@((aB~m3lfltEw6VIAjxl&Gqlg&Z-(Yw6wL@ z4F2Gv2Wt)OAMgJ?-mA$bwCeiR*B5++;kv0UV>u3nd^wr~)&Wg4@=K$R%g2seQ7ZHh zu)QB2wJFB8?h@?^D_dKAUE_F9TL2~}-t@F4yV0bsE~y-yX~t2*hYv0GxN~*}x*e76 z?{K4f=ouCq3=nHwXM#o+PW*@GofVo);GoBb*THl^mkBX~qzh0C8bm0lQ^N{+d_7
    ?+nZgkX${sHY{2?5=n>kQnq&lppgTw^i8WuEx4qg4`5(|sV#I=22F znyxWAlCE2KII(Rz6Wi9rwlNdiwr$(C?Myfmo0Ftt`}X^-b^liN>e_Ww=j{DJ^wfDC z#vp)^xgf7x^LK_GXtuRZYntAIY6e&GiD-#d?lM1bns$T+=sruc?K^-`YxEid`KEu0 z5-%6g>fe*5Ee$(;j(9=-g?2)o?|b3I_^b^kdyDwXb=nWx-}nZP63iBAQ2-^Bn5$7K zCxhZpnXH(H0MmsciTb|RU)I*uPybZ+?M--l>iwh({J*luvWt$yqQEk=;pmn6&$TKh zkyb>ZFk z9F0k0A|Wxl%0eE``3Iyh`6iYN%Z!FZ%3K~X2j+(PV^Cu}X7HFORz1VC)e=$d8Wlc*Q;S+vM-VHp!zYo+4kH63TeYeK6 z+I%2ga0FO>v0TytqwfYzh$L6xMtqA~l|S&5)VJ2Q$sm>+NFi#ixUt$P7q8dx_?EY} z*U>Olz|;}(9Zgo0bITt}H4Qw!K0shQpM$zudxC1AkyS%5^3|EM-2TfvA^Yt?{i|#n zQn`NK>-fZNbPLRt+*G&(1aZjabpKOE>xycj+dIRL^@Jg27RQLm-&$Ad$4Kp>)WI2m zu)Enjhoc<&Ch%p0pyn^|VyX5IBh%G6JKmu9@feun;5FaFQ51nDs05$n3n$204?W>G@x&ubD|SnwXUPFN;UT0Dr>E6cPv8Wxe|G^z(;|!Os`B5`CMYEMNEi0} zD%cm7+)g#xfkw;`6T%jlmI)DMY5_weWPry9xo`mO6F(^a*QEKb*Rgz%`V5%4Wf!AF zxFD?w)D(CF*o1|O5i<^?6-w}*yNUTbX}Lr$&K{kV5zT5O4gyS@@m!YMvBV6Kop*Sk znV%X>2Es9bEXV!$d9Ba09xYyp7xywD0C5oh#BcV<-^PkNXV>47G=&F` zlJI@)5O!4<_w1J`+iLwUEoWUzPh*GY0>7$ma4OmQc%}=XNrkvM3|~vbdAC&u-M@=3i$41FlLI;EytB>rVZtV z7k$irr%h}n8W+43p#tT{*AJc1|S^M+D()uion@7gnA~EG3H3Yt-^+?L>_P z;iRDZGGPMXM29GdxLI2Hv}{7C%}gyd2Z&yYlCsg`i-09DW=@*1Chqk#&vt+|CDFlw zA$^i$!sLbiqmyZI$ID0nbTE{?EbF@7*Q`#B3e&&nPY`mJaW zbix?2PuQ5%+yFoptE2GtLUZpSm(l9|Yjr48)=t!xqdi$zEdDY!R4mh`%hEv9KB!8n zNpVUpMP(KVyua~3UZp9`z{m(N(FyYu-)*K&Eg4}Q`nle!D9j?KkYs;CE=QfjerBl5gqO^7I73LX^uv>CO8468aFv}bn zrA4}-Ge^gs5n^xglRB3?Qvlo2qDDoeUdn}=C^ZF(V&kqv^~k_!AtmM@)rN~se85Z! zhYeg+MrLhdy$UpRcP?s}IL{s=I>;GZ7m@#Hb|oKxY4%tg!2yJ+ARh+XF3_puh270l zmcIkurZUU?FXT83L_5I4L-PyIO(RyXgD2}Rz z7+Pka#%>NkdH~(R>sS*ebXZxOX-~B*Cp93A98cHcUXx{!IlVkCxsd=7s1; zQ;{e=R-?^^97JN>VMHJf26=%|3^^;^+W#J)qiosLc)C@)mrVmo|6tU@0GI0?RV)WE1}xgN(y5H-=u)#a_8OAF=pb)VjBG#jI#*(BICjkQ;JcsLNN)xa9m5_~UBBU7U8b*)DSxR+T{>dKg7KMv;geX9G zw*->P2ufwWQHfhmmG3rJESAnHm(4f>^E^%~iI&@LX0=KLqVMU;cf+WAASND9z;0u< z7_b;n5fh@2X9Kb|$>mx4?hDGAc6Sr@H;C8pEu*D!+c=aCvWS-IH4D?@ft#rf$kL|3 zC4vt?gSIdloWYR74vMT~MMk3lxPi~FHSfC)z9>5TcW(vPoj0IoQ+PsX4Wu*-WP${S z+0LyQX+YEjS@cXyeerAsxn1$By&9yYq@I=EC+>t-L0RWllshRmlOD0(?}ZuQ&g7WT{6T+(*Fd6 zt#`7`>2>_uRBj~x>ExU=S z^)+N9U0l!(0H_F+&d zO(bWBwe7J?V`}j{ZWa~O0Mm!?@rz6S20fz)4#y21@C*p$zypT)=vOBv2MvHwiU4Zv zk8wDv2zCjvbW`;HKQg78I}3SjA2~4|0vE^(anmnB8}?tZkCZ=ZF41GiD0`p3hf&Jm zfzI*zx%!f;4d5}t;Rhp715*)`qCQG_{Hf%|-TUjmX#x`>3G-=P++gYARAa}oVX$Y& z$sgxSC*9g5E^^l}T2`!5y1aP@UPJ!kaO0hZg=KZhmow>c`yW-gM&&rqzCT9AswuQyAf(19Q1a=@uM4B5l z62->|${Daz>&AmXz$h~n0Wv2a9gY!Kskn6lDXC@;4bo*ZnizO`fypb6?$PT3H9^;} z-xdjf{+25WeYAW_|4yiKmFK#Q3ljXdy+Tb;z6KUFy=$q?47x5h6nc&y(Wsd2FnH8S zV>J0Gj^_!yKGZDyw?momYm)vHAs%>izFrv0ySI`~wh6i)NDg z$`J9nsTC6!`vxS(FIRem3H5dxl1m9g`~E=vG_4NW~*9GyuSOKy;6Vv%i0{F zK=^B?`!@CO<3664Mq63RCu&e{21nT--0}r@Lhnz~MK;uYmhQ~q8g29HQZG4aM!2r?OzapBmSm_gETz>Foiws9xyX zf6*>!-QWEYIQk^e0;B$4i*%M=B+{PFlm2VXsb*M4-xdFwM?t3rllr{QX7OKrg_!cX zT*p}*@fs7}fBlVPC?1>|fi|Fw2Y%D~yMZKRzdN{gbKfuPISYnUtVoH!2ZfRV*`C%I zkp;^jf#O@Vi7McY%qaRid>WE)bx`S(VCw6C(`{L2fM6L%EXDc86D5c<%&o-*`m;Qm z_F$a7y)_K1yRTL%24#@qDZVCcNAt#1< z+l!lG;I$I8T-tKsj$-g6oO=M2rei7?O{tDF))8>O*VFO~A&{0aP%pMY6>NOm#K zu=%KV>#fbD!=N_1cI|yEuB~>9wq7SwEyy9(rKQz^buDu7y9aV(%?WmzT|;C*hh&jG4h}x-9nh*6aK!8xoPJY6mJWp$D>&)g-?hXH(V@_V(WnEnzOL$CuZLMN?PS+XU zxIF$oNHaa|@xQCJwiSw{xvyEjNE#}E6{KQeY9QWdxhlC7LVbPx&dNHKXjHCv-1;Aj!XE3S&XZtGL&X-{TNSt;@~ zNJ?F0uT|A`UBIR}wa&_c$0oLejspe0gL|RRz02P2w(dGP*+8G2IlGO{r&V+X(80H& zzFz2td>pvh_V>vb71hDj^&(Z%|FUV$k1AC*F<`W%4cL7%&h>E}F7$b$=;w1F&((W3 z%Lfqllargi6sWb#Ut%{J2p|pqmghWndR1q*1Cn;WI3Sbi7OsSfEl)20P*ZFs5xxMq<}<;sfNpU zhW;P1AUjj6VJ6zHB?w9DUfZ`z@^b%@!otQzTnba#x7G}Ke0%7YuqrSZ$#hiD^=04&K_jG{yU(G z#OEV?Q1JTtN?xw8)oopV#MTpthWPMsFc!%42gx6Rx zT>tfAr4*qq@p88*2%{^=_Fq$NI!A9<&vAPLm;WFVoORV9UX9H)g~EHcIerTwBZL0? z_irpQ7g&HW8$*fOS|XPI+xaOl?Se0k+eC-^lL=;A$hxDl3M6xNkzdz;EIYopt)LdT zYt3{X&F6B6u#&;lD?pDW8_aXjaXUf@%3 zR@7$(H8=<34P^-Qv|w}mUa8w`H`*R^-Sx_OFFI(kptcqlo4gh8Kwm|phr2b)oW8x> zhg|&&uxgVskWZh>zXZ!X-$7ASfw}2+_;DyV(vA~$O(rc^5%IQ*Rp;kFxULTn&`4Gp zyyjDnw%Hf4Phly-u+_mxcz!eYr})eDsk=ip1YeTUyD$VR@`^ZYXWK&C9}oU7<@KM_ zD<@2TM)%K?aoHA0OM-qX)_1cU_4R(AqnZMR2SQ$~0IcGXlYmlT7!e$wPu(>>JdCpVn*cOYwHEd&vGi2OzYu&)x5HuAXFkg@!lFF#{<4 zJ+0Ki?(^k1Tt25i8Nn-k%^<(~)fSU(Xp%3d<9A1sR@`sc+-$H$KE>^}@MDG9D)RF3 zmbt!E0na_xE%}pXK#QdMX5IFyEY6SBu&}VPFtDX^`3?e#NDp6$r!MQRTV-Ws&a?dR01WK&{KIp7+u{)#59oQSmjA~B^rf4( zm-)@Z@F~LTSo=>uZmGs+Dd*dB9VEANdseh!Kgc{51>hJh8gO5MhOd6<1kTP*n&DMj zZML}e&J(Bo2FF8%@YQeb+X+HlPJB}ei+<3u7MI2Ax>J{n6;1C@M&Ks?5ikatr^TSC zo1MP}M}WUsUu3oRhlsyjW5= z;sx8<+T3;rz*MOe%+Syl)^|F~&DR&k|NQ-n@QZ-ib6-48;it3V_{Pggo_AaJ?3z13 zwo;pIPUvaVc}p$To}w&2Gh&IORa4(})I8mH%yrAeJH`JO>CTlry&Dko#0S_2dnYYU z=I=2&J?+h^9WUmu)zQ|sXKnbF_3%v#a4R6xY`cn@y`DD>r9~x2tx|`n*qG|Y^ z5?lYO(n@-syc2SE>u)QgTHhr->QcDwOLkgw4N`}|;5)6Z?|ILnXOgQ-x{)U%vkKg9 zF(3a!Zuy6c9RZi!;cPG5cCAIZCGyxi?>!Vs;ZP`HK&CTpI4Y?3ae_0nh;!^;I5FJE z+?-Dvh;ZU@tIxWsx^y?hI z?{N+!r7slrNrcM6^*hgdfil0UvX+p!0Hjy`bEC}$)mz@VX8WJ7MIMLur3AKrQ&*YA zK)`>?=|-Dtb;L6?df5-bix#$DzpBEJ@mU|P$vN#AZcpH^cS}}fct$8lq%gY#Qk5P2 z%s}u+;RcaL&3K!gHuEWCsTiRO$H{l8CMrfcel<}urg&2@ zvF7`GGpo^Te4)<;MWyq!So?ZZQv~d*>nByA2h|-KCwUs~oXC`O5O)rVfcttE_1l{H z@TGwFwn4W@I9pFum&Ia<%xOx{jqcOMR;x8K{#t|G$0|!PH8IOyN1lyj?RLA|m1WX( zf6I?`$6_hPJl4PR%HGx+y7skmUU&fq+{02aIEK3C!d=~6%e6c6n~N5^C3=?65vb_r zXIek5dpa=>BPL#Bv&UocL4`lmG0=9R&G%~CV#HUfSR0s!z=p6S#BH=ki2z`s(jesB;>y3ijJH+Lk9QR-v4&-!+k~3&t<8u z_hP?748$#4UwwX^N{^J5(yW4|cY{X41zL8es$fOUfy|p+59bSm`>&7cdH2CAPG)n3 zKH?4CT)d;Pgt)CQbL@TZ_{Oe04_Y;e{EQ+aU}ys{_=54jy&AMm%O+!apt!V{MMQ+8 ziV1`XIKiynb@@NNy;=PJ_4?Ys2}suOy*>Jt&ag0MUnNXTJ~I@@fj#`2{Smv`dL3%zs4N297syB!zbAV7{xI7K1YJEe?A9c z_|mNQH6EvxLSK=iRNr736fmAqz#$mB+jqRC{Tl;z^ZmCoK?qBv+q{IxL+F4#dNUo} zW@P@?`xoc$xm@v&z|}s>tIMpjSCrNfKDVn*;1s~ zJ+B3c9`EsG0g$ElpFa^+5Sw4SrXb+hJE;A3r5}I)r6=-Ic*0@aGB{1IF_+8tG_y2oX6A0U;jq&i?XyEY z&f(w&T)$gOn4>1ruA;EsaNW+4J2Aj$t=15yh=7yOwH~`HLPrR}rJ`KB(BqXuT=ka|d3;wjOLK6CK0CgB`B0^` zL^%8&1?j&#YJ8qm_I>O`c*{|KLM3p9r3=1~m`SjTa0!deE^~QxG zJ!C_~Lw;a&d=-voTg<&KxA&hn^!v#(f3^;s+z%8Ehc9ai-3%m;JuysCDbI?MSZT6e z*ly>i^|GI+uIL%0qA_`o*ZZ{|=OTiJDY5f?%knW6wJwLl>coXNQG4bUQa=Ar;A-Q} z^L9VaKFj+G$V+U+{nN1j->fz&TEltCSR5gr1+XI7;)nZ)rHA#l&RewZ)xE%%d%q#G z_416g5L4@GrYbmx$9zGiWp0<*o-d(*KnXzv)_l+?@?0ig_562=!5=Wpl0<&M$xi1J zzjIx#f%_W2hyVJ|9QTj(T;KH~kMs948*4xFm*(kw1FX1MqlA!EOB^E*zU)E0#d8av zVeITASu9`x#IdY1H|6&s}b1p&YTpj3|t80hPOc zZkIZr7;G+dU8Nq)eK&lJfeF-)UzEQSAg;wV|&skZUVIHndL{-o@d4n69KpKUBWt$4<7+)-W7`tgcxro2;gn|K z1Fe>7Mk^EH9(?Y=&r|>Rnr^;if9zijzJ3`!WD`P^cobt3Xw~8UA6pnN);oHwRtxAP zrlP*ypz}$x8L(%b$}|D)Vxe7{wZ7ixzr|(pQjYG-&i)SWepUCJt3=-7V58ca>A7Oo z!Xq>j}B>FyHcb?!)5n2!Z_fq{z$Njo>{l{^;yZ4yy zzddS;-?^@0Gj?NpGa@H&3@5d=frLF)L>F;u@Q3A`aj+W6r#AM}f*^udTkpQPefPt1 z>-o0lb)gWKJ~y=ilk#%LeQ-R#=U$HXPRAa!`TwGf9RET{z>3{Q;ICe>HIiT|22&m1Kg+|2Qm#X)0(;2 zY+p|tOwVR{E_CbK+1lE2Jtl-hc4It(DnHA>4`ds5c?b<^+|>YC-?4!^y(_*ZRPnK2yW;`b7}x zGI=b{=Xr26FF!S82?FM-nqzFYkvh-r*RR)Ou7241^S|kN-k+J}5sAz@SNgm7%9Us5 zUvok+xNPfz3IDD>Hl9XSzI?}X?R$Hn0osS?N3-SVKQ`(nzSvLl-Csc_qVUWOn6*-m zt$0q zN7bdv*jPN42uxjHpf^1RnVWslb$LHbrRTrjkvCKXz26w^tb}0;TwmAlIQMhlt6nTIIis;s!i-~6~6A=}B8odrU-{Axz= zdAy~+>Nr3GEb!cQ=%NfEBLcAEKA|%d1Rw4filyc<*~+V{&mTAJnQ(|1%8HAN%lZmE z{t+;b6z#3GqQHOq<|G#3rpE!3qF8spJwBJ=wb{7zjT2$U#Z|Byc$02muy*l>zKapS7iH?=5Y)1(xo2Rs_6kMHYHCibs&vTru#!keJT%dq2|LxlbTs z2JTi1zGWiP=Yd`s*`i{q{LhXG(#Ib~V#y>UkmsSTP9o&m_+;xsuhpc~e9Z1Y9UiXiWeeX_~Sc||r2dQzepjQE51o0TdG)o5wuyNJWS%j{CIr8=I9!S>JJ?FGsCpG&`)iK-B3&s{4nRjcFZGqvLfXr*(3kp&K{%#W9W%8Yaq~+ z(yBr;hRcv zxws1{sT#U620qCY2#WT}-cR{B z;w3p*$N(zyT^yP)W-kC35DbJ7)@DZS40R*LjZ|V*z~E;wtB~bB604DSiLqBtkfLLb z>BO~d9{~kAZ_XIh%aan35{4$9f=UO(4q7Noqe+2ak?uzOV?BWPQ*5gF7Bznq`9Vbo zW`VoqHQ#u3m3%eA!Skju3x5p8$r>{Jp79zTH;7bBd}?4JfJRD8_pMa)4c=j<>NvW^ zg9sL(QR?SF2j#JZosXz>34C6-L6kiCEco|jQ^BhDj4u?o@75i9?8LrKz`JK33>j-# zBI=3GUr`0Bi?t&w(r;=eeds6FpL*ET>(MH7vwo$)J&2Nn&LgVOS>PYqB>5ei1e5DG z2gT5F-VYq<{V&J(Ja3dh<@E$KBocnv_Y|`sKbfO7L+kp%p{aIixY6cEm;gzac=rHs5CL3ow7+e$Ubo6b-KO3oWTtZNW;$(bC1GH^Y z`#}he$T!m5>GPcx+0t4NVGzXGyTRS?EPf$^eKVe0uG{VS_tob&B6U~itr%QYA`RIE{wIVU1c2z+vj{qmO@ zZ93D(J^#6X)MU48_s~OxbdZpJhIrK5G|S=08|8N{R7a!zBlu2%6g0ZtzKDnVGGRl~ zYoXy6iMqy2b`%&X+_p@t;+ulq&TE2$;dObpW(OOdM52lI;2MlrLO!g3fd!NV!uBvD zPP@&CxT<=5t(-X60U`evdg-Xk#s)U~S)g*UBu^CAI3^&njl8Dik{8uk0>HXL5&4A>E9~2~uYuNdQk5kUm!h@WG!yJ&ee4o$VJG@C? zDkS~gg7 z1F1;cl?qERCyNOTLBWMQabS-aIw2GJ4vn;UD5Hg8;}AQzj2nO4@;d;ckffGqIIkW` zE4qStD=*|oDM27F`3p(sn=k@Y)J9v8s65(>)(_Dlun_8aBY^ynI0AA&V6cOyX-s3c z6N569YMX8!B+WTxH1yich+C^Y!LyqjL5a(E)0E!a;q4kQMvPn`M0l{F%>nQrD@>PI z*jr+#=pyE9ClnsUBy`k7G)y%#{AgdzP6AyHk^!LS}8b`kGn)-WIPlllx^0G zm{|gWwV@<}V9--Tp3((IKu%NUxFK@ds0$t~D_GnxBns0~{UY>C$bFF?3P><=5skPq zi!hE|$MA;FZ#Tt~LoT*k4SuIXLdlJMGs1OC!z9=-^pDRateiWk4XZQECfps}5WKQW zAh{*saf4VywVjGS5id1i-c~+E8ayk5Zigqh zF>u4?t}npMJax z16V->Ub21zO$6XJI1e7yQweF7v?vcg7`luD3kXLkv&tQJv=j+C1)v?n@S#j1nu!nH zFkl?=kpmh}8d;Jt_UIsQ4==G82<$cN+-@pEosLdk8yk8dSssfZ0np#MLmR2i|OlgIP3*W#J4|M|) zaH!lyn?EWFX)1Epwk<>?zoCM1i0}#|Xyg=!WDOCc+IaG-xf);D%rvGX3`LlpDBjLN z#gid-7z95#0dCPSVF4IFlJupiHkh#Bj1}pJEhN!|THr;lNBi>g&lza~_n=-grNQ-o z8MVS9R8$y-MBIsbizG-1lf}tzKxR0!kF`xN3~s^k^kNKnM*?9Lz(h8@iDMd8-_Mmp zcnPW<8^^oW&&1#oa7Rd3i*RLsq@k|X4}2Dx4G8ql=$(u9;^l}hKRU@cnEBV9-3{;R zCvHQC#?$3&g?i0o)DS`nE3_dDH##}pEkwxSF?-&zq$x6!T9_?4>gbBZeG9_4&7?OB z`uGE$!4H;57_$~VFEFtql)O7%`9K_W*9Yx3&B|48dJt$!uz73h1bZ7=D8nD7SOm6s z_zP|wt)mv?5o|{O36bQ5jyp|P$nPa_X<>I0bQ<|*3a@u$(uv=uhQ02!4QSN?l91DjF`+~_}c^GgYFN_fm`C+2<3IobT*d4ot z6x_Xk!jC8;3?ob1cN*V)VJ5{EhXpi%&iT&Q=MyOJMf=PXx5bB2>^+LX7(@QYK+p%7 zW=oR1Q(EMPuM;v>MoP2D@U1fc^=JR&_rimPUhY(04vim_bLk*EhGlP4w#83}T zI~n7!lpLy4sBUauumUS}0%eDh))~H1^U@-*XbGsWtC9Bu1NfSN+ag#Bl)_!ss0Oq1 z5Nz+v-pN9hGep&9A$y&Ds|A##kDShQo7$km-kXt7m%+hDy(0;niXOV#E`}BO8)@x( z89Q~X_F7KLJozn^rn|c-8Ix{oLGMzlAnaXkFG3s;&cT^wM5Xq-q0>!0rIL+rft5zu zcmBdUoHm-Gq0cGzYBezL!lfU3S&30A7n2B@oT<0ty7%tJCu4^6s!o3%=TjA*dqgcz z#8g5Uj`Hxeek512>LZ}f>fvui81m!9Ut&7WQaoCzJeS!9$p@%ASn>9B7jt)|qsYKU zMM?1nxx*c(LhW4U7kHJ5XRC-)a>!i0t8r4jN&6m`TbUhuaek^A*8VWSd`bH8XTuTh z0@dV|n#{Y@EibkL2shZ{n1{0<2TZiG*+|SEIr%i_VQRB$l%0<2B?*(};mb=XRm*|9 zVEC4u-zLQ#s_A(z8!1L+7)JTv7WuS8OvkY;d!zKOuzAAp+c*yiBQYouMg87yrdYPN zfw7-;>o2?zss;a!ff~geE5=KHlE0iQi`<~1q;zA1l}W7Z$D9F$g_8kT)ccF<>v`|L zVQzK7Pk8p!!#L*?0j9Q5M_Bq&2Dkd>a!@jyj#^!Fr66&>uvFo=slgNmK9umZ8&m5tqKPuN1K^ z{AhC6TTEFEmf*iXF|WyA`0j4HWb2F3I?OE+MJw0mt46jN;xMTsM0sQn2_Hox z4Zh0`q&zzfBG5h4dZD%U3QL!ASQvwZ`KpBtD~AA4sKxBQ^r||5@BD5?B*afEd0g1jqOnI zW2P{5d526cls}=(t!`;L+IihXg%&ihB#&ssHJ}mWZKQxpBZzous=7PWp)A_h^sr?Cy=A1SF@(>?r-~!&@9MA?x>>$uC%X; z9fkJ2j(iVP4j757>A#muAWmGkr5rO8XJ)vh%4+G%ios{n#wd+{T6oi>(ls=X9*i$z zDqiXt(5EGvMdZc+B}zEav$fM|GzrEMX`hULj+4LB##Du8Y<7InqE(2iD@ilGb>~pW zgZuZ*3^Wi!&~Ti~B|`0!=Fb^i$Zb!fX_f9?y?f&28VNTF%4~`>x*lOB0hP6)$n_Cw zG~o7?t6F40+i@FEZXIzL;7v${5zg_n%x}#wW@=04Hb3~;iBgGySNIhTbL+(^L zAT_YR8VxHhDuU8ps}vD-nlD$RcucV>sCTkWv9ghPIQpQ!o&un#{y2cDtyWSEMQmv< zaBwfl*k;N#oX;+eVkG*4R3c%cW5a=iqcvri^4Hoc5H+)sLr#+xN`H#=#yyfCNZVsv zGZ|ZrxsndGIaW8^PL97x#E*5H(nrSg#sr)gx~iAKV#c;>MLR{sM;TU1*OZnE@7d7Z zvLszd012aN49GMmK@uv<`x-svFB$DMbSzK93P(AH7)#2Yk7S>T9D}3{6*=fSvsn;S zkE%zX>bSACQ+?M4aEb*%MtU#qQPoZTPnC}!b~;eRSn>XG-gu2%l&tb09iJuQhGc?% zCTx`bGiXz-tno08-Zw;>TY{&@C0DiQ2LJq>`g-FW;vRI*y}}5;vvR*P{82^ewQqw$FaaTTnwJ2-YX$JW zun2B91;9sH!T)0c&OXEzbS&wOL-lH~;M9AdjFnj1N3V)Uh@7rvD(a@}uQnzmdW1k65fTz$)2nrfTQ)mVFW3MOBMM;JLI9@qSmwc zG`rVm>#b7>;Y;?^dC;Sq8(A}XG>hj>`jQ5U(V27oxx*2waE4QP!2V|o?F9>li^UbgIyuvOb8SeXMjzF2NU2|mfGc+Q=0A6wIE^XfC$UC266N0ve zOHm{7^$HU#P|2C=}_%b z+^5d$Y-y9>H1_a0@9rCD}Hdu&pN|?R!sy#AvN_(=Cr(GL6L(Ja-`^ zgV2%F4CcLZ+#X~dkZ*D+uU>? zIz1a#|LrXO2ph7R;5J>MAl;0hFAm>NRO!_S3MZTgiOT`|dq2^?u)I8;Y8ckfA6E7Y z4D@XprT28ypX=r5BQgjSz1nMDCE#@8!=ayH-#=u;Jx?IKq}1^vIUA!Yds0LrSbwMh z8`PLKn%RyX41A8><_teouJsqDuUwPcKp*J**VAB0%}*q-Yqo!c$*(8{}YMDM*zQyev=(r}vmj5;Ey zvC7fpRd8CnFS(8n@=ESw6c3k46vIV=rILAWyPB_5!ac>YZCEI!50|VG z<+Xy}Td3)^f_9^6PI{Sn2cdK~QJ#?iz*oc1FR;w4~EHybNp59(nEsMlib_&PJwBjia zF_m*CFUbh8@dCv(_QAT;JqQ061}V>o=AQSwqgD}?)L+Vl(3iGEqvv#2ftr_&l%{IY z+cJ9b_jO{j=lP;j`A|oSau|CwcV1Q3za0#ypVSV)u!!>hnYS=ct#Od-pzL_uDNbYF z_or3Zpw0icp7El{DB<3|L4d1xA>XfTX;eXWy9+PvEaP2<+jPeBKk+Esw-=QJehdI|u}78QTk4|N50k<1?+vnCk5AP_eQ5+=RX!J~SBIKS&RSKEeHTA8;+W|q zH8KBcp+#z^d3{uMbWdNbYA&Y1-GPmMZuhU63=01Gv)EVknB+;5N=r-8{VlI2UDx!zMkhDpg&vkkiHEWnTK1%4v;V=2wq(G&l-uO{Pfkix zQtx8f4#s$f%Un1oh0q6ZiGwySM(lQm8&=RMAbEpE<7cI(&W|HWkDoqIekx>bG(Y(d zf3}hJf|8EPXrA}!v7ITBTaT^Vw-volf{Jo`4syS9w|D9xAGD*wrx+~ars9X6you}F zGyD5VQ>U(b9H&k*pJ@XZVU|birb}DfMbdY#Hw-psq z4C2+IV(LO($znh55?{-T$!4c>;%WVvoO@0gP*5YcPVX1cAFBfH@56n}sU5a~EAws~tAMR4U&k;QUVXrLP4%}fsk~+_2ANH~H z?(-tyy_TkrUL~XkHCsy#MXV17S4?oC!O_TF105OsA<1z#FAh4vq1!~5qSKwuO6I?; z;e%)+V!eaHs8d$TdreJRHG_bRNUSFhJAIh_8od!ZlT-93Lnz)QL{Iy^Vc2DQ{ z(GlqyXB)R2k$MU_i!Y9JCxa+xcQ@)mSZlge&rOSJb?M0o*gUdRe^ilDRl8cIIxpx`MbRk#+4A`T>!D~l-m&?C{H-65O(x`Qq@Rn-aC?}k1K}WwX>c-8r)nRRf1}qQ|!(+$+C_Dz7lqiVPp&? zeHOaaA<44G%1N~~n%(oI_>gL9k|cJY42`9?hMAwY4?4!Ljr~Fyf4J&f zVm5(R#xTX<%G^Z=X73@9Vj$7Chf?fTCby=B03|y}teZ@dIfL_DZsYykK?p$ zh=w3j06|JiI&#GDxL?}j*#U*XY6D>Hw1BfVAu&YS&k$CGp3{C$kQomEV>Z`DfG7Vm zFnN@o8c#}d16CcYm9Vni;bXGS5jb?i_WU>9;E#@69bES(k~({Ufl6w6-QGaS?OmNo z)SNX&_0;>zx$@fV8G5hcd%wL}P(E@O;mf>;_FJGgXbOYO)5W~8ny`%p^Sm#4u$&iV zsJC2}#}vRhVfrr)#Hq~u}0&d$pebB~iy{*R`!jB4Zi!Zljl zX@TPIg%)@B;uMFV1qu`^?rm`|P#g+HiaQi{FYfM6a0?;Ho!|f7yH@gT*37J#bI$Cu z_kN!j7hs@0OvFo!qX5n7lww{^W`S{e<7Z2)Zin&fiM`!<>k&k#1xp%%!~=P0gH3zF zvHU+miBq43ulU}yZ{g0)!f1uy4IPZh%!`$(!H_(JyUD0j+M!cVK){PsOG#UK8?!!w8o4f$>N9#AZ3u3e3ptgecw*OZN1 z$XLp(sJDThN=}ui_`f$|LnfcE4-jju&Ia>|^Z>VA$n#o$*Sj*v7Xo|O+}yC{Zvl{? z%eAy3G}5CsWnWG^9-gnF4h@ccrU;Q@Lq}cj!s;_Wb#xGLw_NOuRUZkKZa!OX57gU@ zZQvzH+#Bl8mzlJn9+OA!RWO-TEiO1P~pmGek~e&lv}7WpIzA~ZK>IC_80rE z6V)4swTZ+AWcErbH1G?%X1qGUY}cI zXF2-HW=>1yS@mokk2wmevA^p4Ss`4?Sp{DMkRHYTkrPgOgaF1HFcX`cfg&1LHpGKW?TRZZWUY`)W`ESK}+lS^`xk8A{g9LV|o@6V1SH1%2PKkmoGyyE_J zF(WDR;SSYqGBg0J(> zGZT#P**Ve<^f;1`bM0pEz*|@4Lm>Hh3A6}NH#SR4qNEeQWsDk}{CnkSQ2p7(MJR+` z&{gH?xFpAYkb<_nT-CF&5xzGh)ZFNGwAB74??&uBO6hN8ebJ!I5N3JM7c<|0FEpQc zw3M)U?ehF5A}!@ET2IeiIralN8N#s-b6%LlrEK25t0jKL76I{CG`%{=T+B1R=R&J# z-Z5s*mI@YIG4WjPn|^W6FXszaM}IE=QGb^KAXxdUIL=vQ5ln))JIM@ZY!Aivg*yv+y2pfw9nb{_wDFS3 zOOKM5bCe=Uq_7b}r@RQXYX{h&&a{&ld%C`BJ5V*K!AE{Mg1=hUkW}UU#~t@MGWiqu zVK}GyzN7Djo@DEf-#?(vMV4qUr;vsAt_{K%vlcE%q4atYQc=1{a@sS#-O8mcY9l5 z+MxHs?_k+ADbq^dk8uYIa+R$ zJtjmSvi&d2oRH|rctdy?(CU!T7;U;M=8p3oeY}S{)65NNu)j;VW6Uk)daXDRtaa7= z?ue77>u2QAZmYO#SZLk^sHu*EhHyJG*B~XOG@VZy?Y_?IIY2>!;8S__n3Tln+^mLC z-K{3Wbp*oug7!S4I$c9#dssIKeZVrGr$Fhlx*f=8TA_LU9$Jwo_(Q`$ zGedkkmCPJTrN*3tC_{h{6N{4>SrLo=ehL$bn5Ze=W=WdW5d{Y`+yccY)v!5PI+D5j zk-Xc$3Kw|~i7OsRYI#sroB=D;0yz(bMP-L1vZjauZ8TK3B9e|yhY5*7`a5|qAu4&K zE0P8G85t4^Pc&t3>yKAiSV|Tm_o?=%1%$I@=-!)w-qGk33EN13EzQUb1`UZn%3hyfz!JELNUowrLc0ZcF!y0*d zX{u8;;(%`6~XTg*M+M56HFC;`x2%r8#Mx}Il9!W5K2yovx0|a;%-od6QmeS{02&u6& ziDV5CTB-ehHAz@02!8wXpefeUZb|0%&P_@be>$^|RCY<)n%C+@+}jYsBuzkXd2p+% z8FZdeBD!LNp8bnacA7%$4vlN7-%6R?^M2WGVXhbs+eAo=KaHyKh~CZVl5q!StKo!IMr-`1*8347(;;&BuHnegZ_9`*n=oLAiD z0f--(Ab!r(<>#p#^iCJL#>hsYH+g{8rB4~NAQUepK8}|7)kk`zJ_=!rZWUUIZEtV5 zSKpJ+a^%QY8SO7TWYrNQ`2)=TvM3?7+D=7Ss<>fG&UlpeRaE5z@<4McVLa~-Mbf0Y zU2eUopkYl7L_hCAe8}$l*p3M|y`a8s4IP6Jd2_9=5*-nP4$IuC$Uos!^2r^{D22Ib z`hxD27vz%x5II9>mWpora^wUwMTO0@Nr03Mf-ce~R zDs@%qCY`S=Zpdlr7K_$Dk*^|$b6-Ykd>A^5{I;YjIdv1N`+-<2Hk2xVW zdoZq3Xiqd=^lFLGj!L4VGvDo^%2`vQ^~+eXIz=Hd3CZ^rDa>E@4hiM|Z6ENzFjyk% zj83mC4)P5mRYVDhc+StW|LhQJimnENudIhgg4f~!7oGTD?Gr<~C;qzah6^kdneIbA zPUcF)@A3-@V(XrE9(W5?2z3&85zqu)9$Cs!D@aI~xwKz>WI3g@BG&j$6qeGAQG}Vb z&~ddj1L-56NtIFkCC?wl+UfgwSy!wH{Lf~pP!C!m2^+_ogTt0CjK$TDaLM;dEwgQ&<}%QHEp-2QoE-F!wfZi%Le${b9(*3F7C8s&2EQc+zU80GY#pPLxM||eHD9_20mpokN?SCsx{0k;p zemR`|b<3!_uf<3>I$je(Y(O{G|BkcT4PPdanSGh=pYHSiyu7s5t|TAi{eXx;Qi5do z8FrVp4rbJ6#*0`*_rX^_@Qj20{%Mg@{$S=i{?eJc5)}jYwTP4JO}sY(kOQB61~2UC z%@1w{+roR%E2(ooHkVboEmRos;Y-bhJsgy!k#QNXt>xqCWa`Kf&^#1XXS-ic^;~o| zUIVSdf2-GTwhK>Q1B|nH(E#lClc&{OAixSiNzi*QQHFtm6my7I2$#z1YJiMZ@`{iw z0}yxo%je!N3&tm&>Eg z=ld!-78*bB9YU+9aIhiZ0z-;)2as?T#TfG;m@8~K$|A++SAj@ zB*@!=KRA%y^*z^H_m>1Tdd*(rWj}xBsNQzqoYRUs{bf=5uH%WPUT;S&u&=dx@xT-K zhr4`GGd%#j8JMqn{Cf7f;{w69g*a@!xABCpVSaKm!lRk1xgSsE9&bxM@_)reA%07T z{5DoLK~^z#GvlphNBhYVi6+H69}APyS90V+4ypmUC<0tP;sQT}Gy&QL`5;DJw&eMd zwc|^DDFiNEQBA~1GTEpljYP$EszL6B{qibud+PyMZZICB$aJTc@ioYIzojWqOu>Gn zFZi{Hdm;Ucq)B0$>y+^)Kp>OnJy&GlMGv-B{M+)tmTVfL@3}2dy^p3s z5r-F5f1y=kfYqJV7S%hq-V!O_5wH{#vT?9bHrP{tXznwYqEgYeVF!`mPIAgyqxfm~@-66jbVxCd zy!8sa2sjxfK#l7qMcg2W>yqX#clKZ9&dhqh8;>Zh~b#{HIT~^!}~(YPq4wV z9vDfg%-0TE`UK?Sb{cdD0ie2VF#;Q34;b!Gr>XHfTc>VbyUwZzRs8fg?1p&M1dueZ zmXZzx9hF{Z{conauf>xK+vQ2|fhISbEq7n_{|=I8*v@#dqvKWSc3GpGI}Y{YoTg*k z!6!H7va{ZLH7EAp+OyoCehpqfNlTNu+?uJ19eH)KiihAsbmC<-IdU|4-z++>yjn#P zyR1?8FWGr(@M|Y7o4|QPf}Zc=+EMu1H@{TET>|_`!~&M4Ns8#Lf2D%%&H<2kGc>-c z=@2DS83(6jD9>jGix1!b`bx?B!a?kcQE4*AbHfa9k*H@zeVQ6A`|o>qpM~jaCX}N} zE$;rsz+1`DBj$0+rcOd)Fums+hMm`0t>ss}QF;bz+;9BI6@bFm3%u584PD4Y{>Gx7 z+2FD`v1m}Kcbut`%z{fP>H`2UkDpq%`|ypMz0~cL)HqVFVNN?46+haEuo~=P0GUCc zw)Th^DGGt1~ND7Po(rhNwOCCUeyR=W0zY!InRzQeiMlL?44!sH9MOM`7B*Z_-7^WvuM4@aRz5s zl~TWJ_+3v~aWbO^ItBJ!->8PR0X$cAps%m>ikQ<*zwKCV=$;iniW0j3_>ytOIFl~} z7YU=zcei+&kA>j1C<+p50f$orHeWdqKyv2nAo4nH;_H9^y@p-rd2WjNDiW#uMw0zO zq|7inXd#b}@=Y4W+G+RfZgtWw89XV<)3n?i)HdmO(YfAyFUsWuw##b7Q$|B&$*>8P z3DF|oub>DU*`8QiH9KsVDuZrxI`A`8$jm)_yDBPaJN*)-YgVF}b2IJ|G^=!(L|_^` ztw2`Uj<94v`J11=eJBNUEABe1Q{(yT1&O|ZuM#l#7p0kHQ_wsp2BvNU)ZUWz zd7sw)x=B642_&$kVB3^Gxs2QwVF_}_oK>coRh2Dw7jQdn^t{JQC}4>^5=bDCOW>cy zzq>+vnC;z(yzV~6{eu$lOBuLG7qf%rzKP^WyXiXb%p;UqL_9%Xj|M%<37LM`1~xs= znvM^Ffv`#Vjnr?yw)+)8vg3>#Bd=le>7b!B&GheH8se@#zX$mgn(Xq+r#gEml0k^& zQ%aQmP&_s0d57$fMheEr`3AP_jqv5=qk6p;i*c=AW!iGm+4gE&>S|Q&%?qcg@kII6 zM&{9b|DSW;B9(ihn9k|nHm!-UzK0%Ns|7!<2h)(#;N$l48l7VqnESmgkWPHx4{@(- zp_gKs6mf|P?cD!qBE0B5?R$|a}~bCk23AM|H%THT++h?=;vPlDL>l|{GpbfFFk0ih;^ ztG)%2%VAX@Ac|(BXYJ_WgU{CNR2bKHeU@JN|IZPo=nl5E=k-vR&KqgzJa%*Yc^-`Y$)3I%QfQ^I=YQYU0| zk}>!?>BzVIAEV~dYal<4fM^E2n*-4IkPA3n8cwBwSLyRrfOh)g8({=+Ek-tzyygu* zM>EyU=lm=H@jyqzvFK5ln0zxg_T{Vbt0{R|0lv{2AYti$u5;7_10upb#a`aC6H4JXd_TtU?UdfNP(3|GWSyMi%UBq)f6&$C zn;Xh_|1pJ-IJJ&n@f4XTzYZoKn=ipO6?(5}*8bSN(UC(9rMr`_Ahe07mwM_6mhV|& z+Za?5B^?kAyl=77%$aiEtPZ9wVj?b5<}nXgaD8Li<>0|YoW294^K(kr=A2?a%-PS# zXU#S4D?1*2c(dBzdupxuEaMmmL67lv_PxplRqLsCT)>2YKrA{Bu~))kqkAIRCwrff zDoqK_Yx21QO#RwK?;YpD3XujzaXvg{draoO>x6WQ1?PXqt5sZD^gOy>4Sr{9H5N_z zx!rqyou|F7R>!mM6$LqY;urP4#}QOm#+T77(JGa!Ggq(g^xWf($wNX?bsjFD%n2#z zTxR$aTn4i=L?_L>W9RFb0lyEwtNkg@#iP+1aw49hq;b3l=Z#u#c)>u6DZUlyQc9~W zdnEB==L}ylzf{}PF}$-c#LqHt^%-&##!o+Ih3|Pba{CZ9JUG6pY3}-Qv|7mW;P1~& z0Q4c4OhNUk(HA-a%$61LnEkgQNW_;C<>xPci3r7>9dMhNnAi~#g*JX)ZFa_13Xink z{kb0V>J8~kzx(89TiBM$)<~oTX!xNqSD?t@u3ppA+mr1`?A&zzon=7?f7DOcoA9Rm zP;->()mF!qhKye+3qhw1ySYIJBWIV=nDt)kPsy5+My0GwL|dhQVxn;m6+cm)3~6RA ztxG-v#@;x-U;PRgL{^*5O#iyl%OuNBrtZ*jGG;Y94JreJ?y8yuwKy)yQxn2cyjf$Lh+n;q6+p2ZRpsEHBF?lB z&f-IcEG&b<7{f(|NPkRL`+a-)()1_xel&Ml98p~N_lzVeN+;_3SVwhIrkFEzo4+1=)4@^nA#uE-{WN}}b3@u+9pYxpc$n8wQ=ISSiu8Vz z?sbW$Bp&tJ(<$hgi!8RrfJAG&adFcJjQ6RgyuPv$@7;n+t5U!N=R3|w^%uQJzajyR z)6D;dpCc%^Q1kVRh>(-%FsrhsYo>imofpSr)vfnGVyfRmQz$8nJr}8S0y_!{B#o%@ z;!}~fF)8u!-Z~*H)wj3f?}Onb!5RFekXc&eCj|Tk1H)g*aIRZRQCa3?SUF3Lti|_S zKg&LKq0TPbwF}QgBcGjE#M6Ftto`H-;CEo&@${hI)$z9qT$oHoB%zMt+UQsDTxi@| zX|noW&^n~D?Or`glse_QJ!PKnzem_>uuv^_yr&>PxqqiQ;s!cp-#yhwz%%mmL-Xn9 zn>?X~D#;aau1X!NxDa%iqlg#S`B>pf?%_$V3OlSiIzF2HH$N1P*}lpyPtc&>Gd26} z4ue=!arX3;J;v@?f=#wB{E*GN zm@}0EQa|SiLI8W)(U0M>*gL*^?D2QcS|AsGsW;*R;6=2Sptencys3YR%Ks+6Mfp1a z%vmhf_=AA)X%gV!;o|G8E`XC0vN$dSE6e6Qf`X1olA5^1&f5R|$k`A*9{JlYl#)a3^{FRBEx1>4xiIodk_yk2>SrYSXk$jj>od~Xu+|j6>B;)+^rw>Yi1yugVL@3*Z2z9G%Trp-uG{RIYk^`O zw?{7|lO+$=|L;Y+{<|n%$R~6p^cJFe&zqm%tnorOff6p`zaPy^Ah9KM#=YC`yRPhe z=N581s}wi0-r8F6z7x*2mfMKpD+ycf**LEvawh+D61z*#2yWJd4KpTm-s-Hfx&#HU z)=xHEEVJ`(9I(9F7N~YI_PP@g6YAKBDa|ESAWBkVphEvHfUA?M{veO3fGNXS$XlRM_eO3p1Q4oMwY7=DTeI@7K9EI zmECC%XHGLS%`2nFdVP8u4{k#%boxV|tSgT~b#g8?CxLX;Q_-{4t!T(c7tVS~s;xS5 z?O4_NR5zC5YCN{l;jiGZZ9@l^O$lUpf+Z&`mUIvVT^6fw$5H!xSDE@wPORs6Wf&*w zV(xtl_?lho3S7)o52UL`%)js?=Jh#7w2n8bRtC%aVgs762_Z|6Q6-w50qFX*w;kJB zh_bwABr9oU8#l`ku(y~3N@D*s5>eQ*bq(^{Bw`&p`nDvJ+$C1&Kg^qi3cq!*KDg&{ zIPiqK766$wS1N058B7b}aJKv%_W=^0L@Vd+mjOMwK_^$44V}luSuBnGfLF(9RVt_V z#j++7;j10{BWe+WGGalVj+%oavV~a3imt2HqD3zM3(cUQY7|I6=g$tI&)zF--qka7 zRZPJSpT63RvylVNW|=^NZFWZM0lsfw-r=;Y8QiIg#&A8%*cG_Fe8}`KJVS=)?H*tI zuAK%!eTGlApRV0M@wo*{Db&NE!iGLscT1Az_KmKvx`{?yKdQ+t6t3snO zOzDT-(+>nNp4ulOn(k>T-eXLol z#TCM`n2%y!&9vv^T%vb+d|!MDyP5+ZG=3o$rZQ}2dotsrX=O$t5shNJ)@2Gn%VJ+9 z&!ji})zEFdk0Z{k?8tQY@4N$wRv5Xno3<20PL09uR^gIriJ5|TU}*dciIl@WPNR~w$705Ui#&hROA^g`EV61B?n(|I{X%Lsp?4jnfJMgE>er_$k*lC zd#p1kcs<841-|mai-u`=O6eA(d$-D4r9Ymuv8_HhniCKvC{*r729Dtqk5rOp z58EI6X&wk+j`HmvT(%~*J{-zvdk8vQMeL}MdrqHRE8Q4Zc3CU&SF@7~8WAD$iTpGAPR;dz|rtsZ+ODTNlgdM>HjF#H{B>jj4ECz@<{iKtdUPLYb?(+ zmHgcDGGsx8LECV5f73`Mz2AWh z`FfjofuAvM7+XVK_ulL(Q5JwD&CTs}i@WpC%qBuxzDHfRO*7Ut7KqDr18H4{w|THff2bHfPLTqgRV*eu zLBY$E5+r;)+@S?ITRC|(n;*00qt(uj!$Xqjza_9vbhn$-DO0*10w-@$qTTJ;@Ivqk z6ylL`Ujg#Enq`{Q#@6hhQ78PT%4e<>wwGnMz~UY-(2|s`I3Hc;`03Nn?5;rLvJm^in~_1}*6nDE~&BIA>r=GY(vJ6%lF2?buu=hb5u zbG=ivQ6!>g{5p)l8GS+V{q9NcN~!=IaEzCJlAgS54~r41*wopbZeJdHxVvMQQC9th zJhDg`W0&~(^Jhc1yCO~l*i)*f7&3s=J8W#yh)9gHs!B04mg5Xh*U}kRzZ(NXPMhWC zUZ&+OiaNh2Kysx^{py4EK6`HF-RIt>%C=CO>*e0rBAJ(>ld%@Ro%h*itp{`6RH1&6dg*WK7P zXxQpRanDA}i#s-a)t6VT>w=93CP?1Rlt{t1naRvuFGfKt{)eaP!YhaN-B-~DCKoR? zlgUYPK&{VKMXJ~T>edHMn)c%F)mFBwY{9m8B{t-${t$RKaWpwo)51Z7uThIM^H{Yy z)c$1(G5$Lk!J5aSi$PgtLomPu00qtaBX%*G0c|rkmkoCVR_GYCQX1MBd|-ql=W-psr6d8bvjjqLt?thvi@bL0ioF0b(y!1(L(ws*DY z-jK@CICK}utF!cSt?43QSYe5gi&pNi-pKaJ{^hrt#hgNQTfq8fw5dpi4*QjfHMkWfV>V%J&|Gh(Ft2*r5v zi;CRJH!h2K7ui7Ek6+I9HmNZ!Wod3+yZoiy!mIBhzVH?)Q}+#u6tJ(uxJOT^bg$y6l$>{Uu7pQR?q5WvU`C*rnYkDkavJLLu+tV`PI0`u5AEjC zS~0ivD5HBppBsFA-)c>*WRY-(lYQ{^9s z200lrN(@_{{=^R`lnFxBIQqJC*KwCDza6tbQTn~tyct321mi&8ENT>ayl(N{H~*yo zbrs0Xje_Ip7k~U19Z|DCnwC1fgs(p6P2T&Hr=>WaW+D@lQ~*s#X&}=QX?=|>XO`hx z#K-v@g7vsBalzyBX&}G(Iy0%iX6fJFU$~rioaTQRKHa8;UkZ|prc1pQriyRo8S!SF zUtV5*IlI2T4ggz$1yzak{O+qfyN0jjdS{c|Iurx;Yh!Z$V%-PuYI5cHbJwrgnRJ-8 z^9_b7eLWzKrO*5)6+VY>cfgEiF}nB(tc0`uzDE zd|Llnj{Sp)iI~U2Enu14Oe?%JE7Yy?0Q4o%{>Vfbo`0CS z`i(k2%S<=ETr>e@Y5#ZG68W(unBb{OmxnxwPc-)Ekx)K@c6u7EQ=6urMiUi7$=zcJS=xuUCk{6nMk!J)fUh}#mXMX9!OHNR>_o+Rl{3HgE(m}Nm;94**vyqc zORRZCVxz?NVUhP-7|aR#T9J_qP2}5Lp#xuI-Zj5N^uNfa zwaI{kYQW#@rQ!Ds^`V}sWU}n(sU`8M7Ka`cR_e7bgJswc%&(^UkPoBth0^Aui^h)c zIJD3EOZgh|?gNUH#e3@>%saju>qzBbFUQ8lGI3TGcW#s9J<1ni>~w5E2nNuZ85tPt za$9_kXNvCb8iAi(j;Fr2ck_sCMjwnCLqpP6Mx~8xPjim4KV=hyMD+FwM)~^W`hlQc zg!w(Msw@)SxK>3#s78sSo#tf1EDzHVdyhR=cjnv0?Sz;->T;b2a?h`Y7`Rqct z{b+7rcTnBl!J+j!TCGG~LL*YAG;?{@Bsg&Bf8#U&u9nwznhUl2mwO0hKb24`Y}Z@a z*nEJgzGiBm(pE>hql8?R^V&yC%x#@&j2P0RbURDyj1n2rGECzO8kYIQbX9N!WbeprJ%EuyT~n)!71Rq+2WL>!*^lzan$% zZI`zLczAfbCYguh8FEo^kmJJq;l6y%R6w)( zF1%OM0qX{4TSaMUt-HD3anZe7Rp(P#ks8m#EW34V+&`RdUB6}jYV+3HYdsFz_p$`K zjf>(DecNmU8lvWL@ri}Sn@^ukgXLU>egL;%J1ySms6;{-?3ko<~O<;-s_rC?8LM+^!oo{k&Nty38|Lf_%}f<&D7MPT@0r5S1m!4O&EJ-;v_6S= z92UVy-KJJO9=GzZHF>BGN#1|_z$v^!*l>NviWth9@ExLJE=X$IKMhW%90@wTKwP?hYjod?QLoq5sNOc?Uf1_nG|LJdzLS%p!E+U;f;P zZNeAxiRw>sF@AD)otJ}J(8cR4!D|0Sf(#6W^s1aLJAbcO{ynKiU;f~GzM+& z3us@uYs)|V)78uocH033oa1DA{JOMYvO6d1GHnciTvrFePPbhVF_V?%u(?^)ZuD}a z4WaKUq#nF$iy-fIskzM(N7&#`tQ;66%Aez87J-GMj#WVpOMSoH`}K>Kt?QpZi*wI4 zLP(QXm$!S)F7iNaj}mFm+eOqp!@Sa}TlLEgL6t2wZhs{7(evsG#Uy%oXHmNXtsv2k zUDv7Ai8eqDq&GQ1;#1%gaa=Bl|6T1Z_4(sNFqG`H)qOrS?R`3LE>$UPVzV> z5a=8pk=J>>LQr8SlY=2D+J?WlD4_5_a?zbXktKCMoiST{zR`4b6+x?Y1u-yt=sB%U zn5ZtVB*P-#KqZj$yQ{MWJ)6Nmr0~yu8q_FmxN^2EF8K-!3{dkh&|3)aI46Bk(#441%%kk5&eIb1pqUDoL3( zTapayn|sd!Cp%Rkd_MBu+^n-+jNz#^Y1(-jd(-)z-n%C75Reu)yr)Yc4%OcPS_|JB z9&+X+T;x8C|MQkr!0g+K2d#UI3oHhJPn*AbtO~gp`Y!U|*`Yo@NL(&8E}EOnolDEW zJ;kJ8X`rebhrZTEsdjJ)J)PB#nTwaxJLG(<^T&CB_Stga@);tkQe!nw`+kV5Kfx@C+a_0J5uI9@{Fi=x2 z-^E7Sua+-o@w$yJ!tb*HCr$j`V8n)Zv-9TNmRth2Nzl4M=k?3CeUzY}tK&@xeC&*Q z7n-N_>z9i`+c6-Pk)DdrMt*%RNZkG9@)>SU{S73qyRw zN%k87=an>7#4+f3`Co>17i>`IH~eNaocpyTJ*f{aHD1NvP}O4QwhS}IfB1LQYt5|W z$5Lx~MEiW{8O{;`cjchCyVX9rzV`%D_j7fQ)$r@IF0PM0;G42qTaT+Ksz_t}Mcf11|vclsA>)be6=b z-}R1i|G35WGL^*6*btnA-~$LWN6Ut*u!-2(EgHfR`$e*; z8>M0q3yuhb-``4fG4k^5_;TmFh*CcNuo&^C)I9imI=dhHhk_rJ?!QTpkkBxP>A;Yu96PRnLyD5dhFBmu2{u`P9rR!* zqS!`z9Q<6p&i0(#_GzQ9rA=gDqXelqk&B?So!ZzYc+^T!_Xl0ra=!?n$NTll+cg3Z)i-F{=TI0GNpuQH9J~K^yKLFEOO@YGe{+g^pOxILrAgdAu{beT!mDFo7LzY=GykZhH4#M@>Q6LoF+V%gtRh*k)G%qQ$8 z94Igj@b;zF3vArv%G#0(^hm8r>SD9&sze-KDBp1S?9 zq#I3~u+vtq>a0){!e-d1XQ?URLZv@JVp8KT#8jqW`L0+?NS6ynU6?k(G7Lkpz47P3 z<;0X*g-zJ9g+{MRXbrRDNBx-+O%?L>-E&CzGHO8kw1v06{$GPwv_jf^mQ~0IjxuLW zLiP_r*$^ugPR-ym658bk&sh>paa(6|BwPr65pDyD|Brdz+LnQjS>w(AM`}pg$PiiV z$Q4p8i0d_q?XaCR<0aE!3^HM&lphpDD|?pI9*r=e6n_-)$og4hv=j+dsxT$6iddxF zNwxu_FCd4Ho%yUUGBK3ZX8)~fq>2po_0r*Zf>sod(9~-dW;rI_KejXBNRh8tzSFS~ zacft9EW~vPk-*|d8Mi1%)Im`gPMt$7Pl;q2By$R`dGR@vB|QZ<`OEDCQ1a)mR>PHbE8vc#$cT_vRh*=ER0L(O8}#^>rv2aLOblUKe0@9b5fS5DyPQHK9lPqIrQ8*%u3!JYZlKs zpu~_XWRHJD+~vVF?IxxFIL0btNv>7!d&)x#i1*AP7s`kp}*;?Ej!V7c(< zoN196F%0BZ#alh#C*4!XD51aNk}%h4{A-6NrwK1q)x(ynecqsHJwolE@fxo6g{rT~ zLpE|7OM${&b(A}3qX4e&m%T)127V+Lng-I%&1}Y}05ku?^X&P0TjPbmT(j$lVjkmm z>SJDq7yBj}>A*qF+sB>FVMnk2iT$V!<_h*iI_NWPF0{|13nPWNcvzsg;p1qf&qURs zYy0{yHhg?Sj5*yT6dj};SaMP$=15rLdZbcsl7aHhdrJlmN&J^X`prY=zu4Tp=oc-S zktxtDfB)uY`f9(+ZIRZ9iNwp#W?sge^TQg7at7tLSR@z37Vb+Z4*{!6M&r|{{E6&8 zuSX@tv5>oG68t-$LCX=J%S5_~QK+8%z^Gq^%O-ER_&bHVX|4Sk?M0~7s=ds_k`LLK z18B!t{PAm}Y_tUD&7`BmR)siraiKhQ;NtcXiRK)eDiKLREIl0_(N`H8zYysUBVl%9 z`(K|qUM>P47v`4eUteH(2uNO@uytAbc86H=26WH*pOe%!5!Nq-#LZ$=i_Jkkl(%0@ zu}~rzeiQy6C#1Np6iXo1ugzAJhTO-U>L!Jxs?2$;6{|Weg!Fo#i3@rqALkedYC1oP9kS8v^vEGUECPP3|w zHpV&5PAl#|^%5q6BSuMx&{5w;U5!E9$c)<3 z`r*F$0`{~BXPFK>Tl$`;lO5A$P-sy3*d!^#Wwkmc+Ui!3W--(lNhq6gOrm4uI1*2W zLUO!xHViEr`#oSU4<^_9lecI1=TRUMYYNS|uVH1jAV@8w4ivuQtGeu0{uiP*{94O( zv=<#U5C(lX`yd;Rq`(YLUv!DuRlZMFeewE5=rUpIDTUst3)(&Bgzl9a^oo(#9J8#< znN8&U8rHPp!JNr{mE}a*-2RVuZ*?jz+R$yofOOisSb6o=&W$nTMimRKs`Uj`%rl`+ z{_93KGRpJU7r%wm8I>d|t=dRWFr28FcBS_3La1>F6KhBtUG^9ZN&{O0f-0^%>gn21 z%2jXBk%QnJ%sj1x+A31Dp~Sz{`=i{Gy8c}IYlrPRjj5kR?0KYCd;%J%R+DSf?<>#~ zCnOMG`+IcOuNH3CIp@Ern3y**{@(2QuO`phwO$&~sC(rfBVpKiu^$;Jza9S$71rX^ zRA9zqWVK%{;T9SCo4w^4h8&qL%;8m=tx&Js*z8PKCh<*p0Z#NixY7NUE{CReNWMEl zSe8^4KCBRXU#R4vk&nsr>D`HSwETB8F4D5sA(fiZ7_XA7@{(o_qjxG;2VN2MFBmY; z>fsd85Ek)SPv=^N#(C)FPyKO=sgA=XP@3-?HAWpei_z25Z>+$)(v{agdxk6!G-tXv z-Bk@BMK(k(DOjp(m7V1ORSf(-gCoN80RppL7Itg5YKksndT=@D_;{UEGstjq#3&SO zr*~g1f=xWIY^eiL8LCpTN)=zIHBvX)7f{q`c`1nmr0fK7qEMCev*fQI(X`iAocSx9!c=F}yF~Ss&1P&EcCism8@8 z1pagrrW(SuepsgzD|^gq=J=|r7ER8Kw7WUT$0janh~=6Jl|CUTyi*t7t!iG`PRYr|-80fsJ)DS=hWLX+F+ptt$^AZAluh zXGIeZ=bkcRgfg3#c#K%7cZg?Y~5_)-3B>g*ORSR~>hQ<;m3RRu8Yj5kyw60D% zc9L02T_@O5xMYx*w*2J_l>@K>uqt?@F?n28>X%<8r;+{p0D0e|ZCc2Ce{~i8Ng5l0 zRA!@MO=dm4j)}`&is*MJ8U(QsbG$|$NHD4m5{{n52*XbH7&x&Cadx`aG3*#dVgt*b62&gSRi zGc2!X7ET+krIo7Up=e`qd^%rw$1OIdPl{<$4k|4#K@*09Es8gPin{Jt4QG1M5>&0*o^N9$c23+)-UPr;!FVPRT~mI zq8Z21QzwGP8_33V3lx$SYd;Z~SjMJocFFg}28l~qvTGIOsx2x(T*za81ey3!eK565 z#l4TVW1%ddBC0hmGKH(w&lv9UgQyF+kyf_rdx_uvk1 z=iGbl`Of1{_h9tiwW?}K$y#%&Se*|jLYiBb?XaHqcOrCDIb6jW*yC|Ey-K%R-eH6* zl!Q`*8}NK8c_cb_x!3(!sKvo6(d0Zsdlseo7_Nvm<7QW<$wU-KSu`{;T;ejbZ)cXabDg}Fj5g4yLQ3+Y>T&Qe}M)q(WTikWMvA@A8@IC#=1aBEm>}^E^-bPx7F1Phtmx=l+r79X zYw0>*5ovR%60PKmhdziTx3%!;h}We^4kbs5@|HLZwqJ?0vHiW=Lz)$e#5(U1ylooa z@%BFAeT{o%QoQ+O#~6L9K>Bv~LTG}=zAS>=H=Z@ng5>I^8x4!Tl;>}He{eeRKs+#DnVby#D1$=}UTBPle zOSoIX;yKIM$DNM;oQ6o3U$k}+=dNqq_mL8Iaq+_%CohA&n)mk=Ii5g#t4NvH1D4SZ zo54jcCcqK41Z&^EJH#RE?o=UvEnm97N0%Z0-G|3r<1L0b*L$7{{-AFC;w(}NBWG+6 zMX71qK`lo~+KP)Qsl>G#y=d(UJEN<-2j0y|t2oCwTCoI{@{!WE5-|qfRy17V`weL5FawAD6Onf&H zKi!a&Fm{)2sg&tSKLD(n&v$9p@g-&BxraL#yn7!#F0R%|0aMsaZ;3P|}d3917BC3t@oWC@}@mM>1#{ z-FnBLsAOMNw?BOAGwi(~cuS2&;+pph={+9PixW@$B^|YN@$Qn!DV*+8u3b80wK$hG z^njBk9Z~GFSlq3@d)0~SbLpLwf7D#g*>e-YUFf*G{=|Abia$j1H?%c5oyTukCEamP>E3YR6@diXYh$m* zDBSb6PtN(N1%w)4bm7cGGo5k1XpJz`~hcn}J3P3BUoz_Zg{DC2qGLe>AzER*9j442%t;ns(RIUT! zDPN_3h!Yluoc-2o>?uvG@uGVlWA|?a11eP^p+!b!EqPIiPg%(X<1>JI zhP)6gZX%vaZ=eiVNFrK`MP6l3ee>*t-)5PFB7be>zh@K>CxRH$yw7O44YT!*XpFFT zb+e$2j}e=|5`=C}jpQ@on$ASsSii@Sf@{Y7mUA7s=1-fq{WVlp~oWO6IpLW)vLeyI~K_i`VA5<;9;zdC%{@ zd~j~8(UrVH0fRxo5Xn_LlUH3RB5x-7U<(gX@xRLU?*SXQOo(hzgL{W0V(l^{8CyI1 zxQH5h4#mmQ)MQdu(nl2QqW?`!>v6rSETPs1u}8j7I`cNeek{rR*Y{occPkU^XZMAq zOZA3{;Mg0%J6|-1TOzOZmwu^}OkvQxXr~?bs#P#Z!YpzD4>bv29-8{kl$p1%efQ}K zFI)1+?}0zr<*SIXg-%W=VD%7Ih+CF{4C^#w7F?vfFsX-&dj&IcbF=)KrLKjUMJd%~;X|{ht1fy)lvgrLh+tT-1WUfA$=KMHv*7vCBV;bU*|f`X zPWQG{ceTlJPahO*g2K`hunO2W8Ab42d*;RStQi|?x4$QUn-IeLJViRWsR8+p&fq)$ z+x{q`n13aCp_@}ZAlE0V}UcVh|0IJ->E~8Tu6~y=P!%!!i>k$bs}+;@EGz^$iY22!=wB#y|?A%@2l1u=ocf z#NZPwH+ddy4=Ty>K9AtD8JogUG%x@wyZ;+XQJ{*}8Pr+zb%j!aY7x!+6Q`4TZmP*_ zppHb2L)0Qr{Z#4SR+&Jx7;m9AOl@wAks6;DWk_WPOGUBAzh%Qs28#&|!GkqqjzQsD zQ;YgyMa3RdZjFbBPnJ{Aytpg@;HdakYjBi}RVh3$PFQn-#N9$Zo0yn5JD=SZ1UAvs z!*@wM`jRx(3jD+RBy42xuVZLUd+7v=mO2h>=GvhIi{#)?+_^~bOi3l->QdThi*01O zLdEr5gatZ7nazjdd^1ihINwr)%uV-8Y3c*3TTSvDAT_7Cxj9AI4Pa#b{F%wq69yW! zx@ooD*X(BvP)9>k%PxmEH|cm66>L-HLObM3nExA&MX4)1MG|q~_Bk*RK41BNvXF2@ zr5rg0{^K5OKz|~_`I~0v-&Hu``+$Z1m3rz==swvL_^(Uwdv*6pS38Cknb9= z(!=4gR4(h&t1X~rTmVDg(ZZMv*uqp0kWVb8dMw*AjgAt;4Tm4Eb-e!B==OORP3(R{ z4A2C^y_J23Romd~;CQ|Lk%|I=0ZO6=$=3p1{DV(G>gIp9-<#JhuE|QYh?#G;GBQs~GrY<{S&@2{f zCG$u?>q6|?>XkvCfM^lGjLc=dDvVAIN&-Rof?~)JfRgBX6fs}ANy4fd^mkZg+AXR- zd?~;HkJiW)I2J#r6p5`;5#`?0S7XAO^9~HBgRixcsHo%zcrH5--qB?wm}Bo80f&lfLs0CPh4Q%Ax(1gOQGDL~wrbqdkg1{ivhli1E^ z6sK8-J55-?$NgJB`>Tc&)NfbQblk;#U=an|pQ2k8(5imQVpLTo$cG^Cz1NM=@uC{2 znBG#q*|`%?!pbcV{WFRKLY(jJGFh)kyG{0rBY>2WgJkccClNe-{2o7m zO}~SXzr=U~gB+xshk|EY)>cB|5n>n5Ouy6lc}PC5IMnFB{5TeT76eK!)T`sK6nH?% z7Z8Gp;Djq`ge;Mr2|2pT|1Ts1c_Qr|r}mj5$+`o;1i*ghGT&YPR*l26R|P`<`g}HY zrcjBw^yuI?QiTx1`!)I|*il_oyJr?q@BFI~ z9(j4~BWF(KdgR<;ZrU**pC*;c7g4q^3qpN(*oH49p|>Nil09LHX-uM+Z%X5imSg{; zkXuRl9@N}JDYXO6;~(Ca{*+z=6e_`Bfu7M#6qdf)4XG*$!=RT0geeN42q~2S1%rfz zL%$1^fr6on=1uyUDv2Ywctr`ALKB_8OCR5&V^poSM{|)v*?xhc(AJnyZMJ_ghgw2U z&tv!!EO9Bn9fI7WYfQhtJ?Gg``OhI;R+nqD_S{>djch5bL+xgEy1HVA^e z^PAAf8%4C2oo_MpPD13Dl%{Ur!>Agz__*W!=lSgHQjTT zV|!4W#K{PDzqbFLI7tdA;+qof{eln4Ghph@8+tAkkJe`_Qi{LeB!N*@7iM6=wy|^?OJ*}ayu0h8ThduDgpU^+df~^<`~hH!Q5ed| z$;o-Y`uJiTX~fzTVPFu34>wH$dd~?9fhZNqq)dFej!&F}_f^Ol64~7mw~f1BH14V& ztPzcpe@`K-s)Y#)tLW(H_*RZ*;gaJ!c9`(IAoHK0zMlsvive$C&-p$IWDw%vsdwnJ zg(O|nE*U=R$=?}l)Q10O09!4|Vt^YWa2eA$3B$vQYX&Z?ZwcP!GgCOGKRAw@0`F^h ziWuCN)O7d0^AZ+hQ*Q0G>EG&H>~S?}2*KHq9!k|;uRnecuy1fKSj z5-bP{7OC^kpFaTJ0RkpezK8Y3vPN5aP_nCwi_7TSkC@R>Qup6BT9a3R|1CsS>!M#Y z`NT0RZGV-a@~O@UltJHr;!!pJ;8A7M1^$Pksu%QN7{z(&tpZE4TtaHvD-8mghWf55l?d``>vI*>kH{TSu5iY$!Gv;`_5e_uab|3P)Pi%iI z8{9e!WP_0tygWYK;}R}K^I8u)Jw1ISby22!OxsIFPjpdZ&tj`iGmteXtmOHoMpQMnTLl&$7*!{t+a`G~|D*2xEO2x^8MKuJ{N))cB6K^h zt1NgcAufH#_)Kh`?~}=uoGe#z>r$K_LP}Wrhg_OcRtV9Mi4pjf-WGU}J?$dKer)kL z*xmEK)mvIjNtxDrkB$+_8qWNk>|5WJ*)xEsDrnPDF|+VPuH*3Vt6HomU)Z17Xu{BQdXwc z?r);g0F>%$0ic+XC2`m_qklvyxL8lEw6vI5&}%uv(sBeHAr1lNdvnUw-TBt7#geJ8 zx^x-{J}2JF)RfyL>-l$A#6zDg&6vCRZFa3EMf`G!>M%%+}gAHEsD(j-k=33@n z$y8lZR8uQF6egb#8f~vH_r3YeW`+rObT14m^!al*h1pt1i{UGY2pvpjv1;^BB{*7i zbHrwM)RV=kl_>AVk9Yf{q^9X^%X-$h-Q8NtCli17-}{|;pRid{@;K88j<;mq8xBa% z&2n;ZoS15H@$p&E?jIi$%`GpluAa`!*0OV0tkv39T1VkX&Vu4@vuh@v3y8S{QV-6K zNR2KA#02FbR8tXznWI+=TMDPzVE^xU;3z_trmRxy^~P+OC_;RE5+vxWYc!788FzG> zA}|LBhyQchpb_j2r_@gC%|Be_yuZBqamYYJ$FpS%$g)> z{_gYL#e3^rC~_+Dw^aBeNgxqSgG#24QRwi$tYA3jU7~R%6O5#*nan%XoOudcR)Yj? zmsP4i8)Hh0y}}6EJ6L5N9+m)5J%@)F9QTUz)HwH9xR>=an%?ZwyLhrB`FWT7mBxEj z^#%TK5-ZdfxE+UrmIS{6#|C9l0p;~?Km*`)bY`geur_~Lh_DVBtvdAm8+*^YyJZjt zpbdtGTl0r3bsQW3&I#}GDmgMX{8$nTF+LG1h-}pxxY3=0_HkkN8vsH3#F^0HLrEeK zXehem{Zyg1dHpnf$xadH3k!;wNMMA;#9&^C@rNMr;h}PH5aFq5YilVvKB+H7*P|9I zHt0W{z>*4h+yRav2<=~Apu{X4!(S51(Y?0^qDQXx_F&9PzZg-FiNNzZIU~|Z_3iQm z`pf*PU%Yh+SS0R7%o3Kl$6*1{;=i>l(Q3)92WrAUG+s04zUUpj@piFfsDyTAO2_X| z3e_N|fO*K~ClwG-(+lX-*473pFDOHQ(8!gA_ikx#G+HhJ_qM^`f1wf}2~{e}ki`ol zz(PAQLW7W^T3a~KvGD=|0{qC9=iU^>xj&cRrYUB^H2MlkbcEu9AP0i( zg-sZouCGuE#n_plCacPCXIkD>mnETvU?&R-zn7O1vNzW{+SqkX982+BZ6dW=z>k2i z={zlG=-zB>RS=oIA7e)uMuOx{bg54}5lyjgMZZYU@|}hu{$vWusW(Izu0Khl%=M4{ z-MK!`)>p?qEFrU->f6;N^pUwA7U*7;`bfPcXs#0Zk_ZY4o0Yjn__qoM2ux1*@$gL^ zy37=^=KVwwad?k~7C|6X`}6t#O&ahevcqZ_01-{5v3Gxb^t*T6#762nfZN%lu0q4` zjqYSN`m>*5#h{9Q`r~`Vf-qd0rhBnG0&v{9vf)~7v~jU@@cCpB*$eSN++FuLCJXr1p$O0 zQwG9eqfA2bRi|@UGfDU~(ho+0^<|xRdY$WFn+u8@CzG<(zSllPyg=b7=v%m3~3za}fPLwFzVqa>Z9@4X-w|9|<=fB%e|AMasg zagV>Ai|UzfrmlL;;dQhar%o{&1QN^~3;waakocUime|N{=dww5zoDOg2j3c;n{t0l|Pof)_#z+Dt+YkS7;(t}>75}MM z=S{R`F47dKa5*DhdPib=E9nRGv&iu&z;pCxVlFryO4Z)*<|aL(H2`j-~zd?doq||J>JvdYO*@*Ra2&Z0w#si+;^vfvy&oKwQE= zR;#10N)k-BcRg@y1Rgrq=r<|0N{0q*Zk#A0O@4z$#qk)4p4$5_-z@M#@JFo~+ekU= z6@|_{Guy@&7=jlU<`+7wjh6wR+?M|J7#Z)no8^DD{c5l0oSpq!@b0b<$~z@KJ+4m2Bv5JQl*LFX`einbXjPO7Roi)eqRV{3}LcRBWY@>k6p~U3$S>L4MF6 zqoJ2g$zU0euP)!NRZ)yT`-b2u(iI+OdVfnBxhT9~3J(%_`V&|EQOOM@OZo$=QYh?! zRz^?>Hwzz){R;<7p}^YLhFyLhILl8e#nSmSk8o!d#T8%8z~YHk=C=5kiHSUEfjh!-iLE#_m2qyn2plw65|r%}4lPA%pxY~OpZ zD)_LMA`+{MuekjMzdP283m4z4@RRHW?STf~w2if!Cj`PDftYqGiLv6`H9|ABu+f?r z+va-ML5$ixGCYF(Jxca<(Cw;!Q~BxpvY<>!AT&tqV$t`@YNLs9o$~e-488<828^#r z=Qs73mnB0=&3dK0a|c|352WixTk*rB-1oR`(EG`W+-aOKPBHe#BD*QV72nQU&IHcF zNuSn(ysH|%xqu-OnxA?RqA^+0pt848=DdamXKe-#Sg=4N zf0m1Qhlejuj=pbzj0&QC2MIKo6#?(#F+#;bcj@Jg1S8=^nMt6*!;%pa(Wqi34JzxQ z`*$gTY?zDC5w<{zw9ufCK(06hBn)3-kRCH0m>iv!oSK4xiYoe#f-L3xP5wcbHyE!2 zN3qv#XL;s%+k1B8=LI?^X+2n%l?)j(2SvNCql+f-xwaEY{u=8#|;M!-`kJ3?RX_jEqPkL*>A}m(L=CDvN5p zt!ZzS{L1+H4-x9Ms*-KZ{`EOx?pc`6QQ;ph^z`;529?!%mz)t zooB%$Nblav1LqGzJGn#E{}7(cwm+IOha8Z#!xNuz9L{Jh%0A-a3RRZvmHMMK_VN<@ z3L5WA+nviJrz30IfT^pxX~HME*B6=j2K_#kLT#z}s!8G{!5GDYTZRTd^oe4Iv0?}x zVAYHj%=+{QCV~tXg17X3&!pWa>L@@m6 zF05^oJ2ICSg`TYu8QY#sjRVOy2bu(pPyK?{8JyTYEfh`RH!x z4#0Wt1fM?NqV{DHr%{xINyX$|7{q?W!ySm-`oVzeHAx{tW`w>{W2s}r`x6dUSfVC7 zBmHse>et`s3XL2bYzQnaf0iuQUByycKt>Dwnp<%E%N`Ii-Lc@rc(%@!cg^pse z{e_<^b>VdS#BB_N*H@&l17L#=zJ*U$ug(3r1MIas3#8U&c|^}T*@oRTWnbgD@{6aR ziEia&!iv5-Ziv>X>ePUHDFnT?gJ80ggr^X8J&(M-+w86+U|?wni;hjNOa$*HCL&N% z!S5UV@Ij8e0hD4tzcg)@4Pt4C-B_$OALNf_2gjK1;3oc5cRk@Tkrj?OSvubc{7m@R zZJUg2YkYmV6}2|=Fth}T6SAH{_z1Gl$`@t{o2dX;kNY*z{vQ6)-R1mzc72B z#tJ^(%9{e48spP;Rsxoh4y$_(IH$QY&Yx}P5xhA>>d&RuT-xqGKTTX6mR9tI=YaJm z1lMj=A_Bc$_T_Rhv?y}$8@DoeO=MEVr08iD9c*qN90?>J;AZ*x->e?G6D5`Sf1M^Z1J5$(eb8alQYaVJ9-a$d1F zf6m~yZhIzaR5IC2Kd$qnc%QUDfB@ZoYkgX@Rt3$g4nwi!Zqgp)Gv69$PfbWqbLQ1 zB;0&gElm@VX_Mt;{R*D%>#W9^8%wp%?`6T`d)dDwH9JC;Cd-;;kvrt@2eF^tFS*kX z4QD*9c4x1yO7DKa| zb1Z_JQ(`*|Z+rt$4#O(=-14#*FUR-Lwy!rJrYO;b`sSczj4iWJI%+5Z>#U{t_ZK5$ zP=1>YvBD&|@@V)M0eiRA53xG?9$SRB$9Ma=xAhh)XOvY0p08>K;ivH8lXqZ2 zBM36WzgL{s6m&c0rar>k4QMud^!hPWu#@#gZwcP5Qi{97Lycyq-B0tSK&Vh&GgCqh9pxD@rMPbnZvz1n-c$~Drrb-@&kqs8gE+6b@0t8=r#|s0 zw>{nmX25xLFj)AD$3i|9O`5HZoF^@Nk$RL6un+AA5S~`OOx}XrTpLL&i;u-sgOda> zjK4~wV0aR`iHAbXKYpNONd>bzcg4Sb_eBStnG)uC&3k*|r9BE}pvWP@MsqD0%180( z-N!h@WXf_yT1jL0d}kp{jLS;NIxE#&&K0wTE}pfZFLq;K&gQMJcF2l-40S@g?7ipP zuNj29{4ZPLEtHc`!ty=^2L4gIOi4hwcKkgQ3!dHmdzL2N)NVG(g91W^3GDZ)(;tLR z8&8VAHZ^}oyN4WT_dzmNiqnjLx6G}N4x_l=1`3n!j+h}}V#ZLgE$p#Xr>SrvZe&z= zeqCE_>ppVVR3($A=l6us5835lD8KBxeT?+DH617OL8PfM1UpFnTEinkkZc&HcgIGL zCO;5nhR!=CWuU_L<*~;N`|!QMv*k^WR3Wqr`StYt!$Dw-?9Oqp@eh=~f2lkhSNDf> zDdT;4Cq-vhQ|cM$%;f7#v7`oJBYDl=Q(wpJo$H=~vT^`QXy&rZ8QoZLWZAs}jDQRg1RHU{!KXrL| z03C!N1kxXu7)h!{NKCYk9}Gjq`3-X~AS8awlM+5Y>3xU_1q>=IOeG{8I?$l;xyNC@ zI5{|>T%E;AcvQN>{p$-(A{ng|(Qdq=!2R^0zdK>D{^8a6a7EK6qC*pK`a)sN##KET z3?g}Xhu2qbq8IPiGv8l)mkbSznY_-Yfr)Dn?BN7$|8>4UoiGJa4J(ZXfJp$QtFWL> z_*^NoAiNcRF)G8CWd_RZ3`W77Jv(z>a7YkPW_SZYRI4sW zCu^3P);80MEd90!Wq3tMnWN!=MB*F_jtah!HeR!y#$31`-?z%;2QC za(DfsrL%Y2$#Wlc?YdF8$~3P_lc1>;)B(WcSb>Li|Jr#gsZ}Yv6}X--8Ad!1Q8u+N z%FUN=4Y!)ui1IYt(0&x@Sy6h9R>k1WvM88_ORnpe;|SMd{_G^du8ZOloE!^v-z30-t=pk!NfvD zvIl*DqfjztzubOlzo?OM{@XOs5f;KO(D(3E&W@)^1{KQ7*NxM71Ws0FWMX|I7x*d(qeb8xj`Mfe5s$}mTdvW|p`qYKaAirOFCjl#4 zzEW1rs1sXF4ehh>Y2{JaSHbZ+D@EDXGiZ?U01c95AO?1p)bZ6fTumqju3U|59l2ei zkHijOsd5xNL?66lZ8ehEwXY)36e*a#agaE_cwvKjkUY7hk>?~*m%f;h{Zzr{j~!Q1 zJHbyK9eZxjZ?Nq*2Xth;c9F-c60mAPh#I9y5cJL-3Dm5NRP;E#{`l{4l)hP9)A&&O zQ+=brqf0{=aNgeID8*Ms^H_fR_MKScLgOW$@a1VNOGk)QRx~X8@vy-qF9e|&3KAeF zokzqrFoUQZHT1V zeJ6#Cte2JUL-xm<-6lfmcs`Q6La8|MLx~xnV}@79#0-CU{^c$3{6i%CQKBjo#z#IZ zOn7N<`pnNt;|(dlB^<>7BX$-OCkge1{}%jYUDvG6oLups&C@q^2Afc|beBM%qC zbiwCT9j}81!=bmY8|6ocMB~bb+2R^Kh`Hj{raN)NSU3kB@C+C`>A+IWNy7@t6su_$2CQz!=m;W&Dh z3ZXuYOl0r*rK+jR0Ej<`)6(r=CsC)EK;!Ob%hWbeQl4`l*yM(Kw|LstdAzMTC)Jdc zNKZ64v={xdU^R5WiYC!}fJzAD0~Mk3n~p31o`F?^M%9nvV@YZlXTkk_un#@8BPG__ z>&q?Hp1o48yAc&`5h691dj;!FCeQxw3_}QgVH{zgZQcu>G#0p+&;0_fP9FP>-c*(5 zQ6wr+IKdFKu~;uHRT9Ku zx@Rp(=?290M>9C4JlUon53Rl3|A75r5=hrCH@4XPe7@dXIXF8&Xyj5r>lv4JYU@5= zvCocQ9eZg-Y#0evYMj@y!V|uRSN#0Iz(C#uz%z9-jYht=Q+rN4J(WMufG zj4IN@4fr4x<<*vXK+LqHhw(WmuM6`Hxi~lvB{Uu<#zGZDeZe63O`pLE;H=xr!A!rC z)k4cDgIk|KOwx=F;qc_pN?_VlI%u%0q?m-)Esl;BlW@7FP~dPNmgL*OJQU@6g1@`h z<3;m?N<60)at~w3=A>-ocZy`e23@~?%Ov89@$tqdD=#m+9UzW&SH>n>-1guwM# zX)Z>;5Hs?GV=Zu5`E$e9*G9r}tsMJZt;c+9y-k8%FXg&DyHORby`+68=%+tInmdULm)V~xF&g}NFVB4f-0 z8v7MjSrBGr?N+@do3ygxr3z><0#;GC(PcIOH?76(8f%50ye|t5Xq5_#@tW$(XPX3=_-i$ zFM1~WI67WbOmt;(Fc@jI>n#@U*$R{O&eonMnwzgZ$(MvE)s^c#?(Nb`>%4EpkGL$3 z@#T?Z5LHEI?mAuz8A`<>EqMM6T6t+D+PX69yx2d7j!AbLb&ufYNHing0$vsg*7Bf-^XATam}IRzty zikAZ$*d`?!|Ik_Qr>?&nSa`a@pvma?T(LB6H`CDy@0kVM4G4+0)K?K_v>S8Xj>lMH z-oH!s9Q8&Qg|Ch$hlSoe`kUMC9WSFSWHgEF^QwtMdGwSXl^Vs*^Cr93d*kuR$dAkWW{Uo{^rriKMdh>CpGztQ#HSbf6IO&fbYx ztTCp#ZgzHEVP?>lOnaJE_Cq1JE6G#iRLM^t{OkUtl+uJ{Vxyph{UjkKJl4A1d^^e{ zPKAd#;+OMe?Qt?3!C!QeTIjHFLruGF z#>Tw0%^L-4|Eq`Y)@$p`&+k&jxnwzT>x;bV>A^%hF{#yjWxF6nJnGlH_s2f03kjGO zlD3lgSNrFh0MIDU)cahlg_^DDzN1~jg0P>_Nhj5!1iKoI+ zt$bNyPS$4xha^>@QSWdwa?t#Yd&KDwKt8dC&2PzbakurFeXV@dTU8IB`)n9i6&aw0 zl@mQT+^#;Iy}s1s_CF?tS7G(9c^&~J4CeP8s^=;-#Zr$7n#R!W}}odH0>0feFv z1S&7iuSa7hnV!w(zcJwY8bh$Go+n5Ds*kl?qC)@=-8hmAya)wBAh~i`MU4oLZGKdz zUf$E>1r9?6ulpgkVzei(K-p5GzMlLiJ;v6)=hOL;P}p>FY-oGUKN-GMgt_98ce`At zE~g8HglNS2S#0k=x9;qI{P{2_*X;i2pHHb)*7_m#HSjLAqshHLB67|B))RDL=Yo^F z8+p3|h)ME|2RdZN>kyJmzxS1z5;!v3KjpqEUE}l4;o6m`B-UTr4^J#~?$Y1s@OgZ> z_6@>OP?CO>Bk}%ZPy@}543zc8Zl}^kUq}rDEdegUBqnV3?8&A(5XBRt7<21r;`P^J zb6~>iV#*?~mG-Rd-S4E7#q<`Z&pekSvk3`nR`c83vJ`KalMy+Tlgnbsb3;w-bXKdj zwr*FS(G~6TaSS(tC%9H!p06aaWP%-ucxtUyngeHZk-~T8);t!z;-+#Z#hJVd-d(wE zzA=(i<*BLo4d<^h!Bk!J6?l)M-mR_m2Fk;} zaykH+^Oh$kBAd|O<3I)<-zcN%7w?)qny0Hbf+2nRUG=%03EL9NjCk^Ia&ZWFPUhuE zA0veb6Lh_|@)LD)9xgNv~G{I&RT1@cKI5NFT)wEE!JAhZ_6X`!$M1Jyf(Ki+AZ-_5%WXbLr{PIFIp(_UjmjZHaO^@v+{k)7~DxK=5Yc z24hwDJQ~|zqWk62lB_#lJlze?8};#|=#sh33{!UJQx1U(!`#}HScDK9Vm3;B1xa$y zbtO&~ zfl*0B6A{YFK!Sn-ftN3vE|(LCkwE?>n}z`sRkr=8RWHR(G*(Ka&C*`K8@^^aNf-zs z%RfFkvOg|TEI-5f0pYSQwf0ZLF>!N3ZvSYgXUBEjHfo zWUN9NUC44AxV;-70|z`j&;bb>1`ejo$W)=w>M_6i_T?GvVX95VUvLXrz4_J!!G;TZ(tXl zm+iJ%T`xH;yi`n#u;0CVxkxquKDFl%xl}3CtCWzPp+{%G6T&ofKbDegNsAD>^Cc*?jFy7oRDmt@O%k^%1yMUg^ z4HpZ2Nwux$)Enz31II1dhyu>>v_MQ<{udAqH7}uT`MjBf} z@w{^m=Pz8ZgPAWzWPG$ZixE(9Vv&1dB)k)@S9c-B0O)(`4M_~Q`(o`4+#xU>P~Nb>M3IFq`p`+^pn z*LH^rXIuRI{B-#rWs1&vHbw0dbz3&~1hsI&VX{nU-UQjF1~n6WM^xB{-g6Z)pD9s+ zK6dZ0o7zd#Z9dPR`BnKE1s`j2{m z`bbsU-x70ad)?{yDTc+-HBIo_`tDiMhcxdL8MYc z8-YE}Kti3@^>I!1>Ib>CEnD%gpBIAXXJ%?Tvdw@UB^U7`I2d>AQ{e&FQ&*QX`o7oV z#o7&KnO@FJ&DA9+R;i=Kpf@!0)k0<3DKPrduQhA;lDI*;!Stn-!<>ta0WmL%wf_g! z#p_BC6VB&>SHgu0QWFekKNSwF)7Cw6S0ROOqCl+c*a3Zq-%PQVF4r`I3kxFk0B2p@ z6`UA(dA)qhbTzK*Fq!zpGpzRIbu05#)0ZqNm9yQYcUjbS>7^IgJBGWTxt`StUO70{ zWNnzd%As+hJAP`B8jN>}-1%a6kAoEMeV^#Pao5>8{2`6i%;0h(m(yMwz!xT*&9I;S zUWz*^Jy1ia@VSbnT+shJ5GN7PP(2=(1eq#(PP*^3_lx=R7EgFCciZJ}Q)EqhkX=C_ zpXO%A)z&bHp2Br~oKr~52^K#DOOw{zDo<>z#+ zy$f8f4$ouAUe(x#6Tv8E<1|O(r413!wiU3!=`T^Zfc!yM%hR;$RGraYZs+#K!;)T6 zQ8+rU-JHES%bTZs3!cmlGSAgCUX$ltHMTxatD#2kjp|`)y~E4C*|9XQ0qG8J81%`- zCXbuAL97%s((uv|B&JWccl%Z4ZwqHN6X1Hf^g64b?Ev%ko0&U% zxd_8&tM{nHk{rq4U#c$CeSJuTu)AIu#mD2h&^sE2ZAY`ic4h-9L+$JIY>4s$5g2sk zb@cw!Z~*~Pd(739{+8~t$o0}OP}}U~8uzX>#_26b-Ohq8mmN8JQpl|8b#;`%MOzY= z6ZPF71TZK3&!=BN3B$?LpY)&Btpp;{zEDM==bpa_s#;(O1bVDC5|kAu0`1qOS0pD& z!4gC+pQNU&>|nZfDnUijb-*{Ft*SV=x@xrzH!KxOaOjM36@*Vt5Q_=$Vnw5y(+?Z^ zd#Z}bepK)%OpwzBoj)Yg-PLM#KsSiOmapm?2R`cBH*eM2RBBE$_Rx*C5z zJah-R3K_sDVkj`<5$&s;cL}w%2psZ95a&p4hfMaVDPFb~16&u{E(K zwrzd=yytvpt<$T2c2#%Xd-uMpu6^MJDuTf4E(f`UrpxD~(I4j9`%avvpbx3uu%C=fT{ zdtW`|;BF%DL*aYrbT(nai0k0}Z!nw{V-^LpW>eV~2k)|;Is`z=Rb?KRJzkNBGWfaa z1!F<5ki}+akMnEn9b=Qon6#wzbtv*?f<%8J?1e2KM~7dErw>Y_Y2+wAcQp(k)IPN6 z4Vj?2wsvvPJjl+jJYfVH#rY~Yt;urQ45Qr3bi)VAF2uCr82ABGSv5k-B2PLTN4weh zP?LMP2e>|a!2cdgV<`eoNdgRX{MZ&@4qEfiZ?Cjr$fpd`G0?b~?sOa~BK?`8hOqsb zPG*3rhxmXHHJ(LSH;nFdt~XXPpaktG2fZqsG&%!?M%5WQmX#k3l7B9K(2lZi!P0v) zO15Mdo{&Ss1izP2F2!e0;#QCM9o~eJ1ufzJ!jVtF{SfNL>)%65>RSB|LBQ{ASj^Sd zFA;?)ZFIXdi`dyj5t0&ioYvGhDpFF*c~kU30!iZYP0yX`7*Nn*Yw+ng`$+fWn(enx zmJp3a>pdR#*VRZp^nXqXoyU7)L1~ow-bBa3V1RYci#)CihTeQy8rv>9TzOIkEk&}> zZ-c_C-IlNS72qr*7xD5Zue))394KM3J}y})=v)eH^F8a;2NJO4t$rco+C*1YNKa3I z-p!A=dCOc=ojxxC3czmY8t}x`((tj*dI2b*9I^+4|H#)M3bPP=NUov5DPMjdpr?dn z+;{Wg`-F7wPCC%C@ivpmWk5>#uz@a31Hq4su$Vgcp<(|6BlJSMg*@41 z0t|WprO-ouHm&9=%!eOrv{{aV?e#Q={k9fmET8(xE0HzD5TAnX`+m>UW@mx&7hn$< zQL0ijcdmCk59g0@PGGlkBr)(!WT4=`UlJ)3kO`{oyboz_;&>Eq8hLPUq6dWoJ)ACr zes?`4pH*WOb_@AUsKJ{>US0{dMekzFXg);ld)JsqQRT?TMk`Rq4lbmP@X~j`bL;9n zjyykT=9j>R@edpPK zm(A93r=>Sx9BA^H>78q?Gwa16skHaszoep)O2XvnBly!0u-VE^=T*8%u)IM>K% z{L@mt+~X)?GXNkJv0@uAj$+`4D&2KtErz^9M#2?qxMN+AL7cEt}z7`smN*sOM zrgq^Y+~e0ueEi=)kK)HSIX;h)rI;K+8UmAw3`7YdcJXvB5liJLtvt#oi zA+I)u5^aXGygXr>h0*E9zaRt`IuafNT!&W$I+FfR-}OE&VD3W3*Ia`!*&Lp=gW-7x zMvpwv$?ex8HxX!PcaB#97S=XO{d)+%AyKUf?n$KY~jBS`relyA#-R zfX%Iw?9v>>*2F5B->l$<2%biXCH5ki%gE|Jf5r^m8KQ=5J9V@iP}P*1`&_7}WH zulgYs2gYYOwu6zGem{39P2mB*b>P=+YJ~AlofxQ@IscSNRuvaB{31TLvZp%8ug72^ zX(s01{QI|lckS%)YOCjC!2ZJC9#&ahH7y<6&e0xg9PEdThymZof8;j8W7Q>ESCPzd zki1wNDbBQNctKN(P}t*KTJI;zUl!}(*b_*rP_WUPPjq-vW1LAkqBv9<7{n58F&tdl z-`F??LMBQd0o}jHv=>uTQx!73q}F%RK(Qj{E)zkrojxE5k9-FQOfn;{G-NgOK!JE1 z%!pDWiWHgG-(pZVIg{P-ZEetfvjNmWE85oTje&5Zh5eD|*jKEuU>(Ymq40vIR%Dg?Am1X+D@Pj)bNFLVasCHw%ObG^~ z7eJ68c^>frgZDI{V%~>qnRmBSyEL~ucVkMQr~5HzOff7{*Yj}6Xw*R~EG+y_m?VRQ z(lG}B$)DI4U_LiGe!2;)C7Z7$RCD+~fV%#p7^EUOUorMn(h4gESJB~r^zw(Hg9Fq7 zAJ;~Rm$&@irXKXR1lN3Eox;GSoPIC15WZ19e~-XO3+_$HY|mr5mI0o0OBnN9om(SdPcherjC!c#t}ZMf zUQ*DcCP5;`!vC6ux2$c2+hzAX#^txxM&;w zwZjR*x*0duGHP-v0DOZ5!DcMULWI9X%WBOdV9|<;xUDH+sbW^e!gScxj>oVS;%b5< zg1}RgSQ8vYLcrViao8gOKv6X*W{`1vReO&TMQG7%Z}|~z zK7s#qiBgR`Ym66oJ6S4@0Oy+uWya z005^HohLpM)CP(otW8X87V;vMhw;7x8KGJnk{q)8$r zcp$p=LUn6d`esTHu+SRK|I6;h6@V}(E3#g1cJ>Q2a8j}Y*)HbdcoJ6gpHC=>I>AipgYmh07)c{DcNU6z zZB9Q#Ng~IY#<3NC;=vNMZexIhlh=c*cC_I`we2RDmfVv0u0Q@ z*tOd`I`({hzA>7k4=S_-!;ugJB8BB1?Y(F!=30pLze}tmJ)O!|pf!VGN`H?@Rqv$; zLct)XH-nQUD_kNM2h0E8jGP-8s_`0VrOJi6uog#Ve^6YG;DTvV5IKQryn~XGp*BD2Pq%qo-H!)fSCsVNxe#4tf za%mI8^YQnHCGg9r}B{xFdmSaNkEVr-%*(!tl9;$gHU(o5!X9vmFJ zx#;XiGU|mburg2k_9f zN0rptO56?_B88SjQ!Jjya{8$P;29x^VJ}uoQ>H%@{CMt|+hknPk{nLO$$hD2%+hgn z=3T=xW5>7fYj~3M{Rg8XXc16!dkE6%Q*&12HTvTe5I`c6ke1!LbvxXhuaNMz^9}Oh zlHU;nQh^cK{`)r&vS(*=Atj+!{0iaq>=e^-1ozTzuWl`>r)T=V>x)-^<+KM6<|V>A zR@1DnZfv<56p<-#my+cDE|#dRj)ohSZ)a9ws{;<_-#AXv_21eqa0U)-W6j@E79fo1 z&-$dSYFVs_NNC6;EKFiX^8eFrLWul-+Rd@awRxE1Io%&5;1O*u^$r7vr)$nl%)om& z&&|%ajW3@IWJh?dT`++1+hu^=3`jwSqskRFmp^euItq#c1S!~2hlbRs;^Njlw0J=G zI>-xwY&<9~f0d;}2&KfJ;8{^PF0>_DNdS~V;4}yBl?@zD7olm(>3$ex8H_E$p_!YyNYFdH2(_ zSD``BY&H@6%f(jlt~7>Efs=( z%OxzNR*C}Z3((JtALCG95#aB3Z`#n)5qp`w|9TJ-6XsU&F6nBXExK7-eH=JG=nVum143WT|_ z^86m-hMo^LmT{han82#3^W90_h#PfVETpfhvI*J%B9Pt_*uX-kebWQ!4(ZdGs@Jb~ zsXOt(jK})hIa3#7dt);dA;A#ICnmseNSbj2)_o2G~_*|DFq+}QXO8JZsA?RAG3*M7EuZWR^c8*0A!i=_ff9(j=a6EwMS; z3)Jptvg5HG`}%Z&0`by=hw-`P7^Wk!T)h@72r)72&uRv08KW1f$3p~Q0CkH;tWaQ6 z2nr6!*==rNPG~QiJqGE>$yB_!E+HRsRSH*$@H-^q=J(zd z${D6FTvpdE5z=)AQ~agwBvNX4ICd^+Hw+vNo3{{=Mi=$^KW!|ut+m*!|(ZZy8QyE6wgo}mC3bllAC-AksMMHPE0*(mk0yYe}GQ+3$ycSZgykNh34 zo&##HnchjW;IFvS&aTmK$SFD*#1jjnE46JN#H5YFMUf-*oNLhBR(E>U%m0P#fYr;G|N~#i2>zZ z@OQGoDU;Gt{SGvmigY=6xj^v<%LAz#41o0d%k>8`mcCHTo*=QK?^e95=%~Y68c;A4 zk_OY#!>yQ^)ks<#K+mt2>*}xN{3@%#S_BFg=yG%EUF~l!2o-YJSQ3M6=syJtun$)~{IdyY=Ry;q z3m@r74p>fstKFhh(@y_H?-P+IEh~eKr$)|jD;yUpPDZSz_8P68nLuxx(;#*K%kED= zT2Ui2L%LeG03>Cl<62z`g{9ejJzXB;T&;~S@?C81H+H^>%ytQvH|S0juOZCMAvi~1 zkELN^n)2GZQ~KBCh|^DOA(o$bKuIo@!o|S>2S_!RRGlWxKkzf%h~f9-X0!6C!~}12 zA%kGH*MIBegNL3pm&0ZT`kW+J1r!uF!uWO{$)z(Ix=b}v=Xyn?ASo&aicHMBYAuDP zKkg?h$-)ad;An@MlU6B6oj08-eArOB+6|H(s@}{o(qK8Th=Eb(Jk>s;iQVd3lu} zWt!S-N{Kre$pFO6E#*- zYhV{SRuii8`AP%GcEBKq8bu{?P@~=O#F_hvK(Ob#kyxcdo0v&2WVjRuo;!g#j+70; z013H4Vwm-Buz429#(Mc5I6i^{AU<=~{=2J(!#ZBv5G8yGbF>Nk{y`up9H;x^ah+j; z3}8lv(PS}`+iLuI2O_F1-DicobBwm<5i0L zZ=BF9_iMi-07C|p@q%In5RYCsT}!m!$V@FyojNHw_I@{Yb zI@@244!;-WDD~340lyNbY)u3FX!whzZ1vu-d!C23UUIe^9Hz+)w}pTQV>9pPM<7B0 zAK%(ijlqWhVQggt*@UCcOO(>Tt+_c4%g0HLD$N`L?<9^kuN#xE-f$S=SGVuqM<#I% zkT(Oj8j8M4Nl7Kq^Uvw*1f!@Dv~%!)6!~(YMdIQ`cZVB1Ue+r~mhbPoP~4vbo}VLE zAD@BGTc6u8U)Rht4BCJG-3b(>8RL>xffzdJKJCr;j@V+^@R2dW8=X&~pj@qGZ9r$w z;M%pD(pTmK{`wj}|2jWcwZXdUp;_U8Q|oOz4{VsWc;Zp6ZvpKVC_e0Smicp+`I|w9 z_P9b?lAf9x)2g}~XHsEJjr-VciOf1?m&rlp^z_O6+WJzR4*TCgO^wtjkf2t5t*n{R zuuCEXc4=)}{^|t@|LXYo_|z`%F@u_D%k8XVp@R&QkrHfht-iC<^Wn7l#ZM@GLhy0L z6nXOtKnBxP9jF6xdRK@A2PE#`6#v~^>Ht-XjH6N)p-S}^{tpISjfH!wQQfP19 ztvdnnrCvOY(0r@JEJ4S_8&}{+#7321xAU@qZl>kIA| zW`m{A>+*(=-g0clJpVdzz;Q7C(KOkL^7%>bCuggJL&8J*)1*u@7vew@g!mcPKai`E z?kV$%t`E3Kc;xRpZ|0*n;FY(Xv>l4u$jAEZtn=Ag^F}SPoKMVSOUqD9I*O!E)>5)w z8l$_@7T<$Z^T@=g6;z4D;7ghXx8f^sCYlDu6t{uDs+w-28=`8-E_-Qdc5xPHF%+G9 zIbVs(icxknE;YvD+Aj=n{#myODsM}7cT;z_|G--F(KO$75ZYx}>w1kIKz7Y0>U*#3O!_OzChCjYG``32@O-`C~pt?U(*nG7{| zD)I=QdQs&*C7PB6quFN<&`6%GjemkyH2?VO9nxEN%;c&WdPvuz{)r;f)hlC?UgAvN zh#>8&w_Ye2v$1alE04YcHLwb$wKBv=pMLqTTWgW0v%Ew&?5~8j7eGgVj}MQeFQ`>*Q)C}xF6fIe$k3& zukaxUDM%X{+szLutD2K0=b`$O{2N7coZf31b4k zWgLVx@~M@cYIR!MYr0(z?`t`$G;@W##W>6k854wVE0p}{(kFCm5yHjDX5{_jnV92? zCMz8fWF_nrdg#?4WB`T@F7A4I-|et1;Y>6L@-7Gs*vmX?4BK}nE0XDox9^|#-h_^L z29!o$fIkv;LnBQC(12ohR;%u0Cds5ak@XY4`HE?R;C(N3PK5C5 zaZ%puLTG@HV+|u#OmoM-7yRh${%}_WM?-JStH z_6zU|*L69kwL!|)r@6UG)*%QxqCk)Jb5J#|nnM-^=XT8I3V7Epb~d_w?Gy-Y!>q73 zknVA@_Rzq8$rPQ<6F`-2iwAn)5_Wr!%>h5bz_wUAtCVTNOum1Y`hP!%)HkA-JML{^ zcWmP6sFba38C_}>U5j1F<3)kD@u;W_b)c7-&ZxI^{#L)U9sb$m51}mRvOn4n#m#8c z&2;4K?00^>QzYuFZSQn`ofwYbpbIpK%VST6{`rl7r#KiBR&C!;Mx8}mFdS#b(ZTxS z{5)IOGd_y}=5>ac$8S*TZC#$O`Ws*n@rUO<=D99`u;4)Ow4d5gUGr;U4u=e&J9uCP_ zxO-e~G>1l351&{>VxWbElfV`tV$f+m=jHi0OrR_IUDyUzgJl1;gmvBb1XW04A4^+b z^PpSemptHOvy-3M$RqdG+ACgYsL1Ex`qoC{LBrNpD6@F)Tc`Ub5Vu3wK&3DA3{fQEi-U*ju7-!J>6JKJh0NRjgQd)GRloBcHAz?X^QaAXhQ z+v@~#iIQp&q67&dNE-bh~d_HRB5kK1Jh_NLch>$nz?W#n$`E`4z#2y_QIz%kc^N>g#@Y$4i@VUtj z0Rh1{RZ|(<>-(C4-D>B3{{*~;$;A@>*ZLN2nX4KGxs}5)OS4nNE@qpW>+_iUojCBm zD)5;zVL<%t$^Yv?3xzi_>C)&|x$Y7ycj!tPhjQue_I5j(u;2#7(JOkuf5*)Df3h3B zDH)vzdB4B6yG95q)0kB5zl6;GjrP8LS#A+%KbSCjt@QmorDyK^)tWrO{4w4GJc(#8 zOE?ca2tSfx7^Shp%}d;xVeWoCG))n=y_c4 z**cxg74!w(Xl*^ZO!a(b^{@u^#Sx{D2Hpm~M}_rm`Q0%;oxh=M%ZWdRFSiK4fQBVK zOAv0j=LP%gVmpz%5?EP!0MTHqc(Pb~9lT!68fNUove?rt*x-i&D>yQfJMeWG`0;@y zd^qCx@xc847M3UcDNG*VarMT+{23Mcwno&>&j9^X@ewiwTSVf&0x;?o1_(3;ZY>1{ zgrh`Xv(NNA4kqgXpM3+3i|Yfg-k1fyVhc%IR zdU)KPkJ)E{?<+k%yYxmc@MeBjwVtiO_pT8>9vsDdYNNM-OSO?1QR7 zXD9Aw-t!B!P^hb=FX0y?tonj*;Ael``&v(yu;t+mgs-9h4OBW-5Y&M9j$B>$-Tu+n z1vB>%2hC?|UEs&xogyW{w>i@U$BHD&ejCXEnzIRQ$EO?5M}oKJtxwge@134kpH)4d z#KF%qLQkh@mCm%mfu)Ye(4egF{W;~Eq2>Aa_o0G;k-3kDzG)Ci?nN@ezN2UQ(IbzT z=3)6M&;Q^^=qW?^wW#hfTWIGMVcb}XuG#31bwOptK&k?+}o|lO);MX>35X>~P#&KDHIbXh2b~?O?G7bTQMZ6Zg zLTT5Z4P0;(EGecOL4P(J;!|Qgm2rdy@!q0n99aw$F-oH!kdUzG5-4LAaz>O$LM!dv zez^*S7j)tSse`)$!V!33LpyyM=zz1*h*?I`86k9U zP;agJ4xW5%Zt#EI)_wh~#Dj+=wHagN{8k@Li6UuG{g(zg7&?|kXyOD+vdBgq!)}8} zYB#TZI7)x~1M4XpYWvt*88$|=SZozc2uu``8F)JkYQvAIb$&AaHwIT-506=nAVc5t zmG-YO(6qY#gpon8>`d@~A_iKDKit-QeIoRD@AYTVJ-qx401W1)eb<6hoxhUhD4X0Q zy`Ih!zB=C>?CY)rK4+6DPjx?!<0lBcT8#y+yi)ZV);?@~Eq%GnD?EIJ8GRfaW<&06 zy}ud}2>5~UTxYwjT*0m9!&ctq_p_cC{N^={Z(3#{v=iv6iXa;%X!>$d(gVle!JF(+ zh7GduAqza-T*U&$wy)IA-4)&vuqd--x&GD@dkLSP#C{nn0`*59*ty=i8W}0d<#TtA z1Oh(0FwA!T&rx}Sw@q%Bp#COeB}y9k$m${2J};*`FFVeT0(a73Zx6HdkfN&#D+sVxyVc^~ z^BMJ8{bGO>- zx{j1S7+e^bnwmP=ZTP%SW+*v&+YQI+6s~RgzQ(2J3I^OR(CEf(xxEd80^K(ME^e|4 zd7RWN8~HzV6BhWS1r!#QsBuUN+he>T_DAn;;;U#_0HhejckNY`iteCx6XP7pT-tpB zLBB=5HP>G9=41F$%F{t(1%*e69>ND{5L7M~+-1X@(6QNoX|r*DTJ(C$C!N2KDEw-W z8wh_ccGB@xtCFROaUBN~rJ=Jj; z!2mH`!-e!ECtw6A3EKnp-?Q&dj=suf4BMP`h`(;~Bn{@nP|(5_AIQVHf)K@HV|^W3 zb>ggY1;4aDruToOEF*6vd}i zBD}`vs?+_Jk)nOyM>SJ<|8}9lO8(+B`A29Y8w5?V_h)DTM;uD(7ya25jmTv?O)O;# z3p7{yHN;k9$}<@3k`^kC+!Qz^23*=Ag@+a?w+804#sqGV({yFSXy~Be)5a*YFaa(*CasY7$Two1?B0 zMN4N&fJ<|pj6@+Mjz|kh;|=4iwq=dt7y8SyjRwuWeslN2k}LPOCK5Kfq92L~ z0B|<=1PrlU$~~u|TREh z=2nv~F~3c+8Wjl&8-`nhocVw!?!XiNt*};gVj-{o(~>z z@Q}9Of8Khxq^vREZ_>ce6t-8nt~r7hj?d5aK;Pr*6PheKx)B+Qy&p?{1SSmu3=qMU#E@b8pr#?Dr`}R3zNExsx+s~mwH+iwgD{=HN=tpy zfdHWjFC6nDW>h_sNUkXJ12SI+!iVx#E4p;oR(<`9H31fI0*p9?LGN^L z8bJ)1Vr*$8*~HwR5=G{0wXd)`)k$!R2&-}rMcVe?+I;Eaxqq3-U3poXm=V_7P9wV|^+jY=z27yC`8Ba*6PA)__&w!aNoiOiwfp-+X*EfS zb@_!RBS;tb<$#;5%G&u4%@nQ&s@<#%ho@mP!?nm#ms z8lfsq8h}ZnmPxBjg|X>mNEY@Gl7(|ShH*h(Me;~ox^2!9TNkAXe7FqCsS z9y*$&H?C2mp~d2hPUl zW6Oo3BgD3`W+5^C&;Sv&bpZ&tOB#T5=;TslAx0tu_h?nCKm#jgB81!> zymNLy0A>GH1}${FV;p3S&dSg);vDwMNQ!9$+TO~zZFQsYZ0=a^YhI>!(lE{VaM*vp z5hX{FV~WuLh!~3uQ<`F@&Rv=jAV>S*NeBg3LN!RBa%)DqS{USh!|x0VYWr|&6*Mm> znGcIsYQfuzJ6LfgYH90FP0c4$%r%C>ftNdfM-K6VHRvuZ-tY9mTx#<-Pz9sM)zDxO zP4XkFsM(YCrfO&^?k1s;FcyJCT9rW@PPN96n1up&#zIfmlEj2h|2bsekr-#=*0(#} z8C0}}L@!4I`Kw$iQApupeFVebHnq!d!try-k)gwxXnj^~1X=Jyi8)ytd?>xP5qu2H zEaaIEpig&s45iwOCMGosD?@r+^e$`j;dMa~Nq!&;dB%2^K7pv2!#0_)W#ALj?Wu<6ZqrNyJeqaeT#Q_366v0Tu=aM@5-DFUff<1 zeyYVkspRUi+Z)K}U?N7XWa|oTnJ`G;^_i4es#w6a(&NNh!;YVU^MP&MaM4<#D*Z0h zNalsAUv%yAxh22QtL}b~RcXPJ((J*I;IvZH;Za7Y;Y9T>ojmew@!|C}9-wTco}lel zxNJ+Y#GSKVGNaQbK(LJ*6h91hGw;VYvSEZAzvB#u*{9f!88E~r5sW`bz%Bin_`+Ju zI))=%QY!c*@FdXCd@swd%Oqt9(=ir3E-KpWsSApGY*hTd`?a?cPp|_9SHn{dvgw++&9QMkY&PU; z3d9|x4m8&!{dotbCMvn6j@(&*i=;outn%ZLZ()_A(Um0Ohf=nzVGTEtrl8V!c}F*`N{X@n{&Wi&2W(v0YoflDQffJeEP^v? zP-BeppvzK90N)1;2{^d~9b{!nRtTJQ0t9@^wdqT`#8D|VRWNj*x0PDguh zB-%k{&viI61JIet?0vFf^ePF_FQE-l6fs?H+B=q4H2kCEMcpKzni=)7&g5N%W$5Gb z9a8SR>+JrW$!z?6#jO(roU*H`&E4J7#%4gB?kdce&AjJqebiXb>%x z1@c7k@Z7QyC2C70h$HDB`QyicUH(Rchf0Eun)zF2OxhmBAB0YcMcx8<+%&-GHovD0 zLRoAW3k@^`RdW-u>=N#dVDiKsT}3+qc#~mt1tgF-;0h!?M??f1ktCvgoX>pPrA)x- z`&(WrfqPuBRfPSjYX9E~vlU{7UW?D-Ykuifr_3pV#w1XOGGhIu3i#%z@pj>}jG^yT&|U zyvVK+Nw(rdrx0AVtbL#G$fA?=0ER?q_Hd4&>G8;``7jF_K_|=q4$*rpt8xVvX~k30 z?$xAyGgDP{IZXVci}qW&-WhJ1xH0ZdO+~Du@K%=oAYV+~9oZDA7O&htDHGQj|GT`b zYFv_Fi4V1g3tzvgqBVV$k^_?3IE+K81^-;tJgsm4*XV_W(a1Lc`!Tne>PS4;gaxKG$%A!Jag;{zxoL>jim> zGpa&1BwE6}*iV`dJVJ6xC5>jfMRo6sR9!9Hw+ROWe_1ZcHd22q%1WvEVma{-?=mcd z4a(jJUZDA+*jSBlF|C`Di$0y^C=XRFk~HlGLh7E9*bQ1N#5j9@7a^+|#JKJ0pDXiU zZ@=0P1@wBZ!}6$b&BMSXTc0%j&{&Novc~aR=}#Fu1vN9Y$tUYjhJH$pTleE~nFgYA zPSrDL#zXI?$s91&aME)M>1oUr`(diL(`)4MFE*ZF|6V1efg2?mhVFpX5?REl#c*{0 zerVO77j_|cyBJod8AGX7Rb`n+(E-CU$|k0SfCA;sdVR-Mp^HA={cXh1Ufw|lgb`b` z&=XeF9pH0VWsRLDG&a=UE7JDnY%r&OBBHY1^@k&{ZwWht2teIw);T(ospIMzdo9LA z`Ekv(FtXPqVuq}9AbRg!p+nl0c~lvzcW9(R z!lEAyE%JzAOBO7oInY&g+Q}lroH)~(LTE=7Hc$kh7+|CZh*jPkntk}#)}B_;^4qAL z=-B+htJ7vF?x|ri>-S0Bb^+IPLaVbm#Y}Cds&>=iSjR8l)1tJ>d>-%-GH)k1ea%Ok z`1$LgNZq;I!Kh0;Gv-XAE`VNBvv?`tuOYZodG0`Ji*-1A10vSoY z%9_UFYu!Waj_zUA7^+B@=nKbcJlgBPAZ5Fnv1{!idG}sHLOIzd6*0_#&eF>m*E8>61y08BH~M7zl4i}TDTzP{ z)|dq)a}NXD4^a%RN;_+4R%@H9O1f`)$OGDMGPrke=5}`SD0&QUT%Bpe1J9bak{2j_ zQI~MLSsZEveD#49hBe?tluaV9^eN}`DLUEmb74Pe>zR1(B zvU^hY$O?t$fwYy%&nr1JS^g&GPAE=iL8;)v{r1ZU7mHEj*6ze5-SHg0i=D6)V&fiE zyWCI>ovNs|;?|c;?rw$9iw05{8`&kZ&b3yhik-nSR?MaxRn9}QIi2 z-&10J7%DF(m%fZ>eD!O23B^bki^PWB79j@zx9lFNn|7ys*dFqur>#EM>Df~>N;-q~z2l-)4VtSIzVjlLzrwwb-Ts$dE~st8_ecL>uB*f)p+ zkp5}7{XLFvq6ksGNgY-<&^ljt4%a^Ysei_z+2_|$F28x2zPhc1$?>8n@7IW-T`k2M zMR9gF&(nI6MD16$hHqWpYdfg4@zgHrr9KqWl9nsav$@22F;UJvKi?_9TKbcx#@Tk$ z9AjO&+iLQXilN6xzeCKXxTi9(YZZ#5@!$=Q?swI zG!eC}@!F+xBHNPM{E7)ZI|TiorUpgf;k))tmtyyUOR{{$Qf3)|0#yot*-flf>>gx# zLL)QOiasscOf`N#V3SvHbCfQsResI%7riVrh=1?on+n(H?=09_Xnb5sBPo>(JUzUT zf1-A9!N;*4{tLMX$||@*HsFDsV2@}yGgvOVGx+)m#UmLu!niuJhCRI0-q z?WTYSR-tAV9<4J5?=(;S~V0Fu?$RU^&WnB zv`XDN8ZW9xtyLM-%?q}EE7!Ao(bDlAS6N+NT$?){tHzm_|GV0O2cB`h+B!bpiFsfZ z+nn7?H<9SJuZ*T~ReZL{)5fyeY|BX{_#qI-951QPn4GCviRrq);5Tk|{eX)|o{L&; zv!56F?^%=6mPZ0nYlhIB#i%^|J-R|AZX6Fz4fgL!MLFkT85j@dgGD?YCOd}eVMYMs zFutaPvqod))XHGhwDuWJ$+^pkiG6sPmdBD1xr8=eGgWLXbxqYLZRt5W^37J9u4qJ> z)Q4n7sB%q1ho7c9M!_S1E?GWKA3nK&W48^cw0l1fz(R)` zWu9%o;OPc4B;N$(*eE6?r4WuZHys_&2$idH3X}pVHRA?%9EK!0KGkL0*03|HsXB)t zt`+lya(v#^5Duyi&K5M5WCmgWYlHuF!V!t|GKoTIn9s{em+wAcMmI@b} z0~lJ)0s$@QA|Z$&t>z_Eu@^;`kGWsF1;x}_g?SJD)zy!h7T@T~d~_32k7_lq$xYr>-2Bwsfa2#d2Q4f-`Xb_HfwREZ z^=h^a*%aH+vfF)8##lYZ%AB%BtP-bPt$jlC@$Z#^Q<>uU4gbb(c;kYYW^Q?%;Top7 zcL&V!Xe&zL1@OjeVpB9Q^=&Qc4q5y?MpAW7W;m=8tc>MrcZ+pf+|STT)n`;)@?@Y@ z>#&XOR@>VCPoa zX3+2K4~%^d7F=dt7T8`_@>}SZ=TIa`8h;-s4qRek+J~t zI9x36KK|nor{`%qi(%vVzm0rCP9VfTFKbp_NU)$+Y>P!&(yXc|k4M>NOO+`8Y}vEF9C(dC^|r8E(fiF-gMMI@2BA+)qF?SRE_%$yPF+D9)61Q@?Gmdad=~ zJ?YX~JkUB+s50M(GiGXZnOqbLQq(u)%uCNOaty&O%fy>)#gm@Q^ihqF6~Aa%9-sF# zFYhjYnuvIfN|!^I6H$ArTKcRMBBDE_)@|S}U!?Tri*|sFXX>o;+7?`|a@o-_$QOXB)%NJ8yE1n+pS6Q|%rbx+^%myYmtd8TcuO)TRcBbVPU)I;= ztIP4)mtB7NXcCL3S6Y_X5mA{Z6pzjw(Gjt|&%LG;7^nxxy-5hEQZ8O8xTm{tIBAer zO4!Oq_WK~3A=M+g0ZIOSb#3{BW_Joe-YT|? z2CMew7=1)_ZXfKDpskGm-frd`I>OsNtB8&i!+dPJnhkd^NCO3wAx)XL(FhsUW*|v8JD(@%8*knCLct7uVtNxh- z(|Njmh~8yz@@m73VgAPD_*(m=lstxLF<&!=6wGq?9ZsG%%asZYB%6q|@rU=yA4K@h zbG2BsV8*5cUCjXaVMukt%pO%XldOI~Z=0o8c`%@U zS%^AU>*JW3;ZHg|hUGo-O;QQS+Dq5|T~%(t#L4&z1|7ecvZ=%W7TfOvOi9j;JOF&D62HjQrq^j=e?HLI zw`tAh#A;*pef)4F>SjVKq|}iC+ptL8R;o7KEL-5{l3v9NOaOqGg-DXzmrJc~UG{%y z`o{1&x-Z@nHc6YtXw1e=V>D=N+iGkZjh)7}ZQE>Y8z(+F;XUvF-ur!?J!`GmvuDrR zKXgYeftC8y_yO=xf43QUSLq!cv}rxEuVutwk}h3ZXNi*T)QEJt-B14&>H9qnk?xJP zw>N2D!c*EJMWQfjw>z)y)d;@q<kGj)$|9(jMPhEp4tXE;j9FwUDOJ{8D{zdS`mw zn8^|pyuMdvilF`uTk(YH1$_OAR9CuRyUFtmQOfF4fM(e`;LAQ zox_Hw?_;I!_Y?*?L3q`rdM=Scc`mAAVy+8V+`8ImNvmm}LkL<}xrdNg;L=L%J*! zX9bVZkD$NcJ*(j-`N2`rK@}t<_F|BR1%4h@_cw%swJztRVqy}Nhyb${&t$P#9p{l7 zFQJQv$5E+5lZ{jyv7E7g3!&NzXJ!}qfa8?QaTvJJ1QHAISu(`CeVd4pv0KL&3x6`w z972!vRE*A+sE0uR!Mu|aRh!hRNok3*7*^xwT}zm;6nF`BI&1hK(ex_QzwW#0M((Hoy6h ztkcn?trd^`Nv^7t?r$&IjjEj26Fp>eAtC8xhO8d@JpXw`MRS8>_pLxU97Y{3#}kMy zHrsJ?JssC*5jH8PJ1kSs&)33YMfg|KlH&GRLgH+;*&g!$U6!ly>g)#Etcg1JV-7jD zFUt5`E_UwLn~9MJdfglPaysvtu_EoJgQhdNo?n}7LaUGwZeBj~HBrS9c^bZ~kI*wZ z5c%7gChyIC67;D5xHt!P@%Ed@!tyc^pZ_<{l92E0-1lOpuBZHbp4%`AjNt~vC_Ji( zK33Uga&QUha91uQWB3sXb6V)N7DTz6_D=5Bw*&|xSdNy7pAPD|8zP5|eNJr*md(k16D)+$XOvX)J#RBb zjskc&OC|)-*9>du0qN6Fg@eAh)7X5^J5=&0_g(GApLQCI7&Bt+)>~Cw*Z&q4ZQ%?JM1(f!VFKxs{kX9>>?oylJ#I(a4!LiH9e&2?UF>%LRYBam* z8y>1k{I0`0Y^`$GJ~C#y>Mi$&+hmXaa_SLJwnU+k{|Hb!&Zx+p%s>qv4bts!pIcL{ zkOSR6e)$StOXDu#lW+pSq~N(L&6IOe-uGpAq{z&B8Z4f&N};!XZ2g$LIo|U0@?*g* z?nGxhU89U#uGaA=FKIEiXI3eyG#NV#>Cj=g>Y2N8+mjv)JX~L{^tHaTxisu!it#@G zw7t#3E|4?rwV~~CG0C^@|Fej<4u+<=UGnvw z>&K!~dp&Gms?=(^s+($YS;<$&l{>5IQPpH08%SL$ZcQ zpZm;<9sjmcYf{BJFSOvo^tU^^^e+3Q7oaU15;s}^EA^EBONZP=ZxiKN4R_=);{*Ia zNrqw+NN|=}O}hT-R*u3<$M_4d|)j?J>VF5#KqnHvW>n zY^m(w6tqf1MG9f-Xz7|&B9PgN?H=!sr@VD|TM0IT^>@_HJG_(olZ22jS2O-pdDXgB zJcz}lLQ(>u=fcFOX}0=+pS^Ayl4+Fl?9G<7XHr@?f0(y9)C_wkeWRIZu2BI?$%vws zxSo=X;7(drjOM+B)Mix$)_b(*G$F+O%O?KGK&SmMY(($ z!%%Xgu27YDz}5ICNg7o8YD^V_0fsCW=dX-R1n-YMuM;tOMso+ck-ft~)!Oy$Q{$@U z?HwJV@Kp+lC&yPCe0sPqC0Sn3}hKz|UPuIUFHPN*=SCC>7Uiq-Pqm`~S6e{X`YBohSpv zy@7-+Ek{n<;0`+vrPuo|Hv#+gVmsSy@O|FmC0^+~t!V)|8|%o`daVBY_9w>!=%a*7ODN?~K%MV_Ak*kE z@Z4dmSNl@MX}8(PVkaVBIBGh>nUtNOgQz{khS z`r-BLj6V(UD3mv~rU~}u;+n<$1NXJ}3<`CkkR^H6uy}7CjrOZCWpFXGNomkaod%1F z!8ZDFaC(sawp9(0r_(vY%fU;f46O$uUlA&7ZftP+~VgUv<~=~Rd+pJTQDZe*QKX@(H~{r1K>SSSMzgL2BFg2 zU#xJ)PMoQ#qL@_vSW2fRp+g*=dN4-#Au1_|joh(f;iN!87HW%uvpSj3p!oaiKr-_^ zC3|_^`lX85;%}kak1FEs>#LWlOPg4~i>p{WjgL#+7v-YD{*IfMPt70KMQ&~heZ!QI z0uS*W@?B47A)j%yBKNP_a@dGm|p%AgIG?c!*vZI4($LF(!n$H3G3Xiza)=PI%?Qj@NAp(9cKWvtT$g$Ck1^`R zIt!wrX|!y{g)(auk?-g0kVd#=^8r*pYA^aA0^nw+ zhZMe|{0D+JViXdZroBbJ>!Imj7}8w2h>u}Y;h9(I{zxy@aQ!p$MLZR{w{r|@T z{2cufR)YOA?7Qic66!dEMx6umDA3o*OPNup!)&z^9|y8CC8ZF|Rg)ej@z-ERt1N*^ zB5PBm;ABh*df>?(|Wq{5kqLbhbAg>xJ~0Q9%iUG#$2`IfG1VrZXTQ`*-P9B zZyJqNK1P_qEv`14m*n{ZL20UuDd|yLGGUMe)a08)Skw=S!C^w5J&97tW2C!|m^Toz zlneXD-_m?3FhwRD{E=%WeIbMuP>%a2o-7I4wuuDBfOoUa(WEIPhj@%z$ch_(2OXm-f*Hn;&mbc`>?vov ze#a_|^m$wDAqDvSIg6anC%6P9Lr&>W4A`=&-*KX~(e!%%p=mm`&T%ml+C!Gc7o8U_ zq%!?P`*u1pXq&Go;-CThD0rnj;{tyRlbC1)j5TGBgUUC*dzqRWjsMgZ*JFZp9) z*OrqMA-zbA#~vZ8M%=Tl5j9+kFg7i(#w$W)K>whjy(k@wZLq316H~?>t%4F>5t^8gC&V^9=-tCLnXG7(*?L|E7C@m6{UR2d*uWpn;Djo)e+Ylpd7K0qk zfrtiKSz$%6vGKo!;ruIliRrV)G`LX(dU?U?RZgGY8Zk#()Ot2m7vV&DDz)8WU(05Ku4Go z-_F4M{tX3`rMPb50*W_ov0alH6qt+FwFyPJIAx3qx_%50A?^ac>oDS!X5-_pdl-ED zp8MW+MNYM3wWgR(@rQ}Kcb-c+;_j+`P#}@gJy?%JS>-W=7zw@!SYIK_Mawu zx`0>B$wzjbPmc16^bTo!MKalUyd|oCw+Hiui-WrS5|c23lSPw!`Zi*!*O#k3-xhJ0 zLi$p-P12(OPA3-+jKpM9V@9228&lbKyf7L^^^@rtC)udr0d)-N_o$L|Lu0GAz47m| z*>k4UAmsRN-#Z&o71EDhW|GS9Qh3lRuws&Z;~w~OD1nD}cCA(e&yo3l5C1Ym22!-r z)4lh=txtgksa z9=2SV0NMPUp10}gr?+e2kNFxqeuYM;T`?xmHNP8Ey5GPF*ePW?b9^fEt%RwYNA59O zwE}x4^QJH6@&$#s+n^I7V?JGbGdB^jlL$3sX~Vrf?_+6FYS0P1e9v{-dJVkckM~;M zdX`8hw)(r$vUjHIz2CVT0C9#=ZcA%;zYkgdb<|v^U3B zD+`Cmh5sc@*U5FgU{%8lo`jIxiMw8o@a?GYL0Iwx6Vjt`fxATR7pJK?IIfoP{?W+h zNCMs9=D*U(2a<9`ytn@zTVk~V63-4-nVMjOi$fc`o_RlkozB4ZvDdXX;+&;_uhBUA zR}6ok%4)~PM+QrMepJQ55}~jkzXTa9_mR^rSF?2pL_M$#IG`s4<~7K6Gw-W_rZP;- z%tuDs1YihR7IG#Kce6yqn=AF!m;PCg4CAs+J1>Dk`S-;N9gfUY$!PV%S(AMBJP50( zVu%H_a4GYBM&zrtdbWCTv5digR-L9}qGLf1jM=>B1Ii_0UEUENNADyXF4v>uE>=pf z1!VXsOpbPi$08!+ycByB$Ei_UmCj!xQ0;$n}<uFJ zW#_V<8cAVnH<|7x)1&$fJI&>bPcUy(F@)YdF0X->TdwrlwXaL3x|fDr8(v4(W8|N1 zr63XwSxqKjdxJOci8d!3Gl(Ga&18I@G4D@;Mu^RvqlmUw1G3o6Bf%+mT@PhzlDMKO zRwgiAsI9m!p!++oZJ~joWE>M2RThi)C9!+FB| zrxxVc=<1Do`)VIw2@lGEZvxH0%R1$kC_-fhD|J2S0_s`zQ=KN0!#}?n7X02fTDL2m z5lw{IKUGEIRB3E)&5`Y{tY(v&#}HdH!V}J$(`$72zC3bjA>pWWCgU$}JKyCGtY?$m zb{2d4fe-~x9wP_LA0IssGz0xhrnEmS&}6I3D!Uf(35fYS?LLWl<@q|S?PaK2L&OL$ zc(<0_4({_d8HOUc82}2R-e14oHxe5SBK z+L6S`f54&heppxYIoN*z0Y86=5jq{#7G&WedL(rByXwq1s%p2JY{Zsze|fvpz8u=U z5}|su#&I{CfZVqlcxHh-FqeHbz(0c02)l2omAv_|vijMWZQY5%<4fi_w}U+u(y^nK z!lGz)5R4DEHiZz>(MB#OgNi!G znl^`+_emfc`Jgc_1alX)O6YvA{uae|)><6a^m%pL?e!@y3zs)16kwF*M2MCni(b#X zpuzq_R=?8%e74ldzwY-s7}}vV92%XY?X&FjBDj$_YUAzgz0-@5W&s|SKQd#)`!N*+ zGt#krIZb~3fKDSZiy3FfyNSruLi5eO!MR$i(NVSb20 z*_4*0uo~a>@>|b}m-CoUM=fiSDboaeb>_Sah2T@47G9L|U36Hj*<|P!9{%W;uVtD% z>hzjTWj0zwxWw(VUe4Pc>>N82?kRrOIWde62$9bskgbZGq&dzmKU^|3IQc94ii(ZXTUsmrS<&{e!R=XKnh z%}(Ac8tjg8RXrmszp)@tVz+g6ZeBBJYBNSSNx%GxLkna$w;(j_Or#>b2i7J;e0!rkYCn>B->qgLO|(z4#p?9e0!mV&)O*_ zF%{5FDje1I@M=H4agH0fZRqF$ds_D&r@PaOne-aHZL9?9YmRR2BD}dIz369*E{>QW z<>rNyyRNQ^VTM|#bH(c-h!FQ1>Vx%5J>0~yJn*d6wGcG9dsWau(|S;WWy3muRN48~ z-Iw=%mBr-l$|WR#hreMk83A_E``FFEdGn5c{a)8&Ka-WhsQX-^`vUO3ID)HCuX%rF zn}N%B;sVWzUzMj4|D7Bue@(o6J7zlcrshMOKg9VGBEvu0P*RX>9u9@?L2xc zGPYl`(bBj6z2c+r-~N+k>h^P>Ze(A+6TFR&?QC|+VTclJJZ+wPZy?mwh0u08Jgz#I zx4b@B{)QFA&-Qv&=T=ETl*OirR+nxI<%H{K`|Fr&(&)>?NG!oRj_a-lbeVMPusG^Ml+c{jjYKYCxm5j~SWRU50?4lF(H!M8(WIB%I`!LhtXa4nP zh`0M@sy$1Ksb&RhW$eB+hdwXwXHg^tQQM7<^Yy}^nNaDcF%oyUR;?;BUbAV@^wG0h z3P2FaboJ$?fn$h?*`ILKcn#D54+jEK6OsneY&S9SWoOVq(l@%$N$;ESotuHsnwjYm z#YSN7yAuwhmem}{)FTgqwV~SE;&*BulOK8Bx=H&+mCDzC#l|KVC#3fGV+56EH&7a6 zyLLa>%}*;io&9!lJDPf@J;3!VK&cEfe8JMNV@{XbDNn1R^y`-|)@K*K`+~0wOGkpB zd*vKnZ~LbKC%N3DVnR_teiutu-=n(Aw;4iSj;CjBNe#MU2e+NK2YkE2->--J;yJwT zJ4>HRl;Mu2vpW3Ff?$R=AevGdp=#LwtUq(;KEKEl^VpvcG*I6z^iDGdeq5{j70n;e zjbs~S;MV2Mt^;2!oo#07-n?RocpRViLK@s`%nhKe6>+*3Hd$pDFWN{yi5F zS*+#YWgi>{^tL%2Uu8WT`1Sei7U4*8-BEQ8dwo1Jebm2iE#R4js>oz`7wdDo-h&4| z0WrdvIg%&n$sIOE1W&T`bEH3DC(wl`cO*T633zx~xRVHu;0ai#u?ZFGNH;hEeq7hJ3 zn#2moB|PBTZ1mmC8Vk4Zyqew8g)^Xp_MZUblpiwz{A8YOijR@IEJukAW|H5tz28Ut zZnD^QR@$samRzi6WTwMLM+#RH{2fa@muj-I)0`cq+`bm-1ilb(udJ~V zZ+1kT*lc;&w;wU;_zE8HZozo1TPk%6p5?~*gXaE9!@LlAY5a{}trOvOo8H97?CTCF z*jIhp*$W}*g$Z?4&1WWTqs>IJ2rV{$TPs_2z*mjT1ON~N7QAE72uwaDk_6GHYqmUS z7@Jq^3cFdFX|@E{nI^{y@LPo@#yPaV;`ITFV1gn4pZjv%)))egfCPR zW2+k(rPo)J+%%m#yH91&Vnr*plm+vxpwJyVSnCkv)koFHZ3GV9E_$E5<{_w%DC)T z%2}$Sw_Po=+Ku_dZf_u8g*OEH6vFz}Hv#mEyK|rE_U$m7f4wI0U#&O(&?%hJZQFrs z{MeaZ3X$^s=-+#q@XAIwS$ts2u=^#@>D(=_edy|ZcF#RC+D#t1wv=4pl_CU};dVuw zRTZ7dgMsO|{*^HH7@NS-$m@M$wVcQnQf5_F07W46K`mp>JaN_o-<62B-;@hrsY+~zqiX&j=J50CTL$(C{kR7B=HPE~s~CwQacE-o`nc+z%3zY$S3r8^S@X!#$(>P_Ln|TEfxh zoz8woXoAn%#vYON(=i0waZY~tEtH17%?L-4u+1wsC}YQo6Hi0j9bE>VUCd|nsSg|@ z46N_BRz~?l*8-Rt2~N;7C3)C*f3Q1egnY-|+x}SI-l(naUtF7e4az484`S{BOa@UHia(M3Y^S(UNNj>Do&*jhX(q8MR z_O(|M)vzvbf%k}UyIfI<+20*Qe2k1_@}yu}3&$PY%y?N_<85#NXvPuv3v~_RSc+ow zCjWXr3z`R@()~?bm=+ltO!TgD#jDX+ z3f=17E+ot1e{KVIda>LFs)H`7H9?R}2+KQ+QCj3=-nJr{@}kT4QU3g~Z9r5bxE7@# zTlE~0`h8^W!)d#9Kd`Ld()-n=G8f$Gc2`W?_3pXU!_fB&Y(AJ8(8plZ9fpuEtPpVB z62rfESe6Lvar5%-w3L}bL06JQ*Q9DoRwSkaMQrKol|MlCh7l*1u%E-vGTN4k3bfdw2TPRA+>uVKZ)@9Ci z(_vkA^v6{d$f!scUQXr1RhjH7fUG}u`1?4Optqv|$edV^$Hngnhf01+V!r%l0#Ct0 z?FDPUq_CIUuyEK5s4Wf|`#WT8-pO*Br>%i8L^O`)>A?YAFL+!;iL7?-!`syc29Do# zrzh95#c!V$6T+qvZXZ37h!F{iRvKKE1sGhB*LTytQC=wcx&OYtUA=SRMnSPz^NY)@ zg$A^O>>XubogcTlzDPk;(;NXe+FY1q0oVUdo|ZN*HIuE13#6gtDSpD2))ctx9X*t) zsTu&cS~=`5XecLzDVTLDwot4f<>Te=5d+gwE3`#fO!WG#HsUmqRejlMx%iaw+g*gQ z;$Hc;x9j{|Ko_qovAq~#2{2p0#m=^GUhJnJORXEI+NW@C5nIm-xamkhjs`=xQRg($ zn;rHOVVm3OC|WutgH1m=v}lkl1`TuNYY|6d%iG!6qDl3uOx~<6?S^zRxM1qz#o~D6 zC{aZSj(YyC{A=X#YvgER^*%`QO*t@KxcM4F%S#-;~trF97UdE?~n#iR^!qnRvDz%y5Kco z6f~CHGLVb3z*^(3jioVfcN-q{xj-EjF1Q1KXkF+wAY7`XFw`cus$~H+T0T1|R;jUR z(V-A>HHQO@;FLuP8e6We&OPjJa4g@ukDO1Rf`Zf$3BurS8Xp4*wmW?&b7JlYLY#QR z;)j%nmBGn#dD_s;YSL6@1K!&5td6=|Y3bF@u6Fo@_!A!GMME9Dd^(tzSXRV@yUuq7avoMAkO+{Bs9T!O{CP(DKBSBS zY5ecBo!-!em?U8!9XF$y@Y^OO20ls?X7%mtctE!_^6x1Xh-$RydGIPq96K?x?*abt z#YWoP4tx{)WJH)_RYZ%xn*r;~l5;n2vAgvU$*ngEb!X|?x0i<~)6K=>{5g1Eiw8|}P{547B7|x3n%u*iRlX6?s?UH)At$bOlO#!9d z-L8@GMsHrk^W#{}CpIR3&+F+@P;LT(DV$p9ZPQnGS3u#DD}Nf|UE-a5ZM3N90=Q=4O~ zr~F(g3eZP_7DUFm+HOtS*OF^wZPWQV?*@N&0~2oIYi8Wt$`LFC`#nmB-Saqz7`lE& z(U={bMAmb=nE}OCuSygD1qz0tN*%RS#3KU2tF--lrx*mNyvos7NH_Udf?Ts{5rsk* zCz77m)wt_7ZTX@2aW_3UnnjK2e*f0k{9%#_q*X~SZ^)1vNoc?dW<8GpjE&SiwmO`EAo3GSw3s?`ao7F zYL9z6tk4JJk?b4!HJWBY(fu7~h2hxx=E-HB9uMRkwpBb+n)p`Ijk{p}Aw zzE_UV-U+_l`-M=OWmo6_Wn_zl~!D*X$XTKsP-z14Ij z@U8u&EX>)(8|>+@2^bn`IOO;)>0~LcruJZyc)3I_3P)5P&ow+@wd6570Oh) z@3MfH38VhfSPMz9dQbp_UT%S#_ZRxss)od~lV5S@9$(kcKsJ{>?hmhgdj(ZmDWgzW zMo!)9>%0oE^|3BWUpgTe8B{cJ)ZZZqh8&9yPHY}mTMH09PI15Ir5Pd>+NDN5T6u#% z!%&Ha$)x#@_tx?M9}6(C;~NsFb9KYt!IbmNTpfZ`oAiBLYsvsi06aL?TM7}h>gYXf z2z~6n?*BZ{^Gq`FrszLxhCmou6(HGbRS7(|d$%ssYVv)1FKfH1`M8aaxBT%CG#PI> znv*gO+P8f8F| zwN5E7PCoyEF|9Hrx|c#bPT@S_pCsnTDvIyPPptFX00k>3dkF+2v~RsM5t^Y|l!dTo zzc5y5>0kKEf2o*>{yg|w8z}=loH~7VrR4Q8I_Ulww3zn=k>T#LW5Bq@&IQ%`YV6F*?qevBQe&>vS0AsEhf0SCGsq_{c2HYrvwegb{Fi+t2R~ko`**Yj8j)WD9e~uBsb?SaGWLx;*#x{>lq}T=WkH&Ao`| zT#RS-{@2!>ukpnq=L%G?8ilKsI*4yD5X#Eqc?Y19AyUUZ6bXOs-?)zwT`W{Oz0`8@ zr5J$rO8SVUgp$A$K|a86dCiphMfd(A?uO3Hj;P&2>4~2m@{N4>*cX>HIsfrKsFBQa!9Padx3V} z4Soce-d>tE>#6iWQOUpzcnXtqjsjO#BQs`KOl{ijCgEvpO(b_Xj3)fN^Q8=m1(Y5a zvycRRpH&EjZV9gB{(<@oalYU$JEFK<2{^z$w(3CBDgQigeAxadoKLByFvf@viwxkN zP2%S|A78l}L8WLD07Z?TL!qiOUk}Jt{MlP^*QS$$Q>8>8nO{Wvv-P0s%L!~9 zY0Dqf%=mgZn-!Wr@JJhuf&r{x_H?3r>augyc^b>ns+O|Q2RCT{Tz~l4hr{7#PY+x$ z0Ujc+18WkBhO3iNjv9#m5L!iIIw#ff`QVm=jCJ4#3*0#*bpy^(Fufq4KjX zs(&4vHI>`lpm3jcemh8Yte?(IOOx@>57Uo7FFgdHX&`z5NM!n_EET6_7yq>vH=G}g zfs~;?Yv8Z}0MtJL6r!WAu5lGDcCH7V12cllTWdVLoNq^`|GjW&IvOTre*L2Uh2MCz zwRm}X<+Hs7DIMt;>@@F~`kvcfOFs|+h#eYeIm2jH4zBUN&iCJwczppPj&x8W@0x|2AX|&~{i~q( zk8*OKDLw(%)b!Lw$#U-;G(Z^YSvZO?@&b0VB+n0)@aT{8xV857?x(X9@oN32#f%CG?NoDc(}C)2#U@Q2B{^r@lgd&wf|9m0N^ z54k_E>Bbe#^^J#9lfo(tuNtMJ+y8nggT`i^uDY*&oIey#}ErEZ-3}t`{F?0r$<7jU1}(Mz;Tt@qSm3IIfdS~`Y1D0- z7i{2LZSDP=;@hGzexZB$2XAlfSlJY)ev#pzm10gH@|Uw)!S&aUEcDEYFud2@B|`H)!RZr zPb0d^O6+;6)Wr)cxhV}I=5YC4nJ)t=BBXv_pjx&C>ecrkEY(%R1q7&_c$C)%*JFXL z=)%wHkf-w%=^|M8gllO43-|?#i+QA_!?!6zn{eu+IZnU6ODP_we)G^fNXBrxn+Ebj zw0@f^Q1GJVjDNBD+V)Y8S|&>V-kt2K@%wBvEU#-zQ3wmri2Ruzh>B>Z-(p*3_x^OQ z&ob@Owl?{!$Ioh$;@80eHrM#Er1@bfocm>yAb>{MJ~-^M184f}_Foa-s5A_(p6BaL zMa>5vv{{}wjQ3Yn8JkP17c+6D7h64s)06gF2|%&+XSl;zR9panY^3^hp3y8p%FHur zXEbD)N%!8|&Pq}k3E>|ZZ>U)lh=UDD2*6e$j@HjCH4x@f=W_WvibXAEb2;SL2moNe zm1_#tf+ReiA2)*Tx(6weeBgeqtaUg0x!xNwW=(h+ExdH*y6z<63K@}!!K1>_IJ&?6 z`^Dy^Esf6kgqiE?J3YK~zN)36otoZyu}+H=Pvcq%B+g3xlHj3U6)j(RYO62UQh7)c zUyFhTRWw)#AQ2lhXU1M_tQN=Q<@}Osvk3VPK{Dni7>7KBn{U&1n85(xsTdB^(gM?d#OAF*BleF#jWOQNIC1x zkCR&I9YlM%6{E8G>h(s2T}A@y1B##CR^LQ;n3QZU0#CN%y8CHHr z{$;1C+0I@_>G#T9eWtZU)xJfzuGdh^S^tI6j1HiM&ykLduKO7+KKVlZeUcFQX5^|O zj?pD9-4SE*J{TWJ=?lxBJca-c%ntJ>LoEeyyx3fzDYBfyc1ffFxa`a&jZjra2?RU6 zz+H;fii|6*Kn?TlzEE|!RBJ*~ZEyeO0 zA3UW+YPpMWAr=Mz7tp9C8ce-+zxnyt*B2~Iw0qa#+I_!*p}y@tur}1OS@a%mG+MXb zWIFM5aUoIE**IQe>*&&C2&;$oRd3fX!M~<)+;b|Ad@{yrT5}b0*~;W(&@J?_6kSa! z&LRMHE+N~blbgr4)@`$}LKnZ&{q(E}v8N03fGBD5WLBXGUBv59kei~#8u~*u^id5} z+iicXsj`g~V1huunj_op>NDGB8gg!vr8|Bnzz2R`8GS1oyRmFa_wYiPoE!jMeN2Hq zcDq1Z>pgDrEIs40cKnFKKT&lCXkM9kXa3*Uc3VQqz{?J34nciBO!Kl8kP9TpYa#OP zb~HF;e0sz}3hAA>TxqehG$d`rMr*O#cv{^b5%hk!EQ|jnkRV=?8?W02Azr%t{H%Kq zsoP9IWh!tWB!sz%A`WIc=qYT?fvC;jKr{>rDr$*I@a2A4@FOI_4sS}tzzbN66HEAU za_MF!mOPq>1OD>55#!@ke1uc(BG9M&d^aQT@{pp>2>u>PxRCI~Db-0Q98$ul5$l%5N&KK3+ciejB)J9G3$O10A zG6cL{pFROQ`Pq2(&@79>vK;G;2Z~c-2>&bZ{D0*YFU3u=_t8cDyI=gec-BgrnXeMf zi7sxJ$pr@bT@S3ZALsZ@qehJCG-;h_3Vs`!FPn0!PLtomoI}eco1DdUYP*v4sr!em zpguK^dXhImS^lR+(o`QttmX7@{bF{}pqbRD^YI4$YrPk(08zqAa& zG&|vaeUcfzck3U|5VWxDK9^H*_z7g+MAOv#x-aj0zm{RQU9ED|NupLUD4y#(67!b9 z$^W*H$Pm(b?C}Nzb2zw^4{Jy%2p3PF{mI+N-hOfD{vruH+d7ZdBBV^zF=ysC(hWdcR5+y=E3~}a8HCR{QcLMR9wV9y}sb|1=0ynEsJ0=HKx@H zmD0o%3j14asrm>3Ps}GOcKQGX&@h98nN^XZ{z?5V!i187x0ILlT_b4mKcFx@nggG$ zy;tcDDRlrrHrqTLJqBnJ=S}C!*{>%-A9wN}2itEDSBUFNHz})Suk280>?RS2>hH(y zSSZcjND&Kd06_;qL#PB9++d&ti7)~L5X|$q;lzW(6-N>6TM#J+t<}=_Qna)>rkJA) zhaLATc=op>CIu24yxIk8 zd=cceT1pJRu^kuVXys6qCCTyCAZ}48?S-E=dEp6N{c^HtyDNtMr#cgnh0a~DV@#4i@hXIBG=ACoSbD#75zWlc*$y&+E zo|Wum<+@B0t@s=d4_u+e9eik5jzsSpil)VOzt&*C3H&t9?K-(Q8{qg2&gRWLC`5_eNt}@CIx^WE*MrJD z_nJ(F(>@c!=BoN%pSeZn;*xnulX0`@HohEn&tqUEJij40S#B;ej}YC@5*sVEoL?uZ zDG@TxUHaWofyf1#>E)r+T7p|mbP4JTX}Y}Lt+bwUEDpq>)X>>eQk-&vpn5%Kw&zOB zr2QPKO*X{#Viqrncq0w#EjH+k*q$%Q9>_5d<479|d{?kZZ|;yGxuVX9M0O#vsioP> zw39~(hf)2dgG;N?TDxELnw?5K0|N^8&f`sSxfp|uY+?#;?9yH3WI2mx1yN0Z&eX0_ zXdlrZ*WT<^eu?Ed%eN^t{5NYRk3Sq6HY*mWzzQXvqg#&}Bmr@={~jlsr#uLc6DE#z@oj9?FpUZ}eX z7L>_=l94LE$A%;6-`n2vh(r!5m*ju2(LaxtXd7E@QVqxb#Xiu}0Taqa#$}j^dxvh^ zoD$fVN9(GdN_g*7>d2Hw$ttEYjc`|!vqR{O?^vJz)^2ow4B`04jDB$Kk#9ZQWqBi{ z=e8!LWRShFD>f;B)p%`TuS11eUIG+$J>tkvFIegXEY1x?Vi-)?ZCg`O)Uokcrb2n3 zLyoCpf5_kDDpY-;p`uwwMnM&Yz*+XkUiB+dG5Xy22(Zgo|7vR%tfstYD`Jlo4kOjsWs5KWcTV4(ladWpwOrb-;VR(7*>k;VhzV1B92L}V1aFRD0 zSU+2LdiG*lv0qp`$2D*~UY-X7g1pDkyXv}TS&%HX5kKxJolML2IgI}xPJiml6#Z*O zLZ;sBreO`NygMQTS7Z@m5XV1)!ya@1Lhv!pMQhRJdzp~!W6XDVPiu?X1)3=+j!3%B zWZSV!s}SGA^YR}72K;{lohHJ#3b6ULd12_%IvK0@rTaWL^jV<3B&jT9tBj)pZ*?O z&;GhS-$0ZTv%dU!!r%R8$!PgEym~S8rQISpefJb3N<>a5|IszC6p|G8{Blf323P_H z9n>@&(uSS-lcFFV$~{&K3X+J_1RF5w%B<}CU69##}S4!!IJ1{J3L=I*L z{O`I~^pGH1gnO7*S7)$D` z6+UdcK;E5dOi2UU!63WrhpVEB2OP$o6U}SwVp9c|*Ai05VIUoPL@2HlwtxCuhIwO&2Pkh6y0+Z)mu0|Dgn3SP@mNed3Ux@zgy&8;G4VP-JgHf(PA_+lnb z0}Q#5Q2!>RWp%PrU+Pdx$AThr4+YwA@dCJ2Rk_@*yHkIbz5x}A!?VACewYfLPJ(rHUe)gZQa$jrfxjWtecXF z@`|g3zuaW|3^~jPB)8{zTT1IlndcYgQUuCc4!wNAA6LzMmanlW4c`&a=}E!S3%#2J zd8~FC>UWmWL&&O?7X`((ymNX(pJs4$BpdecX1{iH9Dc!?+clL>)YTK8>D8}-S!twT z_lgBiQralR4ge}QD51FZ#h zmTnV(VR%}1Hr@{ow0R_7NE-HFLQ`#u1JIHrd#?#^sW#cMdkG$nui**_&6gUrVb`@D zY@TR@455fM;x_lSgs)KQ z--eYg$lNcqrHjfvCkneb17Uh>gzn#KIl94VL-(sG0`Jy)3iBmiF=iwvt<8HKp2ViX zy6M{0+bd064^Oa%hZXGbX~ly6UTnXkvIGN;CveDY=f*JV|2!BsJuWy{uWocn&q5HV zxxPK(z|;GQ%}uh;GLIgJ`SZOLGJBe*t({n*cbJTvfY)l_R9iW|F6H|UGxemRnL@@4 zKG<&yU8EP^wqmn+8g44e#o;n8PTrnMx5GLz-ONHn{Wz(>XE+p>lG-8hfVX3pXpLiX zN(xN4KcXD&{i#>44vqVi0cj%Tw}%mA383B9NA#+ zE4&0}6X5UlUO&N{E%lkN)&P(=7K3{AZdwv`#x|BZsEL=_AH`4pd9b({JK4n{w6R!&h@wk8iSy(d$2^^cM~Y z@?xA#3i2qjCA|-vN=-`v#@fAS1Itk^4y|X>Gw^GT6X)$)P#J+oKq_He%qS8+U_pH& zZCHB$z1zv;*M{88k?7mHyd{3miu0vRwc_jsN@V*fs8rvt4cdwEGrXvOURftz=mqGB ztJ%8m7M6lwca0)Z_bR7z3rtYyBFuJfs2(+^n*<2X4V8H))wrI~86Pm>v!Y(d9}$K;MZDKV+{(kt%y6ayp8N{1Sl?6cM1FUb)grX zoKyDqVnS~hbs7OttX6IB0?vAweHmF_M7*xy9$)p~^a$PhvyZ!=d~Q+D&gZE44sprz$HD=7>`sj{@!`#M z?BW(LIb~S86Fj2VQiXh`qvKLS!GT0H(LR37D`r3ZSHgEq1iTQ4AzDBQ=93bt>r2NT zJT;d9S%k>Yu6-UZTKKxr;I?rQZB5KX;f!>?%N&n7H1IrvmWCn?t)8Qm5YMr4Yt-a@ zR(EZkBu7s(i{!Bk&RykNHipPQgCS)N$p^(4UbL~AvcK5o9p8DEu_Js!FMd*Tf1Z8mnv+X#|=@`^!-#!%b8?3ZR1i8cnv<$tVMna*&qR#vkguG7)^?C4oNp2&#N z5HmMHurBR|jv_zqA$f)N0) z`!c``pAQSRW=8wqcv4Nkl1vqpVNjI{8>4+%E$v2*Xy!3&^>6&58x<1R<+AALfEiuL z#`pH>kPAIF=PHk;qmDjkj)wC1IRn0~$(yDLSxdz{fP3)6=FlmY@VWF6on;m^HHCC} z5oV2Yo#kqyfcfy~j|PdBhH9~Rx#xOcghO=&)!z7`U|or>O|Pvd1iQF4BI6dwugS|5 z&+*puPc0E^#$eZ60hZg@04I;@5>ipVTHW8KKec#DcOgg#ABYF}*hQZ0R zj)HL<6n;JlspIz2CirN)K5{AvMfLi~F1^A0x z{$XtPKVI59o0G0FZmO52^0Yh{MLlWIIo$<5uK1l${HbjrEjbx)PzyR;{(EXJl9E{p z64y=khTr0Md!95TPiFi)tT*cQ+cO3SAr`rg{kyM@7K6mq6VhV0Z#P2WrxHKX;;w-k zPx*&S*^8m~q(mX`+g@=20qFfPSXK5oco^{z8|Ha)N^30%6ff%&qUus|YSIbyXcp9Z)%4kK8z-r$Md?m1EigSxsk&jT4S0c8g94i*b;U(7 zL@t-MMLnG~UUF`Jtgts=afoouQ-OI!3tr~y+vYx)yp5jQt$~UeZmE7No3F91|A>4x z$^}_sBz_Q5V@QY_ZcPyBb;NierOs5aBSq7>>lMsxel+I5O6CpX= zk?N`Z>LILf`5ZJ^LpYL#<9lRNj42 z{BCJU+%ICM_wd~xv6?9e$gTFrgezgvFF7Nqvi)TY+=61UgeH?6f~v%L10RJ-_j@#f zO913AD}tW%j0T#>NS8^o;;I|+qVyq@6!Q1N+UrB=f9Rvlc(SM%Up3xxj}Aw(IK&GA zqFD! z@?&r5DDOKu3qZU>HN}$q7VJrQ0ySmz53WkgNOBk zo%<3isXqgAg`tvfO`zQ-PMEt5E^4-1!t~fJ+_@GtK4p_Ab8J$x_?lb3Zos2?$~O11 zZDTZc^l)B=DiXt+d~(m{vh*1!eVNF(cU{fOlwz!`XOa$dEc-7a8U0k>P{`q+#Y0Lp z$yFYvP{PpoAuJc@W@j0R39JZuFAAM_tW8wQq zCcTe;gEM*WUL7T{7nT}x#06LGS;lo<{JKLyVZY;RH5oURN71tt`Y2YHDl6~Cw}Zq8 z+zQ{9eDSIzA-=!l^_&69FAR}=wN)zCK|1rSIoRR@rI8|m+$wWXEI+#9| z$|fgGOh*XewiBXcB=Z$5fStR5edifmoGV+DdE?}fP zSk}BSbJnQMUFv3ZH`l*fpRv~miLD&3n;U&`?t~PI( zjiI!^BrH0&&Ao&;6ODVZh1wt?RF&%mCTTO~@sCfeMWk58FWE{>e(hwWnd-<7Iu_{~ zQNOScZ^HdmFqhyyx}SpbmhHop&I?@RllqJDKiQEEFBDCpW(na`k=!SvJHlSptadBg(!jsx;Z%`NX!|=nK@!#id>Jr zs7O(d^zHgJP8FU5|HMO&@-r!<>?zk!jjf-~j0Hi?B^>4b0WnGRdi#&lP8rWtYP)a7b$v%L)6M+X>Bb}7-S%4oMv7@{@QUqklhPSnayFk zexnuoOJJMPzgl5G-P6=}Fbf@pUhbVv6-I$!3e#&8&tw;rKUA^LUOGPRpH_43lD9-cKTG8E$<>Z+JAHYty|J{eUl!X1AVso0px| z7tV^#M=zWDBClKR*G_l#M;l6>EYUx)&!XN7uVhUOHMaaosI~0uG;ZrUg7t*W&CIv> z7+bG#R)Ts9w*evZZTFSs1iu{@g-i>7EwTOxWuHQRp*@QrtAZ=v?&+fn;XFt_ zM7}(w??-yjc9Id!-B%H?m$&!Tp%4z6KW95qoN{{i}iyqzJjSh_d53|wq-O2m3SHWTH1F*!Pb;de{_25JCxcty% z^|&$R+pU||O9xh?bOrnQ!B-;?GHy&>zR+*PXc5@pg# zH4`YwPXTzRl$?x1g=5%aVlXZn(UU6j0(zbmWM@i}0%X5=4Hc^Khdf6qs)}k2rxbQ= zKiDmA9k{8>^jp2X06#b)c8ZkWSFje^>h@{)Wd{I>X=Q()ZInX2SgcfRbafsm>w1p{ zG7D#pG*6dWAnReUeOhW!pFbXV+`MvZy@?qb7S~-4+-|gK?IBxp0t!FGv=+3#y@Xc1 zk8H-DHrn|%=#6HGb?P)lco>|vi~YU8uJ~|PQ+X=yyc2?_q}ALd#R|7<9k+ecj_@mR z**_2^jQ+E=9nKmj-Yv+Qo%696TX3MS$%l73K*kqM=^@np9$&rt<*XlXQfqa^nDZ39 z^Sg-?o+8J$g}hQ{&{l=l)M02}38lkX*1iF%&9@@-{k!sjJ>vt5X=#-RAaScpvZ|5M5(|)p~^u~T(-w-rs~!u z*`iobvugW9fW|%I9%t1>9HlTD|M7xe+j1pS7gzg%4W}}KoojW3vYW7WXAsh-pz>E< z5LHjWjCF5WV85QS~0YOu8^wZ`}5n0(kU6vBTH2zagm}%}^feU)iIJU+)Ys(eJgfgwK z%UT?{4(V3ATvnQ7?%g35RvXLVua+IuQJU-xdIti3&0C{8i7E1|M$@20e_T*q)OZ_) z5*L%3a;N<>wvw*GSx2@QN}0v#e6%dBIqv`T06^094(rJH1}tVMix%mfrxbmhP}=Xg z9Ho_cyxz<=&lP5mTjA2$=CVjs^@ATS=nrNa8|n~WyHC9o_gdx+z_0Q=4CcZI0P&I9 zvr^f5o+DA8rjLX0jvZI0omZb&2%q5{w}w&SeQK@tksw*DXIEkssPOXd;D>OTT4O^j zZHnp5=)0LZ8oH%BPU*WsoL7ZRx;KQ}_$BgK>57Cjg;pl@+`ij};Aj#Szvxq-oQ!uo z_p_r}ezo!y^`)D!vmA>ZbC61rKQJ-W|L`|4v(zNnGMKow(Fbv5Y0msw6-nVjGb_7JSLGyC`5sa8EtGkyUUz9iJ+u>dq*uRF~hsl2H^X+MRuD>e=~d*bd5)m zfRz>Nhs3W!X%1?eqC0$v1PY@0{y0FOG%>F7r@1mCX$psc8y`PlA^WQPUctmDL z9lqqVy?9k0sk8#x?PJ;q+N_G*I-THsvv<$GnS@^N)zHS!yy<+3zb`ieBn&`V1;y)2 zhT9IL2Lu*z(TJSjp(qi`9$QCmH^R<+J*3?FB^L|sNA?Ospws6kc#&!#B44dDa*B7i zPlN0~fBVRJE0yCfzs?o$*eosr+^+j9iTCFvo!Nm7w_SU8e%abrEgAQX+HHu8_!7v} zx?_{CF&2~SiFAt8tTu2z@g9|m0*8_$<|*8h1-y6OfZi>|`d-6q&CE62&1MQ!OuW6+ z(|z9xds;m@e5q-9+Rrz0so(z~-j0F&o%Y&YCdB>=m*iwewUA3RPA&^1a&qP-{XFtV zftGMi>;o!!zR(~UE}B5%{p~MqvmfuQ2;aET-VA83+_F$u32w}k(m0d-a*G&zapduS z@MZBkwMMss(6>!N0w4NFMU(74eW$E&J#wPucBbI|X7OioRzBtMnvN!@)EpMTgEO5q zgOOPWc30nP`(Af2Hd;9J78f>{+I4&Qwa_hP$x8prOb@!$@Xppu>7}#=){2}Oh_F_e zROj%4ojTF`*i3lZGa{#Y(FoS@VM@`-@K|tKm&?hXfg)OxrH8L?uR}r2fD=1fkfo=> z`eDS@fb17m(Q$R>>5=xgX@IgvhVUm;yjKnZxb2^hyP)_z<>?D<6)qrKE$@|Judb@M zE%sO;ZIf^P%h@@(oOLp5k9>I8HTF2)uIsrTdW$uSH}2GryNtaqS(9(JZflzUvC6k| zKs8#mCo#8sP!Bz6Dj7M4wG@V~*K3ar(LR-TQTnOYO3lsdi67rvi=`q#CP1u-Wth}B zk#?seu>SNut-#KmTPnFY(o>C^nTpsDq|pm3QTWp+uh%c553=8wsIm>qMNHI7##XO`3q( zB}jSuIU;~uARBc_Nu=^ikWp{9S4iV9q=0>Pg=2@{sRu({Fva3l2c3*d#IDX1(@Kp5 z?3j&ZOiDmh(J=*dFlg+4^vz`^udL-&drF&uzU{VFI%gmQtyl3hQ5lG1-dPR1#J3b% zlt&!qp+i?om@PwhNNc}`gQz_;YcNh(tdv%wu5bGI<_0U|ZAOCL8~`XSUe(Hal(f9& z5lxjSWy>)V5_Sil@7mS*3w(bx($fD_Jj6vy9hS;O8skp}!( zbft2=oBrSiMP8q4+j2jKTF5B!^^Rj#Sp(=MH+}efVxNC^!XyxBlEPsqW;B+RdG*6> zX#=iVC~#5L?A&-?espH70a}UyS8_hCrIsf;m3%`GN#6o>fTTyJ;20=lfuZ#qtEU@8 zw>@(?(e;-E#roA&V;Rhxs+^ghEmA%urnd%erwIjW(XrIZm;cG8k=$X@*2+B zy0>%dhY3;d`5rqyX>X=4{Tv;4@AKnhxxi0*UyNAb?+VRFrLp?5ZSBz@eX{nm z-jIB8`Z-o3A|#aTemkTC7c@I`<_LWqmbyM}AkA7~7<$!~IyIBCxEdORN0PC-&w&DX z*Gm``{R*pYt0|!*?``&Bezuyp^!qjpOouKqhcyY)bb3SIYopf*Ix(r%#hJ+1pV~EB z{uV;qWC}9%Gh@WN>mjxo;pX7Ht>^pZfW}oLgDbAFhc~%p{KfBfr)1Pc;-xni9|U`> zPXlcY8dY#T759_HFK>>Z5~uuFnO~feMArbn8E)2eloT=NWV!-X{d;?MeF_&6@&3@_ z6hAbcWQnc0xthSft^>W<-PesIzIIIsIT*qOm^ReD#6`;<_z%;Zgb`WQa9|U#92@U` z)OG#sc#5AV8dNO~Uu?vaN7eEt*0_GeCYt6Wq)pC)iaBKc^oUT}mMQ`o_~<3)<(}>8 z7+lqpOrMr&+`CM5Uu93^a&mSACBxnWJADot7pwEKMH3+};LO}m>ADkk@PnX-E(vI{ zYs`v9H_He|=d)azHb|ZO+`A2XX{sr$bk97mRncRjP!={?HEtXmnH-<^srymG`< z=2BB#&db*@$w*eJs;G!~)YmxH`1L|8DF&~+)#5Jo8XWv51=~vlqnSr+?R?o3Rf~M( zfvVfx*%UX{pVi=^M4UA9iNl*S%SPj+A2_vS$S&0t-j9HmCJepZXz~rb_4m%$XN1Wv zM{mA8WEZzRtv&nh-pn1Ao_MV5>Xr!6;%8@t?MJR<90E>+G(OL8Yt7Y4&akKyJuFW@ zBx0EQCO@cUuQ7ADBkcSm?U$_Rud1)I)&m&iO>P5bU+ zVzKVJaw8_aW5#0~I`7qGQvE?k4Md!3<6oXeo$g=-oG{jI$XDBDTuj_{>1DWs`e zkv4>CQo8+>l6@)@2~)`jeNcOhm0X##ST`f{jK0pBkuhA8T$ItX~{g~l|=EVNY zQO75Ns&g3y{Ppr)j$No?hkFFHym$EO%{=FEv6cRqS%J{wT*EE%->Gy)9GHg)PgdEw zaF6LqSl0eJ8K*YU-KH}tZ%On@kRCH}fK!PM=X@?xCn%b&)Ojzt*+v$?>NfS#3D5mf4pXC(&WKIFQ>X`H~VB*R6~7a5wN((D*d&K>NHFeRhQM> zJkJ-QgSk-k)O{3I+D7r1aqt|3UDhCBd?Q>tJK9Y3D(A3;Ng^dq>^X^-&Ny?&Wk_;U zG_cXpuAP|mqq^evfoB;$mM1nT?uA4h>`flRs2q_cGKcOUldk}Fi*KyPl7|s`-=#}Z z0nULk(=QDPAM0jP{qYR8EW$0q4A(rq&UZ*`R0!pnmMyM$cU-02j2Kw~KC`)Ouq*Yn zG9TimvgmcLkEkDS&pdS;sq!ozJ(J?}+3+}e6g(5)W?fpp+ReQAY{Un0)QVp6W=a#u zmSWtWnv`<5siN8FEN;;?t-kwGOIkKA{qp?g@&lUcgIQV}YDURd!<)|;F!$C6O%Q`?c?T#J|s;Cs9P zu4z@H?o@C_GrBVhbW7Vv^q{nRwV0^U6sHb)+0|wq(&jL7%Bcq~8i{CV0}VX@lb2vO zt)8uuhO-hoi|wcjO(fa1o#HL@Yd0xAH^#?x^iA0O=EhS-Sh(W&1}p;xuJo7lp{ri@ zH``~y>Omf#l9ND}azu6Yp8RH<=(z5*)iZ2t_Ylw)5)Saw_TEP5eNx*|n*{3aR ze%?llw`e3=&r&;7Cuu}xydm#w)!r^Nt;qW6&DJaW6lEl7+g@{#2<^DW6QD&JWOQfr+$XOF*TS+=LMEs@lk2pLgbW0r4Q4!v*u@T=7*+kI9t1xmS|Oe!uN4 zsQWil@%OSUMs(cHPA(vWS;6q41R0f_ySL$}Y+$c~$Ta8#=Ejtp_Vzrd_nJMMQ(Eke zbl_P?XF=FqUN&&|MKi}rY1XaYcx8<^hM`ni;l%OW6S?PZqO~F9D99cn)W{{Es3ZU- zcTeOR`O3M}c`s8tasK(oVu?v;V1}Y$uQ|OwZU^p{M)6FR2z;QHNpDGM{-xo`niMRm z^xmO`1X|Z$Rnv2}3UoR=XEseya4kH%=dv7s^8IyA)T44T^a?!?0iN zN?F$ho=e;|a7)|O*BInh#}=sgl<2kMsp;Lk zdV#g6VEgK2`6KEPc$oCH{YDqj-qAvsd6f)kYu;?R($_=%j&N!8G9!&;A|9HjAV7S( z2i(gP;gnW@mI&{U9X?4{G_7q;k{y2fc^|xJ$HG?d9g@9hxT2|taEUs~XDPI%1t;pm zb0pjB=VLTBAgHoP>e3yN__za6k?~h6f(7N&J>_Igv=swXLD&8kq8!z(G4mZ$+3}hw z#QP1_Ys_|`6Y=)V$+!ySS(+kI=D^dRJ=>cnep^(x_CMkmf&^ZEO2z{Ud0&6lo&{do zmL)R@W!|<2Nt9?BhW7FKH&<_ZtUN^$mzM$FG#mE_zA^=V*}S@nfsqnT*GQyt7M;xQ zW%nf4mP&(`P64k1}*Wc^mC(ew9cC);4y34aT3m^obU~5aURYVt$2JwST?f zdeg=18z=tQu)do2B&C^@AqR65YEmiBFO{}dJ8o8ELp5w<$Ysh?*MP^&!?HLvQ{(m_mr~T)Ej}Lk7e4}jz{=h zgp)C6!}LKMX`SnhbL?&=0xh(7ZJ~*nP*P)QJ7_K{jOysHU;?gV-3$IuUZYBTJS`(o ztsUcQcx*Udw7%48;eqMyMUD4oo4?YHStUkZJ%BH>wmijlZ3a9!r2jEUW%(ZH1rENb za=kn}L=4G#d=7nF$E!NxqF-EAxb511#!x_cpYOxKAih4a_dWK|`iss~{95effpi<~ zT25wee8`dGxdarCa*Sr@fag%tKKSG?Qly+G?86a^APs>HG zhXSF4=UNi1?sK462O08WUmf%23!Ht~VKzUhFyI!%=Tr7E=P-BC4HUd0UpW@{!3{!#ZSJ+w~OLZ(RlnPUM<$N&hs$3wetkj zt(q>>^ac7fn5a$R%z>!nP9zz?wM$HHX}xSTnF(rMvmtDn4@jTVsA_VIvYkMN?+Z5g zXW@d*tjj6S&k&PxVTKx+=MpuOZ3$euttU&TMG{&Fikm&Fc~iz+^T}_NArr1cyFkbM znIp<{z?z3zY#4>eYABy$4=&KqG@mia3pE89P)!(Wk~LaINR*piOd7 zX!=F0F-6nx=O>Zd#v2BKjC0fXly4?oerI(nDmLvq#u~)X_P=i>ahX8`F1y~W2tD?G zJjxW%?A@9EjR^EMWy!bieY?z~7QYlww0$S0M3CD;mVGE08AmD&GGSEN%@Q~X0-Y_g z!x!~%WeIL8z+qGB(1e_9qnGKQal{<0N<+Lmf?Pa3g%73;S&7T5cRm?$Dzt{yjS@Zv z8)|0rAZI}{VURc=Z#~<^lfY}1vlhG_YN;W8{{99Ir|izZmN7F5;jI3H_6Tx_3B)Q6 zUAqEQe!sXG`b2e>+?}!{E<&B(Mwl?WXu3Tg!gy2o-Y{m((@k{tF^vj68i$(v(!V`E z?*EeZYwW(0_CPZ&q~ZRPTG;tvK@0<`v_$`xrYkOb9Dl9G`AU2S@F(=75dM>qFZ-}N zmbrfgRs676XE|IY|BnbogEYjC^aF3+|EK7~ z4p|KBA8m2UpAZsYh=e`n!D-t@Xbq>rKQa{+E(WuaoBDruUN6S{xA@UEPRD;sd{3B` z`bX>kuccQqmm`k6c+q|1ihpkZ*8|6kFnC)SrL3Smw*v6Lau^loQ=YK;#*=-&{#Q{& zMlcwtx$(bVy$X$M2T2Xr|4*3er3`FDCnr0Ol_8ib^PifIAK6mdyvUdTSCb1Cdizgq zoTEXy;;m>s=6{W<$`*f?O};V1{7+BBpDN>wKV9;Le2#OR^a|$;lH)`k^insBI|F82$SKo&r3yVoT0QgU5`wGAC_=Iwqm~Cy;m?h zL&zx8@NPnW50>$+m{(8yM>8R>b883L)_k0iPv~DAPQPwO$^Oeq#oK1VBLy)xf6o6m z7!?;fR!@x<3k7i}%PnIs3g?N5CbS+Q-Hk9uDBdVVZc@SW48I1jJ(qlN;a4a>K;hDU zs$)?4ncn5%nIN?BCgv3C!2*22^l$m5Tj$FyoR3%=Q^-*n&FQ$K{`Z7pxzS@^!87qvG) z?dcFDOO44`+prttYFakCV#7b2>XDtz&4`tfGeR;;7f^!tTA#W^RJo}vx5zJLG@P#To+ z=ptf)k%S7f zFpt1$udeI3pGz|i-o!yjBv-2GXy?U!yjbSj#s&^a?Ac)p_#vFtq2BC~#241?}ZNkfReNd2A(?-^uro8Ih+^jspLb$ALw_E^LH%5rW)Ve4Jz zz`r|bqz?KN50Z3S(?d{9hm*xjG+{-Ln5 zkXWJgvVI8ioNcKMpn?p@f#E|my9o>S18J5=_Ln^mvV6vI(0r@3Q{zf1iSye&z@Q?o zL8l!KrlRCri~aGugki4zTP`A2w-B1eLzZrH9o*Vc-p1=AxgEC(i>Yebum%cQ++{J3t$D1BhJ~4qH-N zv<`cWuAZ8?xcE$ixUuwK^RnUhxx^VfpuRN94;s^!hVJEhNW09l^SEy``k!+Pz{bEZ z8M$!$ZfzBLB7H9}(RqmB1|q_|FyN#i3^oK#2r#y4dD<=S+2VP+@|X;mFg<@vLWu4~ z;|zq*!VbQh;^aJ8&C=zJh~Xp7r1|5(%>?}B9F@*<4|yYf zA)luw{dA=6Bdlr-c2DuJ%(GFCL?jJszb%^8mN=CeAIb}#W<#(f!o~FfPAGbK6kj)7e+^6DG+>H@ z?>Ya~)|<&Lgr4GpRUkA{DN*JIj`(h;4}Pi^0rfmIN}L7i&UfAC;GItH#L7Hu!nDpa z{GYC#9^vrjtG$U_nd>5gd}!5Iu-|s)6k?$XhCkIKM(EWa+H>5xei;Rm*(;^4QsW(18n|{3I z6sZAGh=40N^akyA30uNoGO)Rb`1lIxBlw~PKnq~C%}5JBkUV_2=kfqQTxp!cjzs2d z;ub-jlfaD-2r{52Xt;-fP2fK>d@}IU84=M0xgM#>EuLw-HKQ*IP0+#dr9oMYvO>?S zb?@~TkH@ifDIU~2#96z;(#$LT+L-(D!p0r*&%xV$Pxg(5%6B1ileCu@Srp;W^Dd5%qZZ zz0*HZsq?s5-c*RCjZdBPywqt8|EY8AxcW9he|tAFF~db@j?BTj_n5n>KBBMfhYajc zzdZj&-61JPSUY6t!6O8+d3kvmWiJC;+C%cKw#yp_p**X5m!GUKOp*UPkTfR$hkibt z;*Y=ZUZhB}(lKa0Ygl&>)L9=?ydeXk4;oE{ZG>K%r=`W>>m2AHJwQpCws&e+hiQa6 zB6@2^P6>rAZ+|(QN*G6c&y>)QrCgl5IiuegbxX#%&2e-ooQ?A+K`3hQ=`T+{)g%kEkvCzI%;O;#}ZJCL~bobES#FU)W ztIMBCP@+ruSXZ6ELmS3Rx`F2;G;}Y0?Hs2MdU};ksancKvtsazd;6N0x-(CMdG8%4 zB*t2b;^Gp;)iuTFPMLXWRusnrBY#tU)72?%P^&Sg)p4Em_2}{Yfhn|j<$Byo#&pju z-+DTr&J}J#T;#eysd`yq`&s*iAW# z>y3=lNMmOH_4Vuu?z?NTpU8-Hn|CNU(;=7o6D3GooU}DKOW|7Qb|lV}=?T_K0c`1K z#3Kieg{eu!hmRISuYcxsv`c%T)loCW_!<-u!QQxkR8`IqvyVTwXI&2 zEvE2VoH$7UJ{JYxt{>WB#_?$Pd(8I_xr}Emj5wU9BMI_Ldd8_a?M4M5^sulKhj2}?%{ML55Q1dr8l&?rIa`+x-G=OP|u5Nwc6u9q2yuG z?)fy9!7TDs#i!! zHfRbmbuNl{5UMq*UUIeaU-LD#3bZ4stT6-s=esQrvJQVDs4;6fLWX>D^u&>wbgj8J zo2h{96%`F8BK;(AqjHEHI^6&$bi6;BR{Xl}KQ(zQHy+>M<|+EOq%zk$Ken_(ilSiLIp1!QWy#R=3W}y*8_!ce|CWGEaZr7dhE?aIZg| z=wEuP>A+R<#vITzwLT#iW1mA$BS2DD2jDu%)8+;k4lSZTjhE=f%4wzUgNw-eU29vn z-Q)6Dt2CrZ3V7qX*5ESFIdyj{;?{Y$B_av^fh3jILCnQRHq^&oM&V@U`#fq%L6;gI z2b-qE&dVr0EZS~TD=*Orr0)LiY_=x!or&wNDxiBxv%D&bB7^cd{62snudlp$l;jX3!P{idi)G)CpQ0ym z8j~c3UhRrV%;>hqXa$ifpBIShr72VP4#0$W>7StxvpWC&;F|J~`g$%t=0ftLRxZgi z`&!wDac_(Q-(uK}3xiXizIF3bign=d1n~9@Hj{bA&&qP7nRl;eRp3F5Q~~1R!n!14 z8Y?Xd&~pgpc8{&MODn*jH;cF{5=8m1ESE2+5T%-B4mQ+K|M#DxsYGnStlg%P@`^z+ zl=ZfGR=aC6hVtp@98mz;M1D9sdFiSR+E4?E;$91qVC2bR)De}>ibY}S{#rKfwk&5iYVb{LR`$3K++j-Py@Y52PNIo`}#J&DH+CwB45 zNRMYeRhclaL+m@WZ(~X8E_I;^zA~1#yBp4xSjL z&d=eTm&f~0tF&UX%$?(h?D6E|ly)XbgJxhtj{n0?4O_C1MasV$8ba=dqVmnjFbhp! zA+PunyH20=`40NS&-sC3rz^a^>Ee&KI<%Wy#G&LHW@JoIb&Dz2_lf86WQGg&tGrHP zoIt%teO={ycO0&h!vfb4g_+kifyuG~+pl!gM}OeXYZNWSUV)talV-2(fVJCIe0pb^ zcp2RClP*h27rCEQFMM`(`LPsAj?jmrlYJJo{}>nb-12GuyLb17Ckv zA*u4VsudNuZ-)?~2Ghf!b@{wDs?Oj{b>c%8Usb#JpDBR~3_0ewCjOWARV@0!hCUl& z*LW7Q0C_5DAO#};Hp9Q+6(_08X-%nP=6S+70PUFiwp_yPkxo; zQ^x;>vc^YV|EEa)N2>h4M*oj3%k^@;#9!Nhd4U9`Rp>vlebUg=imAvi3w4 zI!hPuSQUDNx=(UgoVnq~Av;2{(GZUQ#xMWytzF6OxVy>>k;-K>1;c+g3}7{pyHcSM zu~OH+f}d)BqT+hZ*Cfw&)vQ0B*|X;S^s*8VV70|5MHKCDT{LW0`9oaURH;p|(x~^M z74K7z0aZn`Xsxp*M8L$;#_YAf+*Ka$+lyJn7BqYLHZ%&p5%lCEUu_bl_zy{FcYs8! zp{b0GtgZcw%$VPPV0U){TE?LA2R%NVz&*&2AP3@95IH*`P&I! zq=c`J`7i2Z6S3=nd8hV=jWWufN8u1f*MjK3s7innLxFAF_;gad9eIU^p=p(O!45cj zYOHHVBHPa1_*C|H%zC_)6-2$s#lYgOMd2h$2;v#%Y^T4 zor^b^dzT)42Y}>a)_o-4rxOG@1{U^aU%7N|5wkC1(He4_@Jv-s){k%Zp z=*IUjqOFDYoo@J8XtXRpTMrD{9>hwJ8rS7^+)ZS)U(wF`T{U!6GvN>AWFd)NekU%X z>{q!MRz8e^XY;CMnn_;wNVD6kq(f5tpeuL7w@T{B4YTyLw%I*W@tIJrNlVR1gQ#}$ zN~--;8MQnvzjc-z;Y*^$8qs_nf|b^4hKt>`2Q!)43Ef}>#h~uiYt0JBURqgME#242 zXyoB9t?`-voGb};j1c)gp@`e~0ua*+ug1>bV_7`M)sd!-*Tk5#zbiF!w_wop#8IfdJ9A@@<4pPz>s% znzz=$%e*?sQ@?`_LYh3Z=Az`=kav35OV4enR5$6<=?tn_vSK2dp#S0Ke5GV*7;S4d zv}bGRDRHj{74$=f%svU$ng1&jj@>G0sCy(ja-g^-l51-xpUcFK|i}IeBeo73R#_lZ3WUn@F5F-}r zc#lfx&)-%yzm8pN2EIh2MB37FwAB&IOZ%U`Xhx_qVKpC~1W zHjR$5fs1<7?hrxjtSuZnS#C;>7(WK$pOyLm;6eY{PQ=NZ7!I425^U&&irlKv`wncV zpoXc>m3Ifne90FIuN8-G6q7TFtIsqQ_uAmJJdl>pJ6bwncMIqV;`$h0yW}DQ;Ss(7 zP?@|izV|FzmeBzeo(|Ep!qV$R&js(QQIg5*4RWm6~++9=Qa@J4bqvlXAWa! zaH3w|t0Xq4)9GZFutS>)3LNv;upjN8-s`zHc+QlL`c$Fx+60qHR{W*BB14hw=E$H}k*npZXEIpzTz!o8DC| z!dcR-R!9C%6|z50tUrkhEl$fr?Iy?L#?V<@U5J0EpRg7oDi2LJ=|Xcw=4FpyJ!eiO z*wRm_?+e+8DdgKGk$iG4Zg*nL4JFXtwadkW_rAR(*`vKiJ~a11L@gW(Zz+NJcwuXu zU;|D|8$15{R5Fq1$qEA?-h+r8-dGP7iMR=X?6ufCktlI5jMJ=51;6vpRG%N;@+1w* zxrn3@g}sx876*!)aIn!R(u={Rvp!w;PI!5c2V))% z5fHFA4Ab#!Y?NK#)>W3VYcc}wem+9>)>Ui7=gK5O++89)htH$M*ex%H>B;h19(w(; zZ(K*n;?2YaKmkk#o_mi{qYQ~gT!RAsCe9Bm(|O3ucB7P-fBK4Y80G|Z8gzhF&3~fr zMo{=l8CmZwzs0AS0scI?J2!GP^z%+BKm~xARaRP9@zu zYXz`2(MBsSDg4J1(vm0Ujo&G6=QY!4+E2P7WdnX@bbA~@e{vjGQHrp+&dNy`RAzEwplc>{XRsZbX_ix|qDu(K* z!*@%uer!Fm&-=gZ3Ud0w#Jq@QlqhZ_qI)veR1{q6(UBqg0WRPUV()wy6+?25`yLo+ zWw#dMW{lxFi@JX^`G~T`hD3*Pla=r3xo5ar-ID--3z^vRz7uSYANN|xF)Br@i-Xuj z;y#+zZIsr`t#gZZI%B=QYP+LOrm~Z3iAj?9c6?U6dEU9)z1z-OVKGZ|8&8~}B9Po;Gqm7&>zZcPd0Ia(Q2LKDKP@}Zp z69}X+sGEEjA!itPCD-VO!fhO zE~%T5!h3T(nNy#IefqckhBZowe)RMaL5!D2#qYzoafnrpo3LB z=F{D=`?R<~+bl|kEr8kT`aKYPWi19@2BdWc9XE$1HL**NY>S60!X78-Q8$I2!P$+V z76KY))O>xq%b#kF22&hL$9#t0DkAq*S_~8k-pG|A|7&?|>;0wQ&m4*2Q02}_2U!VC z5w|+4_U0cTw+G?+wX5&bTq&Y@^k~JyISZK7na~l3v2uD++Xrt%C;>f6Xn5!U3!sJe zHsgNw@gaY94Ppm)5A#2`eDrL!)>{H#pCW#gwXi;QX9^Z*-OA^{yI_=r{!bw8V{T^t zHncQsl&~~hbGL24TMl5i?yA?z*UHHHytZ~rOp`3&J0%=9B&AFp8qKcXDzA*77m+@< zw9JC-!wqeY%jJlZ6!=msE?>Sz(@r56HO!^u-jRexg5}VG!rlY}&?{(`2i3vlH;Bhf zO>?@IH|v+@(U@q?>bOk)*l12GA+f$!EY$!bYyar4P+*xC=-G8ew{c4_;cgg5^!wZB zM3FXfLY@!yatIfUH9z`ncM25$iDUSvftnaTQ6&Dy7wCa34=%!mZ|_YAQTzZROM!xb z-IW|@wS^Ma;RI1{&1q>|crP|0j9fj{_Ugj@8r=ZhK`QK6{Ylj9%k)(ZVFLdK%C^Wt zDLEf?Za$wIbh!IB%k!Y>d=~`?_n$S0yU9!iPO$)Nn@F6QHrnBd=*=`@JSsoV?j3thPw1UmqF~HoFt!tIh z=W{E^qh)r-#S6`2GlS_v?#p$QW3yT5iaj?@vzYC6WeKgTto`X+Qk~ekFeMMnSXVfV zXLGN9bIlj3c|$E?Kw^LRl4sBb2_rVYjX5RAn;L{w()}g_kn|m~@W$NZq8Cpjg_SQl zFwMM$=I=miSdQ!Q=Zk~$q)F{t9N-kB+UIqqG(0~4#>ia@n9d3yQf=7U&1^iA_00Cw zg`JtnmJuT7>j4&@c(VhUAL>d&?xPM5F{WqVAqS)GCmwYuFd;%plo$4B1@+gn-?_qp zn#K7W;+0cnM*ny;zmFE64`0*)l|tDJbwO@J_4OW&jt681Ai<}ZP3tyeIJUn^cfJeb zr2CN)6*CwS^BG8)Vw4PK^O-;842+4bOvl>frjS1ejtkY=c?YP{T#WB zkzLmtEPy;vK)6~k4O1E0s?Y3vG zHx2;lz?*LSGG|=QzuL%BvdOu)l=^hkUr-=f?r6h)N+#(8q9$zgUM5dL`zk=R5Nb*L z2oCD!Hhrx`-R=2eG-@LmfQ6C*CNfPvb3DwKkb;jS8mA^5>>$jW z^F$UR5L}2gf|_0&xe~Vt+@$DviLaX`L;hKi0;t;@9?zqfBapiMF12Q`kYHO#PdRTW z%j?t2`_l-%uC52vd&{wz?7Q2U6SFuum^D&O;rkw>B35~4|`UEG<;oFEb>2^tOU`gE&hGzvLLq+lox{!^tYbs$L}2MOX!=~ zqKK-ZlSevvs^CPdW#Y1`6SbTn?i{;-=^}|ir4Vgfl$OT7pYhN0ve@+V{3%Lstk=?e zvhE^7B|Of00y&ZxZHBl2*yF+viaKU+hhAjk7IXQvbYg;= Date: Fri, 8 Nov 2024 17:08:14 +0800 Subject: [PATCH 054/211] [VL] Clean up some legacy code and correct minimum required GCC version (#7865) --- .github/workflows/build_bundle_package.yml | 1 - .github/workflows/velox_backend.yml | 95 ---------------------- .github/workflows/velox_backend_cache.yml | 7 -- dev/ci-velox-buildshared-centos-8.sh | 2 - dev/ci-velox-buildstatic-centos-7.sh | 2 - dev/vcpkg/README.md | 2 +- docs/developers/NewToGluten.md | 2 +- ep/build-velox/src/get_velox.sh | 11 --- 8 files changed, 2 insertions(+), 120 deletions(-) diff --git a/.github/workflows/build_bundle_package.yml b/.github/workflows/build_bundle_package.yml index d4f7046f2b0c..a28720656651 100644 --- a/.github/workflows/build_bundle_package.yml +++ b/.github/workflows/build_bundle_package.yml @@ -51,7 +51,6 @@ jobs: - name: Build Gluten velox third party run: | df -a - yum install ccache -y cd $GITHUB_WORKSPACE/ bash dev/ci-velox-buildstatic-centos-7.sh - name: Upload native libs diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 737895be57bd..c316dba6f887 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -69,7 +69,6 @@ jobs: - name: Build Gluten native libraries run: | df -a - yum install ccache -y cd $GITHUB_WORKSPACE/ bash dev/ci-velox-buildstatic-centos-7.sh - name: "Save ccache" @@ -544,14 +543,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.2.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.2 @@ -590,14 +581,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.2.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.2 @@ -629,14 +612,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.3.1 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.3 @@ -676,14 +651,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.3.1 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.3 @@ -716,14 +683,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.4.3 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.4 @@ -763,14 +722,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.4.3 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.4 @@ -803,14 +754,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.5.3 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 @@ -849,14 +792,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.5.3 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5-scala2.13 @@ -895,14 +830,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 @@ -935,14 +862,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.5.3 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 @@ -980,14 +899,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 @@ -1015,12 +926,6 @@ jobs: key: ccache-centos8-release-default-${{github.sha}} restore-keys: | ccache-centos8-release-default - - name: Setup java and maven - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Build Gluten native libraries run: | df -a diff --git a/.github/workflows/velox_backend_cache.yml b/.github/workflows/velox_backend_cache.yml index 71a7a5840b4e..e969ffb7415e 100644 --- a/.github/workflows/velox_backend_cache.yml +++ b/.github/workflows/velox_backend_cache.yml @@ -44,7 +44,6 @@ jobs: - name: Build Gluten native libraries run: | df -a - yum install ccache -y bash dev/ci-velox-buildstatic-centos-7.sh - name: Save Ccache uses: actions/cache/save@v3 @@ -65,12 +64,6 @@ jobs: key: ccache-centos8-release-default-${{github.sha}} restore-keys: | ccache-centos8-release-default - - name: Setup java and maven - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - yum install sudo patch java-1.8.0-openjdk-devel wget -y - bash .github/workflows/util/setup_helper.sh install_maven - name: Build Gluten native libraries run: | df -a diff --git a/dev/ci-velox-buildshared-centos-8.sh b/dev/ci-velox-buildshared-centos-8.sh index f295632ffe35..ff9a62b798b5 100755 --- a/dev/ci-velox-buildshared-centos-8.sh +++ b/dev/ci-velox-buildshared-centos-8.sh @@ -2,8 +2,6 @@ set -e -# TODO: will be removed after docker is updated. -dnf install -y --setopt=install_weak_deps=False gcc-toolset-11 source /opt/rh/gcc-toolset-11/enable ./dev/builddeps-veloxbe.sh --run_setup_script=OFF --build_arrow=OFF --enable_ep_cache=OFF --build_tests=ON \ --build_examples=ON --build_benchmarks=ON diff --git a/dev/ci-velox-buildstatic-centos-7.sh b/dev/ci-velox-buildstatic-centos-7.sh index 76bd33d7ffa7..6895891a98e9 100755 --- a/dev/ci-velox-buildstatic-centos-7.sh +++ b/dev/ci-velox-buildstatic-centos-7.sh @@ -2,8 +2,6 @@ set -e -# TODO: will be removed after docker is updated. -sudo yum install -y devtoolset-11 source /opt/rh/devtoolset-11/enable export NUM_THREADS=4 ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --build_arrow=OFF --build_tests=OFF --build_benchmarks=OFF \ diff --git a/dev/vcpkg/README.md b/dev/vcpkg/README.md index 229e11cc7204..986a8ec689eb 100644 --- a/dev/vcpkg/README.md +++ b/dev/vcpkg/README.md @@ -15,7 +15,7 @@ Please install build depends on your system to compile all libraries: sudo $GLUTEN_REPO/dev/vcpkg/setup-build-depends.sh ``` -For CentOS user, gcc 11 needs to be enabled manually before next step: +GCC-11 is the minimum required compiler. It needs to be enabled beforehand. Take Centos-7/8 as example: ``` sh # CentOS 8 diff --git a/docs/developers/NewToGluten.md b/docs/developers/NewToGluten.md index 8228c919e850..347213a41824 100644 --- a/docs/developers/NewToGluten.md +++ b/docs/developers/NewToGluten.md @@ -58,7 +58,7 @@ spark.executor.extraJavaOptions=-Dio.netty.tryReflectionSetAccessible=true [Maven Dowload Page](https://maven.apache.org/docs/history.html) And then set the environment setting. -## GCC 9.4 or above +## GCC 11 or above # Compile gluten using debug mode diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index a3220cbb3455..37dc20aedb36 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -69,17 +69,6 @@ function process_setup_ubuntu { # Do not install libunwind which can cause interruption when catching native exception. ensure_pattern_matched '\${SUDO} apt install -y libunwind-dev' scripts/setup-ubuntu.sh sed -i 's/${SUDO} apt install -y libunwind-dev//' scripts/setup-ubuntu.sh - # Overwrite gcc installed by build-essential. - ensure_pattern_matched '\${SUDO} pip3 install cmake==3.28.3' scripts/setup-ubuntu.sh - sed -i '/^ ${SUDO} pip3 install cmake==3.28.3/a\ - \VERSION=`cat /etc/os-release | grep VERSION_ID`\ - if [[ $VERSION =~ "20.04" ]]; then\ - sudo apt install -y software-properties-common\ - sudo add-apt-repository ppa:ubuntu-toolchain-r/test\ - sudo apt update && sudo apt install -y gcc-11 g++-11\ - sudo ln -sf /usr/bin/gcc-11 /usr/bin/gcc\ - sudo ln -sf /usr/bin/g++-11 /usr/bin/g++\ - fi' scripts/setup-ubuntu.sh ensure_pattern_matched 'ccache' scripts/setup-ubuntu.sh sed -i '/ccache/a\ *thrift* \\' scripts/setup-ubuntu.sh sed -i '/ccache/a\ libiberty-dev \\' scripts/setup-ubuntu.sh From 904f1d113e32797c53206245fd4e0b48d53baa6c Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Fri, 8 Nov 2024 17:22:28 +0800 Subject: [PATCH 055/211] [VL] Do not use --version-script link option on Darwin (#7820) --- cpp/velox/CMakeLists.txt | 4 ++-- cpp/velox/symbols.map | 2 +- cpp/velox/tests/CMakeLists.txt | 8 +++----- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index b16de0032a04..17defc498ec1 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -197,8 +197,8 @@ endif() add_library(velox SHARED ${VELOX_SRCS}) -if(ENABLE_GLUTEN_VCPKG) - # Hide symbols of static dependencies +if(ENABLE_GLUTEN_VCPKG AND NOT CMAKE_SYSTEM_NAME MATCHES "Darwin") + # Hide some symbols to avoid conflict. target_link_options( velox PRIVATE -Wl,--version-script=${CMAKE_CURRENT_SOURCE_DIR}/symbols.map) endif() diff --git a/cpp/velox/symbols.map b/cpp/velox/symbols.map index 525faf3526a1..427560c8f122 100644 --- a/cpp/velox/symbols.map +++ b/cpp/velox/symbols.map @@ -9,6 +9,6 @@ JNI_OnLoad; JNI_OnUnload; local: - # Hide symbols of static dependencies + # Hide all other symbols except the above global symbols. *; }; diff --git a/cpp/velox/tests/CMakeLists.txt b/cpp/velox/tests/CMakeLists.txt index c4cf6bfb0756..55cab342665b 100644 --- a/cpp/velox/tests/CMakeLists.txt +++ b/cpp/velox/tests/CMakeLists.txt @@ -26,11 +26,9 @@ function(add_velox_test TEST_EXEC) message(FATAL_ERROR "No sources specified for test ${TEST_NAME}") endif() add_executable(${TEST_EXEC} ${SOURCES} ${VELOX_TEST_COMMON_SRCS}) - target_include_directories( - ${TEST_EXEC} PRIVATE ${CMAKE_SOURCE_DIR}/velox ${CMAKE_SOURCE_DIR}/src - ${VELOX_BUILD_PATH}/_deps/duckdb-src/src/include) - target_link_libraries(${TEST_EXEC} velox GTest::gtest GTest::gtest_main - google::glog) + target_include_directories(${TEST_EXEC} PRIVATE ${CMAKE_SOURCE_DIR}/velox + ${CMAKE_SOURCE_DIR}/src) + target_link_libraries(${TEST_EXEC} velox GTest::gtest GTest::gtest_main) gtest_discover_tests(${TEST_EXEC} DISCOVERY_MODE PRE_TEST) endfunction() From c2a578c3379adafb79da9c06df3df4e41eb9139a Mon Sep 17 00:00:00 2001 From: lgbo Date: Fri, 8 Nov 2024 17:48:33 +0800 Subject: [PATCH 056/211] enable lazy expand for avg and sum(decimal) (#7840) --- .../extension/LazyAggregateExpandRule.scala | 15 +++++--- ...enClickHouseTPCHSaltNullParquetSuite.scala | 35 +++++++++++++++++++ 2 files changed, 46 insertions(+), 4 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala index e06503a5e1e6..86b28ab1f717 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -190,10 +190,12 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan // 2. if any aggregate function uses attributes which is not from expand's child, we don't // enable this if ( - !aggregate.aggregateExpressions.forall( + !aggregate.aggregateExpressions.forall { e => isValidAggregateFunction(e) && - e.aggregateFunction.references.forall(expandOutputAttributes.contains(_))) + e.aggregateFunction.references.forall( + attr => expandOutputAttributes.find(_.semanticEquals(attr)).isDefined) + } ) { logDebug(s"xxx Some aggregate functions are not supported") return false @@ -267,7 +269,8 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan case _: Count => true case _: Max => true case _: Min => true - case sum: Sum => !sum.dataType.isInstanceOf[DecimalType] + case _: Average => true + case _: Sum => true case _ => false } } @@ -275,7 +278,11 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan def getReplaceAttribute( toReplace: Attribute, attributesToReplace: Map[Attribute, Attribute]): Attribute = { - attributesToReplace.getOrElse(toReplace, toReplace) + val kv = attributesToReplace.find(kv => kv._1.semanticEquals(toReplace)) + kv match { + case Some((_, v)) => v + case None => toReplace + } } def buildReplaceAttributeMap(expand: ExpandExecTransformer): Map[Attribute, Attribute] = { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 12047b300c9c..9affdeb7f7fc 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -3068,6 +3068,41 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) } + test("GLUTEN-7647 lazy expand for avg and sum") { + val create_table_sql = + """ + |create table test_7647(x bigint, y bigint, z bigint, v decimal(10, 2)) using parquet + |""".stripMargin + spark.sql(create_table_sql) + val insert_data_sql = + """ + |insert into test_7647 values + |(1, 1, 1, 1.0), + |(2, 2, 2, 2.0), + |(3, 3, 3, 3.0), + |(2,2,1, 4.0) + |""".stripMargin + spark.sql(insert_data_sql) + + def checkLazyExpand(df: DataFrame): Unit = { + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: ExpandExecTransformer if (e.child.isInstanceOf[HashAggregateExecBaseTransformer]) => + e + } + assert(expands.size == 1) + } + + var sql = "select x, y, avg(z), sum(v) from test_7647 group by x, y with cube order by x, y" + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + sql = + "select x, y, count(distinct z), avg(v) from test_7647 group by x, y with cube order by x, y" + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + sql = + "select x, y, count(distinct z), sum(v) from test_7647 group by x, y with cube order by x, y" + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + spark.sql("drop table if exists test_7647") + } + test("GLUTEN-7759: Fix bug of agg pre-project push down") { val table_create_sql = "create table test_tbl_7759(id bigint, name string, day string) using parquet" From 43bf2fc3559808512310f0badcd4bad224346b40 Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Fri, 8 Nov 2024 17:58:13 +0800 Subject: [PATCH 057/211] Exclude metastore_db from git (#7871) --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index d30ea813ac04..4ea83cbf7e12 100644 --- a/.gitignore +++ b/.gitignore @@ -95,3 +95,6 @@ dist/ /cpp-ch/local-engine/Parser/*_udf !/cpp-ch/local-engine/Parser/example_udf + +# For Hive +metastore_db/ From 5b496e86b6e1fbca55866616fa1519a532b07b51 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Fri, 8 Nov 2024 18:44:50 +0800 Subject: [PATCH 058/211] [GLUTEN-7760] Fix udf implicit cast & update doc (#7852) --- .../sql/hive/VeloxHiveUDFTransformer.scala | 8 +- .../gluten/expression/VeloxUdfSuite.scala | 27 +++-- docs/developers/VeloxUDF.md | 19 +-- .../WholeStageTransformerSuite.scala | 108 +---------------- .../apache/spark/sql/GlutenQueryTest.scala | 109 +++++++++++++++++- 5 files changed, 142 insertions(+), 129 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala index d895faa31702..b3524e20f019 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala @@ -37,11 +37,11 @@ object VeloxHiveUDFTransformer { } if (UDFResolver.UDFNames.contains(udfClassName)) { - UDFResolver + val udfExpression = UDFResolver .getUdfExpression(udfClassName, udfName)(expr.children) - .getTransformer( - ExpressionConverter.replaceWithExpressionTransformer(expr.children, attributeSeq) - ) + udfExpression.getTransformer( + ExpressionConverter.replaceWithExpressionTransformer(udfExpression.children, attributeSeq) + ) } else { HiveUDFTransformer.genTransformerFromUDFMappings(udfName, expr, attributeSeq) } diff --git a/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala index f85103deb847..61ba927cd457 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala @@ -16,11 +16,13 @@ */ package org.apache.gluten.expression +import org.apache.gluten.execution.ProjectExecTransformer import org.apache.gluten.tags.{SkipTestTags, UDFTest} import org.apache.spark.SparkConf import org.apache.spark.sql.{GlutenQueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.expression.UDFResolver import java.nio.file.Paths @@ -158,16 +160,24 @@ abstract class VeloxUdfSuite extends GlutenQueryTest with SQLHelper { |AS 'org.apache.spark.sql.hive.execution.UDFStringString' |""".stripMargin) - val nativeResultWithImplicitConversion = - spark.sql(s"""SELECT hive_string_string(col1, 'a') FROM $tbl""").collect() - val nativeResult = - spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""").collect() + val offloadWithImplicitConversionDF = + spark.sql(s"""SELECT hive_string_string(col1, 'a') FROM $tbl""") + checkGlutenOperatorMatch[ProjectExecTransformer](offloadWithImplicitConversionDF) + val offloadWithImplicitConversionResult = offloadWithImplicitConversionDF.collect() + + val offloadDF = + spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""") + checkGlutenOperatorMatch[ProjectExecTransformer](offloadDF) + val offloadResult = offloadWithImplicitConversionDF.collect() + // Unregister native hive udf to fallback. UDFResolver.UDFNames.remove("org.apache.spark.sql.hive.execution.UDFStringString") - val fallbackResult = - spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""").collect() - assert(nativeResultWithImplicitConversion.sameElements(fallbackResult)) - assert(nativeResult.sameElements(fallbackResult)) + val fallbackDF = + spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""") + checkSparkOperatorMatch[ProjectExec](fallbackDF) + val fallbackResult = fallbackDF.collect() + assert(offloadWithImplicitConversionResult.sameElements(fallbackResult)) + assert(offloadResult.sameElements(fallbackResult)) // Add an unimplemented udf to the map to test fallback of registered native hive udf. UDFResolver.UDFNames.add("org.apache.spark.sql.hive.execution.UDFIntegerToString") @@ -176,6 +186,7 @@ abstract class VeloxUdfSuite extends GlutenQueryTest with SQLHelper { |AS 'org.apache.spark.sql.hive.execution.UDFIntegerToString' |""".stripMargin) val df = spark.sql(s"""select hive_int_to_string(col1) from $tbl""") + checkSparkOperatorMatch[ProjectExec](df) checkAnswer(df, Seq(Row("1"), Row("2"), Row("3"))) } finally { spark.sql(s"DROP TABLE IF EXISTS $tbl") diff --git a/docs/developers/VeloxUDF.md b/docs/developers/VeloxUDF.md index 4f685cc41ed7..4cbdcfa99292 100644 --- a/docs/developers/VeloxUDF.md +++ b/docs/developers/VeloxUDF.md @@ -172,22 +172,23 @@ or Start `spark-sql` and run query. You need to add jar "spark-hive_2.12--tests.jar" to the classpath for hive udf `org.apache.spark.sql.hive.execution.UDFStringString` ``` +spark-sql (default)> create table tbl as select * from values ('hello'); +Time taken: 3.656 seconds spark-sql (default)> CREATE TEMPORARY FUNCTION hive_string_string AS 'org.apache.spark.sql.hive.execution.UDFStringString'; -Time taken: 0.808 seconds -spark-sql (default)> select hive_string_string("hello", "world"); +Time taken: 0.047 seconds +spark-sql (default)> select hive_string_string(col1, 'world') from tbl; hello world -Time taken: 3.208 seconds, Fetched 1 row(s) +Time taken: 1.217 seconds, Fetched 1 row(s) ``` You can verify the offload with "explain". ``` -spark-sql (default)> explain select hive_string_string("hello", "world"); -== Physical Plan == -VeloxColumnarToRowExec -+- ^(2) ProjectExecTransformer [hello world AS hive_string_string(hello, world)#8] - +- ^(2) InputIteratorTransformer[fake_column#9] +spark-sql (default)> explain select hive_string_string(col1, 'world') from tbl; +VeloxColumnarToRow ++- ^(2) ProjectExecTransformer [HiveSimpleUDF#org.apache.spark.sql.hive.execution.UDFStringString(col1#11,world) AS hive_string_string(col1, world)#12] + +- ^(2) InputIteratorTransformer[col1#11] +- RowToVeloxColumnar - +- *(1) Scan OneRowRelation[fake_column#9] + +- Scan hive spark_catalog.default.tbl [col1#11], HiveTableRelation [`spark_catalog`.`default`.`tbl`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [col1#11], Partition Cols: []] ``` ## Configurations diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala index 146d6fde58ec..fd250834d078 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala @@ -17,15 +17,13 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.test.FallbackUtil import org.apache.gluten.utils.Arm import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, GlutenQueryTest, Row} -import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.DoubleType @@ -33,7 +31,6 @@ import java.io.File import java.util.concurrent.atomic.AtomicBoolean import scala.io.Source -import scala.reflect.ClassTag case class Table(name: String, partitionColumns: Seq[String]) @@ -179,109 +176,6 @@ abstract class WholeStageTransformerSuite result } - def checkLengthAndPlan(df: DataFrame, len: Int = 100): Unit = { - assert(df.collect().length == len) - val executedPlan = getExecutedPlan(df) - assert(executedPlan.exists(plan => plan.find(_.isInstanceOf[TransformSupport]).isDefined)) - } - - /** - * Get all the children plan of plans. - * @param plans: - * the input plans. - * @return - */ - def getChildrenPlan(plans: Seq[SparkPlan]): Seq[SparkPlan] = { - if (plans.isEmpty) { - return Seq() - } - - val inputPlans: Seq[SparkPlan] = plans.map { - case stage: ShuffleQueryStageExec => stage.plan - case plan => plan - } - - var newChildren: Seq[SparkPlan] = Seq() - inputPlans.foreach { - plan => - newChildren = newChildren ++ getChildrenPlan(plan.children) - // To avoid duplication of WholeStageCodegenXXX and its children. - if (!plan.nodeName.startsWith("WholeStageCodegen")) { - newChildren = newChildren :+ plan - } - } - newChildren - } - - /** - * Get the executed plan of a data frame. - * @param df: - * dataframe. - * @return - * A sequence of executed plans. - */ - def getExecutedPlan(df: DataFrame): Seq[SparkPlan] = { - df.queryExecution.executedPlan match { - case exec: AdaptiveSparkPlanExec => - getChildrenPlan(Seq(exec.executedPlan)) - case cmd: CommandResultExec => - getChildrenPlan(Seq(cmd.commandPhysicalPlan)) - case plan => - getChildrenPlan(Seq(plan)) - } - } - - /** - * Check whether the executed plan of a dataframe contains the expected plan. - * @param df: - * the input dataframe. - * @param tag: - * class of the expected plan. - * @tparam T: - * type of the expected plan. - */ - def checkGlutenOperatorMatch[T <: GlutenPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { - val executedPlan = getExecutedPlan(df) - assert( - executedPlan.exists(plan => tag.runtimeClass.isInstance(plan)), - s"Expect ${tag.runtimeClass.getSimpleName} exists " + - s"in executedPlan:\n ${executedPlan.last}" - ) - } - - def checkSparkOperatorMatch[T <: SparkPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { - val executedPlan = getExecutedPlan(df) - assert(executedPlan.exists(plan => tag.runtimeClass.isInstance(plan))) - } - - /** - * Check whether the executed plan of a dataframe contains the expected plan chain. - * - * @param df - * : the input dataframe. - * @param tag - * : class of the expected plan. - * @param childTag - * : class of the expected plan's child. - * @tparam T - * : type of the expected plan. - * @tparam PT - * : type of the expected plan's child. - */ - def checkSparkOperatorChainMatch[T <: UnaryExecNode, PT <: UnaryExecNode]( - df: DataFrame)(implicit tag: ClassTag[T], childTag: ClassTag[PT]): Unit = { - val executedPlan = getExecutedPlan(df) - assert( - executedPlan.exists( - plan => - tag.runtimeClass.isInstance(plan) - && childTag.runtimeClass.isInstance(plan.children.head)), - s"Expect an operator chain of [${tag.runtimeClass.getSimpleName} ->" - + s"${childTag.runtimeClass.getSimpleName}] exists in executedPlan: \n" - + s"${executedPlan.last}" - ) - } - /** * run a query with native engine as well as vanilla spark then compare the result set for * correctness check diff --git a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala index 53abaa9ac297..164083a8d8b0 100644 --- a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala +++ b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala @@ -21,6 +21,8 @@ package org.apache.spark.sql * 1. We need to modify the way org.apache.spark.sql.CHQueryTest#compare compares double */ import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.TransformSupport +import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.SPARK_VERSION_SHORT @@ -28,7 +30,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan, SQLExecution, UnaryExecNode} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.storage.StorageLevel @@ -38,6 +41,7 @@ import org.scalatest.Assertions import java.util.TimeZone import scala.collection.JavaConverters._ +import scala.reflect.ClassTag import scala.reflect.runtime.universe abstract class GlutenQueryTest extends PlanTest { @@ -306,6 +310,109 @@ abstract class GlutenQueryTest extends PlanTest { query.queryExecution.executedPlan.missingInput.isEmpty, s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}") } + + def checkLengthAndPlan(df: DataFrame, len: Int = 100): Unit = { + assert(df.collect().length == len) + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => plan.find(_.isInstanceOf[TransformSupport]).isDefined)) + } + + /** + * Get all the children plan of plans. + * @param plans: + * the input plans. + * @return + */ + def getChildrenPlan(plans: Seq[SparkPlan]): Seq[SparkPlan] = { + if (plans.isEmpty) { + return Seq() + } + + val inputPlans: Seq[SparkPlan] = plans.map { + case stage: ShuffleQueryStageExec => stage.plan + case plan => plan + } + + var newChildren: Seq[SparkPlan] = Seq() + inputPlans.foreach { + plan => + newChildren = newChildren ++ getChildrenPlan(plan.children) + // To avoid duplication of WholeStageCodegenXXX and its children. + if (!plan.nodeName.startsWith("WholeStageCodegen")) { + newChildren = newChildren :+ plan + } + } + newChildren + } + + /** + * Get the executed plan of a data frame. + * @param df: + * dataframe. + * @return + * A sequence of executed plans. + */ + def getExecutedPlan(df: DataFrame): Seq[SparkPlan] = { + df.queryExecution.executedPlan match { + case exec: AdaptiveSparkPlanExec => + getChildrenPlan(Seq(exec.executedPlan)) + case cmd: CommandResultExec => + getChildrenPlan(Seq(cmd.commandPhysicalPlan)) + case plan => + getChildrenPlan(Seq(plan)) + } + } + + /** + * Check whether the executed plan of a dataframe contains the expected plan chain. + * + * @param df + * : the input dataframe. + * @param tag + * : class of the expected plan. + * @param childTag + * : class of the expected plan's child. + * @tparam T + * : type of the expected plan. + * @tparam PT + * : type of the expected plan's child. + */ + def checkSparkOperatorChainMatch[T <: UnaryExecNode, PT <: UnaryExecNode]( + df: DataFrame)(implicit tag: ClassTag[T], childTag: ClassTag[PT]): Unit = { + val executedPlan = getExecutedPlan(df) + assert( + executedPlan.exists( + plan => + tag.runtimeClass.isInstance(plan) + && childTag.runtimeClass.isInstance(plan.children.head)), + s"Expect an operator chain of [${tag.runtimeClass.getSimpleName} ->" + + s"${childTag.runtimeClass.getSimpleName}] exists in executedPlan: \n" + + s"${executedPlan.last}" + ) + } + + /** + * Check whether the executed plan of a dataframe contains the expected plan. + * @param df: + * the input dataframe. + * @param tag: + * class of the expected plan. + * @tparam T: + * type of the expected plan. + */ + def checkGlutenOperatorMatch[T <: GlutenPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert( + executedPlan.exists(plan => tag.runtimeClass.isInstance(plan)), + s"Expect ${tag.runtimeClass.getSimpleName} exists " + + s"in executedPlan:\n ${executedPlan.last}" + ) + } + + def checkSparkOperatorMatch[T <: SparkPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => tag.runtimeClass.isInstance(plan))) + } } object GlutenQueryTest extends Assertions { From 76b375d9983d096111a438cadf71017124297194 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 8 Nov 2024 20:36:36 +0800 Subject: [PATCH 059/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_08) (#7854) Upstream Velox's New Commits: d76c05c7d by Deepak Majeti, Add support for S3 bucket config (11321) 327fa89fd by Christian Zentgraf, fix: Installation of HDFS on Ubuntu (11420) 8c69b8b76 by Chengcheng Jin, Support prefix sort in spilling (11384) 64661a31d by Jaime Pan, Return null for invalid datetime format when legacy date formatter is used (11131) d1bf9da3d by Bikramjeet Vig, Add support for using negative array indices in json path (11451) 789ce652f by Ke, Rename tableStorageFormat in HiveInsertTableHandle (11458) aba7ba815 by Kevin Wilfong, Fix inconsistencies between Velox and Presto Java when formatting time zone short/long names --- cpp/velox/utils/ConfigExtractor.cc | 39 ++++++++++++++++++++---------- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 27 insertions(+), 14 deletions(-) diff --git a/cpp/velox/utils/ConfigExtractor.cc b/cpp/velox/utils/ConfigExtractor.cc index a83045b03e5e..3175c375fafc 100644 --- a/cpp/velox/utils/ConfigExtractor.cc +++ b/cpp/velox/utils/ConfigExtractor.cc @@ -22,6 +22,7 @@ #include "utils/Exception.h" #include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Config.h" namespace { @@ -98,27 +99,39 @@ std::shared_ptr getHiveConfig( } if (useInstanceCredentials) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3UseInstanceCredentials] = "true"; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kUseInstanceCredentials)] = "true"; } else if (!iamRole.empty()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3IamRole] = iamRole; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kIamRole)] = iamRole; if (!iamRoleSessionName.empty()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3IamRoleSessionName] = iamRoleSessionName; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kIamRoleSessionName)] = iamRoleSessionName; } } else { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3AwsAccessKey] = awsAccessKey; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3AwsSecretKey] = awsSecretKey; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kAccessKey)] = awsAccessKey; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kSecretKey)] = awsSecretKey; } // Only need to set s3 endpoint when not use instance credentials. if (!useInstanceCredentials) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3Endpoint] = awsEndpoint; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kEndpoint)] = awsEndpoint; } - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3SSLEnabled] = sslEnabled ? "true" : "false"; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3PathStyleAccess] = pathStyleAccess ? "true" : "false"; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3LogLevel] = awsSdkLogLevel; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3MaxAttempts] = retryMaxAttempts; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3RetryMode] = retryMode; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3MaxConnections] = maxConnections; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3ConnectTimeout] = connectTimeout; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kSSLEnabled)] = sslEnabled ? "true" : "false"; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kPathStyleAccess)] = pathStyleAccess ? "true" : "false"; + hiveConfMap[facebook::velox::filesystems::S3Config::kS3LogLevel] = awsSdkLogLevel; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kMaxAttempts)] = retryMaxAttempts; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kRetryMode)] = retryMode; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kMaxConnections)] = maxConnections; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kConnectTimeout)] = connectTimeout; #endif #ifdef ENABLE_GCS diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 37dc20aedb36..60c7ea0a48bd 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_07 +VELOX_BRANCH=2024_11_08 VELOX_HOME="" OS=`uname -s` From 2b62a40d18bde64ff4bba53d3fc4cb184d1df601 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sun, 10 Nov 2024 09:13:42 +0800 Subject: [PATCH 060/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_09) (#7875) Upstream Velox's New Commits: 31aa90c1e by Amit Dutta, Back out "Use fallocate for file size extension when supported" (11480) c5232cd31 by Krishna Pai, Back out "Add runner for local distributed execution" (11475) e0939eac7 by Yenda Li, Add ipprefix cast operators for varchar [1/n] (11460) f2e137a58 by rui-mo, Add ExprToSubfieldFilterParser and allow custom parser registration (11452) 396b2bbb6 by mohsaka, Change IPPREFIX type to row type in typeof (11449) 1d4e89d29 by Orri Erling, Add runner for local distributed execution (11261) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 60c7ea0a48bd..1d2718069907 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_08 +VELOX_BRANCH=2024_11_09 VELOX_HOME="" OS=`uname -s` From be0435a86b74615117afbac81e5e0e50dc8d398e Mon Sep 17 00:00:00 2001 From: Chang chen Date: Sun, 10 Nov 2024 12:39:48 +0800 Subject: [PATCH 061/211] [GLUTEN-7028][CH][Part-7] Feature/onepipeline2 (#7788) * [Refactor] Remvoe object ClickhouseOptimisticTransaction * [Refactor] Fix typo * [Improve] we now parse partition at java side * [Improve] Make BasicWriteTaskStats correct * [Bug Fix] Fix header mismatch issue: We need to get header from pipeline builder instead of query plan, since query plan only contains read rel output header. * [Refactor] Pass write proto for one pipeline write * [Improve] Introduce RuntimeConfig and RuntiemSettings [Improve] Using callback to update config without copying config [Refactor] NativeExpressionEvaluator and refactor Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniWrapper_createMergeTreeWriter [Refactor] Simplify the logic of evaluating tmp_path [Refactor] Remove Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_injectWriteFilesTempPath * [New Feature] one pipeline --- .../ClickhouseOptimisticTransaction.scala | 1 - .../ClickhouseOptimisticTransaction.scala | 1 - .../ClickhouseOptimisticTransaction.scala | 34 ++-- .../MergeTreeDelayedCommitProtocol.scala | 48 +++++ ...ite.scala => FileDeltaColumnarWrite.scala} | 85 +++----- .../MergeTreeDeltaColumnarWrite.scala | 189 ++++++++++++++++++ .../CHNativeExpressionEvaluator.java | 6 - .../ExpressionEvaluatorJniWrapper.java | 25 +-- .../vectorized/SimpleExpressionEval.java | 2 +- .../datasources/CHDatasourceJniWrapper.java | 9 +- .../clickhouse/WriteReturnedMetric.java | 8 +- .../datasources/v1/write_optimization.proto | 2 + .../backendsapi/clickhouse/CHConf.scala | 17 ++ .../clickhouse/CHIteratorApi.scala | 14 +- .../clickhouse/CHTransformerApi.scala | 35 ++-- .../clickhouse/RuntimeConfig.scala | 45 +++++ .../clickhouse/RuntimeSettings.scala | 61 ++++++ .../NativeExpressionEvaluator.scala | 39 ++++ .../spark/sql/delta/MergeTreeFileFormat.scala | 28 ++- .../files/MergeTreeFileCommitProtocol.scala | 38 ++-- .../delta/util/MergeTreePartitionUtils.scala | 58 ++++++ .../sql/execution/CHColumnarToRowExec.scala | 6 +- .../spark/sql/execution/CHColumnarWrite.scala | 110 ++++++---- .../execution/CHColumnarWriteFilesExec.scala | 20 +- .../v1/CHFormatWriterInjects.scala | 2 +- .../v1/CHMergeTreeWriterInjects.scala | 45 ++--- .../datasources/v1/FakeRowOutputWriter.scala | 6 +- .../clickhouse/MergeTreeCommiterHelper.scala | 4 + .../v2/clickhouse/metadata/AddFileTags.scala | 83 ++++---- .../sql/execution/utils/CHExecUtil.scala | 14 +- ...ClickHouseWholeStageTransformerSuite.scala | 5 +- .../GlutenClickHouseMergeTreeWriteSuite.scala | 170 ++++++++-------- cpp-ch/local-engine/Common/BlockIterator.cpp | 2 +- cpp-ch/local-engine/Common/BlockIterator.h | 2 +- cpp-ch/local-engine/Common/CHUtil.cpp | 22 +- cpp-ch/local-engine/Common/CHUtil.h | 9 +- cpp-ch/local-engine/Common/GlutenConfig.cpp | 30 ++- cpp-ch/local-engine/Common/GlutenConfig.h | 10 +- cpp-ch/local-engine/Common/GlutenSettings.cpp | 10 +- cpp-ch/local-engine/Common/GlutenSettings.h | 18 +- cpp-ch/local-engine/Parser/LocalExecutor.cpp | 8 +- cpp-ch/local-engine/Parser/LocalExecutor.h | 6 +- .../Parser/RelParsers/WriteRelParser.cpp | 134 ++++++++----- .../Parser/RelParsers/WriteRelParser.h | 8 +- .../Parser/SerializedPlanParser.cpp | 2 +- .../Storages/MergeTree/MetaDataHelper.cpp | 10 +- .../Storages/MergeTree/MetaDataHelper.h | 3 - .../Storages/MergeTree/SparkMergeTreeMeta.cpp | 38 ++-- .../Storages/MergeTree/SparkMergeTreeMeta.h | 3 +- .../Storages/MergeTree/SparkMergeTreeSink.cpp | 2 - .../Storages/MergeTree/SparkMergeTreeSink.h | 83 ++++++++ .../MergeTree/SparkMergeTreeWriteSettings.h | 8 +- .../MergeTree/SparkMergeTreeWriter.cpp | 46 ++--- .../Storages/MergeTree/SparkMergeTreeWriter.h | 20 +- .../MergeTree/SparkStorageMergeTree.cpp | 8 +- .../Storages/Output/NormalFileWriter.h | 26 ++- cpp-ch/local-engine/local_engine_jni.cpp | 69 ++----- .../local-engine/tests/gluten_test_util.cpp | 14 +- cpp-ch/local-engine/tests/gluten_test_util.h | 11 + .../local-engine/tests/gtest_local_engine.cpp | 6 +- .../tests/gtest_write_pipeline.cpp | 29 +-- .../tests/gtest_write_pipeline_mergetree.cpp | 31 ++- .../tests/json/mergetree/1_mergetree.json | 2 +- .../json/mergetree/1_mergetree_hdfs.json | 2 +- ..._one_pipeline.json => 3_one_pipeline.json} | 13 +- .../json/mergetree/3_one_pipeline_input.json | 28 +++ .../json/native_write_one_partition.json | 7 +- .../tests/json/native_write_plan.json | 7 +- 68 files changed, 1270 insertions(+), 667 deletions(-) rename backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/{CHDeltaColumnarWrite.scala => FileDeltaColumnarWrite.scala} (62%) create mode 100644 backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala create mode 100644 backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala rename cpp-ch/local-engine/tests/json/mergetree/{2_one_pipeline.json => 3_one_pipeline.json} (96%) create mode 100644 cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 1ea6fe249e81..06b6fa8c4c20 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -36,7 +36,6 @@ import org.apache.commons.lang3.exception.ExceptionUtils import scala.collection.mutable.ListBuffer -object ClickhouseOptimisticTransaction {} class ClickhouseOptimisticTransaction( override val deltaLog: DeltaLog, override val snapshot: Snapshot)(implicit override val clock: Clock) diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 2a97ef951506..11acfb026054 100644 --- a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -36,7 +36,6 @@ import org.apache.commons.lang3.exception.ExceptionUtils import scala.collection.mutable.ListBuffer -object ClickhouseOptimisticTransaction {} class ClickhouseOptimisticTransaction( override val deltaLog: DeltaLog, override val snapshot: Snapshot)(implicit override val clock: Clock) diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 9097a02b9337..6d62e70d2ebb 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.delta +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.spark.SparkException @@ -24,12 +25,12 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 import org.apache.spark.sql.delta.constraints.{Constraint, Constraints} -import org.apache.spark.sql.delta.files.{DelayedCommitProtocol, DeltaFileFormatWriter, MergeTreeDelayedCommitProtocol, TransactionalWrite} +import org.apache.spark.sql.delta.files._ import org.apache.spark.sql.delta.hooks.AutoCompact import org.apache.spark.sql.delta.schema.{InnerInvariantViolationException, InvariantViolationException} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker -import org.apache.spark.sql.execution.{CHDelayedCommitProtocol, QueryExecution, SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormatWriter, GlutenWriterColumnarRules, WriteFiles, WriteJobStatsTracker} import org.apache.spark.sql.execution.datasources.v1.MergeTreeWriterInjects @@ -43,8 +44,6 @@ import org.apache.hadoop.fs.Path import scala.collection.mutable.ListBuffer -object ClickhouseOptimisticTransaction {} - class ClickhouseOptimisticTransaction( override val deltaLog: DeltaLog, override val catalogTable: Option[CatalogTable], @@ -66,7 +65,12 @@ class ClickhouseOptimisticTransaction( inputData: Dataset[_], writeOptions: Option[DeltaOptions], additionalConstraints: Seq[Constraint]): Seq[FileAction] = { - if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { + + // TODO: update FallbackByBackendSettings for mergetree always return true + val onePipeline = GlutenConfig.getConf.enableNativeWriter.getOrElse( + false) && CHConf.get.enableOnePipelineMergeTreeWrite + + if (!onePipeline && ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { hasWritten = true val spark = inputData.sparkSession @@ -169,12 +173,6 @@ class ClickhouseOptimisticTransaction( } committer.addedStatuses.toSeq ++ committer.changeFiles } else { - // TODO: support native delta parquet write - // 1. insert FakeRowAdaptor - // 2. DeltaInvariantCheckerExec transform - // 3. DeltaTaskStatisticsTracker collect null count / min values / max values - // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeApplicable', - // 'nativeFormat' in the LocalProperty of the sparkcontext super.writeFiles(inputData, writeOptions, additionalConstraints) } } @@ -188,7 +186,7 @@ class ClickhouseOptimisticTransaction( } override protected def getCommitter(outputPath: Path): DelayedCommitProtocol = - new CHDelayedCommitProtocol("delta", outputPath.toString, None, deltaDataSubdir) + new FileDelayedCommitProtocol("delta", outputPath.toString, None, deltaDataSubdir) override def writeFiles( inputData: Dataset[_], @@ -231,7 +229,17 @@ class ClickhouseOptimisticTransaction( WriteFiles(logicalPlan, fileFormat, partitioningColumns, None, options, Map.empty) val queryExecution = new QueryExecution(spark, write) - val committer = getCommitter(outputPath) + val committer = fileFormat.toString match { + case "MergeTree" => + val tableV2 = ClickHouseTableV2.getTable(deltaLog) + new MergeTreeDelayedCommitProtocol2( + outputPath.toString, + None, + deltaDataSubdir, + tableV2.dataBaseName, + tableV2.tableName) + case _ => getCommitter(outputPath) + } // If Statistics Collection is enabled, then create a stats tracker that will be injected during // the FileFormatWriter.write call below and will collect per-file stats using diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala index f2c22234a692..e90df0f02634 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.delta.files +import org.apache.hadoop.mapreduce.TaskAttemptContext + class MergeTreeDelayedCommitProtocol( val outputPath: String, randomPrefixLength: Option[Int], @@ -24,3 +26,49 @@ class MergeTreeDelayedCommitProtocol( val tableName: String) extends DelayedCommitProtocol("delta-mergetree", outputPath, randomPrefixLength, subdir) with MergeTreeFileCommitProtocol {} + +/** + * A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields in a pipeline + * write for parquet. + */ +class FileDelayedCommitProtocol( + jobId: String, + val outputPath: String, + randomPrefixLength: Option[Int], + subdir: Option[String]) + extends DelayedCommitProtocol(jobId, outputPath, randomPrefixLength, subdir) { + + override def getFileName( + taskContext: TaskAttemptContext, + ext: String, + partitionValues: Map[String, String]): String = { + super.getFileName(taskContext, ext, partitionValues) + } + + def updateAddedFiles(files: Seq[(Map[String, String], String)]): Unit = { + assert(addedFiles.isEmpty) + addedFiles ++= files + } + + override def parsePartitions(dir: String): Map[String, String] = + super.parsePartitions(dir) +} + +/** + * A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields in a pipeline + * write for mergetree. + */ +class MergeTreeDelayedCommitProtocol2( + val outputPath: String, + randomPrefixLength: Option[Int], + subdir: Option[String], + val database: String, + val tableName: String) + extends DelayedCommitProtocol("delta-mergetree", outputPath, randomPrefixLength, subdir) { + + override def newTaskTempFile( + taskContext: TaskAttemptContext, + dir: Option[String], + ext: String): String = outputPath + +} diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala similarity index 62% rename from backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala rename to backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala index 66f502038fcd..709f5682246d 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala @@ -21,45 +21,19 @@ import org.apache.gluten.exception.GlutenNotSupportException import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.delta.files.DelayedCommitProtocol +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.delta.files.{FileDelayedCommitProtocol, MergeTreeDelayedCommitProtocol2} import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils -import org.apache.hadoop.mapreduce.TaskAttemptContext - -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -/** A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields. */ -class CHDelayedCommitProtocol( - jobId: String, - val outputPath: String, - randomPrefixLength: Option[Int], - subdir: Option[String]) - extends DelayedCommitProtocol(jobId, outputPath, randomPrefixLength, subdir) { - - override def getFileName( - taskContext: TaskAttemptContext, - ext: String, - partitionValues: Map[String, String]): String = { - super.getFileName(taskContext, ext, partitionValues) - } - - def updateAddedFiles(files: Seq[(Map[String, String], String)]): Unit = { - assert(addedFiles.isEmpty) - addedFiles ++= files - } - - override def parsePartitions(dir: String): Map[String, String] = - super.parsePartitions(dir) -} - -case class CHDelayedCommitProtocolWrite( +case class FileDeltaColumnarWrite( override val jobTrackerID: String, override val description: WriteJobDescription, - override val committer: CHDelayedCommitProtocol) - extends CHColumnarWrite[CHDelayedCommitProtocol] + override val committer: FileDelayedCommitProtocol) + extends CHColumnarWrite[FileDelayedCommitProtocol] with Logging { override def doSetupNativeTask(): Unit = { @@ -73,39 +47,41 @@ case class CHDelayedCommitProtocolWrite( private def doCollectNativeResult( cb: ColumnarBatch): Option[(Seq[(Map[String, String], String)], ExecutedWriteSummary)] = { - val numFiles = cb.numRows() + val basicNativeStats = BasicNativeStats(cb) + + // TODO: we need close iterator here before processing the result. + // Write an empty iterator - if (numFiles == 0) { + if (basicNativeStats.isEmpty) { None } else { - val file_col = cb.column(0) - val partition_col = cb.column(1) - val count_col = cb.column(2) - - val partitions: mutable.Set[String] = mutable.Set[String]() + // process stats val addedFiles: ArrayBuffer[(Map[String, String], String)] = new ArrayBuffer[(Map[String, String], String)] - var numWrittenRows: Long = 0 - Range(0, cb.numRows()).foreach { - i => - val fileName = file_col.getUTF8String(i).toString - val partition = partition_col.getUTF8String(i).toString - if (partition == "__NO_PARTITION_ID__") { - addedFiles.append((Map.empty[String, String], fileName)) + + basicNativeStats.foreach { + stat => + val absolutePath = s"${description.path}/${stat.relativePath}" + if (stat.partition_id == "__NO_PARTITION_ID__") { + addedFiles.append((Map.empty[String, String], stat.filename)) } else { - val partitionValues = committer.parsePartitions(partition) - addedFiles.append((partitionValues, s"$partition/$fileName")) + val partitionValues = committer.parsePartitions(stat.partition_id) + addedFiles.append((partitionValues, stat.relativePath)) + basicWriteJobStatsTracker.newPartition( + new GenericInternalRow(Array[Any](stat.partition_id))) } - numWrittenRows += count_col.getLong(i) + basicWriteJobStatsTracker.newFile(absolutePath) + basicWriteJobStatsTracker.closeFile(absolutePath) + numWrittenRows += stat.record_count } - val updatedPartitions = partitions.toSet + Some( ( addedFiles.toSeq, ExecutedWriteSummary( - updatedPartitions = updatedPartitions, - stats = Seq(CreateBasicWriteTaskStats(numFiles, updatedPartitions, numWrittenRows))))) + updatedPartitions = Set.empty, + stats = Seq(finalStats.copy(numRows = numWrittenRows))))) } } @@ -134,8 +110,11 @@ object CHDeltaColumnarWrite { jobTrackerID: String, description: WriteJobDescription, committer: FileCommitProtocol): CHColumnarWrite[FileCommitProtocol] = committer match { - case c: CHDelayedCommitProtocol => - CHDelayedCommitProtocolWrite(jobTrackerID, description, c) + case c: FileDelayedCommitProtocol => + FileDeltaColumnarWrite(jobTrackerID, description, c) + .asInstanceOf[CHColumnarWrite[FileCommitProtocol]] + case m: MergeTreeDelayedCommitProtocol2 => + MergeTreeDeltaColumnarWrite(jobTrackerID, description, m) .asInstanceOf[CHColumnarWrite[FileCommitProtocol]] case _ => throw new GlutenNotSupportException( diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala new file mode 100644 index 000000000000..fdc6cdfd3077 --- /dev/null +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings +import org.apache.gluten.vectorized.NativeExpressionEvaluator + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.delta.actions.{AddFile, FileAction} +import org.apache.spark.sql.delta.files.MergeTreeDelayedCommitProtocol2 +import org.apache.spark.sql.delta.stats.DeltaStatistics +import org.apache.spark.sql.delta.util.{JsonUtils, MergeTreePartitionUtils} +import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} +import org.apache.spark.sql.execution.datasources.mergetree.StorageMeta +import org.apache.spark.sql.execution.utils.CHExecUtil +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.Utils + +import org.apache.hadoop.fs.Path + +import java.util.UUID + +/** + * {{{ + * val schema = + * StructType( + * StructField("part_name", StringType, false) :: + * StructField("partition_id", StringType, false) :: + * StructField("record_count", LongType, false) :: + * StructField("marks_count", LongType, false) :: + * StructField("size_in_bytes", LongType, false) :: Nil) + * }}} + */ +case class BasicMergeTreeNativeStat( + part_name: String, + partition_id: String, + record_count: Long, + marks_count: Long, + size_in_bytes: Long) { + def apply( + database: String, + table: String, + path: Path, + modificationTime: Long, + hostName: Seq[String]): FileAction = { + val partitionValues = if (partition_id == "__NO_PARTITION_ID__") { + Map.empty[String, String] + } else { + MergeTreePartitionUtils.parsePartitions(partition_id) + } + val tags = Map[String, String]( + "database" -> database, + "table" -> table, + "engine" -> "MergeTree", + "path" -> path.toUri.getPath, + "targetNode" -> hostName.map(_.trim).mkString(","), + "partition" -> "", + "uuid" -> "", + "rows" -> record_count.toString, + "bytesOnDisk" -> size_in_bytes.toString, + "dataCompressedBytes" -> -1L.toString, + "dataUncompressedBytes" -> -1L.toString, + "modificationTime" -> modificationTime.toString, + "partitionId" -> "", + "minBlockNumber" -> -1L.toString, + "maxBlockNumber" -> -1L.toString, + "level" -> -1L.toString, + "dataVersion" -> -1L.toString, + "defaultCompressionCodec" -> "lz4", + "bucketNum" -> "", + "dirName" -> path.toString, + "marks" -> marks_count.toString + ) + + val stats = Map[String, Any]( + DeltaStatistics.NUM_RECORDS -> record_count, + DeltaStatistics.MIN -> "", + DeltaStatistics.MAX -> "", + DeltaStatistics.NULL_COUNT -> "" + ) + AddFile( + part_name, + partitionValues, + size_in_bytes, + modificationTime, + dataChange = true, + JsonUtils.toJson(stats), + tags) + } +} + +object BasicMergeTreeNativeStats { + + /** + * we assume the number of records is less than 10,000.So the memory overhead is acceptable. + * otherwise, we need to access ColumnarBatch row by row, which is not efficient. + */ + def apply(cb: ColumnarBatch): Seq[BasicMergeTreeNativeStat] = + CHExecUtil + .c2r(cb) + .map( + row => + BasicMergeTreeNativeStat( + row.getUTF8String(0).toString, + row.getUTF8String(1).toString, + row.getLong(2), + row.getLong(3), + row.getLong(4) + )) + .toSeq +} + +case class MergeTreeDeltaColumnarWrite( + override val jobTrackerID: String, + override val description: WriteJobDescription, + override val committer: MergeTreeDelayedCommitProtocol2) + extends CHColumnarWrite[MergeTreeDelayedCommitProtocol2] + with Logging { + override def doSetupNativeTask(): Unit = { + assert(description.path == committer.outputPath) + val writePath = StorageMeta.normalizeRelativePath(committer.outputPath) + val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId + val partPrefixWithoutPartitionAndBucket = s"${UUID.randomUUID.toString}_$split" + logDebug( + s"Pipeline write path: $writePath with part prefix: $partPrefixWithoutPartitionAndBucket") + val settings = + Map( + RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath, + RuntimeSettings.PART_NAME_PREFIX.key -> partPrefixWithoutPartitionAndBucket) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) + } + + private def doCollectNativeResult(cb: ColumnarBatch): Option[WriteTaskResult] = { + val basicNativeStats = BasicMergeTreeNativeStats(cb) + + // TODO: we need close iterator here before processing the result. + + if (basicNativeStats.isEmpty) { + None + } else { + val modificationTime = System.currentTimeMillis() + val hostName = Seq(Utils.localHostName()) + val path = new Path(committer.outputPath) + var numRows: Long = 0 + var numBytes: Long = 0 + val numFiles = basicNativeStats.size + val addFiles = basicNativeStats.map { + stat => + if (stat.partition_id != "__NO_PARTITION_ID__") { + basicWriteJobStatsTracker.newPartition( + new GenericInternalRow(Array[Any](stat.partition_id))) + } + numRows += stat.record_count + numBytes += stat.size_in_bytes + stat(committer.database, committer.tableName, path, modificationTime, hostName) + } + + Some { + WriteTaskResult( + new TaskCommitMessage(addFiles), + ExecutedWriteSummary( + updatedPartitions = Set.empty, + stats = + Seq(finalStats.copy(numFiles = numFiles, numBytes = numBytes, numRows = numRows))) + ) + } + } + } + + override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = { + doCollectNativeResult(batch) + } +} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java index a4f272a85355..370f434a3c88 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java @@ -25,7 +25,6 @@ import org.apache.spark.sql.internal.SQLConf; -import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -58,11 +57,6 @@ private static Map getNativeBackendConf() { return GlutenConfig.getNativeBackendConf(Backend.get().name(), SQLConf.get().getAllConfs()); } - public static void injectWriteFilesTempPath(String path, String fileName) { - ExpressionEvaluatorJniWrapper.injectWriteFilesTempPath( - path.getBytes(StandardCharsets.UTF_8), fileName.getBytes(StandardCharsets.UTF_8)); - } - // Used by WholeStageTransform to create the native computing pipeline and // return a columnar result iterator. public static BatchIterator createKernelWithBatchIterator( diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java index 8cc3046436ab..76fddcafdd41 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java @@ -16,12 +16,7 @@ */ package org.apache.gluten.vectorized; -import org.apache.gluten.backendsapi.clickhouse.CHConf; import org.apache.gluten.execution.ColumnarNativeIterator; -import org.apache.gluten.utils.ConfigUtil; - -import java.util.Map; -import java.util.stream.Collectors; /** * This class is implemented in JNI. This provides the Java interface to invoke functions in JNI. @@ -48,23 +43,5 @@ public static native long nativeCreateKernelWithIterator( byte[] confArray, boolean materializeInput); - /** - * Set the temp path for writing files. - * - * @param path the temp path for writing files - */ - public static native void injectWriteFilesTempPath(byte[] path, byte[] filename); - - /// The following methods are used to update the query settings in the native engine. - public static void updateQueryRuntimeSettings(Map settings) { - Map newSettings = - settings.entrySet().stream() - .filter(entry -> CHConf.startWithSettingsPrefix(entry.getKey())) - .collect( - Collectors.toMap( - e -> CHConf.removeSettingsPrefix(e.getKey()), Map.Entry::getValue)); - updateQueryRuntimeSettings(ConfigUtil.serialize(newSettings)); - } - - private static native void updateQueryRuntimeSettings(byte[] settings); + public static native void updateQueryRuntimeSettings(byte[] settings); } diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java index b09cccb4580f..c73e1076ddf8 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java @@ -34,7 +34,7 @@ public class SimpleExpressionEval implements AutoCloseable, Iterator { public SimpleExpressionEval(ColumnarNativeIterator blockStream, PlanNode planNode) { Plan plan = planNode.toProtobuf(); if (LOG.isDebugEnabled()) { - LOG.debug(String.format("SimpleExpressionEval exec plan: %s", plan.toString())); + LOG.debug("SimpleExpressionEval exec plan: {}", plan.toString()); } byte[] planData = plan.toByteArray(); instance = createNativeInstance(blockStream, planData); diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java index fb2b50798836..ee000d5ae157 100644 --- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java @@ -26,10 +26,8 @@ public CHDatasourceJniWrapper(String filePath, WriteRel write) { this.instance = createFilerWriter(filePath, write.toByteArray()); } - public CHDatasourceJniWrapper( - String prefix, String partition, String bucket, WriteRel write, byte[] confArray) { - this.instance = - createMergeTreeWriter(prefix, partition, bucket, write.toByteArray(), confArray); + public CHDatasourceJniWrapper(WriteRel write, byte[] confArray) { + this.instance = createMergeTreeWriter(write.toByteArray(), confArray); } public void write(long blockAddress) { @@ -48,8 +46,7 @@ public void close() { private native long createFilerWriter(String filePath, byte[] writeRel); /// MergeTreeWriter - private native long createMergeTreeWriter( - String prefix, String partition, String bucket, byte[] writeRel, byte[] confArray); + private native long createMergeTreeWriter(byte[] writeRel, byte[] confArray); public static native String nativeMergeMTParts( byte[] splitInfo, String partition_dir, String bucket_dir); diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java index 223e92c28d1d..49311504ced0 100644 --- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java +++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java @@ -18,8 +18,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Map; - public class WriteReturnedMetric { @JsonProperty("part_name") @@ -38,7 +36,7 @@ public class WriteReturnedMetric { protected String bucketId; @JsonProperty("partition_values") - protected Map partitionValues; + protected String partitionValues; public String getPartName() { return partName; @@ -72,11 +70,11 @@ public void setRowCount(long rowCount) { this.rowCount = rowCount; } - public Map getPartitionValues() { + public String getPartitionValues() { return partitionValues; } - public void setPartitionValues(Map partitionValues) { + public void setPartitionValues(String partitionValues) { this.partitionValues = partitionValues; } diff --git a/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto b/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto index 8404f81f19f4..89f606e4ffd3 100644 --- a/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto +++ b/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto @@ -6,6 +6,8 @@ package local_engine; option java_package = "org.apache.spark.sql.execution.datasources.v1"; option java_multiple_files = true; +//TODO : set compression codec + message Write { message Common { string format = 1; diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala index 3f653ad200c0..83cc75bc0ac7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala @@ -19,6 +19,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenConfig import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf object CHConf { private[clickhouse] val BACKEND_NAME: String = "ch" @@ -47,10 +48,26 @@ object CHConf { } } + /** CH configuration prefix at Java side */ def prefixOf(key: String): String = s"$CONF_PREFIX.$key" def runtimeConfig(key: String): String = s"$RUNTIME_CONFIG.$key" def runtimeSettings(key: String): String = s"$RUNTIME_SETTINGS.$key" def startWithSettingsPrefix(key: String): Boolean = key.startsWith(RUNTIME_SETTINGS) def removeSettingsPrefix(key: String): String = key.substring(RUNTIME_SETTINGS.length + 1) + + def get: CHConf = new CHConf(SQLConf.get) + + import SQLConf._ + + val ENABLE_ONEPIPELINE_MERGETREE_WRITE = + buildConf(prefixOf("mergetree.write.pipeline")) + .doc("Using one pipeline to write data to MergeTree table in Spark 3.5") + .booleanConf + .createWithDefault(false) +} + +class CHConf(conf: SQLConf) extends GlutenConfig(conf) { + def enableOnePipelineMergeTreeWrite: Boolean = + conf.getConf(CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala index 2fa2e4402b79..dd5a736e7571 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala @@ -21,13 +21,12 @@ import org.apache.gluten.backendsapi.IteratorApi import org.apache.gluten.execution._ import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.logging.LogLevelUtil -import org.apache.gluten.memory.CHThreadGroup import org.apache.gluten.metrics.IMetrics import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.plan.PlanNode import org.apache.gluten.substrait.rel._ import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat -import org.apache.gluten.vectorized.{BatchIterator, CHNativeExpressionEvaluator, CloseableCHColumnBatchIterator} +import org.apache.gluten.vectorized.{BatchIterator, CHNativeExpressionEvaluator, CloseableCHColumnBatchIterator, NativeExpressionEvaluator} import org.apache.spark.{InterruptibleIterator, SparkConf, TaskContext} import org.apache.spark.affinity.CHAffinity @@ -325,9 +324,16 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { createNativeIterator(splitInfoByteArray, wsPlan, materializeInput, inputIterators)) } + /** + * This function used to inject the staging write path before initializing the native plan.Only + * used in a pipeline model (spark 3.5) for writing parquet or orc files. + */ override def injectWriteFilesTempPath(path: String, fileName: String): Unit = { - CHThreadGroup.registerNewThreadGroup() - CHNativeExpressionEvaluator.injectWriteFilesTempPath(path, fileName) + val settings = + Map( + RuntimeSettings.TASK_WRITE_TMP_DIR.key -> path, + RuntimeSettings.TASK_WRITE_FILENAME.key -> fileName) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala index bf909c52ac20..438c6cd3dfb0 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.backendsapi.TransformerApi -import org.apache.gluten.execution.{CHHashAggregateExecTransformer, WriteFilesExecTransformer} +import org.apache.gluten.execution.CHHashAggregateExecTransformer import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.substrait.expression.{BooleanLiteralNode, ExpressionBuilder, ExpressionNode} import org.apache.gluten.utils.{CHInputPartitionsUtil, ExpressionDocUtil} @@ -26,18 +26,22 @@ import org.apache.spark.internal.Logging import org.apache.spark.rpc.GlutenDriverEndpoint import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.delta.MergeTreeFileFormat import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 import org.apache.spark.sql.delta.files.TahoeFileIndex import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation, PartitionDirectory} +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v1.Write import org.apache.spark.sql.execution.datasources.v2.clickhouse.source.DeltaMergeTreeFileFormat import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types._ import org.apache.spark.util.collection.BitSet import com.google.common.collect.Lists -import com.google.protobuf.{Any, Message, StringValue} +import com.google.protobuf.{Any, Message} import java.util @@ -246,16 +250,23 @@ class CHTransformerApi extends TransformerApi with Logging { register.shortName case _ => "UnknownFileFormat" } - val compressionCodec = - WriteFilesExecTransformer.getCompressionCodec(writeOptions).capitalize - val writeParametersStr = new StringBuffer("WriteParameters:") - writeParametersStr.append("is").append(compressionCodec).append("=1") - writeParametersStr.append(";format=").append(fileFormatStr).append("\n") + val write = Write + .newBuilder() + .setCommon( + Write.Common + .newBuilder() + .setFormat(fileFormatStr) + .setJobTaskAttemptId("") // we can get job and task id at the driver side + .build()) - packPBMessage( - StringValue - .newBuilder() - .setValue(writeParametersStr.toString) - .build()) + fileFormat match { + case d: MergeTreeFileFormat => + write.setMergetree(MergeTreeFileFormat.createWrite(d.metadata)) + case _: ParquetFileFormat => + write.setParquet(Write.ParquetWrite.newBuilder().build()) + case _: OrcFileFormat => + write.setOrc(Write.OrcWrite.newBuilder().build()) + } + packPBMessage(write.build()) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala new file mode 100644 index 000000000000..12bb8d05d953 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.backendsapi.clickhouse + +import org.apache.spark.sql.internal.SQLConf + +object RuntimeConfig { + import CHConf._ + import SQLConf._ + + val PATH = + buildConf(runtimeConfig("path")) + .doc( + "https://clickhouse.com/docs/en/operations/server-configuration-parameters/settings#path") + .stringConf + .createWithDefault("/") + + // scalastyle:off line.size.limit + val TMP_PATH = + buildConf(runtimeConfig("tmp_path")) + .doc("https://clickhouse.com/docs/en/operations/server-configuration-parameters/settings#tmp-path") + .stringConf + .createWithDefault("/tmp/libch") + // scalastyle:on line.size.limit + + val USE_CURRENT_DIRECTORY_AS_TMP = + buildConf(runtimeConfig("use_current_directory_as_tmp")) + .doc("Use the current directory as the temporary directory.") + .booleanConf + .createWithDefault(false) +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala new file mode 100644 index 000000000000..1e37e21ba70d --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.backendsapi.clickhouse + +import org.apache.spark.sql.internal.SQLConf + +object RuntimeSettings { + + import CHConf._ + import SQLConf._ + + val NATIVE_WRITE_RESERVE_PARTITION_COLUMNS = + buildConf(runtimeSettings("gluten.write.reserve_partition_columns")) + .doc("Whether reserve partition columns for Native write or not, default is false") + .booleanConf + .createWithDefault(false) + + val TASK_WRITE_TMP_DIR = + buildConf(runtimeSettings("gluten.task_write_tmp_dir")) + .doc("The temporary directory for writing data") + .stringConf + .createWithDefault("") + + val TASK_WRITE_FILENAME = + buildConf(runtimeSettings("gluten.task_write_filename")) + .doc("The temporary file name for writing data") + .stringConf + .createWithDefault("") + + val PART_NAME_PREFIX = + buildConf(runtimeSettings("gluten.part_name_prefix")) + .doc("The part name prefix for writing data") + .stringConf + .createWithDefault("") + + val PARTITION_DIR = + buildConf(runtimeSettings("gluten.partition_dir")) + .doc("The partition directory for writing data") + .stringConf + .createWithDefault("") + + val BUCKET_DIR = + buildConf(runtimeSettings("gluten.bucket_dir")) + .doc("The bucket directory for writing data") + .stringConf + .createWithDefault("") +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala new file mode 100644 index 000000000000..35533a4b3e9c --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.vectorized + +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.utils.ConfigUtil + +import scala.collection.JavaConverters._ + +// TODO: move CHNativeExpressionEvaluator to NativeExpressionEvaluator +/** Scala Wrapper for ExpressionEvaluatorJniWrapper */ +object NativeExpressionEvaluator { + + def updateQueryRuntimeSettings(settings: Map[String, String]): Unit = { + ExpressionEvaluatorJniWrapper.updateQueryRuntimeSettings( + ConfigUtil.serialize( + settings + .filter(t => CHConf.startWithSettingsPrefix(t._1) && t._2.nonEmpty) + .map { + case (k, v) => + (CHConf.removeSettingsPrefix(k), v) + } + .asJava)) + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala index dd24d475340b..bea22404eafe 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala @@ -21,7 +21,8 @@ import org.apache.gluten.execution.datasource.GlutenFormatFactory import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.actions.Metadata import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} -import org.apache.spark.sql.execution.datasources.mergetree.DeltaMetaReader +import org.apache.spark.sql.execution.datasources.mergetree.{DeltaMetaReader, StorageMeta} +import org.apache.spark.sql.execution.datasources.v1.Write import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.StructType @@ -64,3 +65,28 @@ trait MergeTreeFileFormat extends FileFormat with DataSourceRegister { } } } + +object MergeTreeFileFormat { + def createWrite(outputPath: Option[String], conf: Map[String, String]): Write.MergeTreeWrite = { + Write.MergeTreeWrite + .newBuilder() + .setDatabase(conf(StorageMeta.DB)) + .setTable(conf(StorageMeta.TABLE)) + .setSnapshotId(conf(StorageMeta.SNAPSHOT_ID)) + .setOrderByKey(conf(StorageMeta.ORDER_BY_KEY)) + .setLowCardKey(conf(StorageMeta.LOW_CARD_KEY)) + .setMinmaxIndexKey(conf(StorageMeta.MINMAX_INDEX_KEY)) + .setBfIndexKey(conf(StorageMeta.BF_INDEX_KEY)) + .setSetIndexKey(conf(StorageMeta.SET_INDEX_KEY)) + .setPrimaryKey(conf(StorageMeta.PRIMARY_KEY)) + .setRelativePath(outputPath.map(StorageMeta.normalizeRelativePath).getOrElse("")) + .setAbsolutePath("") + .setStoragePolicy(conf(StorageMeta.POLICY)) + .build() + } + def createWrite(metadata: Metadata): Write.MergeTreeWrite = { + // we can get the output path at the driver side + val deltaMetaReader = DeltaMetaReader(metadata) + createWrite(None, deltaMetaReader.storageConf) + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala index bce43b0ef288..13a9efa35989 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala @@ -16,9 +16,9 @@ */ package org.apache.spark.sql.delta.files -import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings import org.apache.gluten.memory.CHThreadGroup -import org.apache.gluten.vectorized.ExpressionEvaluatorJniWrapper +import org.apache.gluten.vectorized.NativeExpressionEvaluator import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage @@ -31,8 +31,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext import java.util.UUID -import scala.collection.JavaConverters._ - trait MergeTreeFileCommitProtocol extends FileCommitProtocol { def outputPath: String @@ -45,8 +43,8 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol { val jobID = taskContext.getJobID.toString val taskAttemptID = taskContext.getTaskAttemptID.toString MergeTreeCommiterHelper.prepareTaskWriteInfo(jobID, taskAttemptID) - val settings = Map(CHConf.runtimeSettings("gluten.write.reserve_partition_columns") -> "true") - ExpressionEvaluatorJniWrapper.updateQueryRuntimeSettings(settings.asJava) + val settings = Map(RuntimeSettings.NATIVE_WRITE_RESERVE_PARTITION_COLUMNS.key -> "true") + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) } override def newTaskTempFile( @@ -54,22 +52,22 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol { dir: Option[String], ext: String): String = { - taskContext.getConfiguration.set( - "mapreduce.task.gluten.mergetree.partition", - dir.map(p => new Path(p).toUri.toString).getOrElse("")) - + val partitionStr = dir.map(p => new Path(p).toUri.toString) val bucketIdStr = ext.split("\\.").headOption.filter(_.startsWith("_")).map(_.substring(1)) - taskContext.getConfiguration.set( - "mapreduce.task.gluten.mergetree.bucketid", - bucketIdStr.getOrElse("")) + val split = taskContext.getTaskAttemptID.getTaskID.getId - val partition = dir.map(p => new Path(p).toUri.toString + "/").getOrElse("") + // The partPrefix is used to generate the part name in the MergeTree table. + // outputPath/partition-dir/bucket-id/UUID_split + val partition = partitionStr.map(_ + "/").getOrElse("") val bucket = bucketIdStr.map(_ + "/").getOrElse("") - val taskID = taskContext.getTaskAttemptID.getTaskID.getId.toString - val partPrefix = s"$partition$bucket${UUID.randomUUID.toString}_$taskID" - - taskContext.getConfiguration.set("mapreduce.task.gluten.mergetree.partPrefix", partPrefix) + val partPrefix = s"$partition$bucket${UUID.randomUUID.toString}_$split" + val settings = Map( + RuntimeSettings.PART_NAME_PREFIX.key -> partPrefix, + RuntimeSettings.PARTITION_DIR.key -> partitionStr.getOrElse(""), + RuntimeSettings.BUCKET_DIR.key -> bucketIdStr.getOrElse("") + ) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) outputPath } @@ -87,4 +85,8 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol { ) new TaskCommitMessage(statuses) } + + override def abortTask(taskContext: TaskAttemptContext): Unit = { + MergeTreeCommiterHelper.resetCurrentTaskWriteInfo() + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala new file mode 100644 index 000000000000..03b45e540dc9 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.delta.util + +import org.apache.spark.sql.catalyst.expressions.Cast +import org.apache.spark.sql.types.StringType + +import org.apache.hadoop.fs.Path + +/** + * `OptimizeTableCommandOverwrites` does not use `DelayedCommitProtocol`, so we can't use + * `DelayedCommitProtocol.parsePartitions`. This is a copied version.
    TODO: Remove it. + */ +object MergeTreePartitionUtils { + + private val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]" + + def parsePartitions(dir: String): Map[String, String] = { + // TODO: timezones? + // TODO: enable validatePartitionColumns? + val dateFormatter = DateFormatter() + val timestampFormatter = + TimestampFormatter(timestampPartitionPattern, java.util.TimeZone.getDefault) + val parsedPartition = + PartitionUtils + .parsePartition( + new Path(dir), + typeInference = false, + Set.empty, + Map.empty, + validatePartitionColumns = false, + java.util.TimeZone.getDefault, + dateFormatter, + timestampFormatter) + ._1 + .get + parsedPartition.columnNames + .zip( + parsedPartition.literals + .map(l => Cast(l, StringType).eval()) + .map(Option(_).map(_.toString).orNull)) + .toMap + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala index 29fa0d0aba96..174493e5f4f6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.gluten.extension.ValidationResult import org.apache.gluten.metrics.GlutenTimeMetric -import org.apache.gluten.vectorized.CHNativeBlock import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} import org.apache.spark.broadcast.Broadcast @@ -103,10 +102,7 @@ class CHColumnarToRowRDD( logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols") Iterator.empty } else { - val blockAddress = GlutenTimeMetric.millis(convertTime) { - _ => CHNativeBlock.fromColumnarBatch(batch).blockAddress() - } - CHExecUtil.getRowIterFromSparkRowInfo(blockAddress, batch.numCols(), batch.numRows()) + GlutenTimeMetric.millis(convertTime)(_ => CHExecUtil.c2r(batch)) } } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala index 6a5c19a4f939..e1496527d0bf 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala @@ -23,7 +23,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec, HadoopMapReduceCommitProtocol} import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.execution.datasources.{BasicWriteTaskStats, ExecutedWriteSummary, PartitioningUtils, WriteJobDescription, WriteTaskResult} +import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, BasicWriteTaskStats, ExecutedWriteSummary, PartitioningUtils, WriteJobDescription, WriteTaskResult, WriteTaskStatsTracker} +import org.apache.spark.sql.execution.utils.CHExecUtil import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -53,6 +54,16 @@ trait CHColumnarWrite[T <: FileCommitProtocol] { } def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] + lazy val basicWriteJobStatsTracker: WriteTaskStatsTracker = description.statsTrackers + .find(_.isInstanceOf[BasicWriteJobStatsTracker]) + .map(_.newTaskInstance()) + .get + + // TODO: task commit time + def finalStats: BasicWriteTaskStats = basicWriteJobStatsTracker + .getFinalStats(0) + .asInstanceOf[BasicWriteTaskStats] + lazy val (taskAttemptContext: TaskAttemptContext, jobId: String) = { // Copied from `SparkHadoopWriterUtils.createJobID` to be compatible with multi-version def createJobID(jobTrackerID: String, id: Int): JobID = { @@ -90,25 +101,6 @@ object CreateFileNameSpec { } } -object CreateBasicWriteTaskStats { - def apply( - numFiles: Int, - updatedPartitions: Set[String], - numWrittenRows: Long): BasicWriteTaskStats = { - val partitionsInternalRows = updatedPartitions.map { - part => - val parts = new Array[Any](1) - parts(0) = part - new GenericInternalRow(parts) - }.toSeq - BasicWriteTaskStats( - partitions = partitionsInternalRows, - numFiles = numFiles, - numBytes = 101, - numRows = numWrittenRows) - } -} - /** [[HadoopMapReduceAdapter]] for [[HadoopMapReduceCommitProtocol]]. */ case class HadoopMapReduceAdapter(sparkCommitter: HadoopMapReduceCommitProtocol) { private lazy val committer: OutputCommitter = { @@ -148,6 +140,42 @@ case class HadoopMapReduceAdapter(sparkCommitter: HadoopMapReduceCommitProtocol) } } +/** + * {{{ + * val schema = + * StructType( + * StructField("filename", StringType, false) :: + * StructField("partition_id", StringType, false) :: + * StructField("record_count", LongType, false) :: Nil) + * }}} + */ +case class BasicNativeStat(filename: String, partition_id: String, record_count: Long) { + + lazy val relativePath: String = if (partition_id == "__NO_PARTITION_ID__") { + filename + } else { + s"$partition_id/$filename" + } +} +object BasicNativeStats { + + /** + * we assume the number of records is less than 10,000.So the memory overhead is acceptable. + * otherwise, we need to access ColumnarBatch row by row, which is not efficient. + */ + def apply(cb: ColumnarBatch): Seq[BasicNativeStat] = + CHExecUtil + .c2r(cb) + .map( + row => + BasicNativeStat( + row.getUTF8String(0).toString, + row.getUTF8String(1).toString, + row.getLong(2) + )) + .toSeq +} + case class HadoopMapReduceCommitProtocolWrite( override val jobTrackerID: String, override val description: WriteJobDescription, @@ -169,46 +197,44 @@ case class HadoopMapReduceCommitProtocolWrite( } def doCollectNativeResult(cb: ColumnarBatch): Option[WriteTaskResult] = { - val numFiles = cb.numRows() + val basicNativeStats = BasicNativeStats(cb) + + // TODO: we need close iterator here before processing the result. // Write an empty iterator - if (numFiles == 0) { + if (basicNativeStats.isEmpty) { None } else { - val file_col = cb.column(0) - val partition_col = cb.column(1) - val count_col = cb.column(2) - - val outputPath = description.path val partitions: mutable.Set[String] = mutable.Set[String]() val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]() var numWrittenRows: Long = 0 - Range(0, cb.numRows()).foreach { - i => - val targetFileName = file_col.getUTF8String(i).toString - val partition = partition_col.getUTF8String(i).toString - if (partition != "__NO_PARTITION_ID__") { - partitions += partition - val tmpOutputPath = outputPath + "/" + partition + "/" + targetFileName + basicNativeStats.foreach { + stat => + val tmpAbsolutePath = s"${description.path}/${stat.relativePath}" + if (stat.partition_id != "__NO_PARTITION_ID__") { + partitions += stat.partition_id val customOutputPath = description.customPartitionLocations.get( - PartitioningUtils.parsePathFragment(partition)) + PartitioningUtils.parsePathFragment(stat.partition_id)) if (customOutputPath.isDefined) { - addedAbsPathFiles(tmpOutputPath) = customOutputPath.get + "/" + targetFileName + addedAbsPathFiles(tmpAbsolutePath) = customOutputPath.get + "/" + stat.filename } + basicWriteJobStatsTracker.newPartition( + new GenericInternalRow(Array[Any](stat.partition_id))) } - numWrittenRows += count_col.getLong(i) + basicWriteJobStatsTracker.newFile(tmpAbsolutePath) + basicWriteJobStatsTracker.closeFile(tmpAbsolutePath) + numWrittenRows += stat.record_count } val updatedPartitions = partitions.toSet - val summary = - ExecutedWriteSummary( - updatedPartitions = updatedPartitions, - stats = Seq(CreateBasicWriteTaskStats(numFiles, updatedPartitions, numWrittenRows))) Some( WriteTaskResult( new TaskCommitMessage(addedAbsPathFiles.toMap -> updatedPartitions), - summary)) + ExecutedWriteSummary( + updatedPartitions = updatedPartitions, + stats = Seq(finalStats.copy(numRows = numWrittenRows))) + )) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala index bf051671fbba..ab3f652cda23 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution +import org.apache.gluten.memory.CHThreadGroup + import org.apache.spark.{Partition, SparkException, TaskContext, TaskOutputFileAlreadyExistException} import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.rdd.RDD @@ -46,14 +48,15 @@ class CHColumnarWriteFilesRDD( extends RDD[WriterCommitMessage](prev) { private def reportTaskMetrics(writeTaskResult: WriteTaskResult): Unit = { - val stats = writeTaskResult.summary.stats.head.asInstanceOf[BasicWriteTaskStats] - val (numBytes, numWrittenRows) = (stats.numBytes, stats.numRows) - // Reports bytesWritten and recordsWritten to the Spark output metrics. - // We should update it after calling `commitTask` to overwrite the metrics. - Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach { - outputMetrics => - outputMetrics.setBytesWritten(numBytes) - outputMetrics.setRecordsWritten(numWrittenRows) + writeTaskResult.summary.stats.headOption.map(_.asInstanceOf[BasicWriteTaskStats]).foreach { + stats => + // Reports bytesWritten and recordsWritten to the Spark output metrics. + // We should update it after calling `commitTask` to overwrite the metrics. + Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach { + outputMetrics => + outputMetrics.setBytesWritten(stats.numBytes) + outputMetrics.setRecordsWritten(stats.numRows) + } } } @@ -78,6 +81,7 @@ class CHColumnarWriteFilesRDD( } override def compute(split: Partition, context: TaskContext): Iterator[WriterCommitMessage] = { + CHThreadGroup.registerNewThreadGroup() val commitProtocol = CHColumnarWrite(jobTrackerID, description, committer) commitProtocol.setupTask() diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala index b360e286e504..7f80c134e5d6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala @@ -78,7 +78,7 @@ trait CHFormatWriterInjects extends GlutenFormatWriterInjectsBase { val datasourceJniWrapper = new CHDatasourceJniWrapper(outputPath, createWriteRel(outputPath, dataSchema, context)) - new FakeRowOutputWriter(datasourceJniWrapper, outputPath) + new FakeRowOutputWriter(Some(datasourceJniWrapper), outputPath) } override def inferSchema( diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala index 76f4b467ded2..3e0e393459a6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.v1 +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.substrait.`type`.ColumnTypeNode @@ -26,6 +27,7 @@ import org.apache.gluten.substrait.rel.RelBuilder import org.apache.gluten.utils.ConfigUtil import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.delta.MergeTreeFileFormat import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.{CHDatasourceJniWrapper, FakeRowAdaptor, OutputWriter} @@ -75,22 +77,7 @@ class CHMergeTreeWriterInjects extends CHFormatWriterInjects { .setFormat(formatName) .setJobTaskAttemptId(s"$jobID/$taskAttemptID") .build()) - .setMergetree( - Write.MergeTreeWrite - .newBuilder() - .setDatabase(conf(StorageMeta.DB)) - .setTable(conf(StorageMeta.TABLE)) - .setSnapshotId(conf(StorageMeta.SNAPSHOT_ID)) - .setOrderByKey(conf(StorageMeta.ORDER_BY_KEY)) - .setLowCardKey(conf(StorageMeta.LOW_CARD_KEY)) - .setMinmaxIndexKey(conf(StorageMeta.MINMAX_INDEX_KEY)) - .setBfIndexKey(conf(StorageMeta.BF_INDEX_KEY)) - .setSetIndexKey(conf(StorageMeta.SET_INDEX_KEY)) - .setPrimaryKey(conf(StorageMeta.PRIMARY_KEY)) - .setRelativePath(StorageMeta.normalizeRelativePath(outputPath)) - .setAbsolutePath("") - .setStoragePolicy(conf(StorageMeta.POLICY)) - .build()) + .setMergetree(MergeTreeFileFormat.createWrite(Some(outputPath), conf)) .build() } @@ -100,14 +87,24 @@ class CHMergeTreeWriterInjects extends CHFormatWriterInjects { context: TaskAttemptContext, nativeConf: ju.Map[String, String]): OutputWriter = { - val datasourceJniWrapper = new CHDatasourceJniWrapper( - context.getConfiguration.get("mapreduce.task.gluten.mergetree.partPrefix"), - context.getConfiguration.get("mapreduce.task.gluten.mergetree.partition"), - context.getConfiguration.get("mapreduce.task.gluten.mergetree.bucketid"), - createWriteRel(outputPath, dataSchema, context), - ConfigUtil.serialize(nativeConf) - ) - new FakeRowOutputWriter(datasourceJniWrapper, outputPath) + val wrapper = if (CHConf.get.enableOnePipelineMergeTreeWrite) { + + /** + * In pipeline mode, CHColumnarWriteFilesRDD.writeFilesForEmptyIterator will create a JNI + * wrapper which is not needed in this case. + * + * TODO: We should refactor the code to avoid creating the JNI wrapper in this case. + */ + None + } else { + val datasourceJniWrapper = new CHDatasourceJniWrapper( + createWriteRel(outputPath, dataSchema, context), + ConfigUtil.serialize(nativeConf) + ) + Some(datasourceJniWrapper) + } + + new FakeRowOutputWriter(wrapper, outputPath) } override val formatName: String = "mergetree" diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala index 9ebce9d3b455..574c9784257d 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.datasources.{CHDatasourceJniWrapper, FakeR import scala.collection.mutable.ArrayBuffer -class FakeRowOutputWriter(datasourceJniWrapper: CHDatasourceJniWrapper, outputPath: String) +class FakeRowOutputWriter(datasourceJniWrapper: Option[CHDatasourceJniWrapper], outputPath: String) extends OutputWriter { protected var addFiles: ArrayBuffer[AddFile] = new ArrayBuffer[AddFile]() @@ -35,12 +35,12 @@ class FakeRowOutputWriter(datasourceJniWrapper: CHDatasourceJniWrapper, outputPa if (nextBatch.numRows > 0) { val col = nextBatch.column(0).asInstanceOf[CHColumnVector] - datasourceJniWrapper.write(col.getBlockAddress) + datasourceJniWrapper.foreach(_.write(col.getBlockAddress)) } // else ignore this empty block } override def close(): Unit = { - datasourceJniWrapper.close() + datasourceJniWrapper.foreach(_.close()) } // Do NOT add override keyword for compatibility on spark 3.1. diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala index 1d6a488afebf..436c6fa66a29 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala @@ -48,4 +48,8 @@ object MergeTreeCommiterHelper { require(currentTaskWriteInfo.get() == null, "currentTaskWriteInfo is not null") currentTaskWriteInfo.set(TaskWriteInfo(s"$jobID/$taskAttemptID")) } + + def resetCurrentTaskWriteInfo(): Unit = { + currentTaskWriteInfo.remove() + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala index 8c610929a3d8..c4c971633a11 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.util.MergeTreePartitionUtils import org.apache.spark.sql.execution.datasources.clickhouse.WriteReturnedMetric import com.fasterxml.jackson.core.`type`.TypeReference @@ -26,7 +27,6 @@ import org.apache.hadoop.fs.Path import java.util.{List => JList} import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer @SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) class AddMergeTreeParts( @@ -192,46 +192,47 @@ object AddFileTags { tableName: String, originPathStr: String, returnedMetrics: String, - hostName: Seq[String]): ArrayBuffer[AddFile] = { + hostName: Seq[String]): Seq[AddFile] = { + val mapper: ObjectMapper = new ObjectMapper() - try { - val values: JList[WriteReturnedMetric] = - mapper.readValue(returnedMetrics, new TypeReference[JList[WriteReturnedMetric]]() {}) - var addFiles = new ArrayBuffer[AddFile]() - val path = new Path(originPathStr) - val modificationTime = System.currentTimeMillis() - addFiles.appendAll(values.asScala.map { - value => - AddFileTags.partsInfoToAddFile( - database, - tableName, - "MergeTree", - path.toUri.getPath, - hostName.map(_.trim).mkString(","), - value.getPartName, - "", - value.getRowCount, - value.getDiskSize, - -1L, - -1L, - modificationTime, - "", - -1L, - -1L, - -1, - -1L, - value.getBucketId, - path.toString, - dataChange = true, - "", - partitionValues = value.getPartitionValues.asScala.toMap, - marks = value.getMarkCount - ) - }) - addFiles - } catch { - case e: Exception => - ArrayBuffer.empty[AddFile] - } + val values: JList[WriteReturnedMetric] = + mapper.readValue(returnedMetrics, new TypeReference[JList[WriteReturnedMetric]]() {}) + val path = new Path(originPathStr) + val modificationTime = System.currentTimeMillis() + + values.asScala.map { + value => + val partitionValues = if (value.getPartitionValues.isEmpty) { + Map.empty[String, String] + } else { + MergeTreePartitionUtils.parsePartitions(value.getPartitionValues) + } + + AddFileTags.partsInfoToAddFile( + database, + tableName, + "MergeTree", + path.toUri.getPath, + hostName.map(_.trim).mkString(","), + value.getPartName, + "", + value.getRowCount, + value.getDiskSize, + -1L, + -1L, + modificationTime, + "", + -1L, + -1L, + -1, + -1L, + value.getBucketId, + path.toString, + dataChange = true, + "", + partitionValues = partitionValues, + marks = value.getMarkCount + ) + }.toSeq } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala index 6d91108d22da..cc172ac4b543 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala @@ -113,12 +113,8 @@ object CHExecUtil extends Logging { iter => iter.flatMap( batch => { - val blockAddress = CHNativeBlock.fromColumnarBatch(batch).blockAddress() - // generate rows from a columnar batch - val rowItr: Iterator[InternalRow] = - getRowIterFromSparkRowInfo(blockAddress, batch.numCols(), batch.numRows()) - + val rowItr: Iterator[InternalRow] = c2r(batch) val projection = UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) val mutablePair = new MutablePair[InternalRow, Null]() @@ -164,6 +160,14 @@ object CHExecUtil extends Logging { val rowInfo = CHBlockConverterJniWrapper.convertColumnarToRow(blockAddress, null) getRowIterFromSparkRowInfo(rowInfo, columns, rows) } + + def c2r(batch: ColumnarBatch): Iterator[InternalRow] = { + getRowIterFromSparkRowInfo( + CHNativeBlock.fromColumnarBatch(batch).blockAddress(), + batch.numCols(), + batch.numRows()) + } + private def buildPartitionedBlockIterator( cbIter: Iterator[ColumnarBatch], options: IteratorOptions, diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index 5c6f669198c2..6ed106e2c75a 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -17,6 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.clickhouse.RuntimeConfig import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} @@ -82,8 +83,8 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.sql.warehouse.dir", warehouse) .setCHConfig("user_defined_path", "/tmp/user_defined") - .setCHConfig("path", UTSystemParameters.diskOutputDataPath) - .setCHConfig("tmp_path", s"/tmp/libch/$SPARK_DIR_NAME") + .set(RuntimeConfig.PATH.key, UTSystemParameters.diskOutputDataPath) + .set(RuntimeConfig.TMP_PATH.key, s"/tmp/libch/$SPARK_DIR_NAME") if (UTSystemParameters.testMergeTreeOnObjectStorage) { conf .set("spark.hadoop.fs.s3a.access.key", S3_ACCESS_KEY) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index 033d51467177..79931ec9490f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -66,97 +66,99 @@ class GlutenClickHouseMergeTreeWriteSuite } test("test mergetree table write") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree; + |""".stripMargin) - // write.format.default = mergetree - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |TBLPROPERTIES (write.format.default = 'mergetree') - |LOCATION '$basePath/lineitem_mergetree' - |""".stripMargin) + // write.format.default = mergetree + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |TBLPROPERTIES (write.format.default = 'mergetree') + |LOCATION '$basePath/lineitem_mergetree' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree + | select * from lineitem + |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { - df => - val plans = collect(df.queryExecution.executedPlan) { - case f: FileSourceScanExecTransformer => f - case w: WholeStageTransformer => w - } - assertResult(4)(plans.size) + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_mergetree + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr) { + df => + val plans = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + case w: WholeStageTransformer => w + } + assertResult(4)(plans.size) - val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] + assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) - val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).primaryKeyOption.isEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).partitionColumns.isEmpty) - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) - assertResult(6)(addFiles.size) - assertResult(600572)(addFiles.map(_.rows).sum) + val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).primaryKeyOption.isEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).partitionColumns.isEmpty) + val addFiles = + fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) + assertResult(6)(addFiles.size) + assertResult(600572)(addFiles.map(_.rows).sum) - // GLUTEN-5060: check the unnecessary FilterExec - val wholeStageTransformer = plans(2).asInstanceOf[WholeStageTransformer] - val planNodeJson = wholeStageTransformer.substraitPlanJson - assert( - !planNodeJson - .replaceAll("\n", "") - .replaceAll(" ", "") - .contains("\"input\":{\"filter\":{")) + // GLUTEN-5060: check the unnecessary FilterExec + val wholeStageTransformer = plans(2).asInstanceOf[WholeStageTransformer] + val planNodeJson = wholeStageTransformer.substraitPlanJson + assert( + !planNodeJson + .replaceAll("\n", "") + .replaceAll(" ", "") + .contains("\"input\":{\"filter\":{")) + } } - } test("test mergetree insert overwrite") { diff --git a/cpp-ch/local-engine/Common/BlockIterator.cpp b/cpp-ch/local-engine/Common/BlockIterator.cpp index 464701893207..150b5c908c51 100644 --- a/cpp-ch/local-engine/Common/BlockIterator.cpp +++ b/cpp-ch/local-engine/Common/BlockIterator.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes namespace local_engine { -void local_engine::BlockIterator::checkNextValid() +void local_engine::BlockIterator::checkNextValid() const { if (consumed) { diff --git a/cpp-ch/local-engine/Common/BlockIterator.h b/cpp-ch/local-engine/Common/BlockIterator.h index a57935665e3e..4b9d6abcc30e 100644 --- a/cpp-ch/local-engine/Common/BlockIterator.h +++ b/cpp-ch/local-engine/Common/BlockIterator.h @@ -23,7 +23,7 @@ namespace local_engine class BlockIterator { protected: - void checkNextValid(); + void checkNextValid() const; // make current block available void produce(); // consume current block diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 369dceb61afc..d253de82938f 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -583,7 +583,7 @@ std::vector BackendInitializerUtil::wrapDiskPathConfig( return changed_paths; } -DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const std::map & spark_conf_map) +DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const SparkConfigs::ConfigMap & spark_conf_map) { DB::Context::ConfigurationPtr config; @@ -618,7 +618,7 @@ DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const std::map< if (spark_conf_map.contains(GLUTEN_TASK_OFFHEAP)) config->setString(MemoryConfig::CH_TASK_MEMORY, spark_conf_map.at(GLUTEN_TASK_OFFHEAP)); - const bool use_current_directory_as_tmp = config->getBool("use_current_directory_as_tmp", false); + const bool use_current_directory_as_tmp = config->getBool(PathConfig::USE_CURRENT_DIRECTORY_AS_TMP, false); char buffer[PATH_MAX]; if (use_current_directory_as_tmp && getcwd(buffer, sizeof(buffer)) != nullptr) wrapDiskPathConfig(String(buffer), "", *config); @@ -636,7 +636,7 @@ DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const std::map< return config; } -String BackendInitializerUtil::tryGetConfigFile(const std::map & spark_conf_map) +String BackendInitializerUtil::tryGetConfigFile(const SparkConfigs::ConfigMap & spark_conf_map) { if (spark_conf_map.contains(CH_RUNTIME_CONFIG_FILE)) return spark_conf_map.at(CH_RUNTIME_CONFIG_FILE); @@ -701,7 +701,7 @@ DB::Field BackendInitializerUtil::toField(const String & key, const String & val return DB::Field(value); } -void BackendInitializerUtil::initSettings(const std::map & spark_conf_map, DB::Settings & settings) +void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_conf_map, DB::Settings & settings) { /// Initialize default setting. settings.set("date_time_input_format", "best_effort"); @@ -811,17 +811,15 @@ void BackendInitializerUtil::initContexts(DB::Context::ConfigurationPtr config) global_context->makeGlobalContext(); global_context->setConfig(config); - auto getDefaultPath = [config] -> auto + auto tmp_path = config->getString("tmp_path", PathConfig::DEFAULT_TEMP_FILE_PATH); + if(config->getBool(PathConfig::USE_CURRENT_DIRECTORY_AS_TMP, false)) { - const bool use_current_directory_as_tmp = config->getBool("use_current_directory_as_tmp", false); char buffer[PATH_MAX]; - if (use_current_directory_as_tmp && getcwd(buffer, sizeof(buffer)) != nullptr) - return std::string(buffer) + "/tmp/libch"; - else - return std::string("/tmp/libch"); + if (getcwd(buffer, sizeof(buffer)) != nullptr) + tmp_path = std::string(buffer) + tmp_path; }; - global_context->setTemporaryStoragePath(config->getString("tmp_path", getDefaultPath()), 0); + global_context->setTemporaryStoragePath(tmp_path, 0); global_context->setPath(config->getString("path", "/")); String uncompressed_cache_policy = config->getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY); @@ -931,7 +929,7 @@ void BackendInitializerUtil::initCompiledExpressionCache(DB::Context::Configurat #endif } -void BackendInitializerUtil::initBackend(const std::map & spark_conf_map) +void BackendInitializerUtil::initBackend(const SparkConfigs::ConfigMap & spark_conf_map) { DB::Context::ConfigurationPtr config = initConfig(spark_conf_map); diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index c3d067f245f0..cff69090ee31 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -27,6 +27,7 @@ #include #include #include +#include namespace DB { @@ -158,8 +159,8 @@ class BackendInitializerUtil /// Initialize two kinds of resources /// 1. global level resources like global_context/shared_context, notice that they can only be initialized once in process lifetime /// 2. session level resources like settings/configs, they can be initialized multiple times following the lifetime of executor/driver - static void initBackend(const std::map & spark_conf_map); - static void initSettings(const std::map & spark_conf_map, DB::Settings & settings); + static void initBackend(const SparkConfigs::ConfigMap & spark_conf_map); + static void initSettings(const SparkConfigs::ConfigMap & spark_conf_map, DB::Settings & settings); inline static const String CH_BACKEND_PREFIX = "spark.gluten.sql.columnar.backend.ch"; @@ -199,8 +200,8 @@ class BackendInitializerUtil friend class BackendFinalizerUtil; friend class JNIUtils; - static DB::Context::ConfigurationPtr initConfig(const std::map & spark_conf_map); - static String tryGetConfigFile(const std::map & spark_conf_map); + static DB::Context::ConfigurationPtr initConfig(const SparkConfigs::ConfigMap & spark_conf_map); + static String tryGetConfigFile(const SparkConfigs::ConfigMap & spark_conf_map); static void initLoggers(DB::Context::ConfigurationPtr config); static void initEnvs(DB::Context::ConfigurationPtr config); diff --git a/cpp-ch/local-engine/Common/GlutenConfig.cpp b/cpp-ch/local-engine/Common/GlutenConfig.cpp index eb6c2dcab622..0cefbc383977 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.cpp +++ b/cpp-ch/local-engine/Common/GlutenConfig.cpp @@ -30,28 +30,26 @@ namespace local_engine void SparkConfigs::updateConfig(const DB::ContextMutablePtr & context, std::string_view plan) { - std::map spark_conf_map = load(plan); - // configs cannot be updated per query - // settings can be updated per query - auto settings = context->getSettingsCopy(); // make a copy - - // TODO: Remove BackendInitializerUtil::initSettings - BackendInitializerUtil::initSettings(spark_conf_map, settings); - context->setSettings(settings); + update( + plan, + [&](const ConfigMap & spark_conf_map) + { + // configs cannot be updated per query + // settings can be updated per query + auto settings = context->getSettingsCopy(); // make a copy + + // TODO: Remove BackendInitializerUtil::initSettings + BackendInitializerUtil::initSettings(spark_conf_map, settings); + context->setSettings(settings); + }); } -std::map SparkConfigs::load(std::string_view plan, bool processStart) +void SparkConfigs::update(std::string_view plan, const std::function & callback, bool processStart) { - std::map configs{}; auto configMaps = local_engine::BinaryToMessage(plan); - if (!processStart) debug::dumpMessage(configMaps, "Update Config Map Plan"); - - for (const auto & pair : configMaps.configs()) - configs.emplace(pair.first, pair.second); - - return configs; + callback(configMaps.configs()); } MemoryConfig MemoryConfig::loadFromContext(const DB::ContextPtr & context) diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h index 11220afb4878..8af83329b6b7 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.h +++ b/cpp-ch/local-engine/Common/GlutenConfig.h @@ -19,6 +19,7 @@ #include #include +#include namespace Poco::Util { @@ -33,8 +34,9 @@ namespace local_engine struct SparkConfigs { + using ConfigMap = google::protobuf::Map; static void updateConfig(const DB::ContextMutablePtr &, std::string_view); - static std::map load(std::string_view plan, bool processStart = false); + static void update(std::string_view plan, const std::function & callback, bool processStart = false); }; struct MemoryConfig @@ -151,4 +153,10 @@ struct MergeTreeCacheConfig static MergeTreeCacheConfig loadFromContext(const DB::ContextPtr & context); }; + +namespace PathConfig +{ +inline constexpr const char * USE_CURRENT_DIRECTORY_AS_TMP = "use_current_directory_as_tmp"; +inline constexpr const char * DEFAULT_TEMP_FILE_PATH = "/tmp/libch"; +}; } diff --git a/cpp-ch/local-engine/Common/GlutenSettings.cpp b/cpp-ch/local-engine/Common/GlutenSettings.cpp index ceac59ae1fc3..ab015b56362d 100644 --- a/cpp-ch/local-engine/Common/GlutenSettings.cpp +++ b/cpp-ch/local-engine/Common/GlutenSettings.cpp @@ -41,8 +41,12 @@ bool settingsEqual(const DB::Settings & settings, std::string_view name, const s } void updateSettings(const DB::ContextMutablePtr & context, std::string_view plan) { - std::map spark_conf_map = SparkConfigs::load(plan); - for (const auto & [key, value] : spark_conf_map) - context->setSetting(key, value); + SparkConfigs::update( + plan, + [&](const SparkConfigs::ConfigMap & config_map) + { + for (const auto & [key, value] : config_map) + context->setSetting(key, value); + }); } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/GlutenSettings.h b/cpp-ch/local-engine/Common/GlutenSettings.h index 74bb0bb23d50..99c01368b42b 100644 --- a/cpp-ch/local-engine/Common/GlutenSettings.h +++ b/cpp-ch/local-engine/Common/GlutenSettings.h @@ -27,41 +27,41 @@ namespace local_engine #define SKIP_ALIAS(ALIAS_NAME) -#define DECLARE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) TYPE NAME{DEFAULT}; +#define DECLARE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION) TYPE NAME{DEFAULT}; #define GLUTEN_SETTING_STATIC_MEMBER_(NAME) s_##NAME##_ -#define INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) \ - static constexpr std::string_view GLUTEN_SETTING_STATIC_MEMBER_(NAME) = "g." #UNIQ "." #NAME; +#define INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ + static constexpr std::string_view GLUTEN_SETTING_STATIC_MEMBER_(NAME) = "gluten." #NAME; #define DECLARE_GLUTEN_SETTINGS(SETTINGS_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ struct SETTINGS_CLASS_NAME \ { \ - LIST_OF_SETTINGS_MACRO(DECLARE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS, _) \ + LIST_OF_SETTINGS_MACRO(DECLARE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS) \ static SETTINGS_CLASS_NAME get(const DB::ContextPtr & context); \ void set(const DB::ContextMutablePtr & context) const; \ \ private: \ - LIST_OF_SETTINGS_MACRO(INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS, __COUNTER__) \ + LIST_OF_SETTINGS_MACRO(INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS) \ }; -#define IMPLEMENT_GLUTEN_GET_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) \ +#define IMPLEMENT_GLUTEN_GET_(TYPE, NAME, DEFAULT, DESCRIPTION) \ if (DB::Field field_##NAME; settings.tryGet(GLUTEN_SETTING_STATIC_MEMBER_(NAME), field_##NAME)) \ result.NAME = field_##NAME.safeGet(); -#define IMPLEMENT_GLUTEN_SET_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) context->setSetting(GLUTEN_SETTING_STATIC_MEMBER_(NAME), NAME); +#define IMPLEMENT_GLUTEN_SET_(TYPE, NAME, DEFAULT, DESCRIPTION) context->setSetting(GLUTEN_SETTING_STATIC_MEMBER_(NAME), NAME); #define IMPLEMENT_GLUTEN_SETTINGS(SETTINGS_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ SETTINGS_CLASS_NAME SETTINGS_CLASS_NAME::get(const DB::ContextPtr & context) \ { \ SETTINGS_CLASS_NAME result; \ const DB::Settings & settings = context->getSettingsRef(); \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_GET_, SKIP_ALIAS, _) \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_GET_, SKIP_ALIAS) \ return result; \ } \ void SETTINGS_CLASS_NAME::SETTINGS_CLASS_NAME::set(const DB::ContextMutablePtr & context) const \ { \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_SET_, SKIP_ALIAS, _) \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_SET_, SKIP_ALIAS) \ } // workaround for tryGetString diff --git a/cpp-ch/local-engine/Parser/LocalExecutor.cpp b/cpp-ch/local-engine/Parser/LocalExecutor.cpp index ebbf5064c92d..6ef860aff92d 100644 --- a/cpp-ch/local-engine/Parser/LocalExecutor.cpp +++ b/cpp-ch/local-engine/Parser/LocalExecutor.cpp @@ -72,7 +72,7 @@ bool LocalExecutor::hasNext() return true; } -bool LocalExecutor::fallbackMode() +bool LocalExecutor::fallbackMode() const { return executor.get() || fallback_mode; } @@ -110,7 +110,7 @@ Block * LocalExecutor::nextColumnar() return columnar_batch; } -void LocalExecutor::cancel() +void LocalExecutor::cancel() const { if (executor) executor->cancel(); @@ -118,7 +118,7 @@ void LocalExecutor::cancel() push_executor->cancel(); } -void LocalExecutor::setSinks(std::function setter) +void LocalExecutor::setSinks(const std::function & setter) const { setter(*query_pipeline_builder); } @@ -137,7 +137,7 @@ Block LocalExecutor::getHeader() LocalExecutor::LocalExecutor(QueryPlanPtr query_plan, QueryPipelineBuilderPtr pipeline_builder, bool dump_pipeline_) : query_pipeline_builder(std::move(pipeline_builder)) - , header(query_plan->getCurrentHeader().cloneEmpty()) + , header(query_pipeline_builder->getHeader().cloneEmpty()) , dump_pipeline(dump_pipeline_) , ch_column_to_spark_row(std::make_unique()) , current_query_plan(std::move(query_plan)) diff --git a/cpp-ch/local-engine/Parser/LocalExecutor.h b/cpp-ch/local-engine/Parser/LocalExecutor.h index cce6cb20a227..ea1445c605e1 100644 --- a/cpp-ch/local-engine/Parser/LocalExecutor.h +++ b/cpp-ch/local-engine/Parser/LocalExecutor.h @@ -49,11 +49,11 @@ class LocalExecutor : public BlockIterator DB::Block * nextColumnar(); bool hasNext(); - bool fallbackMode(); + bool fallbackMode() const; /// Stop execution, used when task receives shutdown command or executor receives SIGTERM signal - void cancel(); - void setSinks(std::function setter); + void cancel() const; + void setSinks(const std::function & setter) const; void execute(); DB::Block getHeader(); RelMetricPtr getMetric() const { return metric; } diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp index 569e736b3890..ea1239a8093a 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp @@ -21,18 +21,31 @@ #include #include #include +#include #include +#include #include +#include +#include +#include #include #include #include #include -#include +#include #include +namespace DB::Setting +{ +extern const SettingsUInt64 min_insert_block_size_rows; +extern const SettingsUInt64 min_insert_block_size_bytes; +} + using namespace local_engine; using namespace DB; +namespace +{ DB::ProcessorPtr make_sink( const DB::ContextPtr & context, const DB::Names & partition_by, @@ -41,33 +54,26 @@ DB::ProcessorPtr make_sink( const std::string & base_path, const std::string & filename, const std::string & format_hint, - const std::shared_ptr & stats) + const std::shared_ptr & stats) { if (partition_by.empty()) { - auto file_sink = std::make_shared(context, base_path, "", filename, format_hint, input_header); + auto file_sink = std::make_shared(context, base_path, "", filename, format_hint, input_header); file_sink->setStats(stats); return file_sink; } - auto file_sink = std::make_shared( + auto file_sink = std::make_shared( context, partition_by, input_header, output_header, base_path, filename, format_hint); file_sink->setStats(stats); return file_sink; } -bool need_fix_tuple(const DB::DataTypePtr & input, const DB::DataTypePtr & output) -{ - const auto original = typeid_cast(input.get()); - const auto output_type = typeid_cast(output.get()); - return original != nullptr && output_type != nullptr && !original->equals(*output_type); -} - DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB::Block & output) { DB::NamesWithAliases aliases; - for (auto ouput_name = output.begin(), input_iter = input.begin(); ouput_name != output.end(); ++ouput_name, ++input_iter) - aliases.emplace_back(DB::NameWithAlias(input_iter->name, ouput_name->name)); + for (auto output_name = output.begin(), input_iter = input.begin(); output_name != output.end(); ++output_name, ++input_iter) + aliases.emplace_back(DB::NameWithAlias(input_iter->name, output_name->name)); ActionsDAG actions_dag{blockToNameAndTypeList(input)}; actions_dag.project(aliases); @@ -138,12 +144,39 @@ void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block }); } -namespace local_engine +void addMergeTreeSinkTransform( + const DB::ContextPtr & context, + const DB::QueryPipelineBuilderPtr & builder, + const MergeTreeTable & merge_tree_table, + const DB::Block & output, + const DB::Names & /*partitionCols*/) { + const DB::Settings & settings = context->getSettingsRef(); + const auto dest_storage = merge_tree_table.getStorage(context->getGlobalContext()); + StorageMetadataPtr metadata_snapshot = dest_storage->getInMemoryMetadataPtr(); + ASTPtr none; + auto sink = dest_storage->write(none, metadata_snapshot, context, false); + Chain chain; + chain.addSink(sink); + const SinkHelper & sink_helper = assert_cast(*sink).sinkHelper(); + // + auto stats = std::make_shared(output, sink_helper); + chain.addSink(stats); + // + chain.addSource(std::make_shared( + output, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + chain.addSource(std::make_shared( + output, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + + builder->addChain(std::move(chain)); +} -IMPLEMENT_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) - -void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder) +void addNormalFileWriterSinkTransform( + const DB::ContextPtr & context, + const DB::QueryPipelineBuilderPtr & builder, + const std::string & format_hint, + const DB::Block & output, + const DB::Names & partitionCols) { GlutenWriteSettings write_settings = GlutenWriteSettings::get(context); @@ -153,24 +186,7 @@ void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel if (write_settings.task_write_filename.empty()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name."); - assert(write_rel.has_named_table()); - const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - google::protobuf::StringValue optimization; - named_table.advanced_extension().optimization().UnpackTo(&optimization); - auto config = local_engine::parse_write_parameter(optimization.value()); - - //TODO : set compression codec according to format - assert(config["isSnappy"] == "1"); - assert(config.contains("format")); - - assert(write_rel.has_table_schema()); - const substrait::NamedStruct & table_schema = write_rel.table_schema(); - auto blockHeader = TypeParser::buildBlockFromNamedStruct(table_schema); - const auto partitionCols = collect_partition_cols(blockHeader, table_schema); - - auto stats = std::make_shared(blockHeader); - - adjust_output(builder, blockHeader); + auto stats = std::make_shared(output); builder->addSimpleTransform( [&](const Block & cur_header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -181,10 +197,10 @@ void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel context, partitionCols, cur_header, - blockHeader, + output, write_settings.task_write_tmp_dir, write_settings.task_write_filename, - config["format"], + format_hint, stats); }); builder->addSimpleTransform( @@ -195,30 +211,40 @@ void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel return stats; }); } +} -std::map parse_write_parameter(const std::string & input) +namespace local_engine { - std::map reuslt; - const std::string prefix = "WriteParameters:"; - const size_t prefix_pos = input.find(prefix); - if (prefix_pos == std::string::npos) - return reuslt; - const size_t start_pos = prefix_pos + prefix.length(); - const size_t end_pos = input.find('\n', start_pos); +IMPLEMENT_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) - if (end_pos == std::string::npos) - return reuslt; +void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder) +{ + assert(write_rel.has_named_table()); + const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - for (const Poco::StringTokenizer tok(input.substr(start_pos, end_pos - start_pos), ";", Poco::StringTokenizer::TOK_TRIM); - const auto & parameter : tok) + local_engine::Write write; + if (!named_table.advanced_extension().optimization().UnpackTo(&write)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to unpack write optimization with local_engine::Write."); + assert(write.has_common()); + const substrait::NamedStruct & table_schema = write_rel.table_schema(); + auto output = TypeParser::buildBlockFromNamedStruct(table_schema); + adjust_output(builder, output); + const auto partitionCols = collect_partition_cols(output, table_schema); + if (write.has_mergetree()) { - const size_t pos = parameter.find('='); - if (pos == std::string::npos) - continue; - reuslt[parameter.substr(0, pos)] = parameter.substr(pos + 1); + local_engine::MergeTreeTable merge_tree_table(write, table_schema); + GlutenWriteSettings write_settings = GlutenWriteSettings::get(context); + if (write_settings.task_write_tmp_dir.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "MergeTree Write Pipeline need inject relative path."); + if (!merge_tree_table.relative_path.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Non empty relative path for MergeTree table in pipeline mode."); + + merge_tree_table.relative_path = write_settings.task_write_tmp_dir; + addMergeTreeSinkTransform(context, builder, merge_tree_table, output, partitionCols); } - return reuslt; + else + addNormalFileWriterSinkTransform(context, builder, write.common().format(), output, partitionCols); } DB::Names collect_partition_cols(const DB::Block & header, const substrait::NamedStruct & struct_) diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h index b083d2f637d2..65676c45380e 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h @@ -40,13 +40,11 @@ namespace local_engine void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder); -/// Visible for UTs -std::map parse_write_parameter(const std::string & input); DB::Names collect_partition_cols(const DB::Block & header, const substrait::NamedStruct & struct_); -#define WRITE_RELATED_SETTINGS(M, ALIAS, UNIQ) \ - M(String, task_write_tmp_dir, , "The temporary directory for writing data", UNIQ) \ - M(String, task_write_filename, , "The filename for writing data", UNIQ) +#define WRITE_RELATED_SETTINGS(M, ALIAS) \ + M(String, task_write_tmp_dir, , "The temporary directory for writing data") \ + M(String, task_write_filename, , "The filename for writing data") DECLARE_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 3b70069ca410..4e49c577bf86 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -218,7 +218,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list auto rel_parser = RelParserFactory::instance().getBuilder(rel.rel_type_case())(parser_context); auto all_input_rels = rel_parser->getInputs(rel); - assert(all_input_rels.size() == 0 || all_input_rels.size() == 1 || all_input_rels.size() == 2); + assert(all_input_rels.empty() || all_input_rels.size() == 1 || all_input_rels.size() == 2); std::vector input_query_plans; rel_stack.push_back(&rel); for (const auto * input_rel : all_input_rels) diff --git a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp index a904fd087677..958421022ba2 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp @@ -253,7 +253,6 @@ void saveFileStatus( std::vector mergeParts( std::vector selected_parts, - std::unordered_map & partition_values, const String & new_part_uuid, SparkStorageMergeTree & storage, const String & partition_dir, @@ -264,13 +263,8 @@ std::vector mergeParts( future_part->assign(std::move(selected_parts)); future_part->part_info = MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION; + future_part->name = partition_dir.empty() ? "" : partition_dir + "/"; - future_part->name = ""; - if(!partition_dir.empty()) - { - future_part->name = partition_dir + "/"; - extractPartitionValues(partition_dir, partition_values); - } if(!bucket_dir.empty()) { future_part->name = future_part->name + bucket_dir + "/"; @@ -294,6 +288,8 @@ std::vector mergeParts( return merged; } +/** TODO: Remove it. + * Extract partition values from partition directory, we implement it in the java */ void extractPartitionValues(const String & partition_dir, std::unordered_map & partition_values) { Poco::StringTokenizer partitions(partition_dir, "/"); diff --git a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h index 503517cc8c7e..827788a35a65 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h @@ -34,11 +34,8 @@ void saveFileStatus( std::vector mergeParts( std::vector selected_parts, - std::unordered_map & partition_values, const String & new_part_uuid, SparkStorageMergeTree & storage, const String & partition_dir, const String & bucket_dir); - -void extractPartitionValues(const String & partition_dir, std::unordered_map & partition_values); } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp index 63c6225a1f94..199bd455085d 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp @@ -137,7 +137,7 @@ void doParseMergeTreeTableString(MergeTreeTable & table, ReadBufferFromString & assertChar('\n', in); String schema; readString(schema, in); - google::protobuf::util::JsonStringToMessage(schema, &table.schema); + table.schema = JsonStringToMessage(schema); assertChar('\n', in); readString(table.order_by_key, in); assertChar('\n', in); @@ -243,27 +243,23 @@ std::shared_ptr MergeTreeTable::buildMetaData(const return doBuildMetadata(header.getNamesAndTypesList(), context, *this); } -MergeTreeTable::MergeTreeTable(const substrait::WriteRel & write_rel) +MergeTreeTable::MergeTreeTable(const local_engine::Write & write, const substrait::NamedStruct & table_schema) { - assert(write_rel.has_named_table()); - const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - local_engine::Write write_opt; - named_table.advanced_extension().optimization().UnpackTo(&write_opt); - assert(write_opt.has_mergetree()); - const Write_MergeTreeWrite & write = write_opt.mergetree(); - database = write.database(); - table = write.table(); - snapshot_id = write.snapshot_id(); - schema = write_rel.table_schema(); - order_by_key = write.order_by_key(); - low_card_key = write.low_card_key(); - minmax_index_key = write.minmax_index_key(); - bf_index_key = write.bf_index_key(); - set_index_key = write.set_index_key(); - primary_key = write.primary_key(); - relative_path = write.relative_path(); - absolute_path = write.absolute_path(); // always empty, see createNativeWrite in java - table_configs.storage_policy = write.storage_policy(); + assert(write.has_mergetree()); + const Write_MergeTreeWrite & merge_tree_write = write.mergetree(); + database = merge_tree_write.database(); + table = merge_tree_write.table(); + snapshot_id = merge_tree_write.snapshot_id(); + schema = table_schema; + order_by_key = merge_tree_write.order_by_key(); + low_card_key = merge_tree_write.low_card_key(); + minmax_index_key = merge_tree_write.minmax_index_key(); + bf_index_key = merge_tree_write.bf_index_key(); + set_index_key = merge_tree_write.set_index_key(); + primary_key = merge_tree_write.primary_key(); + relative_path = merge_tree_write.relative_path(); + absolute_path = merge_tree_write.absolute_path(); // always empty, see createNativeWrite in java + table_configs.storage_policy = merge_tree_write.storage_policy(); } std::unique_ptr buildMergeTreeSettings(const MergeTreeTableSettings & config) diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h index d015e78b3471..87b2d8403be3 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h @@ -32,6 +32,7 @@ class ReadBufferFromString; } namespace local_engine { +class Write; class SparkStorageMergeTree; using SparkStorageMergeTreePtr = std::shared_ptr; using namespace DB; @@ -79,7 +80,7 @@ struct MergeTreeTable std::shared_ptr buildMetaData(const DB::Block & header, const ContextPtr & context) const; MergeTreeTable() = default; - explicit MergeTreeTable(const substrait::WriteRel & write_rel); + MergeTreeTable(const local_engine::Write & write, const substrait::NamedStruct & table_schema); }; struct MergeTreeTableInstance : MergeTreeTable diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp index 5a66f5933163..43a3a78295d2 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp @@ -134,10 +134,8 @@ void SinkHelper::doMergePartsAsync(const std::vector & for (const auto & prepare_merge_part : prepare_merge_parts) before_size += prepare_merge_part->getBytesOnDisk(); - std::unordered_map partition_values; const auto merged_parts = mergeParts( prepare_merge_parts, - partition_values, toString(UUIDHelpers::generateV4()), dataRef(), write_settings.partition_settings.partition_dir, diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h index 8f02d6fa4e7d..d19e5dc4e969 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h @@ -16,10 +16,12 @@ */ #pragma once +#include #include #include #include #include +#include namespace local_engine { @@ -178,4 +180,85 @@ class SparkMergeTreeSink : public DB::SinkToStorage int part_num = 1; }; + +class MergeTreeStats : public DB::ISimpleTransform +{ + bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks + const SinkHelper & sink_helper; + + static DB::Block statsHeader() + { + return makeBlockHeader( + {{STRING(), "part_name"}, + {STRING(), "partition_id"}, + {BIGINT(), "record_count"}, + {BIGINT(), "marks_count"}, + {BIGINT(), "size_in_bytes"}}); + } + + DB::Chunk final_result() const + { + // TODO: remove it + const std::string NO_PARTITION_ID{"__NO_PARTITION_ID__"}; + + auto parts = sink_helper.unsafeGet(); + + const size_t size = parts.size(); + auto file_col = STRING()->createColumn(); + file_col->reserve(size); + + auto partition_col = STRING()->createColumn(); + partition_col->reserve(size); + + auto countCol = BIGINT()->createColumn(); + countCol->reserve(size); + auto & countColData = static_cast &>(*countCol).getData(); + + auto marksCol = BIGINT()->createColumn(); + marksCol->reserve(size); + auto & marksColData = static_cast &>(*marksCol).getData(); + + auto bytesCol = BIGINT()->createColumn(); + bytesCol->reserve(size); + auto & bytesColData = static_cast &>(*bytesCol).getData(); + + for (const auto & part : parts) + { + file_col->insertData(part->name.c_str(), part->name.size()); + partition_col->insertData(NO_PARTITION_ID.c_str(), NO_PARTITION_ID.size()); + countColData.emplace_back(part->rows_count); + marksColData.emplace_back(part->getMarksCount()); + bytesColData.emplace_back(part->getBytesOnDisk()); + } + const DB::Columns res_columns{ + std::move(file_col), std::move(partition_col), std::move(countCol), std::move(marksCol), std::move(bytesCol)}; + return DB::Chunk(res_columns, size); + } + +public: + explicit MergeTreeStats(const DB::Block & input_header_, const SinkHelper & sink_helper_) + : ISimpleTransform(input_header_, statsHeader(), true), sink_helper(sink_helper_) + { + } + Status prepare() override + { + if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) + { + all_chunks_processed_ = true; + /// return Ready to call transform() for generating filling rows after latest chunk was processed + return Status::Ready; + } + + return ISimpleTransform::prepare(); + } + + String getName() const override { return "MergeTreeStats"; } + void transform(DB::Chunk & chunk) override + { + if (all_chunks_processed_) + chunk = final_result(); + else + chunk = {}; + } +}; } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h index 3554a092a470..1fbbdbe3461d 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h @@ -20,10 +20,10 @@ namespace local_engine { -#define MERGE_TREE_WRITE_RELATED_SETTINGS(M, ALIAS, UNIQ) \ - M(String, part_name_prefix, , "The part name prefix for writing data", UNIQ) \ - M(String, partition_dir, , "The parition directory for writing data", UNIQ) \ - M(String, bucket_dir, , "The bucket directory for writing data", UNIQ) +#define MERGE_TREE_WRITE_RELATED_SETTINGS(M, ALIAS) \ + M(String, part_name_prefix, , "The part name prefix for writing data") \ + M(String, partition_dir, , "The partition directory for writing data") \ + M(String, bucket_dir, , "The bucket directory for writing data") DECLARE_GLUTEN_SETTINGS(SparkMergeTreeWritePartitionSettings, MERGE_TREE_WRITE_RELATED_SETTINGS) diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp index d5c119ef44ba..a8fdfff6ff75 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp @@ -103,13 +103,7 @@ std::string PartInfo::toJson(const std::vector & part_infos) writer.Key("bucket_id"); writer.String(item.bucket_id.c_str()); writer.Key("partition_values"); - writer.StartObject(); - for (const auto & key_value : item.partition_values) - { - writer.Key(key_value.first.c_str()); - writer.String(key_value.second.c_str()); - } - writer.EndObject(); + writer.String(item.partition_values.c_str()); writer.EndObject(); } writer.EndArray(); @@ -117,10 +111,7 @@ std::string PartInfo::toJson(const std::vector & part_infos) } std::unique_ptr SparkMergeTreeWriter::create( - const MergeTreeTable & merge_tree_table, - const SparkMergeTreeWritePartitionSettings & write_settings_, - const DB::ContextMutablePtr & context, - const std::string & spark_job_id) + const MergeTreeTable & merge_tree_table, const DB::ContextMutablePtr & context, const std::string & spark_job_id) { const DB::Settings & settings = context->getSettingsRef(); const auto dest_storage = merge_tree_table.getStorage(context); @@ -130,34 +121,22 @@ std::unique_ptr SparkMergeTreeWriter::create( Chain chain; auto sink = dest_storage->write(none, metadata_snapshot, context, false); chain.addSink(sink); + const SinkHelper & sink_helper = assert_cast(*sink).sinkHelper(); + // + // auto stats = std::make_shared(header, sink_helper); + // chain.addSink(stats); + // chain.addSource(std::make_shared( header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); chain.addSource(std::make_shared( header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); - std::unordered_map partition_values; - if (!write_settings_.partition_dir.empty()) - extractPartitionValues(write_settings_.partition_dir, partition_values); - return std::make_unique( - header, - assert_cast(*sink).sinkHelper(), - QueryPipeline{std::move(chain)}, - std::move(partition_values), - spark_job_id); + return std::make_unique(header, sink_helper, QueryPipeline{std::move(chain)}, spark_job_id); } SparkMergeTreeWriter::SparkMergeTreeWriter( - const DB::Block & header_, - const SinkHelper & sink_helper_, - DB::QueryPipeline && pipeline_, - std::unordered_map && partition_values_, - const std::string & spark_job_id_) - : header{header_} - , sink_helper{sink_helper_} - , pipeline{std::move(pipeline_)} - , executor{pipeline} - , partition_values{partition_values_} - , spark_job_id(spark_job_id_) + const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_) + : header{header_}, sink_helper{sink_helper_}, pipeline{std::move(pipeline_)}, executor{pipeline}, spark_job_id(spark_job_id_) { } @@ -175,7 +154,8 @@ void SparkMergeTreeWriter::close() { executor.finish(); std::string result = PartInfo::toJson(getAllPartInfo()); - SparkMergeTreeWriterJNI::setCurrentTaskWriteInfo(spark_job_id, result); + if (spark_job_id != CPP_UT_JOB_ID) + SparkMergeTreeWriterJNI::setCurrentTaskWriteInfo(spark_job_id, result); } std::vector SparkMergeTreeWriter::getAllPartInfo() const @@ -191,7 +171,7 @@ std::vector SparkMergeTreeWriter::getAllPartInfo() const part->getMarksCount(), part->getBytesOnDisk(), part->rows_count, - partition_values, + sink_helper.write_settings.partition_settings.partition_dir, sink_helper.write_settings.partition_settings.bucket_dir}); } return res; diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h index 58c9a6659169..806963817871 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h @@ -41,7 +41,7 @@ struct PartInfo size_t mark_count; size_t disk_size; size_t row_count; - std::unordered_map partition_values; + String partition_values; String bucket_id; bool operator<(const PartInfo & rhs) const { return disk_size < rhs.disk_size; } @@ -52,29 +52,23 @@ struct PartInfo class SparkMergeTreeWriter : public NativeOutputWriter { public: - static std::unique_ptr create( - const MergeTreeTable & merge_tree_table, - const SparkMergeTreeWritePartitionSettings & write_settings_, - const DB::ContextMutablePtr & context, - const std::string & spark_job_id); + static std::unique_ptr + create(const MergeTreeTable & merge_tree_table, const DB::ContextMutablePtr & context, const std::string & spark_job_id); SparkMergeTreeWriter( - const DB::Block & header_, - const SinkHelper & sink_helper_, - DB::QueryPipeline && pipeline_, - std::unordered_map && partition_values_, - const std::string & spark_job_id_); + const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_); void write(DB::Block & block) override; void close() override; + /// visible for UTs + static constexpr auto CPP_UT_JOB_ID = "__UT_JOB_ID__"; + private: DB::Block header; const SinkHelper & sink_helper; DB::QueryPipeline pipeline; DB::PushingPipelineExecutor executor; - std::unordered_map partition_values; - const std::string spark_job_id; std::vector getAllPartInfo() const; diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index d44873196b93..a6a252be3d63 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -160,12 +160,12 @@ SparkStorageMergeTree::SparkStorageMergeTree( std::atomic SparkStorageMergeTree::part_num; -void SparkStorageMergeTree::prefetchPartDataFile(const std::unordered_set& parts) const +void SparkStorageMergeTree::prefetchPartDataFile(const std::unordered_set & parts) const { prefetchPartFiles(parts, CompactObjectStorageDiskTransaction::PART_DATA_FILE_NAME); } -void SparkStorageMergeTree::prefetchPartFiles(const std::unordered_set& parts, String file_name) const +void SparkStorageMergeTree::prefetchPartFiles(const std::unordered_set & parts, String file_name) const { auto disk = getDisks().front(); if (!disk->isRemote()) @@ -185,7 +185,7 @@ void SparkStorageMergeTree::prefetchPartFiles(const std::unordered_set& parts) const +void SparkStorageMergeTree::prefetchMetaDataFile(const std::unordered_set & parts) const { prefetchPartFiles(parts, CompactObjectStorageDiskTransaction::PART_META_FILE_NAME); } @@ -514,6 +514,8 @@ SinkToStoragePtr SparkWriteStorageMergeTree::write( const ASTPtr &, const StorageMetadataPtr & /*storage_in_memory_metadata*/, ContextPtr context, bool /*async_insert*/) { SparkMergeTreeWriteSettings settings{.partition_settings{SparkMergeTreeWritePartitionSettings::get(context)}}; + if (settings.partition_settings.part_name_prefix.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "empty part_name_prefix is not allowed."); settings.load(context); SinkHelperPtr sink_helper = SparkMergeTreeSink::create(table, settings, getContext()); #ifndef NDEBUG diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index ba918928607c..8a25d7b21fa6 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -67,7 +67,7 @@ class WriteStats : public DB::ISimpleTransform DB::Arena partition_keys_arena_; std::string filename_; - absl::flat_hash_map fiel_to_count_; + absl::flat_hash_map file_to_count_; static DB::Block statsHeader() { @@ -76,18 +76,24 @@ class WriteStats : public DB::ISimpleTransform DB::Chunk final_result() const { - ///TODO: improve performance + const size_t size = file_to_count_.size(); + auto file_col = STRING()->createColumn(); + file_col->reserve(size); auto partition_col = STRING()->createColumn(); + partition_col->reserve(size); auto countCol = BIGINT()->createColumn(); + countCol->reserve(size); + auto & countColData = static_cast &>(*countCol).getData(); + UInt64 num_rows = 0; - for (const auto & [relative_path, rows] : fiel_to_count_) + for (const auto & [relative_path, rows] : file_to_count_) { if (rows == 0) continue; file_col->insertData(filename_.c_str(), filename_.size()); partition_col->insertData(relative_path.data, relative_path.size); - countCol->insert(rows); + countColData.emplace_back(rows); num_rows++; } @@ -130,12 +136,12 @@ class WriteStats : public DB::ISimpleTransform if (patition_id.empty()) return; - fiel_to_count_.emplace(copyStringInArena(partition_keys_arena_, patition_id), 0); + file_to_count_.emplace(copyStringInArena(partition_keys_arena_, patition_id), 0); } void collectStats(const String & file_path, size_t rows) { - if (const auto it = fiel_to_count_.find(file_path); it != fiel_to_count_.end()) + if (const auto it = file_to_count_.find(file_path); it != file_to_count_.end()) { it->second += rows; return; @@ -233,7 +239,7 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink private: const std::string base_path_; - const std::string filenmame_; + const std::string filename_; DB::ContextPtr context_; const DB::Block sample_block_; const std::string format_hint_; @@ -250,7 +256,7 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink const std::string & format_hint) : PartitionedSink(make_partition_expression(partition_by), context, input_header) , base_path_(base_path) - , filenmame_(filename) + , filename_(filename) , context_(context) , sample_block_(sample_block) , format_hint_(format_hint) @@ -259,9 +265,9 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink DB::SinkPtr createSinkForPartition(const String & partition_id) override { assert(stats_); - const auto partition_path = fmt::format("{}/{}", partition_id, filenmame_); + const auto partition_path = fmt::format("{}/{}", partition_id, filename_); PartitionedSink::validatePartitionKey(partition_path, true); - auto file_sink = std::make_shared(context_, base_path_, partition_id, filenmame_, format_hint_, sample_block_); + auto file_sink = std::make_shared(context_, base_path_, partition_id, filename_, format_hint_, sample_block_); file_sink->setStats(stats_); return file_sink; } diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 952f6e50e94d..24c5950065ea 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -205,8 +205,11 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_n LOCAL_ENGINE_JNI_METHOD_START const auto conf_plan_a = local_engine::getByteArrayElementsSafe(env, conf_plan); const std::string::size_type plan_buf_size = conf_plan_a.length(); - local_engine::BackendInitializerUtil::initBackend( - local_engine::SparkConfigs::load({reinterpret_cast(conf_plan_a.elems()), plan_buf_size}, true)); + local_engine::SparkConfigs::update( + {reinterpret_cast(conf_plan_a.elems()), plan_buf_size}, + [&](const local_engine::SparkConfigs::ConfigMap & spark_conf_map) + { local_engine::BackendInitializerUtil::initBackend(spark_conf_map); }, + true); LOCAL_ENGINE_JNI_METHOD_END(env, ) } @@ -217,25 +220,6 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_n LOCAL_ENGINE_JNI_METHOD_END(env, ) } -/// TODO: remvoe this method -JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_injectWriteFilesTempPath( - JNIEnv * env, jclass, jbyteArray temp_path, jbyteArray filename) -{ - LOCAL_ENGINE_JNI_METHOD_START - const auto query_context = local_engine::QueryContext::instance().currentQueryContext(); - - const auto path_array = local_engine::getByteArrayElementsSafe(env, temp_path); - const auto filename_array = local_engine::getByteArrayElementsSafe(env, filename); - - local_engine::GlutenWriteSettings settings{ - .task_write_tmp_dir = {reinterpret_cast(path_array.elems()), static_cast(path_array.length())}, - .task_write_filename = {reinterpret_cast(filename_array.elems()), static_cast(filename_array.length())}, - }; - settings.set(query_context); - - LOCAL_ENGINE_JNI_METHOD_END(env, ) -} - /// Set settings for the current query. It assumes that all parameters are started with `CH_RUNTIME_SETTINGS_PREFIX` prefix, /// and the prefix is removed by java before passing to C++. JNIEXPORT void @@ -824,14 +808,14 @@ JNIEXPORT jobject Java_org_apache_gluten_vectorized_CHBlockConverterJniWrapper_c spark_row_info = converter.convertCHColumnToSparkRow(*block, mask); auto * offsets_arr = env->NewLongArray(spark_row_info->getNumRows()); - const auto * offsets_src = reinterpret_cast(spark_row_info->getOffsets().data()); + const auto * offsets_src = spark_row_info->getOffsets().data(); env->SetLongArrayRegion(offsets_arr, 0, spark_row_info->getNumRows(), offsets_src); auto * lengths_arr = env->NewLongArray(spark_row_info->getNumRows()); - const auto * lengths_src = reinterpret_cast(spark_row_info->getLengths().data()); + const auto * lengths_src = spark_row_info->getLengths().data(); env->SetLongArrayRegion(lengths_arr, 0, spark_row_info->getNumRows(), lengths_src); int64_t address = reinterpret_cast(spark_row_info->getBufferAddress()); - int64_t column_number = reinterpret_cast(spark_row_info->getNumCols()); - int64_t total_size = reinterpret_cast(spark_row_info->getTotalBytes()); + int64_t column_number = spark_row_info->getNumCols(); + int64_t total_size = spark_row_info->getTotalBytes(); jobject spark_row_info_object = env->NewObject(spark_row_info_class, spark_row_info_constructor, offsets_arr, lengths_arr, address, column_number, total_size); @@ -953,38 +937,30 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW } JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniWrapper_createMergeTreeWriter( - JNIEnv * env, jobject, jstring prefix, jstring partition, jstring bucket, jbyteArray writeRel, jbyteArray conf_plan) + JNIEnv * env, jobject, jbyteArray writeRel, jbyteArray conf_plan) { LOCAL_ENGINE_JNI_METHOD_START auto query_context = local_engine::QueryContext::instance().currentQueryContext(); - // by task update new configs ( in case of dynamic config update ) + // by task update new configs (in case of dynamic config update) const auto conf_plan_a = local_engine::getByteArrayElementsSafe(env, conf_plan); local_engine::SparkConfigs::updateConfig( query_context, {reinterpret_cast(conf_plan_a.elems()), static_cast(conf_plan_a.length())}); - const auto part_dir_prefix = jstring2string(env, prefix); - const auto partition_ = jstring2string(env, partition); - const auto bucket_ = jstring2string(env, bucket); - - local_engine::SparkMergeTreeWritePartitionSettings settings{ - .part_name_prefix{part_dir_prefix}, .partition_dir{partition_}, .bucket_dir{bucket_}}; - settings.set(query_context); - const auto writeRelBytes = local_engine::getByteArrayElementsSafe(env, writeRel); substrait::WriteRel write_rel = local_engine::BinaryToMessage( {reinterpret_cast(writeRelBytes.elems()), static_cast(writeRelBytes.length())}); - local_engine::MergeTreeTable merge_tree_table(write_rel); assert(write_rel.has_named_table()); const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - local_engine::Write write_opt; - named_table.advanced_extension().optimization().UnpackTo(&write_opt); - assert(write_opt.has_common()); - auto * writer - = local_engine::SparkMergeTreeWriter::create(merge_tree_table, settings, query_context, write_opt.common().job_task_attempt_id()) - .release(); - - return reinterpret_cast(writer); + local_engine::Write write; + if (!named_table.advanced_extension().optimization().UnpackTo(&write)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to unpack write optimization with local_engine::Write."); + assert(write.has_common()); + assert(write.has_mergetree()); + local_engine::MergeTreeTable merge_tree_table(write, write_rel.table_schema()); + const std::string & id = write.common().job_task_attempt_id(); + + return reinterpret_cast(local_engine::SparkMergeTreeWriter::create(merge_tree_table, query_context, id).release()); LOCAL_ENGINE_JNI_METHOD_END(env, 0) } @@ -1058,16 +1034,15 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn std::vector selected_parts = local_engine::StorageMergeTreeFactory::getDataPartsByNames(temp_storage->getStorageID(), "", merge_tree_table.getPartNames()); - std::unordered_map partition_values; std::vector loaded - = local_engine::mergeParts(selected_parts, partition_values, uuid_str, *temp_storage, partition_dir, bucket_dir); + = local_engine::mergeParts(selected_parts, uuid_str, *temp_storage, partition_dir, bucket_dir); std::vector res; for (auto & partPtr : loaded) { saveFileStatus(*temp_storage, context, partPtr->name, const_cast(partPtr->getDataPartStorage())); res.emplace_back(local_engine::PartInfo{ - partPtr->name, partPtr->getMarksCount(), partPtr->getBytesOnDisk(), partPtr->rows_count, partition_values, bucket_dir}); + partPtr->name, partPtr->getMarksCount(), partPtr->getBytesOnDisk(), partPtr->rows_count, partition_dir, bucket_dir}); } auto json_info = local_engine::PartInfo::toJson(res); diff --git a/cpp-ch/local-engine/tests/gluten_test_util.cpp b/cpp-ch/local-engine/tests/gluten_test_util.cpp index 7d0961094363..455abdb62e41 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.cpp +++ b/cpp-ch/local-engine/tests/gluten_test_util.cpp @@ -22,8 +22,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -76,10 +76,9 @@ std::optional parseFilter(const std::string & filter, const AnotherR return ActionsDAG::buildFilterActionsDAG({visitor_data.getActions().getOutputs().back()}, node_name_to_input_column); } -std::pair> create_plan_and_executor( - std::string_view json_plan, std::string_view split_template, std::string_view file, const std::optional & context) +std::pair> +create_plan_and_executor(std::string_view json_plan, std::string_view split, const std::optional & context) { - const std::string split = replaceLocalFilesWildcards(split_template, file); const auto plan = local_engine::JsonStringToMessage(json_plan); auto parser_context = ParserContext::build(context.value_or(QueryContext::globalContext()), plan); SerializedPlanParser parser(parser_context); @@ -87,6 +86,13 @@ std::pair> create_plan_and_execu return {plan, parser.createExecutor(plan)}; } +std::pair> create_plan_and_executor( + std::string_view json_plan, std::string_view split_template, std::string_view file, const std::optional & context) +{ + const std::string split = replaceLocalFilesWildcards(split_template, file); + return create_plan_and_executor(json_plan, split, context); +} + const char * get_data_dir() { const auto * const result = std::getenv("PARQUET_TEST_DATA"); diff --git a/cpp-ch/local-engine/tests/gluten_test_util.h b/cpp-ch/local-engine/tests/gluten_test_util.h index a9d8af37b47a..799a6d7967dc 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.h +++ b/cpp-ch/local-engine/tests/gluten_test_util.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include #include @@ -71,6 +72,9 @@ AnotherRowType readParquetSchema(const std::string & file); std::optional parseFilter(const std::string & filter, const AnotherRowType & name_and_types); +std::pair> +create_plan_and_executor(std::string_view json_plan, std::string_view split, const std::optional & context = std::nullopt); + std::pair> create_plan_and_executor( std::string_view json_plan, std::string_view split_template, @@ -85,6 +89,13 @@ inline std::string replaceLocalFilesWildcards(const std::string_view haystack, c return boost::replace_all_copy(std::string{haystack}, wildcard, replaced); } +inline std::string replaceLocalFilesWithTPCH(const std::string_view haystack) +{ + static constexpr auto wildcard = "{replace_GLUTEN_SOURCE_TPCH_DIR}"; + constexpr std::string_view replaced = GLUTEN_SOURCE_TPCH_DIR(""); + return boost::replace_all_copy(std::string{haystack}, wildcard, replaced); +} + inline BlockFieldType toBlockFieldType(const AnotherFieldType & type) { return BlockFieldType(type.type, type.name); diff --git a/cpp-ch/local-engine/tests/gtest_local_engine.cpp b/cpp-ch/local-engine/tests/gtest_local_engine.cpp index 973f2fa6192f..5f9b6f280e58 100644 --- a/cpp-ch/local-engine/tests/gtest_local_engine.cpp +++ b/cpp-ch/local-engine/tests/gtest_local_engine.cpp @@ -94,8 +94,10 @@ void registerOutputFormatParquet(DB::FormatFactory & factory); int main(int argc, char ** argv) { - BackendInitializerUtil::initBackend( - SparkConfigs::load(local_engine::JsonStringToBinary(EMBEDDED_PLAN(_config_json)), true)); + SparkConfigs::update( + local_engine::JsonStringToBinary(EMBEDDED_PLAN(_config_json)), + [&](const SparkConfigs::ConfigMap & spark_conf_map) { BackendInitializerUtil::initBackend(spark_conf_map); }, + true); auto & factory = FormatFactory::instance(); DB::registerOutputFormatParquet(factory); diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp index 233456992dff..9f0515d3d384 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -138,14 +139,6 @@ TEST(WritePipeline, SubstraitFileSink) EXPECT_TRUE(write_rel.has_named_table()); const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - - google::protobuf::StringValue optimization; - named_table.advanced_extension().optimization().UnpackTo(&optimization); - auto config = local_engine::parse_write_parameter(optimization.value()); - EXPECT_EQ(2, config.size()); - EXPECT_EQ("parquet", config["format"]); - EXPECT_EQ("1", config["isSnappy"]); - EXPECT_TRUE(write_rel.has_table_schema()); const substrait::NamedStruct & table_schema = write_rel.table_schema(); auto block = TypeParser::buildBlockFromNamedStruct(table_schema); @@ -194,15 +187,6 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) EXPECT_TRUE(write_rel.has_named_table()); const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - - google::protobuf::StringValue optimization; - named_table.advanced_extension().optimization().UnpackTo(&optimization); - auto config = local_engine::parse_write_parameter(optimization.value()); - EXPECT_EQ(2, config.size()); - EXPECT_EQ("parquet", config["format"]); - EXPECT_EQ("1", config["isSnappy"]); - - EXPECT_TRUE(write_rel.has_table_schema()); const substrait::NamedStruct & table_schema = write_rel.table_schema(); auto block = TypeParser::buildBlockFromNamedStruct(table_schema); @@ -222,20 +206,27 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) // EXPECT_EQ(16, col_b.getInt(0)); } +/*DB::ASTPtr printColumn(const std::string& column) +{ + // printf('%05d',col) + DB::ASTs arguments {std::make_shared("%05d"), std::make_shared(column)}; + return DB::makeASTFunction("printf", std::move(arguments)); +}*/ + TEST(WritePipeline, ComputePartitionedExpression) { const auto context = DB::Context::createCopy(QueryContext::globalContext()); auto partition_by = SubstraitPartitionedFileSink::make_partition_expression({"s_nationkey", "name"}); + // auto partition_by = printColumn("s_nationkey"); ASTs arguments(1, partition_by); ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); Block sample_block{{STRING(), "name"}, {UINT(), "s_nationkey"}}; + auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); auto partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); - - auto partition_by_column_name = partition_by_string->getColumnName(); Chunk chunk = testChunk(); diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp index dbff85806147..ddbf113353b1 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -194,7 +195,6 @@ INCBIN(_1_read_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/1_p TEST(MergeTree, SparkMergeTree) { - GTEST_SKIP(); // TODO: fix test ThreadStatus thread_status; const auto context = DB::Context::createCopy(QueryContext::globalContext()); @@ -226,7 +226,7 @@ TEST(MergeTree, SparkMergeTree) }; gm_write_settings.set(context); - auto writer = local_engine::SparkMergeTreeWriter::create(merge_tree_table, gm_write_settings, context, "no job id"); + auto writer = local_engine::SparkMergeTreeWriter::create(merge_tree_table, context, SparkMergeTreeWriter::CPP_UT_JOB_ID); SparkMergeTreeWriter & spark_merge_tree_writer = *writer; auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_1_read_), split_template, file); @@ -255,25 +255,24 @@ TEST(MergeTree, SparkMergeTree) } } -INCBIN(_2_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/2_one_pipeline.json"); +INCBIN(_3_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline.json"); +INCBIN(_3_mergetree_plan_input_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline_input.json"); TEST(MergeTree, Pipeline) { - GTEST_SKIP(); const auto context = DB::Context::createCopy(QueryContext::globalContext()); - GlutenWriteSettings settings{ - .task_write_tmp_dir = "file:///tmp/lineitem_mergetree", - .task_write_filename = "part-00000-a09f9d59-2dc6-43bc-a485-dcab8384b2ff.c000.mergetree", - }; + GlutenWriteSettings settings{.task_write_tmp_dir = "tmp/lineitem_mergetree"}; settings.set(context); + SparkMergeTreeWritePartitionSettings partition_settings{.part_name_prefix = "pipline_prefix"}; + partition_settings.set(context); - constexpr std::string_view split_template - = R"({"items":[{"uriFile":"{replace_local_files}","length":"19230111","parquet":{},"schema":{},"metadataColumns":[{}],"properties":{"fileSize":"19230111","modificationTime":"1722330598029"}}]})"; auto [_, local_executor] = test::create_plan_and_executor( - EMBEDDED_PLAN(_2_mergetree_plan_), - split_template, - GLUTEN_SOURCE_TPCH_DIR("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"), - context); - EXPECT_TRUE(local_executor->hasNext()); - const Block & x = *local_executor->nextColumnar(); + EMBEDDED_PLAN(_3_mergetree_plan_), replaceLocalFilesWithTPCH(EMBEDDED_PLAN(_3_mergetree_plan_input_)), context); + size_t sum = 0; + while (local_executor->hasNext()) + { + const Block & x = *local_executor->nextColumnar(); + EXPECT_EQ(1, x.rows()); + //-debug::headBlock("pipeline write", x); + } } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json index eecb8d7484d2..81f3fdf77241 100644 --- a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json +++ b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json @@ -1,6 +1,6 @@ { "detail": { "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "MergeTree;default\nlineitem_mergetree\n1724764750266_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\nlineitem_mergetree\n\n{\"storage_policy\":\"default\"}\n" + "value": "MergeTree;default\nlineitem_mergetree\n1724764750266_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\n\nlineitem_mergetree\n\n{\"storage_policy\":\"default\"}\n" } } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json index d1ed674596e5..efbd37e11ea5 100644 --- a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json +++ b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json @@ -1,6 +1,6 @@ { "detail": { "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "MergeTree;default\nlineitem_mergetree_hdfs\n1724766584676_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\n3.5/test/lineitem_mergetree_hdfs\n\n{\"storage_policy\":\"__hdfs_main\"}\n" + "value": "MergeTree;default\nlineitem_mergetree_hdfs\n1724766584676_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\n\n3.5/test/lineitem_mergetree_hdfs\n\n{\"storage_policy\":\"__hdfs_main\"}\n" } } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/2_one_pipeline.json b/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline.json similarity index 96% rename from cpp-ch/local-engine/tests/json/mergetree/2_one_pipeline.json rename to cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline.json index fbc593267464..b1de89a63d3a 100644 --- a/cpp-ch/local-engine/tests/json/mergetree/2_one_pipeline.json +++ b/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline.json @@ -7,8 +7,17 @@ "namedTable": { "advancedExtension": { "optimization": { - "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "WriteParameters:isSnappy=1;format=mergetree\n" + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "mergetree" + }, + "mergetree": { + "database": "default", + "table": "lineitem_mergetree", + "snapshotId": "1729651573302_0", + "orderByKey": "tuple()", + "storagePolicy": "default" + } }, "enhancement": { "@type": "type.googleapis.com/substrait.Type", diff --git a/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json b/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json new file mode 100644 index 000000000000..1471adf0a25f --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json @@ -0,0 +1,28 @@ +{ + "items": [ + { + "uriFile": "{replace_GLUTEN_SOURCE_TPCH_DIR}lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet", + "length": "19230111", + "parquet": {}, + "schema": {}, + "metadataColumns": [ + { + "key": "input_file_name", + "value": "{replace_GLUTEN_SOURCE_TPCH_DIR}lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet" + }, + { + "key": "input_file_block_length", + "value": "19230111" + }, + { + "key": "input_file_block_start", + "value": "0" + } + ], + "properties": { + "fileSize": "19230111", + "modificationTime": "1719819166344" + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/native_write_one_partition.json b/cpp-ch/local-engine/tests/json/native_write_one_partition.json index 45b3f60e41fa..d09df9bd9b47 100644 --- a/cpp-ch/local-engine/tests/json/native_write_one_partition.json +++ b/cpp-ch/local-engine/tests/json/native_write_one_partition.json @@ -7,8 +7,11 @@ "namedTable": { "advancedExtension": { "optimization": { - "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "WriteParameters:isSnappy=1;format=parquet\n" + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "parquet" + }, + "parquet": {} }, "enhancement": { "@type": "type.googleapis.com/substrait.Type", diff --git a/cpp-ch/local-engine/tests/json/native_write_plan.json b/cpp-ch/local-engine/tests/json/native_write_plan.json index 8d5ffce7b163..c7142fb0e843 100644 --- a/cpp-ch/local-engine/tests/json/native_write_plan.json +++ b/cpp-ch/local-engine/tests/json/native_write_plan.json @@ -7,8 +7,11 @@ "namedTable": { "advancedExtension": { "optimization": { - "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "WriteParameters:isSnappy=1;format=parquet\n" + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "parquet" + }, + "parquet": {} }, "enhancement": { "@type": "type.googleapis.com/substrait.Type", From 2322962739801ae8e10f88310f4c8033d4d61fd2 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Mon, 11 Nov 2024 15:38:54 +0800 Subject: [PATCH 062/211] [VL] Fix weekly scheduled GHA job (#7888) --- .github/workflows/velox_weekly.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/velox_weekly.yml b/.github/workflows/velox_weekly.yml index 8cd74723f0ae..666f0b033b62 100644 --- a/.github/workflows/velox_weekly.yml +++ b/.github/workflows/velox_weekly.yml @@ -93,8 +93,8 @@ jobs: sudo apt install -y software-properties-common sudo add-apt-repository ppa:ubuntu-toolchain-r/test sudo apt update && sudo apt install -y gcc-11 g++-11 - sudo ln -sf /usr/bin/gcc-11 /usr/bin/gcc - sudo ln -sf /usr/bin/g++-11 /usr/bin/g++ + export CC=/usr/bin/gcc-11 + export CXX=/usr/bin/g++-11 fi sudo apt-get install -y openjdk-8-jdk export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 From b8d9e48bd1edcf3ae5f30bc22cb8a261551294f8 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Mon, 11 Nov 2024 16:41:35 +0800 Subject: [PATCH 063/211] [VL] Enable array test for GlutenParquetIOSuite (#7841) --- .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 7 ------- .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 7 ------- .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 7 ------- 3 files changed, 21 deletions(-) diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 4e8f13ee4414..1fc64577d2a6 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -704,13 +704,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-35640: int as long should throw schema incompatible error") // Velox only support read Timestamp with INT96 for now. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") - // TODO: Unsupported Array schema in Parquet. - .exclude("vectorized reader: optional array with required elements") - .exclude("vectorized reader: required array with required elements") - .exclude("vectorized reader: required array with optional elements") - .exclude("vectorized reader: required array with legacy format") - // add support in native reader - .exclude("SPARK-41096: FIXED_LEN_BYTE_ARRAY support") .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index ec13265f9e85..8c65afd1a3a2 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -688,13 +688,6 @@ class VeloxTestSettings extends BackendTestSettings { // Velox only support read Timestamp with INT96 for now. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") .exclude("Read TimestampNTZ and TimestampLTZ for various logical TIMESTAMP types") - // TODO: Unsupported Array schema in Parquet. - .exclude("vectorized reader: optional array with required elements") - .exclude("vectorized reader: required array with required elements") - .exclude("vectorized reader: required array with optional elements") - .exclude("vectorized reader: required array with legacy format") - // add support in native reader - .exclude("SPARK-41096: FIXED_LEN_BYTE_ARRAY support") .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 876d213b0583..18c46a304282 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -699,13 +699,6 @@ class VeloxTestSettings extends BackendTestSettings { // Velox only support read Timestamp with INT96 for now. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") .exclude("Read TimestampNTZ and TimestampLTZ for various logical TIMESTAMP types") - // TODO: Unsupported Array schema in Parquet. - .exclude("vectorized reader: optional array with required elements") - .exclude("vectorized reader: required array with required elements") - .exclude("vectorized reader: required array with optional elements") - .exclude("vectorized reader: required array with legacy format") - // add support in native reader - .exclude("SPARK-41096: FIXED_LEN_BYTE_ARRAY support") .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] From 2d6d78cd8357a6e3eb31bf6d0939749de99fbfa2 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Mon, 11 Nov 2024 17:09:25 +0800 Subject: [PATCH 064/211] [Core] Add gluten icon (#7894) [Core] Add gluten icon Co-authored-by: liuneng1994 --- .idea/icon.svg | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 .idea/icon.svg diff --git a/.idea/icon.svg b/.idea/icon.svg new file mode 100644 index 000000000000..3a8604285c28 --- /dev/null +++ b/.idea/icon.svg @@ -0,0 +1,7 @@ + + + + Layer 1 + + + \ No newline at end of file From 84d849e46b0d3b390a773413efdc053681edfec4 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Mon, 11 Nov 2024 03:26:01 -0600 Subject: [PATCH 065/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241111) (#7884) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241111) * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/58400 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- .../Parser/scalar_function_parser/arrayExcept.cpp | 3 ++- .../Parser/scalar_function_parser/arrayRemove.cpp | 3 ++- .../Parser/scalar_function_parser/arrayRepeat.cpp | 3 ++- .../Parser/scalar_function_parser/lambdaFunction.cpp | 3 ++- 5 files changed, 10 insertions(+), 6 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 20e3f8828cd6..b5e11ca13f06 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241107 -CH_COMMIT=6528c9d384d \ No newline at end of file +CH_BRANCH=rebase_ch/20241111 +CH_COMMIT=c988219798d \ No newline at end of file diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp index e33d5f8717d1..8857fbbf5df6 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp @@ -81,7 +81,8 @@ class FunctionParserArrayExcept : public FunctionParser captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, - lambda_output->result_name); + lambda_output->result_name, + false); const auto * lambda_function = &actions_dag.addFunction(function_capture, {arr2_not_null}, lambda_output->result_name); // Apply arrayFilter with the lambda function diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp index b4192ed33017..f45544cfa4d1 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp @@ -79,7 +79,8 @@ class FunctionParserArrayRemove : public FunctionParser captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, - lambda_output->result_name); + lambda_output->result_name, + false); const auto * lambda_function = &actions_dag.addFunction(function_capture, {elem_not_null}, lambda_output->result_name); /// Apply arrayFilter with the lambda function diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp index ce1ce8e16c70..581fd6f66589 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp @@ -78,7 +78,8 @@ class FunctionParserArrayRepeat : public FunctionParser captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, - lambda_output->result_name); + lambda_output->result_name, + false); const auto * lambda_function = &actions_dag.addFunction(function_capture, {elem_arg}, lambda_output->result_name); /// Apply arrayMap with the lambda function diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp index 77c0a4725684..2d61633e80ce 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp @@ -147,7 +147,8 @@ class FunctionParserLambda : public FunctionParser captured_column_names, lambda_function_args, lambda_body_node->result_type, - lambda_body_node->result_name); + lambda_body_node->result_name, + false); const auto * result = &actions_dag.addFunction(function_capture, lambda_children, lambda_body_node->result_name); return result; From 7e5172ab2131faa22a2f933d8020d03ac1936b36 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 11 Nov 2024 18:44:18 +0800 Subject: [PATCH 066/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_10) (#7881) 62e589ecc by Xiaoxuan Meng, Support different vector serialization format for streaming shuffle (11445) ec825034e by Ke, Add faulty injection in writer fuzzer (11375) 5e0f52c52 by Richard Barnes, Remove unused-variable in velox/common/memory/tests/MockSharedArbitratorTest.cpp (11485) 9a1718142 by Richard Barnes, Remove unused-variable in velox/experimental/wave/exec/Wave.cpp (11484) 4a79bc55f by Bikramjeet Vig, Add support for testing peeling in expression fuzzer (11379) a33e8d733 by Jimmy Lu, Fix integer overflow while skipping on a stream (11477) 5544d0a83 by Bikramjeet Vig, Add option to disable flatenning optimization in PrestoSerializer (11465) b146deea0 by Jialiang Tan, Fix MockSharedArbitrationTest.localArbitrationRunInParallelWithGlobalArbitration (11476) --- cpp/velox/compute/VeloxBackend.cc | 4 ++++ cpp/velox/shuffle/VeloxShuffleReader.cc | 5 +++-- cpp/velox/shuffle/VeloxShuffleReader.h | 1 + cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h | 4 ++++ ep/build-velox/src/get_velox.sh | 2 +- 5 files changed, 13 insertions(+), 3 deletions(-) diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index d39b0902c250..c1e907bea1d5 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -162,6 +162,10 @@ void VeloxBackend::init(const std::unordered_map& conf // serde, for spill facebook::velox::serializer::presto::PrestoVectorSerde::registerVectorSerde(); } + if (!isRegisteredNamedVectorSerde(facebook::velox::VectorSerde::Kind::kPresto)) { + // RSS shuffle serde. + facebook::velox::serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } velox::exec::Operator::registerOperator(std::make_unique()); initUdf(); diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index 32e346d737a3..92751d454aa9 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -538,6 +538,7 @@ VeloxRssSortShuffleReaderDeserializer::VeloxRssSortShuffleReaderDeserializer( rowType_(rowType), batchSize_(batchSize), veloxCompressionType_(veloxCompressionType), + serde_(getNamedVectorSerde(facebook::velox::VectorSerde::Kind::kPresto)), deserializeTime_(deserializeTime) { constexpr uint64_t kMaxReadBufferSize = (1 << 20) - AlignedBuffer::kPaddedSize; auto buffer = AlignedBuffer::allocate(kMaxReadBufferSize, veloxPool_.get()); @@ -553,7 +554,7 @@ std::shared_ptr VeloxRssSortShuffleReaderDeserializer::next() { ScopedTimer timer(&deserializeTime_); RowVectorPtr rowVector; - VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, &rowVector, &serdeOptions_); + VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, serde_, &rowVector, &serdeOptions_); if (rowVector->size() >= batchSize_) { return std::make_shared(std::move(rowVector)); @@ -561,7 +562,7 @@ std::shared_ptr VeloxRssSortShuffleReaderDeserializer::next() { while (rowVector->size() < batchSize_ && in_->hasNext()) { RowVectorPtr rowVectorTemp; - VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, &rowVectorTemp, &serdeOptions_); + VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, serde_, &rowVectorTemp, &serdeOptions_); rowVector->append(rowVectorTemp.get()); } diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index f7ff05c5d13e..d39e38936369 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -125,6 +125,7 @@ class VeloxRssSortShuffleReaderDeserializer : public ColumnarBatchIterator { int32_t rowCount_; int32_t batchSize_; facebook::velox::common::CompressionKind veloxCompressionType_; + facebook::velox::VectorSerde* const serde_; facebook::velox::serializer::presto::PrestoVectorSerde::PrestoOptions serdeOptions_; int64_t& deserializeTime_; std::shared_ptr in_; diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h index f5dd6f4f384a..4b4abfb525a5 100644 --- a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -90,6 +90,10 @@ class VeloxShuffleWriterTestBase : public facebook::velox::test::VectorTestBase protected: void setUp() { + if (!isRegisteredNamedVectorSerde(facebook::velox::VectorSerde::Kind::kPresto)) { + // RSS shuffle serde. + facebook::velox::serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } // Set up test data. children1_ = { makeNullableFlatVector({1, 2, 3, std::nullopt, 4, std::nullopt, 5, 6, std::nullopt, 7}), diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 1d2718069907..759400de87e2 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_09 +VELOX_BRANCH=2024_11_10 VELOX_HOME="" OS=`uname -s` From 63059cf22da47ba8309dcd782dbe49ac633a6a5c Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 11 Nov 2024 21:17:31 +0800 Subject: [PATCH 067/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_11) (#7883) 12b52e70e by duanmeng, Add TableScan Replayer (11205) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 759400de87e2..44679ae6ee2a 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_10 +VELOX_BRANCH=2024_11_11 VELOX_HOME="" OS=`uname -s` From 36a5d18c9d7512f69bb01bd1dec46b86e939c181 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Tue, 12 Nov 2024 09:15:34 +0800 Subject: [PATCH 068/211] [GLUTEN-7847][CORE] Distinguish between native scan and vanilla spark scan in plan tree string (#7877) --- ...utenClickHouseDeltaParquetWriteSuite.scala | 16 +++++----- .../execution/GlutenClickHouseTPCHSuite.scala | 2 +- ...tenClickHouseMergeTreeCacheDataSuite.scala | 10 +++--- ...ickHouseMergeTreePathBasedWriteSuite.scala | 20 ++++++------ ...nClickHouseMergeTreeWriteOnHDFSSuite.scala | 10 +++--- ...eTreeWriteOnHDFSWithRocksDBMetaSuite.scala | 10 +++--- ...tenClickHouseMergeTreeWriteOnS3Suite.scala | 12 +++---- .../GlutenClickHouseMergeTreeWriteSuite.scala | 22 ++++++------- .../v1-bhj-ras/spark32/1.txt | 4 +-- .../v1-bhj-ras/spark32/10.txt | 16 +++++----- .../v1-bhj-ras/spark32/11.txt | 12 +++---- .../v1-bhj-ras/spark32/12.txt | 8 ++--- .../v1-bhj-ras/spark32/13.txt | 8 ++--- .../v1-bhj-ras/spark32/14.txt | 8 ++--- .../v1-bhj-ras/spark32/15.txt | 8 ++--- .../v1-bhj-ras/spark32/16.txt | 8 ++--- .../v1-bhj-ras/spark32/18.txt | 16 +++++----- .../v1-bhj-ras/spark32/19.txt | 8 ++--- .../v1-bhj-ras/spark32/20.txt | 20 ++++++------ .../v1-bhj-ras/spark32/21.txt | 24 +++++++------- .../v1-bhj-ras/spark32/22.txt | 8 ++--- .../v1-bhj-ras/spark32/3.txt | 12 +++---- .../v1-bhj-ras/spark32/4.txt | 8 ++--- .../v1-bhj-ras/spark32/5.txt | 24 +++++++------- .../v1-bhj-ras/spark32/6.txt | 4 +-- .../v1-bhj-ras/spark32/7.txt | 20 ++++++------ .../v1-bhj-ras/spark32/8.txt | 32 +++++++++---------- .../v1-bhj-ras/spark32/9.txt | 24 +++++++------- .../v1-bhj-ras/spark33/1.txt | 4 +-- .../v1-bhj-ras/spark33/10.txt | 16 +++++----- .../v1-bhj-ras/spark33/11.txt | 16 +++++----- .../v1-bhj-ras/spark33/12.txt | 8 ++--- .../v1-bhj-ras/spark33/13.txt | 8 ++--- .../v1-bhj-ras/spark33/14.txt | 8 ++--- .../v1-bhj-ras/spark33/15.txt | 12 +++---- .../v1-bhj-ras/spark33/16.txt | 8 ++--- .../v1-bhj-ras/spark33/18.txt | 16 +++++----- .../v1-bhj-ras/spark33/19.txt | 8 ++--- .../v1-bhj-ras/spark33/20.txt | 20 ++++++------ .../v1-bhj-ras/spark33/21.txt | 24 +++++++------- .../v1-bhj-ras/spark33/22.txt | 14 ++++---- .../v1-bhj-ras/spark33/3.txt | 12 +++---- .../v1-bhj-ras/spark33/4.txt | 8 ++--- .../v1-bhj-ras/spark33/5.txt | 24 +++++++------- .../v1-bhj-ras/spark33/6.txt | 4 +-- .../v1-bhj-ras/spark33/7.txt | 20 ++++++------ .../v1-bhj-ras/spark33/8.txt | 32 +++++++++---------- .../v1-bhj-ras/spark33/9.txt | 24 +++++++------- .../v1-bhj-ras/spark34/1.txt | 4 +-- .../v1-bhj-ras/spark34/10.txt | 16 +++++----- .../v1-bhj-ras/spark34/11.txt | 16 +++++----- .../v1-bhj-ras/spark34/12.txt | 8 ++--- .../v1-bhj-ras/spark34/13.txt | 8 ++--- .../v1-bhj-ras/spark34/14.txt | 8 ++--- .../v1-bhj-ras/spark34/15.txt | 12 +++---- .../v1-bhj-ras/spark34/16.txt | 8 ++--- .../v1-bhj-ras/spark34/18.txt | 16 +++++----- .../v1-bhj-ras/spark34/19.txt | 8 ++--- .../v1-bhj-ras/spark34/20.txt | 20 ++++++------ .../v1-bhj-ras/spark34/21.txt | 24 +++++++------- .../v1-bhj-ras/spark34/22.txt | 14 ++++---- .../v1-bhj-ras/spark34/3.txt | 12 +++---- .../v1-bhj-ras/spark34/4.txt | 8 ++--- .../v1-bhj-ras/spark34/5.txt | 24 +++++++------- .../v1-bhj-ras/spark34/6.txt | 4 +-- .../v1-bhj-ras/spark34/7.txt | 20 ++++++------ .../v1-bhj-ras/spark34/8.txt | 32 +++++++++---------- .../v1-bhj-ras/spark34/9.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark32/1.txt | 4 +-- .../tpch-approved-plan/v1-bhj/spark32/10.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark32/11.txt | 12 +++---- .../tpch-approved-plan/v1-bhj/spark32/12.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/13.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/14.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/15.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/16.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/18.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark32/19.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/20.txt | 20 ++++++------ .../tpch-approved-plan/v1-bhj/spark32/21.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark32/22.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/3.txt | 12 +++---- .../tpch-approved-plan/v1-bhj/spark32/4.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark32/5.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark32/6.txt | 4 +-- .../tpch-approved-plan/v1-bhj/spark32/7.txt | 20 ++++++------ .../tpch-approved-plan/v1-bhj/spark32/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1-bhj/spark32/9.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark33/1.txt | 4 +-- .../tpch-approved-plan/v1-bhj/spark33/10.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark33/11.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark33/12.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark33/13.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark33/14.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark33/15.txt | 12 +++---- .../tpch-approved-plan/v1-bhj/spark33/16.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark33/18.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark33/19.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark33/20.txt | 20 ++++++------ .../tpch-approved-plan/v1-bhj/spark33/21.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark33/22.txt | 14 ++++---- .../tpch-approved-plan/v1-bhj/spark33/3.txt | 12 +++---- .../tpch-approved-plan/v1-bhj/spark33/4.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark33/5.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark33/6.txt | 4 +-- .../tpch-approved-plan/v1-bhj/spark33/7.txt | 20 ++++++------ .../tpch-approved-plan/v1-bhj/spark33/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1-bhj/spark33/9.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark34/1.txt | 4 +-- .../tpch-approved-plan/v1-bhj/spark34/10.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark34/11.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark34/12.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark34/13.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark34/14.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark34/15.txt | 12 +++---- .../tpch-approved-plan/v1-bhj/spark34/16.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark34/18.txt | 16 +++++----- .../tpch-approved-plan/v1-bhj/spark34/19.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark34/20.txt | 20 ++++++------ .../tpch-approved-plan/v1-bhj/spark34/21.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark34/22.txt | 14 ++++---- .../tpch-approved-plan/v1-bhj/spark34/3.txt | 12 +++---- .../tpch-approved-plan/v1-bhj/spark34/4.txt | 8 ++--- .../tpch-approved-plan/v1-bhj/spark34/5.txt | 24 +++++++------- .../tpch-approved-plan/v1-bhj/spark34/6.txt | 4 +-- .../tpch-approved-plan/v1-bhj/spark34/7.txt | 20 ++++++------ .../tpch-approved-plan/v1-bhj/spark34/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1-bhj/spark34/9.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark32/1.txt | 4 +-- .../tpch-approved-plan/v1-ras/spark32/10.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark32/11.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark32/12.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/13.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/14.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/15.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/16.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/17.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark32/18.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark32/19.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/20.txt | 20 ++++++------ .../tpch-approved-plan/v1-ras/spark32/21.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark32/22.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/3.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark32/4.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark32/5.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark32/6.txt | 4 +-- .../tpch-approved-plan/v1-ras/spark32/7.txt | 20 ++++++------ .../tpch-approved-plan/v1-ras/spark32/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1-ras/spark32/9.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark33/1.txt | 4 +-- .../tpch-approved-plan/v1-ras/spark33/10.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark33/11.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark33/12.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark33/13.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark33/14.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark33/15.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark33/16.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark33/17.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark33/18.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark33/19.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark33/20.txt | 20 ++++++------ .../tpch-approved-plan/v1-ras/spark33/21.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark33/22.txt | 14 ++++---- .../tpch-approved-plan/v1-ras/spark33/3.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark33/4.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark33/5.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark33/6.txt | 4 +-- .../tpch-approved-plan/v1-ras/spark33/7.txt | 20 ++++++------ .../tpch-approved-plan/v1-ras/spark33/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1-ras/spark33/9.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark34/1.txt | 4 +-- .../tpch-approved-plan/v1-ras/spark34/10.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark34/11.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark34/12.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark34/13.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark34/14.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark34/15.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark34/16.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark34/17.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark34/18.txt | 16 +++++----- .../tpch-approved-plan/v1-ras/spark34/19.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark34/20.txt | 20 ++++++------ .../tpch-approved-plan/v1-ras/spark34/21.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark34/22.txt | 14 ++++---- .../tpch-approved-plan/v1-ras/spark34/3.txt | 12 +++---- .../tpch-approved-plan/v1-ras/spark34/4.txt | 8 ++--- .../tpch-approved-plan/v1-ras/spark34/5.txt | 24 +++++++------- .../tpch-approved-plan/v1-ras/spark34/6.txt | 4 +-- .../tpch-approved-plan/v1-ras/spark34/7.txt | 20 ++++++------ .../tpch-approved-plan/v1-ras/spark34/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1-ras/spark34/9.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark32/1.txt | 4 +-- .../tpch-approved-plan/v1/spark32/10.txt | 16 +++++----- .../tpch-approved-plan/v1/spark32/11.txt | 12 +++---- .../tpch-approved-plan/v1/spark32/12.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/13.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/14.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/15.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/16.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/17.txt | 12 +++---- .../tpch-approved-plan/v1/spark32/18.txt | 16 +++++----- .../tpch-approved-plan/v1/spark32/19.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/20.txt | 20 ++++++------ .../tpch-approved-plan/v1/spark32/21.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark32/22.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/3.txt | 12 +++---- .../tpch-approved-plan/v1/spark32/4.txt | 8 ++--- .../tpch-approved-plan/v1/spark32/5.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark32/6.txt | 4 +-- .../tpch-approved-plan/v1/spark32/7.txt | 20 ++++++------ .../tpch-approved-plan/v1/spark32/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1/spark32/9.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark33/1.txt | 4 +-- .../tpch-approved-plan/v1/spark33/10.txt | 16 +++++----- .../tpch-approved-plan/v1/spark33/11.txt | 16 +++++----- .../tpch-approved-plan/v1/spark33/12.txt | 8 ++--- .../tpch-approved-plan/v1/spark33/13.txt | 8 ++--- .../tpch-approved-plan/v1/spark33/14.txt | 8 ++--- .../tpch-approved-plan/v1/spark33/15.txt | 12 +++---- .../tpch-approved-plan/v1/spark33/16.txt | 8 ++--- .../tpch-approved-plan/v1/spark33/17.txt | 12 +++---- .../tpch-approved-plan/v1/spark33/18.txt | 16 +++++----- .../tpch-approved-plan/v1/spark33/19.txt | 8 ++--- .../tpch-approved-plan/v1/spark33/20.txt | 20 ++++++------ .../tpch-approved-plan/v1/spark33/21.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark33/22.txt | 14 ++++---- .../tpch-approved-plan/v1/spark33/3.txt | 12 +++---- .../tpch-approved-plan/v1/spark33/4.txt | 8 ++--- .../tpch-approved-plan/v1/spark33/5.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark33/6.txt | 4 +-- .../tpch-approved-plan/v1/spark33/7.txt | 20 ++++++------ .../tpch-approved-plan/v1/spark33/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1/spark33/9.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark34/1.txt | 4 +-- .../tpch-approved-plan/v1/spark34/10.txt | 16 +++++----- .../tpch-approved-plan/v1/spark34/11.txt | 16 +++++----- .../tpch-approved-plan/v1/spark34/12.txt | 8 ++--- .../tpch-approved-plan/v1/spark34/13.txt | 8 ++--- .../tpch-approved-plan/v1/spark34/14.txt | 8 ++--- .../tpch-approved-plan/v1/spark34/15.txt | 12 +++---- .../tpch-approved-plan/v1/spark34/16.txt | 8 ++--- .../tpch-approved-plan/v1/spark34/17.txt | 12 +++---- .../tpch-approved-plan/v1/spark34/18.txt | 16 +++++----- .../tpch-approved-plan/v1/spark34/19.txt | 8 ++--- .../tpch-approved-plan/v1/spark34/20.txt | 20 ++++++------ .../tpch-approved-plan/v1/spark34/21.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark34/22.txt | 14 ++++---- .../tpch-approved-plan/v1/spark34/3.txt | 12 +++---- .../tpch-approved-plan/v1/spark34/4.txt | 8 ++--- .../tpch-approved-plan/v1/spark34/5.txt | 24 +++++++------- .../tpch-approved-plan/v1/spark34/6.txt | 4 +-- .../tpch-approved-plan/v1/spark34/7.txt | 20 ++++++------ .../tpch-approved-plan/v1/spark34/8.txt | 32 +++++++++---------- .../tpch-approved-plan/v1/spark34/9.txt | 24 +++++++------- .../FileSourceScanExecTransformer.scala | 4 +-- .../GlutenCustomerExtensionSuite.scala | 2 +- .../TestFileSourceScanExecTransformer.scala | 2 +- .../GlutenCustomerExtensionSuite.scala | 2 +- .../TestFileSourceScanExecTransformer.scala | 2 +- .../GlutenCustomerExtensionSuite.scala | 2 +- .../TestFileSourceScanExecTransformer.scala | 2 +- .../GlutenCustomerExtensionSuite.scala | 2 +- .../TestFileSourceScanExecTransformer.scala | 2 +- .../v2/AbstractBatchScanExec.scala | 2 +- .../v2/AbstractBatchScanExec.scala | 2 +- 265 files changed, 1786 insertions(+), 1788 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala index 8871aaec18f5..b5ab401ec6a2 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala @@ -128,7 +128,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(plans.size === 4) val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(parquetScan.nodeName.startsWith("Scan parquet ")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet ")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -348,7 +348,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -678,7 +678,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) assert(parquetScan.metrics("numFiles").value === 201) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -736,7 +736,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -863,7 +863,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(plans.size === 4) val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -975,7 +975,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -997,7 +997,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -1192,7 +1192,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) assert(parquetScan.metrics("numFiles").value === 200) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala index 8dc178e46ce5..bbe51ef3894c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala @@ -47,7 +47,7 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { } assert(scanExec.size == 1) - assert(scanExec(0).nodeName.startsWith("Scan mergetree")) + assert(scanExec(0).nodeName.startsWith("ScanTransformer mergetree")) val sortExec = df.queryExecution.executedPlan.collect { case sortExec: SortExecTransformer => sortExec diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala index a85a9094d38f..14a8c68bdb51 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala @@ -178,7 +178,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -286,7 +286,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -389,7 +389,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -478,7 +478,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -582,7 +582,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala index 62b9ee3bcb31..6d5835957c82 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala @@ -148,7 +148,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(4)(plans.size) val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -239,7 +239,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(4)(plans.size) val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -383,7 +383,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -416,7 +416,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -608,7 +608,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -715,7 +715,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(3744)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -804,7 +804,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -908,7 +908,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1077,7 +1077,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1251,7 +1251,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala index ca1c9eef2a77..af9b541d91fd 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala @@ -144,7 +144,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -229,7 +229,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -401,7 +401,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(6)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -504,7 +504,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -585,7 +585,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala index 26c6dd852b89..9bd919f20013 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala @@ -144,7 +144,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -229,7 +229,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -401,7 +401,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(6)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -504,7 +504,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -585,7 +585,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index 331009d12c96..571dc4ba9258 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -158,7 +158,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -289,7 +289,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -461,7 +461,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(6)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -565,7 +565,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -646,7 +646,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -765,7 +765,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index 79931ec9490f..b33a2065f665 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -136,7 +136,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(4)(plans.size) val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -602,7 +602,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -770,7 +770,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(3745)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -878,7 +878,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1084,7 +1084,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1297,7 +1297,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1383,7 +1383,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1507,7 +1507,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -1588,7 +1588,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = @@ -1797,7 +1797,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec @@ -1821,7 +1821,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt index 5b913a06db13..7197e83986a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt index 946c150d8654..4687bd81bcc8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt index 39b1c80d441b..cd765112a70f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt @@ -13,7 +13,7 @@ AdaptiveSparkPlan (46) : +- BroadcastQueryStage (5) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -24,7 +24,7 @@ AdaptiveSparkPlan (46) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -71,7 +71,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt index f7b9f730e38b..10972d6c2f1a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt index 2cd26d020ae7..88d48cf5dfa3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -207,7 +207,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -324,7 +324,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt index 2ef6280501aa..a1adaf31bf28 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt @@ -20,34 +20,34 @@ AdaptiveSparkPlan (93) : : : +- BroadcastQueryStage (5) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (92) +- HashAggregate (91) @@ -83,7 +83,7 @@ AdaptiveSparkPlan (93) +- Scan parquet (83) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -112,7 +112,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -127,7 +127,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -203,7 +203,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -245,7 +245,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt index 271e9e810ec3..afb719f19903 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt index f0fca318bf20..9335e4fbe92c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt index b8cdf1656c05..9ff9aba0a986 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt index 0ef8d9d4fc1a..df5fa26f0098 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt index cac53895b981..88f83c467e85 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt index 33544f0796a5..c389ab06a45f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt index 7112bb021b0c..830259ba82bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt index 42cf8e1b7f58..c15913c284f5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt index 8956ddaa2184..b06cd55156e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt index d7603e49ffb6..74246ef67225 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -252,7 +252,7 @@ AdaptiveSparkPlan (69) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == HashAggregate (68) +- HashAggregate (67) @@ -264,7 +264,7 @@ AdaptiveSparkPlan (69) +- Scan parquet (61) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt index 394bbd2469c2..8cc2b417eb3f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt index 6c279f2d124d..8fd71a81a264 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -323,7 +323,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt index eae4758613ee..505dc59dc4cb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -155,7 +155,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -202,7 +202,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt index a3d3f0a08eeb..9113355acbdd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -227,7 +227,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -237,7 +237,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -344,7 +344,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt index e7fa21b6f1f1..befea454305b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt index a1eaaca4d448..52c99692cf4d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt index 638a442362e3..a076ea86feb5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt index 86db01e58b53..5fdae3d7334b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt index ccc7941882b7..0b00093c0316 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt index 212413568214..8c1db077485f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt index 47dc04d100db..a4a335e80831 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt index 6d2233b3e09f..61b9acc60dc0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt index 1ec03a2f9edd..a65fc78a2f03 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -254,7 +254,7 @@ AdaptiveSparkPlan (69) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == HashAggregate (68) +- HashAggregate (67) @@ -266,7 +266,7 @@ AdaptiveSparkPlan (69) +- Scan parquet (61) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt index e64cb0990026..3a9b17afd60c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt index c92587c9d791..c3ab86e5acfb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(5) Scan parquet +(5) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -288,7 +288,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt index 8382dbcd504e..5fab9fa9da42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -205,7 +205,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -248,7 +248,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt index 7f3836740155..0c2ce3c56b5a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -229,7 +229,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -239,7 +239,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -346,7 +346,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt index 1035e0eb7715..cc68e1c9b249 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -104,7 +104,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt index 150788a3d7c1..a77b48298d37 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt index 035c5c388972..58442a8ca86e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -144,7 +144,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -183,7 +183,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -222,7 +222,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt index cd877f54bd7a..eaa84cfa46c2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -134,7 +134,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -212,7 +212,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt index d381d21da73d..9d1bc4a17558 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -169,7 +169,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -208,7 +208,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -247,7 +247,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -286,7 +286,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -325,7 +325,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -364,7 +364,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt index f0ebba6efac5..8baec4075ad3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -220,7 +220,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -259,7 +259,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt index 5b913a06db13..7197e83986a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt index 946c150d8654..4687bd81bcc8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt index 39b1c80d441b..cd765112a70f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt @@ -13,7 +13,7 @@ AdaptiveSparkPlan (46) : +- BroadcastQueryStage (5) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -24,7 +24,7 @@ AdaptiveSparkPlan (46) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -71,7 +71,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt index f7b9f730e38b..10972d6c2f1a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt index 2cd26d020ae7..88d48cf5dfa3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -207,7 +207,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -324,7 +324,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt index 2ef6280501aa..a1adaf31bf28 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt @@ -20,34 +20,34 @@ AdaptiveSparkPlan (93) : : : +- BroadcastQueryStage (5) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (92) +- HashAggregate (91) @@ -83,7 +83,7 @@ AdaptiveSparkPlan (93) +- Scan parquet (83) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -112,7 +112,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -127,7 +127,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -203,7 +203,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -245,7 +245,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt index 271e9e810ec3..afb719f19903 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt index f0fca318bf20..9335e4fbe92c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt index b8cdf1656c05..9ff9aba0a986 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt index 0ef8d9d4fc1a..df5fa26f0098 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt index cac53895b981..88f83c467e85 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt index 33544f0796a5..c389ab06a45f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt index 7112bb021b0c..830259ba82bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt index 42cf8e1b7f58..c15913c284f5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt index 8956ddaa2184..b06cd55156e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt index d7603e49ffb6..74246ef67225 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -252,7 +252,7 @@ AdaptiveSparkPlan (69) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == HashAggregate (68) +- HashAggregate (67) @@ -264,7 +264,7 @@ AdaptiveSparkPlan (69) +- Scan parquet (61) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt index 394bbd2469c2..8cc2b417eb3f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt index 6c279f2d124d..8fd71a81a264 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -323,7 +323,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt index eae4758613ee..505dc59dc4cb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -155,7 +155,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -202,7 +202,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt index a3d3f0a08eeb..9113355acbdd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -227,7 +227,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -237,7 +237,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -344,7 +344,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt index e7fa21b6f1f1..befea454305b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt index a1eaaca4d448..52c99692cf4d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt index 638a442362e3..a076ea86feb5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt index 86db01e58b53..5fdae3d7334b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt index ccc7941882b7..0b00093c0316 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt index 212413568214..8c1db077485f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt index 47dc04d100db..a4a335e80831 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt index 6d2233b3e09f..61b9acc60dc0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt index 1ec03a2f9edd..a65fc78a2f03 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -254,7 +254,7 @@ AdaptiveSparkPlan (69) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == HashAggregate (68) +- HashAggregate (67) @@ -266,7 +266,7 @@ AdaptiveSparkPlan (69) +- Scan parquet (61) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt index e64cb0990026..3a9b17afd60c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt index c92587c9d791..c3ab86e5acfb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(5) Scan parquet +(5) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -288,7 +288,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt index 8382dbcd504e..5fab9fa9da42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -205,7 +205,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -248,7 +248,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt index 7f3836740155..0c2ce3c56b5a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -229,7 +229,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -239,7 +239,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -346,7 +346,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt index 1035e0eb7715..cc68e1c9b249 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -104,7 +104,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt index 150788a3d7c1..a77b48298d37 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt index 035c5c388972..58442a8ca86e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -144,7 +144,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -183,7 +183,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -222,7 +222,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt index cd877f54bd7a..eaa84cfa46c2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -134,7 +134,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -212,7 +212,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt index d381d21da73d..9d1bc4a17558 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -169,7 +169,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -208,7 +208,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -247,7 +247,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -286,7 +286,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -325,7 +325,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -364,7 +364,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt index f0ebba6efac5..8baec4075ad3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -220,7 +220,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -259,7 +259,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt index a7ced66a82e7..7c8451dfd6dd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt index b385b7719de2..f943422808e8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt index 71dee80fab86..48e7e4d4e0d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt index edbcb50206df..91444cca0528 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt @@ -24,14 +24,14 @@ AdaptiveSparkPlan (58) : +- ColumnarExchange (5) : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == Sort (57) +- Exchange (56) @@ -52,7 +52,7 @@ AdaptiveSparkPlan (58) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +84,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt index 91df642595a3..33c0dfb71e1e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt @@ -13,14 +13,14 @@ AdaptiveSparkPlan (39) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (38) +- HashAggregate (37) @@ -37,7 +37,7 @@ AdaptiveSparkPlan (39) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +74,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt index 56169a244bf9..9317c2f895a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (50) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -26,7 +26,7 @@ AdaptiveSparkPlan (50) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (49) +- Exchange (48) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (50) +- Scan parquet (38) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt index 8ef7d4d51441..2c41822beeeb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt index fe915989f9e6..94e892d1e8c9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt @@ -15,14 +15,14 @@ AdaptiveSparkPlan (63) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) +- ^ RegularHashAggregateExecTransformer (31) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (63) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == HashAggregate (62) +- HashAggregate (61) @@ -59,7 +59,7 @@ AdaptiveSparkPlan (63) +- Scan parquet (52) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +96,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt index a2345ca9ed35..550cd076c1c2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36) : +- ColumnarExchange (35) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) : +- ^ RegularHashAggregateExecTransformer (28) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) + : +- ^ ScanTransformer parquet (19) +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) + : +- ^ ScanTransformer parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) +- ^ RegularHashAggregateExecTransformer (60) @@ -99,7 +99,7 @@ AdaptiveSparkPlan (110) +- Scan parquet (97) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +136,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +289,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt index d06cdb0fecff..f7778691955a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (38) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (37) +- HashAggregate (36) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (38) +- Scan parquet (30) -(1) Scan parquet +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt index 6fa1271a5a39..5dd6bbfccab6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt @@ -21,7 +21,7 @@ AdaptiveSparkPlan (146) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66) : +- ColumnarExchange (65) @@ -40,14 +40,14 @@ AdaptiveSparkPlan (146) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) + : : +- ^ ScanTransformer parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58) : +- ColumnarExchange (57) @@ -64,7 +64,7 @@ AdaptiveSparkPlan (146) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) + : : +- ^ ScanTransformer parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46) : +- ReusedExchange (45) @@ -74,7 +74,7 @@ AdaptiveSparkPlan (146) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + +- ^ ScanTransformer parquet (77) +- == Initial Plan == Sort (145) +- Exchange (144) @@ -127,7 +127,7 @@ AdaptiveSparkPlan (146) +- Scan parquet (137) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -164,7 +164,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -201,7 +201,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +269,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -434,7 +434,7 @@ Input [3]: [s_name#X, s_address#X, s_nationkey#X] (76) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(77) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt index e70a145e9c1d..e72762dd9bfe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt @@ -30,7 +30,7 @@ AdaptiveSparkPlan (138) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) @@ -44,34 +44,34 @@ AdaptiveSparkPlan (138) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (137) +- HashAggregate (136) @@ -120,7 +120,7 @@ AdaptiveSparkPlan (138) +- Scan parquet (127) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -157,7 +157,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -330,7 +330,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -398,7 +398,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt index 674e0b96983f..81af91c0ee2a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt index 3b9b7fd93b96..c4df05de1618 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt @@ -20,21 +20,21 @@ AdaptiveSparkPlan (67) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) +- HashAggregate (65) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (67) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +98,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt index ffb35d1813ae..40cfa020ce73 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt index 0e48f73c7795..37faa880d8af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt index f089996aa4e3..f3ec7ef3a620 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt index 80b0d02a6cb2..8bd2945e4859 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt index a41822a60a19..6cca8ba5a479 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt index 1b89e64860f3..a14969cbc44b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt index 6ca992b70988..cc1f4d31b697 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -444,7 +444,7 @@ AdaptiveSparkPlan (136) : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) + : : +- ^ ScanTransformer parquet (83) : +- ^ InputIteratorTransformer (95) : +- ShuffleQueryStage (93), Statistics(X) : +- ReusedExchange (92) @@ -475,7 +475,7 @@ AdaptiveSparkPlan (136) +- Scan parquet (127) -(83) Scan parquet +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt index 346a6866342e..f998361cddc5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt index 9e927782f6c3..bd84f11c6ac5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt @@ -24,14 +24,14 @@ AdaptiveSparkPlan (58) : +- ColumnarExchange (5) : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == Sort (57) +- Exchange (56) @@ -52,7 +52,7 @@ AdaptiveSparkPlan (58) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +84,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt index d82e4f2b21db..5fdc418d02ef 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt @@ -13,14 +13,14 @@ AdaptiveSparkPlan (39) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (38) +- HashAggregate (37) @@ -37,7 +37,7 @@ AdaptiveSparkPlan (39) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +74,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt index 950441af70e9..2d327719c3bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -272,7 +272,7 @@ AdaptiveSparkPlan (73) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == HashAggregate (72) +- HashAggregate (71) @@ -284,7 +284,7 @@ AdaptiveSparkPlan (73) +- Scan parquet (65) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt index aa7aa005a9b2..cac61db4a6bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt index e48379edc9fa..c5033fa773a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt @@ -15,14 +15,14 @@ AdaptiveSparkPlan (63) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) +- ^ RegularHashAggregateExecTransformer (31) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (63) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == HashAggregate (62) +- HashAggregate (61) @@ -59,7 +59,7 @@ AdaptiveSparkPlan (63) +- Scan parquet (52) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +96,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt index 8dbf9429961a..b07b35495922 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) : +- ^ RegularHashAggregateExecTransformer (28) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) + : +- ^ ScanTransformer parquet (19) +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) + : +- ^ ScanTransformer parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) +- ^ RegularHashAggregateExecTransformer (60) @@ -99,7 +99,7 @@ AdaptiveSparkPlan (110) +- Scan parquet (97) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +136,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +289,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt index 41da1d28e987..348cc4ce85d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (38) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (37) +- HashAggregate (36) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (38) +- Scan parquet (30) -(1) Scan parquet +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt index 1fad7160bf64..703c4a99d63c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt @@ -20,7 +20,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) @@ -39,14 +39,14 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) + : : +- ^ ScanTransformer parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) + : : +- ^ ScanTransformer parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46), Statistics(X) : +- ReusedExchange (45) @@ -73,7 +73,7 @@ AdaptiveSparkPlan (143) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + +- ^ ScanTransformer parquet (77) +- == Initial Plan == Sort (142) +- Exchange (141) @@ -126,7 +126,7 @@ AdaptiveSparkPlan (143) +- Scan parquet (134) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -268,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -433,7 +433,7 @@ Input [3]: [s_name#X, s_address#X, s_nationkey#X] (76) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(77) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt index fd61236faaf8..a675841a475a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -230,7 +230,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -329,7 +329,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -397,7 +397,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt index 44c57da7482f..c48c13779d64 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -293,7 +293,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -400,7 +400,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt index a698975568fb..f6dc9c8015ea 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt @@ -20,21 +20,21 @@ AdaptiveSparkPlan (67) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) +- HashAggregate (65) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (67) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +98,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt index 8b00d867cd70..23fb123956dc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt index dd869d25d416..ea8f7dbd3371 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt index ac6dda065bb0..ccf77e181d75 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt index 6f5ed0d82b80..76e38b48de31 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt index 22cd2b9967fb..3c152f565673 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt index a052989ba525..1a64123cc8c0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -263,7 +263,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt index 68e87c39a078..5278039ec385 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -178,7 +178,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -448,7 +448,7 @@ AdaptiveSparkPlan (136) : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) + : : +- ^ ScanTransformer parquet (83) : +- ^ InputIteratorTransformer (95) : +- ShuffleQueryStage (93), Statistics(X) : +- ReusedExchange (92) @@ -479,7 +479,7 @@ AdaptiveSparkPlan (136) +- Scan parquet (127) -(83) Scan parquet +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt index 9175845083ca..3bfd39bdaf49 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt index b527c2bc3bc3..ca06d654b432 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt @@ -24,14 +24,14 @@ AdaptiveSparkPlan (58) : +- ColumnarExchange (5) : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == Sort (57) +- Exchange (56) @@ -52,7 +52,7 @@ AdaptiveSparkPlan (58) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +84,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt index cc6d364dea22..c61d21b92ed1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt @@ -13,14 +13,14 @@ AdaptiveSparkPlan (39) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (38) +- HashAggregate (37) @@ -37,7 +37,7 @@ AdaptiveSparkPlan (39) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +74,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt index 8f1a1b4415db..3585c0cc2f34 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -274,7 +274,7 @@ AdaptiveSparkPlan (73) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == HashAggregate (72) +- HashAggregate (71) @@ -286,7 +286,7 @@ AdaptiveSparkPlan (73) +- Scan parquet (65) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt index ee7826c8aa7e..3441216f140f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt index eaad0429ab40..c6d0e0103363 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt @@ -15,14 +15,14 @@ AdaptiveSparkPlan (63) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) +- ^ RegularHashAggregateExecTransformer (31) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (63) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == HashAggregate (62) +- HashAggregate (61) @@ -59,7 +59,7 @@ AdaptiveSparkPlan (63) +- Scan parquet (52) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +96,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt index 85b11fa8ca18..50a076b1ef8f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) : +- ^ RegularHashAggregateExecTransformer (28) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) + : +- ^ ScanTransformer parquet (19) +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) + : +- ^ ScanTransformer parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) +- ^ RegularHashAggregateExecTransformer (60) @@ -99,7 +99,7 @@ AdaptiveSparkPlan (110) +- Scan parquet (97) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +136,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -291,7 +291,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt index c72ca859ce37..fb7e231fa5b3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (38) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (37) +- HashAggregate (36) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (38) +- Scan parquet (30) -(1) Scan parquet +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt index ed09215178a4..4e427961a4f6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt @@ -20,7 +20,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) @@ -39,14 +39,14 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) + : : +- ^ ScanTransformer parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) + : : +- ^ ScanTransformer parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46), Statistics(X) : +- ReusedExchange (45) @@ -73,7 +73,7 @@ AdaptiveSparkPlan (143) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + +- ^ ScanTransformer parquet (77) +- == Initial Plan == Sort (142) +- Exchange (141) @@ -126,7 +126,7 @@ AdaptiveSparkPlan (143) +- Scan parquet (134) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +269,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -437,7 +437,7 @@ Input [3]: [s_name#X, s_address#X, s_nationkey#X] (76) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(77) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt index 271d0c6d5098..96750b2b3ecc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -332,7 +332,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -401,7 +401,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt index b8123fb6dc0e..c6e147da04d8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -285,7 +285,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -295,7 +295,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -402,7 +402,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt index 8eea7581a417..4aa745832681 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt @@ -20,21 +20,21 @@ AdaptiveSparkPlan (67) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) +- HashAggregate (65) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (67) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +98,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -167,7 +167,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt index 286881fb99b1..71a8f6974ed1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt index 0a9af460526c..da9339abd628 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -240,7 +240,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -309,7 +309,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -378,7 +378,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -447,7 +447,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt index aa69dbcd11f7..a0894489f0a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -234,7 +234,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -303,7 +303,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -372,7 +372,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt index 05f9c2c90e5d..8ce3620f30d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -352,7 +352,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -421,7 +421,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -490,7 +490,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -559,7 +559,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -628,7 +628,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt index e298d2a75196..4f537e91e132 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -308,7 +308,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -377,7 +377,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -446,7 +446,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt index a7ced66a82e7..7c8451dfd6dd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt index b385b7719de2..f943422808e8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt index 71dee80fab86..48e7e4d4e0d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt index edbcb50206df..91444cca0528 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt @@ -24,14 +24,14 @@ AdaptiveSparkPlan (58) : +- ColumnarExchange (5) : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == Sort (57) +- Exchange (56) @@ -52,7 +52,7 @@ AdaptiveSparkPlan (58) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +84,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt index 91df642595a3..33c0dfb71e1e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt @@ -13,14 +13,14 @@ AdaptiveSparkPlan (39) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (38) +- HashAggregate (37) @@ -37,7 +37,7 @@ AdaptiveSparkPlan (39) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +74,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt index 56169a244bf9..9317c2f895a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (50) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -26,7 +26,7 @@ AdaptiveSparkPlan (50) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (49) +- Exchange (48) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (50) +- Scan parquet (38) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt index 8ef7d4d51441..2c41822beeeb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt index fe915989f9e6..94e892d1e8c9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt @@ -15,14 +15,14 @@ AdaptiveSparkPlan (63) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) +- ^ RegularHashAggregateExecTransformer (31) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (63) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == HashAggregate (62) +- HashAggregate (61) @@ -59,7 +59,7 @@ AdaptiveSparkPlan (63) +- Scan parquet (52) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +96,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt index a2345ca9ed35..550cd076c1c2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36) : +- ColumnarExchange (35) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) : +- ^ RegularHashAggregateExecTransformer (28) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) + : +- ^ ScanTransformer parquet (19) +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) + : +- ^ ScanTransformer parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) +- ^ RegularHashAggregateExecTransformer (60) @@ -99,7 +99,7 @@ AdaptiveSparkPlan (110) +- Scan parquet (97) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +136,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +289,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt index d06cdb0fecff..f7778691955a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (38) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (37) +- HashAggregate (36) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (38) +- Scan parquet (30) -(1) Scan parquet +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt index 6fa1271a5a39..5dd6bbfccab6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt @@ -21,7 +21,7 @@ AdaptiveSparkPlan (146) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66) : +- ColumnarExchange (65) @@ -40,14 +40,14 @@ AdaptiveSparkPlan (146) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) + : : +- ^ ScanTransformer parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58) : +- ColumnarExchange (57) @@ -64,7 +64,7 @@ AdaptiveSparkPlan (146) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) + : : +- ^ ScanTransformer parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46) : +- ReusedExchange (45) @@ -74,7 +74,7 @@ AdaptiveSparkPlan (146) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + +- ^ ScanTransformer parquet (77) +- == Initial Plan == Sort (145) +- Exchange (144) @@ -127,7 +127,7 @@ AdaptiveSparkPlan (146) +- Scan parquet (137) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -164,7 +164,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -201,7 +201,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +269,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -434,7 +434,7 @@ Input [3]: [s_name#X, s_address#X, s_nationkey#X] (76) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(77) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt index e70a145e9c1d..e72762dd9bfe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt @@ -30,7 +30,7 @@ AdaptiveSparkPlan (138) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) @@ -44,34 +44,34 @@ AdaptiveSparkPlan (138) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (137) +- HashAggregate (136) @@ -120,7 +120,7 @@ AdaptiveSparkPlan (138) +- Scan parquet (127) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -157,7 +157,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -330,7 +330,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -398,7 +398,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt index 674e0b96983f..81af91c0ee2a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt index 3b9b7fd93b96..c4df05de1618 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt @@ -20,21 +20,21 @@ AdaptiveSparkPlan (67) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) +- HashAggregate (65) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (67) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +98,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt index ffb35d1813ae..40cfa020ce73 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt index 0e48f73c7795..37faa880d8af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt index f089996aa4e3..f3ec7ef3a620 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt index 80b0d02a6cb2..8bd2945e4859 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt index a41822a60a19..6cca8ba5a479 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt index 1b89e64860f3..a14969cbc44b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt index 6ca992b70988..cc1f4d31b697 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -444,7 +444,7 @@ AdaptiveSparkPlan (136) : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) + : : +- ^ ScanTransformer parquet (83) : +- ^ InputIteratorTransformer (95) : +- ShuffleQueryStage (93), Statistics(X) : +- ReusedExchange (92) @@ -475,7 +475,7 @@ AdaptiveSparkPlan (136) +- Scan parquet (127) -(83) Scan parquet +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt index 346a6866342e..f998361cddc5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt index 9e927782f6c3..bd84f11c6ac5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt @@ -24,14 +24,14 @@ AdaptiveSparkPlan (58) : +- ColumnarExchange (5) : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == Sort (57) +- Exchange (56) @@ -52,7 +52,7 @@ AdaptiveSparkPlan (58) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +84,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt index d82e4f2b21db..5fdc418d02ef 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt @@ -13,14 +13,14 @@ AdaptiveSparkPlan (39) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (38) +- HashAggregate (37) @@ -37,7 +37,7 @@ AdaptiveSparkPlan (39) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +74,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt index 950441af70e9..2d327719c3bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -272,7 +272,7 @@ AdaptiveSparkPlan (73) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == HashAggregate (72) +- HashAggregate (71) @@ -284,7 +284,7 @@ AdaptiveSparkPlan (73) +- Scan parquet (65) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt index aa7aa005a9b2..cac61db4a6bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt index e48379edc9fa..c5033fa773a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt @@ -15,14 +15,14 @@ AdaptiveSparkPlan (63) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) +- ^ RegularHashAggregateExecTransformer (31) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (63) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == HashAggregate (62) +- HashAggregate (61) @@ -59,7 +59,7 @@ AdaptiveSparkPlan (63) +- Scan parquet (52) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +96,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt index 8dbf9429961a..b07b35495922 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) : +- ^ RegularHashAggregateExecTransformer (28) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) + : +- ^ ScanTransformer parquet (19) +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) + : +- ^ ScanTransformer parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) +- ^ RegularHashAggregateExecTransformer (60) @@ -99,7 +99,7 @@ AdaptiveSparkPlan (110) +- Scan parquet (97) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +136,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +289,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt index 41da1d28e987..348cc4ce85d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (38) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (37) +- HashAggregate (36) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (38) +- Scan parquet (30) -(1) Scan parquet +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt index 1fad7160bf64..703c4a99d63c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt @@ -20,7 +20,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) @@ -39,14 +39,14 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) + : : +- ^ ScanTransformer parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) + : : +- ^ ScanTransformer parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46), Statistics(X) : +- ReusedExchange (45) @@ -73,7 +73,7 @@ AdaptiveSparkPlan (143) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + +- ^ ScanTransformer parquet (77) +- == Initial Plan == Sort (142) +- Exchange (141) @@ -126,7 +126,7 @@ AdaptiveSparkPlan (143) +- Scan parquet (134) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -268,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -433,7 +433,7 @@ Input [3]: [s_name#X, s_address#X, s_nationkey#X] (76) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(77) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt index fd61236faaf8..a675841a475a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -230,7 +230,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -329,7 +329,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -397,7 +397,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt index 44c57da7482f..c48c13779d64 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -293,7 +293,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -400,7 +400,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt index a698975568fb..f6dc9c8015ea 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt @@ -20,21 +20,21 @@ AdaptiveSparkPlan (67) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) +- HashAggregate (65) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (67) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +98,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt index 8b00d867cd70..23fb123956dc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt index dd869d25d416..ea8f7dbd3371 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt index ac6dda065bb0..ccf77e181d75 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt index 6f5ed0d82b80..76e38b48de31 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt index 22cd2b9967fb..3c152f565673 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt index a052989ba525..1a64123cc8c0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -263,7 +263,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt index 68e87c39a078..5278039ec385 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -178,7 +178,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -448,7 +448,7 @@ AdaptiveSparkPlan (136) : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) + : : +- ^ ScanTransformer parquet (83) : +- ^ InputIteratorTransformer (95) : +- ShuffleQueryStage (93), Statistics(X) : +- ReusedExchange (92) @@ -479,7 +479,7 @@ AdaptiveSparkPlan (136) +- Scan parquet (127) -(83) Scan parquet +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt index 9175845083ca..3bfd39bdaf49 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt index b527c2bc3bc3..ca06d654b432 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt @@ -24,14 +24,14 @@ AdaptiveSparkPlan (58) : +- ColumnarExchange (5) : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == Sort (57) +- Exchange (56) @@ -52,7 +52,7 @@ AdaptiveSparkPlan (58) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +84,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt index cc6d364dea22..c61d21b92ed1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt @@ -13,14 +13,14 @@ AdaptiveSparkPlan (39) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (38) +- HashAggregate (37) @@ -37,7 +37,7 @@ AdaptiveSparkPlan (39) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +74,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt index 8f1a1b4415db..3585c0cc2f34 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -274,7 +274,7 @@ AdaptiveSparkPlan (73) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == HashAggregate (72) +- HashAggregate (71) @@ -286,7 +286,7 @@ AdaptiveSparkPlan (73) +- Scan parquet (65) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt index ee7826c8aa7e..3441216f140f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt index eaad0429ab40..c6d0e0103363 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt @@ -15,14 +15,14 @@ AdaptiveSparkPlan (63) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ FilterExecTransformer (33) +- ^ ProjectExecTransformer (32) +- ^ RegularHashAggregateExecTransformer (31) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (63) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == HashAggregate (62) +- HashAggregate (61) @@ -59,7 +59,7 @@ AdaptiveSparkPlan (63) +- Scan parquet (52) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +96,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt index 85b11fa8ca18..50a076b1ef8f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt @@ -19,7 +19,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) @@ -32,7 +32,7 @@ AdaptiveSparkPlan (110) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ ProjectExecTransformer (30) : +- ^ FilterExecTransformer (29) : +- ^ RegularHashAggregateExecTransformer (28) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) + : +- ^ ScanTransformer parquet (19) +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (110) : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) + : +- ^ ScanTransformer parquet (47) +- ^ ProjectExecTransformer (62) +- ^ FilterExecTransformer (61) +- ^ RegularHashAggregateExecTransformer (60) @@ -99,7 +99,7 @@ AdaptiveSparkPlan (110) +- Scan parquet (97) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +136,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -291,7 +291,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt index c72ca859ce37..fb7e231fa5b3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (38) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == HashAggregate (37) +- HashAggregate (36) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (38) +- Scan parquet (30) -(1) Scan parquet +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt index ed09215178a4..4e427961a4f6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt @@ -20,7 +20,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) @@ -39,14 +39,14 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) + : : +- ^ ScanTransformer parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (143) : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) + : : +- ^ ScanTransformer parquet (36) : +- ^ InputIteratorTransformer (48) : +- ShuffleQueryStage (46), Statistics(X) : +- ReusedExchange (45) @@ -73,7 +73,7 @@ AdaptiveSparkPlan (143) +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + +- ^ ScanTransformer parquet (77) +- == Initial Plan == Sort (142) +- Exchange (141) @@ -126,7 +126,7 @@ AdaptiveSparkPlan (143) +- Scan parquet (134) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +269,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -437,7 +437,7 @@ Input [3]: [s_name#X, s_address#X, s_nationkey#X] (76) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(77) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt index 271d0c6d5098..96750b2b3ecc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -332,7 +332,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -401,7 +401,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt index b8123fb6dc0e..c6e147da04d8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -285,7 +285,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -295,7 +295,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -402,7 +402,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt index 8eea7581a417..4aa745832681 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt @@ -20,21 +20,21 @@ AdaptiveSparkPlan (67) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == TakeOrderedAndProject (66) +- HashAggregate (65) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (67) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +98,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -167,7 +167,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt index 286881fb99b1..71a8f6974ed1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt index 0a9af460526c..da9339abd628 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -240,7 +240,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -309,7 +309,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -378,7 +378,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -447,7 +447,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt index aa69dbcd11f7..a0894489f0a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -234,7 +234,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -303,7 +303,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -372,7 +372,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt index 05f9c2c90e5d..8ce3620f30d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -352,7 +352,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -421,7 +421,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -490,7 +490,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -559,7 +559,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -628,7 +628,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt index e298d2a75196..4f537e91e132 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -308,7 +308,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -377,7 +377,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -446,7 +446,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala index d64c5ae016c5..d2f8237b6969 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala @@ -157,10 +157,8 @@ abstract class FileSourceScanExecTransformerBase( override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genFileSourceScanTransformerMetricsUpdater(metrics) - override val nodeNamePrefix: String = "NativeFile" - override val nodeName: String = { - s"Scan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" + s"ScanTransformer $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" } override def getProperties: Map[String, String] = { diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 88d771ec8a4f..8b5efa91673f 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -60,7 +60,7 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" override def doCanonicalize(): TestFileSourceScanExecTransformer = { TestFileSourceScanExecTransformer( diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 85af9a623853..98ef4eff7869 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -58,5 +58,5 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 85af9a623853..98ef4eff7869 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -58,5 +58,5 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 85af9a623853..98ef4eff7869 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -58,5 +58,5 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" } diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala index cb25da3d3d20..e3e9659ed33a 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala @@ -260,6 +260,6 @@ abstract class AbstractBatchScanExec( } override def nodeName: String = { - s"BatchScan ${table.name()}".trim + s"BatchScanTransformer ${table.name()}".trim } } diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala index 3aa3be6ead3d..8f51ea2c72b1 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala @@ -260,6 +260,6 @@ abstract class AbstractBatchScanExec( } override def nodeName: String = { - s"BatchScan ${table.name()}".trim + s"BatchScanTransformer ${table.name()}".trim } } From fca7d8c7a2ae47acde30eea13aebde2b68b1d478 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 12 Nov 2024 11:24:15 +0800 Subject: [PATCH 069/211] [GLUTEN-7886][VL] Fix broken Ubuntu 20.04 + VCPKG + GCS + ABFS build Closes #7886 --- cpp/core/CMakeLists.txt | 2 ++ cpp/velox/CMakeLists.txt | 6 ++++++ 2 files changed, 8 insertions(+) diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 14b9b2d6f249..78422e791a15 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -53,8 +53,10 @@ if(CCACHE_FOUND) endif(CCACHE_FOUND) macro(find_protobuf) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") find_package(Protobuf) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) if("${Protobuf_LIBRARY}" STREQUAL "Protobuf_LIBRARY-NOTFOUND") message(FATAL_ERROR "Protobuf Library Not Found") endif() diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 17defc498ec1..586c08653dfc 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -130,21 +130,27 @@ macro(find_re2) endmacro() macro(find_awssdk) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") find_package(AWSSDK REQUIRED COMPONENTS s3;identity-management) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() macro(find_gcssdk) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) set(CMAKE_FIND_LIBRARY_SUFFIXES ".so") find_package(google_cloud_cpp_storage REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() macro(find_azure) find_package(CURL REQUIRED) find_package(LibXml2 REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") find_package(azure-storage-blobs-cpp CONFIG REQUIRED) find_package(azure-storage-files-datalake-cpp CONFIG REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() # Build Velox backend. From 6a684a7c94a111558438ce962eba7994b09d3cbe Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Tue, 12 Nov 2024 13:38:04 +0800 Subject: [PATCH 070/211] [GLUTEN-7890][UI] Optimize cleanup gluten sql executions ui data (#7891) --- .../spark/sql/execution/ui/GlutenSQLAppStatusListener.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala b/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala index 25150ca31f9f..a2f608f64d2e 100644 --- a/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala +++ b/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala @@ -32,7 +32,7 @@ private class GlutenSQLAppStatusListener(conf: SparkConf, kvstore: ElementTracki private val executionIdToDescription = new mutable.HashMap[Long, String] private val executionIdToFallbackEvent = new mutable.HashMap[Long, GlutenPlanFallbackEvent] - kvstore.addTrigger(classOf[SQLExecutionUIData], conf.get[Int](UI_RETAINED_EXECUTIONS)) { + kvstore.addTrigger(classOf[GlutenSQLExecutionUIData], conf.get[Int](UI_RETAINED_EXECUTIONS)) { count => cleanupExecutions(count) } @@ -71,7 +71,7 @@ private class GlutenSQLAppStatusListener(conf: SparkConf, kvstore: ElementTracki fallbackEvent.get.physicalPlanDescription, fallbackEvent.get.fallbackNodeToReason.toSeq.sortBy(_._1) ) - kvstore.write(uiData) + kvstore.write(uiData, checkTriggers = true) executionIdToFallbackEvent.remove(event.executionId) } executionIdToDescription.put(event.executionId, event.description) From 0637a227b93052c2136a6b91ffe86d4568db8b69 Mon Sep 17 00:00:00 2001 From: lgbo Date: Tue, 12 Nov 2024 13:42:35 +0800 Subject: [PATCH 071/211] [GLUTEN-7907][CH] Fixed data race in `ExpresionParser::getUniqueName` (#7908) --- cpp-ch/local-engine/Parser/ExpressionParser.cpp | 2 +- cpp-ch/local-engine/Parser/ExpressionParser.h | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.cpp b/cpp-ch/local-engine/Parser/ExpressionParser.cpp index 30ef92f17607..9899e90ddcda 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.cpp +++ b/cpp-ch/local-engine/Parser/ExpressionParser.cpp @@ -624,7 +624,7 @@ const DB::ActionsDAG::Node * ExpressionParser::toFunctionNode( return &actions_dag.addFunction(function_builder, args, result_name); } -UInt64 ExpressionParser::unique_name_counter = 0; +std::atomic ExpressionParser::unique_name_counter = 0; String ExpressionParser::getUniqueName(const String & name) const { return name + "_" + std::to_string(unique_name_counter++); diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.h b/cpp-ch/local-engine/Parser/ExpressionParser.h index 5aedb94f2c35..95c66146ab3d 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.h +++ b/cpp-ch/local-engine/Parser/ExpressionParser.h @@ -20,6 +20,7 @@ #include #include #include +#include #include "SerializedPlanParser.h" @@ -74,7 +75,7 @@ class ExpressionParser String safeGetFunctionName(const substrait::Expression_ScalarFunction & func_) const; private: - static UInt64 unique_name_counter; + static std::atomic unique_name_counter; std::shared_ptr context; DB::ActionsDAG::NodeRawConstPtrs From bfb631afe86cc5bf6722dba2bf1248500dd87514 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 12 Nov 2024 13:46:57 +0800 Subject: [PATCH 072/211] [VL] CI: Fix out-of-date module name in labeler.yml (#7915) --- .github/labeler.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/labeler.yml b/.github/labeler.yml index abc62156f40f..5deb2714ca6f 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -60,7 +60,7 @@ CORE: VELOX: - changed-files: - any-glob-to-any-file: [ - 'gluten-data/**/*', + 'gluten-arrow/**/*', 'backends-velox/**/*', 'ep/build-velox/**/*', 'cpp/**/*' From 106fd80b6441162577caa49c57a057b2ab7b27b7 Mon Sep 17 00:00:00 2001 From: wang-zhun <61445191+wang-zhun@users.noreply.github.com> Date: Tue, 12 Nov 2024 14:27:25 +0800 Subject: [PATCH 073/211] [CORE] The fallback check for Scan should not be skipped when DPP is present (#7080) --- .../extension/columnar/FallbackRules.scala | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala index b557abe001a1..1268d36d3290 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala @@ -272,21 +272,15 @@ case class AddFallbackTagRule() extends Rule[SparkPlan] { try { plan match { case plan: BatchScanExec => - // If filter expressions aren't empty, we need to transform the inner operators. - if (plan.runtimeFilters.isEmpty) { - val transformer = - ScanTransformerFactory - .createBatchScanTransformer(plan, validation = true) - .asInstanceOf[BasicScanExecTransformer] - transformer.doValidate().tagOnFallback(plan) - } + val transformer = + ScanTransformerFactory + .createBatchScanTransformer(plan, validation = true) + .asInstanceOf[BasicScanExecTransformer] + transformer.doValidate().tagOnFallback(plan) case plan: FileSourceScanExec => - // If filter expressions aren't empty, we need to transform the inner operators. - if (plan.partitionFilters.isEmpty) { - val transformer = - ScanTransformerFactory.createFileSourceScanTransformer(plan) - transformer.doValidate().tagOnFallback(plan) - } + val transformer = + ScanTransformerFactory.createFileSourceScanTransformer(plan) + transformer.doValidate().tagOnFallback(plan) case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => HiveTableScanExecTransformer.validate(plan).tagOnFallback(plan) case plan: ProjectExec => From 2d03b003c1b0cf18faae0154e5370a1bd3e829e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Tue, 12 Nov 2024 16:16:16 +0800 Subject: [PATCH 074/211] [GLUTEN-7868][CH] Nested column pruning for Project(Filter(Generate)) (#7869) * save test configs * wip * finish dev * fix import * fix style * fix style * fix failed uts * fix failec uts * fix style --- .../backendsapi/clickhouse/CHRuleApi.scala | 1 + ...xtendedGeneratorNestedColumnAliasing.scala | 126 ++++++++++++++++++ .../hive/GlutenClickHouseHiveTableSuite.scala | 51 ++++++- .../org/apache/gluten/GlutenConfig.scala | 10 ++ 4 files changed, 187 insertions(+), 1 deletion(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 4107844f322d..ccb124b61313 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -57,6 +57,7 @@ private object CHRuleApi { injector.injectResolutionRule(spark => new RewriteToDateExpresstionRule(spark)) injector.injectResolutionRule(spark => new RewriteDateTimestampComparisonRule(spark)) injector.injectOptimizerRule(spark => new CommonSubexpressionEliminateRule(spark)) + injector.injectOptimizerRule(spark => new ExtendedGeneratorNestedColumnAliasing(spark)) injector.injectOptimizerRule(spark => CHAggregateFunctionRewriteRule(spark)) injector.injectOptimizerRule(_ => CountDistinctWithoutExpand) injector.injectOptimizerRule(_ => EqualToRewrite) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala new file mode 100644 index 000000000000..a97e625ae618 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing.canPruneGenerator +import org.apache.spark.sql.catalyst.optimizer.NestedColumnAliasing +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.AlwaysProcess +import org.apache.spark.sql.internal.SQLConf + +// ExtendedGeneratorNestedColumnAliasing process Project(Filter(Generate)), +// which is ignored by vanilla spark in optimization rule: ColumnPruning +class ExtendedGeneratorNestedColumnAliasing(spark: SparkSession) + extends Rule[LogicalPlan] + with Logging { + + override def apply(plan: LogicalPlan): LogicalPlan = + plan.transformWithPruning(AlwaysProcess.fn) { + case pj @ Project(projectList, f @ Filter(condition, g: Generate)) + if canPruneGenerator(g.generator) && + GlutenConfig.getConf.enableExtendedGeneratorNestedColumnAliasing && + (SQLConf.get.nestedPruningOnExpressions || SQLConf.get.nestedSchemaPruningEnabled) => + val attrToExtractValues = NestedColumnAliasing.getAttributeToExtractValues( + projectList ++ g.generator.children :+ condition, + Seq.empty) + if (attrToExtractValues.isEmpty) { + pj + } else { + val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput) + val (_, attrToExtractValuesNotOnGenerator) = + attrToExtractValues.partition { + case (attr, _) => + attr.references.subsetOf(generatorOutputSet) + } + + val pushedThrough = rewritePlanWithAliases(pj, attrToExtractValuesNotOnGenerator) + pushedThrough + } + case p => + p + } + + private def rewritePlanWithAliases( + plan: LogicalPlan, + attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = { + val attributeToExtractValuesAndAliases = + attributeToExtractValues.map { + case (attr, evSeq) => + val evAliasSeq = evSeq.map { + ev => + val fieldName = ev match { + case g: GetStructField => g.extractFieldName + case g: GetArrayStructFields => g.field.name + } + ev -> Alias(ev, s"_extract_$fieldName")() + } + + attr -> evAliasSeq + } + + val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.map { + case (field, alias) => field.canonicalized -> alias + }.toMap + + // A reference attribute can have multiple aliases for nested fields. + val attrToAliases = + AttributeMap(attributeToExtractValuesAndAliases.mapValues(_.map(_._2)).toSeq) + + plan match { + // Project(Filter(Generate)) + case p @ Project(projectList, child) + if child + .isInstanceOf[Filter] && child.asInstanceOf[Filter].child.isInstanceOf[Generate] => + val f = child.asInstanceOf[Filter] + val g = f.child.asInstanceOf[Generate] + + val newProjectList = NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias) + val newCondition = getNewExpression(f.condition, nestedFieldToAlias) + val newGenerator = getNewExpression(g.generator, nestedFieldToAlias).asInstanceOf[Generator] + + val tmpG = NestedColumnAliasing + .replaceWithAliases(g, nestedFieldToAlias, attrToAliases) + .asInstanceOf[Generate] + val newG = Generate( + newGenerator, + tmpG.unrequiredChildIndex, + tmpG.outer, + tmpG.qualifier, + tmpG.generatorOutput, + tmpG.children.head) + val newF = Filter(newCondition, newG) + val newP = Project(newProjectList, newF) + newP + case _ => plan + } + } + + private def getNewExpression( + expr: Expression, + nestedFieldToAlias: Map[Expression, Alias]): Expression = { + expr.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) => + nestedFieldToAlias(f.canonicalized).toAttribute + } + } +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index 8d311614c7de..ff2d13996dc6 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.hive import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.{GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport} +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport} import org.apache.gluten.test.AllDataTypesWithComplexType import org.apache.gluten.utils.UTSystemParameters @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType import org.apache.hadoop.fs.Path @@ -1450,4 +1451,52 @@ class GlutenClickHouseHiveTableSuite spark.sql("DROP TABLE test_tbl_7054") } + test("Nested column pruning for Project(Filter(Generate))") { + spark.sql("drop table if exists aj") + spark.sql( + """ + |CREATE TABLE if not exists aj ( + | country STRING, + | event STRUCT, event_id:STRING, event_info:MAP> + |) + |USING orc + """.stripMargin) + + spark.sql(""" + |INSERT INTO aj VALUES + | ('USA', named_struct('time', 1622547800, 'lng', -122, 'lat', 37, 'net', + | 'wifi', 'log_extra', map('key1', 'value1'), 'event_id', 'event1', + | 'event_info', map('tab_type', '5', 'action', '13'))), + | ('Canada', named_struct('time', 1622547801, 'lng', -79, 'lat', 43, 'net', + | '4g', 'log_extra', map('key2', 'value2'), 'event_id', 'event2', + | 'event_info', map('tab_type', '4', 'action', '12'))) + """.stripMargin) + + val df = + spark.sql(""" + | SELECT * FROM ( + | SELECT + | game_name, + | CASE WHEN + | event.event_info['tab_type'] IN (5) THEN '1' ELSE '0' END AS entrance + | FROM aj + | LATERAL VIEW explode(split(nvl(event.event_info['game_name'],'0'),',')) + | as game_name + | WHERE event.event_info['action'] IN (13) + |) WHERE game_name = 'xxx' + """.stripMargin) + + val scan = df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + + val schema = scan.schema + assert(schema.size == 1) + val fieldType = schema.fields.head.dataType.asInstanceOf[StructType] + assert(fieldType.size == 1) + + spark.sql("drop table if exists aj") + } + } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 3e491eb2753f..6579e30dab8a 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -107,6 +107,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableCountDistinctWithoutExpand: Boolean = conf.getConf(ENABLE_COUNT_DISTINCT_WITHOUT_EXPAND) + def enableExtendedGeneratorNestedColumnAliasing: Boolean = + conf.getConf(ENABLE_EXTENDED_GENERATOR_NESTED_COLUMN_ALIASING) + def veloxOrcScanEnabled: Boolean = conf.getConf(VELOX_ORC_SCAN_ENABLED) @@ -1929,6 +1932,13 @@ object GlutenConfig { .booleanConf .createWithDefault(false) + val ENABLE_EXTENDED_GENERATOR_NESTED_COLUMN_ALIASING = + buildConf("spark.gluten.sql.extendedGeneratorNestedColumnAliasing") + .internal() + .doc("Do nested column aliasing for Project(Filter(Generator))") + .booleanConf + .createWithDefault(true) + val COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS = buildConf("spark.gluten.sql.columnar.backend.velox.bloomFilter.expectedNumItems") .internal() From f0d54412e831fc2075fdd89d8281599b61ec1844 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 12 Nov 2024 18:09:02 +0800 Subject: [PATCH 075/211] [CORE] Consolidate RewriteSparkPlanRulesManager, AddFallbackTagRule, TransformPreOverrides into a single rule (#7918) --- .../backendsapi/clickhouse/CHRuleApi.scala | 21 +- .../clickhouse/CHSparkPlanExecApi.scala | 15 +- .../FallbackBroadcastHashJoinRules.scala | 1 - .../backendsapi/velox/VeloxRuleApi.scala | 18 +- .../org/apache/gluten/backend/Backend.scala | 4 +- .../extension/GlutenSessionExtensions.scala | 4 +- .../{RuleInjector.scala => Injector.scala} | 6 +- .../apache/gluten/backendsapi/RuleApi.scala | 4 +- .../gluten/backendsapi/SubstraitBackend.scala | 4 +- .../apache/gluten/extension/GlutenPlan.scala | 11 + .../EnsureLocalSortRequirements.scala | 20 +- .../extension/columnar/FallbackRules.scala | 402 ------------------ .../extension/columnar/FallbackTag.scala | 140 ++++++ .../columnar/MiscColumnarRules.scala | 29 -- .../enumerated/EnumeratedTransform.scala | 14 +- .../columnar/enumerated/RasOffload.scala | 2 +- .../columnar/heuristic/AddFallbackTags.scala | 298 +++++++++++++ .../heuristic/HeuristicTransform.scala | 55 +++ .../columnar/heuristic/LegacyOffload.scala | 48 +++ .../{ => heuristic}/OffloadSingleNode.scala | 3 +- .../columnar/rewrite/RewriteJoin.scala | 2 +- .../RewriteSparkPlanRulesManager.scala | 13 +- .../GlutenFormatWriterInjectsBase.scala | 14 +- 23 files changed, 624 insertions(+), 504 deletions(-) rename gluten-core/src/main/scala/org/apache/gluten/extension/injector/{RuleInjector.scala => Injector.scala} (89%) create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala rename gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/{ => heuristic}/OffloadSingleNode.scala (99%) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index ccb124b61313..a1b8fbdd5bc3 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -19,10 +19,10 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.backendsapi.RuleApi import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides} -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} -import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector} +import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.parser.{GlutenCacheFilesSqlParser, GlutenClickhouseSqlParser} import org.apache.gluten.sql.shims.SparkShimLoader @@ -31,14 +31,14 @@ import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRew import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.delta.DeltaLogFileIndex import org.apache.spark.sql.delta.rules.CHOptimizeMetadataOnlyDeltaQuery -import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, CommandResultExec, FileSourceScanExec, GlutenFallbackReporter, RDDScanExec, SparkPlan} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.util.SparkPlanRules class CHRuleApi extends RuleApi { import CHRuleApi._ - override def injectRules(injector: RuleInjector): Unit = { + override def injectRules(injector: Injector): Unit = { injectSpark(injector.spark) injectLegacy(injector.gluten.legacy) injectRas(injector.gluten.ras) @@ -65,8 +65,7 @@ private object CHRuleApi { } def injectLegacy(injector: LegacyInjector): Unit = { - - // Gluten columnar: Transform rules. + // Legacy: Pre-transform rules. injector.injectTransform(_ => RemoveTransitions) injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) @@ -74,9 +73,11 @@ private object CHRuleApi { injector.injectTransform(_ => RewriteSubqueryBroadcast()) injector.injectTransform(c => FallbackBroadcastHashJoin.apply(c.session)) injector.injectTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session)) - injector.injectTransform(_ => intercept(RewriteSparkPlanRulesManager())) - injector.injectTransform(_ => intercept(AddFallbackTagRule())) - injector.injectTransform(_ => intercept(TransformPreOverrides())) + + // Legacy: The Legacy transform rule. + injector.injectTransform(_ => intercept(HeuristicTransform())) + + // Legacy: Post-transform rules. injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectTransform(c => intercept(RewriteTransformer.apply(c.session))) injector.injectTransform(_ => PushDownFilterToScan) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index 190fcb13eaeb..f08f21056d14 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -23,8 +23,7 @@ import org.apache.gluten.execution._ import org.apache.gluten.expression._ import org.apache.gluten.expression.ExpressionNames.MONOTONICALLY_INCREASING_ID import org.apache.gluten.extension.ExpressionExtensionTrait -import org.apache.gluten.extension.columnar.AddFallbackTagRule -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode, WindowFunctionNode} import org.apache.gluten.utils.{CHJoinValidateUtil, UnknownJoinStrategy} @@ -224,9 +223,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { } // FIXME: The operation happens inside ReplaceSingleNode(). // Caller may not know it adds project on top of the shuffle. - val project = TransformPreOverrides().apply( - AddFallbackTagRule().apply( - ProjectExec(plan.child.output ++ projectExpressions, plan.child))) + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val project = + HeuristicTransform()(ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newExprs = Seq[Expression]() for (i <- exprs.indices) { val pos = newExpressionsPosition(i) @@ -249,9 +248,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { } // FIXME: The operation happens inside ReplaceSingleNode(). // Caller may not know it adds project on top of the shuffle. - val project = TransformPreOverrides().apply( - AddFallbackTagRule().apply( - ProjectExec(plan.child.output ++ projectExpressions, plan.child))) + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val project = + HeuristicTransform()(ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newOrderings = Seq[SortOrder]() for (i <- orderings.indices) { val oldOrdering = orderings(i) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala index 207bb0e3a4d7..d1116c857ab6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala @@ -19,7 +19,6 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.FallbackTags.EncodeFallbackTagImplicits import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 3554bc5c9c01..30e92a18b23c 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -20,11 +20,11 @@ import org.apache.gluten.backendsapi.RuleApi import org.apache.gluten.datasource.ArrowConvertorRule import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides} +import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} -import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector} +import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.sql.shims.SparkShimLoader @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFa class VeloxRuleApi extends RuleApi { import VeloxRuleApi._ - override def injectRules(injector: RuleInjector): Unit = { + override def injectRules(injector: Injector): Unit = { injectSpark(injector.spark) injectLegacy(injector.gluten.legacy) injectRas(injector.gluten.ras) @@ -49,7 +49,7 @@ private object VeloxRuleApi { } def injectLegacy(injector: LegacyInjector): Unit = { - // Gluten columnar: Transform rules. + // Legacy: Pre-transform rules. injector.injectTransform(_ => RemoveTransitions) injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) @@ -57,9 +57,11 @@ private object VeloxRuleApi { injector.injectTransform(_ => RewriteSubqueryBroadcast()) injector.injectTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) injector.injectTransform(c => ArrowScanReplaceRule.apply(c.session)) - injector.injectTransform(_ => RewriteSparkPlanRulesManager()) - injector.injectTransform(_ => AddFallbackTagRule()) - injector.injectTransform(_ => TransformPreOverrides()) + + // Legacy: The Legacy transform rule. + injector.injectTransform(_ => HeuristicTransform()) + + // Legacy: Post-transform rules. injector.injectTransform(c => PartialProjectRule.apply(c.session)) injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectTransform(c => RewriteTransformer.apply(c.session)) diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala index ef22d97e773b..346181e140ac 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backend import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector import org.apache.spark.SparkContext import org.apache.spark.api.plugin.PluginContext @@ -51,7 +51,7 @@ trait Backend { def convFuncOverride(): ConventionFunc.Override = ConventionFunc.Override.Empty /** Query planner rules. */ - def injectRules(injector: RuleInjector): Unit + def injectRules(injector: Injector): Unit } object Backend { diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala index d5afc6b7e7b0..addcad8dd05c 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala @@ -18,7 +18,7 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSessionExtensions @@ -28,7 +28,7 @@ private[gluten] class GlutenSessionExtensions with Logging { import GlutenSessionExtensions._ override def apply(exts: SparkSessionExtensions): Unit = { - val injector = new RuleInjector(exts) + val injector = new Injector(exts) injector.control.disableOn { session => val glutenEnabledGlobally = session.conf diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala similarity index 89% rename from gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala index c497a24a07b7..81b5239a8aca 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala @@ -18,8 +18,8 @@ package org.apache.gluten.extension.injector import org.apache.spark.sql.SparkSessionExtensions -/** Injector used to inject query planner rules into Spark and Gluten. */ -class RuleInjector(extensions: SparkSessionExtensions) { +/** Injector used to inject extensible components into Spark and Gluten. */ +class Injector(extensions: SparkSessionExtensions) { val control = new InjectorControl() val spark: SparkInjector = new SparkInjector(control, extensions) val gluten: GlutenInjector = new GlutenInjector(control) @@ -31,4 +31,4 @@ class RuleInjector(extensions: SparkSessionExtensions) { } } -object RuleInjector {} +object Injector {} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala index 7c4c8577f421..745bbcb72d61 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala @@ -16,9 +16,9 @@ */ package org.apache.gluten.backendsapi -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector trait RuleApi { // Injects all Spark query planner rules used by the Gluten backend. - def injectRules(injector: RuleInjector): Unit + def injectRules(injector: Injector): Unit } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala index d7785663d577..37be117105e2 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backendsapi import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector import org.apache.spark.SparkContext import org.apache.spark.api.plugin.PluginContext @@ -35,7 +35,7 @@ trait SubstraitBackend extends Backend { final override def onExecutorShutdown(): Unit = { listenerApi().onExecutorShutdown() } - final override def injectRules(injector: RuleInjector): Unit = { + final override def injectRules(injector: Injector): Unit = { ruleApi().injectRules(injector) } def iteratorApi(): IteratorApi diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala index 06d798e50fc0..c252bacc9a18 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala @@ -21,6 +21,7 @@ import org.apache.gluten.backend.Backend import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.expression.TransformerState +import org.apache.gluten.extension.columnar.FallbackTags.add import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.substrait.SubstraitContext @@ -28,6 +29,7 @@ import org.apache.gluten.substrait.plan.PlanBuilder import org.apache.gluten.substrait.rel.RelNode import org.apache.gluten.test.TestStats +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.SparkPlan import com.google.common.collect.Lists @@ -50,6 +52,15 @@ object ValidationResult { def succeeded: ValidationResult = Succeeded def failed(reason: String): ValidationResult = Failed(reason) + + implicit class EncodeFallbackTagImplicits(result: ValidationResult) { + def tagOnFallback(plan: TreeNode[_]): Unit = { + if (result.ok()) { + return + } + add(plan, result) + } + } } /** Every Gluten Operator should extend this trait. */ diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala index ff989d796f8e..73dc617e0853 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala @@ -16,9 +16,7 @@ */ package org.apache.gluten.extension.columnar -import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.rules.Rule @@ -33,24 +31,14 @@ import org.apache.spark.sql.execution.{SortExec, SparkPlan} * SortAggregate with the same key. So, this rule adds local sort back if necessary. */ object EnsureLocalSortRequirements extends Rule[SparkPlan] { - private lazy val offload = TransformPreOverrides.apply() + private lazy val transform = HeuristicTransform() private def addLocalSort( originalChild: SparkPlan, requiredOrdering: Seq[SortOrder]): SparkPlan = { + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. val newChild = SortExec(requiredOrdering, global = false, child = originalChild) - if (!GlutenConfig.getConf.enableColumnarSort) { - FallbackTags.add(newChild, "columnar Sort is not enabled in SortExec") - newChild - } else { - val rewrittenPlan = RewriteSparkPlanRulesManager.apply().apply(newChild) - if (rewrittenPlan.eq(newChild) && FallbackTags.nonEmpty(rewrittenPlan)) { - // The sort can not be offloaded - rewrittenPlan - } else { - offload.apply(rewrittenPlan) - } - } + transform.apply(newChild) } override def apply(plan: SparkPlan): SparkPlan = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala index 1268d36d3290..432ecd1584d8 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala @@ -17,147 +17,13 @@ package org.apache.gluten.extension.columnar import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.execution._ -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} -import org.apache.gluten.extension.columnar.FallbackTags.EncodeFallbackTagImplicits -import org.apache.gluten.extension.columnar.validator.{Validator, Validators} -import org.apache.gluten.sql.shims.SparkShimLoader -import org.apache.spark.api.python.EvalPythonExecTransformer import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, QueryStageExec} -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.datasources.WriteFilesExec -import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} -import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} -import org.apache.spark.sql.hive.HiveTableScanExecTransformer - -import org.apache.commons.lang3.exception.ExceptionUtils - -sealed trait FallbackTag { - val stacktrace: Option[String] = - if (FallbackTags.DEBUG) { - Some(ExceptionUtils.getStackTrace(new Throwable())) - } else None - - def reason(): String -} - -object FallbackTag { - - /** A tag that stores one reason text of fall back. */ - case class Appendable(override val reason: String) extends FallbackTag - - /** - * A tag that stores reason text of fall back. Other reasons will be discarded when this tag is - * added to plan. - */ - case class Exclusive(override val reason: String) extends FallbackTag - - trait Converter[T] { - def from(obj: T): Option[FallbackTag] - } - - object Converter { - implicit def asIs[T <: FallbackTag]: Converter[T] = (tag: T) => Some(tag) - - implicit object FromString extends Converter[String] { - override def from(reason: String): Option[FallbackTag] = Some(Appendable(reason)) - } - - implicit object FromValidationResult extends Converter[ValidationResult] { - override def from(result: ValidationResult): Option[FallbackTag] = { - if (result.ok()) { - return None - } - Some(Appendable(result.reason())) - } - } - } -} - -object FallbackTags { - val TAG: TreeNodeTag[FallbackTag] = - TreeNodeTag[FallbackTag]("org.apache.gluten.FallbackTag") - - val DEBUG = false - - /** - * If true, the plan node will be guaranteed fallback to Vanilla plan node while being - * implemented. - * - * If false, the plan still has chance to be turned into "non-transformable" in any another - * validation rule. So user should not consider the plan "transformable" unless all validation - * rules are passed. - */ - def nonEmpty(plan: SparkPlan): Boolean = { - getOption(plan).nonEmpty - } - - /** - * If true, it implies the plan maybe transformable during validation phase but not guaranteed, - * since another validation rule could turn it to "non-transformable" before implementing the plan - * within Gluten transformers. If false, the plan node will be guaranteed fallback to Vanilla plan - * node while being implemented. - */ - def maybeOffloadable(plan: SparkPlan): Boolean = !nonEmpty(plan) - - def add[T](plan: TreeNode[_], t: T)(implicit converter: FallbackTag.Converter[T]): Unit = { - val tagOption = getOption(plan) - val newTagOption = converter.from(t) - - val mergedTagOption: Option[FallbackTag] = - (tagOption ++ newTagOption).reduceOption[FallbackTag] { - // New tag comes while the plan was already tagged, merge. - case (_, exclusive: FallbackTag.Exclusive) => - exclusive - case (exclusive: FallbackTag.Exclusive, _) => - exclusive - case (l: FallbackTag.Appendable, r: FallbackTag.Appendable) => - FallbackTag.Appendable(s"${l.reason}; ${r.reason}") - } - mergedTagOption - .foreach(mergedTag => plan.setTagValue(TAG, mergedTag)) - } - - def addRecursively[T](plan: TreeNode[_], t: T)(implicit - converter: FallbackTag.Converter[T]): Unit = { - plan.foreach { - case _: GlutenPlan => // ignore - case other: TreeNode[_] => add(other, t) - } - } - - def untag(plan: TreeNode[_]): Unit = { - plan.unsetTagValue(TAG) - } - - def get(plan: TreeNode[_]): FallbackTag = { - getOption(plan).getOrElse( - throw new IllegalStateException("Transform hint tag not set in plan: " + plan.toString())) - } - - def getOption(plan: TreeNode[_]): Option[FallbackTag] = { - plan.getTagValue(TAG) - } - - implicit class EncodeFallbackTagImplicits(result: ValidationResult) { - def tagOnFallback(plan: TreeNode[_]): Unit = { - if (result.ok()) { - return - } - add(plan, result) - } - } -} case class FallbackOnANSIMode(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { @@ -237,271 +103,3 @@ case class FallbackMultiCodegens(session: SparkSession) extends Rule[SparkPlan] } else plan } } - -// This rule will try to convert a plan into plan transformer. -// The doValidate function will be called to check if the conversion is supported. -// If false is returned or any unsupported exception is thrown, a row guard will -// be added on the top of that plan to prevent actual conversion. -case class AddFallbackTagRule() extends Rule[SparkPlan] { - import AddFallbackTagRule._ - private val glutenConf: GlutenConfig = GlutenConfig.getConf - private val validator = Validators - .builder() - .fallbackByHint() - .fallbackIfScanOnlyWithFilterPushed(glutenConf.enableScanOnly) - .fallbackComplexExpressions() - .fallbackByBackendSettings() - .fallbackByUserOptions() - .fallbackByTestInjects() - .build() - - def apply(plan: SparkPlan): SparkPlan = { - plan.foreachUp { case p => addFallbackTag(p) } - plan - } - - private def addFallbackTag(plan: SparkPlan): Unit = { - val outcome = validator.validate(plan) - outcome match { - case Validator.Failed(reason) => - FallbackTags.add(plan, reason) - return - case Validator.Passed => - } - - try { - plan match { - case plan: BatchScanExec => - val transformer = - ScanTransformerFactory - .createBatchScanTransformer(plan, validation = true) - .asInstanceOf[BasicScanExecTransformer] - transformer.doValidate().tagOnFallback(plan) - case plan: FileSourceScanExec => - val transformer = - ScanTransformerFactory.createFileSourceScanTransformer(plan) - transformer.doValidate().tagOnFallback(plan) - case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - HiveTableScanExecTransformer.validate(plan).tagOnFallback(plan) - case plan: ProjectExec => - val transformer = ProjectExecTransformer(plan.projectList, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: FilterExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genFilterExecTransformer(plan.condition, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: HashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: SortAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: ObjectHashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: UnionExec => - val transformer = ColumnarUnionExec(plan.children) - transformer.doValidate().tagOnFallback(plan) - case plan: ExpandExec => - val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: WriteFilesExec => - val transformer = WriteFilesExecTransformer( - plan.child, - plan.fileFormat, - plan.partitionColumns, - plan.bucketSpec, - plan.options, - plan.staticPartitions) - transformer.doValidate().tagOnFallback(plan) - case plan: SortExec => - val transformer = - SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffleExchangeExec => - val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffledHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genShuffledHashJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - OffloadJoin.getShjBuildSide(plan), - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastExchangeExec => - val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) - transformer.doValidate().tagOnFallback(plan) - case bhj: BroadcastHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastHashJoinExecTransformer( - bhj.leftKeys, - bhj.rightKeys, - bhj.joinType, - bhj.buildSide, - bhj.condition, - bhj.left, - bhj.right, - isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: SortMergeJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genSortMergeJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: CartesianProductExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastNestedLoopJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastNestedLoopJoinExecTransformer( - plan.left, - plan.right, - plan.buildSide, - plan.joinType, - plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: WindowExec => - val transformer = WindowExecTransformer( - plan.windowExpression, - plan.partitionSpec, - plan.orderSpec, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => - val windowGroupLimitPlan = SparkShimLoader.getSparkShims - .getWindowGroupLimitExecShim(plan) - .asInstanceOf[WindowGroupLimitExecShim] - val transformer = WindowGroupLimitExecTransformer( - windowGroupLimitPlan.partitionSpec, - windowGroupLimitPlan.orderSpec, - windowGroupLimitPlan.rankLikeFunction, - windowGroupLimitPlan.limit, - windowGroupLimitPlan.mode, - windowGroupLimitPlan.child - ) - transformer.doValidate().tagOnFallback(plan) - case plan: CoalesceExec => - ColumnarCoalesceExec(plan.numPartitions, plan.child) - .doValidate() - .tagOnFallback(plan) - case plan: GlobalLimitExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) - val transformer = LimitExecTransformer(plan.child, offset, limit) - transformer.doValidate().tagOnFallback(plan) - case plan: LocalLimitExec => - val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) - transformer.doValidate().tagOnFallback(plan) - case plan: GenerateExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( - plan.generator, - plan.requiredChildOutput, - plan.outer, - plan.generatorOutput, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: BatchEvalPythonExec => - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: ArrowEvalPythonExec => - // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not - // enabled, we will try offloading through EvalPythonExecTransformer - if ( - !BackendsApiManager.getSettings.supportColumnarArrowUdf() || - !GlutenConfig.getConf.enableColumnarArrowUDF - ) { - // Both CH and Velox will try using backend's built-in functions for calculate - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - } - case plan: TakeOrderedAndProjectExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) - val transformer = TakeOrderedAndProjectExecTransformer( - limit, - plan.sortOrder, - plan.projectList, - plan.child, - offset) - transformer.doValidate().tagOnFallback(plan) - case plan: SampleExec => - val transformer = - BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( - plan.lowerBound, - plan.upperBound, - plan.withReplacement, - plan.seed, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case _ => - // Currently we assume a plan to be offload-able by default. - } - } catch { - case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => - FallbackTags.add( - plan, - s"${e.getMessage}, original Spark plan is " + - s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") - if (!e.isInstanceOf[GlutenNotSupportException]) { - logDebug("Just a warning. This exception perhaps needs to be fixed.", e) - } - } - } -} - -object AddFallbackTagRule { - implicit private class ValidatorBuilderImplicits(builder: Validators.Builder) { - - /** - * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes - * validation on filter for the exception that filter + scan is detected. Because filters can be - * pushed into scan then the filter conditions will be processed only in scan. - */ - def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validators.Builder = { - builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) - builder - } - } - - private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { - override def validate(plan: SparkPlan): Validator.OutCome = { - if (!scanOnly) { - return pass() - } - // Scan-only mode - plan match { - case _: BatchScanExec => pass() - case _: FileSourceScanExec => pass() - case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() - case filter: FilterExec => - val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || - filter.child.isInstanceOf[BatchScanExec] - if (childIsScan) { - pass() - } else { - fail(filter) - } - case other => fail(other) - } - } - } -} - -case class RemoveFallbackTagRule() extends Rule[SparkPlan] { - override def apply(plan: SparkPlan): SparkPlan = { - plan.foreach(FallbackTags.untag) - plan - } -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala new file mode 100644 index 000000000000..c2d844c19bc0 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar + +import org.apache.gluten.extension.{GlutenPlan, ValidationResult} + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} +import org.apache.spark.sql.execution.SparkPlan + +import org.apache.commons.lang3.exception.ExceptionUtils + +sealed trait FallbackTag { + val stacktrace: Option[String] = + if (FallbackTags.DEBUG) { + Some(ExceptionUtils.getStackTrace(new Throwable())) + } else None + + def reason(): String +} + +object FallbackTag { + + /** A tag that stores one reason text of fall back. */ + case class Appendable(override val reason: String) extends FallbackTag + + /** + * A tag that stores reason text of fall back. Other reasons will be discarded when this tag is + * added to plan. + */ + case class Exclusive(override val reason: String) extends FallbackTag + + trait Converter[T] { + def from(obj: T): Option[FallbackTag] + } + + object Converter { + implicit def asIs[T <: FallbackTag]: Converter[T] = (tag: T) => Some(tag) + + implicit object FromString extends Converter[String] { + override def from(reason: String): Option[FallbackTag] = Some(Appendable(reason)) + } + + implicit object FromValidationResult extends Converter[ValidationResult] { + override def from(result: ValidationResult): Option[FallbackTag] = { + if (result.ok()) { + return None + } + Some(Appendable(result.reason())) + } + } + } +} + +object FallbackTags { + val TAG: TreeNodeTag[FallbackTag] = + TreeNodeTag[FallbackTag]("org.apache.gluten.FallbackTag") + + val DEBUG = false + + /** + * If true, the plan node will be guaranteed fallback to Vanilla plan node while being + * implemented. + * + * If false, the plan still has chance to be turned into "non-transformable" in any another + * validation rule. So user should not consider the plan "transformable" unless all validation + * rules are passed. + */ + def nonEmpty(plan: SparkPlan): Boolean = { + getOption(plan).nonEmpty + } + + /** + * If true, it implies the plan maybe transformable during validation phase but not guaranteed, + * since another validation rule could turn it to "non-transformable" before implementing the plan + * within Gluten transformers. If false, the plan node will be guaranteed fallback to Vanilla plan + * node while being implemented. + */ + def maybeOffloadable(plan: SparkPlan): Boolean = !nonEmpty(plan) + + def add[T](plan: TreeNode[_], t: T)(implicit converter: FallbackTag.Converter[T]): Unit = { + val tagOption = getOption(plan) + val newTagOption = converter.from(t) + + val mergedTagOption: Option[FallbackTag] = + (tagOption ++ newTagOption).reduceOption[FallbackTag] { + // New tag comes while the plan was already tagged, merge. + case (_, exclusive: FallbackTag.Exclusive) => + exclusive + case (exclusive: FallbackTag.Exclusive, _) => + exclusive + case (l: FallbackTag.Appendable, r: FallbackTag.Appendable) => + FallbackTag.Appendable(s"${l.reason}; ${r.reason}") + } + mergedTagOption + .foreach(mergedTag => plan.setTagValue(TAG, mergedTag)) + } + + def addRecursively[T](plan: TreeNode[_], t: T)(implicit + converter: FallbackTag.Converter[T]): Unit = { + plan.foreach { + case _: GlutenPlan => // ignore + case other: TreeNode[_] => add(other, t) + } + } + + def untag(plan: TreeNode[_]): Unit = { + plan.unsetTagValue(TAG) + } + + def get(plan: TreeNode[_]): FallbackTag = { + getOption(plan).getOrElse( + throw new IllegalStateException("Transform hint tag not set in plan: " + plan.toString())) + } + + def getOption(plan: TreeNode[_]): Option[FallbackTag] = { + plan.getTagValue(TAG) + } +} + +case class RemoveFallbackTagRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + plan.foreach(FallbackTags.untag) + plan + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala index 11b4b8650842..a199b5920ca4 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala @@ -17,7 +17,6 @@ package org.apache.gluten.extension.columnar import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, Transitions} -import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.SparkSession @@ -32,34 +31,6 @@ import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.internal.SQLConf object MiscColumnarRules { - object TransformPreOverrides { - def apply(): TransformPreOverrides = { - TransformPreOverrides( - List(), - List( - OffloadOthers(), - OffloadExchange(), - OffloadJoin() - ) - ) - } - } - - // This rule will conduct the conversion from Spark plan to the plan transformer. - case class TransformPreOverrides( - topDownRules: Seq[OffloadSingleNode], - bottomUpRules: Seq[OffloadSingleNode]) - extends Rule[SparkPlan] - with LogLevelUtil { - - def apply(plan: SparkPlan): SparkPlan = { - val plan0 = - topDownRules.foldLeft(plan)((p, rule) => p.transformDown { case p => rule.offload(p) }) - val plan1 = - bottomUpRules.foldLeft(plan0)((p, rule) => p.transformUp { case p => rule.offload(p) }) - plan1 - } - } // Replaces all SubqueryBroadcastExec used by sub-queries with ColumnarSubqueryBroadcastExec. // This prevents query execution from being failed by fallen-back SubqueryBroadcastExec with diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index e6d1c4859ed4..5abc64ec374f 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.enumerated -import org.apache.gluten.extension.columnar.{OffloadExchange, OffloadJoin, OffloadOthers} +import org.apache.gluten.extension.columnar.heuristic.{OffloadExchange, OffloadJoin, OffloadOthers} import org.apache.gluten.extension.columnar.transition.ConventionReq import org.apache.gluten.extension.columnar.validator.{Validator, Validators} import org.apache.gluten.logging.LogLevelUtil @@ -38,6 +38,18 @@ import org.apache.spark.sql.execution.python.EvalPythonExec import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.hive.HiveTableScanExecTransformer +/** + * Rule to offload Spark query plan to Gluten query plan using a search algorithm and a defined cost + * model. + * + * The effect of this rule is similar to + * [[org.apache.gluten.extension.columnar.heuristic.HeuristicTransform]], except that the 3 stages + * in the heuristic version, known as rewrite, validate, offload, will take place together + * individually for each Spark query plan node in RAS rule + * [[org.apache.gluten.extension.columnar.enumerated.RasOffload]]. + * + * The feature requires enabling RAS to function. + */ case class EnumeratedTransform(session: SparkSession, outputsColumnar: Boolean) extends Rule[SparkPlan] with LogLevelUtil { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala index 52798f712c12..1e97ebfbab85 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.enumerated import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.OffloadSingleNode +import org.apache.gluten.extension.columnar.heuristic.OffloadSingleNode import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode import org.apache.gluten.extension.columnar.validator.Validator import org.apache.gluten.ras.path.Pattern diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala new file mode 100644 index 000000000000..538132c81a2e --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.exception.GlutenNotSupportException +import org.apache.gluten.execution.{BasicScanExecTransformer, ColumnarCoalesceExec, ColumnarUnionExec, ExpandExecTransformer, HashAggregateExecBaseTransformer, LimitExecTransformer, ProjectExecTransformer, ScanTransformerFactory, SortExecTransformer, TakeOrderedAndProjectExecTransformer, WindowExecTransformer, WindowGroupLimitExecTransformer, WriteFilesExecTransformer} +import org.apache.gluten.extension.columnar.FallbackTags +import org.apache.gluten.extension.columnar.validator.{Validator, Validators} +import org.apache.gluten.sql.shims.SparkShimLoader + +import org.apache.spark.api.python.EvalPythonExecTransformer +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{CoalesceExec, ColumnarBroadcastExchangeExec, ColumnarShuffleExchangeExec, ExpandExec, FileSourceScanExec, FilterExec, GenerateExec, GlobalLimitExec, LocalLimitExec, ProjectExec, SampleExec, SortExec, SparkPlan, TakeOrderedAndProjectExec, UnionExec} +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.execution.datasources.WriteFilesExec +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, CartesianProductExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} +import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} +import org.apache.spark.sql.hive.HiveTableScanExecTransformer + +// This rule will try to convert a plan into plan transformer. +// The doValidate function will be called to check if the conversion is supported. +// If false is returned or any unsupported exception is thrown, a row guard will +// be added on the top of that plan to prevent actual conversion. +case class AddFallbackTags() extends Rule[SparkPlan] { + import AddFallbackTags._ + private val glutenConf: GlutenConfig = GlutenConfig.getConf + private val validator = Validators + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(glutenConf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .build() + + def apply(plan: SparkPlan): SparkPlan = { + plan.foreachUp { case p => addFallbackTag(p) } + plan + } + + private def addFallbackTag(plan: SparkPlan): Unit = { + val outcome = validator.validate(plan) + outcome match { + case Validator.Failed(reason) => + FallbackTags.add(plan, reason) + return + case Validator.Passed => + } + + try { + plan match { + case plan: BatchScanExec => + val transformer = + ScanTransformerFactory + .createBatchScanTransformer(plan, validation = true) + .asInstanceOf[BasicScanExecTransformer] + transformer.doValidate().tagOnFallback(plan) + case plan: FileSourceScanExec => + val transformer = + ScanTransformerFactory.createFileSourceScanTransformer(plan) + transformer.doValidate().tagOnFallback(plan) + case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => + HiveTableScanExecTransformer.validate(plan).tagOnFallback(plan) + case plan: ProjectExec => + val transformer = ProjectExecTransformer(plan.projectList, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: FilterExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genFilterExecTransformer(plan.condition, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: HashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().tagOnFallback(plan) + case plan: SortAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().tagOnFallback(plan) + case plan: ObjectHashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().tagOnFallback(plan) + case plan: UnionExec => + val transformer = ColumnarUnionExec(plan.children) + transformer.doValidate().tagOnFallback(plan) + case plan: ExpandExec => + val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: WriteFilesExec => + val transformer = WriteFilesExecTransformer( + plan.child, + plan.fileFormat, + plan.partitionColumns, + plan.bucketSpec, + plan.options, + plan.staticPartitions) + transformer.doValidate().tagOnFallback(plan) + case plan: SortExec => + val transformer = + SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) + transformer.doValidate().tagOnFallback(plan) + case plan: ShuffleExchangeExec => + val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) + transformer.doValidate().tagOnFallback(plan) + case plan: ShuffledHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genShuffledHashJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + OffloadJoin.getShjBuildSide(plan), + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().tagOnFallback(plan) + case plan: BroadcastExchangeExec => + val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) + transformer.doValidate().tagOnFallback(plan) + case bhj: BroadcastHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastHashJoinExecTransformer( + bhj.leftKeys, + bhj.rightKeys, + bhj.joinType, + bhj.buildSide, + bhj.condition, + bhj.left, + bhj.right, + isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) + transformer.doValidate().tagOnFallback(plan) + case plan: SortMergeJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genSortMergeJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().tagOnFallback(plan) + case plan: CartesianProductExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) + transformer.doValidate().tagOnFallback(plan) + case plan: BroadcastNestedLoopJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastNestedLoopJoinExecTransformer( + plan.left, + plan.right, + plan.buildSide, + plan.joinType, + plan.condition) + transformer.doValidate().tagOnFallback(plan) + case plan: WindowExec => + val transformer = WindowExecTransformer( + plan.windowExpression, + plan.partitionSpec, + plan.orderSpec, + plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => + val windowGroupLimitPlan = SparkShimLoader.getSparkShims + .getWindowGroupLimitExecShim(plan) + .asInstanceOf[WindowGroupLimitExecShim] + val transformer = WindowGroupLimitExecTransformer( + windowGroupLimitPlan.partitionSpec, + windowGroupLimitPlan.orderSpec, + windowGroupLimitPlan.rankLikeFunction, + windowGroupLimitPlan.limit, + windowGroupLimitPlan.mode, + windowGroupLimitPlan.child + ) + transformer.doValidate().tagOnFallback(plan) + case plan: CoalesceExec => + ColumnarCoalesceExec(plan.numPartitions, plan.child) + .doValidate() + .tagOnFallback(plan) + case plan: GlobalLimitExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) + val transformer = LimitExecTransformer(plan.child, offset, limit) + transformer.doValidate().tagOnFallback(plan) + case plan: LocalLimitExec => + val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) + transformer.doValidate().tagOnFallback(plan) + case plan: GenerateExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( + plan.generator, + plan.requiredChildOutput, + plan.outer, + plan.generatorOutput, + plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: BatchEvalPythonExec => + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: ArrowEvalPythonExec => + // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not + // enabled, we will try offloading through EvalPythonExecTransformer + if ( + !BackendsApiManager.getSettings.supportColumnarArrowUdf() || + !GlutenConfig.getConf.enableColumnarArrowUDF + ) { + // Both CH and Velox will try using backend's built-in functions for calculate + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().tagOnFallback(plan) + } + case plan: TakeOrderedAndProjectExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) + val transformer = TakeOrderedAndProjectExecTransformer( + limit, + plan.sortOrder, + plan.projectList, + plan.child, + offset) + transformer.doValidate().tagOnFallback(plan) + case plan: SampleExec => + val transformer = + BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( + plan.lowerBound, + plan.upperBound, + plan.withReplacement, + plan.seed, + plan.child) + transformer.doValidate().tagOnFallback(plan) + case _ => + // Currently we assume a plan to be offload-able by default. + } + } catch { + case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => + FallbackTags.add( + plan, + s"${e.getMessage}, original Spark plan is " + + s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") + if (!e.isInstanceOf[GlutenNotSupportException]) { + logDebug("Just a warning. This exception perhaps needs to be fixed.", e) + } + } + } +} + +object AddFallbackTags { + implicit private class ValidatorBuilderImplicits(builder: Validators.Builder) { + + /** + * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes + * validation on filter for the exception that filter + scan is detected. Because filters can be + * pushed into scan then the filter conditions will be processed only in scan. + */ + def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validators.Builder = { + builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) + builder + } + } + + private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { + override def validate(plan: SparkPlan): Validator.OutCome = { + if (!scanOnly) { + return pass() + } + // Scan-only mode + plan match { + case _: BatchScanExec => pass() + case _: FileSourceScanExec => pass() + case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() + case filter: FilterExec => + val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || + filter.child.isInstanceOf[BatchScanExec] + if (childIsScan) { + pass() + } else { + fail(filter) + } + case other => fail(other) + } + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala new file mode 100644 index 000000000000..8a026326e5de --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.logging.LogLevelUtil + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +/** + * Rule to offload Spark query plan to Gluten query plan using programed heuristics. + * + * The procedure consists of 3 stages: + * + * 1. Rewrite 2. Validate 3. Offload + * + * In the rewrite stage, planner will try converting the Spark query plan to various forms of + * possible alternative Spark query plans, then choose the optimal one to send to next stage. During + * which, the same validation code that is about to be used in stage 2 might be invoked early to + * predict on the estimate "cost" of an alternative Spark query plan. + * + * Once the plan is rewritten, query planner will call native validation code in stage 2 to + * determine which part of the plan is offload-able or not, then add fallback tags to the + * non-offload-able parts. + * + * In stage 3, query planner will convert the offload-able Spark plan nodes into Gluten plan nodes. + */ +case class HeuristicTransform() extends Rule[SparkPlan] with LogLevelUtil { + + private val validate = AddFallbackTags() + private val rewrite = RewriteSparkPlanRulesManager(validate) + private val offload = LegacyOffload() + + override def apply(plan: SparkPlan): SparkPlan = { + Seq(rewrite, validate, offload).foldLeft(plan) { + case (plan, stage) => + stage(plan) + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala new file mode 100644 index 000000000000..9cf08ff303cd --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.logging.LogLevelUtil + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +class LegacyOffload(topDownRules: Seq[OffloadSingleNode], bottomUpRules: Seq[OffloadSingleNode]) + extends Rule[SparkPlan] + with LogLevelUtil { + + def apply(plan: SparkPlan): SparkPlan = { + val plan0 = + topDownRules.foldLeft(plan)((p, rule) => p.transformDown { case p => rule.offload(p) }) + val plan1 = + bottomUpRules.foldLeft(plan0)((p, rule) => p.transformUp { case p => rule.offload(p) }) + plan1 + } +} + +object LegacyOffload { + def apply(): LegacyOffload = { + new LegacyOffload( + List(), + List( + OffloadOthers(), + OffloadExchange(), + OffloadJoin() + ) + ) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala similarity index 99% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala index 220a45f7b35c..a8c200e9be44 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala @@ -14,13 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar +package org.apache.gluten.extension.columnar.heuristic import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.execution._ import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.sql.shims.SparkShimLoader diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala index d0cac0b29deb..5fcf37439772 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.rewrite import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.columnar.OffloadJoin +import org.apache.gluten.extension.columnar.heuristic.OffloadJoin import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper} import org.apache.spark.sql.catalyst.plans.logical.Join diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala index c6d5e1cca1f7..3b8ca0df3001 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.rewrite -import org.apache.gluten.extension.columnar.{AddFallbackTagRule, FallbackTag, FallbackTags} +import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.rdd.RDD @@ -45,7 +45,9 @@ case class RewrittenNodeWall(originalChild: SparkPlan) extends LeafExecNode { * * Note that, this rule does not touch and tag these operators who does not need to rewrite. */ -class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode]) +class RewriteSparkPlanRulesManager private ( + validateRule: Rule[SparkPlan], + rewriteRules: Seq[RewriteSingleNode]) extends Rule[SparkPlan] { private def mayNeedRewrite(plan: SparkPlan): Boolean = { @@ -96,7 +98,6 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] } override def apply(plan: SparkPlan): SparkPlan = { - val addHint = AddFallbackTagRule() plan.transformUp { case origin if mayNeedRewrite(origin) => // Add a wall to avoid transforming unnecessary nodes. @@ -114,7 +115,7 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] // We do not add tag and leave it to the outside `AddFallbackTagRule`. origin } else { - addHint.apply(rewrittenPlan) + validateRule.apply(rewrittenPlan) val tag = getFallbackTagBack(rewrittenPlan) if (tag.isDefined) { // If the rewritten plan is still not transformable, return the original plan. @@ -136,7 +137,7 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] } object RewriteSparkPlanRulesManager { - def apply(): Rule[SparkPlan] = { - new RewriteSparkPlanRulesManager(RewriteSingleNode.allRules()) + def apply(validateRule: Rule[SparkPlan]): Rule[SparkPlan] = { + new RewriteSparkPlanRulesManager(validateRule, RewriteSingleNode.allRules()) } } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala index 9ec75aa209d1..11ac62ef5823 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala @@ -18,9 +18,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.gluten.execution.{ProjectExecTransformer, SortExecTransformer, TransformSupport, WholeStageTransformer} import org.apache.gluten.execution.datasource.GlutenFormatWriterInjects -import org.apache.gluten.extension.columnar.AddFallbackTagRule -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -42,12 +40,10 @@ trait GlutenFormatWriterInjectsBase extends GlutenFormatWriterInjects { return plan.execute() } - val rules = List( - RewriteSparkPlanRulesManager(), - AddFallbackTagRule(), - TransformPreOverrides() - ) - val transformed = rules.foldLeft(plan) { case (latestPlan, rule) => rule.apply(latestPlan) } + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val transform = HeuristicTransform() + val transformed = transform(plan) + if (!transformed.isInstanceOf[TransformSupport]) { throw new IllegalStateException( "Cannot transform the SparkPlans wrapped by FileFormatWriter, " + From 5d5a963f46cf05bc2b0c3e1da7b8b07ffbb36d20 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Tue, 12 Nov 2024 18:51:41 +0800 Subject: [PATCH 076/211] [CH] Revert "read data from orc file format - ignore reading except date32" by cherry pick https://github.com/ClickHouse/ClickHouse/pull/71500 (#7917) This reverts commit f498fe74850348e4ac3120da41138fe4dda122b1. --- .../gluten/execution/GlutenClickHouseExcelFormatSuite.scala | 5 ++--- cpp-ch/clickhouse.version | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala index a5b866cb448d..0fd868a8a878 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala @@ -1038,13 +1038,12 @@ class GlutenClickHouseExcelFormatSuite ) } - test("read data from orc file format - except date32") { + test("read data from orc file format") { val filePath = s"$orcDataPath/all_data_types_with_non_primitive_type.snappy.orc" val orcFileFormat = "orc" val sql = s""" - | select string_field, int_field, long_field, float_field, double_field, short_field, - | byte_field, boolean_field, decimal_field + | select * | from $orcFileFormat.`$filePath` | where long_field > 30 |""".stripMargin diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index b5e11ca13f06..e51caf2aa788 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence CH_BRANCH=rebase_ch/20241111 -CH_COMMIT=c988219798d \ No newline at end of file +CH_COMMIT=06ffc32a255 From 0a89a6ac2a6636dd0348ebcf0251d2a5ab1bfa98 Mon Sep 17 00:00:00 2001 From: Yuan Date: Tue, 12 Nov 2024 20:01:20 +0800 Subject: [PATCH 077/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_12) (#7899) * [VL] rebase velox to 2024_11_12 Velox changes: afa657237 (HEAD, upstream/main) Enable common dictionary layer generation in expression fuzzers (#11502) 7a64fa1ff Legacy code cleanup (#11503) 592ac67a7 Extracts pipeline IDs automatically in Trace Replayer (#11492) dfc737d74 CMake: Move resolve dependency macros under velox_ prefix (#11392) cc48ac9a6 Fix MockSharedArbitrationTest.ensureMemoryPoolMaxCapacity (#11470) c8c4707f2 Fix memory arbitration fuzzer failure (#11493) 57040fd78 Fix flaky test ssdWriteOptions (#11491) f69b852c4 Make default abort time ratio small in MockSharedArbitrationTest (#11464) bed3184ba Prefix sort code cleanup and fix benchmark targets (#11490) This patch also removed the unnecessary changes in Velox patch. --------- Signed-off-by: Yuan Zhou --- ep/build-velox/src/get_velox.sh | 2 +- ep/build-velox/src/modify_velox.patch | 99 ++------------------------- 2 files changed, 8 insertions(+), 93 deletions(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 44679ae6ee2a..3d72b950bf93 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_11 +VELOX_BRANCH=2024_11_12 VELOX_HOME="" OS=`uname -s` diff --git a/ep/build-velox/src/modify_velox.patch b/ep/build-velox/src/modify_velox.patch index 3d45ff4b48a3..bf801ead83ef 100644 --- a/ep/build-velox/src/modify_velox.patch +++ b/ep/build-velox/src/modify_velox.patch @@ -1,66 +1,5 @@ -diff --git a/CMake/resolve_dependency_modules/arrow/CMakeLists.txt b/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -index ddf1fac71..ef48ae9d9 100644 ---- a/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -+++ b/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -@@ -14,17 +14,13 @@ - project(Arrow) - - if(VELOX_ENABLE_ARROW) -- find_package(Thrift) -- if(Thrift_FOUND) -- set(THRIFT_SOURCE "SYSTEM") -- else() -- set(THRIFT_SOURCE "BUNDLED") -- endif() -- - set(ARROW_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/arrow_ep") - set(ARROW_CMAKE_ARGS -- -DARROW_PARQUET=OFF -- -DARROW_DEPENDENCY_SOURCE=AUTO -+ -DARROW_PARQUET=ON -+ -DARROW_FILESYSTEM=ON -+ -DARROW_PROTOBUF_USE_SHARED=OFF -+ -DARROW_DEPENDENCY_USE_SHARED=OFF -+ -DARROW_DEPENDENCY_SOURCE=BUNDLED - -DARROW_WITH_THRIFT=ON - -DARROW_WITH_LZ4=ON - -DARROW_WITH_SNAPPY=ON -@@ -38,18 +34,16 @@ if(VELOX_ENABLE_ARROW) - -DCMAKE_INSTALL_PREFIX=${ARROW_PREFIX}/install - -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE} - -DARROW_BUILD_STATIC=ON -- -DThrift_SOURCE=${THRIFT_SOURCE} -+ -DARROW_BUILD_SHARED=OFF - -DCMAKE_PREFIX_PATH=${CMAKE_PREFIX_PATH}) - set(ARROW_LIBDIR ${ARROW_PREFIX}/install/${CMAKE_INSTALL_LIBDIR}) - - add_library(thrift STATIC IMPORTED GLOBAL) -- if(NOT Thrift_FOUND) -- set(THRIFT_ROOT ${ARROW_PREFIX}/src/arrow_ep-build/thrift_ep-install) -- set(THRIFT_LIB ${THRIFT_ROOT}/lib/libthrift.a) -+ set(THRIFT_ROOT ${ARROW_PREFIX}/src/arrow_ep-build/thrift_ep-install) -+ set(THRIFT_LIB ${THRIFT_ROOT}/lib/libthrift.a) - -- file(MAKE_DIRECTORY ${THRIFT_ROOT}/include) -- set(THRIFT_INCLUDE_DIR ${THRIFT_ROOT}/include) -- endif() -+ file(MAKE_DIRECTORY ${THRIFT_ROOT}/include) -+ set(THRIFT_INCLUDE_DIR ${THRIFT_ROOT}/include) - - set_property(TARGET thrift PROPERTY INTERFACE_INCLUDE_DIRECTORIES - ${THRIFT_INCLUDE_DIR}) -@@ -68,6 +62,9 @@ if(VELOX_ENABLE_ARROW) - arrow_ep - PREFIX ${ARROW_PREFIX} - URL ${VELOX_ARROW_SOURCE_URL} -+ PATCH_COMMAND patch -p1 < ${CMAKE_CURRENT_SOURCE_DIR}/modify_arrow.patch -+ COMMAND patch -p1 < -+ ${CMAKE_CURRENT_SOURCE_DIR}/modify_arrow_dataset_scan_option.patch - URL_HASH ${VELOX_ARROW_BUILD_SHA256_CHECKSUM} - SOURCE_SUBDIR cpp - CMAKE_ARGS ${ARROW_CMAKE_ARGS} diff --git a/CMake/resolve_dependency_modules/simdjson.cmake b/CMake/resolve_dependency_modules/simdjson.cmake -index 69e7f2044..777eb5ec1 100644 +index ddd232050..b6ed2432b 100644 --- a/CMake/resolve_dependency_modules/simdjson.cmake +++ b/CMake/resolve_dependency_modules/simdjson.cmake @@ -29,4 +29,6 @@ FetchContent_Declare( @@ -71,40 +10,16 @@ index 69e7f2044..777eb5ec1 100644 + FetchContent_MakeAvailable(simdjson) diff --git a/CMakeLists.txt b/CMakeLists.txt -index 7f7cbc92f..52adb1250 100644 +index 7fd99b6dc..e7e03a800 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt -@@ -386,7 +391,7 @@ resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) +@@ -428,7 +428,7 @@ velox_resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) # for reference. find_package(range-v3) - set_source(gflags) --resolve_dependency(gflags COMPONENTS ${VELOX_GFLAGS_TYPE}) -+resolve_dependency(gflags) + velox_set_source(gflags) +-velox_resolve_dependency(gflags COMPONENTS ${VELOX_GFLAGS_TYPE}) ++velox_resolve_dependency(gflags) if(NOT TARGET gflags::gflags) # This is a bit convoluted, but we want to be able to use gflags::gflags as a # target even when velox is built as a subproject which uses -diff --git a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp b/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -index a8b0df52a..254c56b70 100644 ---- a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -+++ b/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -@@ -38,7 +38,6 @@ std::shared_ptr abfsFileSystemGenerator( - - void registerAbfsFileSystem() { - #ifdef VELOX_ENABLE_ABFS -- LOG(INFO) << "Register ABFS"; - registerFileSystem(isAbfsFile, std::function(abfsFileSystemGenerator)); - #endif - } -diff --git a/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp b/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -index 10ee508ba..027a58ecc 100644 ---- a/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -+++ b/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -@@ -72,7 +72,7 @@ HdfsMiniCluster::HdfsMiniCluster() { - "Failed to find minicluster executable {}'", miniClusterExecutableName); - } - boost::filesystem::path hadoopHomeDirectory = exePath_; -- hadoopHomeDirectory.remove_leaf().remove_leaf(); -+ hadoopHomeDirectory.remove_filename().remove_filename(); - setupEnvironment(hadoopHomeDirectory.string()); - } - + \ No newline at end of file From c1a3f7b502ceafd7b5ca11117a93a295ea5d69ec Mon Sep 17 00:00:00 2001 From: Chang chen Date: Wed, 13 Nov 2024 09:32:32 +0800 Subject: [PATCH 078/211] [GLUTEN-7028][CH][Part-8] Support one pipeline write for partition mergetree (#7924) * [Refactor] simple refactor * [Refactor] Remove setStats * [Refactor] SparkPartitionedBaseSink and WriteStatsBase * [Refactor] Add explicit SparkMergeTreeWriteSettings(const DB::ContextPtr & context); * [New] Support writing partition mergetree in one pipeline --- .../GlutenClickHouseMergeTreeWriteSuite.scala | 180 +++++---- .../Parser/RelParsers/WriteRelParser.cpp | 39 +- .../Parser/SerializedPlanParser.cpp | 8 +- .../Parser/SerializedPlanParser.h | 4 +- .../Storages/MergeTree/SparkMergeTreeSink.cpp | 19 +- .../Storages/MergeTree/SparkMergeTreeSink.h | 160 ++++---- .../MergeTree/SparkMergeTreeWriteSettings.cpp | 3 +- .../MergeTree/SparkMergeTreeWriteSettings.h | 2 +- .../MergeTree/SparkStorageMergeTree.cpp | 8 +- .../Storages/Output/NormalFileWriter.cpp | 2 +- .../Storages/Output/NormalFileWriter.h | 122 +++--- .../tests/gtest_write_pipeline_mergetree.cpp | 42 +- .../tests/json/mergetree/4_one_pipeline.json | 377 ++++++++++++++++++ ...input.json => lineitem_parquet_input.json} | 0 14 files changed, 701 insertions(+), 265 deletions(-) create mode 100644 cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json rename cpp-ch/local-engine/tests/json/mergetree/{3_one_pipeline_input.json => lineitem_parquet_input.json} (100%) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index b33a2065f665..85c8c2d92a52 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -162,103 +162,107 @@ class GlutenClickHouseMergeTreeWriteSuite } test("test mergetree insert overwrite") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite; + |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |LOCATION '$basePath/lineitem_mergetree_insertoverwrite' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |LOCATION '$basePath/lineitem_mergetree_insertoverwrite' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree_insertoverwrite - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree_insertoverwrite + | select * from lineitem + |""".stripMargin) - spark.sql(s""" - | insert overwrite table lineitem_mergetree_insertoverwrite - | select * from lineitem where mod(l_orderkey,2) = 1 - |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_mergetree_insertoverwrite - | - |""".stripMargin - assertResult(300001)( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) - ) + spark.sql(s""" + | insert overwrite table lineitem_mergetree_insertoverwrite + | select * from lineitem where mod(l_orderkey,2) = 1 + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_mergetree_insertoverwrite + | + |""".stripMargin + assertResult(300001)( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) + ) + } } test("test mergetree insert overwrite partitioned table with small table, static") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite2; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, "false")) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite2; + |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite2 - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |PARTITIONED BY (l_shipdate) - |LOCATION '$basePath/lineitem_mergetree_insertoverwrite2' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite2 + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/lineitem_mergetree_insertoverwrite2' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree_insertoverwrite2 - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree_insertoverwrite2 + | select * from lineitem + |""".stripMargin) - spark.sql( - s""" - | insert overwrite table lineitem_mergetree_insertoverwrite2 - | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' - |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_mergetree_insertoverwrite2 - | - |""".stripMargin - assertResult(2418)( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) - ) + spark.sql( + s""" + | insert overwrite table lineitem_mergetree_insertoverwrite2 + | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_mergetree_insertoverwrite2 + | + |""".stripMargin + assertResult(2418)( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) + ) + } } test("test mergetree insert overwrite partitioned table with small table, dynamic") { diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp index ea1239a8093a..25f786a77465 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp @@ -58,15 +58,11 @@ DB::ProcessorPtr make_sink( { if (partition_by.empty()) { - auto file_sink = std::make_shared(context, base_path, "", filename, format_hint, input_header); - file_sink->setStats(stats); - return file_sink; + return std::make_shared(context, base_path, "", filename, format_hint, input_header, stats); } - auto file_sink = std::make_shared( - context, partition_by, input_header, output_header, base_path, filename, format_hint); - file_sink->setStats(stats); - return file_sink; + return std::make_shared( + context, partition_by, input_header, output_header, base_path, filename, format_hint, stats); } DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB::Block & output) @@ -148,25 +144,30 @@ void addMergeTreeSinkTransform( const DB::ContextPtr & context, const DB::QueryPipelineBuilderPtr & builder, const MergeTreeTable & merge_tree_table, - const DB::Block & output, - const DB::Names & /*partitionCols*/) + const DB::Block & header, + const DB::Names & partition_by) { - const DB::Settings & settings = context->getSettingsRef(); - const auto dest_storage = merge_tree_table.getStorage(context->getGlobalContext()); - StorageMetadataPtr metadata_snapshot = dest_storage->getInMemoryMetadataPtr(); - ASTPtr none; - auto sink = dest_storage->write(none, metadata_snapshot, context, false); + Chain chain; - chain.addSink(sink); - const SinkHelper & sink_helper = assert_cast(*sink).sinkHelper(); // - auto stats = std::make_shared(output, sink_helper); + auto stats = std::make_shared(header); chain.addSink(stats); // + + SparkMergeTreeWriteSettings write_settings{context}; + if (partition_by.empty()) + write_settings.partition_settings.partition_dir = SubstraitFileSink::NO_PARTITION_ID; + + auto sink = partition_by.empty() ? + SparkMergeTreeSink::create(merge_tree_table, write_settings, context->getGlobalContext(), {stats}) : + std::make_shared(header, partition_by, merge_tree_table, write_settings, context, stats); + + chain.addSource(sink); + const DB::Settings & settings = context->getSettingsRef(); chain.addSource(std::make_shared( - output, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); chain.addSource(std::make_shared( - output, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); builder->addChain(std::move(chain)); } diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 4e49c577bf86..748ff88acbd1 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -283,7 +283,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list return query_plan; } -DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPlan & query_plan) +DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPlan & query_plan) const { const Settings & settings = parser_context->queryContext()->getSettingsRef(); QueryPriorities priorities; @@ -355,11 +355,7 @@ NonNullableColumnsResolver::NonNullableColumnsResolver( expression_parser = std::make_unique(parser_context); } -NonNullableColumnsResolver::~NonNullableColumnsResolver() -{ -} - -// make it simple at present, if the condition contains or, return empty for both side. +// make it simple at present if the condition contains or, return empty for both side. std::set NonNullableColumnsResolver::resolve() { collected_columns.clear(); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index 2bcc09a8ed9e..f0ec608a330f 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -48,7 +48,7 @@ class NonNullableColumnsResolver public: explicit NonNullableColumnsResolver( const DB::Block & header_, std::shared_ptr parser_context_, const substrait::Expression & cond_rel_); - ~NonNullableColumnsResolver(); + ~NonNullableColumnsResolver() = default; // return column names std::set resolve(); @@ -76,7 +76,7 @@ class SerializedPlanParser /// visible for UT DB::QueryPlanPtr parse(const substrait::Plan & plan); std::unique_ptr createExecutor(const substrait::Plan & plan); - DB::QueryPipelineBuilderPtr buildQueryPipeline(DB::QueryPlan & query_plan); + DB::QueryPipelineBuilderPtr buildQueryPipeline(DB::QueryPlan & query_plan) const; /// std::unique_ptr createExecutor(const std::string_view plan); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp index 43a3a78295d2..de0c244f3e53 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp @@ -62,15 +62,24 @@ void SparkMergeTreeSink::onStart() void SparkMergeTreeSink::onFinish() { sink_helper->finish(context); + if (stats_.has_value()) + (*stats_)->collectStats(sink_helper->unsafeGet(), sink_helper->write_settings.partition_settings.partition_dir); } ///// -SinkHelperPtr SparkMergeTreeSink::create( - const MergeTreeTable & merge_tree_table, const SparkMergeTreeWriteSettings & write_settings_, const DB::ContextMutablePtr & context) +SinkToStoragePtr SparkMergeTreeSink::create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings_, + const DB::ContextMutablePtr & context, + const SinkStatsOption & stats) { + if (write_settings_.partition_settings.part_name_prefix.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "empty part_name_prefix is not allowed."); + auto dest_storage = merge_tree_table.getStorage(context); bool isRemoteStorage = dest_storage->getStoragePolicy()->getAnyDisk()->isRemote(); bool insert_with_local_storage = !write_settings_.insert_without_local_storage; + SinkHelperPtr sink_helper; if (insert_with_local_storage && isRemoteStorage) { auto temp = merge_tree_table.copyToDefaultPolicyStorage(context); @@ -78,10 +87,10 @@ SinkHelperPtr SparkMergeTreeSink::create( &Poco::Logger::get("SparkMergeTreeWriter"), "Create temp table {} for local merge.", temp->getStorageID().getFullNameNotQuoted()); - return std::make_shared(temp, dest_storage, write_settings_); + sink_helper = std::make_shared(temp, dest_storage, write_settings_); } - - return std::make_shared(dest_storage, write_settings_, isRemoteStorage); + sink_helper = std::make_shared(dest_storage, write_settings_, isRemoteStorage); + return std::make_shared(sink_helper, context, stats); } SinkHelper::SinkHelper(const SparkStorageMergeTreePtr & data_, const SparkMergeTreeWriteSettings & write_settings_, bool isRemoteStorage_) diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h index d19e5dc4e969..065b147462f9 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h @@ -21,6 +21,7 @@ #include #include #include +#include #include namespace local_engine @@ -152,39 +153,18 @@ class CopyToRemoteSinkHelper : public SinkHelper } }; -class SparkMergeTreeSink : public DB::SinkToStorage +class MergeTreeStats : public WriteStatsBase { -public: - static SinkHelperPtr create( - const MergeTreeTable & merge_tree_table, - const SparkMergeTreeWriteSettings & write_settings_, - const DB::ContextMutablePtr & context); + DB::MutableColumns columns_; - explicit SparkMergeTreeSink(const SinkHelperPtr & sink_helper_, const ContextPtr & context_) - : SinkToStorage(sink_helper_->metadata_snapshot->getSampleBlock()), context(context_), sink_helper(sink_helper_) + enum ColumnIndex { - } - ~SparkMergeTreeSink() override = default; - - String getName() const override { return "SparkMergeTreeSink"; } - void consume(Chunk & chunk) override; - void onStart() override; - void onFinish() override; - - const SinkHelper & sinkHelper() const { return *sink_helper; } - -private: - ContextPtr context; - SinkHelperPtr sink_helper; - - int part_num = 1; -}; - - -class MergeTreeStats : public DB::ISimpleTransform -{ - bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks - const SinkHelper & sink_helper; + part_name, + partition_id, + record_count, + marks_count, + size_in_bytes + }; static DB::Block statsHeader() { @@ -196,69 +176,109 @@ class MergeTreeStats : public DB::ISimpleTransform {BIGINT(), "size_in_bytes"}}); } - DB::Chunk final_result() const + DB::Chunk final_result() override + { + size_t rows = columns_[part_name]->size(); + return DB::Chunk(std::move(columns_), rows); + } + +public: + explicit MergeTreeStats(const DB::Block & input_header_) + : WriteStatsBase(input_header_, statsHeader()), columns_(statsHeader().cloneEmptyColumns()) { - // TODO: remove it - const std::string NO_PARTITION_ID{"__NO_PARTITION_ID__"}; + } - auto parts = sink_helper.unsafeGet(); + String getName() const override { return "MergeTreeStats"; } - const size_t size = parts.size(); - auto file_col = STRING()->createColumn(); - file_col->reserve(size); + void collectStats(const std::deque & parts, const std::string & partition) const + { + const size_t size = parts.size() + columns_[part_name]->size(); - auto partition_col = STRING()->createColumn(); - partition_col->reserve(size); + columns_[part_name]->reserve(size); + columns_[partition_id]->reserve(size); - auto countCol = BIGINT()->createColumn(); - countCol->reserve(size); - auto & countColData = static_cast &>(*countCol).getData(); + columns_[record_count]->reserve(size); + auto & countColData = static_cast &>(*columns_[record_count]).getData(); - auto marksCol = BIGINT()->createColumn(); - marksCol->reserve(size); - auto & marksColData = static_cast &>(*marksCol).getData(); + columns_[marks_count]->reserve(size); + auto & marksColData = static_cast &>(*columns_[marks_count]).getData(); - auto bytesCol = BIGINT()->createColumn(); - bytesCol->reserve(size); - auto & bytesColData = static_cast &>(*bytesCol).getData(); + columns_[size_in_bytes]->reserve(size); + auto & bytesColData = static_cast &>(*columns_[size_in_bytes]).getData(); for (const auto & part : parts) { - file_col->insertData(part->name.c_str(), part->name.size()); - partition_col->insertData(NO_PARTITION_ID.c_str(), NO_PARTITION_ID.size()); + columns_[part_name]->insertData(part->name.c_str(), part->name.size()); + columns_[partition_id]->insertData(partition.c_str(), partition.size()); + countColData.emplace_back(part->rows_count); marksColData.emplace_back(part->getMarksCount()); bytesColData.emplace_back(part->getBytesOnDisk()); } - const DB::Columns res_columns{ - std::move(file_col), std::move(partition_col), std::move(countCol), std::move(marksCol), std::move(bytesCol)}; - return DB::Chunk(res_columns, size); } +}; +class SparkMergeTreeSink : public DB::SinkToStorage +{ public: - explicit MergeTreeStats(const DB::Block & input_header_, const SinkHelper & sink_helper_) - : ISimpleTransform(input_header_, statsHeader(), true), sink_helper(sink_helper_) + using SinkStatsOption = std::optional>; + static SinkToStoragePtr create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings_, + const DB::ContextMutablePtr & context, + const SinkStatsOption & stats = {}); + + explicit SparkMergeTreeSink(const SinkHelperPtr & sink_helper_, const ContextPtr & context_, const SinkStatsOption & stats) + : SinkToStorage(sink_helper_->metadata_snapshot->getSampleBlock()), context(context_), sink_helper(sink_helper_), stats_(stats) { } - Status prepare() override - { - if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) - { - all_chunks_processed_ = true; - /// return Ready to call transform() for generating filling rows after latest chunk was processed - return Status::Ready; - } + ~SparkMergeTreeSink() override = default; + + String getName() const override { return "SparkMergeTreeSink"; } + void consume(Chunk & chunk) override; + void onStart() override; + void onFinish() override; + + const SinkHelper & sinkHelper() const { return *sink_helper; } - return ISimpleTransform::prepare(); +private: + ContextPtr context; + SinkHelperPtr sink_helper; + std::optional> stats_; + int part_num = 1; +}; + + +class SparkMergeTreePartitionedFileSink final : public SparkPartitionedBaseSink +{ + const SparkMergeTreeWriteSettings write_settings_; + MergeTreeTable table; + +public: + SparkMergeTreePartitionedFileSink( + const DB::Block & input_header, + const DB::Names & partition_by, + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings, + const DB::ContextPtr & context, + const std::shared_ptr & stats) + : SparkPartitionedBaseSink(context, partition_by, input_header, stats), write_settings_(write_settings), table(merge_tree_table) + { } - String getName() const override { return "MergeTreeStats"; } - void transform(DB::Chunk & chunk) override + SinkPtr createSinkForPartition(const String & partition_id) override { - if (all_chunks_processed_) - chunk = final_result(); - else - chunk = {}; + SparkMergeTreeWriteSettings write_settings{write_settings_}; + + assert(write_settings.partition_settings.partition_dir.empty()); + assert(write_settings.partition_settings.bucket_dir.empty()); + write_settings.partition_settings.part_name_prefix + = fmt::format("{}/{}", partition_id, write_settings.partition_settings.part_name_prefix); + write_settings.partition_settings.partition_dir = partition_id; + + return SparkMergeTreeSink::create( + table, write_settings, context_->getGlobalContext(), {std::dynamic_pointer_cast(stats_)}); } }; + } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp index 75472d410c97..e584b003d2c6 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp @@ -26,7 +26,8 @@ namespace local_engine IMPLEMENT_GLUTEN_SETTINGS(SparkMergeTreeWritePartitionSettings, MERGE_TREE_WRITE_RELATED_SETTINGS) -void SparkMergeTreeWriteSettings::load(const DB::ContextPtr & context) +SparkMergeTreeWriteSettings::SparkMergeTreeWriteSettings(const DB::ContextPtr & context) + : partition_settings(SparkMergeTreeWritePartitionSettings::get(context)) { const DB::Settings & settings = context->getSettingsRef(); merge_after_insert = settings.get(MERGETREE_MERGE_AFTER_INSERT).safeGet(); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h index 1fbbdbe3461d..e89b2aaf5e44 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h @@ -35,6 +35,6 @@ struct SparkMergeTreeWriteSettings size_t merge_min_size = 1024 * 1024 * 1024; size_t merge_limit_parts = 10; - void load(const DB::ContextPtr & context); + explicit SparkMergeTreeWriteSettings(const DB::ContextPtr & context); }; } \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index a6a252be3d63..0be7e0d892b7 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -513,17 +513,11 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( SinkToStoragePtr SparkWriteStorageMergeTree::write( const ASTPtr &, const StorageMetadataPtr & /*storage_in_memory_metadata*/, ContextPtr context, bool /*async_insert*/) { - SparkMergeTreeWriteSettings settings{.partition_settings{SparkMergeTreeWritePartitionSettings::get(context)}}; - if (settings.partition_settings.part_name_prefix.empty()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "empty part_name_prefix is not allowed."); - settings.load(context); - SinkHelperPtr sink_helper = SparkMergeTreeSink::create(table, settings, getContext()); #ifndef NDEBUG auto dest_storage = table.getStorage(getContext()); assert(dest_storage.get() == this); #endif - - return std::make_shared(sink_helper, context); + return SparkMergeTreeSink::create(table, SparkMergeTreeWriteSettings{context}, getContext()); } } diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index 8c904ab20539..d572b8538591 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -23,7 +23,7 @@ namespace local_engine { const std::string SubstraitFileSink::NO_PARTITION_ID{"__NO_PARTITION_ID__"}; -const std::string SubstraitPartitionedFileSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; +const std::string SparkPartitionedBaseSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; NormalFileWriter::NormalFileWriter(const OutputFormatFilePtr & file_, const DB::ContextPtr & context_) : file(file_), context(context_) { diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index 8a25d7b21fa6..cd9c848dd418 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -61,20 +61,51 @@ class NormalFileWriter : public NativeOutputWriter OutputFormatFilePtr createOutputFormatFile( const DB::ContextPtr & context, const std::string & file_uri, const DB::Block & preferred_schema, const std::string & format_hint); -class WriteStats : public DB::ISimpleTransform +class WriteStatsBase : public DB::ISimpleTransform { +protected: bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks - DB::Arena partition_keys_arena_; - std::string filename_; + virtual DB::Chunk final_result() = 0; - absl::flat_hash_map file_to_count_; +public: + WriteStatsBase(const DB::Block & input_header_, const DB::Block & output_header_) + : ISimpleTransform(input_header_, output_header_, true) + { + } + + Status prepare() override + { + if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) + { + all_chunks_processed_ = true; + /// return Ready to call transform() for generating filling rows after latest chunk was processed + return Status::Ready; + } + return ISimpleTransform::prepare(); + } + + void transform(DB::Chunk & chunk) override + { + if (all_chunks_processed_) + chunk = final_result(); + else + chunk = {}; + } +}; + +class WriteStats : public WriteStatsBase +{ static DB::Block statsHeader() { return makeBlockHeader({{STRING(), "filename"}, {STRING(), "partition_id"}, {BIGINT(), "record_count"}}); } + DB::Arena partition_keys_arena_; + std::string filename_; + absl::flat_hash_map file_to_count_; - DB::Chunk final_result() const +protected: + DB::Chunk final_result() override { const size_t size = file_to_count_.size(); @@ -102,30 +133,9 @@ class WriteStats : public DB::ISimpleTransform } public: - explicit WriteStats(const DB::Block & input_header_) : ISimpleTransform(input_header_, statsHeader(), true) { } - - Status prepare() override - { - if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) - { - all_chunks_processed_ = true; - /// return Ready to call transform() for generating filling rows after latest chunk was processed - return Status::Ready; - } - - return ISimpleTransform::prepare(); - } - + explicit WriteStats(const DB::Block & input_header_) : WriteStatsBase(input_header_, statsHeader()) { } String getName() const override { return "WriteStats"; } - void transform(DB::Chunk & chunk) override - { - if (all_chunks_processed_) - chunk = final_result(); - else - chunk = {}; - } - - void addFilePath(const String & patition_id, const String & filename) + void addFilePath(const String & partition_id, const String & filename) { assert(!filename.empty()); @@ -134,9 +144,9 @@ class WriteStats : public DB::ISimpleTransform assert(filename_ == filename); - if (patition_id.empty()) + if (partition_id.empty()) return; - file_to_count_.emplace(copyStringInArena(partition_keys_arena_, patition_id), 0); + file_to_count_.emplace(copyStringInArena(partition_keys_arena_, partition_id), 0); } void collectStats(const String & file_path, size_t rows) @@ -155,7 +165,7 @@ class SubstraitFileSink final : public DB::SinkToStorage const std::string partition_id_; const std::string relative_path_; OutputFormatFile::OutputFormatPtr output_format_; - std::shared_ptr stats_{nullptr}; + std::shared_ptr stats_; static std::string makeFilename(const std::string & base_path, const std::string & partition_id, const std::string & relative) { @@ -174,22 +184,18 @@ class SubstraitFileSink final : public DB::SinkToStorage const std::string & partition_id, const std::string & relative, const std::string & format_hint, - const DB::Block & header) + const DB::Block & header, + const std::shared_ptr & stats) : SinkToStorage(header) , partition_id_(partition_id.empty() ? NO_PARTITION_ID : partition_id) , relative_path_(relative) , output_format_(createOutputFormatFile(context, makeFilename(base_path, partition_id, relative), header, format_hint) ->createOutputFormat(header)) + , stats_(std::dynamic_pointer_cast(stats)) { - } - String getName() const override { return "SubstraitFileSink"; } - - ///TODO: remove this function - void setStats(const std::shared_ptr & stats) - { - stats_ = stats; stats_->addFilePath(partition_id_, relative_path_); } + String getName() const override { return "SubstraitFileSink"; } protected: void consume(DB::Chunk & chunk) override @@ -208,7 +214,7 @@ class SubstraitFileSink final : public DB::SinkToStorage } }; -class SubstraitPartitionedFileSink final : public DB::PartitionedSink +class SparkPartitionedBaseSink : public DB::PartitionedSink { static const std::string DEFAULT_PARTITION_NAME; @@ -237,13 +243,27 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink return DB::makeASTFunction("concat", std::move(arguments)); } -private: +protected: + DB::ContextPtr context_; + std::shared_ptr stats_; + +public: + SparkPartitionedBaseSink( + const DB::ContextPtr & context, + const DB::Names & partition_by, + const DB::Block & input_header, + const std::shared_ptr & stats) + : PartitionedSink(make_partition_expression(partition_by), context, input_header), context_(context), stats_(stats) + { + } +}; + +class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink +{ const std::string base_path_; const std::string filename_; - DB::ContextPtr context_; const DB::Block sample_block_; const std::string format_hint_; - std::shared_ptr stats_{nullptr}; public: SubstraitPartitionedFileSink( @@ -253,27 +273,23 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink const DB::Block & sample_block, const std::string & base_path, const std::string & filename, - const std::string & format_hint) - : PartitionedSink(make_partition_expression(partition_by), context, input_header) + const std::string & format_hint, + const std::shared_ptr & stats) + : SparkPartitionedBaseSink(context, partition_by, input_header, stats) , base_path_(base_path) , filename_(filename) - , context_(context) , sample_block_(sample_block) , format_hint_(format_hint) { } + DB::SinkPtr createSinkForPartition(const String & partition_id) override { assert(stats_); const auto partition_path = fmt::format("{}/{}", partition_id, filename_); - PartitionedSink::validatePartitionKey(partition_path, true); - auto file_sink = std::make_shared(context_, base_path_, partition_id, filename_, format_hint_, sample_block_); - file_sink->setStats(stats_); - return file_sink; + validatePartitionKey(partition_path, true); + return std::make_shared(context_, base_path_, partition_id, filename_, format_hint_, sample_block_, stats_); } String getName() const override { return "SubstraitPartitionedFileSink"; } - - ///TODO: remove this function - void setStats(const std::shared_ptr & stats) { stats_ = stats; } }; } diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp index ddbf113353b1..1ad90060f475 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp @@ -255,24 +255,42 @@ TEST(MergeTree, SparkMergeTree) } } -INCBIN(_3_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline.json"); -INCBIN(_3_mergetree_plan_input_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline_input.json"); - -TEST(MergeTree, Pipeline) +INCBIN(_3_mergetree_plan_input_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/lineitem_parquet_input.json"); +namespace +{ +void writeMerge(std::string_view json_plan, + const std::string & outputPath , + const std::function & callback, std::optional input = std::nullopt) { const auto context = DB::Context::createCopy(QueryContext::globalContext()); - GlutenWriteSettings settings{.task_write_tmp_dir = "tmp/lineitem_mergetree"}; + GlutenWriteSettings settings{.task_write_tmp_dir = outputPath}; settings.set(context); SparkMergeTreeWritePartitionSettings partition_settings{.part_name_prefix = "pipline_prefix"}; partition_settings.set(context); - auto [_, local_executor] = test::create_plan_and_executor( - EMBEDDED_PLAN(_3_mergetree_plan_), replaceLocalFilesWithTPCH(EMBEDDED_PLAN(_3_mergetree_plan_input_)), context); - size_t sum = 0; + auto input_json = input.value_or(replaceLocalFilesWithTPCH(EMBEDDED_PLAN(_3_mergetree_plan_input_))); + auto [_, local_executor] = test::create_plan_and_executor(json_plan, input_json, context); + while (local_executor->hasNext()) + callback(*local_executor->nextColumnar()); +} +} +INCBIN(_3_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline.json"); +INCBIN(_4_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/4_one_pipeline.json"); +TEST(MergeTree, Pipeline) +{ + writeMerge(EMBEDDED_PLAN(_3_mergetree_plan_),"tmp/lineitem_mergetree",[&](const DB::Block & block) { - const Block & x = *local_executor->nextColumnar(); - EXPECT_EQ(1, x.rows()); - //-debug::headBlock("pipeline write", x); - } + EXPECT_EQ(1, block.rows()); + debug::headBlock(block); + }); +} + +TEST(MergeTree, PipelineWithPartition) +{ + writeMerge(EMBEDDED_PLAN(_4_mergetree_plan_),"tmp/lineitem_mergetree_p",[&](const DB::Block & block) + { + EXPECT_EQ(2525, block.rows()); + debug::headBlock(block); + }); } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json b/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json new file mode 100644 index 000000000000..14a9b3dda2ad --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json @@ -0,0 +1,377 @@ +{ + "relations": [ + { + "root": { + "input": { + "write": { + "namedTable": { + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "mergetree" + }, + "mergetree": { + "database": "default", + "table": "lineitem_mergetree_insertoverwrite2", + "snapshotId": "1731309448915_0", + "orderByKey": "tuple()", + "storagePolicy": "default" + } + }, + "enhancement": { + "@type": "type.googleapis.com/substrait.Type", + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + }, + "tableSchema": { + "names": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "PARTITION_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + } + } + }, + "outputSchema": { + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json b/cpp-ch/local-engine/tests/json/mergetree/lineitem_parquet_input.json similarity index 100% rename from cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json rename to cpp-ch/local-engine/tests/json/mergetree/lineitem_parquet_input.json From 8b8c5b3f553b711c7e4c044195f5995d0a83119f Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Wed, 13 Nov 2024 13:09:47 +0800 Subject: [PATCH 079/211] [Minor] Fix a typo in Gluten config (#7931) --- .../scala/org/apache/gluten/execution/FallbackSuite.scala | 4 ++-- .../scala/org/apache/gluten/execution/MiscOperatorSuite.scala | 2 +- .../scala/org/apache/gluten/execution/VeloxMetricsSuite.scala | 2 +- .../sql/execution/benchmark/StreamingAggregateBenchmark.scala | 4 ++-- .../spark/sql/execution/joins/GlutenInnerJoinSuite.scala | 4 ++-- .../spark/sql/execution/joins/GlutenOuterJoinSuite.scala | 4 ++-- .../src/main/scala/org/apache/gluten/GlutenConfig.scala | 4 ++-- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala index d8b0ee8981e2..0f7b70bf0032 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala @@ -253,7 +253,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl test("fallback with smj") { val sql = "SELECT /*+ SHUFFLE_MERGE(tmp1) */ * FROM tmp1 join tmp2 on tmp1.c1 = tmp2.c1" withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true", GlutenConfig.COLUMNAR_SHUFFLED_HASH_JOIN_ENABLED.key -> "false") { runQueryAndCompare(sql) { df => @@ -262,7 +262,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl } } withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", GlutenConfig.COLUMNAR_SORTMERGEJOIN_ENABLED.key -> "false") { runQueryAndCompare(sql) { df => diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index da5184f752b3..2b42d7fb8d64 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -845,7 +845,7 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1" ) { val query = diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala index 0b74824832f2..e41f00821c27 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala @@ -63,7 +63,7 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa test("test sort merge join metrics") { withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { // without preproject runQueryAndCompare( diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala index 7f11fd043db4..77370d059caa 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala @@ -51,7 +51,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark { withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" ) { spark.sql(query).noop() } @@ -62,7 +62,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark { withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" ) { spark.sql(query).noop() } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala index d31936211b72..8d9e82df195a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala @@ -24,13 +24,13 @@ import org.apache.spark.sql.GlutenSQLTestsBaseTrait class GlutenInnerJoinSuiteForceShjOn extends InnerJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") } } class GlutenInnerJoinSuiteForceShjOff extends InnerJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala index 9c45eae15b0a..f48939c3d012 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala @@ -24,13 +24,13 @@ import org.apache.spark.sql.GlutenSQLTestsBaseTrait class GlutenOuterJoinSuiteForceShjOn extends OuterJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") } } class GlutenOuterJoinSuiteForceShjOff extends OuterJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") } } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 6579e30dab8a..0bfe16270e5f 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -75,7 +75,7 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableNativeColumnarToRow: Boolean = conf.getConf(COLUMNAR_COLUMNAR_TO_ROW_ENABLED) - def forceShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED) + def forceShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED) def enableColumnarSortMergeJoin: Boolean = conf.getConf(COLUMNAR_SORTMERGEJOIN_ENABLED) @@ -942,7 +942,7 @@ object GlutenConfig { .booleanConf .createWithDefault(true) - val COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED = + val COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED = buildConf("spark.gluten.sql.columnar.forceShuffledHashJoin") .internal() .booleanConf From f44e4c0c1f1503d1f6ce026b2f9b6f4272123e13 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Wed, 13 Nov 2024 13:12:55 +0800 Subject: [PATCH 080/211] [VL] Clean up unused variables in cpp source files (#7929) --- cpp/velox/jni/VeloxJniWrapper.cc | 1 - cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc | 1 - 2 files changed, 2 deletions(-) diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index b8d2b0c3c2ff..6ea60d651a73 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -385,7 +385,6 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_VeloxDataSourceJniWrapp jlong dsHandle, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getRuntime(env, wrapper); auto datasource = ObjectStore::retrieve(dsHandle); auto batch = ObjectStore::retrieve(batchHandle); datasource->write(batch); diff --git a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc index 34796e378e1d..e17ad5e2f713 100644 --- a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc @@ -117,7 +117,6 @@ arrow::Status VeloxRssSortShuffleWriter::write(std::shared_ptr cb } arrow::Status VeloxRssSortShuffleWriter::evictBatch(uint32_t partitionId) { - int64_t rawSize = batch_->size(); bufferOutputStream_->seekp(0); batch_->flush(bufferOutputStream_.get()); auto buffer = bufferOutputStream_->getBuffer(); From a96a09eebdc3ef26e141edb519c9eb8dbf8de0a0 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Wed, 13 Nov 2024 13:49:45 +0800 Subject: [PATCH 081/211] [CH] Fix SIGSEGV on jstring2string #7928 What changes were proposed in this pull request? replace jstring2string implementation, avoid SIGSEGV on getObjectClass How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) manual tests --- cpp-ch/local-engine/local_engine_jni.cpp | 24 ++++++++---------------- 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 24c5950065ea..2ec0622311be 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -74,22 +74,14 @@ static DB::ColumnWithTypeAndName getColumnFromColumnVector(JNIEnv * /*env*/, job return block->getByPosition(column_position); } -static std::string jstring2string(JNIEnv * env, jstring jStr) -{ - if (!jStr) - return ""; - - const jclass string_class = env->GetObjectClass(jStr); - const jmethodID get_bytes = env->GetMethodID(string_class, "getBytes", "(Ljava/lang/String;)[B"); - auto * const string_jbytes - = static_cast(local_engine::safeCallObjectMethod(env, jStr, get_bytes, env->NewStringUTF("UTF-8"))); - SCOPE_EXIT({ - env->DeleteLocalRef(string_jbytes); - env->DeleteLocalRef(string_class); - }); - - const auto string_jbytes_a = local_engine::getByteArrayElementsSafe(env, string_jbytes); - return {reinterpret_cast(string_jbytes_a.elems()), static_cast(string_jbytes_a.length())}; +static std::string jstring2string(JNIEnv * env, jstring string) +{ + if (string == nullptr) + return std::string(); + const char * chars = env->GetStringUTFChars(string, nullptr); + std::string ret(chars); + env->ReleaseStringUTFChars(string, chars); + return ret; } extern "C" { From e63b7d5828133e2a4c085b87d7039ac68997b27e Mon Sep 17 00:00:00 2001 From: lgbo Date: Wed, 13 Nov 2024 14:06:57 +0800 Subject: [PATCH 082/211] fixed bug in finding attributes in replacement map (#7927) --- .../apache/gluten/extension/LazyAggregateExpandRule.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala index 86b28ab1f717..d21cbfa2505a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -335,7 +335,9 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan val groupingExpressions = partialAggregate.groupingExpressions .filter( - e => e.toAttribute != groupIdAttribute && attributesToReplace.contains(e.toAttribute)) + e => + !e.toAttribute.semanticEquals(groupIdAttribute) && + attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined) .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) .distinct logDebug( @@ -344,7 +346,7 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan // Remove group id column from result expressions val resultExpressions = partialAggregate.resultExpressions - .filter(_.toAttribute != groupIdAttribute) + .filter(!_.toAttribute.semanticEquals(groupIdAttribute)) .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) logDebug( s"xxx newResultExpressions: $resultExpressions\n" + From d6c6bba877b677e00ff37e086522a492d9e64a30 Mon Sep 17 00:00:00 2001 From: Yang Zhang Date: Wed, 13 Nov 2024 14:45:24 +0800 Subject: [PATCH 083/211] Revert "[GLUTEN-7336][CORE] Bump Spark version to v3.5.3 (#7537) (#7930) Revert Spark release version from v3.5.3 to v3.5.2 due to datalake issue. --- .../workflows/util/install_spark_resources.sh | 20 ++++++------- .github/workflows/velox_backend.yml | 30 +++++++++---------- pom.xml | 6 ++-- .../sql/shims/spark35/SparkShimProvider.scala | 2 +- tools/gluten-it/pom.xml | 2 +- 5 files changed, 30 insertions(+), 30 deletions(-) diff --git a/.github/workflows/util/install_spark_resources.sh b/.github/workflows/util/install_spark_resources.sh index d5f413ee2116..dd2afec821d4 100755 --- a/.github/workflows/util/install_spark_resources.sh +++ b/.github/workflows/util/install_spark_resources.sh @@ -63,26 +63,26 @@ case "$1" in 3.5) # Spark-3.5 cd ${INSTALL_DIR} && \ - wget -nv https://archive.apache.org/dist/spark/spark-3.5.3/spark-3.5.3-bin-hadoop3.tgz && \ - tar --strip-components=1 -xf spark-3.5.3-bin-hadoop3.tgz spark-3.5.3-bin-hadoop3/jars/ && \ - rm -rf spark-3.5.3-bin-hadoop3.tgz && \ + wget -nv https://archive.apache.org/dist/spark/spark-3.5.2/spark-3.5.2-bin-hadoop3.tgz && \ + tar --strip-components=1 -xf spark-3.5.2-bin-hadoop3.tgz spark-3.5.2-bin-hadoop3/jars/ && \ + rm -rf spark-3.5.2-bin-hadoop3.tgz && \ mkdir -p ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.12 && \ mv jars ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.12 && \ - wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.3.tar.gz && \ - tar --strip-components=1 -xf v3.5.3.tar.gz spark-3.5.3/sql/core/src/test/resources/ && \ + wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.2.tar.gz && \ + tar --strip-components=1 -xf v3.5.2.tar.gz spark-3.5.2/sql/core/src/test/resources/ && \ mkdir -p shims/spark35/spark_home/ && \ mv sql shims/spark35/spark_home/ ;; 3.5-scala2.13) # Spark-3.5, scala 2.13 cd ${INSTALL_DIR} && \ - wget -nv https://archive.apache.org/dist/spark/spark-3.5.3/spark-3.5.3-bin-hadoop3.tgz && \ - tar --strip-components=1 -xf spark-3.5.3-bin-hadoop3.tgz spark-3.5.3-bin-hadoop3/jars/ && \ - rm -rf spark-3.5.3-bin-hadoop3.tgz && \ + wget -nv https://archive.apache.org/dist/spark/spark-3.5.2/spark-3.5.2-bin-hadoop3.tgz && \ + tar --strip-components=1 -xf spark-3.5.2-bin-hadoop3.tgz spark-3.5.2-bin-hadoop3/jars/ && \ + rm -rf spark-3.5.2-bin-hadoop3.tgz && \ mkdir -p ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.13 && \ mv jars ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.13 && \ - wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.3.tar.gz && \ - tar --strip-components=1 -xf v3.5.3.tar.gz spark-3.5.3/sql/core/src/test/resources/ && \ + wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.2.tar.gz && \ + tar --strip-components=1 -xf v3.5.2.tar.gz spark-3.5.2/sql/core/src/test/resources/ && \ mkdir -p shims/spark35/spark_home/ && \ mv sql shims/spark35/spark_home/ ;; diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index c316dba6f887..4b7f5836e745 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -754,19 +754,19 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 (other tests) + - name: Build and Run unit test for Spark 3.5.2 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.12 - $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ + $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Phudi -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags - name: Upload test report @@ -792,15 +792,15 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5-scala2.13 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 with scala-2.13 (other tests) + - name: Build and Run unit test for Spark 3.5.2 with scala-2.13 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.13 @@ -830,13 +830,13 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) + - name: Prepare spark.test.home for Spark 3.5.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 - - name: Build and Run unit test for Spark 3.5.3 (slow tests) + - name: Build and Run unit test for Spark 3.5.2 (slow tests) run: | cd $GITHUB_WORKSPACE/ - $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ + $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Phudi -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest - name: Upload test report @@ -862,15 +862,15 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 (other tests) + - name: Build and Run unit test for Spark 3.5.2 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.12 @@ -899,10 +899,10 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) + - name: Prepare spark.test.home for Spark 3.5.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 - - name: Build and Run unit test for Spark 3.5.3 (slow tests) + - name: Build and Run unit test for Spark 3.5.2 (slow tests) run: | cd $GITHUB_WORKSPACE/ $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ diff --git a/pom.xml b/pom.xml index d3c67f231e9e..030828a327d6 100644 --- a/pom.xml +++ b/pom.xml @@ -336,11 +336,11 @@ 3.5 spark-sql-columnar-shims-spark35 - 3.5.3 + 3.5.2 1.5.0 delta-spark - 3.2.1 - 32 + 3.2.0 + 32 0.15.0 2.15.1 3.3.4 diff --git a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala index ad99f7be2547..eab32ab9d0b9 100644 --- a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala +++ b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala @@ -20,7 +20,7 @@ import org.apache.gluten.sql.shims.{SparkShimDescriptor, SparkShims} import org.apache.gluten.sql.shims.spark35.SparkShimProvider.DESCRIPTOR object SparkShimProvider { - val DESCRIPTOR = SparkShimDescriptor(3, 5, 3) + val DESCRIPTOR = SparkShimDescriptor(3, 5, 2) } class SparkShimProvider extends org.apache.gluten.sql.shims.SparkShimProvider { diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 70b8d20c5ba4..bad4d6087f11 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -164,7 +164,7 @@ spark-3.5 - 3.5.3 + 3.5.2 2.12.18 From 73a1a01872779d5e454cef83da28c68455021b75 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Wed, 13 Nov 2024 15:11:11 +0800 Subject: [PATCH 084/211] [VL][CI] Fix back upload golden files (#7880) --- .github/workflows/velox_backend.yml | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 4b7f5836e745..fafceb1aa06c 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -564,6 +564,12 @@ jobs: with: name: test-report-spark32 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark32 + path: /tmp/tpch-approved-plan/** run-spark-test-spark32-slow: needs: build-native-lib-centos-7 @@ -633,6 +639,12 @@ jobs: with: name: test-report-spark33 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark33 + path: /tmp/tpch-approved-plan/** run-spark-test-spark33-slow: @@ -704,6 +716,12 @@ jobs: with: name: test-report-spark34 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark34 + path: /tmp/tpch-approved-plan/** run-spark-test-spark34-slow: @@ -775,6 +793,12 @@ jobs: with: name: test-report-spark35 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark35 + path: /tmp/tpch-approved-plan/** run-spark-test-spark35-scala213: needs: build-native-lib-centos-7 From 5a2814363b19064561eed656b5161c3eef6dcfe1 Mon Sep 17 00:00:00 2001 From: Yuan Date: Wed, 13 Nov 2024 19:24:02 +0800 Subject: [PATCH 085/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_13) (#7926) Signed-off-by: Yuan Zhou Co-authored-by: Hongze Zhang --- .../gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala | 3 ++- cpp/velox/memory/VeloxMemoryManager.cc | 4 +++- cpp/velox/operators/functions/RegistrationAllFunctions.cc | 8 -------- ep/build-velox/src/get_velox.sh | 2 +- 4 files changed, 6 insertions(+), 11 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index 5925666d2c00..2df2e2718eaa 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -153,7 +153,8 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { left.dataType.isInstanceOf[DecimalType] && right.dataType .isInstanceOf[DecimalType] && !SQLConf.get.decimalOperationsAllowPrecisionLoss ) { - val newName = "not_allow_precision_loss_" + // https://github.com/facebookincubator/velox/pull/10383 + val newName = substraitExprName + "_deny_precision_loss" GenericExpressionTransformer(newName, Seq(left, right), original) } else { GenericExpressionTransformer(substraitExprName, Seq(left, right), original) diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 19a5d458049b..1d281b22dc86 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -99,7 +99,9 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { } bool growCapacity(velox::memory::MemoryPool* pool, uint64_t targetBytes) override { - velox::memory::ScopedMemoryArbitrationContext ctx(pool); + // Set arbitration context to allow memory over-use during recursive arbitration. + // See MemoryPoolImpl::maybeIncrementReservation. + velox::memory::ScopedMemoryArbitrationContext ctx{}; velox::memory::MemoryPool* candidate; { std::unique_lock guard{mutex_}; diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.cc b/cpp/velox/operators/functions/RegistrationAllFunctions.cc index 6e2f90f0105b..ea9d3f4202f7 100644 --- a/cpp/velox/operators/functions/RegistrationAllFunctions.cc +++ b/cpp/velox/operators/functions/RegistrationAllFunctions.cc @@ -76,13 +76,6 @@ void registerFunctionOverwrite() { velox::functions::registerPrestoVectorFunctions(); } -void registerFunctionForConfig() { - const std::string prefix = "not_allow_precision_loss_"; - velox::functions::sparksql::registerDecimalAdd(prefix, false); - velox::functions::sparksql::registerDecimalSubtract(prefix, false); - velox::functions::sparksql::registerDecimalMultiply(prefix, false); - velox::functions::sparksql::registerDecimalDivide(prefix, false); -} } // namespace void registerAllFunctions() { @@ -96,7 +89,6 @@ void registerAllFunctions() { // Using function overwrite to handle function names mismatch between Spark // and Velox. registerFunctionOverwrite(); - registerFunctionForConfig(); } } // namespace gluten diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 3d72b950bf93..d802838b4b70 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_12 +VELOX_BRANCH=2024_11_13 VELOX_HOME="" OS=`uname -s` From 61033b719f8494ef80dd0c46fa6e738afa7e732d Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 13 Nov 2024 19:33:08 +0800 Subject: [PATCH 086/211] [GLUTEN-7243][VL] A follow-up fix for #7748 (#7935) --- cpp/velox/operators/plannodes/RowVectorStream.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/velox/operators/plannodes/RowVectorStream.h b/cpp/velox/operators/plannodes/RowVectorStream.h index ce26305fa054..e5a469afee3d 100644 --- a/cpp/velox/operators/plannodes/RowVectorStream.h +++ b/cpp/velox/operators/plannodes/RowVectorStream.h @@ -46,7 +46,7 @@ class RowVectorStream { // As of now, non-zero running threads usually happens when: // 1. Task A spills task B; // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again. - facebook::velox::exec::SuspendedSection(driverCtx_->driver); + facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); hasNext = iterator_->hasNext(); } if (!hasNext) { @@ -64,7 +64,7 @@ class RowVectorStream { { // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current // driver to make the current task open to spilling. - facebook::velox::exec::SuspendedSection(driverCtx_->driver); + facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); cb = iterator_->next(); } const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, cb); From d6326f06c8158de2eb08bb9ba7e3e4eb6e3964ef Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Wed, 13 Nov 2024 23:39:08 +0800 Subject: [PATCH 087/211] [GLUTEN-6666][VL] Use custom SparkExprToSubfieldFilterParser (#6754) Removes separating filter relevant code from Gluten. With a custom filter parser registered, we are able to use Velox provided filter extraction. --- cpp/velox/CMakeLists.txt | 1 + cpp/velox/compute/VeloxBackend.cc | 3 + .../SparkExprToSubfieldFilterParser.cc | 103 ++ .../SparkExprToSubfieldFilterParser.h | 37 + cpp/velox/substrait/SubstraitToVeloxPlan.cc | 1174 +---------------- cpp/velox/substrait/SubstraitToVeloxPlan.h | 341 ----- .../Substrait2VeloxPlanConversionTest.cc | 10 +- ep/build-velox/src/get_velox.sh | 2 +- 8 files changed, 151 insertions(+), 1520 deletions(-) create mode 100644 cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc create mode 100644 cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 586c08653dfc..329da49497f3 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -168,6 +168,7 @@ set(VELOX_SRCS memory/VeloxMemoryManager.cc operators/functions/RegistrationAllFunctions.cc operators/functions/RowConstructorWithNull.cc + operators/functions/SparkExprToSubfieldFilterParser.cc operators/serializer/VeloxColumnarToRowConverter.cc operators/serializer/VeloxColumnarBatchSerializer.cc operators/serializer/VeloxRowToColumnarConverter.cc diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index c1e907bea1d5..889f979b2869 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -33,6 +33,7 @@ #include "compute/VeloxRuntime.h" #include "config/VeloxConfig.h" #include "jni/JniFileSystem.h" +#include "operators/functions/SparkExprToSubfieldFilterParser.h" #include "udf/UdfLoader.h" #include "utils/Exception.h" #include "velox/common/caching/SsdCache.h" @@ -155,6 +156,8 @@ void VeloxBackend::init(const std::unordered_map& conf velox::parquet::registerParquetReaderFactory(); velox::parquet::registerParquetWriterFactory(); velox::orc::registerOrcReaderFactory(); + velox::exec::ExprToSubfieldFilterParser::registerParserFactory( + []() { return std::make_shared(); }); // Register Velox functions registerAllFunctions(); diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc new file mode 100644 index 000000000000..8ad537d8168a --- /dev/null +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "operators/functions/SparkExprToSubfieldFilterParser.h" + +namespace gluten { + +using namespace facebook::velox; + +bool SparkExprToSubfieldFilterParser::toSparkSubfield(const core::ITypedExpr* field, common::Subfield& subfield) { + std::vector> path; + for (auto* current = field;;) { + if (auto* fieldAccess = dynamic_cast(current)) { + path.push_back(std::make_unique(fieldAccess->name())); + } else if (dynamic_cast(current)) { + return false; + } else if (dynamic_cast(current) == nullptr) { + return false; + } else { + break; + } + + if (!current->inputs().empty()) { + return false; + } else { + break; + } + } + std::reverse(path.begin(), path.end()); + subfield = common::Subfield(std::move(path)); + return true; +} + +std::unique_ptr SparkExprToSubfieldFilterParser::leafCallToSubfieldFilter( + const core::CallTypedExpr& call, + common::Subfield& subfield, + core::ExpressionEvaluator* evaluator, + bool negated) { + if (call.inputs().empty()) { + return nullptr; + } + + const auto* leftSide = call.inputs()[0].get(); + + if (call.name() == "equalto") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeNotEqualFilter(call.inputs()[1], evaluator) : makeEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "lessthanorequal") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeGreaterThanFilter(call.inputs()[1], evaluator) + : makeLessThanOrEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "lessthan") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeGreaterThanOrEqualFilter(call.inputs()[1], evaluator) + : makeLessThanFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "greaterthanorequal") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeLessThanFilter(call.inputs()[1], evaluator) + : makeGreaterThanOrEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "greaterthan") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeLessThanOrEqualFilter(call.inputs()[1], evaluator) + : makeGreaterThanFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "in") { + if (toSparkSubfield(leftSide, subfield)) { + return makeInFilter(call.inputs()[1], evaluator, negated); + } + } else if (call.name() == "isnull") { + if (toSparkSubfield(leftSide, subfield)) { + if (negated) { + return exec::isNotNull(); + } + return exec::isNull(); + } + } else if (call.name() == "isnotnull") { + if (toSparkSubfield(leftSide, subfield)) { + if (negated) { + return exec::isNull(); + } + return exec::isNotNull(); + } + } + return nullptr; +} +} // namespace gluten diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h new file mode 100644 index 000000000000..d050091c0793 --- /dev/null +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "velox/expression/ExprToSubfieldFilter.h" + +namespace gluten { + +/// Parses Spark expression into subfield filter. Differences from Presto's parser include: +/// 1) Some Spark functions are registered under different names. +/// 2) The supported functions vary. +/// 3) Filter push-down on nested fields is disabled. +class SparkExprToSubfieldFilterParser : public facebook::velox::exec::ExprToSubfieldFilterParser { + public: + std::unique_ptr leafCallToSubfieldFilter( + const facebook::velox::core::CallTypedExpr& call, + facebook::velox::common::Subfield& subfield, + facebook::velox::core::ExpressionEvaluator* evaluator, + bool negated) override; + + private: + // Compared to the upstream 'toSubfield', the push-down of filter on nested field is disabled. + bool toSparkSubfield(const facebook::velox::core::ITypedExpr* field, facebook::velox::common::Subfield& subfield); +}; +} // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 9e2959043334..cdd9269e1494 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -73,45 +73,6 @@ EmitInfo getEmitInfo(const ::substrait::RelCommon& relCommon, const core::PlanNo return emitInfo; } -template -// Get the lowest value for numeric type. -T getLowest() { - return std::numeric_limits::lowest(); -} - -// Get the lowest value for string. -template <> -std::string getLowest() { - return ""; -} - -// Get the max value for numeric type. -template -T getMax() { - return std::numeric_limits::max(); -} - -// The max value will be used in BytesRange. Return empty string here instead. -template <> -std::string getMax() { - return ""; -} - -// Substrait function names. -const std::string sIsNotNull = "is_not_null"; -const std::string sIsNull = "is_null"; -const std::string sGte = "gte"; -const std::string sGt = "gt"; -const std::string sLte = "lte"; -const std::string sLt = "lt"; -const std::string sEqual = "equal"; -const std::string sOr = "or"; -const std::string sNot = "not"; - -// Substrait types. -const std::string sI32 = "i32"; -const std::string sI64 = "i64"; - /// @brief Get the input type from both sides of join. /// @param leftNode the plan node of left side. /// @param rightNode the plan node of right side. @@ -1190,37 +1151,10 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: tableHandle = std::make_shared( kHiveConnectorId, "hive_table", filterPushdownEnabled, connector::hive::SubfieldFilters{}, nullptr); } else { - // Flatten the conditions connected with 'and'. - std::vector<::substrait::Expression_ScalarFunction> scalarFunctions; - std::vector<::substrait::Expression_SingularOrList> singularOrLists; - std::vector<::substrait::Expression_IfThen> ifThens; - flattenConditions(readRel.filter(), scalarFunctions, singularOrLists, ifThens); - - // The vector's subscript stands for the column index. - std::vector rangeRecorders(veloxTypeList.size()); - - // Separate the filters to be two parts. The subfield part can be - // pushed down. - std::vector<::substrait::Expression_ScalarFunction> subfieldFunctions; - std::vector<::substrait::Expression_ScalarFunction> remainingFunctions; - std::vector<::substrait::Expression_SingularOrList> subfieldOrLists; - std::vector<::substrait::Expression_SingularOrList> remainingOrLists; - - separateFilters( - rangeRecorders, - scalarFunctions, - subfieldFunctions, - remainingFunctions, - singularOrLists, - subfieldOrLists, - remainingOrLists, - veloxTypeList, - splitInfo->format); - - // Create subfield filters based on the constructed filter info map. - auto subfieldFilters = createSubfieldFilters(colNameList, veloxTypeList, subfieldFunctions, subfieldOrLists); - // Connect the remaining filters with 'and'. - auto remainingFilter = connectWithAnd(colNameList, veloxTypeList, remainingFunctions, remainingOrLists, ifThens); + connector::hive::SubfieldFilters subfieldFilters; + auto names = colNameList; + auto types = veloxTypeList; + auto remainingFilter = exprConverter_->toVeloxExpr(readRel.filter(), ROW(std::move(names), std::move(types))); tableHandle = std::make_shared( kHiveConnectorId, "hive_table", filterPushdownEnabled, std::move(subfieldFilters), remainingFilter); @@ -1386,39 +1320,6 @@ void SubstraitToVeloxPlanConverter::constructFunctionMap(const ::substrait::Plan exprConverter_ = std::make_unique(pool_, functionMap_); } -void SubstraitToVeloxPlanConverter::flattenConditions( - const ::substrait::Expression& substraitFilter, - std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_IfThen>& ifThens) { - auto typeCase = substraitFilter.rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kScalarFunction: { - const auto& sFunc = substraitFilter.scalar_function(); - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, sFunc.function_reference()); - // TODO: Only and relation is supported here. - if (SubstraitParser::getNameBeforeDelimiter(filterNameSpec) == "and") { - for (const auto& sCondition : sFunc.arguments()) { - flattenConditions(sCondition.value(), scalarFunctions, singularOrLists, ifThens); - } - } else { - scalarFunctions.emplace_back(sFunc); - } - break; - } - case ::substrait::Expression::RexTypeCase::kSingularOrList: { - singularOrLists.emplace_back(substraitFilter.singular_or_list()); - break; - } - case ::substrait::Expression::RexTypeCase::kIfThen: { - ifThens.emplace_back(substraitFilter.if_then()); - break; - } - default: - VELOX_NYI("GetFlatConditions not supported for type '{}'", std::to_string(typeCase)); - } -} - std::string SubstraitToVeloxPlanConverter::findFuncSpec(uint64_t id) { return SubstraitParser::findFunctionSpec(functionMap_, id); } @@ -1481,878 +1382,6 @@ void SubstraitToVeloxPlanConverter::extractJoinKeys( } } -connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::createSubfieldFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists) { - // The vector's subscript stands for the column index. - std::vector columnToFilterInfo(inputTypeList.size()); - - // Process scalarFunctions. - for (const auto& scalarFunction : scalarFunctions) { - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getNameBeforeDelimiter(filterNameSpec); - - if (filterName == sNot) { - VELOX_CHECK(scalarFunction.arguments().size() == 1); - auto expr = scalarFunction.arguments()[0].value(); - if (expr.has_scalar_function()) { - // Set its child to filter info with reverse enabled. - setFilterInfo(expr.scalar_function(), inputTypeList, columnToFilterInfo, true); - } else if (expr.has_singular_or_list()) { - auto singularOrList = expr.singular_or_list(); - setFilterInfo(singularOrList, columnToFilterInfo, true); - } else { - VELOX_NYI("Only support push down Not with scalar function or In."); - } - } else if (filterName == sOr) { - VELOX_CHECK(scalarFunction.arguments().size() == 2); - VELOX_CHECK(std::all_of( - scalarFunction.arguments().cbegin(), - scalarFunction.arguments().cend(), - [](const ::substrait::FunctionArgument& arg) { - return arg.value().has_scalar_function() || arg.value().has_singular_or_list(); - })); - - // Set the children functions to filter info. They should be - // effective to the same field. - for (const auto& arg : scalarFunction.arguments()) { - const auto& expr = arg.value(); - if (expr.has_scalar_function()) { - setFilterInfo(arg.value().scalar_function(), inputTypeList, columnToFilterInfo); - } else if (expr.has_singular_or_list()) { - setFilterInfo(expr.singular_or_list(), columnToFilterInfo); - } else { - VELOX_NYI("Scalar function or SingularOrList expected."); - } - } - } else { - setFilterInfo(scalarFunction, inputTypeList, columnToFilterInfo); - } - } - - // Process singularOrLists. - for (const auto& list : singularOrLists) { - setFilterInfo(list, columnToFilterInfo); - } - - return mapToFilters(inputNameList, inputTypeList, columnToFilterInfo); -} - -bool SubstraitToVeloxPlanConverter::fieldOrWithLiteral( - const ::google::protobuf::RepeatedPtrField<::substrait::FunctionArgument>& arguments, - uint32_t& fieldIndex) { - if (arguments.size() == 1) { - if (arguments[0].value().has_selection()) { - // Only field exists. - return SubstraitParser::parseReferenceSegment(arguments[0].value().selection().direct_reference(), fieldIndex); - } else { - return false; - } - } - - if (arguments.size() != 2) { - // Not the field and literal combination. - return false; - } - bool fieldExists = false; - bool literalExists = false; - for (const auto& param : arguments) { - auto typeCase = param.value().rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: { - if (!SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), fieldIndex)) { - return false; - } - fieldExists = true; - break; - } - case ::substrait::Expression::RexTypeCase::kLiteral: { - literalExists = true; - break; - } - default: - break; - } - } - // Whether the field and literal both exist. - return fieldExists && literalExists; -} - -bool SubstraitToVeloxPlanConverter::childrenFunctionsOnSameField( - const ::substrait::Expression_ScalarFunction& function) { - // Get the column indices of the children functions. - std::vector colIndices; - for (const auto& arg : function.arguments()) { - if (arg.value().has_scalar_function()) { - const auto& scalarFunction = arg.value().scalar_function(); - for (const auto& param : scalarFunction.arguments()) { - if (param.value().has_selection()) { - const auto& field = param.value().selection(); - VELOX_CHECK(field.has_direct_reference()); - uint32_t colIdx; - if (!SubstraitParser::parseReferenceSegment(field.direct_reference(), colIdx)) { - return false; - } - colIndices.emplace_back(colIdx); - } - } - } else if (arg.value().has_singular_or_list()) { - const auto& singularOrList = arg.value().singular_or_list(); - colIndices.emplace_back(getColumnIndexFromSingularOrList(singularOrList)); - } else { - return false; - } - } - - if (std::all_of(colIndices.begin(), colIndices.end(), [&](uint32_t idx) { return idx == colIndices[0]; })) { - // All indices are the same. - return true; - } - return false; -} - -bool SubstraitToVeloxPlanConverter::canPushdownFunction( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::string& filterName, - uint32_t& fieldIdx) { - // Condtions can be pushed down. - static const std::unordered_set supportedFunctions = {sIsNotNull, sIsNull, sGte, sGt, sLte, sLt, sEqual}; - - bool canPushdown = false; - if (supportedFunctions.find(filterName) != supportedFunctions.end() && - fieldOrWithLiteral(scalarFunction.arguments(), fieldIdx)) { - // The arg should be field or field with literal. - canPushdown = true; - } - return canPushdown; -} - -bool SubstraitToVeloxPlanConverter::canPushdownNot( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders) { - VELOX_CHECK(scalarFunction.arguments().size() == 1, "Only one arg is expected for Not."); - const auto& notArg = scalarFunction.arguments()[0]; - if (notArg.value().has_singular_or_list()) { - auto singularOrList = notArg.value().singular_or_list(); - if (!canPushdownSingularOrList(singularOrList)) { - return false; - } - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - return rangeRecorders.at(colIdx).setInRange(); - } else if (notArg.value().has_scalar_function()) { - auto argFunction = - SubstraitParser::findFunctionSpec(functionMap_, notArg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(argFunction); - - static const std::unordered_set supportedNotFunctions = {sGte, sGt, sLte, sLt, sEqual}; - - uint32_t fieldIdx; - bool isFieldOrWithLiteral = fieldOrWithLiteral(notArg.value().scalar_function().arguments(), fieldIdx); - - return ( - supportedNotFunctions.find(functionName) != supportedNotFunctions.end() && isFieldOrWithLiteral && - rangeRecorders.at(fieldIdx).setCertainRangeForFunction(functionName, true /*reverse*/)); - } - return false; -} - -bool SubstraitToVeloxPlanConverter::canPushdownOr( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders) { - // OR Conditon whose children functions are on different columns is not - // supported to be pushed down. - if (!childrenFunctionsOnSameField(scalarFunction)) { - return false; - } - - static const std::unordered_set supportedOrFunctions = {sIsNotNull, sGte, sGt, sLte, sLt, sEqual}; - - for (const auto& arg : scalarFunction.arguments()) { - if (arg.value().has_scalar_function()) { - auto nameSpec = - SubstraitParser::findFunctionSpec(functionMap_, arg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); - - uint32_t fieldIdx; - bool isFieldOrWithLiteral = fieldOrWithLiteral(arg.value().scalar_function().arguments(), fieldIdx); - if (supportedOrFunctions.find(functionName) == supportedOrFunctions.end() || !isFieldOrWithLiteral || - !rangeRecorders.at(fieldIdx).setCertainRangeForFunction( - functionName, false /*reverse*/, true /*forOrRelation*/)) { - // The arg should be field or field with literal. - return false; - } - } else if (arg.value().has_singular_or_list()) { - const auto& singularOrList = arg.value().singular_or_list(); - if (!canPushdownSingularOrList(singularOrList, true)) { - return false; - } - uint32_t fieldIdx = getColumnIndexFromSingularOrList(singularOrList); - // Disable IN pushdown for int-like types. - if (!rangeRecorders.at(fieldIdx).setInRange(true /*forOrRelation*/)) { - return false; - } - } else { - // Or relation betweeen other expressions is not supported to be pushded - // down currently. - return false; - } - } - return true; -} - -void SubstraitToVeloxPlanConverter::separateFilters( - std::vector& rangeRecorders, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_SingularOrList>& subfieldOrLists, - std::vector<::substrait::Expression_SingularOrList>& remainingOrLists, - const std::vector& veloxTypeList, - const dwio::common::FileFormat& format) { - for (const auto& singularOrList : singularOrLists) { - if (!canPushdownSingularOrList(singularOrList)) { - remainingOrLists.emplace_back(singularOrList); - continue; - } - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - if (rangeRecorders.at(colIdx).setInRange()) { - subfieldOrLists.emplace_back(singularOrList); - } else { - remainingOrLists.emplace_back(singularOrList); - } - } - - for (const auto& scalarFunction : scalarFunctions) { - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getNameBeforeDelimiter(filterNameSpec); - // Add all decimal filters to remaining functions because their pushdown are not supported. - if (format == dwio::common::FileFormat::ORC && scalarFunction.arguments().size() > 0) { - auto value = scalarFunction.arguments().at(0).value(); - if (value.has_selection()) { - uint32_t fieldIndex; - bool parsed = SubstraitParser::parseReferenceSegment(value.selection().direct_reference(), fieldIndex); - if (!parsed || (!veloxTypeList.empty() && veloxTypeList.at(fieldIndex)->isDecimal())) { - remainingFunctions.emplace_back(scalarFunction); - continue; - } - } - } - - // Check whether NOT and OR functions can be pushed down. - // If yes, the scalar function will be added into the subfield functions. - if (filterName == sNot) { - if (canPushdownNot(scalarFunction, rangeRecorders)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } else if (filterName == sOr) { - if (canPushdownOr(scalarFunction, rangeRecorders)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } else { - // Check if the condition is supported to be pushed down. - uint32_t fieldIdx; - if (canPushdownFunction(scalarFunction, filterName, fieldIdx) && - rangeRecorders.at(fieldIdx).setCertainRangeForFunction(filterName)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } - } -} - -bool SubstraitToVeloxPlanConverter::RangeRecorder::setCertainRangeForFunction( - const std::string& functionName, - bool reverse, - bool forOrRelation) { - if (functionName == sLt || functionName == sLte) { - if (reverse) { - return setLeftBound(forOrRelation); - } else { - return setRightBound(forOrRelation); - } - } else if (functionName == sGt || functionName == sGte) { - if (reverse) { - return setRightBound(forOrRelation); - } else { - return setLeftBound(forOrRelation); - } - } else if (functionName == sEqual) { - if (reverse) { - // Not equal means lt or gt. - return setMultiRange(); - } else { - return setLeftBound(forOrRelation) && setRightBound(forOrRelation); - } - } else if (functionName == sOr) { - if (reverse) { - // Not supported. - return false; - } else { - return setMultiRange(); - } - } else if (functionName == sIsNotNull) { - if (reverse) { - // Not supported. - return false; - } else { - // Is not null can always coexist with the other range. - return true; - } - } else if (functionName == sIsNull) { - if (reverse) { - return setCertainRangeForFunction(sIsNotNull, false, forOrRelation); - } else { - return setIsNull(); - } - } else { - return false; - } -} - -void SubstraitToVeloxPlanConverter::setColumnFilterInfo( - const std::string& filterName, - std::optional literalVariant, - FilterInfo& columnFilterInfo, - bool reverse) { - if (filterName == sIsNotNull) { - if (reverse) { - columnFilterInfo.setNull(); - } else { - columnFilterInfo.forbidsNull(); - } - } else if (filterName == sIsNull) { - if (reverse) { - columnFilterInfo.forbidsNull(); - } else { - columnFilterInfo.setNull(); - } - } else if (filterName == sGte) { - if (reverse) { - columnFilterInfo.setUpper(literalVariant, true); - } else { - columnFilterInfo.setLower(literalVariant, false); - } - } else if (filterName == sGt) { - if (reverse) { - columnFilterInfo.setUpper(literalVariant, false); - } else { - columnFilterInfo.setLower(literalVariant, true); - } - } else if (filterName == sLte) { - if (reverse) { - columnFilterInfo.setLower(literalVariant, true); - } else { - columnFilterInfo.setUpper(literalVariant, false); - } - } else if (filterName == sLt) { - if (reverse) { - columnFilterInfo.setLower(literalVariant, false); - } else { - columnFilterInfo.setUpper(literalVariant, true); - } - } else if (filterName == sEqual) { - if (reverse) { - columnFilterInfo.setNotValue(literalVariant); - } else { - columnFilterInfo.setLower(literalVariant, false); - columnFilterInfo.setUpper(literalVariant, false); - } - } else { - VELOX_NYI("setColumnFilterInfo not supported for filter name '{}'", filterName); - } -} - -template -variant getVariantFromLiteral(const ::substrait::Expression::Literal& literal) { - using LitT = typename facebook::velox::TypeTraits::NativeType; - return variant(SubstraitParser::getLiteralValue(literal)); -} - -void SubstraitToVeloxPlanConverter::setFilterInfo( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo, - bool reverse) { - auto nameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); - - // Extract the column index and column bound from the scalar function. - std::optional colIdx; - std::optional<::substrait::Expression_Literal> substraitLit; - std::vector typeCases; - - for (const auto& param : scalarFunction.arguments()) { - auto typeCase = param.value().rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: { - typeCases.emplace_back("kSelection"); - uint32_t index; - VELOX_CHECK( - SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), index), - "Failed to parse the column index from the selection."); - colIdx = index; - break; - } - case ::substrait::Expression::RexTypeCase::kLiteral: { - typeCases.emplace_back("kLiteral"); - substraitLit = param.value().literal(); - break; - } - default: - VELOX_NYI("Substrait conversion not supported for arg type '{}'", std::to_string(typeCase)); - } - } - - static const std::unordered_map functionRevertMap = { - {sLt, sGt}, {sGt, sLt}, {sGte, sLte}, {sLte, sGte}}; - - // Handle the case where literal is before the variable in a binary function, e.g. "123 < q1". - if (typeCases.size() > 1 && (typeCases[0] == "kLiteral" && typeCases[1] == "kSelection")) { - auto x = functionRevertMap.find(functionName); - if (x != functionRevertMap.end()) { - // Change the function name: lt => gt, gt => lt, gte => lte, lte => gte. - functionName = x->second; - } - } - - if (!colIdx.has_value()) { - VELOX_NYI("Column index is expected in subfield filters creation."); - } - - // Set the extracted bound to the specific column. - uint32_t colIdxVal = colIdx.value(); - std::optional val; - - auto inputType = inputTypeList[colIdxVal]; - switch (inputType->kind()) { - case TypeKind::TINYINT: - case TypeKind::SMALLINT: - case TypeKind::INTEGER: - case TypeKind::BIGINT: - case TypeKind::REAL: - case TypeKind::DOUBLE: - case TypeKind::BOOLEAN: - case TypeKind::VARCHAR: - case TypeKind::HUGEINT: - if (substraitLit) { - auto kind = inputType->kind(); - val = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH(getVariantFromLiteral, kind, substraitLit.value()); - } - break; - case TypeKind::ARRAY: - case TypeKind::MAP: - case TypeKind::ROW: - // Doing nothing here can let filter IsNotNull still work. - break; - default: - VELOX_NYI("Subfield filters creation not supported for input type '{}' in setFilterInfo", inputType->toString()); - } - - setColumnFilterInfo(functionName, val, columnToFilterInfo[colIdxVal], reverse); -} - -template -void SubstraitToVeloxPlanConverter::createNotEqualFilter( - variant notVariant, - bool nullAllowed, - std::vector>& colFilters) { - using NativeType = typename RangeTraits::NativeType; - using RangeType = typename RangeTraits::RangeType; - // Value > lower - std::unique_ptr lowerFilter; - if constexpr (std::is_same_v) { - if (notVariant.value() < getMax()) { - lowerFilter = std::make_unique( - notVariant.value() + 1 /*lower*/, getMax() /*upper*/, nullAllowed); - } - } else { - lowerFilter = std::make_unique( - notVariant.value() /*lower*/, - false /*lowerUnbounded*/, - true /*lowerExclusive*/, - getMax() /*upper*/, - true /*upperUnbounded*/, - false /*upperExclusive*/, - nullAllowed); - } - - // Value < upper - std::unique_ptr upperFilter; - if constexpr (std::is_same_v) { - if (getLowest() < notVariant.value()) { - upperFilter = std::make_unique( - getLowest() /*lower*/, notVariant.value() - 1 /*upper*/, nullAllowed); - } - } else { - upperFilter = std::make_unique( - getLowest() /*lower*/, - true /*lowerUnbounded*/, - false /*lowerExclusive*/, - notVariant.value() /*upper*/, - false /*upperUnbounded*/, - true /*upperExclusive*/, - nullAllowed); - } - - // To avoid overlap of BigintMultiRange, keep this appending order to make sure lower bound of one range is less than - // the upper bounds of others. - if (upperFilter) { - colFilters.emplace_back(std::move(upperFilter)); - } - if (lowerFilter) { - colFilters.emplace_back(std::move(lowerFilter)); - } -} - -template -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) {} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for small int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for tiny int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - std::string value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(values, nullAllowed); - } -} - -template -void SubstraitToVeloxPlanConverter::setSubfieldFilter( - std::vector> colFilters, - const std::string& inputName, - bool nullAllowed, - connector::hive::SubfieldFilters& filters) { - using MultiRangeType = typename RangeTraits::MultiRangeType; - - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters[0]); - } else if (colFilters.size() > 1) { - // BigintMultiRange should have been sorted - if (colFilters[0]->kind() == common::FilterKind::kBigintRange) { - std::sort(colFilters.begin(), colFilters.end(), [](const auto& a, const auto& b) { - return dynamic_cast(a.get())->lower() < - dynamic_cast(b.get())->lower(); - }); - } - if constexpr (std::is_same_v) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed); - } - } -} - -template -void SubstraitToVeloxPlanConverter::constructSubfieldFilters( - uint32_t colIdx, - const std::string& inputName, - const TypePtr& inputType, - const FilterInfo& filterInfo, - connector::hive::SubfieldFilters& filters) { - if (!filterInfo.isInitialized()) { - return; - } - - bool nullAllowed = filterInfo.nullAllowed_; - bool isNull = filterInfo.isNull_; - bool existIsNullAndIsNotNull = filterInfo.forbidsNullSet_ && filterInfo.isNullSet_; - uint32_t rangeSize = std::max(filterInfo.lowerBounds_.size(), filterInfo.upperBounds_.size()); - - if constexpr (KIND == facebook::velox::TypeKind::HUGEINT) { - // TODO: open it when the Velox's modification is ready. - VELOX_NYI("constructSubfieldFilters not support for HUGEINT type"); - } else if constexpr (KIND == facebook::velox::TypeKind::BOOLEAN) { - // Handle bool type filters. - // Not equal. - if (filterInfo.notValue_) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(!filterInfo.notValue_.value().value(), nullAllowed); - } else if (filterInfo.notValues_.size() > 0) { - std::set notValues; - for (auto v : filterInfo.notValues_) { - notValues.emplace(v.value()); - } - if (notValues.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(!(*notValues.begin()), nullAllowed); - } else { - // if there are more than one distinct value in NOT IN list, the filter should be AlwaysFalse - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } - } else if (rangeSize == 0) { - // IsNull/IsNotNull. - if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported in constructSubfieldFilters when no other filter ranges."); - } - return; - } else { - // Equal. - auto value = filterInfo.lowerBounds_[0].value().value(); - VELOX_CHECK(value == filterInfo.upperBounds_[0].value().value(), "invalid state of bool equal"); - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(value, nullAllowed); - } - } else if constexpr ( - KIND == facebook::velox::TypeKind::ARRAY || KIND == facebook::velox::TypeKind::MAP || - KIND == facebook::velox::TypeKind::ROW) { - // Only IsNotNull and IsNull are supported for complex types. - VELOX_CHECK_EQ(rangeSize, 0, "Only IsNotNull and IsNull are supported for complex type."); - if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported for input type '{}'.", inputType->toString()); - } - } else { - using NativeType = typename RangeTraits::NativeType; - using RangeType = typename RangeTraits::RangeType; - using MultiRangeType = typename RangeTraits::MultiRangeType; - - // Handle 'in' filter. - if (filterInfo.values_.size() > 0) { - // To filter out null is a default behaviour of Spark IN expression. - nullAllowed = false; - setInFilter(filterInfo.values_, nullAllowed, false, inputName, filters); - // Currently, In cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after IN filter."); - VELOX_CHECK(!filterInfo.notValue_.has_value(), "Not equal cannot be supported after IN filter."); - VELOX_CHECK(filterInfo.notValues_.size() == 0, "Not in cannot be supported after IN filter."); - return; - } - - // Handle not in filter. - if (filterInfo.notValues_.size() > 0) { - setInFilter(filterInfo.notValues_, filterInfo.nullAllowed_, true, inputName, filters); - // Currently, NOT In cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after NOT IN filter."); - VELOX_CHECK(!filterInfo.notValue_.has_value(), "Not equal cannot be supported after NOT IN filter."); - return; - } - - // Construct the Filters. - std::vector> colFilters; - - // Handle not(equal) filter. - if (filterInfo.notValue_) { - variant notVariant = filterInfo.notValue_.value(); - createNotEqualFilter(notVariant, filterInfo.nullAllowed_, colFilters); - // Currently, Not-equal cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after not-equal filter."); - if constexpr (std::is_same_v) { - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters.front()); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); - } - } else { - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters.front()); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed); - } - } - return; - } - - // Handle null filtering. - if (rangeSize == 0) { - // handle is not null and is null exists at same time - if (existIsNullAndIsNotNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(std::make_unique()); - } else if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported in constructSubfieldFilters when no other filter ranges."); - } - return; - } - - NativeType lowerBound; - if constexpr (KIND == facebook::velox::TypeKind::BIGINT) { - if (inputType->isShortDecimal()) { - lowerBound = DecimalUtil::kShortDecimalMin; - } else { - lowerBound = getLowest(); - } - } else { - lowerBound = getLowest(); - } - - NativeType upperBound; - if constexpr (KIND == facebook::velox::TypeKind::BIGINT) { - if (inputType->isShortDecimal()) { - upperBound = DecimalUtil::kShortDecimalMax; - } else { - upperBound = getMax(); - } - } else { - upperBound = getMax(); - } - - [[maybe_unused]] bool lowerUnbounded = true; - [[maybe_unused]] bool upperUnbounded = true; - bool lowerExclusive = false; - bool upperExclusive = false; - - // Handle other filter ranges. - for (uint32_t idx = 0; idx < rangeSize; idx++) { - if (idx < filterInfo.lowerBounds_.size() && filterInfo.lowerBounds_[idx]) { - lowerUnbounded = false; - variant lowerVariant = filterInfo.lowerBounds_[idx].value(); - lowerBound = lowerVariant.value(); - lowerExclusive = filterInfo.lowerExclusives_[idx]; - } - - if (idx < filterInfo.upperBounds_.size() && filterInfo.upperBounds_[idx]) { - upperUnbounded = false; - variant upperVariant = filterInfo.upperBounds_[idx].value(); - upperBound = upperVariant.value(); - upperExclusive = filterInfo.upperExclusives_[idx]; - } - - std::unique_ptr filter; - if constexpr (std::is_same_v) { - filter = std::move(std::make_unique( - lowerExclusive ? lowerBound + 1 : lowerBound, upperExclusive ? upperBound - 1 : upperBound, nullAllowed)); - } else { - filter = std::move(std::make_unique( - lowerBound, lowerUnbounded, lowerExclusive, upperBound, upperUnbounded, upperExclusive, nullAllowed)); - } - - colFilters.emplace_back(std::move(filter)); - } - - // Set the SubfieldFilter. - setSubfieldFilter(std::move(colFilters), inputName, filterInfo.nullAllowed_, filters); - } -} - bool SubstraitToVeloxPlanConverter::checkTypeExtension(const ::substrait::Plan& substraitPlan) { for (const auto& sExtension : substraitPlan.extensions()) { if (!sExtension.has_extension_type()) { @@ -2367,199 +1396,4 @@ bool SubstraitToVeloxPlanConverter::checkTypeExtension(const ::substrait::Plan& return true; } -connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::mapToFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo) { - // Construct the subfield filters based on the filter info map. - connector::hive::SubfieldFilters filters; - for (uint32_t colIdx = 0; colIdx < inputNameList.size(); colIdx++) { - if (columnToFilterInfo[colIdx].isInitialized()) { - auto inputType = inputTypeList[colIdx]; - if (inputType->isDate()) { - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - continue; - } - switch (inputType->kind()) { - case TypeKind::TINYINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::SMALLINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::INTEGER: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::BIGINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::REAL: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::DOUBLE: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::BOOLEAN: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::VARCHAR: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::HUGEINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::ARRAY: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::MAP: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::ROW: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - default: - VELOX_NYI( - "Subfield filters creation not supported for input type '{}' in mapToFilters", inputType->toString()); - } - } - } - - return filters; -} - -core::TypedExprPtr SubstraitToVeloxPlanConverter::connectWithAnd( - std::vector inputNameList, - std::vector inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - const std::vector<::substrait::Expression_IfThen>& ifThens) { - if (scalarFunctions.size() == 0 && singularOrLists.size() == 0 && ifThens.size() == 0) { - return nullptr; - } - auto inputType = ROW(std::move(inputNameList), std::move(inputTypeList)); - - // Filter for scalar functions. - std::vector allFilters; - for (auto scalar : scalarFunctions) { - auto filter = exprConverter_->toVeloxExpr(scalar, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - - for (auto orList : singularOrLists) { - auto filter = exprConverter_->toVeloxExpr(orList, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - - for (auto ifThen : ifThens) { - auto filter = exprConverter_->toVeloxExpr(ifThen, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - VELOX_CHECK_GT(allFilters.size(), 0, "One filter should be valid."); - core::TypedExprPtr andFilter = allFilters[0]; - for (auto i = 1; i < allFilters.size(); i++) { - andFilter = connectWithAnd(andFilter, allFilters[i]); - } - - return andFilter; -} - -core::TypedExprPtr SubstraitToVeloxPlanConverter::connectWithAnd( - core::TypedExprPtr leftExpr, - core::TypedExprPtr rightExpr) { - std::vector params; - params.reserve(2); - params.emplace_back(leftExpr); - params.emplace_back(rightExpr); - return std::make_shared(BOOLEAN(), std::move(params), "and"); -} - -bool SubstraitToVeloxPlanConverter::canPushdownSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList, - bool disableIntLike) { - VELOX_CHECK(singularOrList.options_size() > 0, "At least one option is expected."); - // Check whether the value is field. - bool hasField = singularOrList.value().has_selection(); - const auto& options = singularOrList.options(); - for (const auto& option : options) { - VELOX_CHECK(option.has_literal(), "Literal is expected as option."); - auto type = option.literal().literal_type_case(); - // Only BigintValues and BytesValues are supported. - if (type != ::substrait::Expression_Literal::LiteralTypeCase::kI32 && - type != ::substrait::Expression_Literal::LiteralTypeCase::kI64 && - type != ::substrait::Expression_Literal::LiteralTypeCase::kString) { - return false; - } - - // BigintMultiRange can only accept BigintRange, so disableIntLike is set to - // true for OR pushdown of int-like types. - if (disableIntLike && - (type == ::substrait::Expression_Literal::LiteralTypeCase::kI32 || - type == ::substrait::Expression_Literal::LiteralTypeCase::kI64)) { - return false; - } - } - return hasField; -} - -uint32_t SubstraitToVeloxPlanConverter::getColumnIndexFromSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList) { - // Get the column index. - ::substrait::Expression_FieldReference selection; - if (singularOrList.value().has_scalar_function()) { - selection = singularOrList.value().scalar_function().arguments()[0].value().selection(); - } else if (singularOrList.value().has_selection()) { - selection = singularOrList.value().selection(); - } else { - VELOX_FAIL("Unsupported type in IN pushdown."); - } - uint32_t index; - VELOX_CHECK( - SubstraitParser::parseReferenceSegment(selection.direct_reference(), index), - "Failed to parse column index from SingularOrList."); - return index; -} - -void SubstraitToVeloxPlanConverter::setFilterInfo( - const ::substrait::Expression_SingularOrList& singularOrList, - std::vector& columnToFilterInfo, - bool reverse) { - VELOX_CHECK(singularOrList.options_size() > 0, "At least one option is expected."); - // Get the column index. - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - - // Get the value list. - const auto& options = singularOrList.options(); - std::vector variants; - variants.reserve(options.size()); - for (const auto& option : options) { - VELOX_CHECK(option.has_literal(), "Literal is expected as option."); - variants.emplace_back(exprConverter_->toVeloxExpr(option.literal())->value()); - } - // Set the value list to filter info. - if (!reverse) { - columnToFilterInfo[colIdx].setValues(variants); - } else { - columnToFilterInfo[colIdx].setNotValues(variants); - } -} - } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.h b/cpp/velox/substrait/SubstraitToVeloxPlan.h index 0e892469d098..51e50ce34767 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.h @@ -215,354 +215,13 @@ class SubstraitToVeloxPlanConverter { /// if output order is 'kDriect'. core::PlanNodePtr processEmit(const ::substrait::RelCommon& relCommon, const core::PlanNodePtr& noEmitNode); - /// Multiple conditions are connected to a binary tree structure with - /// the relation key words, including AND, OR, and etc. Currently, only - /// AND is supported. This function is used to extract all the Substrait - /// conditions in the binary tree structure into a vector. - void flattenConditions( - const ::substrait::Expression& sFilter, - std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_IfThen>& ifThens); - /// Check the Substrait type extension only has one unknown extension. static bool checkTypeExtension(const ::substrait::Plan& substraitPlan); - /// Range filter recorder for a field is used to make sure only the conditions - /// that can coexist for this field being pushed down with a range filter. - class RangeRecorder { - public: - /// Set the existence of values range and returns whether this condition can - /// coexist with existing conditions for one field. Conditions in OR - /// relation can coexist with each other. - bool setInRange(bool forOrRelation = false) { - if (forOrRelation) { - return true; - } - if (inRange_ || multiRange_ || leftBound_ || rightBound_ || isNull_) { - return false; - } - inRange_ = true; - return true; - } - - /// Set the existence of left bound and returns whether it can coexist with - /// existing conditions for this field. - bool setLeftBound(bool forOrRelation = false) { - if (forOrRelation) { - if (!rightBound_) - leftBound_ = true; - return !rightBound_; - } - if (leftBound_ || inRange_ || multiRange_ || isNull_) { - return false; - } - leftBound_ = true; - return true; - } - - /// Set the existence of right bound and returns whether it can coexist with - /// existing conditions for this field. - bool setRightBound(bool forOrRelation = false) { - if (forOrRelation) { - if (!leftBound_) - rightBound_ = true; - return !leftBound_; - } - if (rightBound_ || inRange_ || multiRange_ || isNull_) { - return false; - } - rightBound_ = true; - return true; - } - - /// Set the existence of multi-range and returns whether it can coexist with - /// existing conditions for this field. - bool setMultiRange() { - if (inRange_ || multiRange_ || leftBound_ || rightBound_ || isNull_) { - return false; - } - multiRange_ = true; - return true; - } - - /// Set the existence of IsNull and returns whether it can coexist with - /// existing conditions for this field. - bool setIsNull() { - if (inRange_ || multiRange_ || leftBound_ || rightBound_) { - return false; - } - isNull_ = true; - return true; - } - - /// Set certain existence according to function name and returns whether it - /// can coexist with existing conditions for this field. - bool setCertainRangeForFunction(const std::string& functionName, bool reverse = false, bool forOrRelation = false); - - private: - /// The existence of values range. - bool inRange_ = false; - - /// The existence of left bound. - bool leftBound_ = false; - - /// The existence of right bound. - bool rightBound_ = false; - - /// The existence of multi-range. - bool multiRange_ = false; - - /// The existence of IsNull. - bool isNull_ = false; - }; - - /// Filter info for a column used in filter push down. - class FilterInfo { - public: - // Null is not allowed. - void forbidsNull() { - nullAllowed_ = false; - if (!initialized_) { - initialized_ = true; - } - forbidsNullSet_ = true; - } - - // Only null is allowed. - void setNull() { - isNull_ = true; - nullAllowed_ = true; - if (!initialized_) { - initialized_ = true; - } - isNullSet_ = true; - } - - // Return the initialization status. - bool isInitialized() const { - return initialized_; - } - - // Add a lower bound to the range. Multiple lower bounds are - // regarded to be in 'or' relation. - void setLower(const std::optional& left, bool isExclusive) { - lowerBounds_.emplace_back(left); - lowerExclusives_.emplace_back(isExclusive); - if (!initialized_) { - initialized_ = true; - } - } - - // Add a upper bound to the range. Multiple upper bounds are - // regarded to be in 'or' relation. - void setUpper(const std::optional& right, bool isExclusive) { - upperBounds_.emplace_back(right); - upperExclusives_.emplace_back(isExclusive); - if (!initialized_) { - initialized_ = true; - } - } - - // Set a list of values to be used in the push down of 'in' expression. - void setValues(const std::vector& values) { - for (const auto& value : values) { - values_.emplace_back(value); - } - if (!initialized_) { - initialized_ = true; - } - } - - // Set a value for the not(equal) condition. - void setNotValue(const std::optional& notValue) { - notValue_ = notValue; - if (!initialized_) { - initialized_ = true; - } - } - - // Set a list of values to be used in the push down of 'not in' expression. - void setNotValues(const std::vector& notValues) { - for (const auto& value : notValues) { - notValues_.emplace_back(value); - } - if (!initialized_) { - initialized_ = true; - } - } - - // Whether this filter map is initialized. - bool initialized_ = false; - - bool nullAllowed_ = false; - bool isNull_ = false; - bool forbidsNullSet_ = false; - bool isNullSet_ = false; - - // If true, left bound will be exclusive. - std::vector lowerExclusives_; - - // If true, right bound will be exclusive. - std::vector upperExclusives_; - - // A value should not be equal to. - std::optional notValue_ = std::nullopt; - - // The lower bounds in 'or' relation. - std::vector> lowerBounds_; - - // The upper bounds in 'or' relation. - std::vector> upperBounds_; - - // The list of values used in 'in' expression. - std::vector values_; - - // The list of values should not be equal to. - std::vector notValues_; - }; - /// Returns unique ID to use for plan node. Produces sequential numbers /// starting from zero. std::string nextPlanNodeId(); - /// Returns whether the args of a scalar function being field or - /// field with literal. If yes, extract and set the field index. - static bool fieldOrWithLiteral( - const ::google::protobuf::RepeatedPtrField<::substrait::FunctionArgument>& arguments, - uint32_t& fieldIndex); - - /// Separate the functions to be two parts: - /// subfield functions to be handled by the subfieldFilters in HiveConnector, - /// and remaining functions to be handled by the remainingFilter in - /// HiveConnector. - void separateFilters( - std::vector& rangeRecorders, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_SingularOrList>& subfieldrOrLists, - std::vector<::substrait::Expression_SingularOrList>& remainingrOrLists, - const std::vector& veloxTypeList, - const dwio::common::FileFormat& format); - - /// Returns whether a function can be pushed down. - static bool canPushdownFunction( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::string& filterName, - uint32_t& fieldIdx); - - /// Returns whether a NOT function can be pushed down. - bool canPushdownNot( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders); - - /// Returns whether a OR function can be pushed down. - bool canPushdownOr( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders); - - /// Returns whether a SingularOrList can be pushed down. - static bool canPushdownSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList, - bool disableIntLike = false); - - /// Check whether the children functions of this scalar function have the same - /// column index. Curretly used to check whether the two chilren functions of - /// 'or' expression are effective on the same column. - static bool childrenFunctionsOnSameField(const ::substrait::Expression_ScalarFunction& function); - - /// Extract the scalar function, and set the filter info for different types - /// of columns. If reverse is true, the opposite filter info will be set. - void setFilterInfo( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo, - bool reverse = false); - - /// Extract SingularOrList and set it to the filter info map. - /// If reverse is true, the opposite filter info will be set. - void setFilterInfo( - const ::substrait::Expression_SingularOrList& singularOrList, - std::vector& columnToFilterInfo, - bool reverse = false); - - /// Extract SingularOrList and returns the field index. - static uint32_t getColumnIndexFromSingularOrList(const ::substrait::Expression_SingularOrList&); - - /// Set the filter info for a column base on the information - /// extracted from filter condition. - static void setColumnFilterInfo( - const std::string& filterName, - std::optional literalVariant, - FilterInfo& columnToFilterInfo, - bool reverse); - - /// Create a multirange to specify the filter 'x != notValue' with: - /// x > notValue or x < notValue. - template - void createNotEqualFilter(variant notVariant, bool nullAllowed, std::vector>& colFilters); - - /// Create a values range to handle (not) in filter. - /// variants: the list of values extracted from the (not) in expression. - // negated: false for IN filter, true for NOT IN filter. - /// inputName: the column input name. - template - void setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters); - - /// Set the constructed filters into SubfieldFilters. - /// The FilterType is used to distinguish BigintRange and - /// Filter (the base class). This is needed because BigintMultiRange - /// can only accept the unique ptr of BigintRange as parameter. - template - void setSubfieldFilter( - std::vector> colFilters, - const std::string& inputName, - bool nullAllowed, - connector::hive::SubfieldFilters& filters); - - /// Create the subfield filter based on the constructed filter info. - /// inputName: the input name of a column. - template - void constructSubfieldFilters( - uint32_t colIdx, - const std::string& inputName, - const TypePtr& inputType, - const FilterInfo& filterInfo, - connector::hive::SubfieldFilters& filters); - - /// Construct subfield filters according to the pre-set map of filter info. - connector::hive::SubfieldFilters mapToFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo); - - /// Convert subfield functions into subfieldFilters to - /// be used in Hive Connector. - connector::hive::SubfieldFilters createSubfieldFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists); - - /// Connect all remaining functions with 'and' relation - /// for the use of remaingFilter in Hive Connector. - core::TypedExprPtr connectWithAnd( - std::vector inputNameList, - std::vector inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - const std::vector<::substrait::Expression_IfThen>& ifThens); - - /// Connect the left and right expressions with 'and' relation. - core::TypedExprPtr connectWithAnd(core::TypedExprPtr leftExpr, core::TypedExprPtr rightExpr); - /// Used to convert AggregateRel into Velox plan node. /// The output of child node will be used as the input of Aggregation. std::shared_ptr toVeloxAgg( diff --git a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc index 06d4ea019572..3193f53bfbf1 100644 --- a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc +++ b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc @@ -256,12 +256,7 @@ TEST_F(Substrait2VeloxPlanConversionTest, ifthenTest) { // Convert to Velox PlanNode. auto planNode = planConverter_->toVeloxPlan(substraitPlan, std::vector<::substrait::ReadRel_LocalFiles>{split}); ASSERT_EQ( - "-- Project[1][expressions: ] -> \n " - "-- TableScan[0][table: hive_table, range filters: [(hd_demo_sk, Filter(IsNotNull, deterministic, null not allowed))," - " (hd_vehicle_count, BigintRange: [1, 9223372036854775807] no nulls)], remaining filter: " - "(and(or(equalto(\"hd_buy_potential\",\">10000\"),equalto(\"hd_buy_potential\",\"unknown\"))," - "if(greaterthan(\"hd_vehicle_count\",0),greaterthan(divide(cast \"hd_dep_count\" as DOUBLE," - "cast \"hd_vehicle_count\" as DOUBLE),1.2))))] -> n0_0:BIGINT, n0_1:VARCHAR, n0_2:BIGINT, n0_3:BIGINT\n", + "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, remaining filter: (and(and(and(and(isnotnull(\"hd_vehicle_count\"),or(equalto(\"hd_buy_potential\",\">10000\"),equalto(\"hd_buy_potential\",\"unknown\"))),greaterthan(\"hd_vehicle_count\",0)),if(greaterthan(\"hd_vehicle_count\",0),greaterthan(divide(cast \"hd_dep_count\" as DOUBLE,cast \"hd_vehicle_count\" as DOUBLE),1.2))),isnotnull(\"hd_demo_sk\")))] -> n0_0:BIGINT, n0_1:VARCHAR, n0_2:BIGINT, n0_3:BIGINT\n", planNode->toString(true, true)); } @@ -277,8 +272,7 @@ TEST_F(Substrait2VeloxPlanConversionTest, filterUpper) { // Convert to Velox PlanNode. auto planNode = planConverter_->toVeloxPlan(substraitPlan, std::vector<::substrait::ReadRel_LocalFiles>{split}); ASSERT_EQ( - "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, range filters: " - "[(key, BigintRange: [-2147483648, 2] no nulls)]] -> n0_0:INTEGER\n", + "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, remaining filter: (and(isnotnull(\"key\"),lessthan(\"key\",3)))] -> n0_0:INTEGER\n", planNode->toString(true, true)); } } // namespace gluten diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index d802838b4b70..2be1481fc05d 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_13 +VELOX_BRANCH=2024_11_13_new VELOX_HOME="" OS=`uname -s` From 0b899c046f2f0b9c141518f4ef43d2075540d93c Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Thu, 14 Nov 2024 10:28:16 +0800 Subject: [PATCH 088/211] [GLUTEN-7641][VL] Add Gluten benchmark scripts (#7642) --- tools/workload/benchmark_velox/README.md | 38 + tools/workload/benchmark_velox/init_disks.py | 103 + .../workload/benchmark_velox/initialize.ipynb | 2918 +++++++++++++++++ .../native_sql_initialize.ipynb | 1388 ++++++++ .../benchmark_velox/params.yaml.template | 71 + .../benchmark_velox/run_tpc_workload.sh | 86 + .../benchmark_velox/tpc_workload.ipynb | 381 +++ .../parquet_dataset/tpcds_datagen_parquet.sh | 10 +- .../tpch_convert_parquet_dwrf.scala | 62 - .../dwrf_dataset/tpch_convert_parquet_dwrf.sh | 47 - .../parquet_dataset/tpch_datagen_parquet.sh | 8 +- 11 files changed, 4992 insertions(+), 120 deletions(-) create mode 100644 tools/workload/benchmark_velox/README.md create mode 100644 tools/workload/benchmark_velox/init_disks.py create mode 100644 tools/workload/benchmark_velox/initialize.ipynb create mode 100644 tools/workload/benchmark_velox/native_sql_initialize.ipynb create mode 100644 tools/workload/benchmark_velox/params.yaml.template create mode 100755 tools/workload/benchmark_velox/run_tpc_workload.sh create mode 100644 tools/workload/benchmark_velox/tpc_workload.ipynb delete mode 100644 tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala delete mode 100644 tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh diff --git a/tools/workload/benchmark_velox/README.md b/tools/workload/benchmark_velox/README.md new file mode 100644 index 000000000000..5f080077f606 --- /dev/null +++ b/tools/workload/benchmark_velox/README.md @@ -0,0 +1,38 @@ +# Setup, Build and Benchmark Spark/Gluten with Jupyter Notebook + +This guide provides notebooks and scripts for conducting performance testing in Gluten. The standard approach involves setting up the test environment on a bare-metal machine or cloud instance and running performance tests with TPC-H/TPC-DS workloads. These scripts enable users to reproduce our performance results in their own environment. + +## Environment Setup + +The recommended OS is ubuntu22.04 with kernel 5.15. To prepare the environment, run [initialize.ipynb](./initialize.ipynb), which will: + +- Install system dependencies and set up jupyter notebook +- Configure Hadoop and Spark +- Configure kernel parameters +- Build Gluten using Docker +- Generate TPC-H/TPC-DS tables + +## Running TPC-H/TPC-DS Benchmarks + +To run TPC-H/TPC-DS benchmarks, use [tpc_workload.ipynb](./tpc_workload.ipynb). You can create a copy of the notebook and modify the parameters defined in this notebook to run different workloads. However, creating and modifying a copy each time you change workloads can be inconvenient. Instead, it's recommended to use Papermill to pass parameters via the command line for greater flexibility. + +The required parameters are specified in [params.yaml.template](./params.yaml.template). To use it, create your own YAML file by copying and modifying the template. The command to run the notebook is: + +```bash +papermill tpc_workload.ipynb --inject-output-path -f params.yaml gluten_tpch.ipynb +``` +After execution, the output notebook will be saved as `gluten_tpch.ipynb`. + +If you want to use different parameters, you can specify them via the `-p` option. It will overwrite the previously defined parameters in `params.yaml`. e.g. To switch to the TPC-DS workload, run: + +```bash +papermill tpc_workload.ipynb --inject-output-path -f params.yaml -p workoad tpcds gluten_tpcds.ipynb +``` + +Please refer to the Papermill documentation for additional usage details. + +We also provide a script [run_tpc_workload.sh](./run_tpc_workload.sh). This script wraps the Papermill command, automatically renaming the output notebook with a timestamp and application ID to prevent overwriting existing output files. + +## Analyzing Performance Results + +You can check the **Show Performance** section in the output notebook after execution. It shows the cpu% per query, and draws charts for the cpu%, memory throughput, disk throughput/util%, network throughput and pagefaults. diff --git a/tools/workload/benchmark_velox/init_disks.py b/tools/workload/benchmark_velox/init_disks.py new file mode 100644 index 000000000000..8e47e16e3078 --- /dev/null +++ b/tools/workload/benchmark_velox/init_disks.py @@ -0,0 +1,103 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# To set up the virtual environment required to run this script, +# refer to the `Format and mount disks` subsection under `System Setup` in initialize.ipynb. +import sys +import subprocess +import questionary +import json + +def yes_or_no(question): + while True: + user_input = input(question + '(yes/no/quit): ') + if user_input.lower() == 'yes': + return True + elif user_input.lower() == 'no': + return False + elif user_input.lower() == 'quit': + sys.exit(1) + else: + continue + +def filter_empty_str(l): + return [x for x in l if x] + +def run_and_log(cmd): + # Print command in yellow + print('\033[93m' + '>>> Running command: ' + repr(cmd) + '\033[0m') + result = subprocess.run(cmd, check=True, shell=True, capture_output=True, text=True) + # Print stdout in green + print('\033[92m' + '==========stdout==========' + '\033[0m') + print(result.stdout) + # Print stderr in red + print('\033[91m' + '==========stderr==========' + '\033[0m') + print(result.stderr) + +def init_disks(): + all_disks = filter_empty_str(subprocess.run("lsblk -I 7,8,259 -npd --output NAME".split(' '), capture_output=True, text=True).stdout.split('\n')) + if not all_disks: + print("No disks found on system. Exit.") + sys.exit(0) + + answer = False + disks = [] + while not answer: + disks = questionary.checkbox('Select disks to initialize:', choices=all_disks).ask() + answer = yes_or_no('Confirm selected:\n' + '\n'.join(disks) + '\n') + + if not disks: + print('No disks are selected.') + return + + for d in disks: + print('Initializing {} ...'.format(d)) + run_and_log('wipefs -a {}'.format(d)) + run_and_log('echo "g\nw\n" | fdisk {}'.format(d)) + run_and_log('echo "n\n\n\n\nw\n" | fdisk {}'.format(d)) + run_and_log('mkfs.ext4 {}p1'.format(d)) + +def mount_partitions(): + subprocess.run('lsblk -pf --json > lsblk.json', shell=True) + partitions = [] + with open('lsblk.json', 'r') as f: + data = json.load(f) + for d in data['blockdevices']: + if 'children' in d: + for c in d['children']: + if c['fstype'] == 'ext4': + partitions.append(c['name']) + answer = False + while not answer: + partitions = questionary.checkbox('Select partitions to create mount points:', choices=partitions).ask() + answer = yes_or_no('Confirm selected:\n' + '\n'.join(partitions) + '\n') + + for i, p in enumerate(partitions): + d = 'data{}'.format(i) + run_and_log('e2label {} ""'.format(p)) + run_and_log('e2label {} {}'.format(p, d)) + run_and_log('mkdir -p /{}'.format(d)) + run_and_log('mount -L {} /{}'.format(d, d)) + +def choose(): + choice = questionary.select('Select operation:', choices=['Format disks', 'Mount partitions']).ask() + print(choice) + if choice == 'Format disks': + init_disks() + elif choice == 'Mount partitions': + mount_partitions() + +if __name__ == '__main__': + choose() diff --git a/tools/workload/benchmark_velox/initialize.ipynb b/tools/workload/benchmark_velox/initialize.ipynb new file mode 100644 index 000000000000..cbbc27686951 --- /dev/null +++ b/tools/workload/benchmark_velox/initialize.ipynb @@ -0,0 +1,2918 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# System Setup" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**1. Install system dependencies and python packages. Prepare the environment.**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "First, install all dependencies and python packages as `root`. Run commands and make sure the installations are successful.\n", + "\n", + "```bash\n", + "apt update\n", + "\n", + "apt install -y sudo locales wget tar tzdata git ccache cmake ninja-build build-essential llvm-11-dev clang-11 libiberty-dev libdwarf-dev libre2-dev libz-dev libssl-dev libboost-all-dev libcurl4-openssl-dev openjdk-8-jdk maven vim pip sysstat gcc-9 libjemalloc-dev nvme-cli curl zip unzip bison flex\n", + "\n", + "python3 -m pip install notebook==6.5.2\n", + "python3 -m pip install jupyter_server==1.23.4\n", + "python3 -m pip install jupyter_highlight_selected_word\n", + "python3 -m pip install jupyter_contrib_nbextensions\n", + "python3 -m pip install virtualenv==20.21.1\n", + "python3 -m pip uninstall -y ipython\n", + "python3 -m pip install ipython==8.21.0\n", + "python3 -m pip uninstall -y traitlets\n", + "python3 -m pip install traitlets==5.9.0\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Check that there isn't an entry for your hostname mapped to 127.0.0.1 or 127.0.1.1 in /etc/hosts (Ubuntu is notorious for this). If there is, delete it.\n", + "Then add `` and `` for master and worker nodes.\n", + "\n", + "Example /etc/hosts:\n", + " \n", + "```\n", + "127.0.0.1 localhost\n", + "\n", + "# The following lines are desirable for IPv6 capable hosts\n", + "::1 ip6-localhost ip6-loopback\n", + "fe00::0 ip6-localnet\n", + "ff00::0 ip6-mcastprefix\n", + "ff02::1 ip6-allnodes\n", + "ff02::2 ip6-allrouters\n", + "\n", + "10.0.0.117 sr217\n", + "10.0.0.113 sr213\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**2. Format and mount disks**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Create a python virtual environment to finish the system setup process:\n", + "\n", + "```bash\n", + "virtualenv -p python3 -v venv\n", + "source venv/bin/activate\n", + "```\n", + "\n", + "And install packages under `venv`:\n", + "```bash\n", + "(venv) python3 -m pip install questionary\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Run script [init_disks.py](./init_disks.py) to format and mount disks. **Be careful when choosing the disks to format.** If you see errors like `device or resource busy`, perhaps the partition has been mounted, you should unmount it first. If you still see this error, reboot the system and try again." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Exit `venv`:\n", + "```bash\n", + "(venv) deactivate\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**3. Create user `sparkuser`**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Create user `sparkuser` without password and with sudo priviledge. It's recommended to use one of the disks as the home directory instead of the system drive.\n", + "\n", + "```bash\n", + "mkdir -p /data0/home/sparkuser\n", + "ln -s /data0/home/sparkuser /home/sparkuser\n", + "cp -r /etc/skel/. /home/sparkuser/\n", + "adduser --home /home/sparkuser --disabled-password --gecos \"\" sparkuser\n", + "\n", + "chown -R sparkuser:sparkuser /data*\n", + "\n", + "echo 'sparkuser ALL=(ALL:ALL) NOPASSWD:ALL' | EDITOR='tee -a' visudo\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Generate ssh keys for `sparkuser`\n", + "\n", + "```bashrc\n", + "su - sparkuser\n", + "```\n", + "\n", + "```bashrc\n", + "rm -rf ~/.ssh\n", + "ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa <</dev/null 2>&1\n", + "cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys\n", + "\n", + "exit\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Generate ssh keys for `root`, and enable no password ssh from `sparkuser`\n", + "\n", + "```bash\n", + "rm -rf /root/.ssh\n", + "ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa <</dev/null 2>&1\n", + "cat /root/.ssh/id_rsa.pub >> /root/.ssh/authorized_keys\n", + "cat /home/sparkuser/.ssh/id_rsa.pub >> /root/.ssh/authorized_keys\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Login to `sparkuser` and run the first-time ssh to the `root`\n", + "\n", + "```bash\n", + "su - sparkuser\n", + "```\n", + "\n", + "```bash\n", + "ssh -o StrictHostKeyChecking=no root@localhost ls\n", + "ssh -o StrictHostKeyChecking=no root@127.0.0.1 ls\n", + "ssh -o StrictHostKeyChecking=no root@`hostname` ls\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Run below command to comment out lines starting from `If not running interactively, don't do anything` in ~/.bashrc\n", + "\n", + "```bash\n", + "sed -i '5,9 s/^/# /' ~/.bashrc\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**4. Configure jupyter notebook**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "As `sparkuser`, install python packages\n", + "\n", + "```bash\n", + "cd /home/sparkuser/.local/lib/ && rm -rf python*\n", + "\n", + "python3 -m pip install --upgrade jsonschema\n", + "python3 -m pip install jsonschema[format]\n", + "python3 -m pip install sqlalchemy==1.4.46\n", + "python3 -m pip install papermill Black\n", + "python3 -m pip install NotebookScripter\n", + "python3 -m pip install findspark spylon-kernel matplotlib pandasql pyhdfs\n", + "python3 -m pip install ipywidgets jupyter_nbextensions_configurator ipyparallel\n", + "```\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Configure jupyter notebook. Setup password when it prompts\n", + "\n", + "```bash\n", + "jupyter notebook --generate-config\n", + "\n", + "jupyter notebook password\n", + "\n", + "mkdir -p ~/.jupyter/custom/\n", + "\n", + "echo '.container { width:100% !important; }' >> ~/.jupyter/custom/custom.css\n", + "\n", + "echo 'div.output_stderr { background: #ffdd; display: none; }' >> ~/.jupyter/custom/custom.css\n", + "\n", + "jupyter nbextension install --py jupyter_highlight_selected_word --user\n", + "\n", + "jupyter nbextension enable highlight_selected_word/main\n", + "\n", + "jupyter nbextension install --py widgetsnbextension --user\n", + "\n", + "jupyter contrib nbextension install --user\n", + "\n", + "jupyter nbextension enable codefolding/main\n", + "\n", + "jupyter nbextension enable code_prettify/code_prettify\n", + "\n", + "jupyter nbextension enable codefolding/edit\n", + "\n", + "jupyter nbextension enable code_font_size/code_font_size\n", + "\n", + "jupyter nbextension enable collapsible_headings/main\n", + "\n", + "jupyter nbextension enable highlight_selected_word/main\n", + "\n", + "jupyter nbextension enable ipyparallel/main\n", + "\n", + "jupyter nbextension enable move_selected_cells/main\n", + "\n", + "jupyter nbextension enable nbTranslate/main\n", + "\n", + "jupyter nbextension enable scratchpad/main\n", + "\n", + "jupyter nbextension enable tree-filter/index\n", + "\n", + "jupyter nbextension enable comment-uncomment/main\n", + "\n", + "jupyter nbextension enable export_embedded/main\n", + "\n", + "jupyter nbextension enable hide_header/main\n", + "\n", + "jupyter nbextension enable highlighter/highlighter\n", + "\n", + "jupyter nbextension enable scroll_down/main\n", + "\n", + "jupyter nbextension enable snippets/main\n", + "\n", + "jupyter nbextension enable toc2/main\n", + "\n", + "jupyter nbextension enable varInspector/main\n", + "\n", + "jupyter nbextension enable codefolding/edit\n", + "\n", + "jupyter nbextension enable contrib_nbextensions_help_item/main\n", + "\n", + "jupyter nbextension enable freeze/main\n", + "\n", + "jupyter nbextension enable hide_input/main\n", + "\n", + "jupyter nbextension enable jupyter-js-widgets/extension\n", + "\n", + "jupyter nbextension enable snippets_menu/main\n", + "\n", + "jupyter nbextension enable table_beautifier/main\n", + "\n", + "jupyter nbextension enable hide_input_all/main\n", + "\n", + "jupyter nbextension enable spellchecker/main\n", + "\n", + "jupyter nbextension enable toggle_all_line_numbers/main\n", + "\n", + "jupyter nbextensions_configurator enable --user\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Clone Gluten\n", + "\n", + "```bash\n", + "cd ~\n", + "git clone https://github.com/apache/incubator-gluten.git gluten\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Start jupyter notebook\n", + "\n", + "```bash\n", + "mkdir -p ~/ipython\n", + "cd ~/ipython\n", + "\n", + "nohup jupyter notebook --ip=0.0.0.0 --port=8888 &\n", + "\n", + "cp ~/gluten/tools/workload/benchmark_velox ~/ipython/\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Initialize\n", + " Run this section after notebook restart! " + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify datadir. The directories are used for spark.local.dirs and hadoop namenode/datanode." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "datadir=[f'/data{i}' for i in range(0, 8)]\n", + "datadir" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify clients(workers). Leave it empty if the cluster is setup on the local machine." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "clients=''''''.split()\n", + "print(clients)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify JAVA_HOME" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "java_home = '/usr/lib/jvm/java-8-openjdk-amd64'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "import socket\n", + "import platform\n", + "\n", + "user=os.getenv('USER')\n", + "print(f\"user: {user}\")\n", + "print()\n", + "\n", + "masterip=socket.gethostbyname(socket.gethostname())\n", + "hostname=socket.gethostname() \n", + "print(f\"masterip: {masterip} hostname: {hostname}\")\n", + "print()\n", + "\n", + "hclients=clients.copy()\n", + "hclients.append(hostname)\n", + "print(f\"master and workers: {hclients}\")\n", + "print()\n", + "\n", + "\n", + "if clients:\n", + " cmd = f\"ssh {clients[0]} \" + \"\\\"lscpu | grep '^CPU(s)'\\\"\" + \" | awk '{print $2}'\"\n", + " client_cpu = !{cmd}\n", + " cpu_num = client_cpu[0]\n", + "\n", + " cmd = f\"ssh {clients[0]} \" + \"\\\"cat /proc/meminfo | grep MemTotal\\\"\" + \" | awk '{print $2}'\"\n", + " totalmemory = !{cmd}\n", + " totalmemory = int(totalmemory[0])\n", + "else:\n", + " cpu_num = os.cpu_count()\n", + " totalmemory = !cat /proc/meminfo | grep MemTotal | awk '{print $2}'\n", + " totalmemory = int(totalmemory[0])\n", + " \n", + "print(f\"cpu_num: {cpu_num}\")\n", + "print()\n", + "\n", + "print(\"total memory: \", totalmemory, \"KB\")\n", + "print()\n", + "\n", + "mem_mib = int(totalmemory/1024)-1024\n", + "print(f\"mem_mib: {mem_mib}\")\n", + "print()\n", + "\n", + "is_arm = platform.machine() == 'aarch64'\n", + "print(\"is_arm: \",is_arm)\n", + "print()\n", + "\n", + "sparklocals=\",\".join([f'{l}/{user}/yarn/local' for l in datadir])\n", + "print(f\"SPARK_LOCAL_DIR={sparklocals}\")\n", + "print()\n", + "\n", + "%cd ~" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Set up clients\n", + " SKIP for single node " + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Install dependencies" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Manually configure ssh login without password to all clients\n", + "\n", + "```bash\n", + "ssh-copy-id -o StrictHostKeyChecking=no root@\n", + "```" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} apt update > /dev/null 2>&1\n", + " !ssh root@{l} apt install -y sudo locales wget tar tzdata git ccache cmake ninja-build build-essential llvm-11-dev clang-11 libiberty-dev libdwarf-dev libre2-dev libz-dev libssl-dev libboost-all-dev libcurl4-openssl-dev openjdk-8-jdk maven vim pip sysstat gcc-9 libjemalloc-dev nvme-cli curl zip unzip bison flex > /dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Create user" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh -o StrictHostKeyChecking=no root@{l} ls" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} adduser --disabled-password --gecos '\"\"' {user}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} cp -r .ssh /home/{user}/\n", + " !ssh root@{l} chown -R {user}:{user} /home/{user}/.ssh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} \"echo -e 'sparkuser ALL=(ALL:ALL) NOPASSWD:ALL' | EDITOR='tee -a' visudo\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Run below command to comment out lines starting from If not running interactively, don't do anything in ~/.bashrc" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh {l} sed -i \"'5,9 s/^/# /'\" ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Use /etc/hosts on master node" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp /etc/hosts root@{l}:/etc/hosts" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Setup disks" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} apt update > /dev/null 2>&1\n", + " !ssh root@{l} apt install -y pip > /dev/null 2>&1\n", + " !ssh root@{l} python3 -m pip install virtualenv" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Manually run **2. Format and mount disks** section under [System Setup](#System-Setup)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Configure Spark, Hadoop" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Download packages" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!wget https://archive.apache.org/dist/spark/spark-3.3.1/spark-3.3.1-bin-hadoop3.tgz > /dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!wget https://archive.apache.org/dist/hadoop/common/hadoop-3.2.4/hadoop-3.2.4.tar.gz > /dev/null 2>&1\n", + "# backup url: !wget https://dlcdn.apache.org/hadoop/common/hadoop-3.2.4/hadoop-3.2.4.tar.gz > /dev/null 2>&1\n", + "if is_arm:\n", + " # download both versions\n", + " !wget https://dlcdn.apache.org/hadoop/common/hadoop-3.3.5/hadoop-3.3.5-aarch64.tar.gz > /dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Create directories" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"chown -R {user}:{user} \" + l for l in datadir])\n", + "for l in hclients:\n", + " !ssh root@{l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"rm -rf {l}/tmp; mkdir -p {l}/tmp\" for l in datadir])\n", + "for l in hclients:\n", + " !ssh {l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"mkdir -p {l}/{user}/hdfs/data; mkdir -p {l}/{user}/yarn/local\" for l in datadir])\n", + "for l in hclients:\n", + " !ssh {l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!mkdir -p {datadir[0]}/{user}/hdfs/name\n", + "!mkdir -p {datadir[0]}/{user}/hdfs/namesecondary" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp hadoop-3.2.4.tar.gz {l}:~/\n", + " !scp spark-3.3.1-bin-hadoop3.tgz {l}:~/\n", + " !ssh {l} \"mv -f hadoop hadoop.bak; mv -f spark spark.bak\"\n", + " !ssh {l} \"tar zxvf hadoop-3.2.4.tar.gz > /dev/null 2>&1\"\n", + " !ssh {l} \"tar -zxvf spark-3.3.1-bin-hadoop3.tgz > /dev/null 2>&1\"\n", + " !ssh root@{l} \"apt install -y openjdk-8-jdk > /dev/null 2>&1\"\n", + " !ssh {l} \"ln -s hadoop-3.2.4 hadoop; ln -s spark-3.3.1-bin-hadoop3 spark\"\n", + " if is_arm:\n", + " !ssh {l} \"tar zxvf hadoop-3.3.5-aarch64.tar.gz > /dev/null 2>&1\"\n", + " !ssh {l} \"cd hadoop && mv lib lib.bak && cp -rf ~/hadoop-3.3.5/lib ~/hadoop\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure bashrc" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "\n", + "cfg=f'''export HADOOP_HOME=~/hadoop\n", + "export PATH=$PATH:$HADOOP_HOME/bin:$HADOOP_HOME/sbin\n", + "\n", + "export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop\n", + "export YARN_CONF_DIR=$HADOOP_HOME/etc/hadoop\n", + "\n", + "export SPARK_HOME=~/spark\n", + "export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/lib/py4j-0.10.9.5-src.zip:$PYTHONPATH\n", + "export PATH=$SPARK_HOME/bin:$PATH\n", + "\n", + "'''" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "if is_arm:\n", + " cfg += 'export CPU_TARGET=\"aarch64\"\\nexport JAVA_HOME=/usr/lib/jvm/java-8-openjdk-arm64\\nexport PATH=$JAVA_HOME/bin:$PATH\\n'\n", + "else:\n", + " cfg += f'export JAVA_HOME={java_home}\\nexport PATH=$JAVA_HOME/bin:$PATH\\n'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "with open(\"tmpcfg\",'w') as f:\n", + " f.writelines(cfg)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp tmpcfg {l}:~/tmpcfg.in\n", + " !ssh {l} \"cat ~/tmpcfg.in >> ~/.bashrc\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} tail -n10 ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure Hadoop" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh root@{l} \"apt install -y libiberty-dev libxml2-dev libkrb5-dev libgsasl7-dev libuuid1 uuid-dev > /dev/null 2>&1\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### setup short-circuit " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh root@{l} \"mkdir -p /var/lib/hadoop-hdfs/\"\n", + " !ssh root@{l} 'chown {user}:{user} /var/lib/hadoop-hdfs/'" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### enable security.authorization" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "coresite='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " fs.default.name\n", + " hdfs://{:s}:8020\n", + " true\n", + " \n", + " \n", + " hadoop.security.authentication\n", + " simple\n", + " \n", + " \n", + " hadoop.security.authorization\n", + " true\n", + " \n", + "\n", + "'''.format(hostname)\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/core-site.xml','w') as f:\n", + " f.writelines(coresite)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/core-site.xml {l}:~/hadoop/etc/hadoop/core-site.xml >/dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### set IP check, note the command \", \".join" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "hadooppolicy='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " security.service.authorization.default.hosts\n", + " {:s}\n", + " \n", + " \n", + " security.service.authorization.default.acl\n", + " {:s} {:s}\n", + " \n", + " \n", + " \n", + " \n", + " security.client.protocol.acl\n", + " *\n", + " ACL for ClientProtocol, which is used by user code\n", + " via the DistributedFileSystem.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.client.datanode.protocol.acl\n", + " *\n", + " ACL for ClientDatanodeProtocol, the client-to-datanode protocol\n", + " for block recovery.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.datanode.protocol.acl\n", + " *\n", + " ACL for DatanodeProtocol, which is used by datanodes to\n", + " communicate with the namenode.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.inter.datanode.protocol.acl\n", + " *\n", + " ACL for InterDatanodeProtocol, the inter-datanode protocol\n", + " for updating generation timestamp.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.namenode.protocol.acl\n", + " *\n", + " ACL for NamenodeProtocol, the protocol used by the secondary\n", + " namenode to communicate with the namenode.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.admin.operations.protocol.acl\n", + " *\n", + " ACL for AdminOperationsProtocol. Used for admin commands.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.refresh.user.mappings.protocol.acl\n", + " *\n", + " ACL for RefreshUserMappingsProtocol. Used to refresh\n", + " users mappings. The ACL is a comma-separated list of user and\n", + " group names. The user and group list is separated by a blank. For\n", + " e.g. \"alice,bob users,wheel\". A special value of \"*\" means all\n", + " users are allowed.\n", + " \n", + "\n", + " \n", + " security.refresh.policy.protocol.acl\n", + " *\n", + " ACL for RefreshAuthorizationPolicyProtocol, used by the\n", + " dfsadmin and mradmin commands to refresh the security policy in-effect.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.ha.service.protocol.acl\n", + " *\n", + " ACL for HAService protocol used by HAAdmin to manage the\n", + " active and stand-by states of namenode.\n", + " \n", + "\n", + " \n", + " security.zkfc.protocol.acl\n", + " *\n", + " ACL for access to the ZK Failover Controller\n", + " \n", + " \n", + "\n", + " \n", + " security.qjournal.service.protocol.acl\n", + " *\n", + " ACL for QJournalProtocol, used by the NN to communicate with\n", + " JNs when using the QuorumJournalManager for edit logs.\n", + " \n", + "\n", + " \n", + " security.mrhs.client.protocol.acl\n", + " *\n", + " ACL for HSClientProtocol, used by job clients to\n", + " communciate with the MR History Server job status etc.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + "\n", + " \n", + " security.resourcetracker.protocol.acl\n", + " *\n", + " ACL for ResourceTrackerProtocol, used by the\n", + " ResourceManager and NodeManager to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.resourcemanager-administration.protocol.acl\n", + " *\n", + " ACL for ResourceManagerAdministrationProtocol, for admin commands.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationclient.protocol.acl\n", + " *\n", + " ACL for ApplicationClientProtocol, used by the ResourceManager\n", + " and applications submission clients to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationmaster.protocol.acl\n", + " *\n", + " ACL for ApplicationMasterProtocol, used by the ResourceManager\n", + " and ApplicationMasters to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.containermanagement.protocol.acl\n", + " *\n", + " ACL for ContainerManagementProtocol protocol, used by the NodeManager\n", + " and ApplicationMasters to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.resourcelocalizer.protocol.acl\n", + " *\n", + " ACL for ResourceLocalizer protocol, used by the NodeManager\n", + " and ResourceLocalizer to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.job.task.protocol.acl\n", + " *\n", + " ACL for TaskUmbilicalProtocol, used by the map and reduce\n", + " tasks to communicate with the parent tasktracker.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.job.client.protocol.acl\n", + " *\n", + " ACL for MRClientProtocol, used by job clients to\n", + " communciate with the MR ApplicationMaster to query job status etc.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationhistory.protocol.acl\n", + " *\n", + " ACL for ApplicationHistoryProtocol, used by the timeline\n", + " server and the generic history service client to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + "\n", + "'''.format((\",\").join(hclients),user,user)\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/hadoop-policy.xml','w') as f:\n", + " f.writelines(hadooppolicy)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/hadoop-policy.xml {l}:~/hadoop/etc/hadoop/hadoop-policy.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### hdfs config, set replication to 1 to cache all the data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [], + "hidden": true + }, + "outputs": [], + "source": [ + "hdfs_data=\",\".join([f'{l}/{user}/hdfs/data' for l in datadir])\n", + "\n", + "hdfs_site=f'''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " dfs.namenode.secondary.http-address\n", + " {hostname}:50090\n", + " \n", + " \n", + " dfs.namenode.name.dir\n", + " {datadir[0]}/{user}/hdfs/name\n", + " true\n", + " \n", + "\n", + " \n", + " dfs.datanode.data.dir\n", + " {hdfs_data}\n", + " true\n", + " \n", + "\n", + " \n", + " dfs.namenode.checkpoint.dir\n", + " {datadir[0]}/{user}/hdfs/namesecondary\n", + " true\n", + " \n", + " \n", + " dfs.name.handler.count\n", + " 100\n", + " \n", + " \n", + " dfs.blocksize\n", + " 128m\n", + "\n", + " \n", + " dfs.replication\n", + " 1\n", + "\n", + "\n", + "\n", + " dfs.client.read.shortcircuit\n", + " true\n", + "\n", + "\n", + "\n", + " dfs.domain.socket.path\n", + " /var/lib/hadoop-hdfs/dn_socket\n", + "\n", + "\n", + "\n", + "'''\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/hdfs-site.xml','w') as f:\n", + " f.writelines(hdfs_site)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/hdfs-site.xml {l}:~/hadoop/etc/hadoop/hdfs-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### mapreduce config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "mapreduce='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " mapreduce.framework.name\n", + " yarn\n", + " \n", + "\n", + " \n", + " mapreduce.job.maps\n", + " 288\n", + " \n", + " \n", + " mapreduce.job.reduces\n", + " 64\n", + " \n", + "\n", + " \n", + " mapreduce.map.java.opts\n", + " -Xmx5120M -DpreferIPv4Stack=true\n", + " \n", + " \n", + " mapreduce.map.memory.mb\n", + " 6144\n", + " \n", + "\n", + " \n", + " mapreduce.reduce.java.opts\n", + " -Xmx5120M -DpreferIPv4Stack=true\n", + " \n", + " \n", + " mapreduce.reduce.memory.mb\n", + " 6144\n", + " \n", + " \n", + " yarn.app.mapreduce.am.staging-dir\n", + " /user\n", + " \n", + " \n", + " mapreduce.task.io.sort.mb\n", + " 2000\n", + " \n", + " \n", + " mapreduce.task.timeout\n", + " 3600000\n", + " \n", + "\n", + "\n", + " mapreduce.jobhistory.address\n", + " {:s}:10020\n", + "\n", + "\n", + "\n", + "'''.format(hostname)\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/mapred-site.xml','w') as f:\n", + " f.writelines(mapreduce)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/mapred-site.xml {l}:~/hadoop/etc/hadoop/mapred-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### yarn config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [], + "hidden": true + }, + "outputs": [], + "source": [ + "yarn_site=f'''\n", + "\n", + "\n", + " \n", + " yarn.resourcemanager.hostname\n", + " {hostname}\n", + " \n", + " \n", + " yarn.resourcemanager.address\n", + " {hostname}:8032\n", + " \n", + " \n", + " yarn.resourcemanager.webapp.address\n", + " {hostname}:8088\n", + " \n", + " \n", + " yarn.nodemanager.resource.memory-mb\n", + " {mem_mib}\n", + " \n", + " \n", + " yarn.nodemanager.resource.cpu-vcores\n", + " {cpu_num}\n", + " \n", + " \n", + " yarn.nodemanager.pmem-check-enabled\n", + " false\n", + " \n", + "\n", + " \n", + " yarn.nodemanager.vmem-check-enabled\n", + " false\n", + " \n", + " \n", + " yarn.nodemanager.vmem-pmem-ratio\n", + " 4.1\n", + " \n", + " \n", + " yarn.nodemanager.aux-services\n", + " mapreduce_shuffle,spark_shuffle\n", + " \n", + "\n", + " \n", + " yarn.scheduler.minimum-allocation-mb\n", + " 1024\n", + " \n", + " \n", + " yarn.scheduler.maximum-allocation-mb\n", + " {mem_mib}\n", + " \n", + " \n", + " yarn.scheduler.minimum-allocation-vcores\n", + " 1\n", + " \n", + " \n", + " yarn.scheduler.maximum-allocation-vcores\n", + " {cpu_num}\n", + " \n", + "\n", + " \n", + " yarn.log-aggregation-enable\n", + " false\n", + " \n", + " \n", + " yarn.nodemanager.log.retain-seconds\n", + " 36000\n", + " \n", + " \n", + " yarn.nodemanager.delete.debug-delay-sec\n", + " 3600\n", + " \n", + " \n", + " yarn.log.server.url\n", + " http://{hostname}:19888/jobhistory/logs/\n", + " \n", + "\n", + " \n", + " yarn.nodemanager.log-dirs\n", + " /home/{user}/hadoop/logs/userlogs\n", + " \n", + " \n", + " yarn.nodemanager.local-dirs\n", + " {sparklocals}\n", + " \n", + " \n", + " \n", + " yarn.nodemanager.aux-services.spark_shuffle.class\n", + " org.apache.spark.network.yarn.YarnShuffleService\n", + " \n", + "\n", + "'''\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/yarn-site.xml','w') as f:\n", + " f.writelines(yarn_site)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/yarn-site.xml {l}:~/hadoop/etc/hadoop/yarn-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### hadoop-env" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "#config java home\n", + "if is_arm:\n", + " !echo \"export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-arm64\" >> ~/hadoop/etc/hadoop/hadoop-env.sh\n", + "else:\n", + " !echo \"export JAVA_HOME={java_home}\" >> ~/hadoop/etc/hadoop/hadoop-env.sh\n", + "\n", + "for l in clients:\n", + " !scp hadoop/etc/hadoop/hadoop-env.sh {l}:~/hadoop/etc/hadoop/ >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### workers config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "if clients:\n", + " with open(f'/home/{user}/hadoop/etc/hadoop/workers','w') as f:\n", + " f.writelines(\"\\n\".join(clients))\n", + " for l in clients:\n", + " !scp hadoop/etc/hadoop/workers {l}:~/hadoop/etc/hadoop/ >/dev/null 2>&1\n", + "else:\n", + " !echo {hostname} > ~/hadoop/etc/hadoop/workers" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### Copy jar from Spark for external shuffle service" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp spark/yarn/spark-3.3.1-yarn-shuffle.jar {l}:~/hadoop/share/hadoop/common/lib/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure Spark" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "eventlog_dir=f'hdfs://{hostname}:8020/tmp/sparkEventLog'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "sparkconf=f'''\n", + "spark.eventLog.enabled true\n", + "spark.eventLog.dir {eventlog_dir}\n", + "spark.history.fs.logDirectory {eventlog_dir}\n", + "'''\n", + "\n", + "with open(f'/home/{user}/spark/conf/spark-defaults.conf','w+') as f:\n", + " f.writelines(sparkconf)\n", + " \n", + "for l in clients:\n", + " !scp ~/spark/conf/spark-defaults.conf {l}:~/spark/conf/spark-defaults.conf >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "sparkenv = f'export SPARK_LOCAL_DIRS={sparklocals}\\n'\n", + "with open(f'/home/{user}/.bashrc', 'a+') as f:\n", + " f.writelines(sparkenv)\n", + "for l in clients:\n", + " !scp ~/.bashrc {l}:~/.bashrc >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} tail -n10 ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Configure startup" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "startup=f'''#!/bin/bash\n", + "echo -1 > /proc/sys/kernel/perf_event_paranoid\n", + "echo 0 > /proc/sys/kernel/kptr_restrict\n", + "echo madvise >/sys/kernel/mm/transparent_hugepage/enabled\n", + "echo 1 > /proc/sys/kernel/numa_balancing\n", + "end=$(($(nproc) - 1))\n", + "for i in $(seq 0 $end); do echo performance > /sys/devices/system/cpu/cpu$i/cpufreq/scaling_governor; done\n", + "for file in $(find /sys/devices/system/cpu/cpu*/power/energy_perf_bias); do echo \"0\" > $file; done\n", + "'''\n", + "\n", + "with open('/tmp/tmpstartup', 'w') as f:\n", + " f.writelines(startup)\n", + "\n", + "startup_service=f'''[Unit]\n", + "Description=Configure Transparent Hugepage, Auto NUMA Balancing, CPU Freq Scaling Governor\n", + "\n", + "[Service]\n", + "ExecStart=/usr/local/bin/mystartup.sh\n", + "\n", + "[Install]\n", + "WantedBy=multi-user.target\n", + "'''\n", + "\n", + "with open('/tmp/tmpstartup_service', 'w') as f:\n", + " f.writelines(startup_service)\n", + " \n", + "for l in hclients:\n", + " !scp /tmp/tmpstartup $l:/tmp/tmpstartup\n", + " !scp /tmp/tmpstartup_service $l:/tmp/tmpstartup_service\n", + " !ssh root@$l \"cat /tmp/tmpstartup > /usr/local/bin/mystartup.sh\"\n", + " !ssh root@$l \"chmod +x /usr/local/bin/mystartup.sh\"\n", + " !ssh root@$l \"cat /tmp/tmpstartup_service > /etc/systemd/system/mystartup.service\"\n", + " !ssh $l \"sudo systemctl enable mystartup.service\"\n", + " !ssh $l \"sudo systemctl start mystartup.service\"\n", + " !ssh $l \"sudo systemctl status mystartup.service\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "deletable": false, + "editable": false, + "heading_collapsed": true, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "source": [ + "## Inspect CPU Freq & HT" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "if is_arm:\n", + " t = r'''\n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include //used for parsing the command line arguments\n", + " #include //used for opening the memory device file\n", + " #include //used for rounding functions\n", + " #include \n", + " #include \n", + " #include \n", + "\n", + " static inline uint64_t GetTickCount()\n", + " {//Return ns counts\n", + " struct timeval tp;\n", + " gettimeofday(&tp,NULL);\n", + " return tp.tv_sec*1000+tp.tv_usec/1000;\n", + " }\n", + "\n", + " uint64_t CNT=CNT_DEF;\n", + "\n", + " int main()\n", + " {\n", + "\n", + " uint64_t start, end;\n", + " start=end=GetTickCount();\n", + "\n", + " asm volatile (\n", + " \"1:\\n\"\n", + " \"SUBS %0,%0,#1\\n\"\n", + " \"bne 1b\\n\"\n", + " ::\"r\"(CNT)\n", + " );\n", + "\n", + " end=GetTickCount();\n", + "\n", + " printf(\" total time = %lu, freq = %lu \\n\", end-start, CNT/(end-start)/1000);\n", + "\n", + " return 0;\n", + " }\n", + " '''\n", + "else:\n", + " t=r'''\n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include //used for parsing the command line arguments\n", + " #include //used for opening the memory device file\n", + " #include //used for rounding functions\n", + " #include \n", + " #include \n", + " #include \n", + "\n", + " static inline uint64_t GetTickCount()\n", + " {//Return ns counts\n", + " struct timeval tp;\n", + " gettimeofday(&tp,NULL);\n", + " return tp.tv_sec*1000+tp.tv_usec/1000;\n", + " }\n", + "\n", + " uint64_t CNT=CNT_DEF;\n", + "\n", + " int main()\n", + " {\n", + "\n", + " uint64_t start, end;\n", + " start=end=GetTickCount();\n", + "\n", + " asm volatile (\n", + " \"1:\\n\"\n", + " \"dec %0\\n\"\n", + " \"jnz 1b\\n\"\n", + " ::\"r\"(CNT)\n", + " );\n", + "\n", + " end=GetTickCount();\n", + "\n", + " printf(\" total time = %lu, freq = %lu \\n\", end-start, CNT/(end-start)/1000);\n", + "\n", + " return 0;\n", + " }\n", + " '''" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "%cd ~\n", + "with open(\"t.c\", 'w') as f:\n", + " f.writelines(t)\n", + "!gcc -O3 -DCNT_DEF=10000000000LL -o t t.c; gcc -O3 -DCNT_DEF=1000000000000LL -o t.delay t.c;\n", + "!for j in `seq 1 $(nproc)`; do echo -n $j; (for i in `seq 1 $j`; do taskset -c $i ./t.delay & done); sleep 1; ./t; killall t.delay; sleep 2; done" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Shutdown Jupyter; source ~/.bashrc; reboot Jupyter; run section [Initialize](#Initialize)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Build gluten" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Install docker" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Instructions from https://docs.docker.com/engine/install/ubuntu/\n", + "\n", + "# Add Docker's official GPG key:\n", + "!sudo -E apt-get update\n", + "!sudo -E apt-get install ca-certificates curl\n", + "!sudo -E install -m 0755 -d /etc/apt/keyrings\n", + "!sudo -E curl -fsSL https://download.docker.com/linux/ubuntu/gpg -o /etc/apt/keyrings/docker.asc\n", + "!sudo chmod a+r /etc/apt/keyrings/docker.asc\n", + "\n", + "# Add the repository to Apt sources:\n", + "!echo \\\n", + " \"deb [arch=$(dpkg --print-architecture) signed-by=/etc/apt/keyrings/docker.asc] https://download.docker.com/linux/ubuntu \\\n", + " $(. /etc/os-release && echo \"$VERSION_CODENAME\") stable\" | \\\n", + " sudo -E tee /etc/apt/sources.list.d/docker.list > /dev/null\n", + "!sudo -E apt-get update" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo -E apt-get install -y docker-ce docker-ce-cli containerd.io docker-buildx-plugin docker-compose-plugin >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "if http_proxy or https_proxy:\n", + " !sudo mkdir -p /etc/systemd/system/docker.service.d\n", + " with open('/tmp/http-proxy.conf', 'w') as f:\n", + " s = '''\n", + "[Service]\n", + "{}\n", + "{}\n", + "'''.format(f'Environment=\"HTTP_PROXY={http_proxy}\"' if http_proxy else '', f'Environment=\"HTTPS_PROXY={https_proxy}\"' if https_proxy else '')\n", + " f.writelines(s)\n", + " !sudo cp /tmp/http-proxy.conf /etc/systemd/system/docker.service.d" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!mkdir -p ~/.m2\n", + "\n", + "def get_proxy(proxy):\n", + " pos0 = proxy.rfind('/')\n", + " pos = proxy.rfind(':')\n", + " host = http_proxy[pos0+1:pos]\n", + " port = http_proxy[pos+1:]\n", + " return host, port\n", + "\n", + "if http_proxy or https_proxy:\n", + " with open(f\"/home/{user}/.m2/settings.xml\",\"w+\") as f:\n", + " f.write('''\n", + "\n", + " ''')\n", + " if http_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " f.write(f'''\n", + " \n", + " http_proxy\n", + " true\n", + " http\n", + " {host}\n", + " {port}\n", + " ''')\n", + " if https_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " f.write(f'''\n", + " \n", + " https_proxy\n", + " true\n", + " https\n", + " {host}\n", + " {port}\n", + " ''')\n", + " f.write('''\n", + " \n", + "\n", + "''')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo systemctl daemon-reload" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo systemctl restart docker.service" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Build gluten" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker pull apache/gluten:vcpkg-centos-7" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "container=!sudo docker run -e http_proxy={http_proxy} -e https_proxy={https_proxy} -itd apache/gluten:vcpkg-centos-7\n", + "containerid = container[0]\n", + "containerid" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt && git clone https://github.com/apache/incubator-gluten.git gluten\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt && source /opt/rh/devtoolset-9/enable && cd gluten && ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --enable_hdfs=ON > build.log\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "if os.path.exists(f'/home/{user}/.m2/settings.xml'):\n", + " !sudo docker exec {containerid} bash -c \"mkdir -p ~/.m2\"\n", + " !sudo docker cp /home/{user}/.m2/settings.xml {containerid}:/root/.m2/settings.xml" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt/gluten && mvn clean package -DskipTests -Pspark-3.3 -Pbackends-velox\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker cp {containerid}:/opt/gluten/package/target/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar ~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp ~/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar {l}:~/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Generate data" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Build spark-sql-perf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!echo \"deb https://repo.scala-sbt.org/scalasbt/debian all main\" | sudo tee /etc/apt/sources.list.d/sbt.list\n", + "!echo \"deb https://repo.scala-sbt.org/scalasbt/debian /\" | sudo tee /etc/apt/sources.list.d/sbt_old.list\n", + "!curl -sL \"https://keyserver.ubuntu.com/pks/lookup?op=get&search=0x2EE0EA64E40A89B84B2DF73499E82A75642AC823\" | sudo apt-key add\n", + "!sudo -E apt-get update > /dev/null 2>&1\n", + "!sudo -E apt-get install sbt > /dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "def get_proxy(proxy):\n", + " pos0 = proxy.rfind('/')\n", + " pos = proxy.rfind(':')\n", + " host = http_proxy[pos0+1:pos]\n", + " port = http_proxy[pos+1:]\n", + " return host, port\n", + "\n", + "sbt_opts=''\n", + "\n", + "if http_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " sbt_opts = f'{sbt_opts} -Dhttp.proxyHost={host} -Dhttp.proxyPort={port}'\n", + "if https_proxy:\n", + " host, port = get_proxy(https_proxy)\n", + " sbt_opts = f'{sbt_opts} -Dhttps.proxyHost={host} -Dhttps.proxyPort={port}'\n", + " \n", + "if sbt_opts:\n", + " %env SBT_OPTS={sbt_opts}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!git clone https://github.com/databricks/spark-sql-perf.git ~/spark-sql-perf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cd ~/spark-sql-perf && sbt package" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cp ~/spark-sql-perf/target/scala-2.12/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar ~/ipython/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Start Hadoop/Spark cluster, Spark history server" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/bin/hadoop namenode -format" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/bin/hadoop datanode -format " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/sbin/start-dfs.sh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!hadoop dfsadmin -safemode leave" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!hadoop fs -mkdir -p /tmp/sparkEventLog" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cd ~/spark && sbin/start-history-server.sh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "\n", + "master=''\n", + "if clients:\n", + " !~/hadoop/sbin/start-yarn.sh\n", + " master='yarn'\n", + "else:\n", + " # If we run on single node, we use standalone mode\n", + " !{os.environ['SPARK_HOME']}/sbin/stop-slave.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/stop-master.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/start-master.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/start-worker.sh spark://{hostname}:7077 -c {cpu_num}\n", + " master=f'spark://{hostname}:7077'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!jps\n", + "for l in clients:\n", + " !ssh {l} jps" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## TPCH" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!rm -rf ~/tpch-dbgen\n", + "!git clone https://github.com/databricks/tpch-dbgen.git ~/tpch-dbgen" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp -r ~/tpch-dbgen {l}:~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} cd ~/tpch-dbgen && git checkout 0469309147b42abac8857fa61b4cf69a6d3128a8 && make clean && make OS=LINUX" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "%cd ~/gluten/tools/workload/tpch/gen_data/parquet_dataset" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Suggest 2x cpu# partitions.\n", + "scaleFactor = 1500\n", + "numPartitions = 2*cpu_num if len(clients)==0 else len(clients)*2*cpu_num\n", + "dataformat = \"parquet\" # data format of data source\n", + "dataSourceCodec = \"snappy\"\n", + "rootDir = f\"/tpch_sf{scaleFactor}_{dataformat}_{dataSourceCodec}\" # root directory of location to create data in." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Verify parameters\n", + "print(f'scaleFactor = {scaleFactor}')\n", + "print(f'numPartitions = {numPartitions}')\n", + "print(f'dataformat = {dataformat}')\n", + "print(f'rootDir = {rootDir}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "scala=f'''import com.databricks.spark.sql.perf.tpch._\n", + "\n", + "\n", + "val scaleFactor = \"{scaleFactor}\" // scaleFactor defines the size of the dataset to generate (in GB).\n", + "val numPartitions = {numPartitions} // how many dsdgen partitions to run - number of input tasks.\n", + "\n", + "val format = \"{dataformat}\" // valid spark format like parquet \"parquet\".\n", + "val rootDir = \"{rootDir}\" // root directory of location to create data in.\n", + "val dbgenDir = \"/home/{user}/tpch-dbgen\" // location of dbgen\n", + "\n", + "val tables = new TPCHTables(spark.sqlContext,\n", + " dbgenDir = dbgenDir,\n", + " scaleFactor = scaleFactor,\n", + " useDoubleForDecimal = false, // true to replace DecimalType with DoubleType\n", + " useStringForDate = false) // true to replace DateType with StringType\n", + "\n", + "\n", + "tables.genData(\n", + " location = rootDir,\n", + " format = format,\n", + " overwrite = true, // overwrite the data that is already there\n", + " partitionTables = false, // do not create the partitioned fact tables\n", + " clusterByPartitionColumns = false, // shuffle to get partitions coalesced into single files.\n", + " filterOutNullPartitionValues = false, // true to filter out the partition with NULL key value\n", + " tableFilter = \"\", // \"\" means generate all tables\n", + " numPartitions = numPartitions) // how many dsdgen partitions to run - number of input tasks.\n", + "'''\n", + "\n", + "with open(\"tpch_datagen_parquet.scala\",\"w\") as f:\n", + " f.writelines(scala)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "executor_cores = 8\n", + "num_executors=cpu_num/executor_cores\n", + "executor_memory = (totalmemory - 10*1024*1024)/num_executors - 1*1024*1024\n", + "\n", + "# Verify parameters\n", + "print(f'--master {master}')\n", + "print(f'--num-executors {int(num_executors)}')\n", + "print(f'--executor-cores {int(executor_cores)}')\n", + "print(f'--executor-memory {int(executor_memory)}k')\n", + "print(f'--conf spark.sql.shuffle.partitions={numPartitions}')\n", + "print(f'--conf spark.sql.parquet.compression.codec={dataSourceCodec}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "tpch_datagen_parquet=f'''\n", + "cat tpch_datagen_parquet.scala | {os.environ['SPARK_HOME']}/bin/spark-shell \\\n", + " --master {master} \\\n", + " --name tpch_gen_parquet \\\n", + " --driver-memory 10g \\\n", + " --num-executors {int(num_executors)} \\\n", + " --executor-cores {int(executor_cores)} \\\n", + " --executor-memory {int(executor_memory)}k \\\n", + " --conf spark.executor.memoryOverhead=1g \\\n", + " --conf spark.sql.broadcastTimeout=4800 \\\n", + " --conf spark.driver.maxResultSize=4g \\\n", + " --conf spark.sql.shuffle.partitions={numPartitions} \\\n", + " --conf spark.sql.parquet.compression.codec={dataSourceCodec} \\\n", + " --conf spark.network.timeout=800s \\\n", + " --conf spark.executor.heartbeatInterval=200s \\\n", + " --jars /home/{user}/ipython/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar \\\n", + "'''\n", + "\n", + "with open(\"tpch_datagen_parquet.sh\",\"w\") as f:\n", + " f.writelines(tpch_datagen_parquet)\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!nohup bash tpch_datagen_parquet.sh" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## TPCDS" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!rm -rf ~/tpcds-kit\n", + "!git clone https://github.com/databricks/tpcds-kit.git ~/tpcds-kit" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp -r ~/tpcds-kit {l}:~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} \"cd ~/tpcds-kit/tools && make clean && make OS=LINUX CC=gcc-9\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "%cd ~/gluten/tools/workload/tpcds/gen_data/parquet_dataset" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Suggest 2x cpu# partitions\n", + "scaleFactor = 1500\n", + "numPartitions = 2*cpu_num if len(clients)==0 else len(clients)*2*cpu_num\n", + "dataformat = \"parquet\" # data format of data source\n", + "dataSourceCodec = \"snappy\"\n", + "rootDir = f\"/tpcds_sf{scaleFactor}_{dataformat}_{dataSourceCodec}\" # root directory of location to create data in." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Verify parameters\n", + "print(f'scaleFactor = {scaleFactor}')\n", + "print(f'numPartitions = {numPartitions}')\n", + "print(f'dataformat = {dataformat}')\n", + "print(f'rootDir = {rootDir}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "scala=f'''import com.databricks.spark.sql.perf.tpcds._\n", + "\n", + "val scaleFactor = \"{scaleFactor}\" // scaleFactor defines the size of the dataset to generate (in GB).\n", + "val numPartitions = {numPartitions} // how many dsdgen partitions to run - number of input tasks.\n", + "\n", + "val format = \"{dataformat}\" // valid spark format like parquet \"parquet\".\n", + "val rootDir = \"{rootDir}\" // root directory of location to create data in.\n", + "val dsdgenDir = \"/home/{user}/tpcds-kit/tools/\" // location of dbgen\n", + "\n", + "val tables = new TPCDSTables(spark.sqlContext,\n", + " dsdgenDir = dsdgenDir,\n", + " scaleFactor = scaleFactor,\n", + " useDoubleForDecimal = false, // true to replace DecimalType with DoubleType\n", + " useStringForDate = false) // true to replace DateType with StringType\n", + "\n", + "\n", + "tables.genData(\n", + " location = rootDir,\n", + " format = format,\n", + " overwrite = true, // overwrite the data that is already there\n", + " partitionTables = true, // create the partitioned fact tables\n", + " clusterByPartitionColumns = true, // shuffle to get partitions coalesced into single files.\n", + " filterOutNullPartitionValues = false, // true to filter out the partition with NULL key value\n", + " tableFilter = \"\", // \"\" means generate all tables\n", + " numPartitions = numPartitions) // how many dsdgen partitions to run - number of input tasks.\n", + "'''\n", + "\n", + "with open(\"tpcds_datagen_parquet.scala\",\"w\") as f:\n", + " f.writelines(scala)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "executor_cores = 8\n", + "num_executors=cpu_num/executor_cores\n", + "executor_memory = (totalmemory - 10*1024*1024)/num_executors - 1*1024*1024\n", + "\n", + "# Verify parameters\n", + "print(f'--master {master}')\n", + "print(f'--num-executors {int(num_executors)}')\n", + "print(f'--executor-cores {int(executor_cores)}')\n", + "print(f'--executor-memory {int(executor_memory)}k')\n", + "print(f'--conf spark.sql.shuffle.partitions={numPartitions}')\n", + "print(f'--conf spark.sql.parquet.compression.codec={dataSourceCodec}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "tpcds_datagen_parquet=f'''\n", + "cat tpcds_datagen_parquet.scala | {os.environ['SPARK_HOME']}/bin/spark-shell \\\n", + " --master {master} \\\n", + " --name tpcds_gen_parquet \\\n", + " --driver-memory 10g \\\n", + " --num-executors {int(num_executors)} \\\n", + " --executor-cores {int(executor_cores)} \\\n", + " --executor-memory {int(executor_memory)}k \\\n", + " --conf spark.executor.memoryOverhead=1g \\\n", + " --conf spark.sql.broadcastTimeout=4800 \\\n", + " --conf spark.driver.maxResultSize=4g \\\n", + " --conf spark.sql.shuffle.partitions={numPartitions} \\\n", + " --conf spark.sql.parquet.compression.codec={dataSourceCodec} \\\n", + " --conf spark.network.timeout=800s \\\n", + " --conf spark.executor.heartbeatInterval=200s \\\n", + " --jars /home/{user}/ipython/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar \\\n", + "'''\n", + "\n", + "with open(\"tpcds_datagen_parquet.sh\",\"w\") as f:\n", + " f.writelines(tpcds_datagen_parquet)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!nohup bash tpcds_datagen_parquet.sh" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Install Trace-Viewer (optional)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Clone the master branch\n", + "```\n", + "cd ~\n", + "git clone https://github.com/catapult-project/catapult.git -b master\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Trace-Viewer requires python version 2.7. Create a virtualenv for python2.7\n", + "```\n", + "sudo apt install -y python2.7\n", + "virtualenv -p /usr/bin/python2.7 py27\n", + "source py27/bin/activate\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Apply patch\n", + "\n", + "```\n", + "cd catapult\n", + "```\n", + "```\n", + "git apply <\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# System Settings" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import os\n", + "import pandas as pd\n", + "\n", + "pd.set_option('display.max_rows', None)\n", + "\n", + "# Convert the os.environ object to a dictionary and then to a DataFrame\n", + "env_df = pd.DataFrame(list(dict(os.environ).items()), columns=['Environment Variable', 'Value'])\n", + "\n", + "# Display the DataFrame\n", + "from IPython.display import display\n", + "\n", + "display(env_df)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import socket\n", + "localhost=socket.gethostname()\n", + "local_ip=socket.gethostbyname(localhost)\n", + "\n", + "print(f'localhost: {localhost}')\n", + "print(f'ip: {local_ip}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "spark_version=!head -n1 $SPARK_HOME/RELEASE | awk '{print $2}'\n", + "spark_version = spark_version[0]\n", + "\n", + "print(f\"Spark version from SPARK_HOME: {spark_version}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import logging\n", + "import sys\n", + "\n", + "logging.basicConfig(format='%(levelname)s : %(message)s', level=logging.ERROR, stream=sys.stdout)\n", + "logger = logging.getLogger()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import os\n", + "\n", + "hdfs_event_dir=''\n", + "local_event_dir=''\n", + "\n", + "def get_spark_eventlog_dir(path):\n", + " eventlog_dir = None\n", + " eventlog_enabled = False\n", + " try:\n", + " with open(path, 'r') as f:\n", + " for line in f:\n", + " if line.startswith('spark.eventLog.dir'):\n", + " eventlog_dir = line.split(' ')[-1].strip()\n", + " elif line.startswith('spark.eventLog.enabled'):\n", + " eventlog_enabled = line.split(' ')[-1].strip().lower() == 'true'\n", + " except FileNotFoundError:\n", + " raise SystemExit(f\"'spark-defaults.conf' not found: {path}\")\n", + " if not eventlog_enabled:\n", + " raise SystemExit(\"'spark.eventLog.enabled' must be enabled.\")\n", + " return eventlog_dir\n", + "\n", + "spark_defaults_conf = None\n", + "\n", + "if 'SPARK_CONF_DIR' in os.environ:\n", + " spark_defaults_conf = os.path.join(os.environ['SPARK_CONF_DIR'], 'spark-defaults.conf')\n", + "elif 'SPARK_HOME' in os.environ:\n", + " spark_defaults_conf = os.path.join(os.environ['SPARK_HOME'], 'conf', 'spark-defaults.conf')\n", + "\n", + "if spark_defaults_conf:\n", + " event_log_dir = get_spark_eventlog_dir(spark_defaults_conf)\n", + " if event_log_dir:\n", + " print(f\"spark.eventLog.dir: {event_log_dir}\")\n", + " if event_log_dir[:7] == 'hdfs://':\n", + " hdfs_event_dir = event_log_dir\n", + " elif event_log_dir[:6] == 'file:/':\n", + " local_event_dir = event_log_dir[6:]\n", + " else:\n", + " raise SystemExit(f\"'spark.eventLog.dir' is not configured in {spark_defaults_conf}\")\n", + "else:\n", + " raise SystemExit(\"Cannot get `spark.eventLog.dir`. Neither SPARK_CONF_DIR nor SPARK_HOME defined in envrionment variables.\")\n", + " " + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Monitor" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import findspark\n", + "import os\n", + "\n", + "findspark.init(os.environ['SPARK_HOME'])\n", + "os.environ.setdefault('SPARK_SUBMIT_OPTS', '-Dscala.usejavacp=true')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import warnings\n", + "warnings.filterwarnings('ignore')\n", + "\n", + "import atexit\n", + "import collections\n", + "import gzip\n", + "import importlib\n", + "import json\n", + "import logging\n", + "import math\n", + "import os\n", + "import pathlib\n", + "import shutil\n", + "import signal\n", + "import subprocess\n", + "import tempfile\n", + "import threading\n", + "import time\n", + "import timeit\n", + "import traceback\n", + "\n", + "import matplotlib\n", + "import matplotlib.colors as colors\n", + "import matplotlib.pyplot as plt\n", + "import matplotlib.ticker as mtick\n", + "import numpy as np\n", + "import pandas as pd\n", + "import platform\n", + "import pyspark\n", + "import pyspark.sql.functions as F\n", + "import pyspark.sql.types as T\n", + "import spylon_kernel\n", + "from collections import namedtuple\n", + "from concurrent.futures import ThreadPoolExecutor\n", + "from datetime import date\n", + "from functools import reduce\n", + "from IPython.display import display, HTML\n", + "from matplotlib import rcParams\n", + "from pyspark import SparkConf, SparkContext\n", + "from pyspark.ml import Pipeline\n", + "from pyspark.ml.feature import StringIndexer, VectorAssembler\n", + "from pyspark.sql import SparkSession, SQLContext, Window\n", + "from pyspark.sql.functions import col, floor, lit, rank, to_date\n", + "from pyspark.sql.types import (DoubleType, FloatType, IntegerType,\n", + " StringType, StructField, StructType,\n", + " TimestampType)\n", + "\n", + "from spylon_kernel import register_ipython_magics\n", + "from spylon.spark.utils import SparkJVMHelpers\n", + "\n", + "register_ipython_magics()\n", + "\n", + "rcParams['font.sans-serif'] = 'Courier New'\n", + "rcParams['font.family'] = 'Courier New'\n", + "rcParams['font.size'] = '12'\n", + "\n", + "%matplotlib inline\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import socket\n", + "import os\n", + "import sys\n", + "\n", + "from pathlib import Path\n", + "home = str(Path.home())\n", + "\n", + "def upload_profile(server, base_dir, appid):\n", + " local_profile_dir = os.path.join(home, 'profile')\n", + " !mkdir -p {local_profile_dir}\n", + " !cd {local_profile_dir}; rm -f {appid}.tar.gz; tar zcvf {appid}.tar.gz {appid} >/dev/null 2>&1\n", + " \n", + " server_local_dir=os.path.join('PAUS', base_dir)\n", + " server_local_profile_dir=os.path.join(server_local_dir, 'profile')\n", + " server_hdfs_dir=f'/{base_dir}/'\n", + "\n", + " !ssh {server} \"mkdir -p {server_local_profile_dir}\"\n", + " !ssh {server} \"cd {server_local_profile_dir}; rm {appid}.tar.gz; rm -r {appid} >/dev/null 2>&1\"\n", + " !scp {local_profile_dir}/{appid}.tar.gz {server}:{server_local_profile_dir}/\n", + " !ssh {server} \"cd {server_local_profile_dir} && tar zxf {appid}.tar.gz\"\n", + " !ssh {server} \"hdfs dfs -mkdir -p {server_hdfs_dir}; hdfs dfs -rm -r {server_hdfs_dir}{appid}; hdfs dfs -put {server_local_profile_dir}/{appid} {server_hdfs_dir}\"\n", + " !ssh {server} \"cd {server_local_profile_dir}; rm {appid}.tar.gz; rm -r {appid}\"\n", + "\n", + "def killsar(clients):\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w sar | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w pidstat | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w perf | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh root@$l \"kill $x > /dev/null 2>&1\"\n", + "\n", + "def killnumactl(clients):\n", + " for l in clients:\n", + " out =!ssh $l \"ps aux | grep numactl | grep bash | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + "\n", + "def startmonitor(clients,appid,**kwargs):\n", + " local_profile_dir=os.path.join(home, 'profile')\n", + " prof=os.path.join(local_profile_dir, appid)\n", + " !mkdir -p {prof}\n", + " \n", + " for l in clients:\n", + " !ssh root@{l} date\n", + " \n", + " killsar(clients)\n", + " \n", + " perfsyscalls=kwargs.get(\"collect_perf_syscall\",None)\n", + " \n", + " for l in clients:\n", + " prof_client=os.path.join(prof, l)\n", + " !mkdir -p {prof_client}\n", + " !ssh {l} mkdir -p {prof_client}\n", + " !ssh {l} \"sar -o {prof_client}/sar.bin -r -u -d -B -n DEV 1 >/dev/null 2>&1 &\"\n", + " !ssh root@{l} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f 1 | xargs -I % bash -c '(cat /proc/%/status >> {prof_client}/%.stat; cat /proc/%/io >> {prof_client}/%.stat)'\"\n", + " if kwargs.get(\"collect_pid\",False):\n", + " !ssh {l} \"jps | grep CoarseGrainedExecutorBackend | head -n 1 | cut -d' ' -f 1 | xargs -I % pidstat -h -t -p % 1 > {prof_client}/pidstat.out 2>/dev/null &\"\n", + " !ssh root@{l} 'cat /proc/uptime | cut -d\" \" -f 1 | xargs -I ^ date -d \"- ^ seconds\" +%s.%N' > $prof/$l/uptime.txt\n", + " if kwargs.get(\"collect_sched\",False):\n", + " !ssh root@{l} 'perf trace -e \"sched:sched_switch\" -C 8-15 -o {prof_client}/sched.txt -T -- sleep 10000 >/dev/null 2>/dev/null &'\n", + " if perfsyscalls is not None:\n", + " !ssh root@{l} \"perf stat -e 'syscalls:sys_exit_poll,syscalls:sys_exit_epoll_wait' -a -I 1000 -o {prof_client}/perfstat.txt >/dev/null 2>&1 & \"\n", + " if kwargs.get(\"collect_hbm\",False):\n", + " hbm_nodes = kwargs.get(\"hbm_nodes\")\n", + " if hbm_nodes is not None:\n", + " print(\"collect_hbm\")\n", + " hbm_nodes = '\\|'.join([\"node \" + str(i) for i in hbm_nodes])\n", + " %env hbm_numa_nodes={hbm_nodes}\n", + " %env hbm_l = {l}\n", + " %env hbm_prof = {prof}\n", + " !ssh $hbm_l \"echo timestamp, size, free > $hbm_prof/$hbm_l/numactl.csv\"\n", + " !ssh $hbm_l \"while :; do echo \\$(numactl -H | grep '$hbm_numa_nodes' | grep 'size' | awk '{ print \\$4 }' | awk '{ s += \\$1 } END { print s }'), \\$(numactl -H | grep '$hbm_numa_nodes' | grep 'free' | awk '{ print \\$4 }' | awk '{ s += \\$1 } END { print s }') | ts '%Y-%m-%d %H:%M:%S,' >> $hbm_prof/$hbm_l/numactl.csv; sleep 1; done >/dev/null 2>&1 &\"\n", + " else:\n", + " print(\"Missing argument: hbm_nodes. e.g. hbm_nodes = list(range(8,16))\")\n", + " return prof\n", + "\n", + "def stopmonitor(clients, sc, appid, **kwargs):\n", + " %cd ~\n", + " \n", + " local_profile_dir=os.path.join(home, 'profile')\n", + " prof=os.path.join(local_profile_dir, appid)\n", + " !mkdir -p {prof}\n", + "\n", + " killsar(clients)\n", + " killnumactl(clients) \n", + " \n", + " with open(f\"{prof}/starttime\",\"w\") as f:\n", + " f.write(\"{:d}\".format(int(time.time()*1000)))\n", + " \n", + " for l in clients:\n", + " prof_client=os.path.join(prof, l)\n", + " !ssh {l} \"sar -f {prof_client}/sar.bin -r > {prof_client}/sar_mem.sar;sar -f {prof_client}/sar.bin -u > {prof_client}/sar_cpu.sar;sar -f {prof_client}/sar.bin -d -p > {prof_client}/sar_disk.sar;sar -f {prof_client}/sar.bin -n DEV > {prof_client}/sar_nic.sar;sar -f {prof_client}/sar.bin -B > {prof_client}/sar_page.sar;\" \n", + " !ssh root@{l} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f 1 | xargs -I % bash -c '(cat /proc/%/status >> {prof_client}/%.stat; cat /proc/%/io >> {prof_client}/%.stat)'\"\n", + " !ssh {l} \"sar -V \" > {prof_client}/sarv.txt\n", + " !test -f {prof_client}/perfstat.txt && head -n 1 {prof_client}/perfstat.txt > {prof_client}/perfstarttime\n", + " if l!= socket.gethostname():\n", + " !scp -r {l}:{prof_client} {prof}/ > /dev/null 2>&1\n", + " \n", + " if sc is not None:\n", + " sc.stop()\n", + " \n", + " if hdfs_event_dir != '':\n", + " !hadoop fs -copyToLocal {hdfs_event_dir}/{appid} {prof}/app.log\n", + " elif local_event_dir != '':\n", + " !cp {local_event_dir}/{appid} {prof}/app.log" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def pinexecutor_numa(clients):\n", + " cpunum = !ssh {clients[0]} \"grep 'processor' /proc/cpuinfo | wc -l\"\n", + " cpunum = int(cpunum[0])\n", + " \n", + " numanodes=!ssh {clients[0]} \"cat /sys/devices/system/node/node*/cpulist\"\n", + " numanodes = list(filter(lambda x: x != '', numanodes))\n", + " print(numanodes)\n", + " for client in clients:\n", + " pids=!ssh {client} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f1\"\n", + " print(client,\":\",len(pids),\" \",\"\\t\".join(map(str,pids)))\n", + " \n", + " cpunum_c = !ssh {client} \"grep 'processor' /proc/cpuinfo | wc -l\"\n", + " cpunum_c = int(cpunum_c[0])\n", + " if cpunum_c != cpunum:\n", + " print(f\"client {client} cpunum not match!\")\n", + " return\n", + " numanodes_c=!ssh {client} \"cat /sys/devices/system/node/node*/cpulist\"\n", + " numanodes_c = list(filter(lambda x: x != '', numanodes))\n", + " time.sleep(1)\n", + " print(numanodes_c)\n", + " if numanodes_c != numanodes:\n", + " print(f\"client {client} numanodes not match!\")\n", + " return\n", + " \n", + " idx = 0\n", + " nodes=len(numanodes)\n", + " for i in range(nodes):\n", + " cpus = numanodes[i]\n", + " for l in pids[idx:idx+int(len(pids)/nodes)]: # executors on 1 numanode\n", + " print(f\" {cpus} {l}\")\n", + " !ssh {client} \"taskset -a -p -c $cpus $l > /dev/null 2>&1 \"\n", + " idx += int(len(pids)/nodes)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def config_pagecache(clients, run_gluten=True):\n", + " for l in clients:\n", + " if run_gluten:\n", + " !ssh root@$l \"echo 80 > /proc/sys/vm/dirty_ratio\"\n", + " !ssh root@$l \"echo 50 > /proc/sys/vm/dirty_background_ratio\"\n", + " !ssh root@$l \"echo 360000 > /proc/sys/vm/dirty_expire_centisecs\"\n", + " !ssh root@$l \"echo 3000 > /proc/sys/vm/dirty_writeback_centisecs\"\n", + "\n", + " else:\n", + " !ssh root@$l \"echo 10 > /proc/sys/vm/dirty_ratio\"\n", + " !ssh root@$l \"echo 20 > /proc/sys/vm/dirty_background_ratio\"\n", + " !ssh root@$l \"echo 3000 > /proc/sys/vm/dirty_expire_centisecs\"\n", + " !ssh root@$l \"echo 500 > /proc/sys/vm/dirty_writeback_centisecs\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def print_kernel_params(clietns):\n", + " params = {\n", + " 'transparent hugepage': '/sys/kernel/mm/transparent_hugepage/enabled',\n", + " 'auto numa balancing': '/proc/sys/kernel/numa_balancing',\n", + " 'scaling governor': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_governor',\n", + " 'scaling max freq': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_max_freq',\n", + " 'scaling cur freq': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_cur_freq',\n", + " 'power & perf policy': '/sys/devices/system/cpu/cpu*/power/energy_perf_bias',\n", + " 'dirty_ratio': '/proc/sys/vm/dirty_ratio',\n", + " 'dirty_background_ratio': '/proc/sys/vm/dirty_background_ratio',\n", + " 'dirty_expire_centisecs': '/proc/sys/vm/dirty_expire_centisecs',\n", + " 'dirty_writeback_centisecs': '/proc/sys/vm/dirty_writeback_centisecs'\n", + " }\n", + " for k, param in params.items():\n", + " print()\n", + " print(f'{k} ({param})')\n", + " for l in clients:\n", + " print(l + \": \", end='')\n", + " res = !ssh root@$l \"cat {param}\"\n", + " print(*res)\n", + " # print numactl\n", + " print()\n", + " print(\"numactl -H\")\n", + " for l in clients:\n", + " print(l + \":\")\n", + " res = !ssh $l \"numactl -H\"\n", + " print('\\n'.join(res))\n", + " # print memory freq\n", + " print()\n", + " print(\"Memory Frequency\")\n", + " for l in clients:\n", + " print(l + \":\")\n", + " res= !ssh root@$l \"dmidecode -t memory | grep Speed\"\n", + " print('\\n'.join(res))" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "def dropcache(clients):\n", + " for l in clients:\n", + " !ssh root@$l \"sync && echo 3 > /proc/sys/vm/drop_caches; echo 1 >/proc/sys/vm/compact_memory; free -h\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def config_mem_cgroup(clients):\n", + " mem_cgroup = \"\"\"\n", + "CGROUP_ROOT=/sys/fs/cgroup/gluten\n", + "\n", + "if [ ! -d $CGROUP_ROOT ] ; then\n", + " sudo mkdir $CGROUP_ROOT\n", + " # enable memory for subtree\n", + " sudo bash -c \"echo '+memory' >> $CGROUP_ROOT/cgroup.subtree_control\"\n", + "fi\n", + "\n", + "# move each process to sub memory group\n", + "index=0\n", + "for pid in `jps | grep Coarse | awk '{print $1}'` ; do\n", + " target_cgroup=$CGROUP_ROOT/mem-${index}\n", + " if [ ! -d $target_cgroup ] ; then\n", + " sudo mkdir $target_cgroup\n", + " fi\n", + " proc_file=$target_cgroup/cgroup.procs\n", + " sudo bash -c \"echo $pid >> $proc_file\"\n", + " index=`expr $index + 1`\n", + "done\n", + " \"\"\"\n", + " with open(f'{home}/mem-cgroup.sh', 'w+') as f:\n", + " f.writelines(mem_cgroup)\n", + " for l in clients:\n", + " !scp {home}/mem-cgroup.sh {l}:{home}/ >/dev/null 2>&1\n", + " !ssh {l} \"bash {home}/mem-cgroup.sh >/dev/null 2>&1 &\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import pandas as pd\n", + "import matplotlib.pyplot as plt\n", + "import os\n", + "\n", + "\n", + "from IPython.display import display, HTML\n", + "\n", + "def get_io_stats(appid, client):\n", + " file_path = os.path.join(home,'profile',appid,client)\n", + " statf = [f for f in os.listdir(file_path) if f.endswith('.stat')]\n", + " statmap=[]\n", + " for f in statf:\n", + " statmap.append({'pid':f[:-len(\".stat\")]})\n", + " with open(os.path.join(file_path, f),\"r\") as fi:\n", + " cnts=fi.readlines()\n", + " for l in cnts:\n", + " for fld in ['rchar','wchar','syscr','syscw','read_bytes','write_bytes','cancelled_write_bytes']:\n", + " if l.startswith(fld):\n", + " if not fld in statmap[-1]:\n", + " statmap[-1][fld]=int(l.split(\" \")[-1].strip())\n", + " else:\n", + " statmap[-1][fld]=(int(l.split(\" \")[-1].strip())-statmap[-1][fld])/1024/1024/1024\n", + "\n", + " df = pd.DataFrame(statmap).drop('pid', axis=1).sum().to_frame()\n", + " df.columns = ['sum']\n", + " return df\n", + "\n", + "# Preprocess 'time' column\n", + "def process_time(dataframes):\n", + " for df in dataframes:\n", + " df.columns=['time']+list(df.columns[1:])\n", + " df = df[df.time != 'Average:']\n", + " df['time'] = pd.to_datetime(df['time'], format='%H:%M:%S').dt.time\n", + " df['time'] = df['time'].apply(lambda dt: dt.hour*3600 + dt.minute*60 + dt.second)\n", + "\n", + " offset = 12 * 3600 # half-day seconds\n", + " for i in range(1, len(df)):\n", + " if df['time'].iloc[i] < df['time'].iloc[i-1]: # Detect AM->PM or PM->AM\n", + " for j in range(i, len(df)): # Apply offset until end\n", + " df['time'].iloc[j] += offset\n", + "\n", + " df['time'] = df['time'].astype(int)\n", + " yield df\n", + "\n", + "def draw_sar(appid, qtime=None, disk_dev=None, nic_dev=None, client=None):\n", + " if client is None:\n", + " client = clients[0]\n", + "\n", + " display(HTML('{:s}'.format(client)))\n", + "\n", + " display(get_io_stats(appid, client))\n", + "\n", + " # Read data\n", + " profile_dir = os.path.join(home,'profile',appid,client)\n", + " datafiles = [os.path.join(profile_dir, datafile) for datafile in ['sar_cpu.sar', 'sar_mem.sar', 'sar_disk.sar', 'sar_nic.sar', 'sar_page.sar']]\n", + " dataframes = [pd.read_csv(datafile, header=1, delim_whitespace=True, parse_dates=True) for datafile in datafiles]\n", + " \n", + " num_figs=5\n", + " fig, axs=plt.subplots(num_figs,1,sharex=True,figsize=(30,5*4))\n", + "\n", + " [cpu_df, mem_df, disk_df, nic_df, page_df] = process_time(dataframes)\n", + "\n", + " # CPU usage\n", + " cpu_df['total'] = cpu_df['%user'] + cpu_df['%system'] + cpu_df['%iowait']\n", + "\n", + " starttime = cpu_df[cpu_df['total'] > 50]['time'].min() - 1\n", + " cpu_df['time'] -= starttime\n", + "\n", + " axs[4].stackplot(cpu_df['time'], cpu_df['%user'], cpu_df['%system'], cpu_df['%iowait'], labels=['user','system','iowait'])\n", + " axs[4].legend(loc='upper left')\n", + "\n", + " # Memory usage\n", + " mem_df['dirty_cached'] = mem_df['kbdirty'] * mem_df['%memused'] / mem_df['kbmemused']\n", + " mem_df['clean_cached'] = (mem_df['kbcached'] - mem_df['kbdirty']) * mem_df['%memused'] / mem_df['kbmemused']\n", + " mem_df['used'] = mem_df['kbmemused'] * mem_df['%memused'] / mem_df['kbmemused']\n", + "# mem_df['used'] = (mem_df['kbmemused'] - mem_df['kbbuffers'] - mem_df['kbcached'])* mem_df['%memused'] / mem_df['kbmemused']\n", + "\n", + " mem_df['time'] -= starttime\n", + "\n", + " axs[0].stackplot(mem_df['time'], mem_df['used'], mem_df['clean_cached'], mem_df['dirty_cached'], labels=['used','clean cached','dirty cached'])\n", + " axs[0].legend(loc='upper left')\n", + " axs[0].grid(axis = 'y')\n", + "\n", + " # Disk usage\n", + " if disk_dev is not None:\n", + " disk_df = disk_df[disk_df['DEV'].isin(disk_dev)]\n", + " disk_df['rkB/s'] = disk_df['rkB/s'].astype(float)\n", + " disk_df['wkB/s'] = disk_df['wkB/s'].astype(float)\n", + " disk_df['%util'] = disk_df['%util'].astype(float)\n", + "\n", + "\n", + " disk_df = disk_df.groupby('time').agg({'rkB/s': 'sum', 'wkB/s': 'sum', '%util':'mean'}).reset_index()\n", + " disk_df['read'] = disk_df['rkB/s'] / 1024\n", + " disk_df['write'] = disk_df['wkB/s'] / 1024\n", + "\n", + " disk_df['time'] -= starttime\n", + "\n", + " axs[1].stackplot(disk_df['time'], disk_df['read'], disk_df['write'], labels=['read MB/s','write MB/s'])\n", + " axs[1].grid(axis = 'y')\n", + "\n", + " ax2 = axs[1].twinx()\n", + "\n", + " ax2.plot(disk_df['time'], disk_df['%util'],'g-')\n", + " axs[1].legend(loc='upper left')\n", + "\n", + " \n", + " # Nic usage\n", + " if nic_dev is not None:\n", + " nic_df = nic_df[nic_df['IFACE'].isin(nic_dev)]\n", + " nic_df['rxkB/s'] = nic_df['rxkB/s'].astype(float)\n", + " nic_df['txkB/s'] = nic_df['txkB/s'].astype(float)\n", + " \n", + " nic_df = nic_df.groupby('time').agg({'rxkB/s': 'sum', 'txkB/s': \"sum\"}).reset_index()\n", + " nic_df['rx'] = nic_df['rxkB/s'] / 1024\n", + " nic_df['tx'] = nic_df['txkB/s'] / 1024\n", + " \n", + " nic_df['time'] -= starttime\n", + " \n", + " axs[2].stackplot(nic_df['time'], nic_df['rx'], nic_df['tx'], labels=['rx MB/s','tx MB/s'])\n", + " axs[2].legend(loc='upper left')\n", + " axs[2].grid(axis = 'y')\n", + "\n", + " # Pagefaults\n", + " page_df['minflt/s'] = page_df['fault/s'] - page_df['majflt/s']\n", + " \n", + " page_df['time'] -= starttime\n", + "\n", + " axs[3].stackplot(page_df['time'], page_df['minflt/s'], page_df['majflt/s'], labels=['minor_fault/s','major_fault/s'])\n", + " axs[3].legend(loc='upper left')\n", + " axs[3].grid(axis = 'y')\n", + "\n", + " # Add vertical lines and text for qtime, and calculate per query cpu%\n", + " if qtime is not None:\n", + " for ax in axs:\n", + " x = 0\n", + " ax.axvline(x = x, color = 'b')\n", + " for k, v in qtime.items():\n", + " x += v\n", + " ax.axvline(x = x, color = 'b')\n", + "\n", + " tx = 0\n", + " for k, v in qtime.items():\n", + " if v / x > 15 / 772:\n", + " ax.text(tx + v / 2 - 6 * x / 772, ax.get_ylim()[1] * 1.05, k)\n", + " tx += v\n", + "\n", + " x = 0\n", + " qtime_se = {}\n", + " cols = ['%user','%system','%iowait']\n", + " for k, v in qtime.items():\n", + " filtered_df = cpu_df[(cpu_df['time'] >= x) & (cpu_df['time'] <= x+v)]\n", + " averages = filtered_df[cols].mean()\n", + " qtime_se[k] = averages.tolist()\n", + " x += v\n", + " if qtime_se:\n", + " perqcpu = pd.DataFrame(qtime_se).T\n", + " perqcpu.columns = cols\n", + " display(perqcpu)\n", + "\n", + " plt.show()\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def convert_to_etc_gmt(tz_offset=None):\n", + " # Run the 'date +%z' command and get the output\n", + " if not tz_offset:\n", + " tz_offset = !date +%z\n", + " tz_offset = tz_offset[0]\n", + " \n", + " # Extract the sign and the hour/minute offset\n", + " sign = tz_offset[0]\n", + " hours = int(tz_offset[1:3])\n", + " minutes = int(tz_offset[3:])\n", + "\n", + " # Convert the offset to a GMT value\n", + " gmt_offset = hours + (minutes / 60)\n", + " if sign == '+':\n", + " gmt_offset = -gmt_offset\n", + " else:\n", + " gmt_offset = abs(gmt_offset)\n", + "\n", + " # Construct the Etc/GMT string\n", + " etc_gmt = f\"Etc/GMT{int(gmt_offset):+d}\"\n", + " return etc_gmt" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# TestTPC" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import os\n", + "import socket\n", + "from dataclasses import dataclass\n", + "from functools import wraps\n", + "from pathlib import Path\n", + "from typing import List \n", + "\n", + "class TestTPC:\n", + " @dataclass\n", + " class query_info:\n", + " tables: List[str]\n", + " sql: List[str]\n", + "\n", + " query_infos = {}\n", + " query_ids =[]\n", + "\n", + " tpctables=[]\n", + " tpc_query_path = ''\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, workload, server, base_dir, nb_name, data_source = 'parquet'):\n", + " self.spark = spark\n", + " self.sc = spark.sparkSession.sparkContext\n", + " self.appid = self.sc.applicationId\n", + " self.run_gluten = run_gluten\n", + " self.workload = workload\n", + " self.table_dir = table_dir\n", + " self.server = server\n", + " self.base_dir = base_dir\n", + " self.nb_name = nb_name\n", + " self.data_source = data_source\n", + " self.table_loaded = False\n", + " self.result = {}\n", + " self.stopped = False\n", + " self.perf_html = ''\n", + " self.finished_nb = ''\n", + " for l in os.listdir(self.tpc_query_path):\n", + " if (l[-3:] == 'sql'):\n", + " with open(self.tpc_query_path+l,\"r\") as f:\n", + " self.query_infos[l.split(\".\")[0]]=self.query_info(self.tpctables,[\"\\n\".join(f.readlines())])\n", + " self.query_ids = sorted(self.query_infos.keys(), key=lambda x: str(len(x))+x if x[-1] != 'a' and x[-1] != 'b' else str(len(x)-1) + x)\n", + " print(\"http://{}:18080/history/{}/jobs/\".format(local_ip, self.sc.applicationId))\n", + " \n", + " def start_monitor(self, clients, **kw):\n", + " startmonitor(clients, self.appid, **kw)\n", + " \n", + " def stop_monitor(self, clients, **kw):\n", + " if self.stopped:\n", + " return\n", + " stopmonitor(clients, self.sc, self.appid, **kw)\n", + " if self.server:\n", + " output_nb = f'{self.nb_name[:-6]}-{self.appid}.ipynb'\n", + " if output_nb.startswith(home):\n", + " output_nb_name = os.path.relpath(output_nb, home)\n", + " else:\n", + " output_nb_name = output_nb\n", + " output_nb_dir = os.path.dirname(output_nb_name)\n", + " server_nb_dir = os.path.join('PAUS', self.base_dir, output_nb_dir)\n", + " !ssh {self.server} \"mkdir -p {server_nb_dir}\"\n", + " !scp {output_nb} {self.server}:{server_nb_dir}\n", + " self.finished_nb = f\"http://{self.server}:8888/tree/{self.base_dir}/{output_nb_name}\"\n", + " self.stopped = True\n", + "\n", + " def run_perf_analysis(self, disk_dev, nic_dev):\n", + " if not self.server:\n", + " return\n", + "\n", + " upload_profile(self.server, self.base_dir, self.appid)\n", + "\n", + " ts=time.strftime(\"%Y_%m_%d_%H%M%S\")\n", + " name=f'{self.workload}_gluten' if self.run_gluten else f'{self.workload}_vanilla'\n", + " run_script=f'{gluten_home}/tools/workload/benchmark_velox/analysis/run_perf_analysis.sh'\n", + " \n", + " disk=','.join(disk_dev)\n", + " nic=','.join(nic_dev)\n", + "\n", + " command =' '.join(['bash', run_script, '--ts', ts, '--base-dir', self.base_dir, '--name', name, '--appid', self.appid, '--disk', disk, '--nic', nic, '--tz', convert_to_etc_gmt()])\n", + " print(command)\n", + "\n", + " # Block if running on local cluster.\n", + " if self.server == localhost:\n", + " !ssh {self.server} \"{command} > /dev/null 2>&1\"\n", + " else:\n", + " !ssh {self.server} \"{command} > /dev/null 2>&1 &\"\n", + "\n", + " self.perf_html=f'http://{self.server}:8888/view/{self.base_dir}/html/{ts}_{name}_{self.appid}.html'\n", + " display(HTML(f'
    {self.perf_html}'))\n", + " \n", + " def load_table(self, table):\n", + " if type(self.table_dir)==list:\n", + " return self.spark.read.format(self.data_source).load([os.path.join(t, table) for t in self.table_dir])\n", + " else:\n", + " return self.spark.read.format(self.data_source).load(os.path.join(self.table_dir, table))\n", + " \n", + " def load_tables_as_tempview(self, tables):\n", + " for table in tables:\n", + " df = self.load_table(table)\n", + " df.createOrReplaceTempView(table)\n", + " \n", + " def load_all_tables_as_tempview(self):\n", + " print(f\"Loading all tables: {self.tpctables}\")\n", + " self.load_tables_as_tempview(self.tpctables)\n", + " \n", + " def load_query(self, query):\n", + " info = self.query_infos[query]\n", + " return [self.spark.sql(q) for q in info.sql]\n", + " \n", + " def run_query(self, query, explain = False, print_result=False, load_table=True):\n", + " if load_table:\n", + " self.load_all_tables_as_tempview()\n", + " start_time = timeit.default_timer()\n", + " print(\"start query \" + query + \", application id \" + self.sc.applicationId)\n", + " print(\"{} : {}\".format(\"Start time\", start_time))\n", + " self.sc.setJobDescription(query)\n", + "\n", + " queries = self.load_query(query)\n", + " for q in queries:\n", + " if explain: q.explain()\n", + " collect=q.collect()\n", + " end_time = timeit.default_timer()\n", + " duration = end_time - start_time\n", + " display(HTML(('Completed Query. Time(sec): {:f}'.format(duration))))\n", + " \n", + " self.result[query] = duration\n", + " if print_result:\n", + " print(collect)\n", + "\n", + " def power_run(self, explain=False, print_result=False, load_table=True):\n", + " if load_table:\n", + " self.load_all_tables_as_tempview()\n", + " for l in self.query_ids:\n", + " self.run_query(l, explain=explain, print_result=print_result, load_table=False)\n", + "\n", + " def print_result(self):\n", + " print(self.result)\n", + " print()\n", + " durations = [float(i) for i in self.result.values()]\n", + " print(\"total duration:\")\n", + " print(sum(durations))\n", + " print()\n", + " if self.server:\n", + " print(self.finished_nb)\n", + " print(f\"http://{self.server}:1088/tracing_examples/trace_viewer.html#/tracing/test_data/{self.appid}.json\")\n", + " print(f\"http://{self.server}:18080/history/{self.appid}\")\n", + " print(self.perf_html)\n", + " print(self.appid)\n", + " for i in durations:\n", + " print(i)\n", + " \n", + "class TestTPCH(TestTPC):\n", + " tpctables = ['customer', 'lineitem', 'nation', 'orders', 'part', 'partsupp', 'region', 'supplier']\n", + " tpc_query_path = f'{gluten_home}/tools/gluten-it/common/src/main/resources/tpch-queries/'\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, server, base_dir, nb_name, data_source = 'parquet'):\n", + " TestTPC.__init__(self,spark, table_dir, run_gluten, 'tpch', server, base_dir, nb_name, data_source)\n", + " \n", + "class TestTPCDS(TestTPC):\n", + " tpctables = [ 'call_center',\n", + " 'catalog_page',\n", + " 'catalog_returns',\n", + " 'catalog_sales',\n", + " 'customer',\n", + " 'customer_address',\n", + " 'customer_demographics',\n", + " 'date_dim',\n", + " 'household_demographics',\n", + " 'income_band',\n", + " 'inventory',\n", + " 'item',\n", + " 'promotion',\n", + " 'reason',\n", + " 'ship_mode',\n", + " 'store',\n", + " 'store_returns',\n", + " 'store_sales',\n", + " 'time_dim',\n", + " 'warehouse',\n", + " 'web_page',\n", + " 'web_returns',\n", + " 'web_sales',\n", + " 'web_site']\n", + " tpc_query_path = f'{gluten_home}/tools/gluten-it/common/src/main/resources/tpcds-queries/'\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, server, base_dir, nb_name, data_source = 'parquet'):\n", + " TestTPC.__init__(self,spark, table_dir, run_gluten, 'tpcds', server, base_dir, nb_name, data_source)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Create SparkContext" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## default config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import os\n", + "\n", + "def findjemalloc():\n", + " l = clients[0]\n", + " jemallocDir = !ssh $l \"whereis libjemalloc.so.2\"\n", + " libjemalloc = jemallocDir[0].split(' ')\n", + " return libjemalloc[1]\n", + "\n", + "def get_py4jzip():\n", + " spark_home=os.environ['SPARK_HOME']\n", + " py4jzip = !ls {spark_home}/python/lib/py4j*.zip\n", + " return py4jzip[0]\n", + "\n", + "def default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars='', app_name='', master='yarn', run_gluten=False):\n", + " # Create a temp directory that gets cleaned up on exit\n", + " output_dir = os.path.abspath(tempfile.mkdtemp())\n", + " def cleanup():\n", + " shutil.rmtree(output_dir, True)\n", + " atexit.register(cleanup)\n", + " signal.signal(signal.SIGTERM, cleanup)\n", + "\n", + "##################################################\n", + " def convert_to_bytes(size):\n", + " units = {'k': 1, 'm': 2, 'g': 3}\n", + " size = size.lower()\n", + " if size[-1] in units:\n", + " return int(size[:-1]) * 1024 ** units[size[-1]]\n", + " else:\n", + " return int(size)\n", + "\n", + " def yarn_padding(size):\n", + " min_size = convert_to_bytes('1g')\n", + " step = min_size\n", + " while size > min_size:\n", + " min_size += step\n", + " return min_size - size\n", + " \n", + " num_nodes = len(clients)\n", + " num_executors = num_nodes*executors_per_node\n", + " parallelism = num_executors*cores_per_executor*task_per_core\n", + "\n", + " if run_gluten:\n", + " offheap_ratio = gluten_offheap_ratio\n", + " else:\n", + " offheap_ratio = vanilla_offheap_ratio\n", + " driver_memory = convert_to_bytes('20g')\n", + " executor_memory_overhead = convert_to_bytes('1g')\n", + " \n", + " # Minimun executor memory\n", + " min_memory = convert_to_bytes('1g')\n", + "\n", + " # Calculate executor onheap memory\n", + " num_driver = 1 if localhost in clients else 0\n", + " executor_memory = math.floor((convert_to_bytes(memory_per_node) - (executor_memory_overhead + min_memory)*executors_per_node - (driver_memory + min_memory)*num_driver)/(offheap_ratio*num_driver + (1+offheap_ratio)*executors_per_node))\n", + " executor_memory = max(executor_memory, min_memory)\n", + " # Calculate driver/executor offheap memory in MB\n", + " #offheap_memory_per_node = convert_to_bytes(memory_per_node) - (executor_memory + executor_memory_overhead) * executors_per_node\n", + " if offheap_ratio > 0:\n", + " enable_offheap = True\n", + " offheap_memory = math.floor(executor_memory*offheap_ratio)\n", + " else:\n", + " enable_offheap = False\n", + " offheap_memory = 0\n", + "\n", + " byte_to_mb = lambda x: int(x/(1024 ** 2))\n", + " driver_memory_mb = byte_to_mb(driver_memory)\n", + " executor_memory_overhead_mb = byte_to_mb(executor_memory_overhead)\n", + " executor_memory_mb = byte_to_mb(executor_memory)\n", + " offheap_memory_mb = byte_to_mb(offheap_memory)\n", + " \n", + " executor_totalmem_mb = executor_memory_overhead_mb + executor_memory_mb + offheap_memory_mb\n", + " executor_totalmem_mb = yarn_padding(executor_totalmem_mb)\n", + " if byte_to_mb(convert_to_bytes(memory_per_node)) - executor_totalmem_mb*executors_per_node > executor_totalmem_mb:\n", + " executor_memory_overhead_mb += 1024\n", + " \n", + " print('''\n", + " executors per node: {:d}\n", + " parallelism: {:d}\n", + " executor memory: {:d}m\n", + " offheap memory: {:d}m\n", + " '''.format(executors_per_node, parallelism, executor_memory_mb, offheap_memory_mb))\n", + "\n", + " conf = SparkConf() \\\n", + " .set('spark.app.name', app_name)\\\n", + " .set('spark.master',master)\\\n", + " .set('spark.executor.memory', '{:d}m'.format(executor_memory_mb))\\\n", + " .set('spark.memory.offHeap.enabled', enable_offheap)\\\n", + " .set('spark.memory.offHeap.size','{:d}m'.format(offheap_memory_mb))\\\n", + " .set('spark.sql.shuffle.partitions', parallelism)\\\n", + " .set('spark.executor.instances', '{:d}'.format(num_executors))\\\n", + " .set('spark.executor.cores','{:d}'.format(cores_per_executor))\\\n", + " .set('spark.task.cpus','{:d}'.format(1))\\\n", + " .set('spark.driver.memory', '{:d}m'.format(driver_memory_mb))\\\n", + " .set('spark.executor.memoryOverhead', '{:d}m'.format(executor_memory_overhead_mb))\\\n", + " .set('spark.driver.maxResultSize', '4g')\\\n", + " .set('spark.executor.extraJavaOptions',\\\n", + " f'-XX:+UseParallelOldGC -XX:ParallelGCThreads=2 -XX:NewRatio=1 -XX:SurvivorRatio=1 -XX:+UseCompressedOops -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:ErrorFile={home}/logs/java/hs_err_pid%p.log')\\\n", + " .set('spark.driver.extraClassPath', extra_jars) \\\n", + " .set('spark.executor.extraClassPath', extra_jars) \\\n", + " .set('spark.executorEnv.PYTHONPATH',f\"{os.environ['SPARK_HOME']}python:{get_py4jzip()}\") \\\n", + " .set(\"spark.repl.class.outputDir\", output_dir) \\\n", + " .set(\"spark.sql.broadcastTimeout\", \"4800\") \\\n", + " .set('spark.serializer','org.apache.spark.serializer.KryoSerializer')\\\n", + " .set('spark.kryoserializer.buffer.max','512m')\\\n", + " .set('spark.kryo.unsafe',False)\\\n", + " .set('spark.sql.adaptive.enabled',True)\\\n", + " .set('spark.sql.autoBroadcastJoinThreshold',\"10m\")\\\n", + " .set('spark.sql.catalogImplementation','hive')\\\n", + " .set('spark.sql.optimizer.dynamicPartitionPruning.enabled',True)\\\n", + " .set('spark.cleaner.periodicGC.interval', '10s')\n", + "\n", + " return conf\n", + "\n", + "\n", + "def create_cntx_with_config(conf,conf_overwrite=None):\n", + "\n", + " importlib.reload(pyspark.java_gateway)\n", + "\n", + " def Popen(*args, **kwargs):\n", + " \"\"\"Wraps subprocess.Popen to force stdout and stderr from the child process\n", + " to pipe to this process without buffering.\n", + " \"\"\"\n", + " global spark_jvm_proc\n", + " # Override these in kwargs to avoid duplicate value errors\n", + " # Set streams to unbuffered so that we read whatever bytes are available\n", + " # when ready, https://docs.python.org/3.6/library/subprocess.html#popen-constructor\n", + " kwargs['bufsize'] = 0\n", + " # Capture everything from stdout for display in the notebook\n", + " kwargs['stdout'] = subprocess.PIPE\n", + " print(\"java proc gateway popen\")\n", + " spark_jvm_proc = subprocess.Popen(*args, **kwargs)\n", + " return spark_jvm_proc\n", + " pyspark.java_gateway.Popen = Popen\n", + "\n", + " spylon_kernel.scala_interpreter.scala_intp=None\n", + " \n", + " if conf_overwrite is not None:\n", + " conf=conf_overwrite(conf)\n", + " print(\"spark.serializer: \",conf.get(\"spark.serializer\"))\n", + " print(\"master: \",conf.get(\"spark.master\"))\n", + " \n", + " sc = SparkContext(conf = conf,master=conf.get(\"spark.master\"))\n", + " sc.setLogLevel('ERROR')\n", + " \n", + " sc.addPyFile(f\"{os.environ['SPARK_HOME']}/python/lib/pyspark.zip\")\n", + " sc.addPyFile(get_py4jzip())\n", + " \n", + " spark = SQLContext(sc)\n", + " \n", + " time.sleep(30)\n", + " \n", + " for client in clients:\n", + " pids=!ssh $client \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f1\"\n", + " print(client,\":\",len(pids),\" \",\"\\t\".join(map(str,pids)))\n", + " \n", + " spark_session = SparkSession(sc)\n", + " spark_jvm_helpers = SparkJVMHelpers(spark_session._sc)\n", + " spylon_kernel.scala_interpreter.spark_state = spylon_kernel.scala_interpreter.SparkState(spark_session, spark_jvm_helpers, spark_jvm_proc)\n", + " \n", + " print(\"appid: \",sc.applicationId)\n", + " print(\"SparkConf:\")\n", + "\n", + " df = pd.DataFrame(sc.getConf().getAll(), columns=['key', 'value'])\n", + " display(df)\n", + "\n", + " return sc, spark" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Vanilla Spark" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def vanilla_tpch_conf_overwrite(conf):\n", + " return conf\n", + "\n", + "def vanilla_tpcds_conf_overwrite(conf):\n", + " conf.set('spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold', '0')\\\n", + " .set('spark.sql.optimizer.runtime.bloomFilter.enabled', 'true')\n", + " return conf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx_vanilla(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name='', master='yarn', conf_overwrite=None):\n", + " conf = default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, run_gluten=False)\n", + " conf.set(\"spark.sql.execution.arrow.maxRecordsPerBatch\",20480)\\\n", + " .set(\"spark.sql.parquet.columnarReaderBatchSize\",20480)\\\n", + " .set(\"spark.sql.inMemoryColumnarStorage.batchSize\",20480)\n", + " return create_cntx_with_config(conf,conf_overwrite)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Gluten" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def gluten_tpch_conf_overwrite(conf):\n", + " return conf\n", + "\n", + "def gluten_tpcds_conf_overwrite(conf):\n", + " conf.set('spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold', '0')\\\n", + " .set('spark.sql.optimizer.runtime.bloomFilter.enabled', 'true')\\\n", + " .set('spark.gluten.sql.columnar.joinOptimizationLevel', '18')\\\n", + " .set('spark.gluten.sql.columnar.physicalJoinOptimizeEnable', 'true')\\\n", + " .set('spark.gluten.sql.columnar.physicalJoinOptimizationLevel', '18')\\\n", + " .set('spark.gluten.sql.columnar.logicalJoinOptimizeEnable', 'true')\\\n", + " .set('spark.gluten.sql.columnar.logicalJoinOptimizationLevel', '19')\n", + " return conf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx_gluten(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name='', master='yarn', conf_overwrite=None):\n", + " conf = default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, run_gluten=True)\n", + " conf.set('spark.sql.files.maxPartitionBytes', '4g')\\\n", + " .set('spark.plugins','org.apache.gluten.GlutenPlugin')\\\n", + " .set('spark.shuffle.manager','org.apache.spark.shuffle.sort.ColumnarShuffleManager')\\\n", + " .set('spark.gluten.sql.columnar.backend.lib','velox')\\\n", + " .set('spark.gluten.sql.columnar.maxBatchSize',4096)\\\n", + " .set('spark.gluten.sql.columnar.forceshuffledhashjoin',True)\\\n", + " .set('spark.executorEnv.LD_PRELOAD', findjemalloc())\\\n", + " .set('spark.gluten.sql.columnar.coalesce.batches', 'true')\n", + " \n", + " return create_cntx_with_config(conf,conf_overwrite)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Create Context" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx(run_gluten=False, workload='tpch', app_conf_overwrite=None, server='', base_dir='', nb_name='tpc_workload.ipynb', app_name=''):\n", + " table_dir=''\n", + " extra_jars = ''\n", + " is_tpch_workload=False\n", + " is_tpcds_workload=False\n", + " workload_conf_overwrite=None\n", + " create_cntx_func=None\n", + " test_tpc=None\n", + "\n", + " if workload.lower() == 'tpch':\n", + " if not app_name:\n", + " app_name = 'tpch_power'\n", + " tabledir = tpch_tabledir\n", + " is_tpch_workload=True\n", + " elif workload.lower() == 'tpcds':\n", + " if not app_name:\n", + " app_name = 'tpcds_power'\n", + " tabledir = tpcds_tabledir\n", + " is_tpcds_workload=True\n", + " else:\n", + " raise ValueError(f\"Unknown workload: {workload}\")\n", + "\n", + " lastgit=!git --git-dir {gluten_home}/.git log --format=\"%H\" -n 1\n", + " lastgit = lastgit[0]\n", + " print(f'lastgit: {lastgit}')\n", + "\n", + " nodes=len(clients)\n", + "\n", + " if run_gluten:\n", + " jars_base=f\"{home}/jars/\"+lastgit\n", + " \n", + " for target_jar in gluten_target_jar.split(\",\"):\n", + " !ls -l {target_jar}\n", + " !mkdir -p {jars_base}\n", + " !rm -rf {jars_base}/*\n", + " !cp {target_jar} {jars_base}/\n", + " if target_jar[-4:] != '.jar':\n", + " !cp -f {target_jar} {jars_base}/gluten-{lastgit}.jar\n", + "\n", + " jars=!ls -d {jars_base}/*.jar\n", + " extra_jars=\":\".join([\"file://\"+j for j in jars])\n", + " print(f'extra_jars: {extra_jars}')\n", + "\n", + " for c in clients:\n", + " if c!=localhost:\n", + " !ssh {c} \"rm -rf {jars_base}\"\n", + " !ssh {c} \"mkdir -p {jars_base}\"\n", + " !scp {jars_base}/*.jar {c}:{jars_base} >/dev/null 2>&1\n", + "\n", + " app_name = ' '.join(['gluten', app_name, lastgit[:6]])\n", + " create_cntx_func=create_cntx_gluten\n", + " if is_tpch_workload:\n", + " task_per_core = gluten_tpch_task_per_core\n", + " workload_conf_overwrite = gluten_tpch_conf_overwrite\n", + " elif is_tpcds_workload:\n", + " task_per_core = gluten_tpcds_task_per_core\n", + " workload_conf_overwrite = gluten_tpcds_conf_overwrite\n", + " else:\n", + " app_name = ' '.join(['vanilla', app_name, lastgit[:6]])\n", + " create_cntx_func=create_cntx_vanilla\n", + " if is_tpch_workload:\n", + " task_per_core = vanilla_tpch_task_per_core\n", + " workload_conf_overwrite = vanilla_tpch_conf_overwrite\n", + " elif is_tpcds_workload:\n", + " task_per_core = vanilla_tpcds_task_per_core\n", + " workload_conf_overwrite = vanilla_tpcds_conf_overwrite\n", + " \n", + " conf_overwrite = lambda conf: app_conf_overwrite(workload_conf_overwrite(conf))\n", + " \n", + " sc, spark = create_cntx_func(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, conf_overwrite)\n", + " \n", + " # Pin executors to numa nodes for Gluten\n", + " if run_gluten:\n", + " pinexecutor_numa(clients)\n", + "\n", + " appid = sc.applicationId\n", + " print(\"start run: \", appid)\n", + " \n", + " if is_tpch_workload:\n", + " test_tpc = TestTPCH(spark, tabledir, run_gluten, server, base_dir, nb_name)\n", + " elif is_tpcds_workload:\n", + " test_tpc = TestTPCDS(spark, tabledir, run_gluten, server, base_dir, nb_name)\n", + " \n", + " return sc, spark, appid, test_tpc" + ] + } + ], + "metadata": { + "hide_input": false, + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.12" + }, + "nbTranslate": { + "displayLangs": [ + "*" + ], + "hotkey": "alt-t", + "langInMainMenu": true, + "sourceLang": "en", + "targetLang": "fr", + "useGoogleTranslate": true + }, + "toc": { + "base_numbering": 1, + "nav_menu": {}, + "number_sections": true, + "sideBar": false, + "skip_h1_title": false, + "title_cell": "Table of Contents", + "title_sidebar": "Contents", + "toc_cell": false, + "toc_position": { + "height": "364.469px", + "left": "2086.8px", + "top": "150.516px", + "width": "375px" + }, + "toc_section_display": true, + "toc_window_display": true + }, + "toc-autonumbering": true, + "varInspector": { + "cols": { + "lenName": 16, + "lenType": 16, + "lenVar": 40 + }, + "kernels_config": { + "python": { + "delete_cmd_postfix": "", + "delete_cmd_prefix": "del ", + "library": "var_list.py", + "varRefreshCmd": "print(var_dic_list())" + }, + "r": { + "delete_cmd_postfix": ") ", + "delete_cmd_prefix": "rm(", + "library": "var_list.r", + "varRefreshCmd": "cat(var_dic_list()) " + } + }, + "types_to_exclude": [ + "module", + "function", + "builtin_function_or_method", + "instance", + "_Feature" + ], + "window_display": false + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/tools/workload/benchmark_velox/params.yaml.template b/tools/workload/benchmark_velox/params.yaml.template new file mode 100644 index 000000000000..1c70e428bc99 --- /dev/null +++ b/tools/workload/benchmark_velox/params.yaml.template @@ -0,0 +1,71 @@ +# Local path to gluten project. +gluten_home: /home/sparkuser/gluten + +# Local path to gluten jar. +gluten_target_jar: /home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar + +# Spark app master. +master: yarn + +# List of workers. +clients: + - localhost + +# List of block devices. +disk_dev: + - nvme1n1 + - nvme2n1 + +# List of network devices. +nic_dev: + - ens787f0 + +# Hostname or IP to server for perf analysis. Able to connect via ssh. +server: '' + +# Specify the directory on perf analysis server. Usually a codename for this run. +base_dir: emr + +# Proxy used to connect to server for perf analysis. +proxy: '' + +# Whether to upload profile to perf analysis server and run perf analysis scripts. Only takes effect if server is set. +analyze_perf: True + +# Select workload. Can be either 'tpch' or 'tpcds'. +workload: tpch + +# Run with gluten. If False, run vanilla Spark. +run_gluten: True + +# TPC tables +tpch_tabledir: /tpch_sf3000 +tpcds_tabledir: /tpcds_sf3000 + +# Parallelism +executors_per_node: 32 +cores_per_executor: 8 + +gluten_tpch_task_per_core: 2 +gluten_tpcds_task_per_core: 2 +vanilla_tpch_task_per_core: 4 +vanilla_tpcds_task_per_core: 4 + +# Physical memory on each worker node. +memory_per_node: 1000g + +# Offheap ratio. 0 to disable offheap for vanilla Spark. +# onheap:offheap = 1:2 +vanilla_offheap_ratio: 2.0 +# onheap:offheap = 1:7 +gluten_offheap_ratio: 7.0 + +# spark.io.compression.codec +vanilla_codec: lz4 +# spark.gluten.sql.columnar.shuffle.codec +gluten_codec: lz4 +# spark.gluten.sql.columnar.shuffle.codecBackend +gluten_codec_backend: '' +# spark.gluten.sql.columnar.maxBatchSize +max_batch_size: 4096 + diff --git a/tools/workload/benchmark_velox/run_tpc_workload.sh b/tools/workload/benchmark_velox/run_tpc_workload.sh new file mode 100755 index 000000000000..f6de6ff0f538 --- /dev/null +++ b/tools/workload/benchmark_velox/run_tpc_workload.sh @@ -0,0 +1,86 @@ +#! /bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eu + +PAPERMILL_ARGS=() +OUTPUT_DIR=$PWD + +while [[ $# -gt 0 ]]; do + case $1 in + --notebook) + NOTEBOOK="$2" + shift # past argument + shift # past value + ;; + --output-dir) + OUTPUT_DIR="$2" + shift # past argument + shift # past value + ;; + --output-name) + OUTPUT_NAME="$2" + shift # past argument + shift # past value + ;; + *) + PAPERMILL_ARGS+=("$1") # save positional arg + shift # past argument + ;; + esac +done + +if [ -z ${NOTEBOOK+x} ]; then + echo "Usage: $0 --notebook NOTEBOOK [--output-dir OUTPUT_DIR] [--output-name OUTPUT_NAME] [PAPERMILL_ARGS]" + exit 0 +fi + + +BASEDIR=$(dirname $0) +echo "Script location: ${BASEDIR}" + +nbname=$(basename $NOTEBOOK .ipynb) + +if [ -z ${OUTPUT_NAME+x} ]; then output_name=$nbname; else output_name=$(basename $OUTPUT_NAME .ipynb); fi + +output_dir=$(realpath $OUTPUT_DIR) +mkdir -p $output_dir + +rename_append_appid() { + output_name=$1 + orig_nb=$2 + + output_appid=`grep "appid: " $orig_nb | awk -F' ' '{print $2}' | sed 's/....$//'` + if [ -n "$output_appid" ]; + then + rename_nb=${output_dir}/${output_name}-${output_appid}.ipynb + echo "Rename notebook $orig_nb to $rename_nb" + mv $orig_nb $rename_nb + fi +} + +run() { + output_name=${output_name}-$(date +"%H%M%S") + output_nb=${output_dir}/${output_name}.ipynb + papermill --inject-output-path $NOTEBOOK \ + ${PAPERMILL_ARGS[@]} \ + $output_nb + rename_append_appid $output_name $output_nb +} + +run + diff --git a/tools/workload/benchmark_velox/tpc_workload.ipynb b/tools/workload/benchmark_velox/tpc_workload.ipynb new file mode 100644 index 000000000000..5dcb50a8a066 --- /dev/null +++ b/tools/workload/benchmark_velox/tpc_workload.ipynb @@ -0,0 +1,381 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# initialization" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "tags": [ + "parameters" + ] + }, + "outputs": [], + "source": [ + "# Local path to gluten project.\n", + "gluten_home='/home/sparkuser/gluten'\n", + "\n", + "# Local path to gluten jar.\n", + "gluten_target_jar='/home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar'\n", + "\n", + "# Spark app master. e.g. 'yarn'\n", + "master='yarn'\n", + "\n", + "# List of workers.\n", + "clients=['localhost']\n", + "\n", + "# List of block devices. e.g. ['nvme1n1', 'nvme2n1']\n", + "disk_dev=[]\n", + "\n", + "# List of network devices. e.g. ['ens787f0']\n", + "nic_dev=[]\n", + "\n", + "# Hostname or IP to server for perf analysis. Able to connect via ssh.\n", + "server=''\n", + "\n", + "# Specify the directory on perf analysis server. Usually a codename for this run.\n", + "base_dir=''\n", + "\n", + "# Proxy used to connect to server for perf analysis.\n", + "proxy=''\n", + "\n", + "# Whether to upload profile to perf analysis server and run perf analysis scripts. Only takes effect if server is set.\n", + "analyze_perf=True\n", + "\n", + "# Select workload. Can be either 'tpch' or 'tpcds'.\n", + "workload='tpch'\n", + "\n", + "# Run with gluten. If False, run vanilla Spark.\n", + "run_gluten=True\n", + "\n", + "# TPC tables\n", + "tpch_tabledir=''\n", + "tpcds_tabledir=''\n", + "\n", + "# Parallelism\n", + "executors_per_node=32\n", + "cores_per_executor=8\n", + "\n", + "gluten_tpch_task_per_core=2\n", + "gluten_tpcds_task_per_core=4\n", + "vanilla_tpch_task_per_core=8\n", + "vanilla_tpcds_task_per_core=8\n", + "\n", + "# Physical memory on each worker node.\n", + "memory_per_node='1000g'\n", + "\n", + "# Offheap ratio. 0 to disable offheap for vanilla Spark.\n", + "# onheap:offheap = 1:2\n", + "vanilla_offheap_ratio=2.0\n", + "# onheap:offheap = 1:7\n", + "gluten_offheap_ratio=7.0\n", + "\n", + "# spark.io.compression.codec\n", + "vanilla_codec='lz4'\n", + "# spark.gluten.sql.columnar.shuffle.codec\n", + "gluten_codec='lz4'\n", + "# spark.gluten.sql.columnar.shuffle.codecBackend\n", + "gluten_codec_backend=''\n", + "# spark.gluten.sql.columnar.maxBatchSize\n", + "max_batch_size=4096" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "initialize_ipynb = !realpath native_sql_initialize.ipynb\n", + "print(f\"Running notebook: {initialize_ipynb[0]}\\n\")\n", + "%run {initialize_ipynb[0]}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "newClients = []\n", + "for l in clients:\n", + " if l == 'localhost':\n", + " newClients.append(localhost)\n", + " else:\n", + " newClients.append(l)\n", + "clients = newClients\n", + "\n", + "if server == 'localhost':\n", + " server = localhost" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%javascript\n", + "IPython.notebook.kernel.execute('nb_name = \"' + IPython.notebook.notebook_name + '\"')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "nb_name=PAPERMILL_OUTPUT_PATH" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Application Level Configuration" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "tpch_workload=False\n", + "tpcds_workload=False\n", + "\n", + "if workload.lower() == 'tpch':\n", + " tpch_workload=True\n", + "elif workload.lower() == 'tpcds':\n", + " tpcds_workload=True\n", + "else:\n", + " raise ValueError(f\"Unknown workload: {workload}\")\n", + "\n", + "def gluten_conf_overwrite(conf):\n", + " conf.set('spark.gluten.sql.columnar.shuffle.codec', gluten_codec)\\\n", + " .set('spark.gluten.sql.columnar.shuffle.codecBackend', gluten_codec_backend)\\\n", + " .set('spark.gluten.sql.columnar.maxBatchSize', max_batch_size)\\\n", + " .set('spark.executor.extraJavaOptions',\\\n", + " '-XX:+UseParallelOldGC -XX:ParallelGCThreads=2 -XX:NewRatio=1 -XX:SurvivorRatio=1 -XX:+UseCompressedOops -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:ErrorFile=/home/sparkuser/logs/java/hs_err_pid%p.log')\\\n", + " .set('spark.gluten.memory.overAcquiredMemoryRatio','0')\\\n", + "\n", + " if tpch_workload:\n", + " pass\n", + " elif tpcds_workload:\n", + " pass\n", + " return conf\n", + "\n", + "def vanilla_conf_overwrite(conf):\n", + " conf.set('spark.io.compression.codec', vanilla_codec)\\\n", + " .set('spark.executorEnv.LD_LIBRARY_PATH',f\"{os.getenv('HADOOP_HOME')}/lib/native/\") \\\n", + " .set('spark.yarn.appMasterEnv.LD_LIBRARY_PATH',f\"{os.getenv('HADOOP_HOME')}/lib/native/\") \\\n", + "\n", + " if tpch_workload:\n", + " pass\n", + " elif tpcds_workload:\n", + " pass\n", + " return conf\n", + "\n", + "def app_conf_overwrite(conf):\n", + " if run_gluten:\n", + " return gluten_conf_overwrite(conf)\n", + " return vanilla_conf_overwrite(conf)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Run Workload" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Config and clean pagecache before each run\n", + "config_pagecache(clients, run_gluten)\n", + "dropcache(clients)\n", + "print_kernel_params(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Create SparkSession\n", + "sc, spark, appid, test_tpc=create_cntx(run_gluten, workload, app_conf_overwrite, server, base_dir, nb_name)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "if run_gluten:\n", + " config_mem_cgroup(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.start_monitor(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.power_run(explain=False, print_result=False, load_table=True)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.stop_monitor(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "if analyze_perf:\n", + " test_tpc.run_perf_analysis(disk_dev, nic_dev)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Show Performance" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.print_result()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "for client in clients:\n", + " draw_sar(appid, qtime=test_tpc.result, disk_dev=disk_dev, nic_dev=nic_dev, client=client)" + ] + } + ], + "metadata": { + "celltoolbar": "Tags", + "hide_input": false, + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.12" + }, + "nbTranslate": { + "displayLangs": [ + "*" + ], + "hotkey": "alt-t", + "langInMainMenu": true, + "sourceLang": "en", + "targetLang": "fr", + "useGoogleTranslate": true + }, + "toc": { + "base_numbering": 1, + "nav_menu": {}, + "number_sections": true, + "sideBar": false, + "skip_h1_title": false, + "title_cell": "Table of Contents", + "title_sidebar": "Contents", + "toc_cell": false, + "toc_position": { + "height": "428.672px", + "left": "1339.91px", + "top": "374.297px", + "width": "456.969px" + }, + "toc_section_display": true, + "toc_window_display": true + }, + "varInspector": { + "cols": { + "lenName": 16, + "lenType": 16, + "lenVar": 40 + }, + "kernels_config": { + "python": { + "delete_cmd_postfix": "", + "delete_cmd_prefix": "del ", + "library": "var_list.py", + "varRefreshCmd": "print(var_dic_list())" + }, + "r": { + "delete_cmd_postfix": ") ", + "delete_cmd_prefix": "rm(", + "library": "var_list.r", + "varRefreshCmd": "cat(var_dic_list()) " + } + }, + "types_to_exclude": [ + "module", + "function", + "builtin_function_or_method", + "instance", + "_Feature" + ], + "window_display": false + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh index 8f62560dc168..60f10f094d5b 100644 --- a/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh +++ b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -batchsize=10240 SPARK_HOME=/home/sparkuser/spark/ spark_sql_perf_jar=/PATH/TO/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar cat tpcds_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ @@ -22,14 +21,13 @@ cat tpcds_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ --executor-memory 25g \ --executor-cores 8 \ --master yarn \ - --driver-memory 50g \ + --driver-memory 10g \ --deploy-mode client \ --conf spark.executor.memoryOverhead=1g \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ --conf spark.sql.broadcastTimeout=4800 \ --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.sources.useV1SourceList=avro \ --conf spark.sql.shuffle.partitions=224 \ + --conf spark.sql.parquet.compression.codec=snappy \ + --conf spark.network.timeout=800s \ + --conf spark.executor.heartbeatInterval=200s \ --jars ${spark_sql_perf_jar} diff --git a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala deleted file mode 100644 index 958a98f57020..000000000000 --- a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -import org.apache.spark.sql.execution.debug._ -import scala.io.Source -import java.io.File -import java.util.Arrays -import org.apache.spark.sql.types.{DoubleType, TimestampType, LongType, IntegerType} - -val parquet_file_path = "/PATH/TO/TPCH_PARQUET_PATH" -val dwrf_file_path = "/PATH/TO/TPCH_DWRF_PATH" - -val lineitem_parquet_path = "file://" + parquet_file_path + "/lineitem" -val customer_parquet_path = "file://" + parquet_file_path + "/customer" -val nation_parquet_path = "file://" + parquet_file_path + "/nation" -val orders_parquet_path = "file://" + parquet_file_path + "/orders" -val part_parquet_path = "file://" + parquet_file_path + "/part" -val partsupp_parquet_path = "file://" + parquet_file_path + "/partsupp" -val region_path_path = "file://" + parquet_file_path + "/region" -val supplier_parquet_path = "file://" + parquet_file_path + "/supplier" - -val lineitem = spark.read.format("parquet").load(lineitem_parquet_path) -val customer = spark.read.format("parquet").load(customer_parquet_path) -val nation = spark.read.format("parquet").load(nation_parquet_path) -val orders = spark.read.format("parquet").load(orders_parquet_path) -val part = spark.read.format("parquet").load(part_parquet_path) -val partsupp = spark.read.format("parquet").load(partsupp_parquet_path) -val region = spark.read.format("parquet").load(region_path_path) -val supplier = spark.read.format("parquet").load(supplier_parquet_path) - -val lineitem_dwrf_path = "file://" + dwrf_file_path + "/lineitem" -val customer_dwrf_path = "file://" + dwrf_file_path + "/customer" -val nation_dwrf_path = "file://" + dwrf_file_path + "/nation" -val orders_dwrf_path = "file://" + dwrf_file_path + "/orders" -val part_dwrf_path = "file://" + dwrf_file_path + "/part" -val partsupp_dwrf_path = "file://" + dwrf_file_path + "/partsupp" -val region_dwrf_path = "file://" + dwrf_file_path + "/region" -val supplier_dwrf_path = "file://" + dwrf_file_path + "/supplier" - -lineitem.write.mode("append").format("dwrf").save(lineitem_dwrf_path) -customer.write.mode("append").format("dwrf").save(customer_dwrf_path) -nation.write.mode("append").format("dwrf").save(nation_dwrf_path) -orders.write.mode("append").format("dwrf").save(orders_dwrf_path) -part.write.mode("append").format("dwrf").save(part_dwrf_path) -partsupp.write.mode("append").format("dwrf").save(partsupp_dwrf_path) -region.write.mode("append").format("dwrf").save(region_dwrf_path) -supplier.write.mode("append").format("dwrf").save(supplier_dwrf_path) - - diff --git a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh deleted file mode 100644 index 2dc05c9aa7dd..000000000000 --- a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh +++ /dev/null @@ -1,47 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -batchsize=20480 - -export GLUTEN_HOME=/PATH/TO/gluten/ -#please choose right os system jar -export GLUTEN_JVM_JAR=${GLUTEN_HOME}/package/target/ -SPARK_HOME=/home/sparkuser/spark/ - -cat tpch_convert_parquet_dwrf.scala | ${SPARK_HOME}/bin/spark-shell \ - --name convert_parquet_dwrf \ - --master yarn \ - --deploy-mode client \ - --driver-memory 20g \ - --executor-cores 8 \ - --num-executors 14 \ - --executor-memory 30g \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.driver.extraClassPath=${GLUTEN_JVM_JAR} \ - --conf spark.executor.extraClassPath=${GLUTEN_JVM_JAR} \ - --conf spark.memory.offHeap.size=30g \ - --conf spark.executor.memoryOverhead=5g \ - --conf spark.driver.maxResultSize=32g \ - --conf spark.sql.autoBroadcastJoinThreshold=-1 \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ - --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ - --conf spark.sql.broadcastTimeout=4800 \ - --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.adaptive.enabled=true \ - --conf spark.sql.shuffle.partitions=112 \ - --conf spark.sql.sources.useV1SourceList=avro \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --conf spark.sql.files.maxPartitionBytes=1073741824 \ diff --git a/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh index 29512ed80b15..8db9d443331d 100644 --- a/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh +++ b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -batchsize=10240 SPARK_HOME=/home/sparkuser/spark/ spark_sql_perf_jar=/PATH/TO/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar cat tpch_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ @@ -25,11 +24,10 @@ cat tpch_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ --driver-memory 50g \ --deploy-mode client \ --conf spark.executor.memoryOverhead=1g \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ --conf spark.sql.broadcastTimeout=4800 \ --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.sources.useV1SourceList=avro \ --conf spark.sql.shuffle.partitions=224 \ + --conf spark.sql.parquet.compression.codec=snappy \ + --conf spark.network.timeout=800s \ + --conf spark.executor.heartbeatInterval=200s \ --jars ${spark_sql_perf_jar} From 59d52486b189336ee3fd31a9a8fe98e866d5805a Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 14 Nov 2024 10:31:29 +0800 Subject: [PATCH 089/211] remove duplicated columns in agg results (#7937) --- .../org/apache/gluten/extension/LazyAggregateExpandRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala index d21cbfa2505a..d64300a12fcb 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -353,7 +353,7 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan s"resultExpressions:${partialAggregate.resultExpressions}") partialAggregate.copy( groupingExpressions = groupingExpressions, - resultExpressions = resultExpressions, + resultExpressions = resultExpressions.distinct, child = expand.child) } From 73c06100e454723546b396b02110504b6c76c61e Mon Sep 17 00:00:00 2001 From: Xiuli Wei Date: Thu, 14 Nov 2024 12:27:55 +0800 Subject: [PATCH 090/211] [GLUTEN-7856][CORE] Ensure correct enabling of GlutenCostEvaluator (#7857) Closes #7856 --- .../src/main/scala/org/apache/gluten/GlutenPlugin.scala | 9 ++++++--- .../src/main/scala/org/apache/gluten/GlutenConfig.scala | 9 ++++----- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala index d5f5fd2e2150..e52c53d1d558 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala @@ -65,7 +65,7 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { postBuildInfoEvent(sc) - setPredefinedConfigs(sc, conf) + setPredefinedConfigs(conf) // Initialize Backend. Backend.get().onDriverStart(sc, pluginContext) @@ -123,7 +123,7 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { GlutenEventUtils.post(sc, event) } - private def setPredefinedConfigs(sc: SparkContext, conf: SparkConf): Unit = { + private def setPredefinedConfigs(conf: SparkConf): Unit = { // Spark SQL extensions val extensions = if (conf.contains(SPARK_SESSION_EXTENSIONS.key)) { s"${conf.get(SPARK_SESSION_EXTENSIONS.key)}," + @@ -134,7 +134,10 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { conf.set(SPARK_SESSION_EXTENSIONS.key, extensions) // adaptive custom cost evaluator class - if (GlutenConfig.getConf.enableGluten && GlutenConfig.getConf.enableGlutenCostEvaluator) { + val enableGlutenCostEvaluator = conf.getBoolean( + GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED, + GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE) + if (enableGlutenCostEvaluator) { val costEvaluator = "org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator" conf.set(SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key, costEvaluator) } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 0bfe16270e5f..0e41fc65376c 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -466,8 +466,6 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableCastAvgAggregateFunction: Boolean = conf.getConf(COLUMNAR_NATIVE_CAST_AGGREGATE_ENABLED) - def enableGlutenCostEvaluator: Boolean = conf.getConf(COST_EVALUATOR_ENABLED) - def dynamicOffHeapSizingEnabled: Boolean = conf.getConf(DYNAMIC_OFFHEAP_SIZING_ENABLED) @@ -648,6 +646,7 @@ object GlutenConfig { "spark.gluten.memory.dynamic.offHeap.sizing.memory.fraction" val GLUTEN_COST_EVALUATOR_ENABLED = "spark.gluten.sql.adaptive.costEvaluator.enabled" + val GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE = true var ins: GlutenConfig = _ @@ -2116,15 +2115,15 @@ object GlutenConfig { .createWithDefault(true) val COST_EVALUATOR_ENABLED = - buildConf(GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED) + buildStaticConf(GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED) .internal() .doc( - "If true and gluten enabled, use " + + "If true, use " + "org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator as custom cost " + "evaluator class, else follow the configuration " + "spark.sql.adaptive.customCostEvaluatorClass.") .booleanConf - .createWithDefault(true) + .createWithDefault(GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE) val DYNAMIC_OFFHEAP_SIZING_ENABLED = buildConf(GlutenConfig.GLUTEN_DYNAMIC_OFFHEAP_SIZING_ENABLED) From fc549eee511304e3dbe02a5295c4016b827c4f10 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Thu, 14 Nov 2024 13:30:25 +0800 Subject: [PATCH 091/211] [VL] Fix wrong lib suffix for google_cloud_cpp_storage (#7933) --- cpp/velox/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 329da49497f3..3ed7574d1e4c 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -138,8 +138,8 @@ endmacro() macro(find_gcssdk) set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) - set(CMAKE_FIND_LIBRARY_SUFFIXES ".so") - find_package(google_cloud_cpp_storage REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") + find_package(google_cloud_cpp_storage CONFIG 2.22.0 REQUIRED) set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() From 20136c86acb3e63bf7834e1c1a636db26c92bd88 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Thu, 14 Nov 2024 15:21:22 +0800 Subject: [PATCH 092/211] [VL] Add test for scan operator with filter on decimal/timestamp/binary field (#7945) --- .../gluten/execution/VeloxScanSuite.scala | 36 +++++++++++++++++++ .../RandomParquetDataGenerator.scala | 6 ++-- 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala index a50bffa3edcd..a4f16ecc3c8f 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala @@ -18,11 +18,13 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.velox.VeloxBackendSettings +import org.apache.gluten.benchmarks.RandomParquetDataGenerator import org.apache.gluten.utils.VeloxFileSystemValidationJniWrapper import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.GreaterThan import org.apache.spark.sql.execution.ScalarSubquery +import org.apache.spark.sql.types._ class VeloxScanSuite extends VeloxWholeStageTransformerSuite { protected val rootPath: String = getClass.getResource("/").getPath @@ -114,4 +116,38 @@ class VeloxScanSuite extends VeloxWholeStageTransformerSuite { !VeloxFileSystemValidationJniWrapper.allSupportedByRegisteredFileSystems( Array("file:/test_path/", "unsupported://test_path"))) } + + test("scan with filter on decimal/timestamp/binary field") { + withTempView("t") { + withTempDir { + dir => + val path = dir.getAbsolutePath + val schema = StructType( + Array( + StructField("short_decimal_field", DecimalType(5, 2), nullable = true), + StructField("long_decimal_field", DecimalType(32, 8), nullable = true), + StructField("binary_field", BinaryType, nullable = true), + StructField("timestamp_field", TimestampType, nullable = true) + )) + RandomParquetDataGenerator(0).generateRandomData(spark, schema, 10, Some(path)) + spark.catalog.createTable("t", path, "parquet") + + runQueryAndCompare( + """select * from t where long_decimal_field = 3.14""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where short_decimal_field = 3.14""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where binary_field = '3.14'""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where timestamp_field = current_timestamp()""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + } + } + } } diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala b/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala index a27a4991cf4a..ce2f85af7781 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.types._ import com.github.javafaker.Faker -import java.sql.Date +import java.sql.{Date, Timestamp} import java.util.Random case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { @@ -67,7 +67,7 @@ case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { case DoubleType => faker.number().randomDouble(2, Double.MinValue.toLong, Double.MaxValue.toLong) case DateType => new Date(faker.date().birthday().getTime) -// case TimestampType => new Timestamp(faker.date().birthday().getTime) + case TimestampType => new Timestamp(faker.date().birthday().getTime) case t: DecimalType => BigDecimal( faker.number().randomDouble(t.scale, 0, Math.pow(10, t.precision - t.scale).toLong)) @@ -124,7 +124,7 @@ case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { () => StructField(fieldName, FloatType, nullable = true), () => StructField(fieldName, DoubleType, nullable = true), () => StructField(fieldName, DateType, nullable = true), -// () => StructField(fieldName, TimestampType, nullable = true), + () => StructField(fieldName, TimestampType, nullable = true), () => StructField(fieldName, DecimalType(10, 2), nullable = true), () => StructField(fieldName, DecimalType(30, 10), nullable = true) ) From 21b4e6539b5494ee6e7506368dfeca4711b6971a Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Thu, 14 Nov 2024 16:54:59 +0800 Subject: [PATCH 093/211] [GLUTEN-7362][VL] Add test for 'IN' and 'OR' filter in Scan (#7363) --- .../org/apache/gluten/execution/MiscOperatorSuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index 2b42d7fb8d64..58c748d37bfa 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -208,6 +208,13 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa "select l_orderkey from lineitem " + "where l_partkey in (1552, 674) or l_partkey in (1552) and l_orderkey > 1") { _ => } checkLengthAndPlan(df, 73) + + runQueryAndCompare( + "select count(1) from lineitem " + + "where (l_shipmode in ('TRUCK', 'MAIL') or l_shipmode in ('AIR', 'FOB')) " + + "and l_shipmode in ('RAIL','SHIP')") { + checkGlutenOperatorMatch[FileSourceScanExecTransformer] + } } test("in_not") { From f8a2dca340c2c47e2349b1b4304d54ce2a58e6df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 14 Nov 2024 18:44:01 +0800 Subject: [PATCH 094/211] [GLUTEN-7387][CH] Allow parallel downloading in scan operator for hive text/json table when the whole compresse(not bzip2) file is a single file split (#7598) * enable parallel downloading for text/json * wip * wip * finish dev * update version * update initialization of thread pool * fix style --- cpp-ch/clickhouse.version | 2 +- cpp-ch/local-engine/Common/CHUtil.cpp | 17 +++++- .../SubstraitSource/ReadBufferBuilder.cpp | 55 +++++++++++++++++-- .../SubstraitSource/ReadBufferBuilder.h | 7 +++ 4 files changed, 73 insertions(+), 8 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index e51caf2aa788..beda11ec0a99 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence CH_BRANCH=rebase_ch/20241111 -CH_COMMIT=06ffc32a255 +CH_COMMIT=3f7e46d4e9e diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index d253de82938f..778c4f257b19 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -86,6 +87,14 @@ extern const int BAD_ARGUMENTS; extern const int UNKNOWN_TYPE; extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; } + +namespace ServerSetting +{ +extern const ServerSettingsUInt64 max_thread_pool_size; +extern const ServerSettingsUInt64 thread_pool_queue_size; +extern const ServerSettingsUInt64 max_io_thread_pool_size; +extern const ServerSettingsUInt64 io_thread_pool_queue_size; +} } namespace local_engine @@ -757,6 +766,7 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Set settings key:{} value:{}", key, value); } } + /// Finally apply some fixed kvs to settings. settings.set("join_use_nulls", true); settings.set("input_format_orc_allow_missing_columns", true); @@ -970,7 +980,12 @@ void BackendInitializerUtil::initBackend(const SparkConfigs::ConfigMap & spark_c initCompiledExpressionCache(config); LOG_INFO(logger, "Init compiled expressions cache factory."); - GlobalThreadPool::initialize(); + ServerSettings server_settings; + server_settings.loadSettingsFromConfig(*config); + GlobalThreadPool::initialize( + server_settings[ServerSetting::max_thread_pool_size], 0, server_settings[ServerSetting::thread_pool_queue_size]); + getIOThreadPool().initialize( + server_settings[ServerSetting::max_io_thread_pool_size], 0, server_settings[ServerSetting::io_thread_pool_queue_size]); const size_t active_parts_loading_threads = config->getUInt("max_active_parts_loading_thread_pool_size", 64); DB::getActivePartsLoadingThreadPool().initialize( diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index f5f7f95e6202..c1796f3e3cda 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -70,6 +71,9 @@ namespace Setting extern const SettingsUInt64 s3_max_redirects; extern const SettingsBool s3_disable_checksum; extern const SettingsUInt64 s3_retry_attempts; +extern const SettingsMaxThreads max_download_threads; +extern const SettingsUInt64 max_download_buffer_size; +extern const SettingsBool input_format_allow_seeks; } namespace ErrorCodes { @@ -183,7 +187,7 @@ adjustReadRangeIfNeeded(std::unique_ptr read_buffer, const s return std::move(read_buffer); /// Skip text/json files with compression. - /// TODO implement adjustFileReadPosition when compression method is bzip2 + /// When the file is compressed, its read range is adjusted in [[buildWithCompressionWrapper]] Poco::URI file_uri(file_info.uri_file()); DB::CompressionMethod compression = DB::chooseCompressionMethod(file_uri.getPath(), "auto"); if (compression != CompressionMethod::None) @@ -216,6 +220,8 @@ class LocalFileReadBufferBuilder : public ReadBufferBuilder explicit LocalFileReadBufferBuilder(DB::ContextPtr context_) : ReadBufferBuilder(context_) { } ~LocalFileReadBufferBuilder() override = default; + bool isRemote() const override { return false; } + std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) override { @@ -660,10 +666,6 @@ void registerReadBufferBuilders() #endif } -ReadBufferBuilder::ReadBufferBuilder(DB::ContextPtr context_) : context(context_) -{ -} - DB::ReadSettings ReadBufferBuilder::getReadSettings() const { DB::ReadSettings read_settings = context->getReadSettings(); @@ -678,6 +680,10 @@ DB::ReadSettings ReadBufferBuilder::getReadSettings() const return read_settings; } +ReadBufferBuilder::ReadBufferBuilder(DB::ContextPtr context_) : context(context_) +{ +} + std::unique_ptr ReadBufferBuilder::wrapWithBzip2(std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) { @@ -758,7 +764,11 @@ ReadBufferBuilder::buildWithCompressionWrapper(const substrait::ReadRel::LocalFi if (compression == CompressionMethod::Bzip2) return wrapWithBzip2(std::move(in), file_info); else - return wrapReadBufferWithCompressionMethod(std::move(in), compression); + { + /// In this case we are pretty sure that current split covers the whole file because only bzip2 compression is splittable + auto parallel = wrapWithParallelIfNeeded(std::move(in), file_info); + return wrapReadBufferWithCompressionMethod(std::move(parallel), compression); + } } ReadBufferBuilder::ReadBufferCreator ReadBufferBuilder::wrapWithCache( @@ -843,6 +853,39 @@ void ReadBufferBuilder::updateCaches(const String & key, size_t last_modified_ti } } +std::unique_ptr ReadBufferBuilder::wrapWithParallelIfNeeded( + std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) +{ + /// Only use parallel downloading for text and json format because data are read serially in those formats. + if (!file_info.has_text() && !file_info.has_json()) + return std::move(in); + + const auto & settings = context->getSettingsRef(); + auto max_download_threads = settings[DB::Setting::max_download_threads]; + auto max_download_buffer_size = settings[DB::Setting::max_download_buffer_size]; + + bool parallel_read = isRemote() && max_download_threads > 1 && isBufferWithFileSize(*in); + if (!parallel_read) + return std::move(in); + + size_t file_size = getFileSizeFromReadBuffer(*in); + if (file_size < 4 * max_download_buffer_size) + return std::move(in); + + LOG_TRACE( + getLogger("ReadBufferBuilder"), + "Using ParallelReadBuffer with {} workers with chunks of {} bytes", + max_download_threads, + max_download_buffer_size); + + return wrapInParallelReadBufferIfSupported( + {std::move(in)}, + DB::threadPoolCallbackRunnerUnsafe(DB::getIOThreadPool().get(), "ParallelRead"), + max_download_threads, + max_download_buffer_size, + file_size); +} + ReadBufferBuilderFactory & ReadBufferBuilderFactory::instance() { static ReadBufferBuilderFactory instance; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h index f2c1c10a6f2c..e76e79a9b0ba 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -35,6 +36,8 @@ class ReadBufferBuilder virtual ~ReadBufferBuilder() = default; + virtual bool isRemote() const { return true; } + /// build a new read buffer virtual std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) = 0; @@ -55,7 +58,11 @@ class ReadBufferBuilder size_t last_modified_time, size_t file_size); + std::unique_ptr + wrapWithParallelIfNeeded(std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info); + DB::ReadSettings getReadSettings() const; + DB::ContextPtr context; private: From 9a7814d96f1d49d8eb0ec4fe15d1968bdcdcdb18 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 14 Nov 2024 19:14:16 +0800 Subject: [PATCH 095/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_14) (#7942) Upstream Velox's New Commits: 99979c4e7 by Krishna Pai, Back out "Readd: [velox][PR] Add runner for local distributed execution" (11529) 24a36c059 by Pedro Pedreira, docs: Add CONTRIBUTING guidelines for broken CI jobs (11521) beada2b62 by Kevin Wilfong, Fix Presto's date_trunc function when dealing with daylight savings (11509) 08dba0f73 by Deepak Majeti, Add GCS Writer Sink and fix Gcs naming (11328) f34035b03 by zhli1142015, Support decimal type for Spark unaryminus function (11454) a993e053b by duanmeng, Extracts trace node name automatically in trace replayer (11519) 06ab00135 by NEUpanning, Disable aggregate pushdown for decimal type (11298) 6c25a9120 by Krishna Pai, Throw if Aggregation Filter clause has non boolean expression (11524) 8b1717546 by aditi-pandit, refactor: Combine loops in PrestoSerializerTest.validateLexer (11434) 5ab1a8f45 by Minhan Cao, refactor: Remove Parquet thrift code from Arrow writer (11489) 145f1650e by Orri Erling, Readd: [velox][PR] Add runner for local distributed execution (11507) 711494383 by Richard Barnes, Remove unused-variable in velox/dwio/parquet/reader/DeltaByteArrayDecoder.h (11528) a5a2cad2d by David Reveman, Initial import of breeze (10770) 39aa1caf7 by Jimmy Lu, feat(dwio): Delta update support in selective readers (11501) 1f4ccc3c1 by Joe Abraham, Rename variables in Base64.* for readability (11299) 65c49f4b6 by aditi-pandit, Code cleanup in TopNRowNumber (11422) 05dc84155 by Chengcheng Jin, Support long decimal in prefixsort (10385) 117b5df60 by Jialiang Tan, Add fs fault injection in memory arbitrator fuzzer spill (11432) d4bdc3b0e by zhli1142015, Ensure prefix length does not exceed configured maximum (11496) c42320042 by Jialiang Tan, Make arbitration timed lock TSAN compatible (11517) f1ff2dff0 by Pramod, Fix `l_quantity` data generated by Tpch connector (11511) f6276bb6a by Deepak Majeti, doc: Add S3 Bucket config, fix Generic config and Spark-specific config (11418) --- cpp/velox/compute/VeloxBackend.cc | 4 ++-- cpp/velox/operators/writer/VeloxParquetDataSource.h | 2 +- cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h | 2 +- cpp/velox/utils/ConfigExtractor.cc | 8 ++++---- ep/build-velox/src/get_velox.sh | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index 889f979b2869..0d5e1d41a54d 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -41,7 +41,7 @@ #include "velox/connectors/hive/HiveConnector.h" #include "velox/connectors/hive/HiveDataSource.h" #include "velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.h" // @manual -#include "velox/connectors/hive/storage_adapters/gcs/RegisterGCSFileSystem.h" // @manual +#include "velox/connectors/hive/storage_adapters/gcs/RegisterGcsFileSystem.h" // @manual #include "velox/connectors/hive/storage_adapters/hdfs/RegisterHdfsFileSystem.h" // @manual #include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" // @manual #include "velox/dwio/orc/reader/OrcReader.h" @@ -142,7 +142,7 @@ void VeloxBackend::init(const std::unordered_map& conf velox::filesystems::registerS3FileSystem(); #endif #ifdef ENABLE_GCS - velox::filesystems::registerGCSFileSystem(); + velox::filesystems::registerGcsFileSystem(); #endif #ifdef ENABLE_ABFS velox::filesystems::abfs::registerAbfsFileSystem(); diff --git a/cpp/velox/operators/writer/VeloxParquetDataSource.h b/cpp/velox/operators/writer/VeloxParquetDataSource.h index e7428999f02e..a94ab00b9e2a 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSource.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSource.h @@ -37,7 +37,7 @@ #include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" #endif #ifdef ENABLE_GCS -#include "velox/connectors/hive/storage_adapters/gcs/GCSFileSystem.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsFileSystem.h" #endif #ifdef ENABLE_HDFS #include "velox/connectors/hive/storage_adapters/hdfs/HdfsFileSystem.h" diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h index 22d4e96efe7f..579a425d1833 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h @@ -43,7 +43,7 @@ class VeloxParquetDataSourceGCS final : public VeloxParquetDataSource { void initSink(const std::unordered_map& /* sparkConfs */) override { auto fileSystem = filesystems::getFileSystem(filePath_, nullptr); - auto* gcsFileSystem = dynamic_cast(fileSystem.get()); + auto* gcsFileSystem = dynamic_cast(fileSystem.get()); sink_ = std::make_unique( gcsFileSystem->openFileForWrite(filePath_, {{}, sinkPool_.get()}), filePath_); } diff --git a/cpp/velox/utils/ConfigExtractor.cc b/cpp/velox/utils/ConfigExtractor.cc index 3175c375fafc..78366b37568b 100644 --- a/cpp/velox/utils/ConfigExtractor.cc +++ b/cpp/velox/utils/ConfigExtractor.cc @@ -141,7 +141,7 @@ std::shared_ptr getHiveConfig( std::string gcsEndpoint = gsStorageRootUrl.value(); if (!gcsEndpoint.empty()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSEndpoint] = gcsEndpoint; + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsEndpoint] = gcsEndpoint; } } @@ -149,13 +149,13 @@ std::shared_ptr getHiveConfig( // https://cloud.google.com/cpp/docs/reference/storage/latest/classgoogle_1_1cloud_1_1storage_1_1LimitedErrorCountRetryPolicy auto gsMaxRetryCount = conf->get("spark.hadoop.fs.gs.http.max.retry"); if (gsMaxRetryCount.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSMaxRetryCount] = gsMaxRetryCount.value(); + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsMaxRetryCount] = gsMaxRetryCount.value(); } // https://cloud.google.com/cpp/docs/reference/storage/latest/classgoogle_1_1cloud_1_1storage_1_1LimitedTimeRetryPolicy auto gsMaxRetryTime = conf->get("spark.hadoop.fs.gs.http.max.retry-time"); if (gsMaxRetryTime.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSMaxRetryTime] = gsMaxRetryTime.value(); + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsMaxRetryTime] = gsMaxRetryTime.value(); } // https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/CONFIGURATION.md#authentication @@ -166,7 +166,7 @@ std::shared_ptr getHiveConfig( auto gsAuthServiceAccountJsonKeyfile = conf->get("spark.hadoop.fs.gs.auth.service.account.json.keyfile"); if (gsAuthServiceAccountJsonKeyfile.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSCredentialsPath] = + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsCredentialsPath] = gsAuthServiceAccountJsonKeyfile.value(); } else { LOG(WARNING) << "STARTUP: conf spark.hadoop.fs.gs.auth.type is set to SERVICE_ACCOUNT_JSON_KEYFILE, " diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 2be1481fc05d..06a7215d827a 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_13_new +VELOX_BRANCH=2024_11_14 VELOX_HOME="" OS=`uname -s` From 7b4005055408b7cbdc4f5e9bec5b6ac442e52259 Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 14 Nov 2024 20:01:16 +0800 Subject: [PATCH 096/211] [GLUTEN-7647][CH] Drop literals in aggregation results (#7951) --- .../gluten/extension/LazyAggregateExpandRule.scala | 13 +++++++++++-- .../GlutenClickHouseTPCHSaltNullParquetSuite.scala | 8 ++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala index d64300a12fcb..0f7752339707 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -345,8 +345,17 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan s"groupingExpressions: ${partialAggregate.groupingExpressions}") // Remove group id column from result expressions - val resultExpressions = partialAggregate.resultExpressions - .filter(!_.toAttribute.semanticEquals(groupIdAttribute)) + val groupingKeysCount = + partialAggregate.resultExpressions.length - partialAggregate.aggregateExpressions.length + var resultExpressions = partialAggregate.resultExpressions + .slice(0, groupingKeysCount) + .filter( + e => + !e.toAttribute.semanticEquals(groupIdAttribute) && + attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined) + .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) + resultExpressions = resultExpressions ++ partialAggregate.resultExpressions + .slice(groupingKeysCount, partialAggregate.resultExpressions.length) .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) logDebug( s"xxx newResultExpressions: $resultExpressions\n" + diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 9affdeb7f7fc..865d3fa40c0f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -3066,6 +3066,14 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |order by n_regionkey, n_nationkey |""".stripMargin compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select x, n_regionkey, n_nationkey, + |sum(n_regionkey), count(n_name), max(n_regionkey), min(n_regionkey) + |from (select '123' as x, * from nation) group by x, n_regionkey, n_nationkey with cube + |order by x, n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) } test("GLUTEN-7647 lazy expand for avg and sum") { From f65087ece35fc13a56189c450c56075107aad446 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Thu, 14 Nov 2024 23:51:01 +0800 Subject: [PATCH 097/211] [CH] Fix issues due to https://github.com/ClickHouse/ClickHouse/pull/71539. (#7952) * Fix issues due to https://github.com/ClickHouse/ClickHouse/pull/71539. Issue 1 BuildQueryPipelineSettings is created manually instead of calling BuildQueryPipelineSettings::fromContext(); so even https://github.com/ClickHouse/ClickHouse/pull/71890 disable 'query_plan_merge_filters', UTs are still failed. To fix this issue, we need set correct default parameters in CHUtil.cpp Issue 2 If we set query_plan_merge_filters to true, then https://github.com/ClickHouse/ClickHouse/pull/71539 will try to split the left most AND atom to a separate DAG and hence create FilterTransformer for each And atom, which cause collecting metrics failed. I am not sure the benefits of setting it to true, let's keep it to false. * Calling `QueryPlan::explainPlan` after building pipeline is not correct, due to `action_dag` is [moved](https://github.com/ClickHouse/ClickHouse/blob/22d2c856a70dfb8b6e4c506fcb22ac03d59df9be/src/Processors/QueryPlan/FilterStep.cpp#L161). --- cpp-ch/local-engine/Common/CHUtil.cpp | 11 +++++++ cpp-ch/local-engine/Common/DebugUtils.cpp | 16 +++++---- cpp-ch/local-engine/Common/DebugUtils.h | 2 +- .../Parser/SerializedPlanParser.cpp | 33 ++++++++----------- .../Parser/SerializedPlanParser.h | 4 +-- 5 files changed, 37 insertions(+), 29 deletions(-) diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 778c4f257b19..2413fae9e350 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -80,6 +80,9 @@ namespace Setting { extern const SettingsUInt64 prefer_external_sort_block_bytes; extern const SettingsUInt64 max_bytes_before_external_sort; +extern const SettingsBool query_plan_merge_filters; +extern const SettingsBool compile_expressions; +extern const SettingsShortCircuitFunctionEvaluation short_circuit_function_evaluation; } namespace ErrorCodes { @@ -722,6 +725,14 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ settings.set("max_download_threads", 1); settings.set("input_format_parquet_enable_row_group_prefetch", false); + /// update per https://github.com/ClickHouse/ClickHouse/pull/71539 + /// if true, we can't get correct metrics for the query + settings[Setting::query_plan_merge_filters] = false; + /// We now set BuildQueryPipelineSettings according to config. + settings[Setting::compile_expressions] = true; + settings[Setting::short_circuit_function_evaluation] = ShortCircuitFunctionEvaluation::DISABLE; + /// + for (const auto & [key, value] : spark_conf_map) { // Firstly apply spark.gluten.sql.columnar.backend.ch.runtime_config.local_engine.settings.* to settings diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index 2fcab59bf856..8a4323cb1c13 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -38,7 +38,7 @@ namespace pb_util = google::protobuf::util; namespace debug { -void dumpPlan(DB::QueryPlan & plan, bool force, LoggerPtr logger) +void dumpPlan(DB::QueryPlan & plan, const char * type, bool force, LoggerPtr logger) { if (!logger) { @@ -51,10 +51,12 @@ void dumpPlan(DB::QueryPlan & plan, bool force, LoggerPtr logger) return; auto out = local_engine::PlanUtil::explainPlan(plan); + auto task_id = local_engine::QueryContext::instance().currentTaskIdOrEmpty(); + task_id = task_id.empty() ? "" : "(" + task_id + ")"; if (force) // force - LOG_ERROR(logger, "clickhouse plan({}):\n{}", local_engine::QueryContext::instance().currentTaskIdOrEmpty(), out); + LOG_ERROR(logger, "{}{} =>\n{}", type, task_id, out); else - LOG_DEBUG(logger, "clickhouse plan({}):\n{}", local_engine::QueryContext::instance().currentTaskIdOrEmpty(), out); + LOG_DEBUG(logger, "{}{} =>\n{}", type, task_id, out); } void dumpMessage(const google::protobuf::Message & message, const char * type, bool force, LoggerPtr logger) @@ -70,13 +72,15 @@ void dumpMessage(const google::protobuf::Message & message, const char * type, b return; pb_util::JsonOptions options; std::string json; - if (auto s = google::protobuf::json::MessageToJsonString(message, &json, options); !s.ok()) + if (auto s = MessageToJsonString(message, &json, options); !s.ok()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); + auto task_id = local_engine::QueryContext::instance().currentTaskIdOrEmpty(); + task_id = task_id.empty() ? "" : "(" + task_id + ")"; if (force) // force - LOG_ERROR(logger, "{}({}):\n{}", type, local_engine::QueryContext::instance().currentTaskIdOrEmpty(), json); + LOG_ERROR(logger, "{}{} =>\n{}", type, task_id, json); else - LOG_DEBUG(logger, "{}({}):\n{}", type, local_engine::QueryContext::instance().currentTaskIdOrEmpty(), json); + LOG_DEBUG(logger, "{}{} =>\n{}", type, task_id, json); } void headBlock(const DB::Block & block, size_t count) diff --git a/cpp-ch/local-engine/Common/DebugUtils.h b/cpp-ch/local-engine/Common/DebugUtils.h index 55a0be5140c5..338326b05e0e 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.h +++ b/cpp-ch/local-engine/Common/DebugUtils.h @@ -29,7 +29,7 @@ class QueryPlan; namespace debug { -void dumpPlan(DB::QueryPlan & plan, bool force = false, LoggerPtr = nullptr); +void dumpPlan(DB::QueryPlan & plan, const char * type = "clickhouse plan", bool force = false, LoggerPtr = nullptr); void dumpMessage(const google::protobuf::Message & message, const char * type, bool force = false, LoggerPtr = nullptr); void headBlock(const DB::Block & block, size_t count = 10); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 748ff88acbd1..4e461a5c4954 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -101,8 +101,9 @@ std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) const +void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan) { + const substrait::PlanRel & root_rel = plan.relations().at(0); if (root_rel.root().names_size()) { ActionsDAG actions_dag{blockToNameAndTypeList(query_plan->getCurrentHeader())}; @@ -110,8 +111,8 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con const auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); if (cols.getNames().size() != static_cast(root_rel.root().names_size())) { - debug::dumpPlan(*query_plan, true); - debug::dumpMessage(root_rel, "substrait::PlanRel", true); + debug::dumpPlan(*query_plan, "clickhouse plan", true); + debug::dumpMessage(plan, "substrait::Plan", true); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan name size {}.", @@ -134,8 +135,8 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con const auto & original_cols = original_header.getColumnsWithTypeAndName(); if (static_cast(output_schema.types_size()) != original_cols.size()) { - debug::dumpPlan(*query_plan, true); - debug::dumpMessage(root_rel, "substrait::PlanRel", true); + debug::dumpPlan(*query_plan, "clickhouse plan", true); + debug::dumpMessage(plan, "substrait::Plan", true); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan output schema size {}, subtrait plan name size {}.", @@ -198,7 +199,7 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) std::list rel_stack; auto query_plan = parseOp(first_read_rel, rel_stack); if (!writePipeline) - adjustOutput(query_plan, root_rel); + adjustOutput(query_plan, plan); #ifndef NDEBUG PlanUtil::checkOuputType(*query_plan); @@ -297,12 +298,10 @@ DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPl settings, 0); const QueryPlanOptimizationSettings optimization_settings{.optimize_plan = settings[Setting::query_plan_enable_optimizations]}; - return query_plan.buildQueryPipeline( - optimization_settings, - BuildQueryPipelineSettings{ - .actions_settings - = ExpressionActionsSettings{.can_compile_expressions = true, .min_count_to_compile_expression = 3, .compile_expressions = CompileExpressions::yes}, - .process_list_element = query_status}); + BuildQueryPipelineSettings build_settings = BuildQueryPipelineSettings::fromContext(context); + build_settings.process_list_element = query_status; + build_settings.progress_callback = nullptr; + return query_plan.buildQueryPipeline(optimization_settings,build_settings); } std::unique_ptr SerializedPlanParser::createExecutor(const std::string_view plan) @@ -311,11 +310,10 @@ std::unique_ptr SerializedPlanParser::createExecutor(const std::s return createExecutor(parse(s_plan), s_plan); } -std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) +std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) const { Stopwatch stopwatch; - const Settings & settings = parser_context->queryContext()->getSettingsRef(); DB::QueryPipelineBuilderPtr builder = nullptr; try { @@ -323,7 +321,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla } catch (...) { - LOG_ERROR(getLogger("SerializedPlanParser"), "Invalid plan:\n{}", PlanUtil::explainPlan(*query_plan)); + debug::dumpPlan(*query_plan, "Invalid clickhouse plan", true); throw; } @@ -333,11 +331,6 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla if (root_rel.root().input().has_write()) addSinkTransform(parser_context->queryContext(), root_rel.root().input().write(), builder); LOG_INFO(getLogger("SerializedPlanParser"), "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); - LOG_DEBUG( - getLogger("SerializedPlanParser"), - "clickhouse plan [optimization={}]:\n{}", - settings[Setting::query_plan_enable_optimizations], - PlanUtil::explainPlan(*query_plan)); auto config = ExecutorConfig::loadFromContext(parser_context->queryContext()); return std::make_unique(std::move(query_plan), std::move(builder), config.dump_pipeline); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index f0ec608a330f..eadc7112c266 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -68,7 +68,7 @@ class NonNullableColumnsResolver class SerializedPlanParser { private: - std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan); + std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) const; public: explicit SerializedPlanParser(std::shared_ptr parser_context_); @@ -118,7 +118,7 @@ class SerializedPlanParser private: DB::QueryPlanPtr parseOp(const substrait::Rel & rel, std::list & rel_stack); - void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) const; + static void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan); std::vector input_iters; std::vector split_infos; From 1544307d0c91133fbc059ec3f149869c9701c0f9 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Fri, 15 Nov 2024 10:42:07 +0800 Subject: [PATCH 098/211] [GLUTEN-6896] Add buffered read for hash/sort shuffle reader (#7897) --- .../vectorized/ColumnarBatchSerializer.scala | 2 + cpp/core/jni/JniWrapper.cc | 3 +- cpp/core/shuffle/Options.h | 2 + cpp/core/shuffle/Payload.cc | 18 ++++++--- cpp/core/shuffle/Payload.h | 1 + cpp/velox/compute/VeloxRuntime.cc | 1 + cpp/velox/shuffle/VeloxShuffleReader.cc | 37 ++++++++++++------- cpp/velox/shuffle/VeloxShuffleReader.h | 7 +++- .../utils/tests/VeloxShuffleWriterTestBase.h | 1 + docs/Configuration.md | 1 + .../vectorized/ShuffleReaderJniWrapper.java | 1 + ...VeloxCelebornColumnarBatchSerializer.scala | 2 + .../org/apache/gluten/GlutenConfig.scala | 13 +++++++ 13 files changed, 68 insertions(+), 21 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala index fa4d92652ca3..88215e36bb77 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala @@ -98,6 +98,7 @@ private class ColumnarBatchSerializerInstance( val compressionCodecBackend = GlutenConfig.getConf.columnarShuffleCodecBackend.orNull val batchSize = GlutenConfig.getConf.maxBatchSize + val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize val runtime = Runtimes.contextInstance("ShuffleReader") val jniWrapper = ShuffleReaderJniWrapper.create(runtime) val shuffleReaderHandle = jniWrapper.make( @@ -105,6 +106,7 @@ private class ColumnarBatchSerializerInstance( compressionCodec, compressionCodecBackend, batchSize, + bufferSize, shuffleWriterType) // Close shuffle reader instance as lately as the end of task processing, // since the native reader could hold a reference to memory pool that diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 6a0a5b0057f7..9da55894864e 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -996,6 +996,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrappe jstring compressionType, jstring compressionBackend, jint batchSize, + jlong bufferSize, jstring shuffleWriterType) { JNI_METHOD_START auto ctx = getRuntime(env, wrapper); @@ -1007,7 +1008,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrappe options.codecBackend = getCodecBackend(env, compressionBackend); } options.batchSize = batchSize; - // TODO: Add coalesce option and maximum coalesced size. + options.bufferSize = bufferSize; options.shuffleWriterType = ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterType)); std::shared_ptr schema = diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h index a3dc9f6260b0..6a9e0ec4b340 100644 --- a/cpp/core/shuffle/Options.h +++ b/cpp/core/shuffle/Options.h @@ -38,6 +38,7 @@ static constexpr double kDefaultMergeBufferThreshold = 0.25; static constexpr bool kEnableBufferedWrite = true; static constexpr bool kDefaultUseRadixSort = true; static constexpr int32_t kDefaultSortBufferSize = 4096; +static constexpr int64_t kDefaultReadBufferSize = 1 << 20; enum ShuffleWriterType { kHashShuffle, kSortShuffle, kRssSortShuffle }; enum PartitionWriterType { kLocal, kRss }; @@ -49,6 +50,7 @@ struct ShuffleReaderOptions { ShuffleWriterType shuffleWriterType = kHashShuffle; CodecBackend codecBackend = CodecBackend::NONE; int32_t batchSize = kDefaultBatchSize; + int64_t bufferSize = kDefaultReadBufferSize; }; struct ShuffleWriterOptions { diff --git a/cpp/core/shuffle/Payload.cc b/cpp/core/shuffle/Payload.cc index d5cdb6d08eda..55f3a4339677 100644 --- a/cpp/core/shuffle/Payload.cc +++ b/cpp/core/shuffle/Payload.cc @@ -118,9 +118,9 @@ arrow::Status compressAndFlush( return arrow::Status::OK(); } -arrow::Result> readUncompressedBuffer( - arrow::io::InputStream* inputStream, - arrow::MemoryPool* pool) { +arrow::Result> +readUncompressedBuffer(arrow::io::InputStream* inputStream, arrow::MemoryPool* pool, int64_t& deserializedTime) { + ScopedTimer timer(&deserializedTime); int64_t bufferLength; RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &bufferLength)); if (bufferLength == kNullBuffer) { @@ -135,7 +135,9 @@ arrow::Result> readCompressedBuffer( arrow::io::InputStream* inputStream, const std::shared_ptr& codec, arrow::MemoryPool* pool, + int64_t& deserializeTime, int64_t& decompressTime) { + ScopedTimer timer(&deserializeTime); int64_t compressedLength; RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &compressedLength)); if (compressedLength == kNullBuffer) { @@ -155,7 +157,7 @@ arrow::Result> readCompressedBuffer( ARROW_ASSIGN_OR_RAISE(auto compressed, arrow::AllocateResizableBuffer(compressedLength, pool)); RETURN_NOT_OK(inputStream->Read(compressedLength, compressed->mutable_data())); - ScopedTimer timer(&decompressTime); + timer.switchTo(&decompressTime); ARROW_ASSIGN_OR_RAISE(auto output, arrow::AllocateResizableBuffer(uncompressedLength, pool)); RETURN_NOT_OK(codec->Decompress(compressedLength, compressed->data(), uncompressedLength, output->mutable_data())); return output; @@ -296,7 +298,9 @@ arrow::Result>> BlockPayload::deseria const std::shared_ptr& codec, arrow::MemoryPool* pool, uint32_t& numRows, + int64_t& deserializeTime, int64_t& decompressTime) { + auto timer = std::make_unique(&deserializeTime); static const std::vector> kEmptyBuffers{}; ARROW_ASSIGN_OR_RAISE(auto type, readType(inputStream)); if (type == 0) { @@ -306,6 +310,7 @@ arrow::Result>> BlockPayload::deseria RETURN_NOT_OK(inputStream->Read(sizeof(uint32_t), &numRows)); uint32_t numBuffers; RETURN_NOT_OK(inputStream->Read(sizeof(uint32_t), &numBuffers)); + timer.reset(); bool isCompressionEnabled = type == Type::kCompressed; std::vector> buffers; @@ -313,9 +318,10 @@ arrow::Result>> BlockPayload::deseria for (auto i = 0; i < numBuffers; ++i) { buffers.emplace_back(); if (isCompressionEnabled) { - ARROW_ASSIGN_OR_RAISE(buffers.back(), readCompressedBuffer(inputStream, codec, pool, decompressTime)); + ARROW_ASSIGN_OR_RAISE( + buffers.back(), readCompressedBuffer(inputStream, codec, pool, deserializeTime, decompressTime)); } else { - ARROW_ASSIGN_OR_RAISE(buffers.back(), readUncompressedBuffer(inputStream, pool)); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readUncompressedBuffer(inputStream, pool, deserializeTime)); } } return buffers; diff --git a/cpp/core/shuffle/Payload.h b/cpp/core/shuffle/Payload.h index ea8c897e96d0..611b2310d547 100644 --- a/cpp/core/shuffle/Payload.h +++ b/cpp/core/shuffle/Payload.h @@ -92,6 +92,7 @@ class BlockPayload final : public Payload { const std::shared_ptr& codec, arrow::MemoryPool* pool, uint32_t& numRows, + int64_t& deserializeTime, int64_t& decompressTime); static int64_t maxCompressedLength( diff --git a/cpp/velox/compute/VeloxRuntime.cc b/cpp/velox/compute/VeloxRuntime.cc index 332c75dbd725..4c6b52e6fe04 100644 --- a/cpp/velox/compute/VeloxRuntime.cc +++ b/cpp/velox/compute/VeloxRuntime.cc @@ -259,6 +259,7 @@ std::shared_ptr VeloxRuntime::createShuffleReader( veloxCompressionType, rowType, options.batchSize, + options.bufferSize, memoryManager()->getArrowMemoryPool(), ctxVeloxPool, options.shuffleWriterType); diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index 92751d454aa9..0407be736a70 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -38,7 +38,6 @@ #include #include -// using namespace facebook; using namespace facebook::velox; namespace gluten { @@ -291,14 +290,14 @@ VeloxHashShuffleReaderDeserializer::VeloxHashShuffleReaderDeserializer( const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, std::vector* isValidityBuffer, bool hasComplexType, int64_t& deserializeTime, int64_t& decompressTime) - : in_(std::move(in)), - schema_(schema), + : schema_(schema), codec_(codec), rowType_(rowType), batchSize_(batchSize), @@ -307,13 +306,16 @@ VeloxHashShuffleReaderDeserializer::VeloxHashShuffleReaderDeserializer( isValidityBuffer_(isValidityBuffer), hasComplexType_(hasComplexType), deserializeTime_(deserializeTime), - decompressTime_(decompressTime) {} + decompressTime_(decompressTime) { + GLUTEN_ASSIGN_OR_THROW(in_, arrow::io::BufferedInputStream::Create(bufferSize, memoryPool, std::move(in))); +} std::shared_ptr VeloxHashShuffleReaderDeserializer::next() { if (hasComplexType_) { uint32_t numRows = 0; GLUTEN_ASSIGN_OR_THROW( - auto arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, decompressTime_)); + auto arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { // Reach EOS. return nullptr; @@ -332,7 +334,8 @@ std::shared_ptr VeloxHashShuffleReaderDeserializer::next() { uint32_t numRows = 0; while (!merged_ || merged_->numRows() < batchSize_) { GLUTEN_ASSIGN_OR_THROW( - arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, decompressTime_)); + arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { reachEos_ = true; break; @@ -372,22 +375,24 @@ VeloxSortShuffleReaderDeserializer::VeloxSortShuffleReaderDeserializer( const std::shared_ptr& codec, const RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, int64_t& deserializeTime, int64_t& decompressTime) - : in_(std::move(in)), - schema_(schema), + : schema_(schema), codec_(codec), rowType_(rowType), batchSize_(batchSize), arrowPool_(memoryPool), veloxPool_(veloxPool), deserializeTime_(deserializeTime), - decompressTime_(decompressTime) {} + decompressTime_(decompressTime) { + GLUTEN_ASSIGN_OR_THROW(in_, arrow::io::BufferedInputStream::Create(bufferSize, memoryPool, std::move(in))); +} std::shared_ptr VeloxSortShuffleReaderDeserializer::next() { - if (reachEos_) { + if (reachedEos_) { if (cachedRows_ > 0) { return deserializeToBatch(); } @@ -401,10 +406,11 @@ std::shared_ptr VeloxSortShuffleReaderDeserializer::next() { while (cachedRows_ < batchSize_) { uint32_t numRows = 0; GLUTEN_ASSIGN_OR_THROW( - auto arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, arrowPool_, numRows, decompressTime_)); + auto arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, arrowPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { - reachEos_ = true; + reachedEos_ = true; if (cachedRows_ > 0) { return deserializeToBatch(); } @@ -467,7 +473,8 @@ void VeloxSortShuffleReaderDeserializer::readLargeRow(std::vectorsize(); buffers.emplace_back(std::move(arrowBuffers[0])); @@ -575,6 +582,7 @@ VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( const facebook::velox::common::CompressionKind veloxCompressionType, const RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, std::shared_ptr veloxPool, ShuffleWriterType shuffleWriterType) @@ -583,6 +591,7 @@ VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( veloxCompressionType_(veloxCompressionType), rowType_(rowType), batchSize_(batchSize), + bufferSize_(bufferSize), memoryPool_(memoryPool), veloxPool_(veloxPool), shuffleWriterType_(shuffleWriterType) { @@ -599,6 +608,7 @@ std::unique_ptr VeloxColumnarBatchDeserializerFactory::cr codec_, rowType_, batchSize_, + bufferSize_, memoryPool_, veloxPool_.get(), &isValidityBuffer_, @@ -612,6 +622,7 @@ std::unique_ptr VeloxColumnarBatchDeserializerFactory::cr codec_, rowType_, batchSize_, + bufferSize_, memoryPool_, veloxPool_.get(), deserializeTime_, diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index d39e38936369..af35f977127f 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -21,6 +21,7 @@ #include "shuffle/Payload.h" #include "shuffle/ShuffleReader.h" #include "shuffle/VeloxSortShuffleWriter.h" +#include "utils/Timer.h" #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" @@ -36,6 +37,7 @@ class VeloxHashShuffleReaderDeserializer final : public ColumnarBatchIterator { const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, std::vector* isValidityBuffer, @@ -73,6 +75,7 @@ class VeloxSortShuffleReaderDeserializer final : public ColumnarBatchIterator { const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, int64_t& deserializeTime, @@ -97,7 +100,7 @@ class VeloxSortShuffleReaderDeserializer final : public ColumnarBatchIterator { std::list> cachedInputs_; uint32_t cachedRows_{0}; - bool reachEos_{false}; + bool reachedEos_{false}; uint32_t rowOffset_{0}; size_t byteOffset_{0}; @@ -139,6 +142,7 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { const facebook::velox::common::CompressionKind veloxCompressionType, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, std::shared_ptr veloxPool, ShuffleWriterType shuffleWriterType); @@ -161,6 +165,7 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { facebook::velox::common::CompressionKind veloxCompressionType_; facebook::velox::RowTypePtr rowType_; int32_t batchSize_; + int64_t bufferSize_; arrow::MemoryPool* memoryPool_; std::shared_ptr veloxPool_; diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h index 4b4abfb525a5..d2995e251c68 100644 --- a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -366,6 +366,7 @@ class VeloxShuffleWriterTest : public ::testing::TestWithParam::max(), + kDefaultReadBufferSize, defaultArrowMemoryPool().get(), pool_, GetParam().shuffleWriterType); diff --git a/docs/Configuration.md b/docs/Configuration.md index 76549dd4fe7c..d4ca9c10fb76 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -51,6 +51,7 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.gluten.sql.columnar.shuffle.compression.threshold | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | 100 | | spark.gluten.sql.columnar.shuffle.realloc.threshold | Set the threshold to dynamically adjust the size of shuffle split buffers. The size of each split buffer is recalculated for each incoming batch of data. If the new size deviates from the current partition buffer size by a factor outside the range of [1 - threshold, 1 + threshold], the split buffer will be re-allocated using the newly calculated size | 0.25 | | spark.gluten.sql.columnar.shuffle.merge.threshold | Set the threshold control the minimum merged size. When a partition buffer is full, and the number of rows is below (`threshold * spark.gluten.sql.columnar.maxBatchSize`), it will be saved for merging. | 0.25 | +| spark.gluten.sql.columnar.shuffle.readerBufferSize | Buffer size in bytes for shuffle reader reading input stream from local or remote. | 1MB | | spark.gluten.sql.columnar.numaBinding | Set up NUMABinding, default is false | true | | spark.gluten.sql.columnar.coreRange | Set up the core range for NUMABinding, only works when numaBinding set to true.
    The setting is based on the number of cores in your system. Use 72 cores as an example. | 0-17,36-53 |18-35,54-71 | | spark.gluten.sql.columnar.wholeStage.fallback.threshold | Configure the threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node | \>= 1 | diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java index 09f97cbd06b9..8d031cf0792c 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java @@ -40,6 +40,7 @@ public native long make( String compressionType, String compressionCodecBackend, int batchSize, + long bufferSize, String shuffleWriterType); public native long readStream(long shuffleReaderHandle, JniByteInputStream jniIn); diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala index a1fb190e4593..d5f20c8dea00 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala @@ -88,12 +88,14 @@ private class CelebornColumnarBatchSerializerInstance( .replace(GLUTEN_SORT_SHUFFLE_WRITER, GLUTEN_RSS_SORT_SHUFFLE_WRITER) val jniWrapper = ShuffleReaderJniWrapper.create(runtime) val batchSize = GlutenConfig.getConf.maxBatchSize + val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize val handle = jniWrapper .make( cSchema.memoryAddress(), compressionCodec, compressionCodecBackend, batchSize, + bufferSize, shuffleWriterType ) // Close shuffle reader instance as lately as the end of task processing, diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 0e41fc65376c..f049e045c7b0 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -193,6 +193,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def columnarShuffleCompressionThreshold: Int = conf.getConf(COLUMNAR_SHUFFLE_COMPRESSION_THRESHOLD) + def columnarShuffleReaderBufferSize: Long = + conf.getConf(COLUMNAR_SHUFFLE_READER_BUFFER_SIZE) + def maxBatchSize: Int = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) def columnarToRowMemThreshold: Long = @@ -607,6 +610,9 @@ object GlutenConfig { val GLUTEN_SHUFFLE_WRITER_MERGE_THRESHOLD = "spark.gluten.sql.columnar.shuffle.merge.threshold" val GLUTEN_SHUFFLE_DEFUALT_COMPRESSION_BUFFER_SIZE = 32 * 1024 + // Shuffle reader buffer size. + val GLUTEN_SHUFFLE_READER_BUFFER_SIZE = "spark.gluten.sql.columnar.shuffle.readerBufferSize" + // Controls whether to load DLL from jars. User can get dependent native libs packed into a jar // by executing dev/package.sh. Then, with that jar configured, Gluten can load the native libs // at runtime. This config is just for velox backend. And it is NOT applicable to the situation @@ -1155,6 +1161,13 @@ object GlutenConfig { .checkValue(v => v >= 0 && v <= 1, "Shuffle writer merge threshold must between [0, 1]") .createWithDefault(0.25) + val COLUMNAR_SHUFFLE_READER_BUFFER_SIZE = + buildConf(GLUTEN_SHUFFLE_READER_BUFFER_SIZE) + .internal() + .doc("Buffer size in bytes for shuffle reader reading input stream from local or remote.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("1MB") + val COLUMNAR_MAX_BATCH_SIZE = buildConf(GLUTEN_MAX_BATCH_SIZE_KEY) .internal() From fd716bf1420566608f093e8354bcf577a2511c99 Mon Sep 17 00:00:00 2001 From: Xuedong Luan Date: Fri, 15 Nov 2024 13:01:12 +0800 Subject: [PATCH 099/211] [GLUTEN-7837][VL] Spark driver should not initialize cache if not in local mode Closes #7837 --- .../gluten/backendsapi/velox/VeloxListenerApi.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index 850509db3e91..e5c9afc096f4 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -89,7 +89,7 @@ class VeloxListenerApi extends ListenerApi with Logging { SparkDirectoryUtil.init(conf) UDFResolver.resolveUdfConf(conf, isDriver = true) - initialize(conf) + initialize(conf, isDriver = true) UdfJniWrapper.registerFunctionSignatures() } @@ -116,12 +116,12 @@ class VeloxListenerApi extends ListenerApi with Logging { SparkDirectoryUtil.init(conf) UDFResolver.resolveUdfConf(conf, isDriver = false) - initialize(conf) + initialize(conf, isDriver = false) } override def onExecutorShutdown(): Unit = shutdown() - private def initialize(conf: SparkConf): Unit = { + private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { // Force batch type initializations. VeloxBatch.getClass ArrowJavaBatch.getClass @@ -157,7 +157,11 @@ class VeloxListenerApi extends ListenerApi with Logging { } // Initial native backend with configurations. - val parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap) + var parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap) + // Workaround for https://github.com/apache/incubator-gluten/issues/7837 + if (isDriver && !inLocalMode(conf)) { + parsed += (GlutenConfig.COLUMNAR_VELOX_CACHE_ENABLED.key -> "false") + } NativeBackendInitializer.initializeBackend(parsed) // Inject backend-specific implementations to override spark classes. From 596858adde56caa4167e73ffdf53dedf8700874d Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Fri, 15 Nov 2024 14:19:18 +0800 Subject: [PATCH 100/211] support nested column pruning (#7268) Co-authored-by: zouyunhe <811-zouyunhe@users.noreply.git.sysop.bigo.sg> --- .../backendsapi/clickhouse/CHBackend.scala | 4 + .../hive/GlutenClickHouseHiveTableSuite.scala | 53 ++++ .../backendsapi/BackendSettingsApi.scala | 2 + .../heuristic/OffloadSingleNode.scala | 8 +- .../hive/HiveTableScanExecTransformer.scala | 11 +- .../HiveTableScanNestedColumnPruning.scala | 252 ++++++++++++++++++ .../org/apache/gluten/GlutenConfig.scala | 10 + .../execution/AbstractHiveTableScanExec.scala | 7 +- .../execution/AbstractHiveTableScanExec.scala | 7 +- .../execution/AbstractHiveTableScanExec.scala | 7 +- .../execution/AbstractHiveTableScanExec.scala | 7 +- 11 files changed, 360 insertions(+), 8 deletions(-) create mode 100644 gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 54ab38569bb8..6e73ff6b29d1 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -396,4 +396,8 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } override def supportWindowGroupLimitExec(rankLikeFunction: Expression): Boolean = true + + override def supportHiveTableScanNestedColumnPruning: Boolean = + GlutenConfig.getConf.enableColumnarHiveTableScanNestedColumnPruning + } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index ff2d13996dc6..7e31e73040d4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -1499,4 +1499,57 @@ class GlutenClickHouseHiveTableSuite spark.sql("drop table if exists aj") } + test("test hive table scan nested column pruning") { + val json_table_name = "test_tbl_7267_json" + val pq_table_name = "test_tbl_7267_pq" + val create_table_sql = + s""" + | create table if not exists %s( + | id bigint, + | d1 STRUCT>>, + | d2 STRUCT>>, + | day string, + | hour string + | ) partitioned by(day, hour) + |""".stripMargin + val create_table_json = create_table_sql.format(json_table_name) + + s""" + | ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' + | STORED AS INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + |""".stripMargin + val create_table_pq = create_table_sql.format(pq_table_name) + " Stored as PARQUET" + val insert_sql = + """ + | insert into %s values(1, + | named_struct('c', 'c123', 'd', array(named_struct('x', 'x123', 'y', 'y123'))), + | named_struct('c', 'c124', 'd', map('m124', named_struct('x', 'x124', 'y', 'y124'))), + | '2024-09-26', '12' + | ) + |""".stripMargin + val select_sql = + "select id, d1.c, d1.d[0].x, d2.d['m124'].y from %s where day = '2024-09-26' and hour = '12'" + val table_names = Array.apply(json_table_name, pq_table_name) + val create_table_sqls = Array.apply(create_table_json, create_table_pq) + for (i <- table_names.indices) { + val table_name = table_names(i) + val create_table = create_table_sqls(i) + spark.sql(create_table) + spark.sql(insert_sql.format(table_name)) + withSQLConf(("spark.sql.hive.convertMetastoreParquet" -> "false")) { + compareResultsAgainstVanillaSpark( + select_sql.format(table_name), + compareResult = true, + df => { + val scan = collect(df.queryExecution.executedPlan) { + case l: HiveTableScanExecTransformer => l + } + assert(scan.size == 1) + } + ) + } + spark.sql("drop table if exists %s".format(table_name)) + } + } + } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 177d19c0c709..700571fd2815 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -131,4 +131,6 @@ trait BackendSettingsApi { def supportColumnarArrowUdf(): Boolean = false def needPreComputeRangeFrameBoundary(): Boolean = false + + def supportHiveTableScanNestedColumnPruning(): Boolean = false } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala index a8c200e9be44..bae98bec2ec6 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleEx import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} -import org.apache.spark.sql.hive.HiveTableScanExecTransformer +import org.apache.spark.sql.hive.{HiveTableScanExecTransformer, HiveTableScanNestedColumnPruning} /** * Converts a vanilla Spark plan node into Gluten plan node. Gluten plan is supposed to be executed @@ -226,7 +226,11 @@ object OffloadOthers { case plan: ProjectExec => val columnarChild = plan.child logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - ProjectExecTransformer(plan.projectList, columnarChild) + if (HiveTableScanNestedColumnPruning.supportNestedColumnPruning(plan)) { + HiveTableScanNestedColumnPruning.apply(plan) + } else { + ProjectExecTransformer(plan.projectList, columnarChild) + } case plan: HashAggregateExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") HashAggregateExecBaseTransformer.from(plan) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala index 85432350d4a2..f701c76b1813 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala @@ -45,7 +45,8 @@ import java.net.URI case class HiveTableScanExecTransformer( requestedAttributes: Seq[Attribute], relation: HiveTableRelation, - partitionPruningPred: Seq[Expression])(@transient session: SparkSession) + partitionPruningPred: Seq[Expression], + prunedOutput: Seq[Attribute] = Seq.empty[Attribute])(@transient session: SparkSession) extends AbstractHiveTableScanExec(requestedAttributes, relation, partitionPruningPred)(session) with BasicScanExecTransformer { @@ -63,7 +64,13 @@ case class HiveTableScanExecTransformer( override def getMetadataColumns(): Seq[AttributeReference] = Seq.empty - override def outputAttributes(): Seq[Attribute] = output + override def outputAttributes(): Seq[Attribute] = { + if (prunedOutput.nonEmpty) { + prunedOutput + } else { + output + } + } override def getPartitions: Seq[InputPartition] = partitions diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala new file mode 100644 index 000000000000..7a20e5c37da5 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, ProjectExec, SparkPlan} +import org.apache.spark.sql.hive.HiveTableScanExecTransformer.{ORC_INPUT_FORMAT_CLASS, PARQUET_INPUT_FORMAT_CLASS, TEXT_INPUT_FORMAT_CLASS} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} +import org.apache.spark.sql.util.SchemaUtils._ +import org.apache.spark.util.Utils + +object HiveTableScanNestedColumnPruning extends Logging { + import org.apache.spark.sql.catalyst.expressions.SchemaPruning._ + + def supportNestedColumnPruning(projectExec: ProjectExec): Boolean = { + if (BackendsApiManager.getSettings.supportHiveTableScanNestedColumnPruning()) { + projectExec.child match { + case HiveTableScanExecTransformer(_, relation, _, _) => + relation.tableMeta.storage.inputFormat match { + case Some(inputFormat) + if TEXT_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + relation.tableMeta.storage.serde match { + case Some("org.openx.data.jsonserde.JsonSerDe") | Some( + "org.apache.hive.hcatalog.data.JsonSerDe") => + return true + case _ => + } + case Some(inputFormat) + if ORC_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + return true + case Some(inputFormat) + if PARQUET_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + return true + case _ => + } + case _ => + } + } + false + } + + def apply(plan: SparkPlan): SparkPlan = { + plan match { + case ProjectExec(projectList, child) => + child match { + case h: HiveTableScanExecTransformer => + val newPlan = prunePhysicalColumns( + h.relation, + projectList, + Seq.empty[Expression], + (prunedDataSchema, prunedMetadataSchema) => { + buildNewHiveTableScan(h, prunedDataSchema, prunedMetadataSchema) + }, + (schema, requestFields) => { + h.pruneSchema(schema, requestFields) + } + ) + if (newPlan.nonEmpty) { + return newPlan.get + } else { + return ProjectExecTransformer(projectList, child) + } + case _ => + return ProjectExecTransformer(projectList, child) + } + case _ => + } + plan + } + + private def prunePhysicalColumns( + relation: HiveTableRelation, + projects: Seq[NamedExpression], + filters: Seq[Expression], + leafNodeBuilder: (StructType, StructType) => LeafExecNode, + pruneSchemaFunc: (StructType, Seq[SchemaPruning.RootField]) => StructType) + : Option[SparkPlan] = { + val (normalizedProjects, normalizedFilters) = + normalizeAttributeRefNames(relation.output, projects, filters) + val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters) + // If requestedRootFields includes a nested field, continue. Otherwise, + // return op + if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) { + val prunedDataSchema = pruneSchemaFunc(relation.tableMeta.dataSchema, requestedRootFields) + val metaFieldNames = relation.tableMeta.schema.fieldNames + val metadataSchema = relation.output.collect { + case attr: AttributeReference if metaFieldNames.contains(attr.name) => attr + }.toStructType + val prunedMetadataSchema = if (metadataSchema.nonEmpty) { + pruneSchemaFunc(metadataSchema, requestedRootFields) + } else { + metadataSchema + } + // If the data schema is different from the pruned data schema + // OR + // the metadata schema is different from the pruned metadata schema, continue. + // Otherwise, return None. + if ( + countLeaves(relation.tableMeta.dataSchema) > countLeaves(prunedDataSchema) || + countLeaves(metadataSchema) > countLeaves(prunedMetadataSchema) + ) { + val leafNode = leafNodeBuilder(prunedDataSchema, prunedMetadataSchema) + val projectionOverSchema = ProjectionOverSchema( + prunedDataSchema.merge(prunedMetadataSchema), + AttributeSet(relation.output)) + Some( + buildNewProjection( + projects, + normalizedProjects, + normalizedFilters, + leafNode, + projectionOverSchema)) + } else { + None + } + } else { + None + } + } + + /** + * Normalizes the names of the attribute references in the given projects and filters to reflect + * the names in the given logical relation. This makes it possible to compare attributes and + * fields by name. Returns a tuple with the normalized projects and filters, respectively. + */ + private def normalizeAttributeRefNames( + output: Seq[AttributeReference], + projects: Seq[NamedExpression], + filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = { + val normalizedAttNameMap = output.map(att => (att.exprId, att.name)).toMap + val normalizedProjects = projects + .map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + .map { case expr: NamedExpression => expr } + val normalizedFilters = filters.map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + (normalizedProjects, normalizedFilters) + } + + /** Builds the new output [[Project]] Spark SQL operator that has the `leafNode`. */ + private def buildNewProjection( + projects: Seq[NamedExpression], + normalizedProjects: Seq[NamedExpression], + filters: Seq[Expression], + leafNode: LeafExecNode, + projectionOverSchema: ProjectionOverSchema): ProjectExecTransformer = { + // Construct a new target for our projection by rewriting and + // including the original filters where available + val projectionChild = + if (filters.nonEmpty) { + val projectedFilters = filters.map(_.transformDown { + case projectionOverSchema(expr) => expr + }) + val newFilterCondition = projectedFilters.reduce(And) + FilterExec(newFilterCondition, leafNode) + } else { + leafNode + } + + // Construct the new projections of our Project by + // rewriting the original projections + val newProjects = + normalizedProjects.map(_.transformDown { case projectionOverSchema(expr) => expr }).map { + case expr: NamedExpression => expr + } + + ProjectExecTransformer( + restoreOriginalOutputNames(newProjects, projects.map(_.name)), + projectionChild) + } + + private def buildNewHiveTableScan( + hiveTableScan: HiveTableScanExecTransformer, + prunedDataSchema: StructType, + prunedMetadataSchema: StructType): HiveTableScanExecTransformer = { + val relation = hiveTableScan.relation + val partitionSchema = relation.tableMeta.partitionSchema + val prunedBaseSchema = StructType( + prunedDataSchema.fields.filterNot( + f => partitionSchema.fieldNames.contains(f.name)) ++ partitionSchema.fields) + val finalSchema = prunedBaseSchema.merge(prunedMetadataSchema) + val prunedOutput = getPrunedOutput(relation.output, finalSchema) + var finalOutput = Seq.empty[Attribute] + for (p <- hiveTableScan.output) { + var flag = false + for (q <- prunedOutput if !flag) { + if (p.name.equals(q.name)) { + finalOutput :+= q + flag = true + } + } + } + HiveTableScanExecTransformer( + hiveTableScan.requestedAttributes, + relation, + hiveTableScan.partitionPruningPred, + finalOutput)(hiveTableScan.session) + } + + // Prune the given output to make it consistent with `requiredSchema`. + private def getPrunedOutput( + output: Seq[AttributeReference], + requiredSchema: StructType): Seq[Attribute] = { + // We need to update the data type of the output attributes to use the pruned ones. + // so that references to the original relation's output are not broken + val nameAttributeMap = output.map(att => (att.name, att)).toMap + val requiredAttributes = + requiredSchema.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + requiredAttributes.map { + case att if nameAttributeMap.contains(att.name) => + nameAttributeMap(att.name).withDataType(att.dataType) + case att => att + } + } + + /** + * Counts the "leaf" fields of the given dataType. Informally, this is the number of fields of + * non-complex data type in the tree representation of [[DataType]]. + */ + private def countLeaves(dataType: DataType): Int = { + dataType match { + case array: ArrayType => countLeaves(array.elementType) + case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType) + case struct: StructType => + struct.map(field => countLeaves(field.dataType)).sum + case _ => 1 + } + } +} diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index f049e045c7b0..5b15faf6466a 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -50,6 +50,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableColumnarHiveTableScan: Boolean = conf.getConf(COLUMNAR_HIVETABLESCAN_ENABLED) + def enableColumnarHiveTableScanNestedColumnPruning: Boolean = + conf.getConf(COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED) + def enableVanillaVectorizedReaders: Boolean = conf.getConf(VANILLA_VECTORIZED_READERS_ENABLED) def enableColumnarHashAgg: Boolean = conf.getConf(COLUMNAR_HASHAGG_ENABLED) @@ -865,6 +868,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED = + buildConf("spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan") + .internal() + .doc("Enable or disable nested column pruning in hivetablescan.") + .booleanConf + .createWithDefault(true) + val VANILLA_VECTORIZED_READERS_ENABLED = buildStaticConf("spark.gluten.sql.columnar.enableVanillaVectorizedReaders") .internal() diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 46b59ac306c2..f38c85a49dde 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -232,4 +233,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneDataSchema(schema, requestedFields) + } } diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index dd095f0ff247..d9b6bb936f67 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -239,4 +240,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 87aba00b0f59..3521d496546d 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -257,4 +258,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 87aba00b0f59..3521d496546d 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -257,4 +258,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } From 7e4a13abd956e243ad47d7f44ef66754b1c63a11 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Fri, 15 Nov 2024 14:58:44 +0800 Subject: [PATCH 101/211] [GLUTEN-7499][VL][CI] Print ccache statistics for tracking its efficacy (#7957) Just a minor change in case of ccache loses its efficacy for some reason. P.S. For the current pr, we got the below statistics printed. cache directory /__w/incubator-gluten/incubator-gluten/.ccache primary config /__w/incubator-gluten/incubator-gluten/.ccache/ccache.conf secondary config (readonly) /etc/ccache.conf stats updated Fri Nov 15 03:03:00 2024 cache hit (direct) 112337 cache hit (preprocessed) 899 cache miss 10042 cache hit rate 91.85 % called for link 368 cleanups performed 59 files in cache 11473 cache size 4.3 GB max cache size 5.0 GB --- .github/workflows/velox_backend.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index fafceb1aa06c..42e1f2527816 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -71,6 +71,7 @@ jobs: df -a cd $GITHUB_WORKSPACE/ bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s - name: "Save ccache" uses: actions/cache/save@v3 id: ccache @@ -954,6 +955,7 @@ jobs: run: | df -a bash dev/ci-velox-buildshared-centos-8.sh + ccache -s - name: "Save ccache" uses: actions/cache/save@v3 id: ccache From c7548aa005a0350b30fa987b36b9d7a5b25747f7 Mon Sep 17 00:00:00 2001 From: "shuai.xu" Date: Fri, 15 Nov 2024 15:07:27 +0800 Subject: [PATCH 102/211] [GLUTEN-7594] [CH] support cast const map to string (#7599) * [GLUTEN-7594] [CH] support cast const map to string * fix test * fix test --- .../GlutenClickhouseFunctionSuite.scala | 18 +++++++++++++++++- .../Functions/SparkFunctionMapToString.h | 8 ++++++-- 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index 2cfe935ef7e0..02af10657c73 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.compatibility import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite +import org.apache.gluten.execution.{GlutenClickHouseTPCHAbstractSuite, ProjectExecTransformer} import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf @@ -269,6 +269,22 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { } } + test("GLUTEN-7594: cast const map to string") { + withSQLConf( + ( + "spark.sql.optimizer.excludedRules", + "org.apache.spark.sql.catalyst.optimizer.ConstantFolding," + + "org.apache.spark.sql.catalyst.optimizer.NullPropagation")) { + runQueryAndCompare( + """ + |select cast(map(1,'2') as string) + |""".stripMargin, + true, + false + )(checkGlutenOperatorMatch[ProjectExecTransformer]) + } + } + test("GLUTEN-7550 get_json_object in IN") { withTable("test_7550") { sql("create table test_7550(a string) using parquet") diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h index 444e4542c807..3f8a0c97dc07 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h @@ -76,7 +76,10 @@ class SparkFunctionMapToString : public DB::IFunction arguments[1].type->getName(), arguments[2].type->getName()); } - return makeNullable(std::make_shared()); + if (arguments[0].type->isNullable()) + return makeNullable(std::make_shared()); + else + return std::make_shared(); } ColumnPtr executeImpl(const DB::ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows*/) const override @@ -89,7 +92,8 @@ class SparkFunctionMapToString : public DB::IFunction } const auto & col_with_type_and_name = columnGetNested(arguments[0]); - const IColumn & col_from = *col_with_type_and_name.column; + const IColumn & column = *col_with_type_and_name.column; + const IColumn & col_from = column.isConst() ? reinterpret_cast(column).getDataColumn() : column; size_t size = col_from.size(); auto col_to = removeNullable(result_type)->createColumn(); From 802dfa690dc0eff18cfc6fe0f66e36e968f11436 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Fri, 15 Nov 2024 17:19:45 +0800 Subject: [PATCH 103/211] [CORE] Add buildSide info for BroadcastNestedLoopJoinExecTransformer simpleStringWithNodeId (#7948) --- .../execution/BroadcastNestedLoopJoinExecTransformer.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala index ae407b3b3efa..1e0c3d5a7e70 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.{FullOuter, InnerLike, JoinType, LeftExistence, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ExplainUtils, SparkPlan} import org.apache.spark.sql.execution.joins.BaseJoinExec import org.apache.spark.sql.execution.metric.SQLMetric @@ -45,6 +45,11 @@ abstract class BroadcastNestedLoopJoinExecTransformer( def joinBuildSide: BuildSide = buildSide + override def simpleStringWithNodeId(): String = { + val opId = ExplainUtils.getOpId(this) + s"$nodeName $joinType $joinBuildSide ($opId)".trim + } + override def leftKeys: Seq[Expression] = Nil override def rightKeys: Seq[Expression] = Nil From 19e1caea853570e180ae16ccea5a4178bcfe1f8f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 15 Nov 2024 19:29:20 +0800 Subject: [PATCH 104/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_15) (#7954) Upstream Velox's New Commits: 9ebe36609 by zuyu, Fix OutputBuffer comments regarding bufferedBytes_ (11423) 935d30ee1 by zhli1142015, Combine low selectivity vectors generated by the hash join filter (10987) 5ac572b47 by rui-mo, Fix Spark expression fuzzer test on deny_precision_loss functions (11526) fe87f0b75 by hengjiang.ly, Add a faster string-search method with simd instructions than std::find (10858) 5b274e3cf by NEUpanning, Allow partial date parsing when simple datetime formatter is used (11386) c069192e9 by Jimmy Lu, Buckify breeze (11512) f37dc004e by Xiaoxuan Meng, Allow single thread execution mode for local partition to support spark union for gluten (11531) 837480242 by Christian Zentgraf, Add Clang Linux build to CI pipeline (10767) 427ac31e6 by Yenda Li, Add support for TTL in directories (11498) --- cpp/velox/jni/JniFileSystem.cc | 4 ++-- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/velox/jni/JniFileSystem.cc b/cpp/velox/jni/JniFileSystem.cc index 21cdf2b23a02..fa849664e3ae 100644 --- a/cpp/velox/jni/JniFileSystem.cc +++ b/cpp/velox/jni/JniFileSystem.cc @@ -241,7 +241,7 @@ class FileSystemWrapper : public facebook::velox::filesystems::FileSystem { return fs_->list(rewrite(path)); } - void mkdir(std::string_view path) override { + void mkdir(std::string_view path, const facebook::velox::filesystems::DirectoryOptions& options = {}) override { fs_->mkdir(rewrite(path)); } @@ -349,7 +349,7 @@ class JniFileSystem : public facebook::velox::filesystems::FileSystem { return out; } - void mkdir(std::string_view path) override { + void mkdir(std::string_view path, const facebook::velox::filesystems::DirectoryOptions& options = {}) override { JNIEnv* env = nullptr; attachCurrentThreadAsDaemonOrThrow(vm, &env); env->CallVoidMethod(obj_, jniFileSystemMkdir, createJString(env, path)); diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 06a7215d827a..9e0f3c0160ff 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_14 +VELOX_BRANCH=2024_11_15 VELOX_HOME="" OS=`uname -s` From a7f43625cbe613cab8f70d9c99399d515552a37f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sat, 16 Nov 2024 09:46:21 +0800 Subject: [PATCH 105/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_16) (#7958) Upstream Velox's New Commits: b40cec3f3 by Joe Giardino, - Fix FlatVector copy memory issue (11483) 035750010 by yingsu00, Fix PlanNodeStats overflow (11532) 8aeb51cf4 by Jimmy Lu, fix: SSE only build (11542) ab02900db by Orri Erling, feat: NVRTC work with Wave headers (11500) 213f0a053 by Yenda Li, Add comparison support for RowView (11499) cce529e08 by Wei He, Fix Window operator with NaN as the frame bound (11293) ebd597a02 by Jialiang Tan, Change memory-reclaim-max-wait-time to max-memory-arbitration-time (11405) 88e3e6118 by Jacob Wujciak-Jens, build(ci): Add conventional commit title check (11520) 9494cbca8 by Chengcheng Jin, Clean up prefix sort function defaultPrefixSortConfig (11533) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 9e0f3c0160ff..02656d45e3cb 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_15 +VELOX_BRANCH=2024_11_16 VELOX_HOME="" OS=`uname -s` From 3941a2d1d02864c6d3992c2c6e2b0563912cb058 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 18 Nov 2024 07:25:05 +0800 Subject: [PATCH 106/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_17) (#7961) Upstream Velox's New Commits: d1ce7195b by duanmeng, Init filesystem after registered in query replyaer (11559) 90bbdaae6 by Yenda Li, fix(replace_first): Fix up registration of replace_first (11556) 00e814941 by Pramod, Add function serializeSingleColumn to PrestoVectorSerde (10657) 3265e791c by Jialiang Tan, refactor(arbitrator): Simplify arbitration participant lock (11525) 1029d8b35 by Heidi Han, Add support for array join on json (11446) 0a422f738 by Jimmy Lu, fix: Remove same page hack for simdStrstr (11553) 31ae379d4 by David Reveman, Breeze fixes (11537) c21020c5a by yingsu00, fix: ExchangeBenchmark "Named vector serde 'Presto' is not registered" (11548) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 02656d45e3cb..0a1f168db7b5 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_16 +VELOX_BRANCH=2024_11_17 VELOX_HOME="" OS=`uname -s` From c144443443580baf0f97dbfb721d66bc9cb21faa Mon Sep 17 00:00:00 2001 From: JiaKe Date: Mon, 18 Nov 2024 09:01:19 +0800 Subject: [PATCH 107/211] Add config to support viewfs in Gluten. (#7892) --- .../clickhouse/CHIteratorApi.scala | 4 +- ...tenClickHouseMergeTreeWriteOnS3Suite.scala | 2 +- .../GlutenClickHouseMergeTreeWriteSuite.scala | 6 +-- .../backendsapi/velox/VeloxIteratorApi.scala | 20 +++------- .../execution/IcebergScanTransformer.scala | 5 +-- .../gluten/execution/VeloxIcebergSuite.scala | 6 +-- .../gluten/substrait/rel/LocalFilesNode.java | 5 +++ .../gluten/backendsapi/IteratorApi.scala | 4 +- .../execution/BasicScanExecTransformer.scala | 12 ++---- .../execution/WholeStageTransformer.scala | 37 +++++++++++++++---- .../org/apache/gluten/GlutenConfig.scala | 9 +++++ 11 files changed, 62 insertions(+), 48 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala index dd5a736e7571..ff268b95d8de 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala @@ -43,7 +43,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration import java.lang.{Long => JLong} import java.net.URI @@ -133,8 +132,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo = { + properties: Map[String, String]): SplitInfo = { partition match { case p: GlutenMergeTreePartition => ExtensionTableBuilder diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index 571dc4ba9258..c0f509c68cda 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -771,7 +771,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(1)(plans.head.getSplitInfos(null).size) + assertResult(1)(plans.head.getSplitInfos.size) } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index 85c8c2d92a52..72adee309dd7 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -1807,7 +1807,7 @@ class GlutenClickHouseMergeTreeWriteSuite case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(conf._2)(plans.head.getSplitInfos(null).size) + assertResult(conf._2)(plans.head.getSplitInfos.size) } } }) @@ -1831,7 +1831,7 @@ class GlutenClickHouseMergeTreeWriteSuite case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(1)(plans.head.getSplitInfos(null).size) + assertResult(1)(plans.head.getSplitInfos.size) } } } @@ -1939,7 +1939,7 @@ class GlutenClickHouseMergeTreeWriteSuite case f: BasicScanExecTransformer => f } assertResult(2)(scanExec.size) - assertResult(conf._2)(scanExec(1).getSplitInfos(null).size) + assertResult(conf._2)(scanExec(1).getSplitInfos.size) } } }) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala index 320d1f366c23..061daaac0fad 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala @@ -39,9 +39,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.{ExecutorManager, SerializableConfiguration, SparkDirectoryUtil} - -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.util.{ExecutorManager, SparkDirectoryUtil} import java.lang.{Long => JLong} import java.nio.charset.StandardCharsets @@ -57,8 +55,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo = { + properties: Map[String, String]): SplitInfo = { partition match { case f: FilePartition => val ( @@ -69,7 +66,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { modificationTimes, partitionColumns, metadataColumns) = - constructSplitInfo(partitionSchema, f.files, metadataColumnNames, serializableHadoopConf) + constructSplitInfo(partitionSchema, f.files, metadataColumnNames) val preferredLocations = SoftAffinity.getFilePartitionLocations(f) LocalFilesBuilder.makeLocalFiles( @@ -112,8 +109,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { private def constructSplitInfo( schema: StructType, files: Array[PartitionedFile], - metadataColumnNames: Seq[String], - serializableHadoopConf: SerializableConfiguration) = { + metadataColumnNames: Seq[String]) = { val paths = new JArrayList[String]() val starts = new JArrayList[JLong] val lengths = new JArrayList[JLong]() @@ -125,15 +121,9 @@ class VeloxIteratorApi extends IteratorApi with Logging { file => // The "file.filePath" in PartitionedFile is not the original encoded path, so the decoded // path is incorrect in some cases and here fix the case of ' ' by using GlutenURLDecoder - var filePath = file.filePath.toString - if (filePath.startsWith("viewfs")) { - val viewPath = new Path(filePath) - val viewFileSystem = FileSystem.get(viewPath.toUri, serializableHadoopConf.value) - filePath = viewFileSystem.resolvePath(viewPath).toString - } paths.add( GlutenURLDecoder - .decode(filePath, StandardCharsets.UTF_8.name())) + .decode(file.filePath.toString, StandardCharsets.UTF_8.name())) starts.add(JLong.valueOf(file.start)) lengths.add(JLong.valueOf(file.length)) val (fileSize, modificationTime) = diff --git a/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala b/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala index 10d24c317cc1..1cbeb52a9213 100644 --- a/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala +++ b/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read.{InputPartition, Scan} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration import org.apache.iceberg.spark.source.GlutenIcebergSourceUtil @@ -59,9 +58,7 @@ case class IcebergScanTransformer( override lazy val fileFormat: ReadFileFormat = GlutenIcebergSourceUtil.getFileFormat(scan) - override def getSplitInfosFromPartitions( - partitions: Seq[InputPartition], - serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { + override def getSplitInfosFromPartitions(partitions: Seq[InputPartition]): Seq[SplitInfo] = { val groupedPartitions = SparkShimLoader.getSparkShims.orderPartitions( scan, keyGroupedPartitioning, diff --git a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala index 7f399ce629cf..de71d341db69 100644 --- a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala +++ b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala @@ -128,7 +128,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 3) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) case _ => // do nothing } checkLengthAndPlan(df, 7) @@ -208,7 +208,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 3) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) case _ => // do nothing } checkLengthAndPlan(df, 7) @@ -289,7 +289,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 1) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 1) case _ => // do nothing } checkLengthAndPlan(df, 5) diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java index 04bb9d8cf400..9513f497602a 100644 --- a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java @@ -92,6 +92,11 @@ public List getPaths() { return paths; } + public void setPaths(List newPaths) { + paths.clear(); + paths.addAll(newPaths); + } + public void setFileSchema(StructType schema) { this.fileSchema = schema; } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala index 69c9d37334de..11211bd0da91 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration trait IteratorApi { @@ -38,8 +37,7 @@ trait IteratorApi { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo + properties: Map[String, String]): SplitInfo /** Generate native row partition. */ def genPartitions( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala index d768ac2c5936..73ed35e7190b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} -import org.apache.spark.util.SerializableConfiguration import com.google.protobuf.StringValue import io.substrait.proto.NamedStruct @@ -63,13 +62,11 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource def getProperties: Map[String, String] = Map.empty /** Returns the split infos that will be processed by the underlying native engine. */ - def getSplitInfos(serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { - getSplitInfosFromPartitions(getPartitions, serializableHadoopConf) + def getSplitInfos(): Seq[SplitInfo] = { + getSplitInfosFromPartitions(getPartitions) } - def getSplitInfosFromPartitions( - partitions: Seq[InputPartition], - serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { + def getSplitInfosFromPartitions(partitions: Seq[InputPartition]): Seq[SplitInfo] = { partitions.map( BackendsApiManager.getIteratorApiInstance .genSplitInfo( @@ -77,8 +74,7 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource getPartitionSchema, fileFormat, getMetadataColumns.map(_.name), - getProperties, - serializableHadoopConf)) + getProperties)) } override protected def doValidateInternal(): ValidationResult = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index 70839ffc2eba..beb7fe5f99d2 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -25,7 +25,7 @@ import org.apache.gluten.metrics.{GlutenTimeMetric, MetricsUpdater} import org.apache.gluten.substrait.`type`.{TypeBuilder, TypeNode} import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.plan.{PlanBuilder, PlanNode} -import org.apache.gluten.substrait.rel.{RelNode, SplitInfo} +import org.apache.gluten.substrait.rel.{LocalFilesNode, RelNode, SplitInfo} import org.apache.gluten.utils.SubstraitPlanPrinterUtil import org.apache.spark._ @@ -43,7 +43,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration import com.google.common.collect.Lists +import org.apache.hadoop.fs.{FileSystem, Path} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -127,8 +129,10 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f BackendsApiManager.getMetricsApiInstance.genWholeStageTransformerMetrics(sparkContext) val sparkConf: SparkConf = sparkContext.getConf + val serializableHadoopConf: SerializableConfiguration = new SerializableConfiguration( sparkContext.hadoopConfiguration) + val numaBindingInfo: GlutenNumaBindingInfo = GlutenConfig.getConf.numaBindingInfo @transient @@ -289,10 +293,28 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f */ val allScanPartitions = basicScanExecTransformers.map(_.getPartitions) val allScanSplitInfos = - getSplitInfosFromPartitions( - basicScanExecTransformers, - allScanPartitions, - serializableHadoopConf) + getSplitInfosFromPartitions(basicScanExecTransformers, allScanPartitions) + if (GlutenConfig.getConf.enableHdfsViewfs) { + allScanSplitInfos.foreach { + splitInfos => + splitInfos.foreach { + case splitInfo: LocalFilesNode => + val paths = splitInfo.getPaths.asScala + if (paths.nonEmpty && paths.head.startsWith("viewfs")) { + // Convert the viewfs path into hdfs + val newPaths = paths.map { + viewfsPath => + val viewPath = new Path(viewfsPath) + val viewFileSystem = + FileSystem.get(viewPath.toUri, serializableHadoopConf.value) + viewFileSystem.resolvePath(viewPath).toString + } + splitInfo.setPaths(newPaths.asJava) + } + } + } + } + val inputPartitions = BackendsApiManager.getIteratorApiInstance.genPartitions( wsCtx, @@ -384,8 +406,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f private def getSplitInfosFromPartitions( basicScanExecTransformers: Seq[BasicScanExecTransformer], - allScanPartitions: Seq[Seq[InputPartition]], - serializableHadoopConf: SerializableConfiguration): Seq[Seq[SplitInfo]] = { + allScanPartitions: Seq[Seq[InputPartition]]): Seq[Seq[SplitInfo]] = { // If these are two scan transformers, they must have same partitions, // otherwise, exchange will be inserted. We should combine the two scan // transformers' partitions with same index, and set them together in @@ -404,7 +425,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f val allScanSplitInfos = allScanPartitions.zip(basicScanExecTransformers).map { case (partition, transformer) => - transformer.getSplitInfosFromPartitions(partition, serializableHadoopConf) + transformer.getSplitInfosFromPartitions(partition) } val partitionLength = allScanSplitInfos.head.size if (allScanSplitInfos.exists(_.size != partitionLength)) { diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 5b15faf6466a..107c33a241fd 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -478,6 +478,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableHiveFileFormatWriter: Boolean = conf.getConf(NATIVE_HIVEFILEFORMAT_WRITER_ENABLED) def enableCelebornFallback: Boolean = conf.getConf(CELEBORN_FALLBACK_ENABLED) + + def enableHdfsViewfs: Boolean = conf.getConf(HDFS_VIEWFS_ENABLED) } object GlutenConfig { @@ -2193,4 +2195,11 @@ object GlutenConfig { "Otherwise, do nothing.") .booleanConf .createWithDefault(false) + + val HDFS_VIEWFS_ENABLED = + buildStaticConf("spark.gluten.storage.hdfsViewfs.enabled") + .internal() + .doc("If enabled, gluten will convert the viewfs path to hdfs path in scala side") + .booleanConf + .createWithDefault(false) } From 17d9cd8dfbfa0afc551d71be954553f959ce7f26 Mon Sep 17 00:00:00 2001 From: lgbo Date: Mon, 18 Nov 2024 10:18:43 +0800 Subject: [PATCH 108/211] [GLUTEN-7959][CH] `AdvancedExpandStep` generates less row than expected (#7960) * detect cardinality * fix aggregate params --- .../Operator/AdvancedExpandStep.cpp | 19 ++++++------------- 1 file changed, 6 insertions(+), 13 deletions(-) diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp index 6ac5f5fc8f8b..b777731a9103 100644 --- a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp @@ -115,6 +115,7 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, break; aggregate_grouping_keys.push_back(col.name); } + // partial to partial aggregate DB::Aggregator::Params params( aggregate_grouping_keys, aggregate_descriptions, @@ -122,10 +123,10 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, settings[DB::Setting::max_rows_to_group_by], settings[DB::Setting::group_by_overflow_mode], settings[DB::Setting::group_by_two_level_threshold], - settings[DB::Setting::group_by_two_level_threshold_bytes], - settings[DB::Setting::max_bytes_before_external_group_by], + 0, + 0, settings[DB::Setting::empty_result_for_aggregation_by_empty_set], - context->getTempDataOnDisk(), + nullptr, settings[DB::Setting::max_threads], settings[DB::Setting::min_free_disk_space_for_temporary_data], true, @@ -149,7 +150,7 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, new_processors.push_back(expand_processor); auto expand_output_header = expand_processor->getOutputs().front().getHeader(); - + auto transform_params = std::make_shared(expand_output_header, params, false); auto aggregate_processor = std::make_shared(expand_output_header, transform_params, context, false, false); @@ -188,14 +189,10 @@ AdvancedExpandTransform::AdvancedExpandTransform( , input_header(input_header_) { for (size_t i = 0; i < project_set_exprs.getKinds().size(); ++i) - { is_low_cardinality_expand.push_back(true); - } for (auto & port : outputs) - { output_ports.push_back(&port); - } } DB::IProcessor::Status AdvancedExpandTransform::prepare() @@ -245,9 +242,7 @@ DB::IProcessor::Status AdvancedExpandTransform::prepare() input.setNeeded(); if (!input.hasData()) - { return Status::NeedData; - } input_chunk = input.pull(true); has_input = true; expand_expr_iterator = 0; @@ -265,9 +260,7 @@ void AdvancedExpandTransform::work() has_input = false; } if ((input_finished || cardinality_detect_rows >= rows_for_detect_cardinality) && !cardinality_detect_blocks.empty()) - { detectCardinality(); - } else if (!input_finished && cardinality_detect_rows < rows_for_detect_cardinality) return; @@ -281,7 +274,7 @@ void AdvancedExpandTransform::detectCardinality() std::vector is_col_low_cardinality; for (size_t i = 0; i < grouping_keys; ++i) { - DB::WeakHash32 hash(cardinality_detect_rows); + DB::WeakHash32 hash = block.getByPosition(i).column->getWeakHash32(); std::unordered_set distinct_ids; const auto & data = hash.getData(); for (size_t j = 0; j < cardinality_detect_rows; ++j) From c5c064a8bae9454c6f009a1087efece9b4631f36 Mon Sep 17 00:00:00 2001 From: lgbo Date: Mon, 18 Nov 2024 17:02:24 +0800 Subject: [PATCH 109/211] [GLUTEN-7962][CH] A friendly API to build aggregator params #7963 What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Fixes: #7962 How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- cpp-ch/local-engine/Common/AggregateUtil.cpp | 151 +++++++++++++++--- cpp-ch/local-engine/Common/AggregateUtil.h | 31 +++- .../Operator/AdvancedExpandStep.cpp | 48 +----- .../Parser/RelParsers/AggregateRelParser.cpp | 138 +++------------- 4 files changed, 183 insertions(+), 185 deletions(-) diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp index 851dd2e7fe3d..2290747fa158 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.cpp +++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp @@ -15,8 +15,11 @@ * limitations under the License. */ +#include "AggregateUtil.h" +#include #include #include +#include #include #include #include @@ -26,8 +29,26 @@ namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; +extern const int LOGICAL_ERROR; +extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; +} + +namespace Setting +{ +extern const SettingsUInt64 max_bytes_before_external_group_by; +extern const SettingsBool optimize_group_by_constant_keys; +extern const SettingsUInt64 min_free_disk_space_for_temporary_data; +extern const SettingsMaxThreads max_threads; +extern const SettingsBool empty_result_for_aggregation_by_empty_set; +extern const SettingsUInt64 group_by_two_level_threshold_bytes; +extern const SettingsOverflowModeGroupBy group_by_overflow_mode; +extern const SettingsUInt64 max_rows_to_group_by; +extern const SettingsBool enable_memory_bound_merging_of_aggregation_results; +extern const SettingsUInt64 aggregation_in_order_max_block_bytes; +extern const SettingsUInt64 group_by_two_level_threshold; +extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; +extern const SettingsMaxThreads max_threads; +extern const SettingsUInt64 max_block_size; } template @@ -39,24 +60,23 @@ static Int32 extractMethodBucketsNum(Method & /*method*/) Int32 GlutenAggregatorUtil::getBucketsNum(AggregatedDataVariants & data_variants) { if (!data_variants.isTwoLevel()) - { return 0; - } - + Int32 buckets_num = 0; #define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - buckets_num = extractMethodBucketsNum(*data_variants.NAME); + else if (data_variants.type == AggregatedDataVariants::Type::NAME) buckets_num = extractMethodBucketsNum(*data_variants.NAME); - if (false) {} // NOLINT + if (false) + { + } // NOLINT APPLY_FOR_VARIANTS_TWO_LEVEL(M) #undef M - else - throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); + else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); return buckets_num; } -std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock(Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket) +std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock( + Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket) { if (!variants.isTwoLevel()) return {}; @@ -65,7 +85,7 @@ std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock(Aggregato return aggregator.convertOneBucketToBlock(variants, arena, final, bucket); } -template +template static void releaseOneBucket(Method & method, Int32 bucket) { method.data.impls[bucket].clearAndShrink(); @@ -77,29 +97,26 @@ void GlutenAggregatorUtil::safeReleaseOneBucket(AggregatedDataVariants & variant return; if (bucket >= getBucketsNum(variants)) return; -#define M(NAME) \ - else if (variants.type == AggregatedDataVariants::Type::NAME) \ - releaseOneBucket(*variants.NAME, bucket); +#define M(NAME) else if (variants.type == AggregatedDataVariants::Type::NAME) releaseOneBucket(*variants.NAME, bucket); - if (false) {} // NOLINT + if (false) + { + } // NOLINT APPLY_FOR_VARIANTS_TWO_LEVEL(M) #undef M - else - throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); - + else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); } } namespace local_engine { -AggregateDataBlockConverter::AggregateDataBlockConverter(DB::Aggregator & aggregator_, DB::AggregatedDataVariantsPtr data_variants_, bool final_) +AggregateDataBlockConverter::AggregateDataBlockConverter( + DB::Aggregator & aggregator_, DB::AggregatedDataVariantsPtr data_variants_, bool final_) : aggregator(aggregator_), data_variants(std::move(data_variants_)), final(final_) { if (data_variants->isTwoLevel()) - { buckets_num = DB::GlutenAggregatorUtil::getBucketsNum(*data_variants); - } else if (data_variants->size()) buckets_num = 1; else @@ -168,4 +185,94 @@ DB::Block AggregateDataBlockConverter::next() output_blocks.pop_front(); return block; } + +DB::Aggregator::Params AggregatorParamsHelper::buildParams( + DB::ContextPtr context, + const DB::Names & grouping_keys, + const DB::AggregateDescriptions & agg_descriptions, + Mode mode, + Algorithm algorithm) +{ + const auto & settings = context->getSettingsRef(); + size_t max_rows_to_group_by = mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::max_rows_to_group_by]); + DB::OverflowMode group_by_overflow_mode = settings[DB::Setting::group_by_overflow_mode]; + size_t group_by_two_level_threshold + = algorithm == Algorithm::GlutenGraceAggregate ? static_cast(settings[DB::Setting::group_by_two_level_threshold]) : 0; + size_t group_by_two_level_threshold_bytes = algorithm == Algorithm::GlutenGraceAggregate + ? 0 + : (mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::group_by_two_level_threshold_bytes])); + size_t max_bytes_before_external_group_by = algorithm == Algorithm::GlutenGraceAggregate + ? 0 + : (mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::max_bytes_before_external_group_by])); + bool empty_result_for_aggregation_by_empty_set = algorithm == Algorithm::GlutenGraceAggregate + ? false + : (mode == Mode::PARTIAL_TO_FINISHED ? false : static_cast(settings[DB::Setting::empty_result_for_aggregation_by_empty_set])); + DB::TemporaryDataOnDiskScopePtr tmp_data_scope = algorithm == Algorithm::GlutenGraceAggregate ? nullptr : context->getTempDataOnDisk(); + size_t max_threads = settings[DB::Setting::max_threads]; + size_t min_free_disk_space = algorithm == Algorithm::GlutenGraceAggregate + ? 0 + : static_cast(settings[DB::Setting::min_free_disk_space_for_temporary_data]); + bool compile_aggregate_expressions = mode == Mode::PARTIAL_TO_FINISHED ? false : true; + size_t min_count_to_compile_aggregate_expression = mode == Mode::PARTIAL_TO_FINISHED ? 0 : 3; + size_t max_block_size = PODArrayUtil::adjustMemoryEfficientSize(settings[DB::Setting::max_block_size]); + bool enable_prefetch = mode == Mode::PARTIAL_TO_FINISHED ? false : true; + bool only_merge = mode == Mode::PARTIAL_TO_FINISHED; + bool optimize_group_by_constant_keys + = mode == Mode::PARTIAL_TO_FINISHED ? false : settings[DB::Setting::optimize_group_by_constant_keys]; + double min_hit_rate_to_use_consecutive_keys_optimization = settings[DB::Setting::min_hit_rate_to_use_consecutive_keys_optimization]; + DB::Aggregator::Params params( + grouping_keys, + agg_descriptions, + false, + max_rows_to_group_by, + group_by_overflow_mode, + group_by_two_level_threshold, + group_by_two_level_threshold_bytes, + max_bytes_before_external_group_by, + empty_result_for_aggregation_by_empty_set, + tmp_data_scope, + max_threads, + min_free_disk_space, + compile_aggregate_expressions, + min_count_to_compile_aggregate_expression, + max_block_size, + enable_prefetch, + only_merge, + optimize_group_by_constant_keys, + min_hit_rate_to_use_consecutive_keys_optimization, + {}); + return params; +} + + +#define COMPARE_FIELD(field) \ + if (lhs.field != rhs.field) \ + { \ + LOG_ERROR(getLogger("AggregatorParamsHelper"), "Aggregator::Params field " #field " is not equal. {}/{}", lhs.field, rhs.field); \ + return false; \ + } +bool AggregatorParamsHelper::compare(const DB::Aggregator::Params & lhs, const DB::Aggregator::Params & rhs) +{ + COMPARE_FIELD(overflow_row); + COMPARE_FIELD(max_rows_to_group_by); + COMPARE_FIELD(group_by_overflow_mode); + COMPARE_FIELD(group_by_two_level_threshold); + COMPARE_FIELD(group_by_two_level_threshold_bytes); + COMPARE_FIELD(max_bytes_before_external_group_by); + COMPARE_FIELD(empty_result_for_aggregation_by_empty_set); + COMPARE_FIELD(max_threads); + COMPARE_FIELD(min_free_disk_space); + COMPARE_FIELD(compile_aggregate_expressions); + if ((lhs.tmp_data_scope == nullptr) != (rhs.tmp_data_scope == nullptr)) + { + LOG_ERROR(getLogger("AggregatorParamsHelper"), "Aggregator::Params field tmp_data_scope is not equal."); + return false; + } + COMPARE_FIELD(min_count_to_compile_aggregate_expression); + COMPARE_FIELD(enable_prefetch); + COMPARE_FIELD(only_merge); + COMPARE_FIELD(optimize_group_by_constant_keys); + COMPARE_FIELD(min_hit_rate_to_use_consecutive_keys_optimization); + return true; +} } diff --git a/cpp-ch/local-engine/Common/AggregateUtil.h b/cpp-ch/local-engine/Common/AggregateUtil.h index b14cd59c5490..380e1ea35539 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.h +++ b/cpp-ch/local-engine/Common/AggregateUtil.h @@ -25,7 +25,8 @@ class GlutenAggregatorUtil { public: static Int32 getBucketsNum(AggregatedDataVariants & data_variants); - static std::optional safeConvertOneBucketToBlock(Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket); + static std::optional + safeConvertOneBucketToBlock(Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket); static void safeReleaseOneBucket(AggregatedDataVariants & variants, Int32 bucket); }; } @@ -41,6 +42,7 @@ class AggregateDataBlockConverter ~AggregateDataBlockConverter() = default; bool hasNext(); DB::Block next(); + private: DB::Aggregator & aggregator; DB::AggregatedDataVariantsPtr data_variants; @@ -50,4 +52,31 @@ class AggregateDataBlockConverter Int32 current_bucket = 0; DB::BlocksList output_blocks; }; + +class AggregatorParamsHelper +{ +public: + enum class Algorithm + { + GlutenGraceAggregate, + CHTwoStageAggregate + }; + enum class Mode + { + INIT_TO_PARTIAL, + INIT_TO_COMPLETED, + PARTIAL_TO_PARTIAL, + PARTIAL_TO_FINISHED, + }; + + // for using grace aggregating, never enable ch spill, otherwise there will be data lost. + static DB::Aggregator::Params buildParams( + DB::ContextPtr context, + const DB::Names & grouping_keys, + const DB::AggregateDescriptions & agg_descriptions, + Mode mode, + Algorithm algorithm = Algorithm::GlutenGraceAggregate); + static bool compare(const DB::Aggregator::Params & lhs, const DB::Aggregator::Params & rhs); +}; + } diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp index b777731a9103..ee767b31bd21 100644 --- a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp @@ -16,9 +16,6 @@ */ #include "AdvancedExpandStep.h" -#include -#include -#include #include #include #include @@ -33,33 +30,13 @@ #include #include #include +#include #include #include #include #include -namespace DB -{ -namespace Setting -{ -extern const SettingsUInt64 max_bytes_before_external_group_by; -extern const SettingsBool optimize_group_by_constant_keys; -extern const SettingsUInt64 min_free_disk_space_for_temporary_data; -extern const SettingsMaxThreads max_threads; -extern const SettingsBool empty_result_for_aggregation_by_empty_set; -extern const SettingsUInt64 group_by_two_level_threshold_bytes; -extern const SettingsOverflowModeGroupBy group_by_overflow_mode; -extern const SettingsUInt64 max_rows_to_group_by; -extern const SettingsBool enable_memory_bound_merging_of_aggregation_results; -extern const SettingsUInt64 aggregation_in_order_max_block_bytes; -extern const SettingsUInt64 group_by_two_level_threshold; -extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; -extern const SettingsMaxThreads max_threads; -extern const SettingsUInt64 max_block_size; -} -} - namespace local_engine { @@ -116,27 +93,8 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, aggregate_grouping_keys.push_back(col.name); } // partial to partial aggregate - DB::Aggregator::Params params( - aggregate_grouping_keys, - aggregate_descriptions, - false, - settings[DB::Setting::max_rows_to_group_by], - settings[DB::Setting::group_by_overflow_mode], - settings[DB::Setting::group_by_two_level_threshold], - 0, - 0, - settings[DB::Setting::empty_result_for_aggregation_by_empty_set], - nullptr, - settings[DB::Setting::max_threads], - settings[DB::Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[DB::Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[DB::Setting::optimize_group_by_constant_keys], - settings[DB::Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + auto params = AggregatorParamsHelper::buildParams( + context, aggregate_grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::PARTIAL_TO_PARTIAL); auto input_header = input_headers.front(); auto build_transform = [&](DB::OutputPortRawPtrs outputs) diff --git a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp index 6bc8c7e6e14a..f5498b01ab18 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp @@ -39,19 +39,8 @@ namespace DB { namespace Setting { -extern const SettingsUInt64 max_bytes_before_external_group_by; -extern const SettingsBool optimize_group_by_constant_keys; -extern const SettingsUInt64 min_free_disk_space_for_temporary_data; -extern const SettingsMaxThreads max_threads; -extern const SettingsBool empty_result_for_aggregation_by_empty_set; -extern const SettingsUInt64 group_by_two_level_threshold_bytes; -extern const SettingsOverflowModeGroupBy group_by_overflow_mode; -extern const SettingsUInt64 max_rows_to_group_by; extern const SettingsBool enable_memory_bound_merging_of_aggregation_results; extern const SettingsUInt64 aggregation_in_order_max_block_bytes; -extern const SettingsUInt64 group_by_two_level_threshold; -extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; -extern const SettingsMaxThreads max_threads; extern const SettingsUInt64 max_block_size; } namespace ErrorCodes @@ -184,9 +173,7 @@ void AggregateRelParser::setup(DB::QueryPlanPtr query_plan, const substrait::Rel agg_info.signature_function_name = *parseSignatureFunctionName(measure.measure().function_reference()); auto function_parser = AggregateFunctionParserFactory::instance().get(agg_info.signature_function_name, parser_context); if (!function_parser) - { throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unsupported aggregate function: {}", agg_info.signature_function_name); - } /// Put function_parser, parser_func_info and function_name into agg_info for reducing repeated builds. agg_info.function_parser = function_parser; agg_info.parser_func_info = AggregateFunctionParser::CommonFunctionInfo(measure); @@ -198,16 +185,10 @@ void AggregateRelParser::setup(DB::QueryPlanPtr query_plan, const substrait::Rel if (aggregate_rel->groupings_size() == 1) { for (const auto & expr : aggregate_rel->groupings(0).grouping_expressions()) - { if (expr.has_selection() && expr.selection().has_direct_reference()) - { grouping_keys.push_back(input_header.getByPosition(expr.selection().direct_reference().struct_field().field()).name); - } else - { throw Exception(ErrorCodes::BAD_ARGUMENTS, "unsupported group expression: {}", expr.DebugString()); - } - } } else if (aggregate_rel->groupings_size() != 0) { @@ -345,23 +326,23 @@ void AggregateRelParser::addMergingAggregatedStep() AggregateDescriptions aggregate_descriptions; buildAggregateDescriptions(aggregate_descriptions); const auto & settings = getContext()->getSettingsRef(); - Aggregator::Params params( - grouping_keys, - aggregate_descriptions, - false, - settings[Setting::max_threads], - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization]); auto config = StreamingAggregateConfig::loadFromContext(getContext()); if (config.enable_streaming_aggregating) { - params.group_by_two_level_threshold = settings[Setting::group_by_two_level_threshold]; + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::PARTIAL_TO_FINISHED); auto merging_step = std::make_unique(getContext(), plan->getCurrentHeader(), params, false); steps.emplace_back(merging_step.get()); plan->addStep(std::move(merging_step)); } else { + auto params = AggregatorParamsHelper::buildParams( + getContext(), + grouping_keys, + aggregate_descriptions, + AggregatorParamsHelper::Mode::PARTIAL_TO_FINISHED, + AggregatorParamsHelper::Algorithm::CHTwoStageAggregate); /// We don't use the grouping set feature in CH, so grouping_sets_params_list should always be empty. DB::GroupingSetsParamsList grouping_sets_params_list; auto merging_step = std::make_unique( @@ -389,54 +370,20 @@ void AggregateRelParser::addCompleteModeAggregatedStep() auto config = StreamingAggregateConfig::loadFromContext(getContext()); if (config.enable_streaming_aggregating) { - Aggregator::Params params( - grouping_keys, - aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - settings[Setting::group_by_two_level_threshold_bytes], - 0, /*settings[Setting::max_bytes_before_external_group_by]*/ - settings[Setting::empty_result_for_aggregation_by_empty_set], - getContext()->getTempDataOnDisk(), - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::INIT_TO_COMPLETED); auto merging_step = std::make_unique(getContext(), plan->getCurrentHeader(), params, true); steps.emplace_back(merging_step.get()); plan->addStep(std::move(merging_step)); } else { - Aggregator::Params params( + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - settings[Setting::group_by_two_level_threshold_bytes], - settings[Setting::max_bytes_before_external_group_by], - settings[Setting::empty_result_for_aggregation_by_empty_set], - getContext()->getTempDataOnDisk(), - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + AggregatorParamsHelper::Mode::INIT_TO_COMPLETED, + AggregatorParamsHelper::Algorithm::CHTwoStageAggregate); auto aggregating_step = std::make_unique( plan->getCurrentHeader(), @@ -471,9 +418,7 @@ void AggregateRelParser::addAggregatingStep() { const auto & next_rel = *(rel_stack->back()); if (next_rel.rel_type_case() == substrait::Rel::RelTypeCase::kAggregate) - { is_distinct_aggreate = true; - } } if (config.enable_streaming_aggregating) @@ -484,27 +429,9 @@ void AggregateRelParser::addAggregatingStep() // unreliable. It will appear that a small hash table is converted into a two level structure, resulting in a // lot of small blocks. So we disable this condition, reamain `group_by_two_level_threshold` as the condition to // convert a single level hash table into a two level one. - Aggregator::Params params( - grouping_keys, - aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - 0, // group_by_two_level_threshold_bytes - 0, - settings[Setting::empty_result_for_aggregation_by_empty_set], - nullptr, - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::INIT_TO_PARTIAL); + if (!is_distinct_aggreate) { auto aggregating_step = std::make_unique(getContext(), plan->getCurrentHeader(), params); @@ -532,27 +459,12 @@ void AggregateRelParser::addAggregatingStep() } else { - Aggregator::Params params( + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - settings[Setting::group_by_two_level_threshold_bytes], - settings[Setting::max_bytes_before_external_group_by], - settings[Setting::empty_result_for_aggregation_by_empty_set], - getContext()->getTempDataOnDisk(), - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + AggregatorParamsHelper::Mode::INIT_TO_PARTIAL, + AggregatorParamsHelper::Algorithm::CHTwoStageAggregate); auto aggregating_step = std::make_unique( plan->getCurrentHeader(), @@ -587,12 +499,8 @@ void AggregateRelParser::addPostProjection() for (const auto & agg_info : aggregates) { for (const auto * input_node : project_actions_dag.getInputs()) - { if (input_node->result_name == agg_info.measure_column_name) - { agg_info.function_parser->convertNodeTypeIfNeeded(agg_info.parser_func_info, input_node, project_actions_dag, false); - } - } } } else if (has_complete_stage) @@ -601,12 +509,8 @@ void AggregateRelParser::addPostProjection() for (const auto & agg_info : aggregates) { for (const auto * output_node : project_actions_dag.getOutputs()) - { if (output_node->result_name == agg_info.measure_column_name) - { agg_info.function_parser->convertNodeTypeIfNeeded(agg_info.parser_func_info, output_node, project_actions_dag, true); - } - } } } if (project_actions_dag.dumpDAG() != dag_footprint) From 9d0b9c5c81119ac0cddcc97143b79f8cab044c97 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Mon, 18 Nov 2024 17:38:23 +0800 Subject: [PATCH 110/211] [VL] Clean up some legacy code related to USE_AVX512 (#7956) These are some legacy code inherited from Gazelle. And it's not the correct place to do this setting for compiler if we want it for compiling all native code. --- cpp/CMakeLists.txt | 1 - cpp/core/CMakeLists.txt | 14 -------------- 2 files changed, 15 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index ff934a2a8e14..09d09481f216 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -48,7 +48,6 @@ option(BUILD_EXAMPLES "Build Examples" OFF) option(BUILD_BENCHMARKS "Build Benchmarks" OFF) option(ENABLE_JEMALLOC_STATS "Prints Jemalloc stats for debugging" OFF) option(BUILD_GLOG "Build Glog from Source" OFF) -option(USE_AVX512 "Build with AVX-512 optimizations" OFF) option(ENABLE_HBM "Enable HBM allocator" OFF) option(ENABLE_QAT "Enable QAT for de/compression" OFF) option(ENABLE_IAA "Enable IAA for de/compression" OFF) diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 78422e791a15..94484eab826c 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -66,20 +66,6 @@ macro(find_protobuf) CACHE PATH "Protobuf include path") endmacro() -if(USE_AVX512) - # Only enable additional instruction sets if they are supported - message(STATUS "System processor: ${CMAKE_SYSTEM_PROCESSOR}") - if(CMAKE_SYSTEM_PROCESSOR MATCHES "(x86)|(X86)|(amd64)|(AMD64)") - set(AVX512_FLAG "-march=skylake-avx512") - check_cxx_compiler_flag(${AVX512_FLAG} CXX_SUPPORTS_AVX512) - if(NOT CXX_SUPPORTS_AVX512) - message(FATAL_ERROR "AVX512 required but compiler doesn't support it.") - endif() - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${AVX512_FLAG}") - add_definitions(-DCOLUMNAR_PLUGIN_USE_AVX512) - endif() -endif() - # Set up Proto set(PROTO_OUTPUT_DIR "${CMAKE_CURRENT_BINARY_DIR}/proto") file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/proto) From e34565bb6fd3fcaaa4cd3ccba2461e503765f19f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 18 Nov 2024 22:04:42 +0800 Subject: [PATCH 111/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_18) (#7965) Signed-off-by: glutenperfbot Co-authored-by: glutenperfbot Co-authored-by: Yuan --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 0a1f168db7b5..e70fdad01c4d 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_17 +VELOX_BRANCH=2024_11_18 VELOX_HOME="" OS=`uname -s` From 7e1f497b21da3fe08489644ecbf67e0e81f637f4 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 19 Nov 2024 14:24:11 +0800 Subject: [PATCH 112/211] [GLUTEN-7910][CORE][VL] Flip dependency direction for gluten-iceberg (#7967) Closes #7910 --- backends-velox/pom.xml | 68 +++++++++++++++++++ .../gluten/execution/VeloxIcebergSuite.scala | 20 ++++++ .../execution/VeloxTPCHIcebergSuite.scala | 6 +- gluten-iceberg/pom.xml | 13 ---- ...xIcebergSuite.scala => IcebergSuite.scala} | 5 +- 5 files changed, 94 insertions(+), 18 deletions(-) create mode 100644 backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala rename {gluten-iceberg/src/test => backends-velox/src/test-iceberg}/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala (96%) rename gluten-iceberg/src/test/scala/org/apache/gluten/execution/{VeloxIcebergSuite.scala => IcebergSuite.scala} (98%) diff --git a/backends-velox/pom.xml b/backends-velox/pom.xml index e6900e50dd70..a1fcb52ce763 100755 --- a/backends-velox/pom.xml +++ b/backends-velox/pom.xml @@ -29,6 +29,74 @@ org.apache.gluten.tags.UDFTest + + iceberg + + false + + + + org.apache.gluten + gluten-iceberg + ${project.version} + + + org.apache.gluten + gluten-iceberg + ${project.version} + test-jar + test + + + org.apache.iceberg + iceberg-spark-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + provided + + + org.apache.iceberg + iceberg-spark-runtime-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + test + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-iceberg-sources + generate-sources + + add-source + + + + ${project.basedir}/src/main-iceberg/scala + ${project.basedir}/src/main-iceberg/java + + + + + add-iceberg-test-sources + generate-test-sources + + add-test-source + + + + ${project.basedir}/src/test-iceberg/scala + ${project.basedir}/src/test-iceberg/java + + + + + + + + diff --git a/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala new file mode 100644 index 000000000000..1573042884b2 --- /dev/null +++ b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gluten.execution + +class VeloxIcebergSuite extends IcebergSuite diff --git a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala similarity index 96% rename from gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala rename to backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala index 857cbcb410f8..5bc26e81b7ca 100644 --- a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala +++ b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala @@ -24,13 +24,13 @@ import org.apache.iceberg.spark.SparkWriteOptions import java.io.File class VeloxTPCHIcebergSuite extends VeloxTPCHSuite { - - protected val tpchBasePath: String = new File( - "../backends-velox/src/test/resources").getAbsolutePath + protected val tpchBasePath: String = + getClass.getResource("/").getPath + "../../../src/test/resources" override protected val resourcePath: String = new File(tpchBasePath, "tpch-data-parquet").getCanonicalPath + // FIXME: Unused. override protected val queriesResults: String = new File(tpchBasePath, "queries-output").getCanonicalPath diff --git a/gluten-iceberg/pom.xml b/gluten-iceberg/pom.xml index 728924466002..bad2ec20e8e8 100644 --- a/gluten-iceberg/pom.xml +++ b/gluten-iceberg/pom.xml @@ -50,19 +50,6 @@ test-jar test - - org.apache.gluten - backends-velox - ${project.version} - test - - - org.apache.gluten - backends-velox - ${project.version} - test-jar - test - org.apache.spark spark-core_${scala.binary.version} diff --git a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/IcebergSuite.scala similarity index 98% rename from gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala rename to gluten-iceberg/src/test/scala/org/apache/gluten/execution/IcebergSuite.scala index de71d341db69..f5bf24d1b64f 100644 --- a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala +++ b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/IcebergSuite.scala @@ -21,9 +21,10 @@ import org.apache.gluten.GlutenConfig import org.apache.spark.SparkConf import org.apache.spark.sql.Row -class VeloxIcebergSuite extends WholeStageTransformerSuite { - +abstract class IcebergSuite extends WholeStageTransformerSuite { protected val rootPath: String = getClass.getResource("/").getPath + // FIXME: This folder is in module backends-velox so is not accessible if profile backends-velox + // is not enabled during Maven build. override protected val resourcePath: String = "/tpch-data-parquet" override protected val fileFormat: String = "parquet" From ce45bc4642c6cdc5782951b713a4aaabc0635a06 Mon Sep 17 00:00:00 2001 From: exmy Date: Tue, 19 Nov 2024 14:50:08 +0800 Subject: [PATCH 113/211] [GLUTEN-7983][CH] Fix NPE when disable spark.shuffle.compress (#7984) --- .../apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala index 11c45264dbc4..2263cda7eafb 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala @@ -50,7 +50,8 @@ class CHCelebornColumnarShuffleWriter[K, V]( writeMetrics) { private val capitalizedCompressionCodec = - customizedCompressionCodec.toUpperCase(Locale.ROOT) + if (customizedCompressionCodec != null) customizedCompressionCodec.toUpperCase(Locale.ROOT) + else "NONE" private val jniWrapper = new CHShuffleSplitterJniWrapper From 5dda71ab30a20739a4f8d26b150b5adfcc94af46 Mon Sep 17 00:00:00 2001 From: JiaKe Date: Tue, 19 Nov 2024 15:08:37 +0800 Subject: [PATCH 114/211] [GLUTEN-7887][VL][DOC] Add usage doc about dynamic load jvm libhdfs and native libhdfs3 (#7982) --- docs/developers/MicroBenchmarks.md | 5 +++++ docs/get-started/Velox.md | 7 ++++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/docs/developers/MicroBenchmarks.md b/docs/developers/MicroBenchmarks.md index bd469f34c81c..bc463fcac9aa 100644 --- a/docs/developers/MicroBenchmarks.md +++ b/docs/developers/MicroBenchmarks.md @@ -358,6 +358,11 @@ ShuffleWriteRead/iterations:1/process_time/real_time/threads:1 121637629714 ns Unless `spark.gluten.sql.debug` is set in the INI file via `--conf`, the logging behavior is same as debug mode off. Developers can use `--debug-mode` command line flag to turn on debug mode when needed, and set verbosity/severity level via command line flags `--v` and `--minloglevel`. Note that constructing and deconstructing log strings can be very time-consuming, which may cause benchmark times to be inaccurate. + +## Enable HDFS support + +After enabling the dynamic loading of libhdfs.so at runtime to support HDFS, if you run the benchmark with an HDFS file, you need to set the classpath for Hadoop. You can do this by running `export CLASSPATH=`$HADOOP_HOME/bin/hdfs classpath --glob``. Otherwise, the HDFS connection will fail. If you have replaced ${HADOOP_HOME}/lib/native/libhdfs.so with libhdfs3.so, there is no need to set the `CLASSPATH`. + ## Simulate write tasks The last operator for a write task is a file write operator, and the output from Velox pipeline only diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index 4538f440d2cc..bfaa0200c785 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -120,7 +120,8 @@ shared libraries into another jar named `gluten-thirdparty-lib-$LINUX_OS-$VERSIO ## HDFS support -Hadoop hdfs support is ready via the [libhdfs3](https://github.com/apache/hawq/tree/master/depends/libhdfs3) library. The libhdfs3 provides native API for Hadoop I/O without the drawbacks of JNI. It also provides advanced authentication like Kerberos based. Please note this library has several dependencies which may require extra installations on Driver and Worker node. +Gluten supports dynamically loading both libhdfs.so and libhdfs3.so at runtime by using dlopen, allowing the JVM to load the appropriate shared library file as needed. This means you do not need to set the library path during the compilation phase. +To enable this functionality, you must set the JAVA_HOME and HADOOP_HOME environment variables. Gluten will then locate and load the ${HADOOP_HOME}/lib/native/libhdfs.so file at runtime. If you prefer to use libhdfs3.so instead, simply replace the ${HADOOP_HOME}/lib/native/libhdfs.so file with libhdfs3.so. ### Build with HDFS support @@ -131,7 +132,7 @@ cd /path/to/gluten ./dev/buildbundle-veloxbe.sh --enable_hdfs=ON ``` -### Configuration about HDFS support +### Configuration about HDFS support in Libhdfs3 HDFS uris (hdfs://host:port) will be extracted from a valid hdfs file path to initialize hdfs client, you do not need to specify it explicitly. @@ -172,7 +173,7 @@ You also need to add configuration to the "hdfs-site.xml" as below: ``` -### Kerberos support +### Kerberos support in libhdfs3 Here are two steps to enable kerberos. From 32bf3a21db8e392b49279bef3d18c10b92b8c9c9 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 19 Nov 2024 16:44:53 +0800 Subject: [PATCH 115/211] [GLUTEN-6853][CORE] Move more general query planner APIs from gluten-substrait to gluten-core (#7972) --- .../backendsapi/clickhouse/CHRuleApi.scala | 78 +++-- .../clickhouse/CHSparkPlanExecApi.scala | 6 +- .../FallbackBroadcastHashJoinRules.scala | 2 +- .../backendsapi/velox/VeloxListenerApi.scala | 1 + .../backendsapi/velox/VeloxRuleApi.scala | 173 +++++++--- .../apache/gluten/planner/VeloxRasSuite.scala | 35 +- .../extension/columnar/FallbackTag.scala | 19 -- .../enumerated/EnumeratedApplier.scala | 2 + .../enumerated/EnumeratedTransform.scala | 93 ++++++ .../planner/GlutenOptimization.scala | 8 +- .../enumerated/planner/cost/LongCost.scala | 4 +- .../planner/cost/LongCostModel.scala | 123 ++++++++ .../enumerated/planner/cost/LongCoster.scala | 27 +- .../planner/cost/LongCosterChain.scala | 64 ++++ .../planner/metadata/GlutenMetadata.scala | 2 +- .../metadata/GlutenMetadataModel.scala | 4 +- .../planner/metadata/LogicalLink.scala | 4 +- .../enumerated}/planner/metadata/Schema.scala | 4 +- .../planner/plan/GlutenPlanModel.scala | 6 +- .../enumerated}/planner/property/Conv.scala | 2 +- .../property/GlutenPropertyModel.scala | 2 +- .../columnar/heuristic/AddFallbackTags.scala | 42 +++ .../heuristic/HeuristicTransform.scala | 98 ++++++ .../columnar/heuristic/LegacyOffload.scala | 24 +- .../RewriteSparkPlanRulesManager.scala | 35 +- .../columnar/offload/OffloadSingleNode.scala | 33 +- .../columnar/rewrite/RewriteSingleNode.scala | 7 +- .../columnar/validator/Validator.scala | 89 ++++++ .../extension/injector/GlutenInjector.scala | 88 +++++- .../gluten/extension/injector/Injector.scala | 2 - .../spark/util/SparkReflectionUtil.scala | 7 + .../execution/WriteFilesExecTransformer.scala | 2 +- .../apache/gluten/extension/GlutenPlan.scala | 19 ++ .../EnsureLocalSortRequirements.scala | 2 +- .../enumerated/EnumeratedTransform.scala | 127 -------- .../columnar/enumerated/RasOffload.scala | 25 +- .../planner/cost/LegacyCoster.scala} | 15 +- .../planner/cost/RoughCoster.scala} | 14 +- .../planner/cost/RoughCoster2.scala} | 51 +-- .../columnar/heuristic/AddFallbackTags.scala | 298 ------------------ .../ExpandFallbackPolicy.scala | 19 +- .../heuristic/HeuristicTransform.scala | 55 ---- .../OffloadSingleNodeRules.scala} | 13 +- .../columnar/rewrite/PullOutPostProject.scala | 3 + .../columnar/rewrite/PullOutPreProject.scala | 3 + .../columnar/rewrite/RewriteEligibility.scala | 48 +++ .../columnar/rewrite/RewriteIn.scala | 3 + .../columnar/rewrite/RewriteJoin.scala | 6 +- .../rewrite/RewriteMultiChildrenCount.scala | 4 + .../columnar/validator/Validators.scala | 295 ++++++++++++++--- .../gluten/planner/cost/GlutenCostModel.scala | 48 --- .../gluten/planner/cost/LongCostModel.scala | 55 ---- .../GlutenFormatWriterInjectsBase.scala | 2 +- .../execution/FallbackStrategiesSuite.scala | 4 +- .../execution/FallbackStrategiesSuite.scala | 4 +- .../execution/FallbackStrategiesSuite.scala | 4 +- .../execution/FallbackStrategiesSuite.scala | 4 +- 57 files changed, 1280 insertions(+), 927 deletions(-) rename {gluten-substrait => gluten-core}/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala (88%) create mode 100644 gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/GlutenOptimization.scala (86%) rename gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCost.scala => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCost.scala (87%) create mode 100644 gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala rename gluten-substrait/src/main/scala/org/apache/spark/sql/utils/ReflectionUtil.scala => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCoster.scala (56%) create mode 100644 gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/metadata/GlutenMetadata.scala (94%) rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/metadata/GlutenMetadataModel.scala (92%) rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/metadata/LogicalLink.scala (92%) rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/metadata/Schema.scala (92%) rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/plan/GlutenPlanModel.scala (95%) rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/property/Conv.scala (97%) rename {gluten-substrait/src/main/scala/org/apache/gluten => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated}/planner/property/GlutenPropertyModel.scala (95%) create mode 100644 gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala create mode 100644 gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala rename {gluten-substrait => gluten-core}/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala (66%) rename {gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic}/RewriteSparkPlanRulesManager.scala (81%) rename gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala => gluten-core/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNode.scala (60%) rename {gluten-substrait => gluten-core}/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala (89%) create mode 100644 gluten-core/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala rename gluten-substrait/src/main/scala/org/apache/gluten/{planner/cost/LegacyCostModel.scala => extension/columnar/enumerated/planner/cost/LegacyCoster.scala} (85%) rename gluten-substrait/src/main/scala/org/apache/gluten/{planner/cost/RoughCostModel.scala => extension/columnar/enumerated/planner/cost/RoughCoster.scala} (87%) rename gluten-substrait/src/main/scala/org/apache/gluten/{planner/cost/RoughCostModel2.scala => extension/columnar/enumerated/planner/cost/RoughCoster2.scala} (88%) delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala rename gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/{ => heuristic}/ExpandFallbackPolicy.scala (94%) delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala rename gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/{heuristic/OffloadSingleNode.scala => offload/OffloadSingleNodeRules.scala} (96%) create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteEligibility.scala delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LongCostModel.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index a1b8fbdd5bc3..3525fc768dfd 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -16,12 +16,17 @@ */ package org.apache.gluten.backendsapi.clickhouse +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} -import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform} +import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers} +import org.apache.gluten.extension.columnar.rewrite._ import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} +import org.apache.gluten.extension.columnar.validator.Validator +import org.apache.gluten.extension.columnar.validator.Validators.ValidatorBuilderImplicits import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.parser.{GlutenCacheFilesSqlParser, GlutenClickhouseSqlParser} @@ -45,8 +50,8 @@ class CHRuleApi extends RuleApi { } } -private object CHRuleApi { - def injectSpark(injector: SparkInjector): Unit = { +object CHRuleApi { + private def injectSpark(injector: SparkInjector): Unit = { // Inject the regular Spark rules directly. injector.injectQueryStagePrepRule(FallbackBroadcastHashJoinPrepQueryStage.apply) injector.injectQueryStagePrepRule(spark => CHAQEPropagateEmptyRelation(spark)) @@ -64,36 +69,51 @@ private object CHRuleApi { injector.injectPreCBORule(spark => new CHOptimizeMetadataOnlyDeltaQuery(spark)) } - def injectLegacy(injector: LegacyInjector): Unit = { + private def injectLegacy(injector: LegacyInjector): Unit = { // Legacy: Pre-transform rules. - injector.injectTransform(_ => RemoveTransitions) - injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) - injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) - injector.injectTransform(c => FallbackMultiCodegens.apply(c.session)) - injector.injectTransform(_ => RewriteSubqueryBroadcast()) - injector.injectTransform(c => FallbackBroadcastHashJoin.apply(c.session)) - injector.injectTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session)) - - // Legacy: The Legacy transform rule. - injector.injectTransform(_ => intercept(HeuristicTransform())) + injector.injectPreTransform(_ => RemoveTransitions) + injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) + injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) + injector.injectPreTransform(c => FallbackMultiCodegens.apply(c.session)) + injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) + injector.injectPreTransform(c => FallbackBroadcastHashJoin.apply(c.session)) + injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session)) + + // Legacy: The legacy transform rule. + val validatorBuilder: GlutenConfig => Validator = conf => + Validator + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .fallbackByNativeValidation() + .build() + val rewrites = + Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) + val offloads = Seq(OffloadOthers(), OffloadExchange(), OffloadJoin()) + injector.injectTransform( + c => intercept(HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads))) // Legacy: Post-transform rules. - injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) - injector.injectTransform(c => intercept(RewriteTransformer.apply(c.session))) - injector.injectTransform(_ => PushDownFilterToScan) - injector.injectTransform(_ => PushDownInputFileExpression.PostOffload) - injector.injectTransform(_ => EnsureLocalSortRequirements) - injector.injectTransform(_ => EliminateLocalSort) - injector.injectTransform(_ => CollapseProjectExecTransformer) - injector.injectTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session)) - injector.injectTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session)) - injector.injectTransform(c => LazyAggregateExpandRule.apply(c.session)) - injector.injectTransform( + injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) + injector.injectPostTransform(c => intercept(RewriteTransformer.apply(c.session))) + injector.injectPostTransform(_ => PushDownFilterToScan) + injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload) + injector.injectPostTransform(_ => EnsureLocalSortRequirements) + injector.injectPostTransform(_ => EliminateLocalSort) + injector.injectPostTransform(_ => CollapseProjectExecTransformer) + injector.injectPostTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session)) + injector.injectPostTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session)) + injector.injectPostTransform(c => LazyAggregateExpandRule.apply(c.session)) + injector.injectPostTransform( c => intercept( SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarTransformRules)( c.session))) - injector.injectTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( @@ -105,7 +125,7 @@ private object CHRuleApi { .getExtendedColumnarPostRules() .foreach(each => injector.injectPost(c => intercept(each(c.session)))) injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf)) - injector.injectTransform( + injector.injectPost( c => intercept( SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarPostRules)(c.session))) @@ -116,10 +136,10 @@ private object CHRuleApi { injector.injectFinal(_ => RemoveFallbackTagRule()) } - def injectRas(injector: RasInjector): Unit = { + private def injectRas(injector: RasInjector): Unit = { // CH backend doesn't work with RAS at the moment. Inject a rule that aborts any // execution calls. - injector.inject( + injector.injectPreTransform( _ => new SparkPlanRules.AbortRule( "Clickhouse backend doesn't yet have RAS support, please try disabling RAS and" + diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index f08f21056d14..e413cb8ed470 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -225,7 +225,8 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { // Caller may not know it adds project on top of the shuffle. // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. val project = - HeuristicTransform()(ProjectExec(plan.child.output ++ projectExpressions, plan.child)) + HeuristicTransform.static()( + ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newExprs = Seq[Expression]() for (i <- exprs.indices) { val pos = newExpressionsPosition(i) @@ -250,7 +251,8 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { // Caller may not know it adds project on top of the shuffle. // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. val project = - HeuristicTransform()(ProjectExec(plan.child.output ++ projectExpressions, plan.child)) + HeuristicTransform.static()( + ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newOrderings = Seq[SortOrder]() for (i <- orderings.indices) { val oldOrdering = orderings(i) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala index d1116c857ab6..4af2d0fc0c7b 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala @@ -18,7 +18,7 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.columnar._ +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index e5c9afc096f4..a33fd22812dc 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -158,6 +158,7 @@ class VeloxListenerApi extends ListenerApi with Logging { // Initial native backend with configurations. var parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap) + // Workaround for https://github.com/apache/incubator-gluten/issues/7837 if (isDriver && !inLocalMode(conf)) { parsed += (GlutenConfig.COLUMNAR_VELOX_CACHE_ENABLED.key -> "false") diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 30e92a18b23c..62e3be67ca6b 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -16,19 +16,33 @@ */ package org.apache.gluten.backendsapi.velox +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi import org.apache.gluten.datasource.ArrowConvertorRule import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} -import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform -import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform +import org.apache.gluten.extension.columnar.enumerated.{RasOffload, RemoveSort} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, RoughCoster, RoughCoster2} +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform} +import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers} +import org.apache.gluten.extension.columnar.rewrite._ import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} +import org.apache.gluten.extension.columnar.validator.Validator +import org.apache.gluten.extension.columnar.validator.Validators.ValidatorBuilderImplicits import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.sql.shims.SparkShimLoader -import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFallbackReporter} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.execution.datasources.WriteFilesExec +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase +import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.execution.joins.BaseJoinExec +import org.apache.spark.sql.execution.python.EvalPythonExec +import org.apache.spark.sql.execution.window.WindowExec +import org.apache.spark.sql.hive.HiveTableScanExecTransformer class VeloxRuleApi extends RuleApi { import VeloxRuleApi._ @@ -40,38 +54,53 @@ class VeloxRuleApi extends RuleApi { } } -private object VeloxRuleApi { - def injectSpark(injector: SparkInjector): Unit = { +object VeloxRuleApi { + private def injectSpark(injector: SparkInjector): Unit = { // Inject the regular Spark rules directly. injector.injectOptimizerRule(CollectRewriteRule.apply) injector.injectOptimizerRule(HLLRewriteRule.apply) injector.injectPostHocResolutionRule(ArrowConvertorRule.apply) } - def injectLegacy(injector: LegacyInjector): Unit = { + private def injectLegacy(injector: LegacyInjector): Unit = { // Legacy: Pre-transform rules. - injector.injectTransform(_ => RemoveTransitions) - injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) - injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) - injector.injectTransform(c => FallbackMultiCodegens.apply(c.session)) - injector.injectTransform(_ => RewriteSubqueryBroadcast()) - injector.injectTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) - injector.injectTransform(c => ArrowScanReplaceRule.apply(c.session)) + injector.injectPreTransform(_ => RemoveTransitions) + injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) + injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) + injector.injectPreTransform(c => FallbackMultiCodegens.apply(c.session)) + injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) + injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) + injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session)) - // Legacy: The Legacy transform rule. - injector.injectTransform(_ => HeuristicTransform()) + // Legacy: The legacy transform rule. + val validatorBuilder: GlutenConfig => Validator = conf => + Validator + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .fallbackByNativeValidation() + .build() + val rewrites = + Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) + val offloads = Seq(OffloadOthers(), OffloadExchange(), OffloadJoin()) + injector.injectTransform( + c => HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads)) // Legacy: Post-transform rules. - injector.injectTransform(c => PartialProjectRule.apply(c.session)) - injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) - injector.injectTransform(c => RewriteTransformer.apply(c.session)) - injector.injectTransform(_ => PushDownFilterToScan) - injector.injectTransform(_ => PushDownInputFileExpression.PostOffload) - injector.injectTransform(_ => EnsureLocalSortRequirements) - injector.injectTransform(_ => EliminateLocalSort) - injector.injectTransform(_ => CollapseProjectExecTransformer) - injector.injectTransform(c => FlushableHashAggregateRule.apply(c.session)) - injector.injectTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => PartialProjectRule.apply(c.session)) + injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) + injector.injectPostTransform(c => RewriteTransformer.apply(c.session)) + injector.injectPostTransform(_ => PushDownFilterToScan) + injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload) + injector.injectPostTransform(_ => EnsureLocalSortRequirements) + injector.injectPostTransform(_ => EliminateLocalSort) + injector.injectPostTransform(_ => CollapseProjectExecTransformer) + injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) + injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( @@ -90,37 +119,81 @@ private object VeloxRuleApi { injector.injectFinal(_ => RemoveFallbackTagRule()) } - def injectRas(injector: RasInjector): Unit = { + private def injectRas(injector: RasInjector): Unit = { // Gluten RAS: Pre rules. - injector.inject(_ => RemoveTransitions) - injector.inject(_ => PushDownInputFileExpression.PreOffload) - injector.inject(c => FallbackOnANSIMode.apply(c.session)) - injector.inject(_ => RewriteSubqueryBroadcast()) - injector.inject(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) - injector.inject(c => ArrowScanReplaceRule.apply(c.session)) + injector.injectPreTransform(_ => RemoveTransitions) + injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) + injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) + injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) + injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) + injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session)) // Gluten RAS: The RAS rule. - injector.inject(c => EnumeratedTransform(c.session, c.outputsColumnar)) + val validatorBuilder: GlutenConfig => Validator = conf => + Validator + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .build() + val rewrites = + Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) + injector.injectCoster(_ => LegacyCoster) + injector.injectCoster(_ => RoughCoster) + injector.injectCoster(_ => RoughCoster2) + injector.injectRasRule(_ => RemoveSort) + val offloads: Seq[RasOffload] = Seq( + RasOffload.from[Exchange](OffloadExchange()), + RasOffload.from[BaseJoinExec](OffloadJoin()), + RasOffload.from[FilterExec](OffloadOthers()), + RasOffload.from[ProjectExec](OffloadOthers()), + RasOffload.from[DataSourceV2ScanExecBase](OffloadOthers()), + RasOffload.from[DataSourceScanExec](OffloadOthers()), + RasOffload.from(HiveTableScanExecTransformer.isHiveTableScan(_))(OffloadOthers()), + RasOffload.from[CoalesceExec](OffloadOthers()), + RasOffload.from[HashAggregateExec](OffloadOthers()), + RasOffload.from[SortAggregateExec](OffloadOthers()), + RasOffload.from[ObjectHashAggregateExec](OffloadOthers()), + RasOffload.from[UnionExec](OffloadOthers()), + RasOffload.from[ExpandExec](OffloadOthers()), + RasOffload.from[WriteFilesExec](OffloadOthers()), + RasOffload.from[SortExec](OffloadOthers()), + RasOffload.from[TakeOrderedAndProjectExec](OffloadOthers()), + RasOffload.from[WindowExec](OffloadOthers()), + RasOffload.from(SparkShimLoader.getSparkShims.isWindowGroupLimitExec(_))(OffloadOthers()), + RasOffload.from[LimitExec](OffloadOthers()), + RasOffload.from[GenerateExec](OffloadOthers()), + RasOffload.from[EvalPythonExec](OffloadOthers()), + RasOffload.from[SampleExec](OffloadOthers()) + ) + offloads.foreach( + offload => + injector.injectRasRule( + c => RasOffload.Rule(offload, validatorBuilder(c.glutenConf), rewrites))) // Gluten RAS: Post rules. - injector.inject(_ => RemoveTransitions) - injector.inject(c => PartialProjectRule.apply(c.session)) - injector.inject(_ => RemoveNativeWriteFilesSortAndProject()) - injector.inject(c => RewriteTransformer.apply(c.session)) - injector.inject(_ => PushDownFilterToScan) - injector.inject(_ => PushDownInputFileExpression.PostOffload) - injector.inject(_ => EnsureLocalSortRequirements) - injector.inject(_ => EliminateLocalSort) - injector.inject(_ => CollapseProjectExecTransformer) - injector.inject(c => FlushableHashAggregateRule.apply(c.session)) - injector.inject(c => InsertTransitions(c.outputsColumnar)) - injector.inject(c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext())) + injector.injectPostTransform(_ => RemoveTransitions) + injector.injectPostTransform(c => PartialProjectRule.apply(c.session)) + injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) + injector.injectPostTransform(c => RewriteTransformer.apply(c.session)) + injector.injectPostTransform(_ => PushDownFilterToScan) + injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload) + injector.injectPostTransform(_ => EnsureLocalSortRequirements) + injector.injectPostTransform(_ => EliminateLocalSort) + injector.injectPostTransform(_ => CollapseProjectExecTransformer) + injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) + injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform( + c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext())) SparkShimLoader.getSparkShims .getExtendedColumnarPostRules() - .foreach(each => injector.inject(c => each(c.session))) - injector.inject(c => ColumnarCollapseTransformStages(c.glutenConf)) - injector.inject(c => RemoveGlutenTableCacheColumnarToRow(c.session)) - injector.inject(c => GlutenFallbackReporter(c.glutenConf, c.session)) - injector.inject(_ => RemoveFallbackTagRule()) + .foreach(each => injector.injectPostTransform(c => each(c.session))) + injector.injectPostTransform(c => ColumnarCollapseTransformStages(c.glutenConf)) + injector.injectPostTransform(c => RemoveGlutenTableCacheColumnarToRow(c.session)) + injector.injectPostTransform(c => GlutenFallbackReporter(c.glutenConf, c.session)) + injector.injectPostTransform(_ => RemoveFallbackTagRule()) } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala index 38a6832c4e3f..885b549d0def 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala @@ -17,9 +17,11 @@ package org.apache.gluten.planner import org.apache.gluten.GlutenConfig +import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform +import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, LongCostModel} +import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.planner.cost.GlutenCostModel -import org.apache.gluten.planner.property.Conv import org.apache.gluten.ras.{Cost, CostModel, Ras} import org.apache.gluten.ras.RasSuiteBase._ import org.apache.gluten.ras.path.RasPath @@ -140,23 +142,30 @@ class VeloxRasSuite extends SharedSparkSession { object VeloxRasSuite { def newRas(): Ras[SparkPlan] = { - GlutenOptimization - .builder() - .costModel(GlutenCostModel.find()) - .addRules(List()) - .create() - .asInstanceOf[Ras[SparkPlan]] + newRas(Nil) } - def newRas(RasRules: Seq[RasRule[SparkPlan]]): Ras[SparkPlan] = { + def newRas(rasRules: Seq[RasRule[SparkPlan]]): Ras[SparkPlan] = { GlutenOptimization .builder() - .costModel(GlutenCostModel.find()) - .addRules(RasRules) + .costModel(sessionCostModel()) + .addRules(rasRules) .create() .asInstanceOf[Ras[SparkPlan]] } + private def legacyCostModel(): CostModel[SparkPlan] = { + val registry = LongCostModel.registry() + val coster = LegacyCoster + registry.register(coster) + registry.get(coster.kind()) + } + + private def sessionCostModel(): CostModel[SparkPlan] = { + val transform = EnumeratedTransform.static() + transform.costModel + } + val TRIVIAL_SCHEMA: Seq[AttributeReference] = List(AttributeReference("value", StringType)()) val EMPTY_SCHEMA: Seq[AttributeReference] = List.empty @@ -191,7 +200,7 @@ object VeloxRasSuite { } class UserCostModel1 extends CostModel[SparkPlan] { - private val base = GlutenCostModel.legacy() + private val base = legacyCostModel() override def costOf(node: SparkPlan): Cost = node match { case _: RowUnary => base.makeInfCost() case other => base.costOf(other) @@ -201,7 +210,7 @@ object VeloxRasSuite { } class UserCostModel2 extends CostModel[SparkPlan] { - private val base = GlutenCostModel.legacy() + private val base = legacyCostModel() override def costOf(node: SparkPlan): Cost = node match { case _: ColumnarUnary => base.makeInfCost() case other => base.costOf(other) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala similarity index 88% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala index c2d844c19bc0..ea8f37823b3c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala @@ -16,8 +16,6 @@ */ package org.apache.gluten.extension.columnar -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} - import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} import org.apache.spark.sql.execution.SparkPlan @@ -54,15 +52,6 @@ object FallbackTag { implicit object FromString extends Converter[String] { override def from(reason: String): Option[FallbackTag] = Some(Appendable(reason)) } - - implicit object FromValidationResult extends Converter[ValidationResult] { - override def from(result: ValidationResult): Option[FallbackTag] = { - if (result.ok()) { - return None - } - Some(Appendable(result.reason())) - } - } } } @@ -110,14 +99,6 @@ object FallbackTags { .foreach(mergedTag => plan.setTagValue(TAG, mergedTag)) } - def addRecursively[T](plan: TreeNode[_], t: T)(implicit - converter: FallbackTag.Converter[T]): Unit = { - plan.foreach { - case _: GlutenPlan => // ignore - case other: TreeNode[_] => add(other, t) - } - } - def untag(plan: TreeNode[_]): Unit = { plan.unsetTagValue(TAG) } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala index 7ddeb33c7de5..ff822d1c0a1f 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala @@ -64,3 +64,5 @@ class EnumeratedApplier( } } } + +object EnumeratedApplier {} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala new file mode 100644 index 000000000000..43c5f10bbde3 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.enumerated + +import org.apache.gluten.backend.Backend +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall +import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization +import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv +import org.apache.gluten.extension.columnar.transition.ConventionReq +import org.apache.gluten.extension.injector.Injector +import org.apache.gluten.extension.util.AdaptiveContext +import org.apache.gluten.logging.LogLevelUtil +import org.apache.gluten.ras.CostModel +import org.apache.gluten.ras.property.PropertySet +import org.apache.gluten.ras.rule.RasRule + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ + +/** + * Rule to offload Spark query plan to Gluten query plan using a search algorithm and a defined cost + * model. + * + * The effect of this rule is similar to + * [[org.apache.gluten.extension.columnar.heuristic.HeuristicTransform]], except that the 3 stages + * in the heuristic version, known as rewrite, validate, offload, will take place together + * individually for each Spark query plan node in RAS rule + * [[org.apache.gluten.extension.columnar.enumerated.RasOffload]]. + * + * The feature requires enabling RAS to function. + */ +case class EnumeratedTransform(costModel: CostModel[SparkPlan], rules: Seq[RasRule[SparkPlan]]) + extends Rule[SparkPlan] + with LogLevelUtil { + + private val optimization = { + GlutenOptimization + .builder() + .costModel(costModel) + .addRules(rules) + .create() + } + + private val reqConvention = Conv.any + + private val altConventions = { + val rowBased: Conv = Conv.req(ConventionReq.row) + val backendBatchBased: Conv = Conv.req(ConventionReq.backendBatch) + Seq(rowBased, backendBatchBased) + } + + override def apply(plan: SparkPlan): SparkPlan = { + val constraintSet = PropertySet(List(reqConvention)) + val altConstraintSets = + altConventions.map(altConv => PropertySet(List(altConv))) + val planner = optimization.newPlanner(plan, constraintSet, altConstraintSets) + val out = planner.plan() + out + } +} + +object EnumeratedTransform { + // Creates a static EnumeratedTransform rule for use in certain + // places that requires to emulate the offloading of a Spark query plan. + // + // TODO: Avoid using this and eventually remove the API. + def static(): EnumeratedTransform = { + val exts = new SparkSessionExtensions() + val dummyInjector = new Injector(exts) + Backend.get().injectRules(dummyInjector) + val session = SparkSession.getActiveSession.getOrElse( + throw new GlutenException( + "HeuristicTransform#static can only be called when an active Spark session exists")) + val call = new ColumnarRuleCall(session, AdaptiveContext(session), false) + dummyInjector.gluten.ras.createEnumeratedTransform(call) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/GlutenOptimization.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/GlutenOptimization.scala similarity index 86% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/GlutenOptimization.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/GlutenOptimization.scala index 5b24f596395d..46bec0c59493 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/GlutenOptimization.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/GlutenOptimization.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner +package org.apache.gluten.extension.columnar.enumerated.planner -import org.apache.gluten.planner.metadata.GlutenMetadataModel -import org.apache.gluten.planner.plan.GlutenPlanModel -import org.apache.gluten.planner.property.GlutenPropertyModel +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadataModel +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel +import org.apache.gluten.extension.columnar.enumerated.planner.property.GlutenPropertyModel import org.apache.gluten.ras.{CostModel, Optimization, RasExplain} import org.apache.gluten.ras.rule.RasRule diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCost.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCost.scala similarity index 87% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCost.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCost.scala index 72c1cb8b430b..aa74f7736fbd 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCost.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCost.scala @@ -14,8 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.ras.Cost -case class GlutenCost(value: Long) extends Cost +case class LongCost(value: Long) extends Cost diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala new file mode 100644 index 000000000000..1cfe132d84ed --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.enumerated.planner.cost + +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel.GroupLeafExec +import org.apache.gluten.ras.{Cost, CostModel} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.SparkPlan + +import scala.collection.mutable + +abstract class LongCostModel extends CostModel[SparkPlan] { + private val infLongCost = Long.MaxValue + + override def costOf(node: SparkPlan): LongCost = node match { + case _: GroupLeafExec => throw new IllegalStateException() + case _ => LongCost(longCostOf(node)) + } + + private def longCostOf(node: SparkPlan): Long = node match { + case n => + val selfCost = selfLongCostOf(n) + + // Sum with ceil to avoid overflow. + def safeSum(a: Long, b: Long): Long = { + assert(a >= 0) + assert(b >= 0) + val sum = a + b + if (sum < a || sum < b) Long.MaxValue else sum + } + + (n.children.map(longCostOf).toList :+ selfCost).reduce(safeSum) + } + + def selfLongCostOf(node: SparkPlan): Long + + override def costComparator(): Ordering[Cost] = Ordering.Long.on { + case LongCost(value) => value + case _ => throw new IllegalStateException("Unexpected cost type") + } + + override def makeInfCost(): Cost = LongCost(infLongCost) +} + +object LongCostModel extends Logging { + def registry(): Registry = { + new Registry() + } + + /** + * Kind of a cost model. Output of #name() will be used as alias to identify the cost model + * instance from the registry. + */ + sealed trait Kind { + import Kind._ + values.synchronized { + val n = name() + if (values.contains(n)) { + throw new GlutenException(s"Cost mode kind $n already registered") + } + values += n -> this + } + + def name(): String + } + + object Kind { + val values: mutable.Map[String, Kind] = mutable.Map() + } + + /** + * A cost model that is supposed to drive RAS planner create the same query plan with legacy + * planner. + */ + case object Legacy extends Kind { + override def name(): String = "legacy" + } + + /** A rough cost model with some empirical heuristics. */ + case object Rough extends Kind { + override def name(): String = "rough" + } + + /** Compared with rough, rough2 can be more precise to avoid the costly r2c. */ + case object Rough2 extends Kind { + override def name(): String = "rough2" + } + + class Registry private[LongCostModel] { + private val lookup: mutable.Map[Kind, LongCosterChain.Builder] = mutable.Map() + + // The registered coster will take lower precedence than all the existing + // registered costers in cost estimation. + def register(coster: LongCoster): Unit = synchronized { + val chainBuilder = builderOf(coster.kind()) + chainBuilder.register(coster) + } + + def get(kind: Kind): LongCostModel = synchronized { + builderOf(kind).build() + } + + private def builderOf(kind: Kind): LongCosterChain.Builder = { + lookup.getOrElseUpdate(kind, LongCosterChain.builder()) + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/utils/ReflectionUtil.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCoster.scala similarity index 56% rename from gluten-substrait/src/main/scala/org/apache/spark/sql/utils/ReflectionUtil.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCoster.scala index f674e7ec2e70..f06d1a4db829 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/utils/ReflectionUtil.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCoster.scala @@ -14,15 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.utils +package org.apache.gluten.extension.columnar.enumerated.planner.cost -import org.apache.spark.util.Utils +import org.apache.spark.sql.execution.SparkPlan -object ReflectionUtil { - def classForName[C]( - className: String, - initialize: Boolean = true, - noSparkClassLoader: Boolean = false): Class[C] = { - Utils.classForName(className, initialize, noSparkClassLoader) - } +/** + * Costs one single Spark plan node. The coster returns none if the input plan node is not + * recognizable. + * + * Used by the composite cost model [[LongCosterChain]]. + */ +trait LongCoster { + + /** The coster will be registered as part of the cost model associated with this kind. */ + def kind(): LongCostModel.Kind + + /** + * Calculates the long integer cost of the input query plan node. Note, this calculation should + * omit children's costs. + */ + def selfCostOf(node: SparkPlan): Option[Long] } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala new file mode 100644 index 000000000000..8b0c8b9f2d8a --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.enumerated.planner.cost +import org.apache.gluten.exception.GlutenException + +import org.apache.spark.sql.execution.SparkPlan + +import scala.collection.mutable + +/** + * A [[LongCostModel]] implementation that consists of a set of sub-costers. + * + * The costers will apply in the same order they were registered or added. + */ +private class LongCosterChain private (costers: Seq[LongCoster]) extends LongCostModel { + override def selfLongCostOf(node: SparkPlan): Long = { + // Applies the costers respectively, returns when a coster gives a meaningful non-none number. + // If all costers give none, throw an error. + costers + .foldLeft[Option[Long]](None) { + case (None, coster) => + coster.selfCostOf(node) + case (c @ Some(_), _) => + c + } + .getOrElse(throw new GlutenException(s"Cost node found for node: $node")) + } +} + +object LongCosterChain { + def builder(): Builder = new Builder() + + class Builder private[LongCosterChain] { + private val costers = mutable.ListBuffer[LongCoster]() + private var out: Option[LongCosterChain] = None + + def register(coster: LongCoster): Builder = synchronized { + costers += coster + out = None + this + } + + private[cost] def build(): LongCosterChain = synchronized { + if (out.isEmpty) { + out = Some(new LongCosterChain(costers.toSeq)) + } + return out.get + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadata.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadata.scala similarity index 94% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadata.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadata.scala index f66c5290e95f..fabf6ac1fc79 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadata.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadata.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata import org.apache.gluten.ras.Metadata diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadataModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadataModel.scala similarity index 92% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadataModel.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadataModel.scala index 7b95f1383d04..690964daa68d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadataModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadataModel.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata -import org.apache.gluten.planner.plan.GlutenPlanModel.GroupLeafExec +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel.GroupLeafExec import org.apache.gluten.ras.{Metadata, MetadataModel} import org.apache.spark.internal.Logging diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/LogicalLink.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/LogicalLink.scala similarity index 92% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/LogicalLink.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/LogicalLink.scala index 4c3bffd471ad..1886248e9f4e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/LogicalLink.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/LogicalLink.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata -import org.apache.gluten.planner.metadata.GlutenMetadataModel.Verifier +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadataModel.Verifier import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/Schema.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/Schema.scala similarity index 92% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/Schema.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/Schema.scala index 969d34d5cc82..1e8ff1ae59db 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/Schema.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/Schema.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata -import org.apache.gluten.planner.metadata.GlutenMetadataModel.Verifier +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadataModel.Verifier import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala similarity index 95% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala index abf7908a3872..fa92eacd4d47 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.plan +package org.apache.gluten.extension.columnar.enumerated.planner.plan +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadata +import org.apache.gluten.extension.columnar.enumerated.planner.property.{Conv, ConvDef} import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.gluten.extension.columnar.transition.Convention.{KnownBatchType, KnownRowType} -import org.apache.gluten.planner.metadata.GlutenMetadata -import org.apache.gluten.planner.property.{Conv, ConvDef} import org.apache.gluten.ras.{Metadata, PlanModel} import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.sql.shims.SparkShimLoader diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/Conv.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala similarity index 97% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/property/Conv.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala index 78f290d19555..e9ca836eee8a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/Conv.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.property +package org.apache.gluten.extension.columnar.enumerated.planner.property import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq, Transition} import org.apache.gluten.ras.{Property, PropertyDef} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/GlutenPropertyModel.scala similarity index 95% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/GlutenPropertyModel.scala index 115ab4471fb6..bc7014f0fa1b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/GlutenPropertyModel.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.property +package org.apache.gluten.extension.columnar.enumerated.planner.property import org.apache.gluten.ras._ diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala new file mode 100644 index 000000000000..b3458e1385c8 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.extension.columnar.FallbackTags +import org.apache.gluten.extension.columnar.validator.Validator + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +// Add fallback tags when validator returns negative outcome. +case class AddFallbackTags(validator: Validator) extends Rule[SparkPlan] { + def apply(plan: SparkPlan): SparkPlan = { + plan.foreachUp { case p => addFallbackTag(p) } + plan + } + + private def addFallbackTag(plan: SparkPlan): Unit = { + val outcome = validator.validate(plan) + outcome match { + case Validator.Failed(reason) => + FallbackTags.add(plan, reason) + case Validator.Passed => + } + } +} + +object AddFallbackTags {} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala new file mode 100644 index 000000000000..aa13cd958cb7 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.backend.Backend +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall +import org.apache.gluten.extension.columnar.offload.OffloadSingleNode +import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode +import org.apache.gluten.extension.columnar.validator.Validator +import org.apache.gluten.extension.injector.Injector +import org.apache.gluten.extension.util.AdaptiveContext +import org.apache.gluten.logging.LogLevelUtil + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +/** + * Rule to offload Spark query plan to Gluten query plan using programed heuristics. + * + * The procedure consists of 3 stages: + * + * 1. Rewrite 2. Validate 3. Offload + * + * In the rewrite stage, planner will try converting the Spark query plan to various forms of + * possible alternative Spark query plans, then choose the optimal one to send to next stage. During + * which, the same validation code that is about to be used in stage 2 might be invoked early to + * predict on the estimate "cost" of an alternative Spark query plan. + * + * Once the plan is rewritten, query planner will call native validation code in stage 2 to + * determine which part of the plan is offload-able or not, then add fallback tags to the + * non-offload-able parts. + * + * In stage 3, query planner will convert the offload-able Spark plan nodes into Gluten plan nodes. + */ +class HeuristicTransform private (all: Seq[Rule[SparkPlan]]) + extends Rule[SparkPlan] + with LogLevelUtil { + override def apply(plan: SparkPlan): SparkPlan = { + all.foldLeft(plan) { + case (plan, single) => + single(plan) + } + } +} + +object HeuristicTransform { + def withRules(all: Seq[Rule[SparkPlan]]): HeuristicTransform = { + new HeuristicTransform(all) + } + + case class Single( + validator: Validator, + rewriteRules: Seq[RewriteSingleNode], + offloadRules: Seq[OffloadSingleNode]) + extends Rule[SparkPlan] { + private val validate = AddFallbackTags(validator) + private val rewrite = RewriteSparkPlanRulesManager(validate, rewriteRules) + private val offload = LegacyOffload(offloadRules) + + override def apply(plan: SparkPlan): SparkPlan = { + Seq(rewrite, validate, offload).foldLeft(plan) { + case (plan, stage) => + stage(plan) + } + } + } + + // Creates a static HeuristicTransform rule for use in certain + // places that requires to emulate the offloading of a Spark query plan. + // + // TODO: Avoid using this and eventually remove the API. + def static(): HeuristicTransform = { + val exts = new SparkSessionExtensions() + val dummyInjector = new Injector(exts) + Backend.get().injectRules(dummyInjector) + val session = SparkSession.getActiveSession.getOrElse( + throw new GlutenException( + "HeuristicTransform#static can only be called when an active Spark session exists")) + val call = new ColumnarRuleCall(session, AdaptiveContext(session), false) + dummyInjector.gluten.legacy.createHeuristicTransform(call) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala similarity index 66% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala index 9cf08ff303cd..9249e6ebf742 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala @@ -16,33 +16,23 @@ */ package org.apache.gluten.extension.columnar.heuristic +import org.apache.gluten.extension.columnar.offload.OffloadSingleNode import org.apache.gluten.logging.LogLevelUtil import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan -class LegacyOffload(topDownRules: Seq[OffloadSingleNode], bottomUpRules: Seq[OffloadSingleNode]) - extends Rule[SparkPlan] - with LogLevelUtil { +class LegacyOffload(rules: Seq[OffloadSingleNode]) extends Rule[SparkPlan] with LogLevelUtil { def apply(plan: SparkPlan): SparkPlan = { - val plan0 = - topDownRules.foldLeft(plan)((p, rule) => p.transformDown { case p => rule.offload(p) }) - val plan1 = - bottomUpRules.foldLeft(plan0)((p, rule) => p.transformUp { case p => rule.offload(p) }) - plan1 + val out = + rules.foldLeft(plan)((p, rule) => p.transformUp { case p => rule.offload(p) }) + out } } object LegacyOffload { - def apply(): LegacyOffload = { - new LegacyOffload( - List(), - List( - OffloadOthers(), - OffloadExchange(), - OffloadJoin() - ) - ) + def apply(rules: Seq[OffloadSingleNode]): LegacyOffload = { + new LegacyOffload(rules) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/RewriteSparkPlanRulesManager.scala similarity index 81% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/RewriteSparkPlanRulesManager.scala index 3b8ca0df3001..59a370059015 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/RewriteSparkPlanRulesManager.scala @@ -14,21 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar.rewrite +package org.apache.gluten.extension.columnar.heuristic import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} -import org.apache.gluten.sql.shims.SparkShimLoader +import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.aggregate.BaseAggregateExec -import org.apache.spark.sql.execution.joins.BaseJoinExec -import org.apache.spark.sql.execution.python.ArrowEvalPythonExec -import org.apache.spark.sql.execution.window.WindowExec +import org.apache.spark.sql.execution.{LeafExecNode, ProjectExec, SparkPlan} case class RewrittenNodeWall(originalChild: SparkPlan) extends LeafExecNode { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() @@ -51,27 +47,14 @@ class RewriteSparkPlanRulesManager private ( extends Rule[SparkPlan] { private def mayNeedRewrite(plan: SparkPlan): Boolean = { - FallbackTags.maybeOffloadable(plan) && { - plan match { - case _: SortExec => true - case _: TakeOrderedAndProjectExec => true - case _: BaseAggregateExec => true - case _: BaseJoinExec => true - case _: WindowExec => true - case _: FilterExec => true - case _: FileSourceScanExec => true - case _: ExpandExec => true - case _: GenerateExec => true - case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => true - case _: ArrowEvalPythonExec => true - case _ => false - } - } + FallbackTags.maybeOffloadable(plan) && rewriteRules.forall(_.isRewritable(plan)) } private def getFallbackTagBack(rewrittenPlan: SparkPlan): Option[FallbackTag] = { // The rewritten plan may contain more nodes than origin, for now it should only be // `ProjectExec`. + // TODO: Find a better approach than checking `p.isInstanceOf[ProjectExec]` which is not + // general. val target = rewrittenPlan.collect { case p if !p.isInstanceOf[ProjectExec] && !p.isInstanceOf[RewrittenNodeWall] => p } @@ -137,7 +120,9 @@ class RewriteSparkPlanRulesManager private ( } object RewriteSparkPlanRulesManager { - def apply(validateRule: Rule[SparkPlan]): Rule[SparkPlan] = { - new RewriteSparkPlanRulesManager(validateRule, RewriteSingleNode.allRules()) + def apply( + validateRule: Rule[SparkPlan], + rewriteRules: Seq[RewriteSingleNode]): Rule[SparkPlan] = { + new RewriteSparkPlanRulesManager(validateRule, rewriteRules) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNode.scala similarity index 60% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNode.scala index 000dbac7ba47..232973f53a5d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNode.scala @@ -14,29 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar.validator +package org.apache.gluten.extension.columnar.offload +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan -trait Validator { - import Validator._ - def validate(plan: SparkPlan): OutCome - - final def pass(): OutCome = { - Passed - } - - final def fail(p: SparkPlan): OutCome = { - Validator.Failed(s"[${getClass.getSimpleName}] Validation failed on node ${p.nodeName}") - } - - final def fail(reason: String): OutCome = { - Validator.Failed(reason) - } -} - -object Validator { - sealed trait OutCome - case object Passed extends OutCome - case class Failed private (reason: String) extends OutCome +/** + * Converts a vanilla Spark plan node into Gluten plan node. Gluten plan is supposed to be executed + * in native, and the internals of execution is subject by backend's implementation. + * + * Note: Only the current plan node is supposed to be open to modification. Do not access or modify + * the children node. Tree-walking is done by caller of this trait. + */ +trait OffloadSingleNode extends Logging { + def offload(plan: SparkPlan): SparkPlan } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala similarity index 89% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala index 551cfd599abd..38a8031a5cab 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala @@ -30,11 +30,6 @@ import org.apache.spark.sql.execution.SparkPlan * TODO: Ideally for such API we'd better to allow multiple alternative outputs. */ trait RewriteSingleNode { + def isRewritable(plan: SparkPlan): Boolean def rewrite(plan: SparkPlan): SparkPlan } - -object RewriteSingleNode { - def allRules(): Seq[RewriteSingleNode] = { - Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) - } -} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala new file mode 100644 index 000000000000..105dcb5db4aa --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.validator + +import org.apache.spark.sql.execution.SparkPlan + +import scala.collection.mutable +import scala.collection.mutable.ListBuffer + +trait Validator { + import Validator._ + def validate(plan: SparkPlan): OutCome + + final def pass(): OutCome = { + Passed + } + + final def fail(p: SparkPlan): OutCome = { + Validator.Failed(s"[${getClass.getSimpleName}] Validation failed on node ${p.nodeName}") + } + + final def fail(reason: String): OutCome = { + Validator.Failed(reason) + } +} + +object Validator { + sealed trait OutCome + case object Passed extends OutCome + case class Failed private (reason: String) extends OutCome + + def builder(): Builder = Builder() + + class Builder private { + private val buffer: ListBuffer[Validator] = mutable.ListBuffer() + + /** Add a custom validator to pipeline. */ + def add(validator: Validator): Builder = { + buffer += validator + this + } + + def build(): Validator = { + if (buffer.isEmpty) { + return NoopValidator + } + if (buffer.size == 1) { + return buffer.head + } + new ValidatorPipeline(buffer.toSeq) + } + + private class ValidatorPipeline(validators: Seq[Validator]) extends Validator { + override def validate(plan: SparkPlan): Validator.OutCome = { + val init: Validator.OutCome = pass() + val finalOut = validators.foldLeft(init) { + case (out, validator) => + out match { + case Validator.Passed => validator.validate(plan) + case Validator.Failed(_) => out + } + } + finalOut + } + } + + private object NoopValidator extends Validator { + override def validate(plan: SparkPlan): Validator.OutCome = pass() + } + } + + private object Builder { + def apply(): Builder = new Builder() + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala index 498f040c9075..89314dfcf37f 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala @@ -20,12 +20,17 @@ import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.GlutenColumnarRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall -import org.apache.gluten.extension.columnar.enumerated.EnumeratedApplier -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier +import org.apache.gluten.extension.columnar.enumerated.{EnumeratedApplier, EnumeratedTransform} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LongCoster, LongCostModel} +import org.apache.gluten.extension.columnar.heuristic.{HeuristicApplier, HeuristicTransform} +import org.apache.gluten.ras.CostModel +import org.apache.gluten.ras.rule.RasRule +import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.util.SparkReflectionUtil import scala.collection.mutable @@ -51,15 +56,25 @@ class GlutenInjector private[injector] (control: InjectorControl) { object GlutenInjector { class LegacyInjector { + private val preTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val transformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + private val postTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val fallbackPolicyBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val postBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val finalBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + def injectPreTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + preTransformBuilders += builder + } + def injectTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { transformBuilders += builder } + def injectPostTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + postTransformBuilders += builder + } + def injectFallbackPolicy(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { fallbackPolicyBuilders += builder } @@ -75,22 +90,77 @@ object GlutenInjector { private[injector] def createApplier(session: SparkSession): ColumnarRuleApplier = { new HeuristicApplier( session, - transformBuilders.toSeq, + (preTransformBuilders ++ Seq( + c => createHeuristicTransform(c)) ++ postTransformBuilders).toSeq, fallbackPolicyBuilders.toSeq, postBuilders.toSeq, - finalBuilders.toSeq) + finalBuilders.toSeq + ) + } + + def createHeuristicTransform(call: ColumnarRuleCall): HeuristicTransform = { + val all = transformBuilders.map(_(call)) + HeuristicTransform.withRules(all.toSeq) } } - class RasInjector { - private val ruleBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + class RasInjector extends Logging { + private val preTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + private val rasRuleBuilders = mutable.Buffer.empty[ColumnarRuleCall => RasRule[SparkPlan]] + private val costerBuilders = mutable.Buffer.empty[ColumnarRuleCall => LongCoster] + private val postTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + + def injectPreTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + preTransformBuilders += builder + } + + def injectRasRule(builder: ColumnarRuleCall => RasRule[SparkPlan]): Unit = { + rasRuleBuilders += builder + } - def inject(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { - ruleBuilders += builder + def injectCoster(builder: ColumnarRuleCall => LongCoster): Unit = { + costerBuilders += builder + } + + def injectPostTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + postTransformBuilders += builder } private[injector] def createApplier(session: SparkSession): ColumnarRuleApplier = { - new EnumeratedApplier(session, ruleBuilders.toSeq) + new EnumeratedApplier( + session, + (preTransformBuilders ++ Seq( + c => createEnumeratedTransform(c)) ++ postTransformBuilders).toSeq) + } + + def createEnumeratedTransform(call: ColumnarRuleCall): EnumeratedTransform = { + // Build RAS rules. + val rules = rasRuleBuilders.map(_(call)) + + // Build the cost model. + val costModelRegistry = LongCostModel.registry() + costerBuilders.foreach(cb => costModelRegistry.register(cb(call))) + val aliasOrClass = call.glutenConf.rasCostModel + val costModel = findCostModel(costModelRegistry, aliasOrClass) + + // Create transform. + EnumeratedTransform(costModel, rules.toSeq) + } + + private def findCostModel( + registry: LongCostModel.Registry, + aliasOrClass: String): CostModel[SparkPlan] = { + if (LongCostModel.Kind.values.contains(aliasOrClass)) { + val kind = LongCostModel.Kind.values(aliasOrClass) + val model = registry.get(kind) + return model + } + val clazz = SparkReflectionUtil.classForName(aliasOrClass) + logInfo(s"Using user cost model: $aliasOrClass") + val ctor = clazz.getDeclaredConstructor() + ctor.setAccessible(true) + val model: CostModel[SparkPlan] = ctor.newInstance() + model } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala index 81b5239a8aca..52a0c1580a7f 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala @@ -30,5 +30,3 @@ class Injector(extensions: SparkSessionExtensions) { gluten.inject(extensions) } } - -object Injector {} diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala index 40692346e052..ed243abb7f2b 100644 --- a/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala @@ -20,4 +20,11 @@ object SparkReflectionUtil { def getSimpleClassName(cls: Class[_]): String = { Utils.getSimpleName(cls) } + + def classForName[C]( + className: String, + initialize: Boolean = true, + noSparkClassLoader: Boolean = false): Class[C] = { + Utils.classForName(className, initialize, noSparkClassLoader) + } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala index c93d2dda5121..c69623b06193 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala @@ -20,8 +20,8 @@ import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel.GroupLeafExec import org.apache.gluten.metrics.MetricsUpdater -import org.apache.gluten.planner.plan.GlutenPlanModel.GroupLeafExec import org.apache.gluten.substrait.`type`.ColumnTypeNode import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.extensions.ExtensionBuilder diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala index c252bacc9a18..a6ca9b1dcc5c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala @@ -21,8 +21,10 @@ import org.apache.gluten.backend.Backend import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.expression.TransformerState +import org.apache.gluten.extension.columnar.FallbackTag import org.apache.gluten.extension.columnar.FallbackTags.add import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.validator.Validator import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.plan.PlanBuilder @@ -32,6 +34,7 @@ import org.apache.gluten.test.TestStats import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.SparkPlan +import FallbackTag.{Appendable, Converter} import com.google.common.collect.Lists sealed trait ValidationResult { @@ -40,6 +43,15 @@ sealed trait ValidationResult { } object ValidationResult { + implicit object FromValidationResult extends Converter[ValidationResult] { + override def from(result: ValidationResult): Option[FallbackTag] = { + if (result.ok()) { + return None + } + Some(Appendable(result.reason())) + } + } + private case object Succeeded extends ValidationResult { override def ok(): Boolean = true override def reason(): String = throw new UnsupportedOperationException( @@ -60,6 +72,13 @@ object ValidationResult { } add(plan, result) } + + def toValidatorOutcome(): Validator.OutCome = { + if (result.ok()) { + return Validator.Passed + } + Validator.Failed(result.reason()) + } } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala index 73dc617e0853..29a765288572 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.{SortExec, SparkPlan} * SortAggregate with the same key. So, this rule adds local sort back if necessary. */ object EnsureLocalSortRequirements extends Rule[SparkPlan] { - private lazy val transform = HeuristicTransform() + private lazy val transform = HeuristicTransform.static() private def addLocalSort( originalChild: SparkPlan, diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala deleted file mode 100644 index 5abc64ec374f..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.gluten.extension.columnar.enumerated - -import org.apache.gluten.extension.columnar.heuristic.{OffloadExchange, OffloadJoin, OffloadOthers} -import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.extension.columnar.validator.{Validator, Validators} -import org.apache.gluten.logging.LogLevelUtil -import org.apache.gluten.planner.GlutenOptimization -import org.apache.gluten.planner.cost.GlutenCostModel -import org.apache.gluten.planner.property.Conv -import org.apache.gluten.ras.property.PropertySet -import org.apache.gluten.sql.shims.SparkShimLoader - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.datasources.WriteFilesExec -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase -import org.apache.spark.sql.execution.exchange.Exchange -import org.apache.spark.sql.execution.joins.BaseJoinExec -import org.apache.spark.sql.execution.python.EvalPythonExec -import org.apache.spark.sql.execution.window.WindowExec -import org.apache.spark.sql.hive.HiveTableScanExecTransformer - -/** - * Rule to offload Spark query plan to Gluten query plan using a search algorithm and a defined cost - * model. - * - * The effect of this rule is similar to - * [[org.apache.gluten.extension.columnar.heuristic.HeuristicTransform]], except that the 3 stages - * in the heuristic version, known as rewrite, validate, offload, will take place together - * individually for each Spark query plan node in RAS rule - * [[org.apache.gluten.extension.columnar.enumerated.RasOffload]]. - * - * The feature requires enabling RAS to function. - */ -case class EnumeratedTransform(session: SparkSession, outputsColumnar: Boolean) - extends Rule[SparkPlan] - with LogLevelUtil { - - private val validator: Validator = Validators - .builder() - .fallbackByHint() - .fallbackIfScanOnly() - .fallbackComplexExpressions() - .fallbackByBackendSettings() - .fallbackByUserOptions() - .fallbackByTestInjects() - .build() - - private val rules = List( - RemoveSort - ) - - // TODO: Should obey ReplaceSingleNode#applyScanNotTransformable to select - // (vanilla) scan with cheaper sub-query plan through cost model. - private val offloadRules = - Seq( - RasOffload.from[Exchange](OffloadExchange()), - RasOffload.from[BaseJoinExec](OffloadJoin()), - RasOffload.from[FilterExec](OffloadOthers()), - RasOffload.from[ProjectExec](OffloadOthers()), - RasOffload.from[DataSourceV2ScanExecBase](OffloadOthers()), - RasOffload.from[DataSourceScanExec](OffloadOthers()), - RasOffload - .from(HiveTableScanExecTransformer.isHiveTableScan, OffloadOthers()), - RasOffload.from[CoalesceExec](OffloadOthers()), - RasOffload.from[HashAggregateExec](OffloadOthers()), - RasOffload.from[SortAggregateExec](OffloadOthers()), - RasOffload.from[ObjectHashAggregateExec](OffloadOthers()), - RasOffload.from[UnionExec](OffloadOthers()), - RasOffload.from[ExpandExec](OffloadOthers()), - RasOffload.from[WriteFilesExec](OffloadOthers()), - RasOffload.from[SortExec](OffloadOthers()), - RasOffload.from[TakeOrderedAndProjectExec](OffloadOthers()), - RasOffload.from[WindowExec](OffloadOthers()), - RasOffload - .from(SparkShimLoader.getSparkShims.isWindowGroupLimitExec, OffloadOthers()), - RasOffload.from[LimitExec](OffloadOthers()), - RasOffload.from[GenerateExec](OffloadOthers()), - RasOffload.from[EvalPythonExec](OffloadOthers()), - RasOffload.from[SampleExec](OffloadOthers()) - ).map(RasOffload.Rule(_, validator)) - - private val optimization = { - GlutenOptimization - .builder() - .costModel(GlutenCostModel.find()) - .addRules(rules ++ offloadRules) - .create() - } - - private val reqConvention = Conv.any - - private val altConventions = { - val rowBased: Conv = Conv.req(ConventionReq.row) - val backendBatchBased: Conv = Conv.req(ConventionReq.backendBatch) - Seq(rowBased, backendBatchBased) - } - - override def apply(plan: SparkPlan): SparkPlan = { - val constraintSet = PropertySet(List(reqConvention)) - val altConstraintSets = - altConventions.map(altConv => PropertySet(List(altConv))) - val planner = optimization.newPlanner(plan, constraintSet, altConstraintSets) - val out = planner.plan() - out - } -} - -object EnumeratedTransform {} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala index 1e97ebfbab85..49401797f91a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.enumerated import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.heuristic.OffloadSingleNode +import org.apache.gluten.extension.columnar.offload.OffloadSingleNode import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode import org.apache.gluten.extension.columnar.validator.Validator import org.apache.gluten.ras.path.Pattern @@ -50,8 +50,6 @@ object RasOffload { } } - private val rewrites = RewriteSingleNode.allRules() - def from[T <: SparkPlan: ClassTag](base: OffloadSingleNode): RasOffload = { new RasOffload { override def offload(plan: SparkPlan): SparkPlan = base.offload(plan) @@ -59,19 +57,24 @@ object RasOffload { } } - def from(identifier: TypeIdentifier, base: OffloadSingleNode): RasOffload = { + def from(identifier: SparkPlan => Boolean)(base: OffloadSingleNode): RasOffload = { new RasOffload { override def offload(plan: SparkPlan): SparkPlan = base.offload(plan) - override def typeIdentifier(): TypeIdentifier = identifier + override def typeIdentifier(): TypeIdentifier = new TypeIdentifier { + override def isInstance(node: SparkPlan): Boolean = identifier(node) + } } } object Rule { - def apply(base: RasOffload, validator: Validator): RasRule[SparkPlan] = { - new RuleImpl(base, validator) + def apply( + base: RasOffload, + validator: Validator, + rewrites: Seq[RewriteSingleNode]): RasRule[SparkPlan] = { + new RuleImpl(base, validator, rewrites) } - private class RuleImpl(base: RasOffload, validator: Validator) + private class RuleImpl(base: RasOffload, validator: Validator, rewrites: Seq[RewriteSingleNode]) extends RasRule[SparkPlan] with Logging { private val typeIdentifier: TypeIdentifier = base.typeIdentifier() @@ -119,7 +122,7 @@ object RasOffload { val offloadedNodes = offloadedPlan.collect[GlutenPlan] { case t: GlutenPlan => t } val outComes = offloadedNodes.map(_.doValidate()).filter(!_.ok()) if (outComes.nonEmpty) { - // 4. If native validation fails on at least one of the offloaded nodes, return + // 5. If native validation fails on at least one of the offloaded nodes, return // the original one. // // TODO: Tag the original plan with fallback reason. This is a non-trivial work @@ -130,7 +133,9 @@ object RasOffload { offloadedPlan } case Validator.Failed(reason) => - // TODO: Tag the original plan with fallback reason. + // TODO: Tag the original plan with fallback reason. This is a non-trivial work + // in RAS as the query plan we got here may be a copy so may not propagate tags + // to original plan. from } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala similarity index 85% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala index 2e32e3f06a4b..5cf9b87f2ac1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala @@ -14,23 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.extension.columnar.transition.{ColumnarToColumnarLike, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.execution.{ColumnarToRowExec, ColumnarWriteFilesExec, ProjectExec, RowToColumnarExec, SparkPlan} -/** - * A cost model that is supposed to drive RAS planner create the same query plan with legacy - * planner. - */ -class LegacyCostModel extends LongCostModel { +object LegacyCoster extends LongCoster { + override def kind(): LongCostModel.Kind = LongCostModel.Legacy + + override def selfCostOf(node: SparkPlan): Option[Long] = { + Some(selfCostOf0(node)) + } // A very rough estimation as of now. The cost model basically considers any // fallen back ops as having extreme high cost so offloads computations as // much as possible. - override def selfLongCostOf(node: SparkPlan): Long = { + private def selfCostOf0(node: SparkPlan): Long = { node match { case ColumnarWriteFilesExec.OnNoopLeafPath(_) => 0 case ColumnarToRowExec(_) => 10L diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala similarity index 87% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala index 2576a1008472..d2959d46a13c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala @@ -14,20 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.execution.RowToColumnarExecBase import org.apache.gluten.extension.columnar.transition.{ColumnarToColumnarLike, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} -import org.apache.spark.sql.execution.{ColumnarToRowExec, ColumnarWriteFilesExec, ProjectExec, RowToColumnarExec, SparkPlan} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.types.{ArrayType, MapType, StructType} -/** A rough cost model with some empirical heuristics. */ -class RoughCostModel extends LongCostModel { +object RoughCoster extends LongCoster { + override def kind(): LongCostModel.Kind = LongCostModel.Rough - override def selfLongCostOf(node: SparkPlan): Long = { + override def selfCostOf(node: SparkPlan): Option[Long] = { + Some(selfCostOf0(node)) + } + + private def selfCostOf0(node: SparkPlan): Long = { node match { case ColumnarWriteFilesExec.OnNoopLeafPath(_) => 0 case ProjectExec(projectList, _) if projectList.forall(isCheapExpression) => diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel2.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala similarity index 88% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel2.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala index 4452e777980c..832c524d7b55 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel2.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike} @@ -25,31 +25,15 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase -/** Compared with rough, rough2 can be more precise to avoid the costly r2c. */ -class RoughCostModel2 extends LongCostModel { +// Since https://github.com/apache/incubator-gluten/pull/7686. +object RoughCoster2 extends LongCoster { + override def kind(): LongCostModel.Kind = LongCostModel.Rough2 - private def getSizeFactor(plan: SparkPlan): Long = { - // Get the bytes size that the plan needs to consume. - val sizeBytes = plan match { - case _: DataSourceScanExec | _: DataSourceV2ScanExecBase => getStatSizeBytes(plan) - case _: LeafExecNode => 0L - case p => p.children.map(getStatSizeBytes).sum - } - sizeBytes / GlutenConfig.getConf.rasRough2SizeBytesThreshold - } - - private def getStatSizeBytes(plan: SparkPlan): Long = { - plan match { - case a: AdaptiveSparkPlanExec => getStatSizeBytes(a.inputPlan) - case _ => - plan.logicalLink match { - case Some(logicalPlan) => logicalPlan.stats.sizeInBytes.toLong - case _ => plan.children.map(getStatSizeBytes).sum - } - } + override def selfCostOf(node: SparkPlan): Option[Long] = { + Some(selfCostOf0(node)) } - override def selfLongCostOf(node: SparkPlan): Long = { + private def selfCostOf0(node: SparkPlan): Long = { val sizeFactor = getSizeFactor(node) val opCost = node match { case ProjectExec(projectList, _) if projectList.forall(isCheapExpression) => @@ -70,6 +54,27 @@ class RoughCostModel2 extends LongCostModel { opCost * Math.max(1, sizeFactor) } + private def getSizeFactor(plan: SparkPlan): Long = { + // Get the bytes size that the plan needs to consume. + val sizeBytes = plan match { + case _: DataSourceScanExec | _: DataSourceV2ScanExecBase => getStatSizeBytes(plan) + case _: LeafExecNode => 0L + case p => p.children.map(getStatSizeBytes).sum + } + sizeBytes / GlutenConfig.getConf.rasRough2SizeBytesThreshold + } + + private def getStatSizeBytes(plan: SparkPlan): Long = { + plan match { + case a: AdaptiveSparkPlanExec => getStatSizeBytes(a.inputPlan) + case _ => + plan.logicalLink match { + case Some(logicalPlan) => logicalPlan.stats.sizeInBytes.toLong + case _ => plan.children.map(getStatSizeBytes).sum + } + } + } + private def isCheapExpression(ne: NamedExpression): Boolean = ne match { case Alias(_: Attribute, _) => true case _: Attribute => true diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala deleted file mode 100644 index 538132c81a2e..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala +++ /dev/null @@ -1,298 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.gluten.extension.columnar.heuristic - -import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.execution.{BasicScanExecTransformer, ColumnarCoalesceExec, ColumnarUnionExec, ExpandExecTransformer, HashAggregateExecBaseTransformer, LimitExecTransformer, ProjectExecTransformer, ScanTransformerFactory, SortExecTransformer, TakeOrderedAndProjectExecTransformer, WindowExecTransformer, WindowGroupLimitExecTransformer, WriteFilesExecTransformer} -import org.apache.gluten.extension.columnar.FallbackTags -import org.apache.gluten.extension.columnar.validator.{Validator, Validators} -import org.apache.gluten.sql.shims.SparkShimLoader - -import org.apache.spark.api.python.EvalPythonExecTransformer -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{CoalesceExec, ColumnarBroadcastExchangeExec, ColumnarShuffleExchangeExec, ExpandExec, FileSourceScanExec, FilterExec, GenerateExec, GlobalLimitExec, LocalLimitExec, ProjectExec, SampleExec, SortExec, SparkPlan, TakeOrderedAndProjectExec, UnionExec} -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.datasources.WriteFilesExec -import org.apache.spark.sql.execution.datasources.v2.BatchScanExec -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, CartesianProductExec, ShuffledHashJoinExec, SortMergeJoinExec} -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} -import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} -import org.apache.spark.sql.hive.HiveTableScanExecTransformer - -// This rule will try to convert a plan into plan transformer. -// The doValidate function will be called to check if the conversion is supported. -// If false is returned or any unsupported exception is thrown, a row guard will -// be added on the top of that plan to prevent actual conversion. -case class AddFallbackTags() extends Rule[SparkPlan] { - import AddFallbackTags._ - private val glutenConf: GlutenConfig = GlutenConfig.getConf - private val validator = Validators - .builder() - .fallbackByHint() - .fallbackIfScanOnlyWithFilterPushed(glutenConf.enableScanOnly) - .fallbackComplexExpressions() - .fallbackByBackendSettings() - .fallbackByUserOptions() - .fallbackByTestInjects() - .build() - - def apply(plan: SparkPlan): SparkPlan = { - plan.foreachUp { case p => addFallbackTag(p) } - plan - } - - private def addFallbackTag(plan: SparkPlan): Unit = { - val outcome = validator.validate(plan) - outcome match { - case Validator.Failed(reason) => - FallbackTags.add(plan, reason) - return - case Validator.Passed => - } - - try { - plan match { - case plan: BatchScanExec => - val transformer = - ScanTransformerFactory - .createBatchScanTransformer(plan, validation = true) - .asInstanceOf[BasicScanExecTransformer] - transformer.doValidate().tagOnFallback(plan) - case plan: FileSourceScanExec => - val transformer = - ScanTransformerFactory.createFileSourceScanTransformer(plan) - transformer.doValidate().tagOnFallback(plan) - case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - HiveTableScanExecTransformer.validate(plan).tagOnFallback(plan) - case plan: ProjectExec => - val transformer = ProjectExecTransformer(plan.projectList, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: FilterExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genFilterExecTransformer(plan.condition, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: HashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: SortAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: ObjectHashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: UnionExec => - val transformer = ColumnarUnionExec(plan.children) - transformer.doValidate().tagOnFallback(plan) - case plan: ExpandExec => - val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: WriteFilesExec => - val transformer = WriteFilesExecTransformer( - plan.child, - plan.fileFormat, - plan.partitionColumns, - plan.bucketSpec, - plan.options, - plan.staticPartitions) - transformer.doValidate().tagOnFallback(plan) - case plan: SortExec => - val transformer = - SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffleExchangeExec => - val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffledHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genShuffledHashJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - OffloadJoin.getShjBuildSide(plan), - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastExchangeExec => - val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) - transformer.doValidate().tagOnFallback(plan) - case bhj: BroadcastHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastHashJoinExecTransformer( - bhj.leftKeys, - bhj.rightKeys, - bhj.joinType, - bhj.buildSide, - bhj.condition, - bhj.left, - bhj.right, - isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: SortMergeJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genSortMergeJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: CartesianProductExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastNestedLoopJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastNestedLoopJoinExecTransformer( - plan.left, - plan.right, - plan.buildSide, - plan.joinType, - plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: WindowExec => - val transformer = WindowExecTransformer( - plan.windowExpression, - plan.partitionSpec, - plan.orderSpec, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => - val windowGroupLimitPlan = SparkShimLoader.getSparkShims - .getWindowGroupLimitExecShim(plan) - .asInstanceOf[WindowGroupLimitExecShim] - val transformer = WindowGroupLimitExecTransformer( - windowGroupLimitPlan.partitionSpec, - windowGroupLimitPlan.orderSpec, - windowGroupLimitPlan.rankLikeFunction, - windowGroupLimitPlan.limit, - windowGroupLimitPlan.mode, - windowGroupLimitPlan.child - ) - transformer.doValidate().tagOnFallback(plan) - case plan: CoalesceExec => - ColumnarCoalesceExec(plan.numPartitions, plan.child) - .doValidate() - .tagOnFallback(plan) - case plan: GlobalLimitExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) - val transformer = LimitExecTransformer(plan.child, offset, limit) - transformer.doValidate().tagOnFallback(plan) - case plan: LocalLimitExec => - val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) - transformer.doValidate().tagOnFallback(plan) - case plan: GenerateExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( - plan.generator, - plan.requiredChildOutput, - plan.outer, - plan.generatorOutput, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: BatchEvalPythonExec => - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: ArrowEvalPythonExec => - // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not - // enabled, we will try offloading through EvalPythonExecTransformer - if ( - !BackendsApiManager.getSettings.supportColumnarArrowUdf() || - !GlutenConfig.getConf.enableColumnarArrowUDF - ) { - // Both CH and Velox will try using backend's built-in functions for calculate - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - } - case plan: TakeOrderedAndProjectExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) - val transformer = TakeOrderedAndProjectExecTransformer( - limit, - plan.sortOrder, - plan.projectList, - plan.child, - offset) - transformer.doValidate().tagOnFallback(plan) - case plan: SampleExec => - val transformer = - BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( - plan.lowerBound, - plan.upperBound, - plan.withReplacement, - plan.seed, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case _ => - // Currently we assume a plan to be offload-able by default. - } - } catch { - case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => - FallbackTags.add( - plan, - s"${e.getMessage}, original Spark plan is " + - s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") - if (!e.isInstanceOf[GlutenNotSupportException]) { - logDebug("Just a warning. This exception perhaps needs to be fixed.", e) - } - } - } -} - -object AddFallbackTags { - implicit private class ValidatorBuilderImplicits(builder: Validators.Builder) { - - /** - * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes - * validation on filter for the exception that filter + scan is detected. Because filters can be - * pushed into scan then the filter conditions will be processed only in scan. - */ - def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validators.Builder = { - builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) - builder - } - } - - private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { - override def validate(plan: SparkPlan): Validator.OutCome = { - if (!scanOnly) { - return pass() - } - // Scan-only mode - plan match { - case _: BatchScanExec => pass() - case _: FileSourceScanExec => pass() - case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() - case filter: FilterExec => - val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || - filter.child.isInstanceOf[BatchScanExec] - if (childIsScan) { - pass() - } else { - fail(filter) - } - case other => fail(other) - } - } - } -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/ExpandFallbackPolicy.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala similarity index 94% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/ExpandFallbackPolicy.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala index 9c0ddac16b76..d6e674bbf291 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/ExpandFallbackPolicy.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala @@ -14,15 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar +package org.apache.gluten.extension.columnar.heuristic import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.heuristic.FallbackNode +import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} +import org.apache.gluten.extension.columnar.FallbackTags.add import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike, Transitions} import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec @@ -65,6 +67,7 @@ import org.apache.spark.sql.execution.exchange.Exchange // spotless:on case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkPlan) extends Rule[SparkPlan] { + import ExpandFallbackPolicy._ private def countTransitionCost(plan: SparkPlan): Int = { val ignoreRowToColumnar = GlutenConfig.getConf.fallbackIgnoreRowToColumnar @@ -255,7 +258,7 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP ) { plan } else { - FallbackTags.addRecursively( + addFallbackTagsRecursively( vanillaSparkPlan, FallbackTag.Exclusive(fallbackInfo.reason.getOrElse("Unknown reason"))) FallbackNode(vanillaSparkPlan) @@ -273,3 +276,13 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP def DO_NOT_FALLBACK(): FallbackInfo = FallbackInfo() } } + +object ExpandFallbackPolicy { + private def addFallbackTagsRecursively[T](plan: TreeNode[_], t: T)(implicit + converter: FallbackTag.Converter[T]): Unit = { + plan.foreach { + case _: GlutenPlan => // ignore + case other: TreeNode[_] => add(other, t) + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala deleted file mode 100644 index 8a026326e5de..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.gluten.extension.columnar.heuristic - -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager -import org.apache.gluten.logging.LogLevelUtil - -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.SparkPlan - -/** - * Rule to offload Spark query plan to Gluten query plan using programed heuristics. - * - * The procedure consists of 3 stages: - * - * 1. Rewrite 2. Validate 3. Offload - * - * In the rewrite stage, planner will try converting the Spark query plan to various forms of - * possible alternative Spark query plans, then choose the optimal one to send to next stage. During - * which, the same validation code that is about to be used in stage 2 might be invoked early to - * predict on the estimate "cost" of an alternative Spark query plan. - * - * Once the plan is rewritten, query planner will call native validation code in stage 2 to - * determine which part of the plan is offload-able or not, then add fallback tags to the - * non-offload-able parts. - * - * In stage 3, query planner will convert the offload-able Spark plan nodes into Gluten plan nodes. - */ -case class HeuristicTransform() extends Rule[SparkPlan] with LogLevelUtil { - - private val validate = AddFallbackTags() - private val rewrite = RewriteSparkPlanRulesManager(validate) - private val offload = LegacyOffload() - - override def apply(plan: SparkPlan): SparkPlan = { - Seq(rewrite, validate, offload).foldLeft(plan) { - case (plan, stage) => - stage(plan) - } - } -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala similarity index 96% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala index bae98bec2ec6..64448fc60a51 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar.heuristic +package org.apache.gluten.extension.columnar.offload import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager @@ -39,17 +39,6 @@ import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPyth import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} import org.apache.spark.sql.hive.{HiveTableScanExecTransformer, HiveTableScanNestedColumnPruning} -/** - * Converts a vanilla Spark plan node into Gluten plan node. Gluten plan is supposed to be executed - * in native, and the internals of execution is subject by backend's implementation. - * - * Note: Only the current plan node is supposed to be open to modification. Do not access or modify - * the children node. Tree-walking is done by caller of this trait. - */ -sealed trait OffloadSingleNode extends Logging { - def offload(plan: SparkPlan): SparkPlan -} - // Exchange transformation. case class OffloadExchange() extends OffloadSingleNode with LogLevelUtil { override def offload(plan: SparkPlan): SparkPlan = plan match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala index 6ede36446ef1..1a3ecca16a00 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala @@ -33,6 +33,9 @@ import scala.collection.mutable.ArrayBuffer * when a fallback occurs. */ object PullOutPostProject extends RewriteSingleNode with PullOutProjectHelper { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } private def needsPostProjection(plan: SparkPlan): Boolean = { plan match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala index ee81c4124493..7f32014c2478 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala @@ -37,6 +37,9 @@ import scala.collection.mutable * execution by the native engine. */ object PullOutPreProject extends RewriteSingleNode with PullOutProjectHelper { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } private def needsPreProject(plan: SparkPlan): Boolean = { plan match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteEligibility.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteEligibility.scala new file mode 100644 index 000000000000..dd2847df6ff8 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteEligibility.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.rewrite + +import org.apache.gluten.sql.shims.SparkShimLoader + +import org.apache.spark.sql.execution.{ExpandExec, FileSourceScanExec, FilterExec, GenerateExec, SortExec, SparkPlan, TakeOrderedAndProjectExec} +import org.apache.spark.sql.execution.aggregate.BaseAggregateExec +import org.apache.spark.sql.execution.joins.BaseJoinExec +import org.apache.spark.sql.execution.python.ArrowEvalPythonExec +import org.apache.spark.sql.execution.window.WindowExec + +/** + * TODO: Remove this then implement API #isRewritable in rewrite rules. + * + * Since https://github.com/apache/incubator-gluten/pull/4645 + */ +object RewriteEligibility { + def isRewritable(plan: SparkPlan): Boolean = plan match { + case _: SortExec => true + case _: TakeOrderedAndProjectExec => true + case _: BaseAggregateExec => true + case _: BaseJoinExec => true + case _: WindowExec => true + case _: FilterExec => true + case _: FileSourceScanExec => true + case _: ExpandExec => true + case _: GenerateExec => true + case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => true + case _: ArrowEvalPythonExec => true + case _ => false + } + +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala index da120c39a4a8..5a28576750ab 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala @@ -32,6 +32,9 @@ import org.apache.spark.sql.types.StructType * TODO: Remove this rule once Velox support the list option in `In` is not literal. */ object RewriteIn extends RewriteSingleNode { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } private def shouldRewrite(e: Expression): Boolean = { e match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala index 5fcf37439772..2d844a616c5b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.rewrite import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.columnar.heuristic.OffloadJoin +import org.apache.gluten.extension.columnar.offload.OffloadJoin import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper} import org.apache.spark.sql.catalyst.plans.logical.Join @@ -26,6 +26,10 @@ import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoin /** If force ShuffledHashJoin, convert [[SortMergeJoinExec]] to [[ShuffledHashJoinExec]]. */ object RewriteJoin extends RewriteSingleNode with JoinSelectionHelper { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } + private def getSmjBuildSide(join: SortMergeJoinExec): Option[BuildSide] = { val leftBuildable = canBuildShuffledHashJoinLeft(join.joinType) val rightBuildable = canBuildShuffledHashJoinRight(join.joinType) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala index b395d961a075..1d11dcc91795 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala @@ -48,6 +48,10 @@ import org.apache.spark.sql.types.IntegerType object RewriteMultiChildrenCount extends RewriteSingleNode with PullOutProjectHelper { private lazy val shouldRewriteCount = BackendsApiManager.getSettings.shouldRewriteCount() + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } + private def extractCountForRewrite(aggExpr: AggregateExpression): Option[Count] = { val isPartialCountWithMoreThanOneChild = aggExpr.mode == Partial && { aggExpr.aggregateFunction match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala index 898a2dbae824..7f41e810721b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala @@ -18,34 +18,33 @@ package org.apache.gluten.extension.columnar.validator import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.{BackendsApiManager, BackendSettingsApi} +import org.apache.gluten.exception.GlutenNotSupportException +import org.apache.gluten.execution.{BasicScanExecTransformer, ColumnarCoalesceExec, ColumnarUnionExec, ExpandExecTransformer, HashAggregateExecBaseTransformer, LimitExecTransformer, ProjectExecTransformer, ScanTransformerFactory, SortExecTransformer, TakeOrderedAndProjectExecTransformer, WindowExecTransformer, WindowGroupLimitExecTransformer, WriteFilesExecTransformer} import org.apache.gluten.expression.ExpressionUtils import org.apache.gluten.extension.columnar.FallbackTags +import org.apache.gluten.extension.columnar.offload.OffloadJoin import org.apache.gluten.sql.shims.SparkShimLoader +import org.apache.spark.api.python.EvalPythonExecTransformer +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.datasources.WriteFilesExec import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.window.WindowExec +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} +import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} import org.apache.spark.sql.hive.HiveTableScanExecTransformer -import scala.collection.mutable -import scala.collection.mutable.ListBuffer - object Validators { - def builder(): Builder = Builder() - - class Builder private { + implicit class ValidatorBuilderImplicits(builder: Validator.Builder) { private val conf = GlutenConfig.getConf private val settings = BackendsApiManager.getSettings - private val buffer: ListBuffer[Validator] = mutable.ListBuffer() /** Fails validation if a plan node was already tagged with TRANSFORM_UNSUPPORTED. */ - def fallbackByHint(): Builder = { - buffer += FallbackByHint - this + def fallbackByHint(): Validator.Builder = { + builder.add(FallbackByHint) } /** @@ -53,59 +52,53 @@ object Validators { * executed by native library. By default, we use a threshold option in config to make the * decision. */ - def fallbackComplexExpressions(): Builder = { - buffer += new FallbackComplexExpressions(conf.fallbackExpressionsThreshold) - this + def fallbackComplexExpressions(): Validator.Builder = { + builder.add(new FallbackComplexExpressions(conf.fallbackExpressionsThreshold)) } /** Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. */ - def fallbackIfScanOnly(): Builder = { - buffer += new FallbackIfScanOnly(conf.enableScanOnly) - this + def fallbackIfScanOnly(): Validator.Builder = { + builder.add(new FallbackIfScanOnly(conf.enableScanOnly)) } /** * Fails validation if native-execution of a plan node is not supported by current backend * implementation by checking the active BackendSettings. */ - def fallbackByBackendSettings(): Builder = { - buffer += new FallbackByBackendSettings(settings) - this + def fallbackByBackendSettings(): Validator.Builder = { + builder.add(new FallbackByBackendSettings(settings)) } /** * Fails validation if native-execution of a plan node is disabled by Gluten/Spark * configuration. */ - def fallbackByUserOptions(): Builder = { - buffer += new FallbackByUserOptions(conf) - this + def fallbackByUserOptions(): Validator.Builder = { + builder.add(new FallbackByUserOptions(conf)) } - def fallbackByTestInjects(): Builder = { - buffer += new FallbackByTestInjects() - this + def fallbackByTestInjects(): Validator.Builder = { + builder.add(new FallbackByTestInjects()) } - /** Add a custom validator to pipeline. */ - def add(validator: Validator): Builder = { - buffer += validator - this + /** + * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes + * validation on filter for the exception that filter + scan is detected. Because filters can be + * pushed into scan then the filter conditions will be processed only in scan. + */ + def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validator.Builder = { + builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) } - def build(): Validator = { - if (buffer.isEmpty) { - NoopValidator - } else { - new ValidatorPipeline(buffer.toSeq) - } + /** + * Attempts to offload the input query plan node and check native validation result. Fails when + * native validation failed. + */ + def fallbackByNativeValidation(): Validator.Builder = { + builder.add(new FallbackByNativeValidation()) } } - private object Builder { - def apply(): Builder = new Builder() - } - private object FallbackByHint extends Validator { override def validate(plan: SparkPlan): Validator.OutCome = { if (FallbackTags.nonEmpty(plan)) { @@ -207,21 +200,223 @@ object Validators { } } - private class ValidatorPipeline(validators: Seq[Validator]) extends Validator { + private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { override def validate(plan: SparkPlan): Validator.OutCome = { - val init: Validator.OutCome = pass() - val finalOut = validators.foldLeft(init) { - case (out, validator) => - out match { - case Validator.Passed => validator.validate(plan) - case Validator.Failed(_) => out + if (!scanOnly) { + return pass() + } + // Scan-only mode + plan match { + case _: BatchScanExec => pass() + case _: FileSourceScanExec => pass() + case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() + case filter: FilterExec => + val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || + filter.child.isInstanceOf[BatchScanExec] + if (childIsScan) { + pass() + } else { + fail(filter) } + case other => fail(other) } - finalOut } } - private object NoopValidator extends Validator { - override def validate(plan: SparkPlan): Validator.OutCome = pass() + private class FallbackByNativeValidation() extends Validator with Logging { + override def validate(plan: SparkPlan): Validator.OutCome = { + try { + validate0(plan) + } catch { + case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => + if (!e.isInstanceOf[GlutenNotSupportException]) { + logDebug("Just a warning. This exception perhaps needs to be fixed.", e) + } + fail( + s"${e.getMessage}, original Spark plan is " + + s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") + } + } + + private def validate0(plan: SparkPlan): Validator.OutCome = plan match { + case plan: BatchScanExec => + val transformer = + ScanTransformerFactory + .createBatchScanTransformer(plan, validation = true) + .asInstanceOf[BasicScanExecTransformer] + transformer.doValidate().toValidatorOutcome() + case plan: FileSourceScanExec => + val transformer = + ScanTransformerFactory.createFileSourceScanTransformer(plan) + transformer.doValidate().toValidatorOutcome() + case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => + HiveTableScanExecTransformer.validate(plan).toValidatorOutcome() + case plan: ProjectExec => + val transformer = ProjectExecTransformer(plan.projectList, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: FilterExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genFilterExecTransformer(plan.condition, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: HashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().toValidatorOutcome() + case plan: SortAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().toValidatorOutcome() + case plan: ObjectHashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().toValidatorOutcome() + case plan: UnionExec => + val transformer = ColumnarUnionExec(plan.children) + transformer.doValidate().toValidatorOutcome() + case plan: ExpandExec => + val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: WriteFilesExec => + val transformer = WriteFilesExecTransformer( + plan.child, + plan.fileFormat, + plan.partitionColumns, + plan.bucketSpec, + plan.options, + plan.staticPartitions) + transformer.doValidate().toValidatorOutcome() + case plan: SortExec => + val transformer = + SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) + transformer.doValidate().toValidatorOutcome() + case plan: ShuffleExchangeExec => + val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) + transformer.doValidate().toValidatorOutcome() + case plan: ShuffledHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genShuffledHashJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + OffloadJoin.getShjBuildSide(plan), + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().toValidatorOutcome() + case plan: BroadcastExchangeExec => + val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) + transformer.doValidate().toValidatorOutcome() + case bhj: BroadcastHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastHashJoinExecTransformer( + bhj.leftKeys, + bhj.rightKeys, + bhj.joinType, + bhj.buildSide, + bhj.condition, + bhj.left, + bhj.right, + isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) + transformer.doValidate().toValidatorOutcome() + case plan: SortMergeJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genSortMergeJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().toValidatorOutcome() + case plan: CartesianProductExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) + transformer.doValidate().toValidatorOutcome() + case plan: BroadcastNestedLoopJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastNestedLoopJoinExecTransformer( + plan.left, + plan.right, + plan.buildSide, + plan.joinType, + plan.condition) + transformer.doValidate().toValidatorOutcome() + case plan: WindowExec => + val transformer = WindowExecTransformer( + plan.windowExpression, + plan.partitionSpec, + plan.orderSpec, + plan.child) + transformer.doValidate().toValidatorOutcome() + case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => + val windowGroupLimitPlan = SparkShimLoader.getSparkShims + .getWindowGroupLimitExecShim(plan) + .asInstanceOf[WindowGroupLimitExecShim] + val transformer = WindowGroupLimitExecTransformer( + windowGroupLimitPlan.partitionSpec, + windowGroupLimitPlan.orderSpec, + windowGroupLimitPlan.rankLikeFunction, + windowGroupLimitPlan.limit, + windowGroupLimitPlan.mode, + windowGroupLimitPlan.child + ) + transformer.doValidate().toValidatorOutcome() + case plan: CoalesceExec => + ColumnarCoalesceExec(plan.numPartitions, plan.child) + .doValidate() + .toValidatorOutcome() + case plan: GlobalLimitExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) + val transformer = LimitExecTransformer(plan.child, offset, limit) + transformer.doValidate().toValidatorOutcome() + case plan: LocalLimitExec => + val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) + transformer.doValidate().toValidatorOutcome() + case plan: GenerateExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( + plan.generator, + plan.requiredChildOutput, + plan.outer, + plan.generatorOutput, + plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: BatchEvalPythonExec => + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: ArrowEvalPythonExec => + // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not + // enabled, we will try offloading through EvalPythonExecTransformer + if ( + !BackendsApiManager.getSettings.supportColumnarArrowUdf() || + !GlutenConfig.getConf.enableColumnarArrowUDF + ) { + // Both CH and Velox will try using backend's built-in functions for calculate + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().toValidatorOutcome() + } + pass() + case plan: TakeOrderedAndProjectExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) + val transformer = TakeOrderedAndProjectExecTransformer( + limit, + plan.sortOrder, + plan.projectList, + plan.child, + offset) + transformer.doValidate().toValidatorOutcome() + case plan: SampleExec => + val transformer = + BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( + plan.lowerBound, + plan.upperBound, + plan.withReplacement, + plan.seed, + plan.child) + transformer.doValidate().toValidatorOutcome() + case _ => + // Currently we assume a plan to be offload-able by default. + pass() + } } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala deleted file mode 100644 index 0a58b9f69c12..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.gluten.planner.cost - -import org.apache.gluten.GlutenConfig -import org.apache.gluten.ras.CostModel - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.utils.ReflectionUtil - -object GlutenCostModel extends Logging { - def find(): CostModel[SparkPlan] = { - val aliases: Map[String, Class[_ <: CostModel[SparkPlan]]] = - Map( - "legacy" -> classOf[LegacyCostModel], - "rough" -> classOf[RoughCostModel], - "rough2" -> classOf[RoughCostModel2]) - val aliasOrClass = GlutenConfig.getConf.rasCostModel - val clazz: Class[_ <: CostModel[SparkPlan]] = if (aliases.contains(aliasOrClass)) { - aliases(aliasOrClass) - } else { - val userModel = ReflectionUtil.classForName(aliasOrClass) - logInfo(s"Using user cost model: $aliasOrClass") - userModel - } - val ctor = clazz.getDeclaredConstructor() - ctor.setAccessible(true) - val model = ctor.newInstance() - model - } - - def legacy(): CostModel[SparkPlan] = new LegacyCostModel() -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LongCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LongCostModel.scala deleted file mode 100644 index d3c8410abf88..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LongCostModel.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.gluten.planner.cost - -import org.apache.gluten.planner.plan.GlutenPlanModel.GroupLeafExec -import org.apache.gluten.ras.{Cost, CostModel} - -import org.apache.spark.sql.execution.SparkPlan - -abstract class LongCostModel extends CostModel[SparkPlan] { - private val infLongCost = Long.MaxValue - - override def costOf(node: SparkPlan): GlutenCost = node match { - case _: GroupLeafExec => throw new IllegalStateException() - case _ => GlutenCost(longCostOf(node)) - } - - private def longCostOf(node: SparkPlan): Long = node match { - case n => - val selfCost = selfLongCostOf(n) - - // Sum with ceil to avoid overflow. - def safeSum(a: Long, b: Long): Long = { - assert(a >= 0) - assert(b >= 0) - val sum = a + b - if (sum < a || sum < b) Long.MaxValue else sum - } - - (n.children.map(longCostOf).toList :+ selfCost).reduce(safeSum) - } - - def selfLongCostOf(node: SparkPlan): Long - - override def costComparator(): Ordering[Cost] = Ordering.Long.on { - case GlutenCost(value) => value - case _ => throw new IllegalStateException("Unexpected cost type") - } - - override def makeInfCost(): Cost = GlutenCost(infLongCost) -} diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala index 11ac62ef5823..fc0cd39ecfa8 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, SparkPla import org.apache.spark.sql.execution.ColumnarCollapseTransformStages.transformStageCounter trait GlutenFormatWriterInjectsBase extends GlutenFormatWriterInjects { + private lazy val transform = HeuristicTransform.static() /** * FileFormatWriter wraps some Project & Sort on the top of the original output spark plan, we @@ -41,7 +42,6 @@ trait GlutenFormatWriterInjectsBase extends GlutenFormatWriterInjects { } // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. - val transform = HeuristicTransform() val transformed = transform(plan) if (!transformed.isInstanceOf[TransformSupport]) { diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index fc31289119a1..7abe8228facf 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.BasicScanExecTransformer import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier +import org.apache.gluten.extension.columnar.RemoveFallbackTagRule +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} import org.apache.gluten.extension.columnar.transition.InsertTransitions import org.apache.spark.rdd.RDD diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 3e776721311c..d03619a2e07a 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.BasicScanExecTransformer import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} import org.apache.gluten.extension.columnar.transition.InsertTransitions import org.apache.spark.rdd.RDD diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 3e776721311c..d03619a2e07a 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.BasicScanExecTransformer import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} import org.apache.gluten.extension.columnar.transition.InsertTransitions import org.apache.spark.rdd.RDD diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index a214d9755e69..f43652a7d4ee 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.BasicScanExecTransformer import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} import org.apache.gluten.extension.columnar.transition.InsertTransitions import org.apache.spark.rdd.RDD From 97c423f32558754734aa0326079a2c2e57ea2f8f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 19 Nov 2024 20:47:55 +0800 Subject: [PATCH 116/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_19) (#7978) Upstream Velox's New Commits: aedf91c30 by Eric Liu, fix: Resolve typo in post for simple functions (11555) faf595153 by Wenbin Lin, Revert D65740612: Use Velox fs for ssd cache evictlog file a8dd1474d by Zac Wen, Use Velox fs for ssd cache evictlog file (11495) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index e70fdad01c4d..74d105100bfb 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_18 +VELOX_BRANCH=2024_11_19 VELOX_HOME="" OS=`uname -s` From 90428b9f6f55f13379df47a4480fca76f569c498 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Tue, 19 Nov 2024 21:30:47 +0800 Subject: [PATCH 117/211] [GLUTEN-7969][VL] Enable spill to multiple directories for micro benchmark (#7970) --- cpp/core/shuffle/LocalPartitionWriter.cc | 5 +- cpp/core/shuffle/LocalPartitionWriter.h | 1 - cpp/core/shuffle/Utils.cc | 38 +++-- cpp/core/shuffle/Utils.h | 10 +- cpp/core/utils/StringUtil.cc | 16 +- cpp/core/utils/StringUtil.h | 2 + cpp/velox/benchmarks/GenericBenchmark.cc | 137 ++++++++++++++---- cpp/velox/benchmarks/common/BenchmarkUtils.cc | 90 +++--------- cpp/velox/benchmarks/common/BenchmarkUtils.h | 9 +- cpp/velox/tests/CMakeLists.txt | 2 +- cpp/velox/tests/MyUdfTest.cc | 20 +-- docs/developers/MicroBenchmarks.md | 18 ++- 12 files changed, 187 insertions(+), 161 deletions(-) diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index f0edfa257357..b7bfa19304fd 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -379,7 +379,7 @@ LocalPartitionWriter::LocalPartitionWriter( } std::string LocalPartitionWriter::nextSpilledFileDir() { - auto spilledFileDir = getSpilledShuffleFileDir(localDirs_[dirSelection_], subDirSelection_[dirSelection_]); + auto spilledFileDir = getShuffleSpillDir(localDirs_[dirSelection_], subDirSelection_[dirSelection_]); subDirSelection_[dirSelection_] = (subDirSelection_[dirSelection_] + 1) % options_.numSubDirs; dirSelection_ = (dirSelection_ + 1) % localDirs_.size(); return spilledFileDir; @@ -505,6 +505,9 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { "Merging from spill " + std::to_string(s) + " is not exhausted. pid: " + std::to_string(pid)); } } + if (std::filesystem::exists(spill->spillFile()) && !std::filesystem::remove(spill->spillFile())) { + LOG(WARNING) << "Error while deleting spill file " << spill->spillFile(); + } ++s; } spills_.clear(); diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index 2b86a81e883b..3ed0a1306008 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -17,7 +17,6 @@ #pragma once -#include #include #include "shuffle/PartitionWriter.h" diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index 6854c1978370..a11b6b09aa3d 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -17,15 +17,15 @@ #include "shuffle/Utils.h" #include -#include -#include #include +#include #include #include #include #include #include #include "shuffle/Options.h" +#include "utils/StringUtil.h" #include "utils/Timer.h" namespace gluten { @@ -214,17 +214,10 @@ arrow::Result> makeUncompressedRecordBatch( } } // namespace gluten -std::string gluten::generateUuid() { - boost::uuids::random_generator generator; - return boost::uuids::to_string(generator()); -} - -std::string gluten::getSpilledShuffleFileDir(const std::string& configuredDir, int32_t subDirId) { - auto fs = std::make_shared(); +std::string gluten::getShuffleSpillDir(const std::string& configuredDir, int32_t subDirId) { std::stringstream ss; ss << std::setfill('0') << std::setw(2) << std::hex << subDirId; - auto dir = arrow::fs::internal::ConcatAbstractPath(configuredDir, ss.str()); - return dir; + return std::filesystem::path(configuredDir) / ss.str(); } arrow::Result gluten::createTempShuffleFile(const std::string& dir) { @@ -232,22 +225,25 @@ arrow::Result gluten::createTempShuffleFile(const std::string& dir) return arrow::Status::Invalid("Failed to create spilled file, got empty path."); } - auto fs = std::make_shared(); - ARROW_ASSIGN_OR_RAISE(auto path_info, fs->GetFileInfo(dir)); - if (path_info.type() == arrow::fs::FileType::NotFound) { - RETURN_NOT_OK(fs->CreateDir(dir, true)); + if (std::filesystem::exists(dir)) { + if (!std::filesystem::is_directory(dir)) { + return arrow::Status::Invalid("Invalid directory. File path exists but is not a directory: ", dir); + } + } else { + std::filesystem::create_directories(dir); } + const auto parentPath = std::filesystem::path(dir); bool exist = true; - std::string filePath; + std::filesystem::path filePath; while (exist) { - filePath = arrow::fs::internal::ConcatAbstractPath(dir, "temp_shuffle_" + generateUuid()); - ARROW_ASSIGN_OR_RAISE(auto file_info, fs->GetFileInfo(filePath)); - if (file_info.type() == arrow::fs::FileType::NotFound) { - int fd = open(filePath.c_str(), O_CREAT | O_EXCL | O_RDWR, 0666); + filePath = parentPath / ("temp-shuffle-" + generateUuid()); + if (!std::filesystem::exists(filePath)) { + auto fd = open(filePath.c_str(), O_CREAT | O_EXCL | O_RDWR, 0666); if (fd < 0) { if (errno != EEXIST) { - return arrow::Status::IOError("Failed to open local file " + filePath + ", Reason: " + strerror(errno)); + return arrow::Status::IOError( + "Failed to open local file " + filePath.string() + ", Reason: " + strerror(errno)); } } else { exist = false; diff --git a/cpp/core/shuffle/Utils.h b/cpp/core/shuffle/Utils.h index c4e2409d2da0..64b9292d9d36 100644 --- a/cpp/core/shuffle/Utils.h +++ b/cpp/core/shuffle/Utils.h @@ -18,13 +18,13 @@ #pragma once #include -#include -#include -#include #include #include #include + #include +#include + #include "utils/Compression.h" namespace gluten { @@ -36,9 +36,7 @@ static const size_t kSizeOfBinaryArrayLengthBuffer = sizeof(BinaryArrayLengthBuf static const size_t kSizeOfIpcOffsetBuffer = sizeof(IpcOffsetBufferType); static const std::string kGlutenSparkLocalDirs = "GLUTEN_SPARK_LOCAL_DIRS"; -std::string generateUuid(); - -std::string getSpilledShuffleFileDir(const std::string& configuredDir, int32_t subDirId); +std::string getShuffleSpillDir(const std::string& configuredDir, int32_t subDirId); arrow::Result createTempShuffleFile(const std::string& dir); diff --git a/cpp/core/utils/StringUtil.cc b/cpp/core/utils/StringUtil.cc index fc6ebb83c0bd..505e1972f8d0 100644 --- a/cpp/core/utils/StringUtil.cc +++ b/cpp/core/utils/StringUtil.cc @@ -17,13 +17,16 @@ #include #include -#include #include +#include +#include + #include "Exception.h" #include "StringUtil.h" -std::vector gluten::splitByDelim(const std::string& s, const char delimiter) { +namespace gluten { +std::vector splitByDelim(const std::string& s, const char delimiter) { if (s.empty()) { return {}; } @@ -41,7 +44,7 @@ std::vector gluten::splitByDelim(const std::string& s, const char d return result; } -std::vector gluten::splitPaths(const std::string& s, bool checkExists) { +std::vector splitPaths(const std::string& s, bool checkExists) { if (s.empty()) { return {}; } @@ -61,3 +64,10 @@ std::vector gluten::splitPaths(const std::string& s, bool checkExis } return paths; } + +std::string generateUuid() { + boost::uuids::random_generator generator; + return boost::uuids::to_string(generator()); +} + +} // namespace gluten diff --git a/cpp/core/utils/StringUtil.h b/cpp/core/utils/StringUtil.h index 8880229616a5..3030651aa391 100644 --- a/cpp/core/utils/StringUtil.h +++ b/cpp/core/utils/StringUtil.h @@ -23,4 +23,6 @@ std::vector splitByDelim(const std::string& s, const char delimiter std::vector splitPaths(const std::string& s, bool checkExists = false); +std::string generateUuid(); + } // namespace gluten diff --git a/cpp/velox/benchmarks/GenericBenchmark.cc b/cpp/velox/benchmarks/GenericBenchmark.cc index e42aed9f219f..4e38fb44327f 100644 --- a/cpp/velox/benchmarks/GenericBenchmark.cc +++ b/cpp/velox/benchmarks/GenericBenchmark.cc @@ -106,6 +106,50 @@ void setUpBenchmark(::benchmark::internal::Benchmark* bm) { } } +std::string generateUniqueSubdir(const std::string& parent, const std::string& prefix = "") { + auto path = std::filesystem::path(parent) / (prefix + generateUuid()); + std::error_code ec{}; + while (!std::filesystem::create_directories(path, ec)) { + if (ec) { + LOG(ERROR) << fmt::format("Failed to created spill directory: {}, error code: {}", path, ec.message()); + std::exit(EXIT_FAILURE); + } + path = std::filesystem::path(parent) / (prefix + generateUuid()); + } + return path; +} + +std::vector createLocalDirs() { + static const std::string kBenchmarkDirPrefix = "generic-benchmark-"; + std::vector localDirs; + + auto joinedDirsC = std::getenv(gluten::kGlutenSparkLocalDirs.c_str()); + // Check if local dirs are set from env. + if (joinedDirsC != nullptr && strcmp(joinedDirsC, "") > 0) { + auto joinedDirs = std::string(joinedDirsC); + auto dirs = gluten::splitPaths(joinedDirs); + for (const auto& dir : dirs) { + localDirs.push_back(generateUniqueSubdir(dir, kBenchmarkDirPrefix)); + } + } else { + // Otherwise create 1 temp dir. + localDirs.push_back(generateUniqueSubdir(std::filesystem::temp_directory_path(), kBenchmarkDirPrefix)); + } + return localDirs; +} + +void cleanupLocalDirs(const std::vector& localDirs) { + for (const auto& localDir : localDirs) { + std::error_code ec; + std::filesystem::remove_all(localDir, ec); + if (ec) { + LOG(WARNING) << fmt::format("Failed to remove directory: {}, error message: {}", localDir, ec.message()); + } else { + LOG(INFO) << "Removed local dir: " << localDir; + } + } +} + PartitionWriterOptions createPartitionWriterOptions() { PartitionWriterOptions partitionWriterOptions{}; // Disable writer's merge. @@ -204,11 +248,10 @@ void runShuffle( const std::shared_ptr& resultIter, WriterMetrics& writerMetrics, ReaderMetrics& readerMetrics, - bool readAfterWrite) { - std::string dataFile; - std::vector localDirs; - bool isFromEnv; - GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(dataFile, localDirs, isFromEnv)); + bool readAfterWrite, + const std::vector& localDirs, + const std::string& dataFileDir) { + GLUTEN_ASSIGN_OR_THROW(auto dataFile, gluten::createTempShuffleFile(dataFileDir)); auto partitionWriterOptions = createPartitionWriterOptions(); auto partitionWriter = createPartitionWriter(runtime, partitionWriterOptions, dataFile, localDirs); @@ -252,8 +295,12 @@ void runShuffle( readerMetrics.decompressTime = reader->getDecompressTime(); readerMetrics.deserializeTime = reader->getDeserializeTime(); } - // Cleanup shuffle outputs - cleanupShuffleOutput(dataFile, localDirs, isFromEnv); + + if (std::filesystem::remove(dataFile)) { + LOG(INFO) << "Removed shuffle data file: " << dataFile; + } else { + LOG(WARNING) << "Failed to remove shuffle data file. File does not exist: " << dataFile; + } } void updateBenchmarkMetrics( @@ -292,7 +339,6 @@ void updateBenchmarkMetrics( writerMetrics.bytesWritten, benchmark::Counter::kAvgIterations, benchmark::Counter::OneK::kIs1024); } } - } // namespace using RuntimeFactory = std::function; @@ -301,6 +347,7 @@ auto BM_Generic = [](::benchmark::State& state, const std::string& planFile, const std::vector& splitFiles, const std::vector& dataFiles, + const std::vector& localDirs, RuntimeFactory runtimeFactory, FileReaderType readerType) { setCpu(state); @@ -316,6 +363,19 @@ auto BM_Generic = [](::benchmark::State& state, splits.push_back(getPlanFromFile("ReadRel.LocalFiles", splitFile)); } + const auto tid = std::hash{}(std::this_thread::get_id()); + const auto spillDirIndex = tid % localDirs.size(); + const auto veloxSpillDir = generateUniqueSubdir(std::filesystem::path(localDirs[spillDirIndex]) / "gluten-spill"); + + std::vector shuffleSpillDirs; + std::transform(localDirs.begin(), localDirs.end(), std::back_inserter(shuffleSpillDirs), [](const auto& dir) { + auto path = std::filesystem::path(dir) / "shuffle-write"; + return path; + }); + // Use a different directory for data file. + const auto dataFileDir = gluten::getShuffleSpillDir( + shuffleSpillDirs[(spillDirIndex + 1) % localDirs.size()], state.thread_index() % gluten::kDefaultNumSubDirs); + WriterMetrics writerMetrics{}; ReaderMetrics readerMetrics{}; int64_t readInputTime = 0; @@ -343,11 +403,13 @@ auto BM_Generic = [](::benchmark::State& state, for (auto& split : splits) { runtime->parseSplitInfo(reinterpret_cast(split.data()), split.size(), std::nullopt); } - auto resultIter = runtime->createResultIterator("/tmp/test-spill", std::move(inputIters), runtime->getConfMap()); + + auto resultIter = runtime->createResultIterator(veloxSpillDir, std::move(inputIters), runtime->getConfMap()); listenerPtr->setIterator(resultIter.get()); if (FLAGS_with_shuffle) { - runShuffle(runtime, listenerPtr, resultIter, writerMetrics, readerMetrics, false); + runShuffle( + runtime, listenerPtr, resultIter, writerMetrics, readerMetrics, false, shuffleSpillDirs, dataFileDir); } else { // May write the output into file. auto veloxPlan = dynamic_cast(runtime)->getVeloxPlan(); @@ -405,6 +467,7 @@ auto BM_Generic = [](::benchmark::State& state, auto BM_ShuffleWriteRead = [](::benchmark::State& state, const std::string& inputFile, + const std::vector& localDirs, RuntimeFactory runtimeFactory, FileReaderType readerType) { setCpu(state); @@ -414,6 +477,10 @@ auto BM_ShuffleWriteRead = [](::benchmark::State& state, auto* memoryManager = MemoryManager::create(kVeloxBackendKind, std::move(listener)); auto runtime = runtimeFactory(memoryManager); + const size_t dirIndex = std::hash{}(std::this_thread::get_id()) % localDirs.size(); + const auto dataFileDir = + gluten::getShuffleSpillDir(localDirs[dirIndex], state.thread_index() % gluten::kDefaultNumSubDirs); + WriterMetrics writerMetrics{}; ReaderMetrics readerMetrics{}; int64_t readInputTime = 0; @@ -422,7 +489,15 @@ auto BM_ShuffleWriteRead = [](::benchmark::State& state, ScopedTimer timer(&elapsedTime); for (auto _ : state) { auto resultIter = getInputIteratorFromFileReader(inputFile, readerType); - runShuffle(runtime, listenerPtr, resultIter, writerMetrics, readerMetrics, FLAGS_run_shuffle_read); + runShuffle( + runtime, + listenerPtr, + resultIter, + writerMetrics, + readerMetrics, + FLAGS_run_shuffle_read, + localDirs, + dataFileDir); auto reader = static_cast(resultIter->getInputIter()); readInputTime += reader->getCollectBatchTime(); @@ -600,23 +675,31 @@ int main(int argc, char** argv) { return dynamic_cast(Runtime::create(kVeloxBackendKind, memoryManager, sessionConf)); }; -#define GENERIC_BENCHMARK(READER_TYPE) \ - do { \ - auto* bm = \ - ::benchmark::RegisterBenchmark( \ - "GenericBenchmark", BM_Generic, substraitJsonFile, splitFiles, dataFiles, runtimeFactory, READER_TYPE) \ - ->MeasureProcessCPUTime() \ - ->UseRealTime(); \ - setUpBenchmark(bm); \ + const auto localDirs = createLocalDirs(); + +#define GENERIC_BENCHMARK(READER_TYPE) \ + do { \ + auto* bm = ::benchmark::RegisterBenchmark( \ + "GenericBenchmark", \ + BM_Generic, \ + substraitJsonFile, \ + splitFiles, \ + dataFiles, \ + localDirs, \ + runtimeFactory, \ + READER_TYPE) \ + ->MeasureProcessCPUTime() \ + ->UseRealTime(); \ + setUpBenchmark(bm); \ } while (0) -#define SHUFFLE_WRITE_READ_BENCHMARK(READER_TYPE) \ - do { \ - auto* bm = ::benchmark::RegisterBenchmark( \ - "ShuffleWriteRead", BM_ShuffleWriteRead, dataFiles[0], runtimeFactory, READER_TYPE) \ - ->MeasureProcessCPUTime() \ - ->UseRealTime(); \ - setUpBenchmark(bm); \ +#define SHUFFLE_WRITE_READ_BENCHMARK(READER_TYPE) \ + do { \ + auto* bm = ::benchmark::RegisterBenchmark( \ + "ShuffleWriteRead", BM_ShuffleWriteRead, dataFiles[0], localDirs, runtimeFactory, READER_TYPE) \ + ->MeasureProcessCPUTime() \ + ->UseRealTime(); \ + setUpBenchmark(bm); \ } while (0) if (dataFiles.empty()) { @@ -642,5 +725,7 @@ int main(int argc, char** argv) { gluten::VeloxBackend::get()->tearDown(); + cleanupLocalDirs(localDirs); + return 0; } diff --git a/cpp/velox/benchmarks/common/BenchmarkUtils.cc b/cpp/velox/benchmarks/common/BenchmarkUtils.cc index 345f9da8e16d..d0a26ff07fbd 100644 --- a/cpp/velox/benchmarks/common/BenchmarkUtils.cc +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.cc @@ -23,9 +23,6 @@ #include "utils/StringUtil.h" #include "velox/dwio/common/Options.h" -using namespace facebook; -namespace fs = std::filesystem; - DEFINE_int64(batch_size, 4096, "To set velox::core::QueryConfig::kPreferredOutputBatchSize."); DEFINE_int32(cpu, -1, "Run benchmark on specific CPU"); DEFINE_int32(threads, 1, "The number of threads to run this benchmark"); @@ -34,7 +31,7 @@ DEFINE_int32(iterations, 1, "The number of iterations to run this benchmark"); namespace gluten { namespace { std::unordered_map bmConfMap = defaultConf(); -} +} // namespace std::unordered_map defaultConf() { return { @@ -60,13 +57,13 @@ std::string getPlanFromFile(const std::string& type, const std::string& filePath return gluten::substraitFromJsonToPb(type, msgData); } -velox::dwio::common::FileFormat getFileFormat(const std::string& fileFormat) { +facebook::velox::dwio::common::FileFormat getFileFormat(const std::string& fileFormat) { if (fileFormat.compare("orc") == 0) { - return velox::dwio::common::FileFormat::ORC; + return facebook::velox::dwio::common::FileFormat::ORC; } else if (fileFormat.compare("parquet") == 0) { - return velox::dwio::common::FileFormat::PARQUET; + return facebook::velox::dwio::common::FileFormat::PARQUET; } else { - return velox::dwio::common::FileFormat::UNKNOWN; + return facebook::velox::dwio::common::FileFormat::UNKNOWN; } } @@ -84,7 +81,7 @@ std::shared_ptr getSplitInfos(const std::string& datasetPath, if (endsWith(singleFilePath, "." + fileFormat)) { auto fileAbsolutePath = datasetPath + singleFilePath; scanInfo->starts.emplace_back(0); - scanInfo->lengths.emplace_back(fs::file_size(fileAbsolutePath)); + scanInfo->lengths.emplace_back(std::filesystem::file_size(fileAbsolutePath)); scanInfo->paths.emplace_back("file://" + fileAbsolutePath); } } else { @@ -102,7 +99,7 @@ std::shared_ptr getSplitInfosFromFile(const std::string& file // Set split start, length, and path to scan info. scanInfo->starts.emplace_back(0); - scanInfo->lengths.emplace_back(fs::file_size(fileName)); + scanInfo->lengths.emplace_back(std::filesystem::file_size(fileName)); scanInfo->paths.emplace_back("file://" + fileName); return scanInfo; @@ -125,78 +122,25 @@ bool endsWith(const std::string& data, const std::string& suffix) { return data.find(suffix, data.size() - suffix.size()) != std::string::npos; } -#if 0 -std::shared_ptr createReader(const std::string& path) { - std::unique_ptr parquetReader; - std::shared_ptr recordBatchReader; - parquet::ArrowReaderProperties properties = parquet::default_arrow_reader_properties(); - - GLUTEN_THROW_NOT_OK(parquet::arrow::FileReader::Make( - arrow::default_memory_pool(), parquet::ParquetFileReader::OpenFile(path), properties, &parquetReader)); - GLUTEN_THROW_NOT_OK( - parquetReader->GetRecordBatchReader(arrow::internal::Iota(parquetReader->num_row_groups()), &recordBatchReader)); - return recordBatchReader; -} -#endif - -void setCpu(uint32_t cpuindex) { +void setCpu(uint32_t cpuIndex) { static const auto kTotalCores = std::thread::hardware_concurrency(); - cpuindex = cpuindex % kTotalCores; + cpuIndex = cpuIndex % kTotalCores; cpu_set_t cs; CPU_ZERO(&cs); - CPU_SET(cpuindex, &cs); + CPU_SET(cpuIndex, &cs); if (sched_setaffinity(0, sizeof(cs), &cs) == -1) { - LOG(WARNING) << "Error binding CPU " << std::to_string(cpuindex); - exit(EXIT_FAILURE); - } -} - -arrow::Status -setLocalDirsAndDataFileFromEnv(std::string& dataFile, std::vector& localDirs, bool& isFromEnv) { - auto joinedDirsC = std::getenv(gluten::kGlutenSparkLocalDirs.c_str()); - if (joinedDirsC != nullptr && strcmp(joinedDirsC, "") > 0) { - isFromEnv = true; - // Set local dirs. - auto joinedDirs = std::string(joinedDirsC); - // Split local dirs and use thread id to choose one directory for data file. - auto dirs = gluten::splitPaths(joinedDirs); - for (const auto& dir : dirs) { - localDirs.push_back(arrow::fs::internal::ConcatAbstractPath(dir, "temp_shuffle_" + generateUuid())); - std::filesystem::create_directory(localDirs.back()); - } - size_t id = std::hash{}(std::this_thread::get_id()) % localDirs.size(); - ARROW_ASSIGN_OR_RAISE(dataFile, gluten::createTempShuffleFile(localDirs[id])); - } else { - isFromEnv = false; - // Otherwise create 1 temp dir and data file. - static const std::string kBenchmarkDirsPrefix = "columnar-shuffle-benchmark-"; - { - // Because tmpDir will be deleted in the dtor, allow it to be deleted upon exiting the block and then recreate it - // in createTempShuffleFile. - ARROW_ASSIGN_OR_RAISE(auto tmpDir, arrow::internal::TemporaryDir::Make(kBenchmarkDirsPrefix)) - localDirs.push_back(tmpDir->path().ToString()); - } - ARROW_ASSIGN_OR_RAISE(dataFile, gluten::createTempShuffleFile(localDirs.back())); - } - return arrow::Status::OK(); -} - -void cleanupShuffleOutput(const std::string& dataFile, const std::vector& localDirs, bool isFromEnv) { - std::filesystem::remove(dataFile); - for (auto& localDir : localDirs) { - if (std::filesystem::is_empty(localDir)) { - std::filesystem::remove(localDir); - } + LOG(WARNING) << "Error binding CPU " << std::to_string(cpuIndex); + std::exit(EXIT_FAILURE); } } void BenchmarkAllocationListener::allocationChanged(int64_t diff) { - if (usedBytes_ + diff >= limit_) { + if (diff > 0 && usedBytes_ + diff >= limit_) { LOG(INFO) << fmt::format( "reach hard limit {} when need {}, current used {}.", - velox::succinctBytes(limit_), - velox::succinctBytes(diff), - velox::succinctBytes(usedBytes_)); + facebook::velox::succinctBytes(limit_), + facebook::velox::succinctBytes(diff), + facebook::velox::succinctBytes(usedBytes_)); auto neededBytes = usedBytes_ + diff - limit_; int64_t spilledBytes = 0; if (iterator_) { @@ -207,7 +151,7 @@ void BenchmarkAllocationListener::allocationChanged(int64_t diff) { GLUTEN_THROW_NOT_OK(shuffleWriter_->reclaimFixedSize(neededBytes - spilledBytes, &reclaimed)); spilledBytes += reclaimed; } - LOG(INFO) << fmt::format("spill finish, got {}.", velox::succinctBytes(spilledBytes)); + LOG(INFO) << fmt::format("spill finish, got {}.", facebook::velox::succinctBytes(spilledBytes)); } else { usedBytes_ += diff; } diff --git a/cpp/velox/benchmarks/common/BenchmarkUtils.h b/cpp/velox/benchmarks/common/BenchmarkUtils.h index 0108f1d44838..de3df96f89e0 100644 --- a/cpp/velox/benchmarks/common/BenchmarkUtils.h +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.h @@ -95,12 +95,7 @@ inline std::shared_ptr convertBatch(std::shared_ptr& localDirs, bool& isFromEnv); - -void cleanupShuffleOutput(const std::string& dataFile, const std::vector& localDirs, bool isFromEnv); +void setCpu(uint32_t cpuIndex); class BenchmarkAllocationListener final : public gluten::AllocationListener { public: @@ -118,7 +113,7 @@ class BenchmarkAllocationListener final : public gluten::AllocationListener { private: uint64_t usedBytes_{0L}; - uint64_t limit_{0L}; + const uint64_t limit_{0L}; gluten::ResultIterator* iterator_{nullptr}; gluten::ShuffleWriter* shuffleWriter_{nullptr}; }; diff --git a/cpp/velox/tests/CMakeLists.txt b/cpp/velox/tests/CMakeLists.txt index 55cab342665b..ba90e45076cf 100644 --- a/cpp/velox/tests/CMakeLists.txt +++ b/cpp/velox/tests/CMakeLists.txt @@ -60,5 +60,5 @@ add_velox_test(runtime_test SOURCES RuntimeTest.cc) add_velox_test(velox_memory_test SOURCES MemoryManagerTest.cc) add_velox_test(buffer_outputstream_test SOURCES BufferOutputStreamTest.cc) if(BUILD_EXAMPLES) - add_velox_test(MyUdfTest SOURCES MyUdfTest.cc) + add_velox_test(my_udf_test SOURCES MyUdfTest.cc) endif() diff --git a/cpp/velox/tests/MyUdfTest.cc b/cpp/velox/tests/MyUdfTest.cc index c9849d67d010..8e4f1e7dec7e 100644 --- a/cpp/velox/tests/MyUdfTest.cc +++ b/cpp/velox/tests/MyUdfTest.cc @@ -17,12 +17,13 @@ #include #include "udf/UdfLoader.h" -#include "velox/expression/VectorFunction.h" +#include "velox/expression/SimpleFunctionRegistry.h" #include "velox/functions/prestosql/tests/utils/FunctionBaseTest.h" #include "velox/parse/TypeResolver.h" using namespace facebook::velox::functions::test; using namespace facebook::velox; + class MyUdfTest : public FunctionBaseTest { protected: static void SetUpTestCase() { @@ -35,16 +36,7 @@ class MyUdfTest : public FunctionBaseTest { }; TEST_F(MyUdfTest, hivestringstring) { - auto map = facebook::velox::exec::vectorFunctionFactories(); - const std::string candidate = {"org.apache.spark.sql.hive.execution.UDFStringString"}; - ASSERT(map.withRLock([&candidate](auto& self) -> bool { - auto iter = self.find(candidate); - std::unordered_map values; - const facebook::velox::core::QueryConfig config(std::move(values)); - iter->second.factory( - candidate, - {facebook::velox::exec::VectorFunctionArg{facebook::velox::VARCHAR()}, - facebook::velox::exec::VectorFunctionArg{facebook::velox::VARCHAR()}}, - config) != nullptr; - });) -} \ No newline at end of file + const std::string name = "org.apache.spark.sql.hive.execution.UDFStringString"; + const core::QueryConfig config({}); + EXPECT_EQ(TypeKind::VARCHAR, exec::simpleFunctions().resolveFunction(name, {VARCHAR(), VARCHAR()})->type()->kind()); +} diff --git a/docs/developers/MicroBenchmarks.md b/docs/developers/MicroBenchmarks.md index bc463fcac9aa..eedf5010b634 100644 --- a/docs/developers/MicroBenchmarks.md +++ b/docs/developers/MicroBenchmarks.md @@ -257,7 +257,7 @@ inputs from a first stage. The steps are demonstrated as below: 1. Start spark-shell or pyspark We need to set `spark.gluten.sql.benchmark_task.stageId` and `spark.gluten.saveDir` to dump the inputs. -Normally, the stage id should be greater than 0. You can run the command in step 2 in advance to get the +Normally, the stage id should be greater than 0. You can run the command in step 2 in advance to get the right stage id in your case. We shall set `spark.default.parallelism` to 1 and `spark.sql.files.maxPartitionBytes` large enough to make sure there will be only 1 task in the first stage. @@ -378,17 +378,18 @@ cd /path/to/gluten/cpp/build/velox/benchmarks --write-path /absolute_path/ ``` - ## Simulate task spilling -You can simulate task spilling by specify memory hard limit from `--memory_limit`. +You can simulate task spilling by specify a memory hard limit from `--memory_limit`. By default, spilled files are written to the `/tmp` directory. +To simulate real Gluten workloads, which utilize multiple spill directories, set the environment variable GLUTEN_SPARK_LOCAL_DIRS to a comma-separated string. +Please check [Simulate Gluten workload with multiple processes and threads](#Simulate-Gluten-workload-with-multiple-processes-and-threads) for more details. -## Simulate Spark with multiple processes and threads +## Simulate Gluten workload with multiple processes and threads You can use below command to launch several processes and threads to simulate parallel execution on Spark. Each thread in the same process will be pinned to the core number starting from `--cpu`. -Suppose running on a baremetal machine with 48C, 2-socket, HT-on, launching below command will +Suppose running on a bare-metal machine with 48C, 2-socket, HT-on, launching below command will utilize all vcores. ```shell @@ -400,9 +401,10 @@ for ((i=0; i<${processes}; i++)); do done ``` -If you want to add the shuffle write process, you can specify multiple directories by setting -environment variable `GLUTEN_SPARK_LOCAL_DIRS` to a comma-separated string for shuffle write to -spread the I/O pressure to multiple disks. +To include the shuffle write process or trigger spilling via `--memory-limit`, +you can specify multiple directories by setting the `GLUTEN_SPARK_LOCAL_DIRS` environment variable +to a comma-separated string. This will distribute the I/O load across multiple disks, similar to how it works for Gluten workloads. +Temporary subdirectories will be created under each specified directory at runtime and will be automatically deleted if the process completes normally. ```shell mkdir -p {/data1,/data2,/data3}/tmp # Make sure each directory has been already created. From 9315835274b803da6f18499a2ea071a79ec5263a Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 20 Nov 2024 08:34:36 +0800 Subject: [PATCH 118/211] [CORE] Avoid formatted comments from being messed by non-spotless linters (especially IDE linters) (#7989) --- .../sql/execution/ColumnarCachedBatchSerializer.scala | 4 ++-- .../columnar/heuristic/ExpandFallbackPolicy.scala | 11 +++++------ .../spark/sql/execution/GlutenExplainUtils.scala | 6 ++---- .../apache/spark/sql/execution/GlutenImplicits.scala | 5 +++-- 4 files changed, 12 insertions(+), 14 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala index d6e7aa5b2244..6fd346f64cbd 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala @@ -46,7 +46,7 @@ case class CachedColumnarBatch( bytes: Array[Byte]) extends CachedBatch {} -// spotless:off +// format: off /** * Feature: * 1. This serializer supports column pruning @@ -75,7 +75,7 @@ case class CachedColumnarBatch( * - Deserializer DefaultCachedBatch -> InternalRow (unsupport ColumnarToRow) * -> Convert DefaultCachedBatch to InternalRow using vanilla Spark serializer */ -// spotless:on +// format: on class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHelper with Logging { private lazy val rowBasedCachedBatchSerializer = new DefaultCachedBatchSerializer diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala index d6e674bbf291..e1c8b6f41f44 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.exchange.Exchange -// spotless:off + +// format: off /** * Note, this rule should only fallback to row-based plan if there is no harm. * The follow case should be handled carefully @@ -64,7 +65,7 @@ import org.apache.spark.sql.execution.exchange.Exchange * @param isAdaptiveContext If is inside AQE * @param originalPlan The vanilla SparkPlan without apply gluten transform rules */ -// spotless:on +// format: on case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkPlan) extends Rule[SparkPlan] { import ExpandFallbackPolicy._ @@ -106,14 +107,13 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP transitionCost } + // format: off /** * When making a stage fall back, it's possible that we need a ColumnarToRow to adapt to last * stage's columnar output. So we need to evaluate the cost, i.e., the number of required * ColumnarToRow between entirely fallback stage and last stage(s). Thus, we can avoid possible * performance degradation caused by fallback policy. * - * spotless:off - * * Spark plan before applying fallback policy: * * ColumnarExchange @@ -136,9 +136,8 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP * Project * * So by considering the cost, the fallback policy will not be applied. - * - * spotless:on */ + // format: on private def countStageFallbackTransitionCost(plan: SparkPlan): Int = { var stageFallbackTransitionCost = 0 diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala index fa697789c8cf..d6167c931cd6 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala @@ -147,15 +147,13 @@ object GlutenExplainUtils extends AdaptiveSparkPlanHelper { } } - // spotless:off - // scalastyle:off + // format: off /** * Given a input physical plan, performs the following tasks. * 1. Generates the explain output for the input plan excluding the subquery plans. 2. Generates * the explain output for each subquery referenced in the plan. */ - // scalastyle:on - // spotless:on + // format: on def processPlan[T <: QueryPlan[T]]( plan: T, append: String => Unit, diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala index 4ecc674d4b13..435bf9239e8a 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.internal.SQLConf import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -// spotless:off + +// format: off /** * A helper class to get the Gluten fallback summary from a Spark [[Dataset]]. * @@ -52,7 +53,7 @@ import scala.collection.mutable.ArrayBuffer * df.fallbackSummary * }}} */ -// spotless:on +// format: on object GlutenImplicits { case class FallbackSummary( From 5d7b9635d53107b1ae2b571efce8017e821b1a4e Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Wed, 20 Nov 2024 08:50:03 +0800 Subject: [PATCH 119/211] [GLUTEN-7751][VL] Merge two consecutive aggregates to one in complete mode (#7752) What changes were proposed in this pull request? Fix #7751. In issue #4668, support for the CH backend's merge agg capability was added. We can also support this feature in the velox backend. How was this patch tested? Through existing unit tests and GA verification, and additional new unit tests have been added. --- .../backendsapi/velox/VeloxRuleApi.scala | 1 + .../HashAggregateExecTransformer.scala | 14 +- .../tpch-approved-plan/v1-bhj/spark33/15.txt | 74 ++-- .../tpch-approved-plan/v1-bhj/spark34/15.txt | 74 ++-- .../tpch-approved-plan/v1/spark32/13.txt | 178 ++++---- .../tpch-approved-plan/v1/spark32/14.txt | 110 +++-- .../tpch-approved-plan/v1/spark32/17.txt | 170 ++++---- .../tpch-approved-plan/v1/spark32/18.txt | 284 +++++++------ .../tpch-approved-plan/v1/spark32/19.txt | 106 +++-- .../tpch-approved-plan/v1/spark32/20.txt | 390 +++++++++--------- .../tpch-approved-plan/v1/spark32/3.txt | 180 ++++---- .../tpch-approved-plan/v1/spark33/11.txt | 152 ++++--- .../tpch-approved-plan/v1/spark33/13.txt | 178 ++++---- .../tpch-approved-plan/v1/spark33/14.txt | 110 +++-- .../tpch-approved-plan/v1/spark33/15.txt | 74 ++-- .../tpch-approved-plan/v1/spark33/17.txt | 170 ++++---- .../tpch-approved-plan/v1/spark33/18.txt | 284 +++++++------ .../tpch-approved-plan/v1/spark33/19.txt | 106 +++-- .../tpch-approved-plan/v1/spark33/20.txt | 382 +++++++++-------- .../tpch-approved-plan/v1/spark33/3.txt | 180 ++++---- .../tpch-approved-plan/v1/spark34/11.txt | 152 ++++--- .../tpch-approved-plan/v1/spark34/13.txt | 178 ++++---- .../tpch-approved-plan/v1/spark34/14.txt | 110 +++-- .../tpch-approved-plan/v1/spark34/15.txt | 74 ++-- .../tpch-approved-plan/v1/spark34/17.txt | 170 ++++---- .../tpch-approved-plan/v1/spark34/18.txt | 284 +++++++------ .../tpch-approved-plan/v1/spark34/19.txt | 106 +++-- .../tpch-approved-plan/v1/spark34/20.txt | 382 +++++++++-------- .../tpch-approved-plan/v1/spark34/3.txt | 180 ++++---- .../VeloxAggregateFunctionsSuite.scala | 1 + docs/Configuration.md | 1 + .../MergeTwoPhasesHashBaseAggregate.scala | 5 +- .../GlutenReplaceHashWithSortAggSuite.scala | 9 +- .../GlutenReplaceHashWithSortAggSuite.scala | 2 +- .../GlutenReplaceHashWithSortAggSuite.scala | 37 +- ...MergeTwoPhasesHashBaseAggregateSuite.scala | 203 +++++++++ .../org/apache/gluten/GlutenConfig.scala | 9 + 37 files changed, 2559 insertions(+), 2561 deletions(-) rename {backends-clickhouse/src/main/scala/org/apache/gluten/extension => gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar}/MergeTwoPhasesHashBaseAggregate.scala (96%) create mode 100644 gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 62e3be67ca6b..67f9d21a5e7f 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -68,6 +68,7 @@ object VeloxRuleApi { injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) injector.injectPreTransform(c => FallbackMultiCodegens.apply(c.session)) + injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate(c.session)) injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session)) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala index 61af545e411d..9a6ba3a220f1 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala @@ -214,7 +214,7 @@ abstract class HashAggregateExecTransformer( VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction) ) aggregateNodeList.add(aggFunctionNode) - case Final => + case Final | Complete => val aggFunctionNode = ExpressionBuilder.makeAggregateFunction( VeloxAggregateFunctionsBuilder.create(args, aggregateFunction, aggregateMode), childrenNodeList, @@ -242,7 +242,7 @@ abstract class HashAggregateExecTransformer( aggregateFunction.inputAggBufferAttributes.head.nullable) ) aggregateNodeList.add(partialNode) - case Final => + case Final | Complete => val aggFunctionNode = ExpressionBuilder.makeAggregateFunction( VeloxAggregateFunctionsBuilder.create(args, aggregateFunction, aggregateMode), childrenNodeList, @@ -275,7 +275,7 @@ abstract class HashAggregateExecTransformer( expression.mode match { case Partial | PartialMerge => typeNodeList.add(VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction)) - case Final => + case Final | Complete => typeNodeList.add( ConverterUtils .getTypeNode(aggregateFunction.dataType, aggregateFunction.nullable)) @@ -356,7 +356,7 @@ abstract class HashAggregateExecTransformer( // The process of handling the inconsistency in column types and order between // Spark and Velox is exactly the opposite of applyExtractStruct. aggregateExpression.mode match { - case PartialMerge | Final => + case PartialMerge | Final | Complete => val newInputAttributes = new ArrayBuffer[Attribute]() val childNodes = new JArrayList[ExpressionNode]() val (sparkOrders, sparkTypes) = @@ -467,7 +467,7 @@ abstract class HashAggregateExecTransformer( // by previous projection. childrenNodes.add(ExpressionBuilder.makeSelection(colIdx)) colIdx += 1 - case Partial => + case Partial | Complete => aggFunc.children.foreach { _ => childrenNodes.add(ExpressionBuilder.makeSelection(colIdx)) @@ -600,7 +600,7 @@ abstract class HashAggregateExecTransformer( } val aggregateFunc = aggExpr.aggregateFunction val childrenNodes = aggExpr.mode match { - case Partial => + case Partial | Complete => aggregateFunc.children.toList.map( expr => { ExpressionConverter @@ -784,7 +784,7 @@ case class HashAggregateExecPullOutHelper( expr.mode match { case Partial | PartialMerge => expr.aggregateFunction.aggBufferAttributes - case Final => + case Final | Complete => Seq(aggregateAttributes(index)) case other => throw new GlutenNotSupportException(s"Unsupported aggregate mode: $other.") diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt index 74246ef67225..52a87f43535d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt @@ -237,31 +237,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ ScanTransformer parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) (44) ScanTransformer parquet @@ -326,71 +325,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt index a65fc78a2f03..48b06986ee1b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt @@ -239,31 +239,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ ScanTransformer parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) (44) ScanTransformer parquet @@ -328,71 +327,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt index 91444cca0528..c8ddb5437009 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt @@ -1,55 +1,54 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ ScanTransformer parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) (1) ScanTransformer parquet @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt index 33c0dfb71e1e..40a891738143 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt @@ -1,40 +1,39 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) (1) ScanTransformer parquet @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt index 94e892d1e8c9..30fd4b0b1124 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt @@ -1,62 +1,61 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ ScanTransformer parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) (1) ScanTransformer parquet @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7), true) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt index 550cd076c1c2..dd4cad538e2c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt @@ -1,102 +1,101 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ ScanTransformer parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ ScanTransformer parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ ScanTransformer parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) (1) ScanTransformer parquet @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt index f7778691955a..507b700e37e9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt @@ -1,39 +1,38 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) (1) ScanTransformer parquet @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt index 5dd6bbfccab6..13c10a7bba29 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt @@ -1,20 +1,20 @@ == Physical Plan == -AdaptiveSparkPlan (146) +AdaptiveSparkPlan (145) +- == Final Plan == - VeloxColumnarToRow (96) - +- ^ SortExecTransformer (94) - +- ^ InputIteratorTransformer (93) - +- ShuffleQueryStage (91) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (95) + +- ^ SortExecTransformer (93) + +- ^ InputIteratorTransformer (92) + +- ShuffleQueryStage (90) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) @@ -22,12 +22,12 @@ AdaptiveSparkPlan (146) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) @@ -48,83 +48,82 @@ AdaptiveSparkPlan (146) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ ScanTransformer parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ ScanTransformer parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ ScanTransformer parquet (77) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (145) - +- Exchange (144) - +- Project (143) - +- SortMergeJoin Inner (142) - :- Sort (136) - : +- Exchange (135) - : +- Project (134) - : +- SortMergeJoin LeftSemi (133) - : :- Sort (100) - : : +- Exchange (99) - : : +- Filter (98) - : : +- Scan parquet (97) - : +- Sort (132) - : +- Exchange (131) - : +- Project (130) - : +- SortMergeJoin Inner (129) - : :- Sort (112) - : : +- Exchange (111) - : : +- SortMergeJoin LeftSemi (110) - : : :- Sort (104) - : : : +- Exchange (103) - : : : +- Filter (102) - : : : +- Scan parquet (101) - : : +- Sort (109) - : : +- Exchange (108) - : : +- Project (107) - : : +- Filter (106) - : : +- Scan parquet (105) - : +- Sort (128) - : +- Exchange (127) - : +- Filter (126) - : +- HashAggregate (125) - : +- HashAggregate (124) - : +- SortMergeJoin LeftSemi (123) - : :- Sort (117) - : : +- Exchange (116) - : : +- Project (115) - : : +- Filter (114) - : : +- Scan parquet (113) - : +- Sort (122) - : +- Exchange (121) - : +- Project (120) - : +- Filter (119) - : +- Scan parquet (118) - +- Sort (141) - +- Exchange (140) - +- Project (139) - +- Filter (138) - +- Scan parquet (137) + Sort (144) + +- Exchange (143) + +- Project (142) + +- SortMergeJoin Inner (141) + :- Sort (135) + : +- Exchange (134) + : +- Project (133) + : +- SortMergeJoin LeftSemi (132) + : :- Sort (99) + : : +- Exchange (98) + : : +- Filter (97) + : : +- Scan parquet (96) + : +- Sort (131) + : +- Exchange (130) + : +- Project (129) + : +- SortMergeJoin Inner (128) + : :- Sort (111) + : : +- Exchange (110) + : : +- SortMergeJoin LeftSemi (109) + : : :- Sort (103) + : : : +- Exchange (102) + : : : +- Filter (101) + : : : +- Scan parquet (100) + : : +- Sort (108) + : : +- Exchange (107) + : : +- Project (106) + : : +- Filter (105) + : : +- Scan parquet (104) + : +- Sort (127) + : +- Exchange (126) + : +- Filter (125) + : +- HashAggregate (124) + : +- HashAggregate (123) + : +- SortMergeJoin LeftSemi (122) + : :- Sort (116) + : : +- Exchange (115) + : : +- Project (114) + : : +- Filter (113) + : : +- Scan parquet (112) + : +- Sort (121) + : +- Exchange (120) + : +- Project (119) + : +- Filter (118) + : +- Scan parquet (117) + +- Sort (140) + +- Exchange (139) + +- Project (138) + +- Filter (137) + +- Scan parquet (136) (1) ScanTransformer parquet @@ -327,417 +326,410 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) ScanTransformer parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) InputAdapter +(91) InputAdapter Input [2]: [s_name#X, s_address#X] -(93) InputIteratorTransformer +(92) InputIteratorTransformer Input [2]: [s_name#X, s_address#X] -(94) SortExecTransformer +(93) SortExecTransformer Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(95) WholeStageCodegenTransformer (X) +(94) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(96) VeloxColumnarToRow +(95) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(97) Scan parquet +(96) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(98) Filter +(97) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(99) Exchange +(98) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) Sort +(99) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(101) Scan parquet +(100) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(102) Filter +(101) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(103) Exchange +(102) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(104) Sort +(103) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(105) Scan parquet +(104) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(106) Filter +(105) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(107) Project +(106) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(108) Exchange +(107) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(110) SortMergeJoin +(109) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(111) Exchange +(110) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(112) Sort +(111) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(113) Scan parquet +(112) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(114) Filter +(113) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(115) Project +(114) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(116) Exchange +(115) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(117) Sort +(116) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(118) Scan parquet +(117) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(119) Filter +(118) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(120) Project +(119) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(121) Exchange +(120) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(124) HashAggregate +(123) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(125) HashAggregate +(124) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(126) Filter +(125) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(127) Exchange +(126) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(128) Sort +(127) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(129) SortMergeJoin +(128) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(130) Project +(129) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(131) Exchange +(130) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(132) Sort +(131) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(133) SortMergeJoin +(132) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(134) Project +(133) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(135) Exchange +(134) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(136) Sort +(135) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(137) Scan parquet +(136) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(138) Filter +(137) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(139) Project +(138) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(140) Exchange +(139) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(141) Sort +(140) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(142) SortMergeJoin +(141) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(143) Project +(142) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(144) Exchange +(143) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(145) Sort +(144) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(146) AdaptiveSparkPlan +(145) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt index c4df05de1618..2e427b6bf3ca 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt @@ -1,64 +1,63 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ ScanTransformer parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) (1) ScanTransformer parquet @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt index cc1f4d31b697..55d544b898c6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt @@ -424,55 +424,54 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ ScanTransformer parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) (83) ScanTransformer parquet @@ -581,129 +580,122 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt index bd84f11c6ac5..7c3509c3e726 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt @@ -1,55 +1,54 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ ScanTransformer parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) (1) ScanTransformer parquet @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt index 5fdc418d02ef..1161b32937bc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt @@ -1,40 +1,39 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) (1) ScanTransformer parquet @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt index 2d327719c3bd..398cc1f5b8b9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt @@ -257,31 +257,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ ScanTransformer parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) (48) ScanTransformer parquet @@ -346,71 +345,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt index c5033fa773a5..843ce9b39eb6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt @@ -1,62 +1,61 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ ScanTransformer parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) (1) ScanTransformer parquet @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7)) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt index b07b35495922..1119ddcf80be 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt @@ -1,102 +1,101 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ ScanTransformer parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ ScanTransformer parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ ScanTransformer parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) (1) ScanTransformer parquet @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt index 348cc4ce85d5..31341db339e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt @@ -1,39 +1,38 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) (1) ScanTransformer parquet @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt index 703c4a99d63c..9a2c56040b38 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt @@ -1,19 +1,19 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) @@ -21,12 +21,12 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -47,83 +47,82 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ ScanTransformer parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ ScanTransformer parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ ScanTransformer parquet (77) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) (1) ScanTransformer parquet @@ -326,407 +325,400 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) ScanTransformer parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt index f6dc9c8015ea..d09fe3e19a9b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt @@ -1,64 +1,63 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ ScanTransformer parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) (1) ScanTransformer parquet @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt index 5278039ec385..e2d5e58eba47 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt @@ -428,55 +428,54 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ ScanTransformer parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) (83) ScanTransformer parquet @@ -587,131 +586,124 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join type: Inner Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt index ca06d654b432..59abfb682c42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt @@ -1,55 +1,54 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ ScanTransformer parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) (1) ScanTransformer parquet @@ -134,181 +133,174 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: LeftOuter Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt index c61d21b92ed1..531cc5aaab55 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt @@ -1,40 +1,39 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) (1) ScanTransformer parquet @@ -124,94 +123,87 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt index 3585c0cc2f34..f370146f9206 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt @@ -259,31 +259,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ ScanTransformer parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) (48) ScanTransformer parquet @@ -348,71 +347,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt index c6d0e0103363..c1b43b7fc421 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt @@ -1,62 +1,61 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ ScanTransformer parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) (1) ScanTransformer parquet @@ -215,141 +214,134 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [(0.2 * avg(l_quantity#X)#X) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join type: Inner Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt index 50a076b1ef8f..c1a6b181bb34 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt @@ -1,102 +1,101 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ ScanTransformer parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ ScanTransformer parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ ScanTransformer parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) (1) ScanTransformer parquet @@ -375,223 +374,216 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt index fb7e231fa5b3..4db9ca0c8393 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt @@ -1,39 +1,38 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) (1) ScanTransformer parquet @@ -123,90 +122,83 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt index 4e427961a4f6..4d8eee1f1fb7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt @@ -1,19 +1,19 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) @@ -21,12 +21,12 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -47,83 +47,82 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ ScanTransformer parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ ScanTransformer parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ ScanTransformer parquet (77) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) (1) ScanTransformer parquet @@ -328,415 +327,408 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) ScanTransformer parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt index 4aa745832681..1f86369a62af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt @@ -1,64 +1,63 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ ScanTransformer parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) (1) ScanTransformer parquet @@ -217,143 +216,136 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala index 84d1fe8db919..cf0bf272bc5b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala @@ -49,6 +49,7 @@ abstract class VeloxAggregateFunctionsSuite extends VeloxWholeStageTransformerSu .set("spark.unsafe.exceptionOnMemoryLeak", "true") .set("spark.sql.autoBroadcastJoinThreshold", "-1") .set("spark.sql.sources.useV1SourceList", "avro") + .set("spark.gluten.sql.mergeTwoPhasesAggregate.enabled", "false") } test("count") { diff --git a/docs/Configuration.md b/docs/Configuration.md index d4ca9c10fb76..0c44341c363c 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -59,6 +59,7 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.gluten.sql.columnar.fallback.ignoreRowToColumnar | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | true | | spark.gluten.sql.columnar.fallback.preferColumnar | When true, the fallback policy prefers to use Gluten plan rather than vanilla Spark plan if the both of them contains ColumnarToRow and the vanilla Spark plan ColumnarToRow number is not smaller than Gluten plan. | true | | spark.gluten.sql.columnar.force.hashagg | Force to use hash agg to replace sort agg. | true | +| spark.gluten.sql.mergeTwoPhasesAggregate.enabled | Whether to merge two phases aggregate if there are no other operators between them. | true | | spark.gluten.sql.columnar.vanillaReaders | Enable vanilla spark's vectorized reader. Please note it may bring perf. overhead due to extra data transition. We recommend to disable it if most queries can be fully offloaded to gluten. | false | | spark.gluten.sql.native.bloomFilter | Enable or Disable native runtime bloom filter. | true | | spark.gluten.sql.native.arrow.reader.enabled | Enable or Disable native arrow read CSV file format | false | diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MergeTwoPhasesHashBaseAggregate.scala similarity index 96% rename from backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MergeTwoPhasesHashBaseAggregate.scala index a10659b6d5e7..a034a3229a88 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MergeTwoPhasesHashBaseAggregate.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension +package org.apache.gluten.extension.columnar import org.apache.gluten.GlutenConfig @@ -42,6 +42,7 @@ case class MergeTwoPhasesHashBaseAggregate(session: SparkSession) val scanOnly: Boolean = glutenConf.enableScanOnly val enableColumnarHashAgg: Boolean = !scanOnly && glutenConf.enableColumnarHashAgg val replaceSortAggWithHashAgg: Boolean = GlutenConfig.getConf.forceToUseHashAgg + val mergeTwoPhasesAggEnabled: Boolean = GlutenConfig.getConf.mergeTwoPhasesAggEnabled private def isPartialAgg(partialAgg: BaseAggregateExec, finalAgg: BaseAggregateExec): Boolean = { // TODO: now it can not support to merge agg which there are the filters in the aggregate exprs. @@ -59,7 +60,7 @@ case class MergeTwoPhasesHashBaseAggregate(session: SparkSession) } override def apply(plan: SparkPlan): SparkPlan = { - if (!enableColumnarHashAgg) { + if (!mergeTwoPhasesAggEnabled || !enableColumnarHashAgg) { plan } else { plan.transformDown { diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala index f394b4687d3d..33bf1a1ec97e 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution import org.apache.gluten.execution.HashAggregateExecBaseTransformer -import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} import org.apache.spark.sql.execution.aggregate.{ObjectHashAggregateExec, SortAggregateExec} @@ -85,7 +84,7 @@ class GlutenReplaceHashWithSortAggSuite withTempView("t1", "t2") { spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") - Seq(("COUNT", 0, 1, 2, 0), ("COLLECT_LIST", 2, 0, 2, 0)).foreach { + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { aggExprInfo => val query = s""" @@ -99,11 +98,7 @@ class GlutenReplaceHashWithSortAggSuite |) |GROUP BY key """.stripMargin - if (BackendTestUtils.isCHBackendLoaded()) { - checkAggs(query, 1, 0, 1, 0) - } else { - checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) - } + checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) } } } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala index 332c21418a9b..33bf1a1ec97e 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -84,7 +84,7 @@ class GlutenReplaceHashWithSortAggSuite withTempView("t1", "t2") { spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") - Seq(("COUNT", 0, 1, 2, 0), ("COLLECT_LIST", 2, 0, 2, 0)).foreach { + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { aggExprInfo => val query = s""" diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala index 332c21418a9b..29ccd9b71dcd 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -81,24 +81,27 @@ class GlutenReplaceHashWithSortAggSuite } testGluten("replace partial and final hash aggregate together with sort aggregate") { - withTempView("t1", "t2") { - spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") - spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") - Seq(("COUNT", 0, 1, 2, 0), ("COLLECT_LIST", 2, 0, 2, 0)).foreach { - aggExprInfo => - val query = - s""" - |SELECT key, ${aggExprInfo._1}(key) - |FROM - |( - | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key - | FROM t1 - | JOIN t2 - | ON t1.key = t2.key - |) - |GROUP BY key + // TODO: Should can merge aggregates even if RAS support. + withSQLConf("spark.gluten.ras.enabled" -> "false") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { + aggExprInfo => + val query = + s""" + |SELECT key, ${aggExprInfo._1}(key) + |FROM + |( + | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key """.stripMargin - checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) + checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) + } } } } diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala new file mode 100644 index 000000000000..f65c71f77cf5 --- /dev/null +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + +abstract class BaseMergeTwoPhasesHashBaseAggregateSuite extends WholeStageTransformerSuite { + val fileFormat: String = "parquet" + override protected val resourcePath: String = "/tpch-data-parquet" + + override def beforeAll(): Unit = { + super.beforeAll() + + spark + .sql(s""" + |CREATE TABLE t1 (id int, age int, phone int, date string) + |USING $fileFormat + |PARTITIONED BY (date) + |""".stripMargin) + .show() + + spark + .sql(s""" + |INSERT INTO t1 PARTITION(date = '2020-01-01') + |SELECT id, id % 10 as age, id % 10 as phone + |FROM range(100) + |""".stripMargin) + .show() + } + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.sql.files.maxPartitionBytes", "1g") + .set("spark.sql.shuffle.partitions", "1") + .set("spark.memory.offHeap.size", "2g") + .set("spark.gluten.ras.enabled", "false") + .set("spark.gluten.sql.mergeTwoPhasesAggregate.enabled", "true") + if (BackendTestUtils.isCHBackendLoaded()) { + conf + .set("spark.gluten.sql.enable.native.validation", "false") + .set(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + } + conf + } + + override def afterAll(): Unit = { + try { + sql("DROP TABLE IF EXISTS t1") + } finally { + super.afterAll() + } + } + + test("Merge two phase hash-based aggregate into one aggregate") { + def checkHashAggregateCount(df: DataFrame, expectedCount: Int): Unit = { + df.collect() + val plans = collect(df.queryExecution.executedPlan) { + case agg: HashAggregateExecBaseTransformer => agg + } + assert(plans.size == expectedCount) + } + + withTempView("v1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("v1") + // no exchange hash aggregate, merge to one hash aggregate + checkHashAggregateCount( + spark.sql(""" + |SELECT key, count(key) + |FROM v1 + |GROUP BY key + |""".stripMargin), + 1 + ) + + // with filter hash aggregate + checkHashAggregateCount( + spark.sql(""" + |SELECT key, count(key) FILTER (WHERE key LIKE '%1%') AS pc2 + |FROM v1 + |GROUP BY key + |""".stripMargin), + 2 + ) + } + + // with exchange hash aggregate + checkHashAggregateCount( + spark.sql(""" + |SELECT count(1) FROM t1 + |""".stripMargin), + 2) + } + + test("Merge two phase object-based aggregate into one aggregate") { + def checkObjectAggregateCount(df: DataFrame, expectedCount: Int): Unit = { + df.collect() + val plans = collect(df.queryExecution.executedPlan) { + case agg: HashAggregateExecBaseTransformer => agg + } + assert(plans.size == expectedCount) + } + + withTempView("v1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("v1") + // no exchange object aggregate, merge to one hash aggregate + checkObjectAggregateCount( + spark.sql(""" + |SELECT key, collect_list(key) + |FROM v1 + |GROUP BY key + |""".stripMargin), + 1 + ) + + // with filter object aggregate + checkObjectAggregateCount( + spark.sql(""" + |SELECT key, collect_list(key) FILTER (WHERE key LIKE '%1%') AS pc2 + |FROM v1 + |GROUP BY key + |""".stripMargin), + 2 + ) + } + + // with exchange object aggregate + checkObjectAggregateCount( + spark.sql(""" + |SELECT collect_list(id) FROM t1 + |""".stripMargin), + 2) + } + + test("Merge two phase sort-based aggregate into one aggregate") { + def checkSortAggregateCount(df: DataFrame, expectedCount: Int): Unit = { + df.collect() + val plans = collect(df.queryExecution.executedPlan) { + case agg: HashAggregateExecBaseTransformer => agg + } + assert(plans.size == expectedCount) + } + + withSQLConf("spark.sql.test.forceApplySortAggregate" -> "true") { + withTempView("v1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("v1") + // no exchange sort aggregate, merge to one hash aggregate + checkSortAggregateCount( + spark.sql(""" + |SELECT sum(if(key<0,0,key)) + |FROM v1 + |GROUP BY key + |""".stripMargin), + 1 + ) + + // with filter sort aggregate + checkSortAggregateCount( + spark.sql(""" + |SELECT key, sum(if(key<0,0,key)) FILTER (WHERE key LIKE '%1%') AS pc2 + |FROM v1 + |GROUP BY key + |""".stripMargin), + 2 + ) + } + + // with exchange sort aggregate + checkSortAggregateCount( + spark.sql(""" + |SELECT sum(if(id<0,0,id)) FROM t1 + |""".stripMargin), + 2) + } + } +} + +class MergeTwoPhasesAggregateSuiteAEOn + extends BaseMergeTwoPhasesHashBaseAggregateSuite + with EnableAdaptiveExecutionSuite + +class MergeTwoPhasesAggregateSuiteAEOff + extends BaseMergeTwoPhasesHashBaseAggregateSuite + with DisableAdaptiveExecutionSuite diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 107c33a241fd..c46d87469036 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -59,6 +59,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def forceToUseHashAgg: Boolean = conf.getConf(COLUMNAR_FORCE_HASHAGG_ENABLED) + def mergeTwoPhasesAggEnabled: Boolean = conf.getConf(MERGE_TWO_PHASES_ENABLED) + def enableColumnarProject: Boolean = conf.getConf(COLUMNAR_PROJECT_ENABLED) def enableColumnarFilter: Boolean = conf.getConf(COLUMNAR_FILTER_ENABLED) @@ -898,6 +900,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val MERGE_TWO_PHASES_ENABLED = + buildConf("spark.gluten.sql.mergeTwoPhasesAggregate.enabled") + .internal() + .doc("Whether to merge two phases aggregate if there are no other operators between them.") + .booleanConf + .createWithDefault(true) + val COLUMNAR_PROJECT_ENABLED = buildConf("spark.gluten.sql.columnar.project") .internal() From d796e08db0744417a98b0ecba27f34d118450682 Mon Sep 17 00:00:00 2001 From: Yang Zhang Date: Wed, 20 Nov 2024 11:53:04 +0800 Subject: [PATCH 120/211] [GLUTEN-7800][VL] Add config for max reclaim wait time to avoid dead lock when memory arbitration (#7990) --- cpp/velox/config/VeloxConfig.h | 3 +++ cpp/velox/memory/VeloxMemoryManager.cc | 21 ++++++++++++++----- .../org/apache/gluten/GlutenConfig.scala | 7 +++++++ 3 files changed, 26 insertions(+), 5 deletions(-) diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index 792beda96f7d..cb70dc62780b 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -76,6 +76,9 @@ const bool kMemoryUseHugePagesDefault = false; const std::string kVeloxMemInitCapacity = "spark.gluten.sql.columnar.backend.velox.memInitCapacity"; const uint64_t kVeloxMemInitCapacityDefault = 8 << 20; +const std::string kVeloxMemReclaimMaxWaitMs = "spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs"; +const uint64_t kVeloxMemReclaimMaxWaitMsDefault = 3600000; // 60min + const std::string kHiveConnectorId = "test-hive"; const std::string kVeloxCacheEnabled = "spark.gluten.sql.columnar.backend.velox.cacheEnabled"; diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 1d281b22dc86..32978281ef60 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -41,6 +41,8 @@ static constexpr std::string_view kMemoryPoolInitialCapacity{"memory-pool-initia static constexpr uint64_t kDefaultMemoryPoolInitialCapacity{256 << 20}; static constexpr std::string_view kMemoryPoolTransferCapacity{"memory-pool-transfer-capacity"}; static constexpr uint64_t kDefaultMemoryPoolTransferCapacity{128 << 20}; +static constexpr std::string_view kMemoryReclaimMaxWaitMs{"memory-reclaim-max-wait-time"}; +static constexpr std::string_view kDefaultMemoryReclaimMaxWaitMs{"3600000ms"}; template T getConfig( @@ -74,7 +76,13 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { config.extraConfigs, kMemoryPoolTransferCapacity, std::to_string(kDefaultMemoryPoolTransferCapacity)), - velox::config::CapacityUnit::BYTE)) {} + velox::config::CapacityUnit::BYTE)), + memoryReclaimMaxWaitMs_( + std::chrono::duration_cast(velox::config::toDuration(getConfig( + config.extraConfigs, + kMemoryReclaimMaxWaitMs, + std::string(kDefaultMemoryReclaimMaxWaitMs)))) + .count()) {} std::string kind() const override { return kind_; } @@ -123,7 +131,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { VELOX_CHECK_EQ(candidates_.size(), 1, "ListenableArbitrator should only be used within a single root pool"); pool = candidates_.begin()->first; } - pool->reclaim(targetBytes, 0, status); // ignore the output + pool->reclaim(targetBytes, memoryReclaimMaxWaitMs_, status); // ignore the output return shrinkCapacity0(pool, 0); } @@ -172,6 +180,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { gluten::AllocationListener* listener_; const uint64_t memoryPoolInitialCapacity_; // FIXME: Unused. const uint64_t memoryPoolTransferCapacity_; + const uint64_t memoryReclaimMaxWaitMs_; mutable std::mutex mutex_; inline static std::string kind_ = "GLUTEN"; @@ -210,14 +219,16 @@ VeloxMemoryManager::VeloxMemoryManager(const std::string& kind, std::unique_ptr< kMemoryReservationBlockSize, kMemoryReservationBlockSizeDefault); auto memInitCapacity = VeloxBackend::get()->getBackendConf()->get(kVeloxMemInitCapacity, kVeloxMemInitCapacityDefault); + auto memReclaimMaxWaitMs = + VeloxBackend::get()->getBackendConf()->get(kVeloxMemReclaimMaxWaitMs, kVeloxMemReclaimMaxWaitMsDefault); blockListener_ = std::make_unique(listener_.get(), reservationBlockSize); listenableAlloc_ = std::make_unique(defaultMemoryAllocator().get(), blockListener_.get()); arrowPool_ = std::make_unique(listenableAlloc_.get()); std::unordered_map extraArbitratorConfigs; - extraArbitratorConfigs["memory-pool-initial-capacity"] = folly::to(memInitCapacity) + "B"; - extraArbitratorConfigs["memory-pool-transfer-capacity"] = folly::to(reservationBlockSize) + "B"; - extraArbitratorConfigs["memory-reclaim-max-wait-time"] = folly::to(0) + "ms"; + extraArbitratorConfigs[std::string(kMemoryPoolInitialCapacity)] = folly::to(memInitCapacity) + "B"; + extraArbitratorConfigs[std::string(kMemoryPoolTransferCapacity)] = folly::to(reservationBlockSize) + "B"; + extraArbitratorConfigs[std::string(kMemoryReclaimMaxWaitMs)] = folly::to(memReclaimMaxWaitMs) + "ms"; ArbitratorFactoryRegister afr(listener_.get()); velox::memory::MemoryManagerOptions mmOptions{ diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index c46d87469036..f756eb20a6d3 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -1452,6 +1452,13 @@ object GlutenConfig { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("8MB") + val COLUMNAR_VELOX_MEM_RECLAIM_MAX_WAIT_MS = + buildConf("spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs") + .internal() + .doc("The max time in ms to wait for memory reclaim.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(TimeUnit.MINUTES.toMillis(60)) + val COLUMNAR_VELOX_SSD_CACHE_PATH = buildStaticConf("spark.gluten.sql.columnar.backend.velox.ssdCachePath") .internal() From 1f3e6b5e38e38b7508ac8f67a2a426aa9137c0e1 Mon Sep 17 00:00:00 2001 From: lgbo Date: Wed, 20 Nov 2024 11:55:46 +0800 Subject: [PATCH 121/211] [GLUTEN-7986][CH] Improve lazy expand for high cardinality aggregation (#7995) [CH] Improve lazy expand for high cardinality aggregation --- .../Operator/AdvancedExpandStep.cpp | 91 ++++++------------- .../Operator/AdvancedExpandStep.h | 13 ++- 2 files changed, 34 insertions(+), 70 deletions(-) diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp index ee767b31bd21..72721ce85921 100644 --- a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp @@ -25,7 +25,7 @@ #include #include #include -#include +#include #include #include #include @@ -110,8 +110,7 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, auto expand_output_header = expand_processor->getOutputs().front().getHeader(); auto transform_params = std::make_shared(expand_output_header, params, false); - auto aggregate_processor - = std::make_shared(expand_output_header, transform_params, context, false, false); + auto aggregate_processor = std::make_shared(context, expand_output_header, transform_params); DB::connect(expand_processor->getOutputs().back(), aggregate_processor->getInputs().front()); new_processors.push_back(aggregate_processor); auto aggregate_output_header = aggregate_processor->getOutputs().front().getHeader(); @@ -146,8 +145,15 @@ AdvancedExpandTransform::AdvancedExpandTransform( , project_set_exprs(project_set_exprs_) , input_header(input_header_) { - for (size_t i = 0; i < project_set_exprs.getKinds().size(); ++i) - is_low_cardinality_expand.push_back(true); + for (size_t i = 0; i < project_set_exprs.getExpandRows(); ++i) + { + const auto & kinds = project_set_exprs.getKinds()[i]; + size_t n = 0; + for (size_t k = 0; k < grouping_keys; ++k) + if (kinds[k] == EXPAND_FIELD_KIND_SELECTION) + n += 1; + need_to_aggregate.push_back((n != grouping_keys)); + } for (auto & port : outputs) output_ports.push_back(&port); @@ -167,9 +173,11 @@ DB::IProcessor::Status AdvancedExpandTransform::prepare() if (has_output) { - auto & output_port = *output_ports[is_low_cardinality_expand[expand_expr_iterator - 1]]; + auto & output_port = *output_ports[need_to_aggregate[expand_expr_iterator - 1]]; if (output_port.canPush()) { + output_blocks[need_to_aggregate[expand_expr_iterator - 1]] += 1; + output_rows[need_to_aggregate[expand_expr_iterator - 1]] += output_chunk.getNumRows(); output_port.push(std::move(output_chunk)); has_output = false; auto status = expand_expr_iterator >= project_set_exprs.getExpandRows() ? Status::NeedData : Status::Ready; @@ -185,17 +193,18 @@ DB::IProcessor::Status AdvancedExpandTransform::prepare() { if (input.isFinished()) { - if (!cardinality_detect_blocks.empty()) - { - input_finished = true; - return Status::Ready; - } - else - { - output_ports[0]->finish(); - output_ports[1]->finish(); - return Status::Finished; - } + LOG_DEBUG( + getLogger("AdvancedExpandTransform"), + "Input rows/blocks={}/{}. output rows/blocks=[{}/{}, {}/{}]", + input_rows, + input_blocks, + output_rows[0], + output_blocks[0], + output_rows[1], + output_blocks[1]); + output_ports[0]->finish(); + output_ports[1]->finish(); + return Status::Finished; } input.setNeeded(); @@ -204,6 +213,8 @@ DB::IProcessor::Status AdvancedExpandTransform::prepare() input_chunk = input.pull(true); has_input = true; expand_expr_iterator = 0; + input_blocks += 1; + input_rows += input_chunk.getNumRows(); } return Status::Ready; @@ -211,55 +222,9 @@ DB::IProcessor::Status AdvancedExpandTransform::prepare() void AdvancedExpandTransform::work() { - if (!input_finished && cardinality_detect_rows < rows_for_detect_cardinality) - { - cardinality_detect_blocks.push_back(input_header.cloneWithColumns(input_chunk.detachColumns())); - cardinality_detect_rows += cardinality_detect_blocks.back().rows(); - has_input = false; - } - if ((input_finished || cardinality_detect_rows >= rows_for_detect_cardinality) && !cardinality_detect_blocks.empty()) - detectCardinality(); - else if (!input_finished && cardinality_detect_rows < rows_for_detect_cardinality) - return; - - /// The phase of detecting grouping keys' cardinality is finished here. expandInputChunk(); } -void AdvancedExpandTransform::detectCardinality() -{ - DB::Block block = BlockUtil::concatenateBlocksMemoryEfficiently(std::move(cardinality_detect_blocks)); - std::vector is_col_low_cardinality; - for (size_t i = 0; i < grouping_keys; ++i) - { - DB::WeakHash32 hash = block.getByPosition(i).column->getWeakHash32(); - std::unordered_set distinct_ids; - const auto & data = hash.getData(); - for (size_t j = 0; j < cardinality_detect_rows; ++j) - distinct_ids.insert(data[j]); - size_t distinct_ids_cnt = distinct_ids.size(); - is_col_low_cardinality.push_back(distinct_ids.size() < 1000); - } - - for (size_t i = 0; i < project_set_exprs.getExpandRows(); ++i) - { - const auto & kinds = project_set_exprs.getKinds()[i]; - for (size_t k = 0; k < grouping_keys; ++k) - { - const auto & kind = kinds[k]; - if (kind == EXPAND_FIELD_KIND_SELECTION && !is_col_low_cardinality[k]) - { - is_low_cardinality_expand[i] = false; - break; - } - } - } - LOG_DEBUG(getLogger("AdvancedExpandTransform"), "Low cardinality expand: {}", fmt::join(is_low_cardinality_expand, ",")); - - input_chunk = DB::Chunk(block.getColumns(), block.rows()); - cardinality_detect_blocks.clear(); -} - void AdvancedExpandTransform::expandInputChunk() { const auto & input_columns = input_chunk.getColumns(); diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.h b/cpp-ch/local-engine/Operator/AdvancedExpandStep.h index 295084658531..343a7f5227ee 100644 --- a/cpp-ch/local-engine/Operator/AdvancedExpandStep.h +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.h @@ -78,19 +78,18 @@ class AdvancedExpandTransform : public DB::IProcessor bool has_input = false; bool has_output = false; size_t expand_expr_iterator = 0; - std::vector is_low_cardinality_expand; - std::vector approximate_grouping_keys; - size_t cardinality_detect_rows = 0; - std::vector cardinality_detect_blocks; - static constexpr size_t rows_for_detect_cardinality = 10000; - bool input_finished = false; + std::vector need_to_aggregate; std::vector output_ports; DB::Chunk input_chunk; DB::Chunk output_chunk; - void detectCardinality(); + size_t input_blocks = 0; + size_t input_rows = 0; + std::vector output_blocks = {0, 0}; + std::vector output_rows = {0, 0}; + void expandInputChunk(); }; From 095f4fc2aae26ea0aa27e501596dc60e82f7a940 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 20 Nov 2024 13:30:49 +0800 Subject: [PATCH 122/211] [CORE] Minor: Use lower case for Maven profile names (#8001) --- pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 030828a327d6..8a2968b56905 100644 --- a/pom.xml +++ b/pom.xml @@ -530,7 +530,7 @@ - Darwin-x86 + darwin-x86 mac @@ -543,7 +543,7 @@ - Darwin-aarch64 + darwin-aarch64 mac @@ -556,7 +556,7 @@ - Linux-amd64 + linux-amd64 Linux @@ -569,7 +569,7 @@ - Linux-aarch64 + linux-aarch64 Linux From c0bd7f25e7dacef2b4d76db1dd52078e58db2dc3 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 20 Nov 2024 13:31:00 +0800 Subject: [PATCH 123/211] [CORE] Query planner: A more explicit practice to register columnar batch types (#8002) --- .../clickhouse/CHListenerApi.scala | 2 +- .../apache/gluten/columnarbatch/CHBatch.scala | 6 ++- .../backendsapi/velox/VeloxListenerApi.scala | 6 +-- .../gluten/columnarbatch/VeloxBatch.scala | 10 +++-- .../gluten/columnarbatch/ArrowBatches.scala | 10 +++-- .../enumerated/planner/property/Conv.scala | 4 +- .../columnar/transition/Convention.scala | 43 +++++++++++++++---- .../columnar/transition/Transitions.scala | 9 ++-- .../EnsureLocalSortRequirements.scala | 2 +- .../columnar/transition/TransitionSuite.scala | 26 +++++++---- 10 files changed, 80 insertions(+), 38 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala index 6ae957912ac5..b93c002561f0 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala @@ -71,7 +71,7 @@ class CHListenerApi extends ListenerApi with Logging { private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { // Force batch type initializations. - CHBatch.getClass + CHBatch.ensureRegistered() SparkDirectoryUtil.init(conf) val libPath = conf.get(GlutenConfig.GLUTEN_LIB_PATH, StringUtils.EMPTY) if (StringUtils.isBlank(libPath)) { diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala index 870a731b1176..ac0ca5f8b4b7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala @@ -38,6 +38,8 @@ import org.apache.spark.sql.execution.{CHColumnarToRowExec, RowToCHNativeColumna * }}} */ object CHBatch extends Convention.BatchType { - fromRow(RowToCHNativeColumnarExec.apply) - toRow(CHColumnarToRowExec.apply) + override protected def registerTransitions(): Unit = { + fromRow(RowToCHNativeColumnarExec.apply) + toRow(CHColumnarToRowExec.apply) + } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index a33fd22812dc..85de0a888952 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -123,9 +123,9 @@ class VeloxListenerApi extends ListenerApi with Logging { private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { // Force batch type initializations. - VeloxBatch.getClass - ArrowJavaBatch.getClass - ArrowNativeBatch.getClass + VeloxBatch.ensureRegistered() + ArrowJavaBatch.ensureRegistered() + ArrowNativeBatch.ensureRegistered() // Sets this configuration only once, since not undoable. if (conf.getBoolean(GlutenConfig.GLUTEN_DEBUG_KEEP_JNI_WORKSPACE, defaultValue = false)) { diff --git a/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala b/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala index 0c7600c856b1..5d9a78d31874 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala @@ -20,8 +20,10 @@ import org.apache.gluten.execution.{ArrowColumnarToVeloxColumnarExec, RowToVelox import org.apache.gluten.extension.columnar.transition.{Convention, Transition} object VeloxBatch extends Convention.BatchType { - fromRow(RowToVeloxColumnarExec.apply) - toRow(VeloxColumnarToRowExec.apply) - fromBatch(ArrowBatches.ArrowNativeBatch, ArrowColumnarToVeloxColumnarExec.apply) - toBatch(ArrowBatches.ArrowNativeBatch, Transition.empty) + override protected def registerTransitions(): Unit = { + fromRow(RowToVeloxColumnarExec.apply) + toRow(VeloxColumnarToRowExec.apply) + fromBatch(ArrowBatches.ArrowNativeBatch, ArrowColumnarToVeloxColumnarExec.apply) + toBatch(ArrowBatches.ArrowNativeBatch, Transition.empty) + } } diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala b/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala index 5ae3863c57a8..c23d6eea794e 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala @@ -33,7 +33,9 @@ object ArrowBatches { * implementations. */ object ArrowJavaBatch extends Convention.BatchType { - toBatch(VanillaBatch, Transition.empty) + override protected def registerTransitions(): Unit = { + toBatch(VanillaBatch, Transition.empty) + } } /** @@ -44,7 +46,9 @@ object ArrowBatches { * [[ColumnarBatches]]. */ object ArrowNativeBatch extends Convention.BatchType { - fromBatch(ArrowJavaBatch, OffloadArrowDataExec.apply) - toBatch(ArrowJavaBatch, LoadArrowDataExec.apply) + override protected def registerTransitions(): Unit = { + fromBatch(ArrowJavaBatch, OffloadArrowDataExec.apply) + toBatch(ArrowJavaBatch, LoadArrowDataExec.apply) + } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala index e9ca836eee8a..831b212e1f96 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala @@ -34,7 +34,7 @@ sealed trait Conv extends Property[SparkPlan] { return true } val prop = this.asInstanceOf[Prop] - val out = Transition.factory.satisfies(prop.prop, req.req) + val out = Transition.factory().satisfies(prop.prop, req.req) out } } @@ -52,7 +52,7 @@ object Conv { def findTransition(from: Conv, to: Conv): Transition = { val prop = from.asInstanceOf[Prop] val req = to.asInstanceOf[Req] - val out = Transition.factory.findTransition(prop.prop, req.req, new IllegalStateException()) + val out = Transition.factory().findTransition(prop.prop, req.req, new IllegalStateException()) out } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala index 55bcb84d2b10..840b62fb672d 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala @@ -18,6 +18,8 @@ package org.apache.gluten.extension.columnar.transition import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} +import java.util.concurrent.atomic.AtomicBoolean + /** * Convention of a query plan consists of the row data type and columnar data type it supports to * output. @@ -72,22 +74,43 @@ object Convention { } trait BatchType extends TransitionGraph.Vertex with Serializable { - Transition.graph.addVertex(this) + private val initialized: AtomicBoolean = new AtomicBoolean(false) + + final def ensureRegistered(): Unit = { + if (!initialized.compareAndSet(false, true)) { + // Already registered. + return + } + register() + } - final protected def fromRow(transition: Transition): Unit = { + final private def register(): Unit = { + Transition.graph.addVertex(this) + registerTransitions() + } + + ensureRegistered() + + /** + * User batch type could override this method to define transitions from/to this batch type by + * calling the subsequent protected APIs. + */ + protected[this] def registerTransitions(): Unit + + final protected[this] def fromRow(transition: Transition): Unit = { Transition.graph.addEdge(RowType.VanillaRow, this, transition) } - final protected def toRow(transition: Transition): Unit = { + final protected[this] def toRow(transition: Transition): Unit = { Transition.graph.addEdge(this, RowType.VanillaRow, transition) } - final protected def fromBatch(from: BatchType, transition: Transition): Unit = { + final protected[this] def fromBatch(from: BatchType, transition: Transition): Unit = { assert(from != this) Transition.graph.addEdge(from, this, transition) } - final protected def toBatch(to: BatchType, transition: Transition): Unit = { + final protected[this] def toBatch(to: BatchType, transition: Transition): Unit = { assert(to != this) Transition.graph.addEdge(this, to, transition) } @@ -95,10 +118,14 @@ object Convention { object BatchType { // None indicates that the plan doesn't support batch-based processing. - final case object None extends BatchType + final case object None extends BatchType { + override protected[this] def registerTransitions(): Unit = {} + } final case object VanillaBatch extends BatchType { - fromRow(RowToColumnarExec.apply) - toRow(ColumnarToRowExec.apply) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToColumnarExec.apply) + toRow(ColumnarToRowExec.apply) + } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala index 9987a65b0c4c..2f2840b52b7f 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala @@ -54,7 +54,7 @@ case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { child } else { val transition = - Transition.factory.findTransition(from, convReq, Transition.notFound(node)) + Transition.factory().findTransition(from, convReq, Transition.notFound(node)) val newChild = transition.apply(child) newChild } @@ -108,10 +108,9 @@ object Transitions { private def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { val convFunc = ConventionFunc.create() val removed = RemoveTransitions.removeForNode(plan) - val transition = Transition.factory.findTransition( - convFunc.conventionOf(removed), - req, - Transition.notFound(removed, req)) + val transition = Transition + .factory() + .findTransition(convFunc.conventionOf(removed), req, Transition.notFound(removed, req)) val out = transition.apply(removed) out } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala index 29a765288572..056315186df1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.{SortExec, SparkPlan} * SortAggregate with the same key. So, this rule adds local sort back if necessary. */ object EnsureLocalSortRequirements extends Rule[SparkPlan] { - private lazy val transform = HeuristicTransform.static() + private lazy val transform: HeuristicTransform = HeuristicTransform.static() private def addLocalSort( originalChild: SparkPlan, diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala index 5c35cb50207a..9712bd2c219f 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala @@ -87,23 +87,31 @@ class TransitionSuite extends SharedSparkSession { object TransitionSuite extends TransitionSuiteBase { object TypeA extends Convention.BatchType { - fromRow(RowToBatch(this, _)) - toRow(BatchToRow(this, _)) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToBatch(this, _)) + toRow(BatchToRow(this, _)) + } } object TypeB extends Convention.BatchType { - fromRow(RowToBatch(this, _)) - toRow(BatchToRow(this, _)) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToBatch(this, _)) + toRow(BatchToRow(this, _)) + } } object TypeC extends Convention.BatchType { - fromRow(RowToBatch(this, _)) - toRow(BatchToRow(this, _)) - fromBatch(TypeA, BatchToBatch(TypeA, this, _)) - toBatch(TypeA, BatchToBatch(this, TypeA, _)) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToBatch(this, _)) + toRow(BatchToRow(this, _)) + fromBatch(TypeA, BatchToBatch(TypeA, this, _)) + toBatch(TypeA, BatchToBatch(this, TypeA, _)) + } } - object TypeD extends Convention.BatchType {} + object TypeD extends Convention.BatchType { + override protected[this] def registerTransitions(): Unit = {} + } case class RowToBatch(toBatchType: Convention.BatchType, override val child: SparkPlan) extends RowToColumnarTransition From 57085144ee2ca6228dffdf15a1407e02c23ab41a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 20 Nov 2024 14:13:23 +0800 Subject: [PATCH 124/211] [GLUTEN-7979][CH] Fix exception cause by one child of UnionExec outputs Array(Nothing) while the other outputs Array(String) (#7980) * fix bugs * add uts * fix failed uts * fix style * fix failed uts --- ...enClickHouseTPCHSaltNullParquetSuite.scala | 50 +++++++++++++++++++ cpp-ch/local-engine/Common/CHUtil.cpp | 16 +----- .../local-engine/Common/GlutenStringUtils.h | 3 -- .../PartitionColumnFillingTransform.cpp | 7 +-- .../local-engine/Parser/FunctionExecutor.cpp | 4 -- .../Parser/SerializedPlanParser.cpp | 50 +++++++++---------- .../local-engine/Storages/IO/NativeReader.cpp | 5 ++ .../local-engine/Storages/IO/NativeWriter.cpp | 1 + .../Storages/SubstraitSource/FormatFile.cpp | 13 +++-- .../Storages/SubstraitSource/FormatFile.h | 8 ++- .../SubstraitSource/SubstraitFileSource.cpp | 35 ++++++------- .../clickhouse/ClickHouseTestSettings.scala | 3 ++ .../clickhouse/ClickHouseTestSettings.scala | 3 ++ .../clickhouse/ClickHouseTestSettings.scala | 3 ++ .../clickhouse/ClickHouseTestSettings.scala | 3 ++ 15 files changed, 129 insertions(+), 75 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 865d3fa40c0f..5d7bcf324ae8 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -2458,6 +2458,56 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr runQueryAndCompare(sql, noFallBack = true)({ _ => }) } + test("GLUTEN-7979: fix different output schema array and array before union") { + val sql = + """ + |select + | a.multi_peer_user_id, + | max(a.user_id) as max_user_id, + | max(a.peer_user_id) as max_peer_user_id, + | max(a.is_match_line) as max_is_match_line + |from + |( + | select + | t1.user_id, + | t1.peer_user_id, + | t1.is_match_line, + | t1.pk_type, + | t1.pk_start_time, + | t1.pk_end_time, + | t1.multi_peer_user_id + | from + | ( + | select + | id as user_id, + | id as peer_user_id, + | id % 2 as is_match_line, + | id % 3 as pk_type, + | id as pk_start_time, + | id as pk_end_time, + | array() as multi_peer_user_id + | from range(10) + | + | union all + | + | select + | id as user_id, + | id as peer_user_id, + | id % 2 as is_match_line, + | id % 3 as pk_type, + | id as pk_start_time, + | id as pk_end_time, + | array('a', 'b', 'c') as multi_peer_user_id + | from range(10) + | ) t1 + | where t1.user_id > 0 and t1.peer_user_id > 0 + |) a + |group by + | a.multi_peer_user_id + |""".stripMargin + runQueryAndCompare(sql, noFallBack = true)({ _ => }) + } + test("GLUTEN-4190: crush on flattening a const null column") { val sql = """ diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 2413fae9e350..6e907266ffe9 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -492,21 +492,7 @@ std::optional NestedColumnExtractHelper::extractColum const DB::ColumnWithTypeAndName * NestedColumnExtractHelper::findColumn(const DB::Block & in_block, const std::string & name) const { - if (case_insentive) - { - std::string final_name = name; - boost::to_lower(final_name); - const auto & cols = in_block.getColumnsWithTypeAndName(); - auto found = std::find_if(cols.begin(), cols.end(), [&](const auto & column) { return boost::iequals(column.name, name); }); - if (found == cols.end()) - return nullptr; - return &*found; - } - - const auto * col = in_block.findByName(name); - if (col) - return col; - return nullptr; + return in_block.findByName(name, case_insentive); } const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( diff --git a/cpp-ch/local-engine/Common/GlutenStringUtils.h b/cpp-ch/local-engine/Common/GlutenStringUtils.h index 0d980f228f18..0de185d83cc6 100644 --- a/cpp-ch/local-engine/Common/GlutenStringUtils.h +++ b/cpp-ch/local-engine/Common/GlutenStringUtils.h @@ -21,9 +21,6 @@ namespace local_engine { -using PartitionValue = std::pair; -using PartitionValues = std::vector; - class GlutenStringUtils { public: diff --git a/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp b/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp index 724a02f5b296..ab87e1b39b79 100644 --- a/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp +++ b/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp @@ -57,16 +57,11 @@ ColumnPtr createFloatPartitionColumn(DataTypePtr column_type, const std::string return column_type->createColumnConst(1, value); } -//template <> -//ColumnPtr createFloatPartitionColumn(DataTypePtr column_type, std::string partition_value); -//template <> -//ColumnPtr createFloatPartitionColumn(DataTypePtr column_type, std::string partition_value); - PartitionColumnFillingTransform::PartitionColumnFillingTransform( const DB::Block & input_, const DB::Block & output_, const String & partition_col_name_, const String & partition_col_value_) : ISimpleTransform(input_, output_, true), partition_col_name(partition_col_name_), partition_col_value(partition_col_value_) { - partition_col_type = output_.getByName(partition_col_name_).type; + partition_col_type = output_.getByName(partition_col_name_, true).type; partition_column = createPartitionColumn(); } diff --git a/cpp-ch/local-engine/Parser/FunctionExecutor.cpp b/cpp-ch/local-engine/Parser/FunctionExecutor.cpp index 370dc4bda9ac..2cef922ba475 100644 --- a/cpp-ch/local-engine/Parser/FunctionExecutor.cpp +++ b/cpp-ch/local-engine/Parser/FunctionExecutor.cpp @@ -79,8 +79,6 @@ void FunctionExecutor::parseExpression() /// Notice keep_result must be true, because result_node of current function must be output node in actions_dag const auto * node = expression_parser->parseFunction(expression.scalar_function(), actions_dag, true); result_name = node->result_name; - // std::cout << "actions_dag:" << std::endl; - // std::cout << actions_dag->dumpDAG() << std::endl; expression_actions = std::make_unique(std::move(actions_dag)); } @@ -115,9 +113,7 @@ bool FunctionExecutor::executeAndCompare(const std::vectorgetCurrentHeader(); - const auto & original_cols = original_header.getColumnsWithTypeAndName(); - if (static_cast(output_schema.types_size()) != original_cols.size()) + auto origin_header = query_plan->getCurrentHeader(); + const auto & origin_columns = origin_header.getColumnsWithTypeAndName(); + + if (static_cast(output_schema.types_size()) != origin_columns.size()) { debug::dumpPlan(*query_plan, "clickhouse plan", true); debug::dumpMessage(plan, "substrait::Plan", true); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan output schema size {}, subtrait plan name size {}.", - original_cols.size(), + origin_columns.size(), output_schema.types_size(), root_rel.root().names_size()); } + bool need_final_project = false; - ColumnsWithTypeAndName final_cols; + ColumnsWithTypeAndName final_columns; for (int i = 0; i < output_schema.types_size(); ++i) { - const auto & col = original_cols[i]; - auto type = TypeParser::parseType(output_schema.types(i)); - // At present, we only check nullable mismatch. - // intermediate aggregate data is special, no check here. - if (type->isNullable() != col.type->isNullable() && !typeid_cast(col.type.get())) - { - if (type->isNullable()) - { - auto wrapped = wrapNullableType(true, col.type); - final_cols.emplace_back(type->createColumn(), wrapped, col.name); - need_final_project = !wrapped->equals(*col.type); - } - else - { - final_cols.emplace_back(type->createColumn(), removeNullable(col.type), col.name); - need_final_project = true; - } - } + const auto & origin_column = origin_columns[i]; + const auto & origin_type = origin_column.type; + auto final_type = TypeParser::parseType(output_schema.types(i)); + + /// Intermediate aggregate data is special, no check here. + if (typeid_cast(origin_column.type.get()) || origin_type->equals(*final_type)) + final_columns.push_back(origin_column); else { - final_cols.push_back(col); + need_final_project = true; + + bool need_const = origin_column.column && isColumnConst(*origin_column.column); + ColumnWithTypeAndName final_column( + need_const ? final_type->createColumnConst(0, assert_cast(*origin_column.column).getField()) + : final_type->createColumn(), + final_type, + origin_column.name); + final_columns.emplace_back(std::move(final_column)); } } + if (need_final_project) { - ActionsDAG final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); + ActionsDAG final_project = ActionsDAG::makeConvertingActions(origin_columns, final_columns, ActionsDAG::MatchColumnsMode::Position); QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentHeader(), std::move(final_project)); final_project_step->setStepDescription("Project for output schema"); diff --git a/cpp-ch/local-engine/Storages/IO/NativeReader.cpp b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp index 48e6950e27eb..65c2ad8af9e4 100644 --- a/cpp-ch/local-engine/Storages/IO/NativeReader.cpp +++ b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp @@ -54,6 +54,7 @@ DB::Block NativeReader::read() DB::Block result_block; if (istr.eof()) return result_block; + if (columns_parse_util.empty()) { result_block = prepareByFirstBlock(); @@ -154,6 +155,7 @@ DB::Block NativeReader::prepareByFirstBlock() { if (istr.eof()) return {}; + const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); DB::Block result_block; @@ -246,10 +248,12 @@ bool NativeReader::appendNextBlock(DB::Block & result_block) { if (istr.eof()) return false; + size_t columns = 0; size_t rows = 0; readVarUInt(columns, istr); readVarUInt(rows, istr); + for (size_t i = 0; i < columns; ++i) { // Not actually read type name. @@ -259,6 +263,7 @@ bool NativeReader::appendNextBlock(DB::Block & result_block) if (!rows) [[unlikely]] continue; + auto & column_parse_util = columns_parse_util[i]; auto & column = result_block.getByPosition(i); column_parse_util.parse(istr, column.column, rows, column_parse_util); diff --git a/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp index 7f09721abbb9..c19005cd13dd 100644 --- a/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp +++ b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp @@ -32,6 +32,7 @@ namespace local_engine { const String NativeWriter::AGG_STATE_SUFFIX= "#optagg"; + void NativeWriter::flush() { ostr.next(); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp index fc5acc533d59..3c68c70e666d 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp @@ -19,10 +19,12 @@ #include #include #include +#include #include #include #include + #if USE_PARQUET #include #endif @@ -56,13 +58,18 @@ FormatFile::FormatFile( for (size_t i = 0; i < file_info.partition_columns_size(); ++i) { const auto & partition_column = file_info.partition_columns(i); + std::string unescaped_key; std::string unescaped_value; Poco::URI::decode(partition_column.key(), unescaped_key); Poco::URI::decode(partition_column.value(), unescaped_value); - auto key = std::move(unescaped_key); - partition_keys.push_back(key); - partition_values[key] = std::move(unescaped_value); + + partition_keys.push_back(unescaped_key); + partition_values[unescaped_key] = unescaped_value; + + std::string normalized_key = unescaped_key; + boost::to_lower(normalized_key); + normalized_partition_values[normalized_key] = unescaped_value; } } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h index 14b53f68429c..f93abd916b52 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h @@ -65,9 +65,11 @@ class FormatFile virtual std::optional getTotalRows() { return {}; } /// Get partition keys from file path - inline const std::vector & getFilePartitionKeys() const { return partition_keys; } + const std::vector & getFilePartitionKeys() const { return partition_keys; } - inline const std::map & getFilePartitionValues() const { return partition_values; } + const std::map & getFilePartitionValues() const { return partition_values; } + + const std::map & getFileNormalizedPartitionValues() const { return normalized_partition_values; } virtual String getURIPath() const { return file_info.uri_file(); } @@ -81,6 +83,8 @@ class FormatFile ReadBufferBuilderPtr read_buffer_builder; std::vector partition_keys; std::map partition_values; + /// partition keys are normalized to lower cases for partition column case-insensitive matching + std::map normalized_partition_values; std::shared_ptr key_condition; }; using FormatFilePtr = std::shared_ptr; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp index ffe1d18ae785..ef350472b64c 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -87,8 +88,8 @@ SubstraitFileSource::SubstraitFileSource( /// File partition keys are read from the file path const auto partition_keys = files[0]->getFilePartitionKeys(); for (const auto & key : partition_keys) - if (to_read_header.findByName(key)) - to_read_header.erase(key); + if (const auto * col = to_read_header.findByName(key, true)) + to_read_header.erase(col->name); } } @@ -337,14 +338,15 @@ bool ConstColumnsFileReader::pull(DB::Chunk & chunk) if (const size_t col_num = header.columns()) { res_columns.reserve(col_num); - const auto & partition_values = file->getFilePartitionValues(); + const auto & normalized_partition_values = file->getFileNormalizedPartitionValues(); for (size_t pos = 0; pos < col_num; ++pos) { - auto col_with_name_and_type = header.getByPosition(pos); - auto type = col_with_name_and_type.type; - const auto & name = col_with_name_and_type.name; - auto it = partition_values.find(name); - if (it == partition_values.end()) + const auto & column = header.getByPosition(pos); + const auto & type = column.type; + const auto & name = column.name; + + auto it = normalized_partition_values.find(boost::to_lower_copy(name)); + if (it == normalized_partition_values.end()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknow partition column : {}", name); res_columns.emplace_back(createColumn(it->second, type, to_read_rows)); @@ -377,13 +379,13 @@ bool NormalFileReader::pull(DB::Chunk & chunk) if (!rows) return false; - const auto read_columns = raw_chunk.detachColumns(); - auto columns_with_name_and_type = output_header.getColumnsWithTypeAndName(); - auto partition_values = file->getFilePartitionValues(); + auto read_columns = raw_chunk.detachColumns(); + const auto & columns = output_header.getColumnsWithTypeAndName(); + const auto & normalized_partition_values = file->getFileNormalizedPartitionValues(); DB::Columns res_columns; - res_columns.reserve(columns_with_name_and_type.size()); - for (auto & column : columns_with_name_and_type) + res_columns.reserve(columns.size()); + for (auto & column : columns) { if (to_read_header.has(column.name)) { @@ -392,12 +394,11 @@ bool NormalFileReader::pull(DB::Chunk & chunk) } else { - auto it = partition_values.find(column.name); - if (it == partition_values.end()) - { + auto it = normalized_partition_values.find(boost::to_lower_copy(column.name)); + if (it == normalized_partition_values.end()) throw DB::Exception( DB::ErrorCodes::LOGICAL_ERROR, "Not found column({}) from file({}) partition keys.", column.name, file->getURIPath()); - } + res_columns.push_back(createColumn(it->second, column.type, rows)); } } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 50110f15d457..2eb5bd11ffbe 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -203,6 +203,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude( "SPARK-32376: Make unionByName null-filling behavior work with struct columns - deep expr") .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple .exclude("except all") .exclude("exceptAll - nullability") .exclude("intersectAll") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 9b3b090e326d..a7bf5d4da903 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -225,6 +225,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple .exclude("except all") .exclude("exceptAll - nullability") .exclude("intersectAll") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index e91f1495fbe9..fe96e0abde78 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -227,6 +227,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple enableSuite[GlutenDataFrameStatSuite] enableSuite[GlutenDataFrameSuite] .exclude("Uuid expressions should produce same results at retries in the same DataFrame") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index f0637839a762..cf5122fb66f9 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -227,6 +227,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple enableSuite[GlutenDataFrameStatSuite] enableSuite[GlutenDataFrameSuite] .exclude("Uuid expressions should produce same results at retries in the same DataFrame") From 6058d36e85563e7f17a24a930778be5157ab52ca Mon Sep 17 00:00:00 2001 From: Zhichao Zhang Date: Wed, 20 Nov 2024 15:05:48 +0800 Subject: [PATCH 125/211] [GLUTEN-7971][CH] Support using left side as the build table for the left anti/semi join (#7981) * [GLUTEN-7971][CH] Support using left side as the build table for the left anti/semi join Now Vanilla Spark does not support the right anti/semi join, but CH backend does. According to the runtime statistics, it can convert the A left anti/semi join B to B right anti/semi join A when AQE is on and the side ot A table is the smaller than B table. Close #7971. --- .../backendsapi/clickhouse/CHBackend.scala | 11 +++ .../execution/CHHashJoinExecTransformer.scala | 17 ++-- ...GlutenClickHouseTPCDSParquetAQESuite.scala | 3 - ...eTPCDSParquetColumnarShuffleAQESuite.scala | 79 ++++++++++++++++++- ...ouseTPCDSParquetColumnarShuffleSuite.scala | 3 - .../GlutenClickHouseTPCDSParquetSuite.scala | 3 - .../GlutenClickHouseTPCHParquetAQESuite.scala | 18 +++++ cpp-ch/local-engine/Common/CHUtil.cpp | 4 + .../apache/gluten/execution/JoinUtils.scala | 3 + 9 files changed, 124 insertions(+), 17 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 6e73ff6b29d1..966e95319906 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -137,6 +137,10 @@ object CHBackendSettings extends BackendSettingsApi with Logging { CHConf.prefixOf("delta.metadata.optimize") val GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE: String = "true" + val GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT: String = + CHConf.prefixOf("convert.left.anti_semi.to.right") + val GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT_DEFAULT_VALUE: String = "false" + def affinityMode: String = { SparkEnv.get.conf .get( @@ -377,6 +381,13 @@ object CHBackendSettings extends BackendSettingsApi with Logging { true } else { t match { + case LeftAnti | LeftSemi + if (SQLConf.get + .getConfString( + GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT, + GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT_DEFAULT_VALUE) + .toBoolean) => + true case LeftOuter => true case _ => false } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala index 43f19c30e28f..6bf2248ebe3a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala @@ -58,16 +58,23 @@ object JoinTypeTransform { } else { JoinRel.JoinType.JOIN_TYPE_RIGHT } - case LeftSemi | ExistenceJoin(_) => + case LeftSemi => if (!buildRight) { - throw new IllegalArgumentException("LeftSemi join should not switch children") + JoinRel.JoinType.JOIN_TYPE_RIGHT_SEMI + } else { + JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI } - JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI case LeftAnti => if (!buildRight) { - throw new IllegalArgumentException("LeftAnti join should not switch children") + JoinRel.JoinType.JOIN_TYPE_RIGHT_ANTI + } else { + JoinRel.JoinType.JOIN_TYPE_LEFT_ANTI + } + case ExistenceJoin(_) => + if (!buildRight) { + throw new IllegalArgumentException("Existence join should not switch children") } - JoinRel.JoinType.JOIN_TYPE_LEFT_ANTI + JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI case _ => // TODO: Support cross join with Cross Rel JoinRel.JoinType.UNRECOGNIZED diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala index 389d617f10eb..52a72c44941f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala @@ -35,9 +35,6 @@ class GlutenClickHouseTPCDSParquetAQESuite .set("spark.io.compression.codec", "snappy") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.sql.adaptive.enabled", "true") .set("spark.memory.offHeap.size", "4g") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala index 3e965c67ea9a..e8fea04f2c6f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala @@ -20,6 +20,8 @@ import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression +import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.execution.{ColumnarSubqueryBroadcastExec, ReusedSubqueryExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec @@ -35,9 +37,6 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.sql.adaptive.enabled", "true") .set("spark.memory.offHeap.size", "4g") } @@ -265,4 +264,78 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite } }) } + + test("GLUTEN-7971: Support using left side as the build table for the left anti/semi join") { + withSQLConf( + ("spark.sql.autoBroadcastJoinThreshold", "-1"), + ("spark.gluten.sql.columnar.backend.ch.convert.left.anti_semi.to.right", "true")) { + val sql1 = + s""" + |select + | cd_gender, + | cd_marital_status, + | cd_education_status, + | count(*) cnt1 + | from + | customer c,customer_address ca,customer_demographics + | where + | c.c_current_addr_sk = ca.ca_address_sk and + | ca_county in ('Walker County','Richland County','Gaines County','Douglas County') + | and cd_demo_sk = c.c_current_cdemo_sk and + | exists (select * + | from store_sales + | where c.c_customer_sk = ss_customer_sk) + | group by cd_gender, + | cd_marital_status, + | cd_education_status + | order by cd_gender, + | cd_marital_status, + | cd_education_status + | LIMIT 100 ; + |""".stripMargin + runQueryAndCompare(sql1)( + df => { + assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffledHashJoinExecs = collect(df.queryExecution.executedPlan) { + case h: CHShuffledHashJoinExecTransformer if h.joinType == LeftSemi => h + } + assertResult(1)(shuffledHashJoinExecs.size) + assertResult(BuildLeft)(shuffledHashJoinExecs(0).buildSide) + }) + + val sql2 = + s""" + |select + | cd_gender, + | cd_marital_status, + | cd_education_status, + | count(*) cnt1 + | from + | customer c,customer_address ca,customer_demographics + | where + | c.c_current_addr_sk = ca.ca_address_sk and + | ca_county in ('Walker County','Richland County','Gaines County','Douglas County') + | and cd_demo_sk = c.c_current_cdemo_sk and + | not exists (select * + | from store_sales + | where c.c_customer_sk = ss_customer_sk) + | group by cd_gender, + | cd_marital_status, + | cd_education_status + | order by cd_gender, + | cd_marital_status, + | cd_education_status + | LIMIT 100 ; + |""".stripMargin + runQueryAndCompare(sql2)( + df => { + assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffledHashJoinExecs = collect(df.queryExecution.executedPlan) { + case h: CHShuffledHashJoinExecTransformer if h.joinType == LeftAnti => h + } + assertResult(1)(shuffledHashJoinExecs.size) + assertResult(BuildLeft)(shuffledHashJoinExecs(0).buildSide) + }) + } + } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala index 4675de249c6d..24d8e9db607d 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala @@ -32,9 +32,6 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.memory.offHeap.size", "4g") // .set("spark.sql.planChangeLog.level", "error") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala index 08f4522d9ce4..aa7c6b0f560c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala @@ -35,9 +35,6 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui .set("spark.io.compression.codec", "snappy") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.memory.offHeap.size", "4g") .set("spark.gluten.sql.validation.logLevel", "ERROR") .set("spark.gluten.sql.validation.printStackOnFailure", "true") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala index af72ba84a657..1c627140b694 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala @@ -20,6 +20,7 @@ import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.plans.LeftSemi import org.apache.spark.sql.execution.{ReusedSubqueryExec, SubqueryExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper} @@ -213,6 +214,23 @@ class GlutenClickHouseTPCHParquetAQESuite runTPCHQuery(21) { df => } } + test( + "TPCH Q21 with GLUTEN-7971: Support using left side as the build table for the left anti/semi join") { + withSQLConf( + ("spark.sql.autoBroadcastJoinThreshold", "-1"), + ("spark.gluten.sql.columnar.backend.ch.convert.left.anti_semi.to.right", "true")) { + runTPCHQuery(21, compareResult = false) { + df => + assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffledHashJoinExecs = collect(df.queryExecution.executedPlan) { + case h: CHShuffledHashJoinExecTransformer if h.joinType == LeftSemi => h + } + assertResult(1)(shuffledHashJoinExecs.size) + assertResult(BuildLeft)(shuffledHashJoinExecs(0).buildSide) + } + } + } + test("TPCH Q22") { runTPCHQuery(22) { df => diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 6e907266ffe9..03df93c851e3 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -1081,8 +1081,12 @@ JoinUtil::getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type, bool i return {DB::JoinKind::Left, DB::JoinStrictness::Any}; return {DB::JoinKind::Left, DB::JoinStrictness::Semi}; } + case substrait::JoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI: + return {DB::JoinKind::Right, DB::JoinStrictness::Semi}; case substrait::JoinRel_JoinType_JOIN_TYPE_LEFT_ANTI: return {DB::JoinKind::Left, DB::JoinStrictness::Anti}; + case substrait::JoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI: + return {DB::JoinKind::Right, DB::JoinStrictness::Anti}; case substrait::JoinRel_JoinType_JOIN_TYPE_LEFT: return {DB::JoinKind::Left, DB::JoinStrictness::All}; case substrait::JoinRel_JoinType_JOIN_TYPE_RIGHT: diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala index dae5d51af81a..303c9e818f56 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala @@ -265,6 +265,9 @@ object JoinUtils { case _: ExistenceJoin => inputBuildOutput.indices.map(ExpressionBuilder.makeSelection(_)) :+ ExpressionBuilder.makeSelection(buildOutput.size) + case LeftSemi | LeftAnti => + // When the left semi/anti join support the BuildLeft + leftOutput.indices.map(idx => ExpressionBuilder.makeSelection(idx + streamedOutput.size)) case LeftExistence(_) => leftOutput.indices.map(ExpressionBuilder.makeSelection(_)) case _ => From ccc523a8ebd886e890427fc1f855c2320d40259f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Wed, 20 Nov 2024 21:23:38 +0800 Subject: [PATCH 126/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_20) (#7997) Upstream Velox's New Commits: f5c7e4ace by Abdullah Ozturk, misc: Surface clear parsing error for invalid prestopage header (11552) af6609d5d by Jimmy Lu, fix: Unaligned memory access in ByteStream and PrestoSerializer (11574) 03deeaf88 by Zac Wen, feat: Use Velox fs for ssd cache evictlog file (11579) 7d0b84e89 by Deepak Majeti, refactor: ABFS implementation (11419) ce679246b by Guilherme Kunigami, feat: Add opaque type support to PrestoSerializer (11256) 463ebad27 by Yenda Li, refactor: Make compare return int32_t instead of int64_t (11544) e598534cb by Kevin Wilfong, docs: Fix invalid PR title in examples of valid PR titles (11577) --- cpp/velox/compute/VeloxBackend.cc | 2 +- cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h | 2 +- ep/build-velox/src/get_velox.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index 0d5e1d41a54d..62cc36163d33 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -145,7 +145,7 @@ void VeloxBackend::init(const std::unordered_map& conf velox::filesystems::registerGcsFileSystem(); #endif #ifdef ENABLE_ABFS - velox::filesystems::abfs::registerAbfsFileSystem(); + velox::filesystems::registerAbfsFileSystem(); #endif initJolFilesystem(); diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h index 63658cc54ef1..8a417e1c22d2 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h @@ -46,7 +46,7 @@ class VeloxParquetDataSourceABFS final : public VeloxParquetDataSource { auto hiveConf = getHiveConfig(std::make_shared( std::unordered_map(sparkConfs))); auto fileSystem = filesystems::getFileSystem(filePath_, hiveConf); - auto* abfsFileSystem = dynamic_cast(fileSystem.get()); + auto* abfsFileSystem = dynamic_cast(fileSystem.get()); sink_ = std::make_unique( abfsFileSystem->openFileForWrite(filePath_, {{}, sinkPool_.get()}), filePath_); } diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 74d105100bfb..53615ddccfbe 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_19 +VELOX_BRANCH=2024_11_20 VELOX_HOME="" OS=`uname -s` From 16f68a2203e70075554e83adf15aec7676ad1d1f Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Thu, 21 Nov 2024 09:32:35 +0800 Subject: [PATCH 127/211] [GLUTEN-7267][CORE][CH] Move schema pruning optimization of HiveTableScan to an individual post-transform rule (#8008) --- .../backendsapi/clickhouse/CHBackend.scala | 4 --- .../backendsapi/clickhouse/CHRuleApi.scala | 1 + .../backendsapi/BackendSettingsApi.scala | 2 -- .../PruneNestedColumnsInHiveTableScan.scala | 33 +++++++++++++++++++ .../offload/OffloadSingleNodeRules.scala | 11 ++----- .../HiveTableScanNestedColumnPruning.scala | 12 +++---- 6 files changed, 43 insertions(+), 20 deletions(-) create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PruneNestedColumnsInHiveTableScan.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 966e95319906..98d1eabe57dd 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -407,8 +407,4 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } override def supportWindowGroupLimitExec(rankLikeFunction: Expression): Boolean = true - - override def supportHiveTableScanNestedColumnPruning: Boolean = - GlutenConfig.getConf.enableColumnarHiveTableScanNestedColumnPruning - } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 3525fc768dfd..081e815588fa 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -98,6 +98,7 @@ object CHRuleApi { c => intercept(HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads))) // Legacy: Post-transform rules. + injector.injectPostTransform(_ => PruneNestedColumnsInHiveTableScan) injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectPostTransform(c => intercept(RewriteTransformer.apply(c.session))) injector.injectPostTransform(_ => PushDownFilterToScan) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 700571fd2815..177d19c0c709 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -131,6 +131,4 @@ trait BackendSettingsApi { def supportColumnarArrowUdf(): Boolean = false def needPreComputeRangeFrameBoundary(): Boolean = false - - def supportHiveTableScanNestedColumnPruning(): Boolean = false } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PruneNestedColumnsInHiveTableScan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PruneNestedColumnsInHiveTableScan.scala new file mode 100644 index 000000000000..b8972b6d0e2c --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PruneNestedColumnsInHiveTableScan.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar + +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.hive.HiveTableScanNestedColumnPruning + +// Since https://github.com/apache/incubator-gluten/pull/7268. +// Used only by CH backend as of now. +object PruneNestedColumnsInHiveTableScan extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformUp { + case p: ProjectExecTransformer + if HiveTableScanNestedColumnPruning.supportNestedColumnPruning(p) => + HiveTableScanNestedColumnPruning.apply(p) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala index 64448fc60a51..0fee61acace9 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleEx import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} -import org.apache.spark.sql.hive.{HiveTableScanExecTransformer, HiveTableScanNestedColumnPruning} +import org.apache.spark.sql.hive.HiveTableScanExecTransformer // Exchange transformation. case class OffloadExchange() extends OffloadSingleNode with LogLevelUtil { @@ -188,7 +188,7 @@ object OffloadOthers { // Utility to replace single node within transformed Gluten node. // Children will be preserved as they are as children of the output node. // - // Do not look-up on children on the input node in this rule. Otherwise + // Do not look up on children on the input node in this rule. Otherwise // it may break RAS which would group all the possible input nodes to // search for validate candidates. private class ReplaceSingleNode() extends LogLevelUtil with Logging { @@ -215,11 +215,7 @@ object OffloadOthers { case plan: ProjectExec => val columnarChild = plan.child logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - if (HiveTableScanNestedColumnPruning.supportNestedColumnPruning(plan)) { - HiveTableScanNestedColumnPruning.apply(plan) - } else { - ProjectExecTransformer(plan.projectList, columnarChild) - } + ProjectExecTransformer(plan.projectList, columnarChild) case plan: HashAggregateExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") HashAggregateExecBaseTransformer.from(plan) @@ -247,7 +243,6 @@ object OffloadOthers { plan.bucketSpec, plan.options, plan.staticPartitions) - ColumnarWriteFilesExec( writeTransformer, plan.fileFormat, diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala index 7a20e5c37da5..70b08a2ed8bb 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala @@ -16,13 +16,13 @@ */ package org.apache.spark.sql.hive -import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.ProjectExecTransformer import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, SparkPlan} import org.apache.spark.sql.hive.HiveTableScanExecTransformer.{ORC_INPUT_FORMAT_CLASS, PARQUET_INPUT_FORMAT_CLASS, TEXT_INPUT_FORMAT_CLASS} import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} import org.apache.spark.sql.util.SchemaUtils._ @@ -31,9 +31,9 @@ import org.apache.spark.util.Utils object HiveTableScanNestedColumnPruning extends Logging { import org.apache.spark.sql.catalyst.expressions.SchemaPruning._ - def supportNestedColumnPruning(projectExec: ProjectExec): Boolean = { - if (BackendsApiManager.getSettings.supportHiveTableScanNestedColumnPruning()) { - projectExec.child match { + def supportNestedColumnPruning(project: ProjectExecTransformer): Boolean = { + if (GlutenConfig.getConf.enableColumnarHiveTableScanNestedColumnPruning) { + project.child match { case HiveTableScanExecTransformer(_, relation, _, _) => relation.tableMeta.storage.inputFormat match { case Some(inputFormat) @@ -60,7 +60,7 @@ object HiveTableScanNestedColumnPruning extends Logging { def apply(plan: SparkPlan): SparkPlan = { plan match { - case ProjectExec(projectList, child) => + case ProjectExecTransformer(projectList, child) => child match { case h: HiveTableScanExecTransformer => val newPlan = prunePhysicalColumns( From 7b3c2d875d45d85ff1d54f86d6d7db8950f92636 Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Thu, 21 Nov 2024 09:34:26 +0800 Subject: [PATCH 128/211] [GLUTEN-8005][VL] Add MergeTwoPhasesHashBaseAggregate to injectRas list (#8006) --- .../backendsapi/velox/VeloxRuleApi.scala | 1 + .../v1-bhj-ras/spark33/15.txt | 74 ++-- .../v1-bhj-ras/spark34/15.txt | 74 ++-- .../tpch-approved-plan/v1-ras/spark32/13.txt | 178 ++++---- .../tpch-approved-plan/v1-ras/spark32/14.txt | 110 +++-- .../tpch-approved-plan/v1-ras/spark32/17.txt | 170 ++++---- .../tpch-approved-plan/v1-ras/spark32/18.txt | 284 +++++++------ .../tpch-approved-plan/v1-ras/spark32/19.txt | 106 +++-- .../tpch-approved-plan/v1-ras/spark32/20.txt | 390 +++++++++--------- .../tpch-approved-plan/v1-ras/spark32/3.txt | 180 ++++---- .../tpch-approved-plan/v1-ras/spark33/11.txt | 152 ++++--- .../tpch-approved-plan/v1-ras/spark33/13.txt | 178 ++++---- .../tpch-approved-plan/v1-ras/spark33/14.txt | 110 +++-- .../tpch-approved-plan/v1-ras/spark33/15.txt | 74 ++-- .../tpch-approved-plan/v1-ras/spark33/17.txt | 170 ++++---- .../tpch-approved-plan/v1-ras/spark33/18.txt | 284 +++++++------ .../tpch-approved-plan/v1-ras/spark33/19.txt | 106 +++-- .../tpch-approved-plan/v1-ras/spark33/20.txt | 382 +++++++++-------- .../tpch-approved-plan/v1-ras/spark33/3.txt | 180 ++++---- .../tpch-approved-plan/v1-ras/spark34/11.txt | 152 ++++--- .../tpch-approved-plan/v1-ras/spark34/13.txt | 178 ++++---- .../tpch-approved-plan/v1-ras/spark34/14.txt | 110 +++-- .../tpch-approved-plan/v1-ras/spark34/15.txt | 74 ++-- .../tpch-approved-plan/v1-ras/spark34/17.txt | 170 ++++---- .../tpch-approved-plan/v1-ras/spark34/18.txt | 284 +++++++------ .../tpch-approved-plan/v1-ras/spark34/19.txt | 106 +++-- .../tpch-approved-plan/v1-ras/spark34/20.txt | 382 +++++++++-------- .../tpch-approved-plan/v1-ras/spark34/3.txt | 180 ++++---- .../clickhouse/ClickHouseTestSettings.scala | 1 - .../clickhouse/ClickHouseTestSettings.scala | 1 - .../GlutenReplaceHashWithSortAggSuite.scala | 37 +- ...MergeTwoPhasesHashBaseAggregateSuite.scala | 1 - 32 files changed, 2329 insertions(+), 2550 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 67f9d21a5e7f..c5f46dae67dd 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -125,6 +125,7 @@ object VeloxRuleApi { injector.injectPreTransform(_ => RemoveTransitions) injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) + injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate(c.session)) injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session)) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt index 74246ef67225..52a87f43535d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt @@ -237,31 +237,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ ScanTransformer parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) (44) ScanTransformer parquet @@ -326,71 +325,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt index a65fc78a2f03..48b06986ee1b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt @@ -239,31 +239,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ ScanTransformer parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) (44) ScanTransformer parquet @@ -328,71 +327,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt index 91444cca0528..c8ddb5437009 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt @@ -1,55 +1,54 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ ScanTransformer parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) (1) ScanTransformer parquet @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt index 33c0dfb71e1e..40a891738143 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt @@ -1,40 +1,39 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) (1) ScanTransformer parquet @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt index 94e892d1e8c9..30fd4b0b1124 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt @@ -1,62 +1,61 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ ScanTransformer parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) (1) ScanTransformer parquet @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7), true) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt index 550cd076c1c2..dd4cad538e2c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt @@ -1,102 +1,101 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ ScanTransformer parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ ScanTransformer parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ ScanTransformer parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) (1) ScanTransformer parquet @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt index f7778691955a..507b700e37e9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt @@ -1,39 +1,38 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) (1) ScanTransformer parquet @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt index 5dd6bbfccab6..13c10a7bba29 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt @@ -1,20 +1,20 @@ == Physical Plan == -AdaptiveSparkPlan (146) +AdaptiveSparkPlan (145) +- == Final Plan == - VeloxColumnarToRow (96) - +- ^ SortExecTransformer (94) - +- ^ InputIteratorTransformer (93) - +- ShuffleQueryStage (91) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (95) + +- ^ SortExecTransformer (93) + +- ^ InputIteratorTransformer (92) + +- ShuffleQueryStage (90) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) @@ -22,12 +22,12 @@ AdaptiveSparkPlan (146) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) @@ -48,83 +48,82 @@ AdaptiveSparkPlan (146) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ ScanTransformer parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ ScanTransformer parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ ScanTransformer parquet (77) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (145) - +- Exchange (144) - +- Project (143) - +- SortMergeJoin Inner (142) - :- Sort (136) - : +- Exchange (135) - : +- Project (134) - : +- SortMergeJoin LeftSemi (133) - : :- Sort (100) - : : +- Exchange (99) - : : +- Filter (98) - : : +- Scan parquet (97) - : +- Sort (132) - : +- Exchange (131) - : +- Project (130) - : +- SortMergeJoin Inner (129) - : :- Sort (112) - : : +- Exchange (111) - : : +- SortMergeJoin LeftSemi (110) - : : :- Sort (104) - : : : +- Exchange (103) - : : : +- Filter (102) - : : : +- Scan parquet (101) - : : +- Sort (109) - : : +- Exchange (108) - : : +- Project (107) - : : +- Filter (106) - : : +- Scan parquet (105) - : +- Sort (128) - : +- Exchange (127) - : +- Filter (126) - : +- HashAggregate (125) - : +- HashAggregate (124) - : +- SortMergeJoin LeftSemi (123) - : :- Sort (117) - : : +- Exchange (116) - : : +- Project (115) - : : +- Filter (114) - : : +- Scan parquet (113) - : +- Sort (122) - : +- Exchange (121) - : +- Project (120) - : +- Filter (119) - : +- Scan parquet (118) - +- Sort (141) - +- Exchange (140) - +- Project (139) - +- Filter (138) - +- Scan parquet (137) + Sort (144) + +- Exchange (143) + +- Project (142) + +- SortMergeJoin Inner (141) + :- Sort (135) + : +- Exchange (134) + : +- Project (133) + : +- SortMergeJoin LeftSemi (132) + : :- Sort (99) + : : +- Exchange (98) + : : +- Filter (97) + : : +- Scan parquet (96) + : +- Sort (131) + : +- Exchange (130) + : +- Project (129) + : +- SortMergeJoin Inner (128) + : :- Sort (111) + : : +- Exchange (110) + : : +- SortMergeJoin LeftSemi (109) + : : :- Sort (103) + : : : +- Exchange (102) + : : : +- Filter (101) + : : : +- Scan parquet (100) + : : +- Sort (108) + : : +- Exchange (107) + : : +- Project (106) + : : +- Filter (105) + : : +- Scan parquet (104) + : +- Sort (127) + : +- Exchange (126) + : +- Filter (125) + : +- HashAggregate (124) + : +- HashAggregate (123) + : +- SortMergeJoin LeftSemi (122) + : :- Sort (116) + : : +- Exchange (115) + : : +- Project (114) + : : +- Filter (113) + : : +- Scan parquet (112) + : +- Sort (121) + : +- Exchange (120) + : +- Project (119) + : +- Filter (118) + : +- Scan parquet (117) + +- Sort (140) + +- Exchange (139) + +- Project (138) + +- Filter (137) + +- Scan parquet (136) (1) ScanTransformer parquet @@ -327,417 +326,410 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) ScanTransformer parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) InputAdapter +(91) InputAdapter Input [2]: [s_name#X, s_address#X] -(93) InputIteratorTransformer +(92) InputIteratorTransformer Input [2]: [s_name#X, s_address#X] -(94) SortExecTransformer +(93) SortExecTransformer Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(95) WholeStageCodegenTransformer (X) +(94) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(96) VeloxColumnarToRow +(95) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(97) Scan parquet +(96) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(98) Filter +(97) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(99) Exchange +(98) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) Sort +(99) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(101) Scan parquet +(100) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(102) Filter +(101) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(103) Exchange +(102) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(104) Sort +(103) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(105) Scan parquet +(104) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(106) Filter +(105) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(107) Project +(106) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(108) Exchange +(107) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(110) SortMergeJoin +(109) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(111) Exchange +(110) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(112) Sort +(111) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(113) Scan parquet +(112) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(114) Filter +(113) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(115) Project +(114) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(116) Exchange +(115) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(117) Sort +(116) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(118) Scan parquet +(117) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(119) Filter +(118) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(120) Project +(119) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(121) Exchange +(120) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(124) HashAggregate +(123) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(125) HashAggregate +(124) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(126) Filter +(125) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(127) Exchange +(126) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(128) Sort +(127) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(129) SortMergeJoin +(128) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(130) Project +(129) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(131) Exchange +(130) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(132) Sort +(131) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(133) SortMergeJoin +(132) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(134) Project +(133) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(135) Exchange +(134) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(136) Sort +(135) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(137) Scan parquet +(136) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(138) Filter +(137) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(139) Project +(138) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(140) Exchange +(139) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(141) Sort +(140) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(142) SortMergeJoin +(141) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(143) Project +(142) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(144) Exchange +(143) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(145) Sort +(144) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(146) AdaptiveSparkPlan +(145) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt index c4df05de1618..2e427b6bf3ca 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt @@ -1,64 +1,63 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ ScanTransformer parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) (1) ScanTransformer parquet @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt index cc1f4d31b697..55d544b898c6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt @@ -424,55 +424,54 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ ScanTransformer parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) (83) ScanTransformer parquet @@ -581,129 +580,122 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt index bd84f11c6ac5..7c3509c3e726 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt @@ -1,55 +1,54 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ ScanTransformer parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) (1) ScanTransformer parquet @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt index 5fdc418d02ef..1161b32937bc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt @@ -1,40 +1,39 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) (1) ScanTransformer parquet @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt index 2d327719c3bd..398cc1f5b8b9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt @@ -257,31 +257,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ ScanTransformer parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) (48) ScanTransformer parquet @@ -346,71 +345,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt index c5033fa773a5..843ce9b39eb6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt @@ -1,62 +1,61 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ ScanTransformer parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) (1) ScanTransformer parquet @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7)) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt index b07b35495922..1119ddcf80be 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt @@ -1,102 +1,101 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ ScanTransformer parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ ScanTransformer parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ ScanTransformer parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) (1) ScanTransformer parquet @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt index 348cc4ce85d5..31341db339e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt @@ -1,39 +1,38 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) (1) ScanTransformer parquet @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt index 703c4a99d63c..9a2c56040b38 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt @@ -1,19 +1,19 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) @@ -21,12 +21,12 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -47,83 +47,82 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ ScanTransformer parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ ScanTransformer parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ ScanTransformer parquet (77) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) (1) ScanTransformer parquet @@ -326,407 +325,400 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) ScanTransformer parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt index f6dc9c8015ea..d09fe3e19a9b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt @@ -1,64 +1,63 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ ScanTransformer parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) (1) ScanTransformer parquet @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt index 5278039ec385..e2d5e58eba47 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt @@ -428,55 +428,54 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ ScanTransformer parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) (83) ScanTransformer parquet @@ -587,131 +586,124 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join type: Inner Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt index ca06d654b432..59abfb682c42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt @@ -1,55 +1,54 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ ScanTransformer parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) (1) ScanTransformer parquet @@ -134,181 +133,174 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: LeftOuter Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt index c61d21b92ed1..531cc5aaab55 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt @@ -1,40 +1,39 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) (1) ScanTransformer parquet @@ -124,94 +123,87 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt index 3585c0cc2f34..f370146f9206 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt @@ -259,31 +259,30 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ ScanTransformer parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) (48) ScanTransformer parquet @@ -348,71 +347,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt index c6d0e0103363..c1b43b7fc421 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt @@ -1,62 +1,61 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ ScanTransformer parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) (1) ScanTransformer parquet @@ -215,141 +214,134 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [(0.2 * avg(l_quantity#X)#X) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join type: Inner Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt index 50a076b1ef8f..c1a6b181bb34 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt @@ -1,102 +1,101 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ ScanTransformer parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ ScanTransformer parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ ScanTransformer parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) (1) ScanTransformer parquet @@ -375,223 +374,216 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt index fb7e231fa5b3..4db9ca0c8393 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt @@ -1,39 +1,38 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ ScanTransformer parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ ScanTransformer parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) (1) ScanTransformer parquet @@ -123,90 +122,83 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt index 4e427961a4f6..4d8eee1f1fb7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt @@ -1,19 +1,19 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) @@ -21,12 +21,12 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -47,83 +47,82 @@ AdaptiveSparkPlan (143) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ ScanTransformer parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ ScanTransformer parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ ScanTransformer parquet (77) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) (1) ScanTransformer parquet @@ -328,415 +327,408 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) ScanTransformer parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt index 4aa745832681..1f86369a62af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt @@ -1,64 +1,63 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ ScanTransformer parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ ScanTransformer parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ ScanTransformer parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) (1) ScanTransformer parquet @@ -217,143 +216,136 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index fe96e0abde78..b7e3905740fb 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -846,7 +846,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("do not replace hash aggregate if child does not have sort order") .exclude("do not replace hash aggregate if there is no group-by column") .excludeGlutenTest("replace partial hash aggregate with sort aggregate") - .excludeGlutenTest("replace partial and final hash aggregate together with sort aggregate") enableSuite[GlutenReuseExchangeAndSubquerySuite] enableSuite[GlutenSQLAggregateFunctionSuite] enableSuite[GlutenSQLWindowFunctionSuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index cf5122fb66f9..8ce145735dc3 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -846,7 +846,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("do not replace hash aggregate if child does not have sort order") .exclude("do not replace hash aggregate if there is no group-by column") .excludeGlutenTest("replace partial hash aggregate with sort aggregate") - .excludeGlutenTest("replace partial and final hash aggregate together with sort aggregate") enableSuite[GlutenReuseExchangeAndSubquerySuite] enableSuite[GlutenSQLAggregateFunctionSuite] enableSuite[GlutenSQLWindowFunctionSuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala index 29ccd9b71dcd..33bf1a1ec97e 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -81,27 +81,24 @@ class GlutenReplaceHashWithSortAggSuite } testGluten("replace partial and final hash aggregate together with sort aggregate") { - // TODO: Should can merge aggregates even if RAS support. - withSQLConf("spark.gluten.ras.enabled" -> "false") { - withTempView("t1", "t2") { - spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") - spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") - Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { - aggExprInfo => - val query = - s""" - |SELECT key, ${aggExprInfo._1}(key) - |FROM - |( - | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key - | FROM t1 - | JOIN t2 - | ON t1.key = t2.key - |) - |GROUP BY key + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { + aggExprInfo => + val query = + s""" + |SELECT key, ${aggExprInfo._1}(key) + |FROM + |( + | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key """.stripMargin - checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) - } + checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) } } } diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala index f65c71f77cf5..2aaf1be21bf3 100644 --- a/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala @@ -53,7 +53,6 @@ abstract class BaseMergeTwoPhasesHashBaseAggregateSuite extends WholeStageTransf .set("spark.sql.files.maxPartitionBytes", "1g") .set("spark.sql.shuffle.partitions", "1") .set("spark.memory.offHeap.size", "2g") - .set("spark.gluten.ras.enabled", "false") .set("spark.gluten.sql.mergeTwoPhasesAggregate.enabled", "true") if (BackendTestUtils.isCHBackendLoaded()) { conf From 7a564d81536251ff361a3c0a3b9ea5f44ec02a00 Mon Sep 17 00:00:00 2001 From: jackylee Date: Thu, 21 Nov 2024 10:09:59 +0800 Subject: [PATCH 129/211] [VL] Fallback unsupported orc write for spark32 and spark33 (#7996) --- .../backendsapi/velox/VeloxListenerApi.scala | 4 +-- .../velox/VeloxOrcWriterInjects.scala | 32 ------------------- 2 files changed, 2 insertions(+), 34 deletions(-) delete mode 100644 backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index 85de0a888952..d29d3029709e 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -32,7 +32,7 @@ import org.apache.spark.api.plugin.PluginContext import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules -import org.apache.spark.sql.execution.datasources.velox.{VeloxOrcWriterInjects, VeloxParquetWriterInjects, VeloxRowSplitter} +import org.apache.spark.sql.execution.datasources.velox.{VeloxParquetWriterInjects, VeloxRowSplitter} import org.apache.spark.sql.expression.UDFResolver import org.apache.spark.sql.internal.{GlutenConfigUtil, StaticSQLConf} import org.apache.spark.util.{SparkDirectoryUtil, SparkResourceUtil} @@ -166,7 +166,7 @@ class VeloxListenerApi extends ListenerApi with Logging { NativeBackendInitializer.initializeBackend(parsed) // Inject backend-specific implementations to override spark classes. - GlutenFormatFactory.register(new VeloxParquetWriterInjects, new VeloxOrcWriterInjects) + GlutenFormatFactory.register(new VeloxParquetWriterInjects) GlutenFormatFactory.injectPostRuleFactory( session => GlutenWriterColumnarRules.NativeWritePostRule(session)) GlutenFormatFactory.register(new VeloxRowSplitter()) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala deleted file mode 100644 index 0999121dd8bd..000000000000 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.datasources.velox - -import scala.collection.JavaConverters.mapAsJavaMapConverter -import scala.collection.mutable - -class VeloxOrcWriterInjects extends VeloxFormatWriterInjects { - override def nativeConf( - options: Map[String, String], - compressionCodec: String): java.util.Map[String, String] = { - val sparkOptions = new mutable.HashMap[String, String]() - // TODO: implement it - sparkOptions.asJava - } - - override val formatName: String = "orc" -} From 67ba92b90a097eb369819030581d4157e7084e0b Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Thu, 21 Nov 2024 11:02:00 +0800 Subject: [PATCH 130/211] [CORE][CH] Remove API BackendSettingsApi#supportShuffleWithProject (#8009) --- .../backendsapi/clickhouse/CHBackend.scala | 23 ----------- .../clickhouse/CHSparkPlanExecApi.scala | 4 +- .../clickhouse/CHValidatorApi.scala | 39 ++++++++++++++++--- .../RangePartitionerBoundsGenerator.scala | 7 ++-- .../backendsapi/BackendSettingsApi.scala | 8 ---- 5 files changed, 38 insertions(+), 43 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 98d1eabe57dd..061ec9856e80 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -34,9 +34,6 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -255,26 +252,6 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } } - override def supportShuffleWithProject( - outputPartitioning: Partitioning, - child: SparkPlan): Boolean = { - child match { - case hash: HashAggregateExec => - if (hash.aggregateExpressions.isEmpty) { - true - } else { - outputPartitioning match { - case hashPartitioning: HashPartitioning => - hashPartitioning.expressions.exists(x => !x.isInstanceOf[AttributeReference]) - case _ => - false - } - } - case _ => - true - } - } - override def supportSortExec(): Boolean = { GlutenConfig.getConf.enableColumnarSort } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index e413cb8ed470..dde03d4ad087 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -273,9 +273,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { override def genColumnarShuffleExchange(shuffle: ShuffleExchangeExec): SparkPlan = { val child = shuffle.child - if ( - BackendsApiManager.getSettings.supportShuffleWithProject(shuffle.outputPartitioning, child) - ) { + if (CHValidatorApi.supportShuffleWithProject(shuffle.outputPartitioning, child)) { val (projectColumnNumber, newPartitioning, newChild) = addProjectionForShuffleExchange(shuffle) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala index eed493cffe1e..29d26410b9b0 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.{BackendsApiManager, ValidatorApi} +import org.apache.gluten.backendsapi.ValidatorApi import org.apache.gluten.expression.ExpressionConverter import org.apache.gluten.extension.ValidationResult import org.apache.gluten.substrait.SubstraitContext @@ -28,12 +28,14 @@ import org.apache.gluten.vectorized.CHNativeExpressionEvaluator import org.apache.spark.internal.Logging import org.apache.spark.shuffle.utils.RangePartitionerBoundsGenerator -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.aggregate.HashAggregateExec class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logging { + import CHValidatorApi._ override def doNativeValidateWithFailureReason(plan: PlanNode): ValidationResult = { if (CHNativeExpressionEvaluator.doValidate(plan.toProtobuf.toByteArray)) { @@ -86,10 +88,7 @@ class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logg .doTransform(substraitContext.registeredFunction) node.isInstanceOf[SelectionNode] } - if ( - allSelectionNodes || - BackendsApiManager.getSettings.supportShuffleWithProject(outputPartitioning, child) - ) { + if (allSelectionNodes || supportShuffleWithProject(outputPartitioning, child)) { None } else { Some("expressions are not supported in HashPartitioning") @@ -107,3 +106,31 @@ class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logg } } } + +object CHValidatorApi { + + /** + * A shuffle key may be an expression. We would add a projection for this expression shuffle key + * and make it into a new column which the shuffle will refer to. But we need to remove it from + * the result columns from the shuffle. + * + * Since https://github.com/apache/incubator-gluten/pull/1071. + */ + def supportShuffleWithProject(outputPartitioning: Partitioning, child: SparkPlan): Boolean = { + child match { + case hash: HashAggregateExec => + if (hash.aggregateExpressions.isEmpty) { + true + } else { + outputPartitioning match { + case hashPartitioning: HashPartitioning => + hashPartitioning.expressions.exists(x => !x.isInstanceOf[AttributeReference]) + case _ => + false + } + } + case _ => + true + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala index 87c6ae343d4c..706cc5f34108 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.shuffle.utils -import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.backendsapi.clickhouse.CHValidatorApi import org.apache.gluten.execution.SortExecTransformer import org.apache.gluten.expression.ExpressionConverter import org.apache.gluten.substrait.SubstraitContext @@ -261,8 +261,9 @@ object RangePartitionerBoundsGenerator { break } if ( - !ordering.child.isInstanceOf[Attribute] && !BackendsApiManager.getSettings - .supportShuffleWithProject(rangePartitioning, child) + !ordering.child.isInstanceOf[Attribute] && !CHValidatorApi.supportShuffleWithProject( + rangePartitioning, + child) ) { enableRangePartitioning = false break diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 177d19c0c709..506424b79cc3 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -23,8 +23,6 @@ import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.types.StructField @@ -76,12 +74,6 @@ trait BackendSettingsApi { def recreateJoinExecOnFallback(): Boolean = false - /** - * A shuffle key may be an expression. We would add a projection for this expression shuffle key - * and make it into a new column which the shuffle will refer to. But we need to remove it from - * the result columns from the shuffle. - */ - def supportShuffleWithProject(outputPartitioning: Partitioning, child: SparkPlan): Boolean = false def excludeScanExecFromCollapsedStage(): Boolean = false def rescaleDecimalArithmetic: Boolean = false From d5e55446f0c57173d0a3b5004bf25d824ae54de2 Mon Sep 17 00:00:00 2001 From: Joey Date: Thu, 21 Nov 2024 14:30:52 +0800 Subject: [PATCH 131/211] Add compression codec extension to velox written parquet file (#8000) To align with Vanilla's parquet file name. Like gluten-part-b35dab49-3eb9-434b-abb9-7de2bc180a06.snappy.parquet --- .../execution/VeloxParquetWriteSuite.scala | 21 +++++++++- cpp/velox/substrait/SubstraitToVeloxPlan.cc | 38 +++++++++++++++++-- 2 files changed, 54 insertions(+), 5 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala index 4c76c753b90e..d19d279fbb2c 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala @@ -31,6 +31,22 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { override protected val resourcePath: String = "/tpch-data-parquet" override protected val fileFormat: String = "parquet" + // The parquet compression codec extensions + private val parquetCompressionCodecExtensions = Map( + "none" -> "", + "uncompressed" -> "", + "snappy" -> ".snappy", + "gzip" -> ".gz", + "lzo" -> ".lzo", + "lz4" -> ".lz4", + "brotli" -> ".br", + "zstd" -> ".zstd" + ) + + private def getParquetFileExtension(codec: String): String = { + s"${parquetCompressionCodecExtensions(codec)}.parquet" + } + override def beforeAll(): Unit = { super.beforeAll() createTPCHNotNullTables() @@ -49,8 +65,8 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { spark.sql("select array(struct(1), null) as var1").write.mode("overwrite").save(path) } assert( - testAppender.loggingEvents.exists( - _.getMessage.toString.contains("Use Gluten parquet write for hive")) == false) + !testAppender.loggingEvents.exists( + _.getMessage.toString.contains("Use Gluten parquet write for hive"))) } } @@ -77,6 +93,7 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { parquetFiles.forall { file => val path = new Path(f.getCanonicalPath, file) + assert(file.endsWith(getParquetFileExtension(codec))) val in = HadoopInputFile.fromPath(path, spark.sessionState.newHadoopConf()) Utils.tryWithResource(ParquetFileReader.open(in)) { reader => diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index cdd9269e1494..1efa7338796d 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -493,13 +493,43 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: } } +std::string makeUuid() { + return boost::lexical_cast(boost::uuids::random_generator()()); +} + +std::string compressionFileNameSuffix(common::CompressionKind kind) { + switch (static_cast(kind)) { + case common::CompressionKind_ZLIB: + return ".zlib"; + case common::CompressionKind_SNAPPY: + return ".snappy"; + case common::CompressionKind_LZO: + return ".lzo"; + case common::CompressionKind_ZSTD: + return ".zstd"; + case common::CompressionKind_LZ4: + return ".lz4"; + case common::CompressionKind_GZIP: + return ".gz"; + case common::CompressionKind_NONE: + default: + return ""; + } +} + std::shared_ptr makeLocationHandle( const std::string& targetDirectory, + dwio::common::FileFormat fileFormat, + common::CompressionKind compression, const std::optional& writeDirectory = std::nullopt, const connector::hive::LocationHandle::TableType& tableType = connector::hive::LocationHandle::TableType::kExisting) { + std::string targetFileName = ""; + if (fileFormat == dwio::common::FileFormat::PARQUET) { + targetFileName = fmt::format("gluten-part-{}{}{}", makeUuid(), compressionFileNameSuffix(compression), ".parquet"); + } return std::make_shared( - targetDirectory, writeDirectory.value_or(targetDirectory), tableType); + targetDirectory, writeDirectory.value_or(targetDirectory), tableType, targetFileName); } std::shared_ptr makeHiveInsertTableHandle( @@ -615,6 +645,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: // Do not hard-code connector ID and allow for connectors other than Hive. static const std::string kHiveConnectorId = "test-hive"; + // Currently only support parquet format. + dwio::common::FileFormat fileFormat = dwio::common::FileFormat::PARQUET; return std::make_shared( nextPlanNodeId(), @@ -628,8 +660,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: inputType->children(), partitionedKey, nullptr /*bucketProperty*/, - makeLocationHandle(writePath), - dwio::common::FileFormat::PARQUET, // Currently only support parquet format. + makeLocationHandle(writePath, fileFormat, compressionCodec), + fileFormat, compressionCodec)), (!partitionedKey.empty()), exec::TableWriteTraits::outputType(nullptr), From e872cd8b3f0cc26a8ef069b4c4602c0a3af35984 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Thu, 21 Nov 2024 16:59:25 +0800 Subject: [PATCH 132/211] [GLUTEN-7028][CH][Part-9] Collecting Delta stats for parquet (#7993) * [Feature] Implement Spark Dataset.showString * [Bug Fix] We need to run pipeline write in spark35 * [Refacotr] Rework stats collection. 1. Add DeltaStats, we don't collect partition columns stats 2. Add NativeStatCompute * [Feature] Allow collect parquet stats when writing delta parquets * [Refactor] Parameterized SQL * [UT] Verify Delta stats * [Refactor] using GlutenConfig.NATIVE_WRITER_ENABLED.key instead of spark.gluten.sql.native.writer.enabled * [Bug Fix] Only create output file when there is actual output chunks. * [Bug Fix] Minor --- backends-clickhouse/pom.xml | 12 + .../ClickhouseOptimisticTransaction.scala | 272 ++++----- .../execution/FileDeltaColumnarWrite.scala | 131 +++-- .../MergeTreeDeltaColumnarWrite.scala | 108 ++-- .../clickhouse/RuntimeSettings.scala | 6 + .../spark/sql/execution/CHColumnarWrite.scala | 141 +++-- .../execution/CHColumnarWriteFilesExec.scala | 19 +- .../spark/gluten/delta/DeltaStatsUtils.scala | 30 + .../spark/gluten/delta/DeltaStatsUtils.scala | 30 + .../spark/gluten/delta/DeltaStatsUtils.scala | 118 ++++ ...utenClickHouseDeltaParquetWriteSuite.scala | 527 ++++-------------- .../GlutenClickHouseExcelFormatSuite.scala | 9 +- .../GlutenClickHouseTPCHAbstractSuite.scala | 38 ++ .../GlutenClickhouseFunctionSuite.scala | 2 +- ...lutenClickHouseNativeWriteTableSuite.scala | 4 +- .../GlutenClickHouseTableAfterRestart.scala | 32 +- ...ickHouseMergeTreePathBasedWriteSuite.scala | 257 +-------- ...nClickHouseMergeTreeWriteOnHDFSSuite.scala | 136 +---- ...eTreeWriteOnHDFSWithRocksDBMetaSuite.scala | 136 +---- ...tenClickHouseMergeTreeWriteOnS3Suite.scala | 151 +---- .../GlutenClickHouseMergeTreeWriteSuite.scala | 255 +-------- ...rgeTreeWriteTaskNotSerializableSuite.scala | 27 +- .../spark/gluten/NativeWriteChecker.scala | 5 +- .../apache/spark/sql/wrapper/package.scala | 24 + cpp-ch/local-engine/Common/BlockTypeUtils.cpp | 12 +- cpp-ch/local-engine/Common/BlockTypeUtils.h | 8 +- cpp-ch/local-engine/Common/DebugUtils.cpp | 348 +++++++++--- cpp-ch/local-engine/Common/DebugUtils.h | 8 +- .../Parser/RelParsers/WriteRelParser.cpp | 35 +- .../Parser/RelParsers/WriteRelParser.h | 3 +- .../Storages/MergeTree/SparkMergeTreeSink.cpp | 5 +- .../Storages/MergeTree/SparkMergeTreeSink.h | 1 - .../Storages/Output/NormalFileWriter.h | 223 +++++--- .../Storages/Output/ORCOutputFormatFile.h | 5 +- .../Storages/Output/OutputFormatFile.h | 8 +- .../Storages/Output/ParquetOutputFormatFile.h | 1 - .../tests/gtest_write_pipeline.cpp | 12 +- 37 files changed, 1294 insertions(+), 1845 deletions(-) create mode 100644 backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala create mode 100644 backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala create mode 100644 backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala create mode 100644 backends-clickhouse/src/test/scala/org/apache/spark/sql/wrapper/package.scala diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index 0f593a861bd8..c616266bf218 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -426,6 +426,18 @@ + + add-test-sources + generate-test-sources + + add-test-source + + + + src/test/delta-${delta.binary.version} + + + diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 6d62e70d2ebb..05f7fdbfa423 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.delta.files._ import org.apache.spark.sql.delta.hooks.AutoCompact import org.apache.spark.sql.delta.schema.{InnerInvariantViolationException, InvariantViolationException} import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormatWriter, GlutenWriterColumnarRules, WriteFiles, WriteJobStatsTracker} @@ -50,6 +49,9 @@ class ClickhouseOptimisticTransaction( override val snapshot: Snapshot) extends OptimisticTransaction(deltaLog, catalogTable, snapshot) { + private lazy val writingMergeTree = + ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration) + def this( deltaLog: DeltaLog, catalogTable: Option[CatalogTable], @@ -62,119 +64,137 @@ class ClickhouseOptimisticTransaction( } override def writeFiles( + inputData: Dataset[_], + writeOptions: Option[DeltaOptions], + isOptimize: Boolean, + additionalConstraints: Seq[Constraint]): Seq[FileAction] = { + val nativeWrite = GlutenConfig.getConf.enableNativeWriter.getOrElse(false) + if (writingMergeTree) { + if (isOptimize) { + throw new UnsupportedOperationException("Optimize is not supported for ClickHouse") + } + // TODO: update FallbackByBackendSettings for mergetree always return true + val onePipeline = nativeWrite && CHConf.get.enableOnePipelineMergeTreeWrite + if (onePipeline) + pipelineWriteFiles(inputData, writeOptions, isOptimize = false, additionalConstraints) + else + writeMergeTree(inputData, writeOptions, additionalConstraints) + } else { + if (isOptimize || !nativeWrite) { + super.writeFiles(inputData, writeOptions, isOptimize, additionalConstraints) + } else { + pipelineWriteFiles(inputData, writeOptions, isOptimize = false, additionalConstraints) + } + } + } + + @deprecated("Use pipelineWriteFiles instead") + private def writeMergeTree( inputData: Dataset[_], writeOptions: Option[DeltaOptions], additionalConstraints: Seq[Constraint]): Seq[FileAction] = { - // TODO: update FallbackByBackendSettings for mergetree always return true - val onePipeline = GlutenConfig.getConf.enableNativeWriter.getOrElse( - false) && CHConf.get.enableOnePipelineMergeTreeWrite - - if (!onePipeline && ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { - hasWritten = true - - val spark = inputData.sparkSession - val (data, partitionSchema) = performCDCPartition(inputData) - val outputPath = deltaLog.dataPath - - val (queryExecution, output, generatedColumnConstraints, _) = - normalizeData(deltaLog, writeOptions, data) - - val tableV2 = ClickHouseTableV2.getTable(deltaLog) - val committer = - new MergeTreeDelayedCommitProtocol( - outputPath.toString, - None, - None, - tableV2.dataBaseName, - tableV2.tableName) - - // val (optionalStatsTracker, _) = - // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) - val (optionalStatsTracker, _) = (None, None) - - val constraints = - Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints - - SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) { - val queryPlan = queryExecution.executedPlan - val (newQueryPlan, newOutput) = - MergeTreeWriterInjects.insertFakeRowAdaptor(queryPlan, output) - val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, newOutput) - val partitioningColumns = getPartitioningColumns(partitionSchema, newOutput) - - val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() - - if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { - val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker( - new SerializableConfiguration(deltaLog.newDeltaHadoopConf()), - BasicWriteJobStatsTracker.metrics) - // registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) - statsTrackers.append(basicWriteJobStatsTracker) - } + hasWritten = true - // Iceberg spec requires partition columns in data files - val writePartitionColumns = IcebergCompat.isAnyEnabled(metadata) - // Retain only a minimal selection of Spark writer options to avoid any potential - // compatibility issues - var options = (writeOptions match { - case None => Map.empty[String, String] - case Some(writeOptions) => - writeOptions.options.filterKeys { - key => - key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) || - key.equalsIgnoreCase(DeltaOptions.COMPRESSION) - }.toMap - }) + (DeltaOptions.WRITE_PARTITION_COLUMNS -> writePartitionColumns.toString) - - spark.conf.getAll.foreach( - entry => { - if ( - CHConf.startWithSettingsPrefix(entry._1) - || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) - ) { - options += (entry._1 -> entry._2) - } - }) - - try { - val format = tableV2.getFileFormat(protocol, metadata) - GlutenWriterColumnarRules.injectSparkLocalProperty(spark, Some(format.shortName())) - MergeTreeFileFormatWriter.write( - sparkSession = spark, - plan = newQueryPlan, - fileFormat = format, - // formats. - committer = committer, - outputSpec = outputSpec, - // scalastyle:off deltahadoopconfiguration - hadoopConf = spark.sessionState - .newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options), - // scalastyle:on deltahadoopconfiguration - partitionColumns = partitioningColumns, - bucketSpec = - tableV2.normalizedBucketSpec(output.map(_.name), spark.sessionState.conf.resolver), - statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, - options = options, - constraints = constraints - ) - } catch { - case s: SparkException => - // Pull an InvariantViolationException up to the top level if it was the root cause. - val violationException = ExceptionUtils.getRootCause(s) - if (violationException.isInstanceOf[InvariantViolationException]) { - throw violationException - } else { - throw s - } - } finally { - GlutenWriterColumnarRules.injectSparkLocalProperty(spark, None) - } + val spark = inputData.sparkSession + val (data, partitionSchema) = performCDCPartition(inputData) + val outputPath = deltaLog.dataPath + + val (queryExecution, output, generatedColumnConstraints, _) = + normalizeData(deltaLog, writeOptions, data) + + val tableV2 = ClickHouseTableV2.getTable(deltaLog) + val committer = + new MergeTreeDelayedCommitProtocol( + outputPath.toString, + None, + None, + tableV2.dataBaseName, + tableV2.tableName) + + // val (optionalStatsTracker, _) = + // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) + val (optionalStatsTracker, _) = (None, None) + + val constraints = + Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints + + SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) { + val queryPlan = queryExecution.executedPlan + val (newQueryPlan, newOutput) = + MergeTreeWriterInjects.insertFakeRowAdaptor(queryPlan, output) + val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, newOutput) + val partitioningColumns = getPartitioningColumns(partitionSchema, newOutput) + + val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() + + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker( + new SerializableConfiguration(deltaLog.newDeltaHadoopConf()), + BasicWriteJobStatsTracker.metrics) + // registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + statsTrackers.append(basicWriteJobStatsTracker) + } + + // Iceberg spec requires partition columns in data files + val writePartitionColumns = IcebergCompat.isAnyEnabled(metadata) + // Retain only a minimal selection of Spark writer options to avoid any potential + // compatibility issues + var options = (writeOptions match { + case None => Map.empty[String, String] + case Some(writeOptions) => + writeOptions.options.filterKeys { + key => + key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) || + key.equalsIgnoreCase(DeltaOptions.COMPRESSION) + }.toMap + }) + (DeltaOptions.WRITE_PARTITION_COLUMNS -> writePartitionColumns.toString) + + spark.conf.getAll.foreach( + entry => { + if ( + CHConf.startWithSettingsPrefix(entry._1) + || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) + ) { + options += (entry._1 -> entry._2) + } + }) + + try { + val format = tableV2.getFileFormat(protocol, metadata) + GlutenWriterColumnarRules.injectSparkLocalProperty(spark, Some(format.shortName())) + MergeTreeFileFormatWriter.write( + sparkSession = spark, + plan = newQueryPlan, + fileFormat = format, + // formats. + committer = committer, + outputSpec = outputSpec, + // scalastyle:off deltahadoopconfiguration + hadoopConf = spark.sessionState + .newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options), + // scalastyle:on deltahadoopconfiguration + partitionColumns = partitioningColumns, + bucketSpec = + tableV2.normalizedBucketSpec(output.map(_.name), spark.sessionState.conf.resolver), + statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, + options = options, + constraints = constraints + ) + } catch { + case s: SparkException => + // Pull an InvariantViolationException up to the top level if it was the root cause. + val violationException = ExceptionUtils.getRootCause(s) + if (violationException.isInstanceOf[InvariantViolationException]) { + throw violationException + } else { + throw s + } + } finally { + GlutenWriterColumnarRules.injectSparkLocalProperty(spark, None) } - committer.addedStatuses.toSeq ++ committer.changeFiles - } else { - super.writeFiles(inputData, writeOptions, additionalConstraints) } + committer.addedStatuses.toSeq ++ committer.changeFiles } private def shouldOptimizeWrite( @@ -188,16 +208,21 @@ class ClickhouseOptimisticTransaction( override protected def getCommitter(outputPath: Path): DelayedCommitProtocol = new FileDelayedCommitProtocol("delta", outputPath.toString, None, deltaDataSubdir) - override def writeFiles( + private def getCommitter2(outputPath: Path): DelayedCommitProtocol = { + val tableV2 = ClickHouseTableV2.getTable(deltaLog) + new MergeTreeDelayedCommitProtocol2( + outputPath.toString, + None, + deltaDataSubdir, + tableV2.dataBaseName, + tableV2.tableName) + } + + private def pipelineWriteFiles( inputData: Dataset[_], writeOptions: Option[DeltaOptions], isOptimize: Boolean, additionalConstraints: Seq[Constraint]): Seq[FileAction] = { - - if (isOptimize) { - throw new UnsupportedOperationException("Optimize is not supported for ClickHouse") - } - hasWritten = true val spark = inputData.sparkSession @@ -229,24 +254,19 @@ class ClickhouseOptimisticTransaction( WriteFiles(logicalPlan, fileFormat, partitioningColumns, None, options, Map.empty) val queryExecution = new QueryExecution(spark, write) - val committer = fileFormat.toString match { - case "MergeTree" => - val tableV2 = ClickHouseTableV2.getTable(deltaLog) - new MergeTreeDelayedCommitProtocol2( - outputPath.toString, - None, - deltaDataSubdir, - tableV2.dataBaseName, - tableV2.tableName) - case _ => getCommitter(outputPath) + val (committer, collectStats) = fileFormat.toString match { + case "MergeTree" => (getCommitter2(outputPath), false) + case _ => (getCommitter(outputPath), true) } // If Statistics Collection is enabled, then create a stats tracker that will be injected during // the FileFormatWriter.write call below and will collect per-file stats using // StatisticsCollection - // val (optionalStatsTracker, _) = - // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) - val optionalStatsTracker: Option[DeltaJobStatisticsTracker] = None + val (optionalStatsTracker, _) = if (collectStats) { + getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) + } else { + (None, None) + } val constraints = Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala index 709f5682246d..784614152f63 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala @@ -16,19 +16,51 @@ */ package org.apache.spark.sql.execution -import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings import org.apache.gluten.exception.GlutenNotSupportException +import org.apache.gluten.vectorized.NativeExpressionEvaluator import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Projection, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, DeclarativeAggregate} import org.apache.spark.sql.delta.files.{FileDelayedCommitProtocol, MergeTreeDelayedCommitProtocol2} +import org.apache.spark.sql.delta.stats.DeltaFileStatistics import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} -import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.types.StringType import org.apache.spark.util.Utils import scala.collection.mutable.ArrayBuffer +case class DeltaFileCommitInfo(committer: FileDelayedCommitProtocol) + extends (NativeFileWriteResult => Unit) { + val addedFiles: ArrayBuffer[(Map[String, String], String)] = + new ArrayBuffer[(Map[String, String], String)] + override def apply(stat: NativeFileWriteResult): Unit = { + if (stat.partition_id == "__NO_PARTITION_ID__") { + addedFiles.append((Map.empty[String, String], stat.filename)) + } else { + val partitionValues = committer.parsePartitions(stat.partition_id) + addedFiles.append((partitionValues, stat.relativePath)) + } + } + + def result: Seq[(Map[String, String], String)] = addedFiles.toSeq +} + +case class NativeDeltaStats(projection: Projection) extends (InternalRow => Unit) { + protected val results = new collection.mutable.HashMap[String, String] + + override def apply(row: InternalRow): Unit = { + val filename = row.getString(0) + val jsonStats = projection(row).getString(0) + assert(!results.contains(filename), s"Duplicate filename: $filename") + results.put(filename, jsonStats) + } + + def result: DeltaFileStatistics = DeltaFileStatistics(results.toMap) +} case class FileDeltaColumnarWrite( override val jobTrackerID: String, override val description: WriteJobDescription, @@ -36,57 +68,88 @@ case class FileDeltaColumnarWrite( extends CHColumnarWrite[FileDelayedCommitProtocol] with Logging { + private lazy val nativeDeltaStats: Option[NativeDeltaStats] = { + deltaWriteJobStatsTracker + .map( + delta => { + val r = delta.statsColExpr.transform { + case ae: AggregateExpression + if ae.aggregateFunction.isInstanceOf[DeclarativeAggregate] => + ae.aggregateFunction.asInstanceOf[DeclarativeAggregate].evaluateExpression + } + val z = Seq( + AttributeReference("filename", StringType, nullable = false)(), + AttributeReference("partition_id", StringType, nullable = false)()) + val s = + delta.statsColExpr + .collect { + case ae: AggregateExpression + if ae.aggregateFunction.isInstanceOf[DeclarativeAggregate] => + ae.aggregateFunction.asInstanceOf[DeclarativeAggregate] + } + .asInstanceOf[Seq[DeclarativeAggregate]] + .flatMap(_.aggBufferAttributes) + NativeDeltaStats( + UnsafeProjection.create( + exprs = Seq(r), + inputSchema = z :++ s + )) + }) + } override def doSetupNativeTask(): Unit = { assert(description.path == committer.outputPath) val nameSpec = CreateFileNameSpec(taskAttemptContext, description) val writePath = description.path val writeFileName = committer.getFileName(taskAttemptContext, nameSpec.suffix, Map.empty) - logDebug(s"Native staging write path: $writePath and file name: $writeFileName") - BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath, writeFileName) - } - private def doCollectNativeResult( - cb: ColumnarBatch): Option[(Seq[(Map[String, String], String)], ExecutedWriteSummary)] = { - val basicNativeStats = BasicNativeStats(cb) + /** + * CDC files (CDC_PARTITION_COL = true) are named with "cdc-..." instead of "part-...".So, using + * pattern match to replace guid to {}.See the following example: + * {{{ + * part-00000-7d672b28-c079-4b00-bb0a-196c15112918-c000.snappy.parquet + * => + * part-00000-{}.snappy.parquet + * }}} + */ + val guidPattern = + """.*-([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12})(?:-c(\d+)\..*)?$""".r + val fileNamePattern = + guidPattern.replaceAllIn(writeFileName, m => writeFileName.replace(m.group(1), "{}")) + + logDebug(s"Native staging write path: $writePath and with pattern: $fileNamePattern") + val settings = + Map( + RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath, + RuntimeSettings.TASK_WRITE_FILENAME_PATTERN.key -> fileNamePattern + ) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) + } - // TODO: we need close iterator here before processing the result. + private def doCollectNativeResult(stats: Seq[InternalRow]) + : Option[(Seq[(Map[String, String], String)], ExecutedWriteSummary)] = { // Write an empty iterator - if (basicNativeStats.isEmpty) { + if (stats.isEmpty) { None } else { + // stats.map(row => x.apply(row).getString(0)).foreach(println) // process stats - val addedFiles: ArrayBuffer[(Map[String, String], String)] = - new ArrayBuffer[(Map[String, String], String)] - var numWrittenRows: Long = 0 - - basicNativeStats.foreach { - stat => - val absolutePath = s"${description.path}/${stat.relativePath}" - if (stat.partition_id == "__NO_PARTITION_ID__") { - addedFiles.append((Map.empty[String, String], stat.filename)) - } else { - val partitionValues = committer.parsePartitions(stat.partition_id) - addedFiles.append((partitionValues, stat.relativePath)) - basicWriteJobStatsTracker.newPartition( - new GenericInternalRow(Array[Any](stat.partition_id))) - } - basicWriteJobStatsTracker.newFile(absolutePath) - basicWriteJobStatsTracker.closeFile(absolutePath) - numWrittenRows += stat.record_count - } + val commitInfo = DeltaFileCommitInfo(committer) + val basicNativeStat = NativeBasicWriteTaskStatsTracker(description, basicWriteJobStatsTracker) + val basicNativeStats = Seq(commitInfo, basicNativeStat) + NativeStatCompute(stats)(basicNativeStats, nativeDeltaStats) Some( ( - addedFiles.toSeq, + commitInfo.result, ExecutedWriteSummary( updatedPartitions = Set.empty, - stats = Seq(finalStats.copy(numRows = numWrittenRows))))) + stats = nativeDeltaStats.map(_.result).toSeq ++ Seq(basicNativeStat.result)))) } } - override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = { - doCollectNativeResult(batch).map { + override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = { + doCollectNativeResult(writeResults).map { case (addedFiles, summary) => require(addedFiles.nonEmpty, "No files to commit") diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala index fdc6cdfd3077..3ac9d4c305e7 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala @@ -21,21 +21,24 @@ import org.apache.gluten.vectorized.NativeExpressionEvaluator import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.delta.actions.{AddFile, FileAction} import org.apache.spark.sql.delta.files.MergeTreeDelayedCommitProtocol2 import org.apache.spark.sql.delta.stats.DeltaStatistics import org.apache.spark.sql.delta.util.{JsonUtils, MergeTreePartitionUtils} -import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} +import org.apache.spark.sql.execution.datasources.{BasicWriteTaskStats, ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} import org.apache.spark.sql.execution.datasources.mergetree.StorageMeta -import org.apache.spark.sql.execution.utils.CHExecUtil -import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils import org.apache.hadoop.fs.Path import java.util.UUID +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions + /** * {{{ * val schema = @@ -47,7 +50,7 @@ import java.util.UUID * StructField("size_in_bytes", LongType, false) :: Nil) * }}} */ -case class BasicMergeTreeNativeStat( +case class MergeTreeWriteResult( part_name: String, partition_id: String, record_count: Long, @@ -105,25 +108,45 @@ case class BasicMergeTreeNativeStat( } } -object BasicMergeTreeNativeStats { - - /** - * we assume the number of records is less than 10,000.So the memory overhead is acceptable. - * otherwise, we need to access ColumnarBatch row by row, which is not efficient. - */ - def apply(cb: ColumnarBatch): Seq[BasicMergeTreeNativeStat] = - CHExecUtil - .c2r(cb) - .map( - row => - BasicMergeTreeNativeStat( - row.getUTF8String(0).toString, - row.getUTF8String(1).toString, - row.getLong(2), - row.getLong(3), - row.getLong(4) - )) - .toSeq +object MergeTreeWriteResult { + implicit def apply(row: InternalRow): MergeTreeWriteResult = MergeTreeWriteResult( + row.getString(0), + row.getString(1), + row.getLong(2), + row.getLong(3), + row.getLong(4)) +} + +case class MergeTreeCommitInfo(committer: MergeTreeDelayedCommitProtocol2) + extends (MergeTreeWriteResult => Unit) { + private val modificationTime = System.currentTimeMillis() + private val hostName = Seq(Utils.localHostName()) + private val addedFiles: ArrayBuffer[FileAction] = new ArrayBuffer[FileAction] + private val path = new Path(committer.outputPath) + def apply(stat: MergeTreeWriteResult): Unit = { + addedFiles.append( + stat(committer.database, committer.tableName, path, modificationTime, hostName)) + } + def result: Seq[FileAction] = addedFiles.toSeq +} + +case class MergeTreeBasicWriteTaskStatsTracker() extends (MergeTreeWriteResult => Unit) { + private val partitions: mutable.ArrayBuffer[InternalRow] = mutable.ArrayBuffer.empty + private var numRows: Long = 0 + private var numBytes: Long = 0 + private var numFiles: Int = 0 + + def apply(stat: MergeTreeWriteResult): Unit = { + if (stat.partition_id != "__NO_PARTITION_ID__") { + partitions.append(new GenericInternalRow(Array[Any](stat.partition_id))) + } + numFiles += 1 + numRows += stat.record_count + numBytes += stat.size_in_bytes + } + + def result: BasicWriteTaskStats = + BasicWriteTaskStats(partitions.toSeq, numFiles, numBytes, numRows) } case class MergeTreeDeltaColumnarWrite( @@ -146,44 +169,25 @@ case class MergeTreeDeltaColumnarWrite( NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) } - private def doCollectNativeResult(cb: ColumnarBatch): Option[WriteTaskResult] = { - val basicNativeStats = BasicMergeTreeNativeStats(cb) - - // TODO: we need close iterator here before processing the result. - - if (basicNativeStats.isEmpty) { + private def doCollectNativeResult(stats: Seq[InternalRow]): Option[WriteTaskResult] = { + if (stats.isEmpty) { None } else { - val modificationTime = System.currentTimeMillis() - val hostName = Seq(Utils.localHostName()) - val path = new Path(committer.outputPath) - var numRows: Long = 0 - var numBytes: Long = 0 - val numFiles = basicNativeStats.size - val addFiles = basicNativeStats.map { - stat => - if (stat.partition_id != "__NO_PARTITION_ID__") { - basicWriteJobStatsTracker.newPartition( - new GenericInternalRow(Array[Any](stat.partition_id))) - } - numRows += stat.record_count - numBytes += stat.size_in_bytes - stat(committer.database, committer.tableName, path, modificationTime, hostName) - } + val commitInfo = MergeTreeCommitInfo(committer) + val mergeTreeStat = MergeTreeBasicWriteTaskStatsTracker() + val basicNativeStats = Seq(commitInfo, mergeTreeStat) + NativeStatCompute(stats)(basicNativeStats) Some { WriteTaskResult( - new TaskCommitMessage(addFiles), - ExecutedWriteSummary( - updatedPartitions = Set.empty, - stats = - Seq(finalStats.copy(numFiles = numFiles, numBytes = numBytes, numRows = numRows))) + new TaskCommitMessage(commitInfo.result), + ExecutedWriteSummary(updatedPartitions = Set.empty, stats = Seq(mergeTreeStat.result)) ) } } } - override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = { - doCollectNativeResult(batch) + override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = { + doCollectNativeResult(writeResults) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala index 1e37e21ba70d..b59bb32392df 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala @@ -41,6 +41,12 @@ object RuntimeSettings { .stringConf .createWithDefault("") + val TASK_WRITE_FILENAME_PATTERN = + buildConf(runtimeSettings("gluten.task_write_filename_pattern")) + .doc("The pattern to generate file name for writing delta parquet in spark 3.5") + .stringConf + .createWithDefault("") + val PART_NAME_PREFIX = buildConf(runtimeSettings("gluten.part_name_prefix")) .doc("The part name prefix for writing data") diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala index e1496527d0bf..6c7877cc020b 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala @@ -22,10 +22,10 @@ import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec, HadoopMapReduceCommitProtocol} import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, BasicWriteTaskStats, ExecutedWriteSummary, PartitioningUtils, WriteJobDescription, WriteTaskResult, WriteTaskStatsTracker} -import org.apache.spark.sql.execution.utils.CHExecUtil -import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils import org.apache.hadoop.fs.Path @@ -36,6 +36,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import java.lang.reflect.Field import scala.collection.mutable +import scala.language.implicitConversions trait CHColumnarWrite[T <: FileCommitProtocol] { @@ -52,17 +53,17 @@ trait CHColumnarWrite[T <: FileCommitProtocol] { def abortTask(): Unit = { committer.abortTask(taskAttemptContext) } - def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] + def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] lazy val basicWriteJobStatsTracker: WriteTaskStatsTracker = description.statsTrackers .find(_.isInstanceOf[BasicWriteJobStatsTracker]) .map(_.newTaskInstance()) .get - // TODO: task commit time - def finalStats: BasicWriteTaskStats = basicWriteJobStatsTracker - .getFinalStats(0) - .asInstanceOf[BasicWriteTaskStats] + lazy val deltaWriteJobStatsTracker: Option[DeltaJobStatisticsTracker] = + description.statsTrackers + .find(_.isInstanceOf[DeltaJobStatisticsTracker]) + .map(_.asInstanceOf[DeltaJobStatisticsTracker]) lazy val (taskAttemptContext: TaskAttemptContext, jobId: String) = { // Copied from `SparkHadoopWriterUtils.createJobID` to be compatible with multi-version @@ -95,7 +96,7 @@ trait CHColumnarWrite[T <: FileCommitProtocol] { object CreateFileNameSpec { def apply(taskContext: TaskAttemptContext, description: WriteJobDescription): FileNameSpec = { val fileCounter = 0 - val suffix = f".c$fileCounter%03d" + + val suffix = f"-c$fileCounter%03d" + description.outputWriterFactory.getFileExtension(taskContext) FileNameSpec("", suffix) } @@ -149,31 +150,74 @@ case class HadoopMapReduceAdapter(sparkCommitter: HadoopMapReduceCommitProtocol) * StructField("record_count", LongType, false) :: Nil) * }}} */ -case class BasicNativeStat(filename: String, partition_id: String, record_count: Long) { - +case class NativeFileWriteResult(filename: String, partition_id: String, record_count: Long) { lazy val relativePath: String = if (partition_id == "__NO_PARTITION_ID__") { filename } else { s"$partition_id/$filename" } } -object BasicNativeStats { - /** - * we assume the number of records is less than 10,000.So the memory overhead is acceptable. - * otherwise, we need to access ColumnarBatch row by row, which is not efficient. - */ - def apply(cb: ColumnarBatch): Seq[BasicNativeStat] = - CHExecUtil - .c2r(cb) - .map( - row => - BasicNativeStat( - row.getUTF8String(0).toString, - row.getUTF8String(1).toString, - row.getLong(2) - )) - .toSeq +object NativeFileWriteResult { + implicit def apply(row: InternalRow): NativeFileWriteResult = { + NativeFileWriteResult(row.getString(0), row.getString(1), row.getLong(2)) + } +} + +case class NativeStatCompute(rows: Seq[InternalRow]) { + def apply[T](stats: Seq[T => Unit], extra: Option[InternalRow => Unit] = None)(implicit + creator: InternalRow => T): Unit = { + rows.foreach { + row => + val stat = creator(row) + stats.foreach(agg => agg(stat)) + extra.foreach(_(row)) + } + } +} + +case class NativeBasicWriteTaskStatsTracker( + description: WriteJobDescription, + basicWriteJobStatsTracker: WriteTaskStatsTracker) + extends (NativeFileWriteResult => Unit) { + private var numWrittenRows: Long = 0 + override def apply(stat: NativeFileWriteResult): Unit = { + val absolutePath = s"${description.path}/${stat.relativePath}" + if (stat.partition_id != "__NO_PARTITION_ID__") { + basicWriteJobStatsTracker.newPartition(new GenericInternalRow(Array[Any](stat.partition_id))) + } + basicWriteJobStatsTracker.newFile(absolutePath) + basicWriteJobStatsTracker.closeFile(absolutePath) + numWrittenRows += stat.record_count + } + private def finalStats: BasicWriteTaskStats = basicWriteJobStatsTracker + .getFinalStats(0) + .asInstanceOf[BasicWriteTaskStats] + + def result: BasicWriteTaskStats = finalStats.copy(numRows = numWrittenRows) +} + +case class FileCommitInfo(description: WriteJobDescription) + extends (NativeFileWriteResult => Unit) { + private val partitions: mutable.Set[String] = mutable.Set[String]() + private val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]() + + def apply(stat: NativeFileWriteResult): Unit = { + val tmpAbsolutePath = s"${description.path}/${stat.relativePath}" + if (stat.partition_id != "__NO_PARTITION_ID__") { + partitions += stat.partition_id + val customOutputPath = + description.customPartitionLocations.get( + PartitioningUtils.parsePathFragment(stat.partition_id)) + if (customOutputPath.isDefined) { + addedAbsPathFiles(tmpAbsolutePath) = customOutputPath.get + "/" + stat.filename + } + } + } + + def result: (Set[String], Map[String, String]) = { + (partitions.toSet, addedAbsPathFiles.toMap) + } } case class HadoopMapReduceCommitProtocolWrite( @@ -196,50 +240,29 @@ case class HadoopMapReduceCommitProtocolWrite( BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath, writeFileName) } - def doCollectNativeResult(cb: ColumnarBatch): Option[WriteTaskResult] = { - val basicNativeStats = BasicNativeStats(cb) - - // TODO: we need close iterator here before processing the result. + def doCollectNativeResult(stats: Seq[InternalRow]): Option[WriteTaskResult] = { // Write an empty iterator - if (basicNativeStats.isEmpty) { + if (stats.isEmpty) { None } else { - val partitions: mutable.Set[String] = mutable.Set[String]() - val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]() - - var numWrittenRows: Long = 0 - basicNativeStats.foreach { - stat => - val tmpAbsolutePath = s"${description.path}/${stat.relativePath}" - if (stat.partition_id != "__NO_PARTITION_ID__") { - partitions += stat.partition_id - val customOutputPath = - description.customPartitionLocations.get( - PartitioningUtils.parsePathFragment(stat.partition_id)) - if (customOutputPath.isDefined) { - addedAbsPathFiles(tmpAbsolutePath) = customOutputPath.get + "/" + stat.filename - } - basicWriteJobStatsTracker.newPartition( - new GenericInternalRow(Array[Any](stat.partition_id))) - } - basicWriteJobStatsTracker.newFile(tmpAbsolutePath) - basicWriteJobStatsTracker.closeFile(tmpAbsolutePath) - numWrittenRows += stat.record_count - } - - val updatedPartitions = partitions.toSet + val commitInfo = FileCommitInfo(description) + val basicNativeStat = NativeBasicWriteTaskStatsTracker(description, basicWriteJobStatsTracker) + val basicNativeStats = Seq(commitInfo, basicNativeStat) + NativeStatCompute(stats)(basicNativeStats) + val (partitions, addedAbsPathFiles) = commitInfo.result + val updatedPartitions = partitions Some( WriteTaskResult( - new TaskCommitMessage(addedAbsPathFiles.toMap -> updatedPartitions), + new TaskCommitMessage(addedAbsPathFiles -> updatedPartitions), ExecutedWriteSummary( updatedPartitions = updatedPartitions, - stats = Seq(finalStats.copy(numRows = numWrittenRows))) + stats = Seq(basicNativeStat.result)) )) } } - override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = { - doCollectNativeResult(batch).map( + override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = { + doCollectNativeResult(writeResults).map( nativeWriteTaskResult => { val (_, taskCommitTime) = Utils.timeTakenMs { committer.commitTask(taskAttemptContext) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala index ab3f652cda23..503fd1a90caa 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.write.WriterCommitMessage import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.utils.CHExecUtil import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -48,8 +49,9 @@ class CHColumnarWriteFilesRDD( extends RDD[WriterCommitMessage](prev) { private def reportTaskMetrics(writeTaskResult: WriteTaskResult): Unit = { - writeTaskResult.summary.stats.headOption.map(_.asInstanceOf[BasicWriteTaskStats]).foreach { - stats => + writeTaskResult.summary.stats.find(_.isInstanceOf[BasicWriteTaskStats]).foreach { + s => + val stats = s.asInstanceOf[BasicWriteTaskStats] // Reports bytesWritten and recordsWritten to the Spark output metrics. // We should update it after calling `commitTask` to overwrite the metrics. Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach { @@ -94,8 +96,19 @@ class CHColumnarWriteFilesRDD( assert(iter.hasNext) val resultColumnarBatch = iter.next() assert(resultColumnarBatch != null) + + /** + * we assume the number of records is less than 10,000.So the memory overhead is acceptable. + * otherwise, we need to access ColumnarBatch row by row, which is not efficient. + */ + val writeResults = CHExecUtil.c2r(resultColumnarBatch).map(_.copy()).toSeq + // TODO: we need close iterator here before processing the result. + // TODO: task commit time + // TODO: get the schema from result ColumnarBatch and verify it. + assert(!iter.hasNext) + val writeTaskResult = commitProtocol - .commitTask(resultColumnarBatch) + .commitTask(writeResults) .orElse({ // If we are writing an empty iterator, then gluten backend would do nothing. // Here we fallback to use vanilla Spark write files to generate an empty file for diff --git a/backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala b/backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala new file mode 100644 index 000000000000..5d9f761e8a74 --- /dev/null +++ b/backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.gluten.delta + +import org.apache.spark.sql.{DataFrame, SparkSession} + +object DeltaStatsUtils { + + def statsDF( + sparkSession: SparkSession, + deltaJson: String, + schema: String + ): DataFrame = { + throw new IllegalAccessException("Method not used below spark 3.5") + } +} diff --git a/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala b/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala new file mode 100644 index 000000000000..5d9f761e8a74 --- /dev/null +++ b/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.gluten.delta + +import org.apache.spark.sql.{DataFrame, SparkSession} + +object DeltaStatsUtils { + + def statsDF( + sparkSession: SparkSession, + deltaJson: String, + schema: String + ): DataFrame = { + throw new IllegalAccessException("Method not used below spark 3.5") + } +} diff --git a/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala b/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala new file mode 100644 index 000000000000..08681892dc1d --- /dev/null +++ b/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.gluten.delta + +import org.apache.spark.sql.{wrapper, DataFrame, SparkSession} +import org.apache.spark.sql.delta.{DeltaColumnMappingMode, DeltaLog, DeltaLogFileIndex, NoMapping} +import org.apache.spark.sql.delta.actions.Protocol +import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION +import org.apache.spark.sql.delta.stats.{DeltaStatsColumnSpec, StatisticsCollection} +import org.apache.spark.sql.functions.{col, from_json, regexp_replace} +import org.apache.spark.sql.types._ + +import org.apache.hadoop.fs.{FileSystem, Path} + +import java.util.Locale + +case class Statistics(override val tableSchema: StructType) extends StatisticsCollection { + override val outputAttributeSchema: StructType = tableSchema + // [[outputTableStatsSchema]] is the candidate schema to find statistics columns. + override val outputTableStatsSchema: StructType = tableSchema + override val statsColumnSpec = DeltaStatsColumnSpec(None, Some(32)) + override val columnMappingMode: DeltaColumnMappingMode = NoMapping + override val protocol: Protocol = Protocol( + minReaderVersion = 1, + minWriterVersion = TABLE_FEATURES_MIN_WRITER_VERSION, + readerFeatures = None, + writerFeatures = Some(Set())) + + override def spark: SparkSession = { + throw new Exception("Method not used in statisticsCollectionFromMetadata") + } +} + +object DeltaStatsUtils { + + private def stringToDataType(dataType: String): DataType = + dataType.toLowerCase(Locale.ROOT) match { + case "bigint" => LongType + case "double" => DoubleType + case "string" => StringType + case "date" => DateType + case _ => throw new IllegalArgumentException(s"Unsupported data type: $dataType") + } + + /** + * Parse a schema string as follows into a [[StructType]]. + * {{{ + * l_orderkey bigint, + * l_partkey bigint, + * l_suppkey bigint, + * l_linenumber bigint + * }}} + */ + private def stringToSchema(schemaString: String): StructType = { + val fields = schemaString.trim.split(",\\s*").map { + fieldString => + val parts = fieldString.trim.split("\\s+") + require(parts.length == 2, s"Invalid field definition: $fieldString") + val fieldName = parts(0).trim + val fieldType = stringToDataType(parts(1).trim) + StructField(fieldName, fieldType, nullable = true) + } + StructType(fields) + } + + def statsDF( + sparkSession: SparkSession, + deltaJson: String, + schema: String + ): DataFrame = { + + val statistics = Statistics(stringToSchema(schema)) + + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(Map.empty) + wrapper + .ofRows( + sparkSession, + DeltaLog.indexToRelation( + sparkSession, + DeltaLogFileIndex( + DeltaLogFileIndex.COMMIT_FILE_FORMAT, + FileSystem.get(hadoopConf), + Seq(new Path(deltaJson))), + Map.empty) + ) + .select("add") + .filter("add is not null") + .withColumns(Map( + "path" -> regexp_replace( + col("add.path"), + "-[0-9A-Fa-f]{8}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{12}", + "" + ), // normalize file name + "stats" -> from_json(col("add.stats"), statistics.statsSchema) + )) + .select( + "path", + "stats.numRecords", + "stats.minValues.*", + "stats.maxValues.*", + "stats.nullCount.*") + } + +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala index b5ab401ec6a2..2f55510a7b1f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala @@ -16,7 +16,10 @@ */ package org.apache.gluten.execution +import org.apache.gluten.GlutenConfig + import org.apache.spark.SparkConf +import org.apache.spark.gluten.delta.DeltaStatsUtils import org.apache.spark.sql.SaveMode import org.apache.spark.sql.delta.files.TahoeFileIndex import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -49,7 +52,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") .set("spark.sql.files.maxPartitionBytes", "20000000") - .set("spark.gluten.sql.native.writer.enabled", "true") + .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, spark35.toString) .set("spark.sql.storeAssignmentPolicy", "legacy") .setCHSettings("mergetree.merge_after_insert", false) .set("spark.databricks.delta.retentionDurationCheck.enabled", "false") @@ -59,67 +62,45 @@ class GlutenClickHouseDeltaParquetWriteSuite createNotNullTPCHTablesInParquet(tablesPath) } + private val q1SchemaString: String = + s""" l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string""".stripMargin + + private def doInsert(drop: String, create: String, insert: String): Unit = { + spark.sql(drop) + spark.sql(create) + spark.sql(insert) + } + private def drop(table: String): String = s"DROP TABLE IF EXISTS $table" + private def createLineitem(table: String): String = + s"""CREATE TABLE IF NOT EXISTS $table ($q1SchemaString) USING delta + |TBLPROPERTIES (write.format.default = 'parquet') + |LOCATION '$basePath/$table' + |""".stripMargin + test("test parquet table write with the delta") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_delta_parquet; - |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |TBLPROPERTIES (write.format.default = 'parquet') - |LOCATION '$basePath/lineitem_delta_parquet' - |""".stripMargin) + def insert(table: String): String = + s"insert into table $table select /*+ REPARTITION(5) */ * from lineitem" - spark.sql(s""" - | insert into table lineitem_delta_parquet - | select /*+ REPARTITION(5) */ * from lineitem - |""".stripMargin) + val table = "lineitem_delta_parquet" + doInsert(drop(table), createLineitem(table), insert(table)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(table)) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -134,106 +115,64 @@ class GlutenClickHouseDeltaParquetWriteSuite val addFiles = fileIndex.matchingFiles(Nil, Nil) assert(addFiles.size === 5) } + + if (spark35) { + val vanillaTable = "lineitem_delta_parquet_vanilla" + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { + doInsert(drop(vanillaTable), createLineitem(vanillaTable), insert(vanillaTable)) + } + val expected = DeltaStatsUtils + .statsDF( + spark, + s"$basePath/$vanillaTable/_delta_log/00000000000000000001.json", + q1SchemaString) + .collect() + + checkAnswer( + DeltaStatsUtils.statsDF( + spark, + s"$basePath/$table/_delta_log/00000000000000000001.json", + q1SchemaString), + expected + ) + } } test("test parquet insert overwrite with the delta") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite; - |""".stripMargin) - - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite' - |""".stripMargin) - - spark.sql(s""" - | insert into table lineitem_delta_parquet_insertoverwrite - | select * from lineitem - |""".stripMargin) + def insert(table: String): String = + s"insert into table $table select * from lineitem" + val table = "lineitem_delta_parquet_insertoverwrite" + doInsert(drop(table), createLineitem(table), insert(table)) spark.sql(s""" - | insert overwrite table lineitem_delta_parquet_insertoverwrite + | insert overwrite table $table | select * from lineitem where mod(l_orderkey,2) = 1 |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_delta_parquet_insertoverwrite - |""".stripMargin - assert( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) === 300001 - ) + + // total rows should remain unchanged + assert(spark.sql(s"select count(*) from $table").collect().apply(0).get(0) === 300001) } test("test parquet insert overwrite partitioned table with small table, static with delta") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite2; - |""".stripMargin) - - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite2 - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |PARTITIONED BY (l_shipdate) - |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite2' - |""".stripMargin) - - spark.sql(s""" - | insert into table lineitem_delta_parquet_insertoverwrite2 - | select * from lineitem - | where l_shipdate BETWEEN date'1993-01-01' AND date'1993-03-31' - |""".stripMargin) - - spark.sql( + val table = "lineitem_delta_parquet_insertoverwrite2" + doInsert( + drop(table), s""" - | insert overwrite table lineitem_delta_parquet_insertoverwrite2 - | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' - |""".stripMargin) - val sql2 = + |CREATE TABLE IF NOT EXISTS $table ($q1SchemaString) USING delta + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/$table' + |""".stripMargin, s""" - | select count(*) from lineitem_delta_parquet_insertoverwrite2 - | + | insert into table $table select * from lineitem + | where l_shipdate BETWEEN date'1993-01-01' AND date'1993-03-31' |""".stripMargin - assert( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) === 2418 ) + spark.sql(s""" + | insert overwrite table $table select * from lineitem + | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + // total rows should remain unchanged + assert(spark.sql(s"select count(*) from $table").collect().apply(0).get(0) === 2418) } test("test parquet insert overwrite partitioned table with small table, dynamic with delta") { @@ -244,24 +183,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite3 - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |PARTITIONED BY (l_shipdate) |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite3' @@ -297,24 +219,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_update - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |LOCATION '$basePath/lineitem_delta_parquet_update' |""".stripMargin) @@ -360,10 +265,9 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from lineitem_delta_parquet_update | |""".stripMargin - assert( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) === 600572 - ) + + // total rows should remain unchanged + assert(spark.sql(sql2).collect().apply(0).get(0) === 600572) } test("test parquet table delete with the delta") { @@ -373,24 +277,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_delete - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |LOCATION '$basePath/lineitem_delta_parquet_delete' |""".stripMargin) @@ -440,24 +327,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_upsert - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |LOCATION '$basePath/lineitem_delta_parquet_upsert' |""".stripMargin) @@ -542,24 +412,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_partition - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |PARTITIONED BY (l_shipdate, l_returnflag) |LOCATION '$basePath/lineitem_delta_parquet_partition' @@ -635,32 +488,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | l_comment from lineitem | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet_partition - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_partition"), compareResult = false) { df => val result = df.collect() assert(result.length === 2) @@ -703,32 +531,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet_ctas1 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_ctas1")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -756,65 +559,14 @@ class GlutenClickHouseDeltaParquetWriteSuite |LOCATION '$basePath/lineitem_mergetree_ctas2' | as select * from lineitem |""".stripMargin) - - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet_ctas2 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } - + runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_ctas2")) { _ => {} } } test("test path based parquet write with the delta") { val dataPath = s"$basePath/lineitem_delta_parquet_filebased" clearDataPath(dataPath) - - val sourceDF = spark.sql(s""" - |select * from lineitem - |""".stripMargin) - - spark.sql(s""" - |CREATE TABLE delta.`$dataPath` ( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |""".stripMargin) + val sourceDF = spark.sql("select * from lineitem") + spark.sql(s"CREATE TABLE delta.`$dataPath` ($q1SchemaString) USING delta") sourceDF.write .format("delta") @@ -829,32 +581,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .mode(SaveMode.Overwrite) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | delta.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`")) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1149,32 +876,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | delta.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`"), compareResult = false) { df => val result = df.collect() assert(result.length === 2) @@ -1215,35 +917,10 @@ class GlutenClickHouseDeltaParquetWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | delta.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`")) { _ => {} } } - testSparkVersionLE33("test parquet optimize basic") { + test("test parquet optimize basic") { withSQLConf("spark.databricks.delta.optimize.maxFileSize" -> "20000000") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize; @@ -1278,7 +955,7 @@ class GlutenClickHouseDeltaParquetWriteSuite } } - testSparkVersionLE33("test parquet optimize partitioned by one low card column") { + test("test parquet optimize partitioned by one low card column") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p2; |""".stripMargin) @@ -1317,7 +994,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(ret2.apply(0).get(0) === 600572) } - testSparkVersionLE33("test parquet optimize parallel delete") { + test("test parquet optimize parallel delete") { withSQLConf("spark.databricks.delta.vacuum.parallelDelete.enabled" -> "true") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p4; diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala index 0fd868a8a878..7c444d572b69 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.execution +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.spark.SparkConf @@ -76,7 +77,7 @@ class GlutenClickHouseExcelFormatSuite // in this case, FakeRowAdaptor does R2C test("parquet native writer writing a in memory DF") { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { val filePath = basePath + "/native_parquet_test" val format = "parquet" @@ -99,7 +100,7 @@ class GlutenClickHouseExcelFormatSuite // in this case, FakeRowAdaptor only wrap&transfer test("parquet native writer writing a DF from file") { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { val filePath = basePath + "/native_parquet_test" val format = "parquet" @@ -123,7 +124,7 @@ class GlutenClickHouseExcelFormatSuite // in this case, FakeRowAdaptor only wrap&transfer test("parquet native writer writing a DF from an aggregate") { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { val filePath = basePath + "/native_parquet_test_agg" val format = "parquet" @@ -1480,7 +1481,7 @@ class GlutenClickHouseExcelFormatSuite | from $format.`$tablePath` | where long_field > 30 |""".stripMargin - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { testFileFormatBase(tablePath, format, sql, df => {}) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala index 0205bae2d612..fbaebdecaaf6 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala @@ -621,4 +621,42 @@ abstract class GlutenClickHouseTPCHAbstractSuite checkDataFrame(noFallBack, customCheck, df) } + def q1(tableName: String): String = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | $tableName + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + + def q6(tableName: String): String = + s""" + |SELECT + | sum(l_extendedprice * l_discount) AS revenue + |FROM + | $tableName + |WHERE + | l_shipdate >= date'1994-01-01' + | AND l_shipdate < date'1994-01-01' + interval 1 year + | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 + | AND l_quantity < 24 + |""".stripMargin } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index 02af10657c73..e35f6bf65bf3 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -64,7 +64,7 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { test("test uuid - write and read") { withSQLConf( - ("spark.gluten.sql.native.writer.enabled", "true"), + (GlutenConfig.NATIVE_WRITER_ENABLED.key, "true"), (GlutenConfig.GLUTEN_ENABLED.key, "true")) { withTable("uuid_test") { spark.sql("create table if not exists uuid_test (id string) using parquet") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala index 53aef16d143e..d0d03eaf1edb 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala @@ -610,7 +610,7 @@ class GlutenClickHouseNativeWriteTableSuite } val table_name_vanilla = table_name_vanilla_template.format(format) spark.sql(s"drop table IF EXISTS $table_name_vanilla") - withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { withNativeWriteCheck(checkNative = false) { spark .table("origin_table") @@ -672,7 +672,7 @@ class GlutenClickHouseNativeWriteTableSuite val table_name_vanilla = table_name_vanilla_template.format(format) spark.sql(s"drop table IF EXISTS $table_name_vanilla") - withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { withNativeWriteCheck(checkNative = false) { spark .table("origin_table") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala index 503c9bd886ed..5c5f93c62c6b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala @@ -107,40 +107,14 @@ class GlutenClickHouseTableAfterRestart | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - // before restart, check if cache works { - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_mergetree"))(_ => {}) val oldMissingCount1 = ClickhouseSnapshot.deltaScanCache.stats().missCount() val oldMissingCount2 = ClickhouseSnapshot.addFileToAddMTPCache.stats().missCount() // for this run, missing count should not increase - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_mergetree"))(_ => {}) val stats1 = ClickhouseSnapshot.deltaScanCache.stats() assertResult(oldMissingCount1)(stats1.missCount()) val stats2 = ClickhouseSnapshot.addFileToAddMTPCache.stats() @@ -152,7 +126,7 @@ class GlutenClickHouseTableAfterRestart restartSpark() - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_mergetree"))(_ => {}) // after restart, additionally check stats of delta scan cache val stats1 = ClickhouseSnapshot.deltaScanCache.stats() diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala index 6d5835957c82..76b3ba9f3724 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala @@ -114,32 +114,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Overwrite) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -205,32 +180,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .option("clickhouse.lowCardKey", "l_returnflag,l_linestatus") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -575,32 +525,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -672,32 +597,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -771,32 +671,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -875,32 +750,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -935,32 +785,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { _ => {} } } @@ -978,32 +803,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { _ => {} } val directory = new File(dataPath) // find a folder whose name is like 48b70783-b3b8-4bf8-9c52-5261aead8e3e_0_006 val partDir = directory.listFiles().filter(f => f.getName.length > 20).head @@ -1057,19 +857,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1218,32 +1006,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite test("GLUTEN-5219: Fix the table metadata sync issue for the CH backend") { def checkQueryResult(tableName: String): Unit = { - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$tableName` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$tableName`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala index af9b541d91fd..b147173d255b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala @@ -111,32 +111,8 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | select * from lineitem |""".stripMargin) FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -196,32 +172,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -358,32 +309,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | where l_returnflag = 'A' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition_hdfs"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -471,32 +397,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -552,32 +453,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite .option("clickhouse.storage_policy", "__hdfs_main") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala index 9bd919f20013..2540186c9984 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala @@ -111,32 +111,8 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | select * from lineitem |""".stripMargin) FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -196,32 +172,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -358,32 +309,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | where l_returnflag = 'A' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition_hdfs"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -471,32 +397,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -552,32 +453,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite .option("clickhouse.storage_policy", "__hdfs_main_rocksdb") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index c0f509c68cda..8e698adb4b51 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -125,32 +125,8 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | select * from lineitem |""".stripMargin) FileUtils.deleteDirectory(new File(S3_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_s3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -256,32 +232,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey_s3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -418,32 +369,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | where l_returnflag = 'A' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition_s3"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -532,32 +458,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket_s3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -613,32 +514,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite .option("clickhouse.storage_policy", "__s3_main") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -743,21 +619,8 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite FileUtils.forceDelete(new File(S3_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | $tableName - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - withSQLConf(runtimeSettings("enabled_driver_filter_mergetree_index") -> "true") { - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6(tableName)) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index 72adee309dd7..cc577609656b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.execution.mergetree +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution._ import org.apache.gluten.utils.Arm @@ -55,7 +56,7 @@ class GlutenClickHouseMergeTreeWriteSuite .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") .set("spark.sql.files.maxPartitionBytes", "20000000") - .set("spark.gluten.sql.native.writer.enabled", "true") + .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, "true") .setCHSettings("min_insert_block_size_rows", 100000) .setCHSettings("mergetree.merge_after_insert", false) .setCHSettings("input_format_parquet_max_block_size", 8192) @@ -102,32 +103,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree")) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -213,7 +189,7 @@ class GlutenClickHouseMergeTreeWriteSuite } test("test mergetree insert overwrite partitioned table with small table, static") { - withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, "false")) { + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)) { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite2; |""".stripMargin) @@ -573,32 +549,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -731,32 +682,8 @@ class GlutenClickHouseMergeTreeWriteSuite | l_comment from lineitem | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -849,32 +776,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1055,32 +957,7 @@ class GlutenClickHouseMergeTreeWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_ctas1 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_ctas1")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1117,32 +994,7 @@ class GlutenClickHouseMergeTreeWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_ctas2 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1("lineitem_mergetree_ctas2")) { _ => {} } } @@ -1181,32 +1033,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_lowcard - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1("lineitem_mergetree_lowcard")) { _ => {} } val directory = new File(s"$basePath/lineitem_mergetree_lowcard") // find a folder whose name is like 48b70783-b3b8-4bf8-9c52-5261aead8e3e_0_006 val partDir = directory.listFiles().filter(f => f.getName.length > 20).head @@ -1281,19 +1108,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_orderbykey2 - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_orderbykey2")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1368,18 +1183,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_orderbykey3 - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_orderbykey3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1584,7 +1388,7 @@ class GlutenClickHouseMergeTreeWriteSuite | l_linestatus; | |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(tableName)) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1777,23 +1581,10 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_pk_pruning_by_driver - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - Seq(("true", 2), ("false", 3)).foreach( conf => { withSQLConf(CHConf.runtimeSettings("enabled_driver_filter_mergetree_index") -> conf._1) { - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_pk_pruning_by_driver")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1817,7 +1608,7 @@ class GlutenClickHouseMergeTreeWriteSuite CHConf.runtimeSettings("enabled_driver_filter_mergetree_index") -> "true", CHConf.prefixOf("files.per.partition.threshold") -> "10" ) { - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_pk_pruning_by_driver")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -2044,19 +1835,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_case_sensitive - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { _ => } + runTPCHQueryBySQL(6, q6("lineitem_mergetree_case_sensitive")) { _ => } } test("test mergetree with partition with whitespace") { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala index 5f03a0d39f50..767585c4c41e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala @@ -86,31 +86,6 @@ class GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_task_not_serializable - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_task_not_serializable"))(_ => {}) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala index 49e368c888e7..631461b6a2f4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.gluten +import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.GlutenClickHouseWholeStageTransformerSuite import org.apache.spark.sql.{Dataset, Row} @@ -70,13 +71,13 @@ trait NativeWriteChecker } def nativeWrite(f: String => Unit): Unit = { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { formats.foreach(f(_)) } } def vanillaWrite(block: => Unit): Unit = { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { block } } diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/wrapper/package.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/wrapper/package.scala new file mode 100644 index 000000000000..c93387bf9dcf --- /dev/null +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/wrapper/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +package object wrapper { + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = + Dataset.ofRows(sparkSession, logicalPlan) +} diff --git a/cpp-ch/local-engine/Common/BlockTypeUtils.cpp b/cpp-ch/local-engine/Common/BlockTypeUtils.cpp index f6dcd0959225..35890cc78131 100644 --- a/cpp-ch/local-engine/Common/BlockTypeUtils.cpp +++ b/cpp-ch/local-engine/Common/BlockTypeUtils.cpp @@ -39,18 +39,12 @@ DB::DataTypePtr wrapNullableType(bool nullable, DB::DataTypePtr nested_type) { if (nested_type->isLowCardinalityNullable()) return nested_type; - else if (!nested_type->lowCardinality()) - return std::make_shared(nested_type); - else + if (nested_type->lowCardinality()) return std::make_shared( std::make_shared(dynamic_cast(*nested_type).getDictionaryType())); - } - - - if (nullable && !nested_type->isNullable()) return std::make_shared(nested_type); - else - return nested_type; + } + return nested_type; } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/BlockTypeUtils.h b/cpp-ch/local-engine/Common/BlockTypeUtils.h index af01b528d6e9..64032af70f68 100644 --- a/cpp-ch/local-engine/Common/BlockTypeUtils.h +++ b/cpp-ch/local-engine/Common/BlockTypeUtils.h @@ -73,14 +73,18 @@ inline DB::DataTypePtr DATE() return std::make_shared(); } -inline DB::Block makeBlockHeader(const DB::ColumnsWithTypeAndName & data_) +inline DB::Block makeBlockHeader(const DB::ColumnsWithTypeAndName & data) { - return DB::Block(data_); + return DB::Block(data); } DB::NamesAndTypesList blockToNameAndTypeList(const DB::Block & header); DB::DataTypePtr wrapNullableType(bool nullable, DB::DataTypePtr nested_type); +inline DB::DataTypePtr wrapNullableType(DB::DataTypePtr nested_type) +{ + return wrapNullableType(true, nested_type); +} inline DB::DataTypePtr wrapNullableType(const substrait::Type_Nullability nullable, const DB::DataTypePtr & nested_type) { return wrapNullableType(nullable == substrait::Type_Nullability_NULLABILITY_NULLABLE, nested_type); diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index 8a4323cb1c13..8cda6734670e 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -19,10 +19,7 @@ #include #include #include -#include -#include #include -#include #include #include #include @@ -37,6 +34,260 @@ namespace pb_util = google::protobuf::util; namespace debug { +namespace Utils +{ + +/** + * Return the number of half widths in a given string. Note that a full width character + * occupies two half widths. + * + * For a string consisting of 1 million characters, the execution of this method requires + * about 50ms. + */ +static size_t stringHalfWidth(const std::string & str) +{ + //TODO: Implement this method + return str.size(); +} + +/** + *

    Left pad a String with spaces (' ').

    + * + *

    The String is padded to the size of {@code size}.

    + * + *
    + * StringUtils.leftPad(null, *)   = null
    + * StringUtils.leftPad("", 3)     = "   "
    + * StringUtils.leftPad("bat", 3)  = "bat"
    + * StringUtils.leftPad("bat", 5)  = "  bat"
    + * StringUtils.leftPad("bat", 1)  = "bat"
    + * StringUtils.leftPad("bat", -1) = "bat"
    + * 
    + * + * @param str the String to pad out, may be null + * @param size the size to pad to + * @return left padded String or original String if no padding is necessary, + * {@code null} if null String input + */ +static std::string leftPad(const std::string & str, int totalWidth) +{ + std::stringstream ss; + ss << std::setw(totalWidth) << std::setfill(' ') << str; + return ss.str(); +} + +/** + *

    Right pad a String with spaces (' ').

    + * + *

    The String is padded to the size of {@code size}.

    + * + *
    + * StringUtils.rightPad(null, *)   = null
    + * StringUtils.rightPad("", 3)     = "   "
    + * StringUtils.rightPad("bat", 3)  = "bat"
    + * StringUtils.rightPad("bat", 5)  = "bat  "
    + * StringUtils.rightPad("bat", 1)  = "bat"
    + * StringUtils.rightPad("bat", -1) = "bat"
    + * 
    + * + * @param str the String to pad out, may be null + * @param totalWidth the size to pad to + * @param padChar the character to pad with + * @param size the size to pad to + * @return right padded String or original String if no padding is necessary, + * {@code null} if null String input + */ +static std::string rightPad(const std::string & str, int totalWidth, char padChar = ' ') +{ + std::stringstream ss; + ss << str << std::setw(totalWidth - str.size()) << std::setfill(padChar) << ""; + return ss.str(); +} + +static std::string truncate(const std::string & str, size_t width) +{ + if (str.size() <= width) + return str; + return str.substr(0, width - 3) + "..."; +} + +using NameAndColumn = std::pair; +using NameAndColumns = std::vector; + +/** + * Get rows represented in Sequence by specific truncate and vertical requirement. + * + * @param block Columns to show + * @param numRows Number of rows to return + * @param truncate If set to more than 0, truncates strings to `truncate` characters and + * all cells will be aligned right. + */ +static std::vector> getRows(const NameAndColumns & block, size_t numRows, size_t truncate) +{ + std::vector> results; + results.reserve(numRows); + results.emplace_back(std::vector()); + auto & headRow = results.back(); + + for (const auto & column : block) + { + const auto & name = column.first; + headRow.emplace_back(debug::Utils::truncate(name, truncate)); + } + + auto getDataType = [](const DB::IColumn * col) + { + if (const auto * column_nullable = DB::checkAndGetColumn(col)) + return column_nullable->getNestedColumn().getDataType(); + return col->getDataType(); + }; + + for (size_t row = 0; row < numRows - 1; ++row) + { + results.emplace_back(std::vector()); + auto & currentRow = results.back(); + currentRow.reserve(block.size()); + + for (const auto & column : block) + { + const auto * const col = column.second.get(); + DB::WhichDataType which(getDataType(col)); + if (which.isAggregateFunction()) + currentRow.emplace_back("Nan"); + else + { + if (col->isNullAt(row)) + currentRow.emplace_back("null"); + else + { + std::string str = DB::toString((*col)[row]); + currentRow.emplace_back(Utils::truncate(str, truncate)); + } + } + } + } + return results; +} + +static std::string showString(const NameAndColumns & block, size_t numRows, size_t truncate, bool vertical) +{ + numRows = std::min(numRows, block[0].second->size()); + bool hasMoreData = block[0].second->size() > numRows; + // Get rows represented by vector[vector[String]], we may get one more line if it has more data. + std::vector> rows = getRows(block, numRows + 1, truncate); + + size_t numCols = block.size(); + // We set a minimum column width at '3' + constexpr size_t minimumColWidth = 3; + + std::stringstream sb; + + if (!vertical) + { + // Initialise the width of each column to a minimum value + std::vector colWidths(numCols, minimumColWidth); + + // Compute the width of each column + for (const auto & row : rows) + for (size_t i = 0; i < row.size(); ++i) + colWidths[i] = std::max(colWidths[i], stringHalfWidth(row[i])); + + std::vector> paddedRows; + for (const auto & row : rows) + { + std::vector paddedRow; + for (size_t i = 0; i < row.size(); ++i) + if (truncate > 0) + paddedRow.push_back(leftPad(row[i], colWidths[i] - stringHalfWidth(row[i]) + row[i].size())); + else + paddedRow.push_back(rightPad(row[i], colWidths[i] - stringHalfWidth(row[i]) + row[i].size())); + paddedRows.push_back(paddedRow); + } + + // Create SeparateLine + std::stringstream sep; + for (int width : colWidths) + sep << "+" << std::string(width, '-'); + sep << "+\n"; + + // column names + sb << sep.str(); + for (const auto & cell : paddedRows[0]) + sb << "|" << cell; + sb << "|\n" << sep.str(); + + // data + for (size_t i = 1; i < paddedRows.size(); ++i) + { + for (const auto & cell : paddedRows[i]) + sb << "|" << cell; + sb << "|\n"; + } + sb << sep.str(); + } + else + { + // Extended display mode enabled + const std::vector & fieldNames = rows[0]; + auto dataRowsBegin = [&]() { return rows.begin() + 1; }; + + // Compute the width of field name and data columns + size_t fieldNameColWidth = minimumColWidth; + for (const auto & fieldName : fieldNames) + fieldNameColWidth = std::max(fieldNameColWidth, Utils::stringHalfWidth(fieldName)); + + size_t dataColWidth = minimumColWidth; + + + for (auto dataRowIter = dataRowsBegin(); dataRowIter != rows.end(); ++dataRowIter) + { + const auto & row = *dataRowIter; + size_t maxWidth = 0; + for (const auto & cell : row) + maxWidth = std::max(maxWidth, stringHalfWidth(cell)); + dataColWidth = std::max(dataColWidth, maxWidth); + } + + // + for (auto dataRowIter = dataRowsBegin(); dataRowIter != rows.end(); ++dataRowIter) + { + // create row header + std::string rowHeader = "-RECORD " + std::to_string(rows.end() - dataRowIter); + rowHeader = rightPad(rowHeader, fieldNameColWidth + dataColWidth + 5, '-'); + sb << rowHeader << "\n"; + + // process each cell in the row + const auto & row = *dataRowIter; + for (size_t j = 0; j < row.size(); j++) + { + const std::string & cell = row[j]; + const std::string & fieldName = fieldNames[j]; + std::string paddedFieldName = rightPad(fieldName, fieldNameColWidth - stringHalfWidth(fieldName) + fieldName.length()); + std::string paddedData = rightPad(cell, dataColWidth - stringHalfWidth(cell) + cell.length()); + sb << " " << paddedFieldName << " | " << paddedData << " \n"; + } + sb << "\n"; + } + } + + // Print a footer + if (vertical && block[0].second->empty()) + { + // In a vertical mode, print an empty row set explicitly + sb << "(0 rows)" << std::endl; + } + else if (hasMoreData) + { + // For Data that has more than "numRows" records + const char * rowsString = (numRows == 1) ? "row" : "rows"; + sb << "only showing top " << numRows << " " << rowsString << std::endl; + } + return sb.str(); +} +} // namespace Utils + + +/// void dumpPlan(DB::QueryPlan & plan, const char * type, bool force, LoggerPtr logger) { @@ -85,80 +336,33 @@ void dumpMessage(const google::protobuf::Message & message, const char * type, b void headBlock(const DB::Block & block, size_t count) { - std::cout << "============Block============" << std::endl; - std::cout << block.dumpStructure() << std::endl; - // print header - for (const auto & name : block.getNames()) - std::cout << name << "\t"; - std::cout << std::endl; - - // print rows - for (size_t row = 0; row < std::min(count, block.rows()); ++row) - { - for (size_t column = 0; column < block.columns(); ++column) - { - const auto type = block.getByPosition(column).type; - auto col = block.getByPosition(column).column; - - if (column > 0) - std::cout << "\t"; - DB::WhichDataType which(type); - if (which.isAggregateFunction()) - std::cout << "Nan"; - else if (col->isNullAt(row)) - std::cout << "null"; - else - std::cout << toString((*col)[row]); - } - std::cout << std::endl; - } + std::cerr << showString(block, count) << std::endl; } -String printBlock(const DB::Block & block, size_t count) +void headColumn(const DB::ColumnPtr & column, size_t count) { - std::ostringstream ss; - ss << std::string("============Block============\n"); - ss << block.dumpStructure() << String("\n"); - // print header - for (const auto & name : block.getNames()) - ss << name << std::string("\t"); - ss << std::string("\n"); - - // print rows - for (size_t row = 0; row < std::min(count, block.rows()); ++row) - { - for (size_t column = 0; column < block.columns(); ++column) - { - const auto type = block.getByPosition(column).type; - auto col = block.getByPosition(column).column; - - if (column > 0) - ss << std::string("\t"); - DB::WhichDataType which(type); - if (which.isAggregateFunction()) - ss << std::string("Nan"); - else if (col->isNullAt(row)) - ss << std::string("null"); - else - ss << toString((*col)[row]); - } - ss << std::string("\n"); - } - return ss.str(); + std::cerr << Utils::showString({{"Column", column}}, count, 20, false) << std::endl; } +/** + * Compose the string representing rows for output + * + * @param block Block to show + * @param numRows Number of rows to show + * @param truncate If set to more than 0, truncates strings to `truncate` characters and + * all cells will be aligned right. + * @param vertical If set to true, prints output rows vertically (one line per column value). + */ -void headColumn(const DB::ColumnPtr & column, size_t count) +std::string showString(const DB::Block & block, size_t numRows, size_t truncate, bool vertical) { - std::cout << "============Column============" << std::endl; - - // print header - std::cout << column->getName() << "\t"; - std::cout << std::endl; - - // print rows - for (size_t row = 0; row < std::min(count, column->size()); ++row) - std::cout << toString((*column)[row]) << std::endl; -} - + std::vector columns = block.getColumnsWithTypeAndName(); + Utils::NameAndColumns name_and_columns; + name_and_columns.reserve(columns.size()); + std::ranges::transform( + columns, + std::back_inserter(name_and_columns), + [](const DB::ColumnWithTypeAndName & col) { return std::make_pair(col.name, col.column); }); + return Utils::showString(name_and_columns, numRows, truncate, vertical); } +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/DebugUtils.h b/cpp-ch/local-engine/Common/DebugUtils.h index 338326b05e0e..7b9fb637dcc6 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.h +++ b/cpp-ch/local-engine/Common/DebugUtils.h @@ -33,7 +33,11 @@ void dumpPlan(DB::QueryPlan & plan, const char * type = "clickhouse plan", bool void dumpMessage(const google::protobuf::Message & message, const char * type, bool force = false, LoggerPtr = nullptr); void headBlock(const DB::Block & block, size_t count = 10); -String printBlock(const DB::Block & block, size_t count = 10); - void headColumn(const DB::ColumnPtr & column, size_t count = 10); +std::string showString(const DB::Block & block, size_t numRows = 20, size_t truncate = 20, bool vertical = false); +inline std::string verticalShowString(const DB::Block & block, size_t numRows = 20, size_t truncate = 20) +{ + return showString(block, numRows, truncate, true); +} + } diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp index 25f786a77465..2dacb3991858 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp @@ -52,17 +52,18 @@ DB::ProcessorPtr make_sink( const DB::Block & input_header, const DB::Block & output_header, const std::string & base_path, - const std::string & filename, + const FileNameGenerator & generator, const std::string & format_hint, const std::shared_ptr & stats) { if (partition_by.empty()) { - return std::make_shared(context, base_path, "", filename, format_hint, input_header, stats); + return std::make_shared( + context, base_path, "", generator.generate(), format_hint, input_header, stats, DeltaStats{input_header.columns()}); } return std::make_shared( - context, partition_by, input_header, output_header, base_path, filename, format_hint, stats); + context, partition_by, input_header, output_header, base_path, generator, format_hint, stats); } DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB::Block & output) @@ -147,7 +148,6 @@ void addMergeTreeSinkTransform( const DB::Block & header, const DB::Names & partition_by) { - Chain chain; // auto stats = std::make_shared(header); @@ -158,9 +158,9 @@ void addMergeTreeSinkTransform( if (partition_by.empty()) write_settings.partition_settings.partition_dir = SubstraitFileSink::NO_PARTITION_ID; - auto sink = partition_by.empty() ? - SparkMergeTreeSink::create(merge_tree_table, write_settings, context->getGlobalContext(), {stats}) : - std::make_shared(header, partition_by, merge_tree_table, write_settings, context, stats); + auto sink = partition_by.empty() + ? SparkMergeTreeSink::create(merge_tree_table, write_settings, context->getGlobalContext(), {stats}) + : std::make_shared(header, partition_by, merge_tree_table, write_settings, context, stats); chain.addSource(sink); const DB::Settings & settings = context->getSettingsRef(); @@ -184,25 +184,22 @@ void addNormalFileWriterSinkTransform( if (write_settings.task_write_tmp_dir.empty()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject temp directory."); - if (write_settings.task_write_filename.empty()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name."); + if (write_settings.task_write_filename.empty() && write_settings.task_write_filename_pattern.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name or file name pattern."); + + FileNameGenerator generator{ + .pattern = write_settings.task_write_filename.empty(), + .filename_or_pattern + = write_settings.task_write_filename.empty() ? write_settings.task_write_filename_pattern : write_settings.task_write_filename}; - auto stats = std::make_shared(output); + auto stats = WriteStats::create(output, partitionCols); builder->addSimpleTransform( [&](const Block & cur_header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return make_sink( - context, - partitionCols, - cur_header, - output, - write_settings.task_write_tmp_dir, - write_settings.task_write_filename, - format_hint, - stats); + return make_sink(context, partitionCols, cur_header, output, write_settings.task_write_tmp_dir, generator, format_hint, stats); }); builder->addSimpleTransform( [&](const Block &, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h index 65676c45380e..0c9bc11f1f10 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h @@ -44,7 +44,8 @@ DB::Names collect_partition_cols(const DB::Block & header, const substrait::Name #define WRITE_RELATED_SETTINGS(M, ALIAS) \ M(String, task_write_tmp_dir, , "The temporary directory for writing data") \ - M(String, task_write_filename, , "The filename for writing data") + M(String, task_write_filename, , "The filename for writing data") \ + M(String, task_write_filename_pattern, , "The pattern to generate file name for writing delta parquet in spark 3.5") DECLARE_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp index de0c244f3e53..6c9dd890d851 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp @@ -89,7 +89,8 @@ SinkToStoragePtr SparkMergeTreeSink::create( temp->getStorageID().getFullNameNotQuoted()); sink_helper = std::make_shared(temp, dest_storage, write_settings_); } - sink_helper = std::make_shared(dest_storage, write_settings_, isRemoteStorage); + else + sink_helper = std::make_shared(dest_storage, write_settings_, isRemoteStorage); return std::make_shared(sink_helper, context, stats); } @@ -130,7 +131,7 @@ void SinkHelper::doMergePartsAsync(const std::vector & for (const auto & selected_part : prepare_merge_parts) tmp_parts.emplace(selected_part->name); - // check thread group initialized in task thread + // check a thread group initialized in task thread currentThreadGroupMemoryUsage(); thread_pool.scheduleOrThrow( [this, prepare_merge_parts, thread_group = CurrentThread::getGroup()]() -> void diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h index 065b147462f9..38f574ea9872 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h @@ -193,7 +193,6 @@ class MergeTreeStats : public WriteStatsBase void collectStats(const std::deque & parts, const std::string & partition) const { const size_t size = parts.size() + columns_[part_name]->size(); - columns_[part_name]->reserve(size); columns_[partition_id]->reserve(size); diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index cd9c848dd418..d55703741845 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -61,6 +61,85 @@ class NormalFileWriter : public NativeOutputWriter OutputFormatFilePtr createOutputFormatFile( const DB::ContextPtr & context, const std::string & file_uri, const DB::Block & preferred_schema, const std::string & format_hint); +struct DeltaStats +{ + size_t row_count; + std::vector min; + std::vector max; + std::vector null_count; + std::set partition_index; + + static DeltaStats create(const DB::Block & output, const DB::Names & partition) + { + size_t size = output.columns() - partition.size(); + std::set partition_index; + std::ranges::transform( + partition, + std::inserter(partition_index, partition_index.end()), + [&](const auto & name) { return output.getPositionByName(name); }); + assert(partition_index.size() == partition.size()); + return DeltaStats(size, partition_index); + } + static DB::Block statsHeader(const DB::Block & output, const DB::Names & partition, DB::ColumnsWithTypeAndName && statsHeaderBase) + { + std::set partition_index; + std::ranges::transform(partition, std::inserter(partition_index, partition_index.end()), [&](const auto & name) { return name; }); + + assert(partition_index.size() == partition.size()); + + auto appendBase = [&](const std::string & prefix) + { + for (const auto & column : output.getColumnsWithTypeAndName()) + if (!partition_index.contains(column.name)) + statsHeaderBase.emplace_back(wrapNullableType(column.type), prefix + column.name); + }; + appendBase("min_"); + appendBase("max_"); + for (const auto & column : output.getColumnsWithTypeAndName()) + if (!partition_index.contains(column.name)) + statsHeaderBase.emplace_back(BIGINT(), "null_count_" + column.name); + + return makeBlockHeader(statsHeaderBase); + } + + explicit DeltaStats(size_t size, const std::set & partition_index_ = {}) + : row_count(0), min(size), max(size), null_count(size, 0), partition_index(partition_index_) + { + } + + void update(const DB::Chunk & chunk) + { + row_count += chunk.getNumRows(); + const auto & columns = chunk.getColumns(); + assert(columns.size() == min.size() + partition_index.size()); + for (size_t i = 0, col = 0; col < columns.size(); ++col) + { + if (partition_index.contains(col)) + continue; + + const auto & column = columns[col]; + Int64 null_count = 0; + if (const auto * nullable_column = typeid_cast(column.get())) + { + const auto & null_map = nullable_column->getNullMapData(); + null_count = std::ranges::count_if(null_map, [](UInt8 value) { return value != 0; }); + } + this->null_count[i] += null_count; + + DB::Field min_value, max_value; + column->getExtremes(min_value, max_value); + assert(min[i].isNull() || min_value.getType() == min[i].getType()); + assert(max[i].isNull() || max_value.getType() == max[i].getType()); + if (min[i].isNull() || min_value < min[i]) + min[i] = min_value; + if (max[i].isNull() || max_value > max[i]) + max[i] = max_value; + + ++i; + } + } +}; + class WriteStatsBase : public DB::ISimpleTransform { protected: @@ -96,67 +175,69 @@ class WriteStatsBase : public DB::ISimpleTransform class WriteStats : public WriteStatsBase { - static DB::Block statsHeader() + DB::MutableColumns columns_; + + enum ColumnIndex { - return makeBlockHeader({{STRING(), "filename"}, {STRING(), "partition_id"}, {BIGINT(), "record_count"}}); - } - DB::Arena partition_keys_arena_; - std::string filename_; - absl::flat_hash_map file_to_count_; + filename, + partition_id, + record_count + }; protected: DB::Chunk final_result() override { - const size_t size = file_to_count_.size(); - - auto file_col = STRING()->createColumn(); - file_col->reserve(size); - auto partition_col = STRING()->createColumn(); - partition_col->reserve(size); - auto countCol = BIGINT()->createColumn(); - countCol->reserve(size); - auto & countColData = static_cast &>(*countCol).getData(); - - UInt64 num_rows = 0; - for (const auto & [relative_path, rows] : file_to_count_) - { - if (rows == 0) - continue; - file_col->insertData(filename_.c_str(), filename_.size()); - partition_col->insertData(relative_path.data, relative_path.size); - countColData.emplace_back(rows); - num_rows++; - } - - const DB::Columns res_columns{std::move(file_col), std::move(partition_col), std::move(countCol)}; - return DB::Chunk(res_columns, num_rows); + size_t rows = columns_[filename]->size(); + return DB::Chunk(std::move(columns_), rows); } public: - explicit WriteStats(const DB::Block & input_header_) : WriteStatsBase(input_header_, statsHeader()) { } - String getName() const override { return "WriteStats"; } - void addFilePath(const String & partition_id, const String & filename) + WriteStats(const DB::Block & input_header_, const DB::Block & output_header_) + : WriteStatsBase(input_header_, output_header_), columns_(output_header_.cloneEmptyColumns()) { - assert(!filename.empty()); - - if (filename_.empty()) - filename_ = filename; - - assert(filename_ == filename); + } - if (partition_id.empty()) - return; - file_to_count_.emplace(copyStringInArena(partition_keys_arena_, partition_id), 0); + static std::shared_ptr create(const DB::Block & input_header_, const DB::Names & partition) + { + return std::make_shared( + input_header_, + DeltaStats::statsHeader( + input_header_, partition, {{STRING(), "filename"}, {STRING(), "partition_id"}, {BIGINT(), "record_count"}})); } - void collectStats(const String & file_path, size_t rows) + String getName() const override { return "WriteStats"; } + + void collectStats(const String & filename, const String & partition, const DeltaStats & stats) const { - if (const auto it = file_to_count_.find(file_path); it != file_to_count_.end()) + // 3 => filename, partition_id, record_count + constexpr size_t baseOffset = 3; + assert(columns_.size() == baseOffset + stats.min.size() + stats.max.size() + stats.null_count.size()); + columns_[ColumnIndex::filename]->insertData(filename.c_str(), filename.size()); + columns_[partition_id]->insertData(partition.c_str(), partition.size()); + auto & countColData = static_cast &>(*columns_[record_count]).getData(); + countColData.emplace_back(stats.row_count); + size_t columnSize = stats.min.size(); + for (int i = 0; i < columnSize; ++i) { - it->second += rows; - return; + size_t offset = baseOffset + i; + columns_[offset]->insert(stats.min[i]); + columns_[columnSize + offset]->insert(stats.max[i]); + auto & nullCountData = static_cast &>(*columns_[(columnSize * 2) + offset]).getData(); + nullCountData.emplace_back(stats.null_count[i]); } - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "File path {} not found in the stats map", file_path); + } +}; + +struct FileNameGenerator +{ + const bool pattern; + const std::string filename_or_pattern; + + std::string generate() const + { + if (pattern) + return fmt::vformat(filename_or_pattern, fmt::make_format_args(toString(DB::UUIDHelpers::generateV4()))); + return filename_or_pattern; } }; @@ -164,10 +245,12 @@ class SubstraitFileSink final : public DB::SinkToStorage { const std::string partition_id_; const std::string relative_path_; + OutputFormatFilePtr format_file_; OutputFormatFile::OutputFormatPtr output_format_; std::shared_ptr stats_; + DeltaStats delta_stats_; - static std::string makeFilename(const std::string & base_path, const std::string & partition_id, const std::string & relative) + static std::string makeAbsoluteFilename(const std::string & base_path, const std::string & partition_id, const std::string & relative) { if (partition_id.empty()) return fmt::format("{}/{}", base_path, relative); @@ -185,32 +268,38 @@ class SubstraitFileSink final : public DB::SinkToStorage const std::string & relative, const std::string & format_hint, const DB::Block & header, - const std::shared_ptr & stats) + const std::shared_ptr & stats, + const DeltaStats & delta_stats) : SinkToStorage(header) , partition_id_(partition_id.empty() ? NO_PARTITION_ID : partition_id) , relative_path_(relative) - , output_format_(createOutputFormatFile(context, makeFilename(base_path, partition_id, relative), header, format_hint) - ->createOutputFormat(header)) + , format_file_(createOutputFormatFile(context, makeAbsoluteFilename(base_path, partition_id, relative), header, format_hint)) , stats_(std::dynamic_pointer_cast(stats)) + , delta_stats_(delta_stats) { - stats_->addFilePath(partition_id_, relative_path_); } + String getName() const override { return "SubstraitFileSink"; } protected: void consume(DB::Chunk & chunk) override { - const size_t row_count = chunk.getNumRows(); + delta_stats_.update(chunk); + if (!output_format_) [[unlikely]] + output_format_ = format_file_->createOutputFormat(); output_format_->output->write(materializeBlock(getHeader().cloneWithColumns(chunk.detachColumns()))); - - if (stats_) - stats_->collectStats(partition_id_, row_count); } void onFinish() override { - output_format_->output->finalize(); - output_format_->output->flush(); - output_format_->write_buffer->finalize(); + if (output_format_) [[unlikely]] + { + output_format_->output->finalize(); + output_format_->output->flush(); + output_format_->write_buffer->finalize(); + assert(delta_stats_.row_count > 0); + if (stats_) + stats_->collectStats(relative_path_, partition_id_, delta_stats_); + } } }; @@ -246,6 +335,7 @@ class SparkPartitionedBaseSink : public DB::PartitionedSink protected: DB::ContextPtr context_; std::shared_ptr stats_; + DeltaStats empty_delta_stats_; public: SparkPartitionedBaseSink( @@ -253,7 +343,10 @@ class SparkPartitionedBaseSink : public DB::PartitionedSink const DB::Names & partition_by, const DB::Block & input_header, const std::shared_ptr & stats) - : PartitionedSink(make_partition_expression(partition_by), context, input_header), context_(context), stats_(stats) + : PartitionedSink(make_partition_expression(partition_by), context, input_header) + , context_(context) + , stats_(stats) + , empty_delta_stats_(DeltaStats::create(input_header, partition_by)) { } }; @@ -261,7 +354,7 @@ class SparkPartitionedBaseSink : public DB::PartitionedSink class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink { const std::string base_path_; - const std::string filename_; + const FileNameGenerator generator_; const DB::Block sample_block_; const std::string format_hint_; @@ -272,12 +365,12 @@ class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink const DB::Block & input_header, const DB::Block & sample_block, const std::string & base_path, - const std::string & filename, + const FileNameGenerator & generator, const std::string & format_hint, const std::shared_ptr & stats) : SparkPartitionedBaseSink(context, partition_by, input_header, stats) , base_path_(base_path) - , filename_(filename) + , generator_(generator) , sample_block_(sample_block) , format_hint_(format_hint) { @@ -286,9 +379,11 @@ class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink DB::SinkPtr createSinkForPartition(const String & partition_id) override { assert(stats_); - const auto partition_path = fmt::format("{}/{}", partition_id, filename_); + std::string filename = generator_.generate(); + const auto partition_path = fmt::format("{}/{}", partition_id, filename); validatePartitionKey(partition_path, true); - return std::make_shared(context_, base_path_, partition_id, filename_, format_hint_, sample_block_, stats_); + return std::make_shared( + context_, base_path_, partition_id, filename, format_hint_, sample_block_, stats_, empty_delta_stats_); } String getName() const override { return "SubstraitPartitionedFileSink"; } }; diff --git a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h index 2ea197cddaa0..ba9b2a3c473a 100644 --- a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h @@ -20,8 +20,8 @@ #include "config.h" #if USE_ORC -# include -# include +#include +#include namespace local_engine { @@ -33,7 +33,6 @@ class ORCOutputFormatFile : public OutputFormatFile const std::string & file_uri_, WriteBufferBuilderPtr write_buffer_builder_, const DB::Block & preferred_schema_); - ~ORCOutputFormatFile() override = default; OutputFormatFile::OutputFormatPtr createOutputFormat(const DB::Block & header) override; }; diff --git a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h index 7dcffc867aa3..e94923f77a43 100644 --- a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h @@ -16,10 +16,6 @@ */ #pragma once -#include -#include -#include - #include #include #include @@ -48,8 +44,8 @@ class OutputFormatFile virtual ~OutputFormatFile() = default; virtual OutputFormatPtr createOutputFormat(const DB::Block & header_) = 0; - - virtual const DB::Block getPreferredSchema() const { return preferred_schema; } + OutputFormatPtr createOutputFormat() { return createOutputFormat(preferred_schema); } + DB::Block getPreferredSchema() const { return preferred_schema; } protected: DB::Block createHeaderWithPreferredSchema(const DB::Block & header); diff --git a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h index cc87da7da854..37b843a628ec 100644 --- a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h @@ -33,7 +33,6 @@ class ParquetOutputFormatFile : public OutputFormatFile const std::string & file_uri_, const WriteBufferBuilderPtr & write_buffer_builder_, const DB::Block & preferred_schema_); - ~ParquetOutputFormatFile() override = default; OutputFormatFile::OutputFormatPtr createOutputFormat(const DB::Block & header) override; }; diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp index 9f0515d3d384..00f2da20c522 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -152,7 +152,7 @@ TEST(WritePipeline, SubstraitFileSink) EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); - debug::headBlock(x); + std::cerr << debug::verticalShowString(x, 10, 50) << std::endl; EXPECT_EQ(1, x.rows()); const auto & col_a = *(x.getColumns()[0]); EXPECT_EQ(settings.task_write_filename, col_a.getDataAt(0)); @@ -186,7 +186,6 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) const substrait::WriteRel & write_rel = root_rel.root().input().write(); EXPECT_TRUE(write_rel.has_named_table()); - const substrait::NamedObjectWrite & named_table = write_rel.named_table(); EXPECT_TRUE(write_rel.has_table_schema()); const substrait::NamedStruct & table_schema = write_rel.table_schema(); auto block = TypeParser::buildBlockFromNamedStruct(table_schema); @@ -202,17 +201,8 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) const Block & x = *local_executor->nextColumnar(); debug::headBlock(x, 25); EXPECT_EQ(25, x.rows()); - // const auto & col_b = *(x.getColumns()[1]); - // EXPECT_EQ(16, col_b.getInt(0)); } -/*DB::ASTPtr printColumn(const std::string& column) -{ - // printf('%05d',col) - DB::ASTs arguments {std::make_shared("%05d"), std::make_shared(column)}; - return DB::makeASTFunction("printf", std::move(arguments)); -}*/ - TEST(WritePipeline, ComputePartitionedExpression) { const auto context = DB::Context::createCopy(QueryContext::globalContext()); From 57ea686b10789b87aeb55d18e7881eed65179f6c Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 21 Nov 2024 21:15:16 +0800 Subject: [PATCH 133/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_21) (#8012) c286451a4 by rui-mo, feat(function): Add Spark locate function (8863) 473902a63 by aditi-pandit, fix: Fix TopNRowNumber documentation about spilling (11569) c13b8ed88 by Max Ma, fix: Add include cstdint explicitly (11594) b1069008c by Xiaoxuan Meng, feat: Add indexed priority queue for auto writer thread scaling (11584) e2bd12099 by Zuyu ZHANG, misc: Fix typo in dwio/IntDecoder (11575) 7a47fb4e2 by Chengcheng Jin, fix: Fix SortBuffer ensureOutputFits estimateOutputSize inaccurate (11534) f03ac2b85 by Yenda Li, Back out "Back out "[velox] misc(revert ipaddress cast): Add ipprefix cast operators for varchar [1/n]"" (11591) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 53615ddccfbe..e2fb3a12b8cc 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_20 +VELOX_BRANCH=2024_11_21 VELOX_HOME="" OS=`uname -s` From 2063cd237b21f4653959e2c9916d8859f5ffe6b3 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Fri, 22 Nov 2024 16:22:42 +0800 Subject: [PATCH 134/211] [VL] Link shared jemalloc lib to work with LD_PRELOAD (#7369) --- ...jemalloc_pic.cmake => Buildjemalloc.cmake} | 18 ++--- ...djemalloc_pic.cmake => Findjemalloc.cmake} | 19 ++--- cpp/velox/CMakeLists.txt | 6 +- cpp/velox/memory/VeloxMemoryManager.cc | 2 +- .../ports/jemalloc/fix-configure-ac.patch | 13 --- dev/vcpkg/ports/jemalloc/portfile.cmake | 79 ------------------- dev/vcpkg/ports/jemalloc/preprocessor.patch | 12 --- dev/vcpkg/ports/jemalloc/vcpkg.json | 8 -- dev/vcpkg/vcpkg.json | 2 - 9 files changed, 20 insertions(+), 139 deletions(-) rename cpp/CMake/{Buildjemalloc_pic.cmake => Buildjemalloc.cmake} (83%) rename cpp/CMake/{Findjemalloc_pic.cmake => Findjemalloc.cmake} (76%) delete mode 100644 dev/vcpkg/ports/jemalloc/fix-configure-ac.patch delete mode 100644 dev/vcpkg/ports/jemalloc/portfile.cmake delete mode 100644 dev/vcpkg/ports/jemalloc/preprocessor.patch delete mode 100644 dev/vcpkg/ports/jemalloc/vcpkg.json diff --git a/cpp/CMake/Buildjemalloc_pic.cmake b/cpp/CMake/Buildjemalloc.cmake similarity index 83% rename from cpp/CMake/Buildjemalloc_pic.cmake rename to cpp/CMake/Buildjemalloc.cmake index 7c2316ea9540..5491bd829533 100644 --- a/cpp/CMake/Buildjemalloc_pic.cmake +++ b/cpp/CMake/Buildjemalloc.cmake @@ -29,7 +29,7 @@ macro(build_jemalloc) ) endif() - set(JEMALLOC_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/jemalloc_ep-install") + set(JEMALLOC_PREFIX "/usr/local") set(JEMALLOC_LIB_DIR "${JEMALLOC_PREFIX}/lib") set(JEMALLOC_INCLUDE_DIR "${JEMALLOC_PREFIX}/include") set(JEMALLOC_STATIC_LIB @@ -41,10 +41,10 @@ macro(build_jemalloc) "CC=${CMAKE_C_COMPILER}" "--prefix=${JEMALLOC_PREFIX}" "--libdir=${JEMALLOC_LIB_DIR}" - "--with-jemalloc-prefix=je_gluten_" - "--with-private-namespace=je_gluten_private_" - "--without-export" - "--disable-shared" + # The below prefix can be enabled if jemalloc is used in some selective + # code for debugging or profiling. "--with-jemalloc-prefix=je_gluten_" + # "--with-private-namespace=je_gluten_private_" This should be commented + # for dynamically linking. "--without-export" "--disable-cxx" "--disable-libdl" # For fixing an issue when loading native lib: cannot allocate memory in @@ -64,11 +64,11 @@ macro(build_jemalloc) INSTALL_COMMAND make install) file(MAKE_DIRECTORY "${JEMALLOC_INCLUDE_DIR}") - add_library(jemalloc::libjemalloc STATIC IMPORTED) + add_library(jemalloc::jemalloc SHARED IMPORTED) set_target_properties( - jemalloc::libjemalloc + jemalloc::jemalloc PROPERTIES INTERFACE_LINK_LIBRARIES Threads::Threads - IMPORTED_LOCATION "${JEMALLOC_STATIC_LIB}" + IMPORTED_LOCATION "${JEMALLOC_LIB_DIR}/libjemalloc.so" INTERFACE_INCLUDE_DIRECTORIES "${JEMALLOC_INCLUDE_DIR}") - add_dependencies(jemalloc::libjemalloc jemalloc_ep) + add_dependencies(jemalloc::jemalloc jemalloc_ep) endmacro() diff --git a/cpp/CMake/Findjemalloc_pic.cmake b/cpp/CMake/Findjemalloc.cmake similarity index 76% rename from cpp/CMake/Findjemalloc_pic.cmake rename to cpp/CMake/Findjemalloc.cmake index ca7b7d213dfc..aa92fe26df29 100644 --- a/cpp/CMake/Findjemalloc_pic.cmake +++ b/cpp/CMake/Findjemalloc.cmake @@ -15,26 +15,21 @@ # specific language governing permissions and limitations # under the License. -# Find Jemalloc macro(find_jemalloc) - # Find the existing jemalloc - set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") - # Find from vcpkg-installed lib path. - find_library( - JEMALLOC_LIBRARY - NAMES jemalloc_pic - PATHS - ${CMAKE_CURRENT_BINARY_DIR}/../../../dev/vcpkg/vcpkg_installed/x64-linux-avx/lib/ - NO_DEFAULT_PATH) + set(SHARED_LIBRARY_SUFFIX ".so") + set(LIB_NAME "jemalloc") + set(LIB_FULL_NAME + ${CMAKE_SHARED_LIBRARY_PREFIX}${LIB_NAME}${SHARED_LIBRARY_SUFFIX}) + find_library(JEMALLOC_LIBRARY NAMES ${LIB_FULL_NAME}) if("${JEMALLOC_LIBRARY}" STREQUAL "JEMALLOC_LIBRARY-NOTFOUND") message(STATUS "Jemalloc Library Not Found.") set(JEMALLOC_NOT_FOUND TRUE) else() message(STATUS "Found jemalloc: ${JEMALLOC_LIBRARY}") find_path(JEMALLOC_INCLUDE_DIR jemalloc/jemalloc.h) - add_library(jemalloc::libjemalloc STATIC IMPORTED) + add_library(jemalloc::jemalloc SHARED IMPORTED) set_target_properties( - jemalloc::libjemalloc + jemalloc::jemalloc PROPERTIES INTERFACE_INCLUDE_DIRECTORIES "${JEMALLOC_INCLUDE_DIR}" IMPORTED_LOCATION "${JEMALLOC_LIBRARY}") endif() diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 3ed7574d1e4c..58f946f9dabf 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -227,14 +227,14 @@ set_target_properties(velox PROPERTIES LIBRARY_OUTPUT_DIRECTORY find_package(Folly REQUIRED CONFIG) if(ENABLE_JEMALLOC_STATS) - include(Findjemalloc_pic) + include(Findjemalloc) find_jemalloc() if(JEMALLOC_NOT_FOUND) - include(Buildjemalloc_pic) + include(Buildjemalloc) build_jemalloc() endif() add_definitions(-DENABLE_JEMALLOC_STATS) - target_link_libraries(velox PUBLIC jemalloc::libjemalloc) + target_link_libraries(velox PUBLIC jemalloc::jemalloc) endif() target_link_libraries(velox PUBLIC gluten) diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 32978281ef60..63a61ae2ddda 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -393,7 +393,7 @@ VeloxMemoryManager::~VeloxMemoryManager() { << "ms as there are still outstanding memory resources. "; } #ifdef ENABLE_JEMALLOC_STATS - je_gluten_malloc_stats_print(NULL, NULL, NULL); + malloc_stats_print(NULL, NULL, NULL); #endif } diff --git a/dev/vcpkg/ports/jemalloc/fix-configure-ac.patch b/dev/vcpkg/ports/jemalloc/fix-configure-ac.patch deleted file mode 100644 index 7799dfb9e80e..000000000000 --- a/dev/vcpkg/ports/jemalloc/fix-configure-ac.patch +++ /dev/null @@ -1,13 +0,0 @@ -diff --git a/configure.ac b/configure.ac -index f6d25f334..3115504e2 100644 ---- a/configure.ac -+++ b/configure.ac -@@ -1592,7 +1592,7 @@ fi - [enable_uaf_detection="0"] - ) - if test "x$enable_uaf_detection" = "x1" ; then -- AC_DEFINE([JEMALLOC_UAF_DETECTION], [ ]) -+ AC_DEFINE([JEMALLOC_UAF_DETECTION], [ ], ["enable UAF"]) - fi - AC_SUBST([enable_uaf_detection]) - diff --git a/dev/vcpkg/ports/jemalloc/portfile.cmake b/dev/vcpkg/ports/jemalloc/portfile.cmake deleted file mode 100644 index 6cac12ca3b7c..000000000000 --- a/dev/vcpkg/ports/jemalloc/portfile.cmake +++ /dev/null @@ -1,79 +0,0 @@ -vcpkg_from_github( - OUT_SOURCE_PATH SOURCE_PATH - REPO jemalloc/jemalloc - REF 54eaed1d8b56b1aa528be3bdd1877e59c56fa90c - SHA512 527bfbf5db9a5c2b7b04df4785b6ae9d445cff8cb17298bf3e550c88890d2bd7953642d8efaa417580610508279b527d3a3b9e227d17394fd2013c88cb7ae75a - HEAD_REF master - PATCHES - fix-configure-ac.patch - preprocessor.patch -) -if(VCPKG_TARGET_IS_WINDOWS) - set(opts "ac_cv_search_log=none required" - "--without-private-namespace" - "--with-jemalloc-prefix=je_gluten_" - "--with-private-namespace=je_gluten_private_" - "--without-export" - "--disable-shared" - "--disable-cxx" - "--disable-libdl" - # For fixing an issue when loading native lib: cannot allocate memory in static TLS block. - "--disable-initial-exec-tls" - "CFLAGS=-fPIC" - "CXXFLAGS=-fPIC") -else() - set(opts - "--with-jemalloc-prefix=je_gluten_" - "--with-private-namespace=je_gluten_private_" - "--without-export" - "--disable-shared" - "--disable-cxx" - "--disable-libdl" - # For fixing an issue when loading native lib: cannot allocate memory in static TLS block. - "--disable-initial-exec-tls" - "CFLAGS=-fPIC" - "CXXFLAGS=-fPIC") -endif() - -vcpkg_configure_make( - SOURCE_PATH "${SOURCE_PATH}" - AUTOCONFIG - NO_WRAPPERS - OPTIONS ${opts} -) - -vcpkg_install_make() - -if(VCPKG_TARGET_IS_WINDOWS) - file(COPY "${SOURCE_PATH}/include/msvc_compat/strings.h" DESTINATION "${CURRENT_PACKAGES_DIR}/include/jemalloc/msvc_compat") - vcpkg_replace_string("${CURRENT_PACKAGES_DIR}/include/jemalloc/jemalloc.h" "" "\"msvc_compat/strings.h\"") - if(VCPKG_LIBRARY_LINKAGE STREQUAL "dynamic") - file(COPY "${CURRENT_BUILDTREES_DIR}/${TARGET_TRIPLET}-rel/lib/jemalloc.lib" DESTINATION "${CURRENT_PACKAGES_DIR}/lib") - file(MAKE_DIRECTORY "${CURRENT_PACKAGES_DIR}/bin") - file(RENAME "${CURRENT_PACKAGES_DIR}/lib/jemalloc.dll" "${CURRENT_PACKAGES_DIR}/bin/jemalloc.dll") - endif() - if(NOT VCPKG_BUILD_TYPE) - if(VCPKG_LIBRARY_LINKAGE STREQUAL "dynamic") - file(COPY "${CURRENT_BUILDTREES_DIR}/${TARGET_TRIPLET}-dbg/lib/jemalloc.lib" DESTINATION "${CURRENT_PACKAGES_DIR}/debug/lib") - file(MAKE_DIRECTORY "${CURRENT_PACKAGES_DIR}/debug/bin") - file(RENAME "${CURRENT_PACKAGES_DIR}/debug/lib/jemalloc.dll" "${CURRENT_PACKAGES_DIR}/debug/bin/jemalloc.dll") - endif() - endif() - if(VCPKG_LIBRARY_LINKAGE STREQUAL "static") - vcpkg_replace_string("${CURRENT_PACKAGES_DIR}/lib/pkgconfig/jemalloc.pc" "install_suffix=" "install_suffix=_s") - if(NOT VCPKG_BUILD_TYPE) - vcpkg_replace_string("${CURRENT_PACKAGES_DIR}/debug/lib/pkgconfig/jemalloc.pc" "install_suffix=" "install_suffix=_s") - endif() - endif() -endif() - -vcpkg_fixup_pkgconfig() - -vcpkg_copy_pdbs() - -file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/debug/include") -file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/debug/share") -file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/tools") - -# Handle copyright -file(INSTALL "${SOURCE_PATH}/COPYING" DESTINATION "${CURRENT_PACKAGES_DIR}/share/${PORT}" RENAME copyright) diff --git a/dev/vcpkg/ports/jemalloc/preprocessor.patch b/dev/vcpkg/ports/jemalloc/preprocessor.patch deleted file mode 100644 index 6e6e2d1403fb..000000000000 --- a/dev/vcpkg/ports/jemalloc/preprocessor.patch +++ /dev/null @@ -1,12 +0,0 @@ -diff --git a/configure.ac b/configure.ac -index 3115504e2..ffb504b08 100644 ---- a/configure.ac -+++ b/configure.ac -@@ -749,6 +749,7 @@ case "${host}" in - so="dll" - if test "x$je_cv_msvc" = "xyes" ; then - importlib="lib" -+ JE_APPEND_VS(CPPFLAGS, -DJEMALLOC_NO_PRIVATE_NAMESPACE) - DSO_LDFLAGS="-LD" - EXTRA_LDFLAGS="-link -DEBUG" - CTARGET='-Fo$@' diff --git a/dev/vcpkg/ports/jemalloc/vcpkg.json b/dev/vcpkg/ports/jemalloc/vcpkg.json deleted file mode 100644 index 007e05b931c9..000000000000 --- a/dev/vcpkg/ports/jemalloc/vcpkg.json +++ /dev/null @@ -1,8 +0,0 @@ -{ - "name": "jemalloc", - "version": "5.3.0", - "port-version": 1, - "description": "jemalloc is a general purpose malloc(3) implementation that emphasizes fragmentation avoidance and scalable concurrency support", - "homepage": "https://jemalloc.net/", - "license": "BSD-2-Clause" -} diff --git a/dev/vcpkg/vcpkg.json b/dev/vcpkg/vcpkg.json index b3d8dc2fcc0c..66ba246f530a 100644 --- a/dev/vcpkg/vcpkg.json +++ b/dev/vcpkg/vcpkg.json @@ -1,7 +1,6 @@ { "$schema": "https://raw.githubusercontent.com/microsoft/vcpkg-tool/main/docs/vcpkg.schema.json", "builtin-baseline": "a7b6122f6b6504d16d96117336a0562693579933", - "dependencies": ["jemalloc"], "default-features": ["velox"], "features": { "velox": { @@ -50,7 +49,6 @@ "xxhash", "protobuf", "benchmark", - "jemalloc", "icu", "thrift", "libstemmer" From cc0f58840291120ef511634c47ba49ed208268fd Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 22 Nov 2024 20:46:44 +0800 Subject: [PATCH 135/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_22) (#8019) Upstream Velox's New Commits: af3f63853 by Kevin Wilfong, fix: Match Presto's behavior for invalid UTF-8 in url_encode (11518) 7d6cedfa3 by Kevin Wilfong, feat(function): Fix Presto URL functions to more closely match Presto Java's behavior (11488) df3266ce1 by rui-mo, fix: Fix importing long decimal vector from Arrow (11404) ebfb1e56f by Yang Zhang, fix(hdfs): Include the connection failure reason (11586) 44e05dd11 by Orri Erling, FEAT: Add runner for local distributed execution (11609) 1b7570454 by rui-mo, fix: Enable Parquet E2E filter tests on decimal type (11602) 5fb3a9930 by Max Ma, fix: Specify type explicitly for avoiding overload error (11573) fe4f5a77b by Bryan Cutler, fix: Fix Presto Java UUID serialization (11197) 9b3fbcd6d by David Reveman, fix: Add support for sm5x CUDA archs to breeze (11593) 581f3fe42 by Christian Zentgraf, build: The setup-adapters.sh script fails to run on macOS (11592) 0f6ba73b8 by Zac Wen, feat: Use fallocate for file size extension when supported (11541) ef91a1cf7 by Jacob Wujciak-Jens, misc: Clear Codeowners for breeze (11547) 63b4b081e by zuyu, build: Minor cleanup in CMakeLists (11453) de6a83dce by Krishna Pai, feat(functions): Support for canonicalization of JSON (11284) --- cpp/velox/CMakeLists.txt | 3 +++ ep/build-velox/src/build_velox.sh | 2 +- ep/build-velox/src/get_velox.sh | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 58f946f9dabf..eff31863d438 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -265,12 +265,15 @@ if(BUILD_TESTS) import_library( facebook::velox::exec_test_lib ${VELOX_COMPONENTS_PATH}/exec/tests/utils/libvelox_exec_test_lib.a) + import_library(facebook::velox::cursor + ${VELOX_COMPONENTS_PATH}/exec/tests/utils/libvelox_cursor.a) target_link_libraries( facebook::velox::exec_test_lib INTERFACE facebook::velox::vector_test_lib facebook::velox::dwio_common_test facebook::velox::file_test_utils facebook::velox::temp_path) target_link_libraries(velox PUBLIC facebook::velox::exec_test_lib) + target_link_libraries(velox PUBLIC facebook::velox::cursor) endif() target_link_libraries(velox PUBLIC facebook::velox) diff --git a/ep/build-velox/src/build_velox.sh b/ep/build-velox/src/build_velox.sh index b48f28a374b2..18599c0f8fa7 100755 --- a/ep/build-velox/src/build_velox.sh +++ b/ep/build-velox/src/build_velox.sh @@ -97,7 +97,7 @@ function compile { set -exu CXX_FLAGS='-Wno-missing-field-initializers' - COMPILE_OPTION="-DCMAKE_CXX_FLAGS=\"$CXX_FLAGS\" -DVELOX_ENABLE_PARQUET=ON -DVELOX_BUILD_TESTING=OFF -DVELOX_MONO_LIBRARY=ON" + COMPILE_OPTION="-DCMAKE_CXX_FLAGS=\"$CXX_FLAGS\" -DVELOX_ENABLE_PARQUET=ON -DVELOX_BUILD_TESTING=OFF -DVELOX_MONO_LIBRARY=ON -DVELOX_BUILD_RUNNER=OFF" if [ $BUILD_TEST_UTILS == "ON" ]; then COMPILE_OPTION="$COMPILE_OPTION -DVELOX_BUILD_TEST_UTILS=ON" fi diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index e2fb3a12b8cc..e6a69a62f1e9 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_21 +VELOX_BRANCH=2024_11_22 VELOX_HOME="" OS=`uname -s` From 114a1dae8ee6bc2fb51dbf57d26c1a818bf3fa16 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 25 Nov 2024 08:14:03 +0800 Subject: [PATCH 136/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_24) (#8028) Upstream Velox's New Commits: 78d761b8d by Jimmy Lu, fix: Delta update reader lifecycle issue (11639) bf3fba765 by Sergey Pershin, Tighten checks for "bins" elements in width_bucket(x, bins) (11629) 059337fca by Jialiang Tan, feat(hashjoin): Add fast row size estimation for hash probe (11558) 20b5728ff by Jialiang Tan, fix: Fix hash join reclaim bytes accounting (11624) 55046e6b4 by Rong Ma, misc: Add macros for Velox::Expected (11570) 42e398f94 by Deepak Majeti, fix(parquet): Schema Evolution (11595) 102c478cb by Deepak Majeti, docs: Add June 2024 monthly (11616) 462071871 by David Reveman, fix: Clamp values in Breeze unit test to prevent overflow (11618) fca62d11a by Wei He, build(fuzzer): Enable biased expression fuzzer with PQR for single functions at PR time (11581) 9ba0197e7 by Kevin Wilfong, fix: Match Presto's behavior for invalid UTF-8 in url_decode (11604) bfaa9a332 by Kevin Wilfong, fix: Handle escaped separators in Presto url_extract_parameter function (11540) d06bdfab5 by Kevin Wilfong, feat(function): Handle unescaped UTF-8 characters in Presto url_extract_* UDFs (11535) b7c2e6565 by Jiaqi Zhang, Add packet sizes for debugging (11615) 8d91c1cce by Bikramjeet Vig, fix: Optimize subscript and array/map filter in favor of memory (11608) e67fe06cf by Zac Wen, refactor: Use CacheTestUtil for testing functions (11610) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index e6a69a62f1e9..f50e244dffa6 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_22 +VELOX_BRANCH=2024_11_24 VELOX_HOME="" OS=`uname -s` From 58e7d831e7a18d707576a983f2b49d91b0b3d67f Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Mon, 25 Nov 2024 09:19:52 +0800 Subject: [PATCH 137/211] [GLUTEN-7953][VL] Fetch and dump all inputs for micro benchmark on middle stage begin (#7998) --- .github/workflows/velox_backend.yml | 8 +- .../NativeBenchmarkPlanGenerator.scala | 98 ++++++---------- cpp/CMakeLists.txt | 4 - cpp/core/compute/Runtime.h | 3 + cpp/core/jni/JniCommon.cc | 44 ++++--- cpp/core/jni/JniCommon.h | 1 + cpp/core/jni/JniWrapper.cc | 14 ++- cpp/core/memory/ColumnarBatch.h | 1 - cpp/core/operators/writer/ArrowWriter.cc | 9 +- cpp/core/operators/writer/ArrowWriter.h | 16 ++- cpp/velox/CMakeLists.txt | 3 + cpp/velox/benchmarks/CMakeLists.txt | 7 +- cpp/velox/benchmarks/GenericBenchmark.cc | 34 ++---- .../benchmarks/common/OrcReaderIterator.h | 107 ------------------ .../benchmarks/common/ParquetReaderIterator.h | 104 ----------------- cpp/velox/benchmarks/exec/OrcConverter.cc | 2 +- cpp/velox/compute/VeloxRuntime.cc | 14 ++- cpp/velox/compute/VeloxRuntime.h | 2 + .../reader}/FileReaderIterator.cc | 41 ++++--- .../reader}/FileReaderIterator.h | 28 ++--- .../operators/reader/ParquetReaderIterator.cc | 97 ++++++++++++++++ .../operators/reader/ParquetReaderIterator.h | 65 +++++++++++ .../VeloxColumnarBatchSerializer.cc | 3 +- .../operators/writer/VeloxArrowWriter.cc | 38 +++++++ cpp/velox/operators/writer/VeloxArrowWriter.h | 35 ++++++ cpp/velox/tests/RuntimeTest.cc | 4 + .../tests/VeloxColumnarBatchSerializerTest.cc | 3 +- cpp/velox/utils/VeloxArrowUtils.cc | 3 + docs/developers/MicroBenchmarks.md | 14 +-- .../org/apache/gluten/utils/DebugUtil.scala | 38 ++++--- .../org/apache/gluten/GlutenConfig.scala | 21 ++-- 31 files changed, 456 insertions(+), 405 deletions(-) delete mode 100644 cpp/velox/benchmarks/common/OrcReaderIterator.h delete mode 100644 cpp/velox/benchmarks/common/ParquetReaderIterator.h rename cpp/velox/{benchmarks/common => operators/reader}/FileReaderIterator.cc (55%) rename cpp/velox/{benchmarks/common => operators/reader}/FileReaderIterator.h (69%) create mode 100644 cpp/velox/operators/reader/ParquetReaderIterator.cc create mode 100644 cpp/velox/operators/reader/ParquetReaderIterator.h create mode 100644 cpp/velox/operators/writer/VeloxArrowWriter.cc create mode 100644 cpp/velox/operators/writer/VeloxArrowWriter.h diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 42e1f2527816..d7445d1a2752 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -969,9 +969,11 @@ jobs: run: | $MVN_CMD test -Pspark-3.5 -Pbackends-velox -pl backends-velox -am \ -DtagsToInclude="org.apache.gluten.tags.GenerateExample" -Dtest=none -DfailIfNoTests=false -Dexec.skip - # This test depends on example.json generated by the above mvn test. - cd cpp/build/velox/benchmarks && sudo chmod +x ./generic_benchmark - ./generic_benchmark --run-example --with-shuffle --threads 1 --iterations 1 + # This test depends on files generated by the above mvn test. + ./cpp/build/velox/benchmarks/generic_benchmark --with-shuffle --partitioning hash --threads 1 --iterations 1 \ + --conf $(realpath backends-velox/generated-native-benchmark/conf_12_0.ini) \ + --plan $(realpath backends-velox/generated-native-benchmark/plan_12_0.json) \ + --data $(realpath backends-velox/generated-native-benchmark/data_12_0_0.parquet),$(realpath backends-velox/generated-native-benchmark/data_12_0_1.parquet) - name: Run UDF test run: | # Depends on --build_example=ON. diff --git a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala index 57fbca17447a..1e378d16f14b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala @@ -19,18 +19,14 @@ package org.apache.gluten.benchmarks import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.{VeloxWholeStageTransformerSuite, WholeStageTransformer} -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.internal.SQLConf import org.apache.commons.io.FileUtils import org.scalatest.Tag import java.io.File -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Paths} - -import scala.collection.JavaConverters._ object GenerateExample extends Tag("org.apache.gluten.tags.GenerateExample") @@ -50,6 +46,11 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { createTPCHNotNullTables() } + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + } + test("Test plan json non-empty - AQE off") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", @@ -67,7 +68,6 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson assert(planJson.nonEmpty) } - spark.sparkContext.setLogLevel(logLevel) } test("Test plan json non-empty - AQE on") { @@ -88,70 +88,42 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { val planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson assert(planJson.nonEmpty) } - spark.sparkContext.setLogLevel(logLevel) } test("generate example", GenerateExample) { - import testImplicits._ withSQLConf( SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.SHUFFLE_PARTITIONS.key -> "2", - GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT.key -> "true" + GlutenConfig.BENCHMARK_SAVE_DIR.key -> generatedPlanDir, + GlutenConfig.BENCHMARK_TASK_STAGEID.key -> "12", + GlutenConfig.BENCHMARK_TASK_PARTITIONID.key -> "0" ) { logWarning(s"Generating inputs for micro benchmark to $generatedPlanDir") - val q4_lineitem = spark - .sql(s""" - |select l_orderkey from lineitem where l_commitdate < l_receiptdate - |""".stripMargin) - val q4_orders = spark - .sql(s""" - |select o_orderkey, o_orderpriority - | from orders - | where o_orderdate >= '1993-07-01' and o_orderdate < '1993-10-01' - |""".stripMargin) - q4_lineitem - .createOrReplaceTempView("q4_lineitem") - q4_orders - .createOrReplaceTempView("q4_orders") - - q4_lineitem - .repartition(1, 'l_orderkey) - .write - .format(outputFileFormat) - .save(generatedPlanDir + "/example_lineitem") - q4_orders - .repartition(1, 'o_orderkey) - .write - .format(outputFileFormat) - .save(generatedPlanDir + "/example_orders") - - val df = - spark.sql(""" - |select * from q4_orders left semi join q4_lineitem on l_orderkey = o_orderkey - |""".stripMargin) - generateSubstraitJson(df, "example.json") + spark + .sql(""" + |select /*+ REPARTITION(1) */ + | o_orderpriority, + | count(*) as order_count + |from + | orders + |where + | o_orderdate >= date '1993-07-01' + | and o_orderdate < date '1993-07-01' + interval '3' month + | and exists ( + | select /*+ REPARTITION(1) */ + | * + | from + | lineitem + | where + | l_orderkey = o_orderkey + | and l_commitdate < l_receiptdate + | ) + |group by + | o_orderpriority + |order by + | o_orderpriority + |""".stripMargin) + .foreach(_ => ()) } - spark.sparkContext.setLogLevel(logLevel) - } - - def generateSubstraitJson(df: DataFrame, file: String): Unit = { - val executedPlan = df.queryExecution.executedPlan - executedPlan.execute() - val finalPlan = - executedPlan match { - case aqe: AdaptiveSparkPlanExec => - aqe.executedPlan match { - case s: ShuffleQueryStageExec => s.shuffle.child - case other => other - } - case plan => plan - } - val lastStageTransformer = finalPlan.find(_.isInstanceOf[WholeStageTransformer]) - assert(lastStageTransformer.nonEmpty) - val plan = - lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson.split('\n') - - val exampleJsonFile = Paths.get(generatedPlanDir, file) - Files.write(exampleJsonFile, plan.toList.asJava, StandardCharsets.UTF_8) } } diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 09d09481f216..67fb9ec721ac 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -217,10 +217,6 @@ if(ENABLE_IAA) add_definitions(-DGLUTEN_ENABLE_IAA) endif() -if(ENABLE_ORC) - add_definitions(-DGLUTEN_ENABLE_ORC) -endif() - # # Subdirectories # diff --git a/cpp/core/compute/Runtime.h b/cpp/core/compute/Runtime.h index a58f770ff74c..3090652b8148 100644 --- a/cpp/core/compute/Runtime.h +++ b/cpp/core/compute/Runtime.h @@ -27,6 +27,7 @@ #include "operators/c2r/ColumnarToRow.h" #include "operators/r2c/RowToColumnar.h" #include "operators/serializer/ColumnarBatchSerializer.h" +#include "operators/writer/ArrowWriter.h" #include "shuffle/ShuffleReader.h" #include "shuffle/ShuffleWriter.h" #include "substrait/plan.pb.h" @@ -124,6 +125,8 @@ class Runtime : public std::enable_shared_from_this { virtual void dumpConf(const std::string& path) = 0; + virtual std::shared_ptr createArrowWriter(const std::string& path) = 0; + const std::unordered_map& getConfMap() { return confMap_; } diff --git a/cpp/core/jni/JniCommon.cc b/cpp/core/jni/JniCommon.cc index 46b53e8c3b00..bb8554568ba0 100644 --- a/cpp/core/jni/JniCommon.cc +++ b/cpp/core/jni/JniCommon.cc @@ -104,22 +104,34 @@ gluten::JniColumnarBatchIterator::~JniColumnarBatchIterator() { std::shared_ptr gluten::JniColumnarBatchIterator::next() { JNIEnv* env = nullptr; attachCurrentThreadAsDaemonOrThrow(vm_, &env); - if (!env->CallBooleanMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorHasNext_)) { - checkException(env); - return nullptr; // stream ended - } - - checkException(env); - jlong handle = env->CallLongMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorNext_); - checkException(env); - auto batch = ObjectStore::retrieve(handle); if (writer_ != nullptr) { - // save snapshot of the batch to file - std::shared_ptr schema = batch->exportArrowSchema(); - std::shared_ptr array = batch->exportArrowArray(); - auto rb = gluten::arrowGetOrThrow(arrow::ImportRecordBatch(array.get(), schema.get())); - GLUTEN_THROW_NOT_OK(writer_->initWriter(*(rb->schema().get()))); - GLUTEN_THROW_NOT_OK(writer_->writeInBatches(rb)); + if (!writer_->closed()) { + // Dump all inputs. + while (env->CallBooleanMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorHasNext_)) { + checkException(env); + jlong handle = env->CallLongMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorNext_); + checkException(env); + auto batch = ObjectStore::retrieve(handle); + + // Save the snapshot of the batch to file. + std::shared_ptr schema = batch->exportArrowSchema(); + std::shared_ptr array = batch->exportArrowArray(); + auto rb = gluten::arrowGetOrThrow(arrow::ImportRecordBatch(array.get(), schema.get())); + GLUTEN_THROW_NOT_OK(writer_->initWriter(*(rb->schema().get()))); + GLUTEN_THROW_NOT_OK(writer_->writeInBatches(rb)); + } + checkException(env); + GLUTEN_THROW_NOT_OK(writer_->closeWriter()); + } + return writer_->retrieveColumnarBatch(); + } else { + if (!env->CallBooleanMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorHasNext_)) { + checkException(env); + return nullptr; // stream ended + } + checkException(env); + jlong handle = env->CallLongMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorNext_); + checkException(env); + return ObjectStore::retrieve(handle); } - return batch; } diff --git a/cpp/core/jni/JniCommon.h b/cpp/core/jni/JniCommon.h index aeab454f1aab..8f40398a4132 100644 --- a/cpp/core/jni/JniCommon.h +++ b/cpp/core/jni/JniCommon.h @@ -26,6 +26,7 @@ #include "compute/Runtime.h" #include "config/GlutenConfig.h" #include "memory/AllocationListener.h" +#include "operators/writer/ArrowWriter.h" #include "shuffle/rss/RssClient.h" #include "utils/Compression.h" #include "utils/Exception.h" diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 9da55894864e..963440f6fc16 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -373,8 +373,16 @@ Java_org_apache_gluten_vectorized_PlanEvaluatorJniWrapper_nativeCreateKernelWith } saveDir = conf.at(kGlutenSaveDir); std::filesystem::path f{saveDir}; - if (!std::filesystem::exists(f)) { - throw GlutenException("Save input path " + saveDir + " does not exists"); + if (std::filesystem::exists(f)) { + if (!std::filesystem::is_directory(f)) { + throw GlutenException("Invalid path for " + kGlutenSaveDir + ": " + saveDir); + } + } else { + std::error_code ec; + std::filesystem::create_directory(f, ec); + if (ec) { + throw GlutenException("Failed to create directory: " + saveDir + ", error message: " + ec.message()); + } } ctx->dumpConf(saveDir + "/conf" + fileIdentifier + ".ini"); } @@ -407,7 +415,7 @@ Java_org_apache_gluten_vectorized_PlanEvaluatorJniWrapper_nativeCreateKernelWith std::shared_ptr writer = nullptr; if (saveInput) { auto file = saveDir + "/data" + fileIdentifier + "_" + std::to_string(idx) + ".parquet"; - writer = std::make_shared(file); + writer = ctx->createArrowWriter(file); } jobject iter = env->GetObjectArrayElement(iterArr, idx); auto arrayIter = makeJniColumnarBatchIterator(env, iter, ctx, writer); diff --git a/cpp/core/memory/ColumnarBatch.h b/cpp/core/memory/ColumnarBatch.h index e0bab254189a..be487f871e74 100644 --- a/cpp/core/memory/ColumnarBatch.h +++ b/cpp/core/memory/ColumnarBatch.h @@ -23,7 +23,6 @@ #include "arrow/c/helpers.h" #include "arrow/record_batch.h" #include "memory/MemoryManager.h" -#include "operators/writer/ArrowWriter.h" #include "utils/ArrowStatus.h" #include "utils/Exception.h" diff --git a/cpp/core/operators/writer/ArrowWriter.cc b/cpp/core/operators/writer/ArrowWriter.cc index 19bab6ddcba3..46ec2fc9ba6c 100644 --- a/cpp/core/operators/writer/ArrowWriter.cc +++ b/cpp/core/operators/writer/ArrowWriter.cc @@ -21,6 +21,7 @@ #include "arrow/table.h" #include "arrow/util/type_fwd.h" +namespace gluten { arrow::Status ArrowWriter::initWriter(arrow::Schema& schema) { if (writer_ != nullptr) { return arrow::Status::OK(); @@ -50,9 +51,15 @@ arrow::Status ArrowWriter::writeInBatches(std::shared_ptr ba } arrow::Status ArrowWriter::closeWriter() { - // Write file footer and close + // Write file footer and close. if (writer_ != nullptr) { ARROW_RETURN_NOT_OK(writer_->Close()); } + closed_ = true; return arrow::Status::OK(); } + +bool ArrowWriter::closed() const { + return closed_; +} +} // namespace gluten diff --git a/cpp/core/operators/writer/ArrowWriter.h b/cpp/core/operators/writer/ArrowWriter.h index 0d0b8ce2cb58..1a7b19606624 100644 --- a/cpp/core/operators/writer/ArrowWriter.h +++ b/cpp/core/operators/writer/ArrowWriter.h @@ -17,15 +17,19 @@ #pragma once -#include "parquet/arrow/writer.h" +#include +#include "memory/ColumnarBatch.h" +namespace gluten { /** * @brief Used to print RecordBatch to a parquet file * */ class ArrowWriter { public: - explicit ArrowWriter(std::string& path) : path_(path) {} + explicit ArrowWriter(const std::string& path) : path_(path) {} + + virtual ~ArrowWriter() = default; arrow::Status initWriter(arrow::Schema& schema); @@ -33,7 +37,13 @@ class ArrowWriter { arrow::Status closeWriter(); - private: + bool closed() const; + + virtual std::shared_ptr retrieveColumnarBatch() = 0; + + protected: std::unique_ptr writer_; std::string path_; + bool closed_{false}; }; +} // namespace gluten diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index eff31863d438..9e110853eb6e 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -169,9 +169,12 @@ set(VELOX_SRCS operators/functions/RegistrationAllFunctions.cc operators/functions/RowConstructorWithNull.cc operators/functions/SparkExprToSubfieldFilterParser.cc + operators/reader/FileReaderIterator.cc + operators/reader/ParquetReaderIterator.cc operators/serializer/VeloxColumnarToRowConverter.cc operators/serializer/VeloxColumnarBatchSerializer.cc operators/serializer/VeloxRowToColumnarConverter.cc + operators/writer/VeloxArrowWriter.cc operators/writer/VeloxParquetDataSource.cc shuffle/VeloxShuffleReader.cc shuffle/VeloxShuffleWriter.cc diff --git a/cpp/velox/benchmarks/CMakeLists.txt b/cpp/velox/benchmarks/CMakeLists.txt index 1aa199b13696..6b2cda358c06 100644 --- a/cpp/velox/benchmarks/CMakeLists.txt +++ b/cpp/velox/benchmarks/CMakeLists.txt @@ -15,8 +15,7 @@ find_arrow_lib(${PARQUET_LIB_NAME}) -set(VELOX_BENCHMARK_COMMON_SRCS common/FileReaderIterator.cc - common/BenchmarkUtils.cc) +set(VELOX_BENCHMARK_COMMON_SRCS common/BenchmarkUtils.cc) add_library(velox_benchmark_common STATIC ${VELOX_BENCHMARK_COMMON_SRCS}) target_include_directories( velox_benchmark_common PUBLIC ${CMAKE_SOURCE_DIR}/velox @@ -38,7 +37,3 @@ add_velox_benchmark(columnar_to_row_benchmark ColumnarToRowBenchmark.cc) add_velox_benchmark(parquet_write_benchmark ParquetWriteBenchmark.cc) add_velox_benchmark(plan_validator_util PlanValidatorUtil.cc) - -if(ENABLE_ORC) - add_velox_benchmark(orc_converter exec/OrcConverter.cc) -endif() diff --git a/cpp/velox/benchmarks/GenericBenchmark.cc b/cpp/velox/benchmarks/GenericBenchmark.cc index 4e38fb44327f..dcb64d7d18d0 100644 --- a/cpp/velox/benchmarks/GenericBenchmark.cc +++ b/cpp/velox/benchmarks/GenericBenchmark.cc @@ -25,12 +25,12 @@ #include #include "benchmarks/common/BenchmarkUtils.h" -#include "benchmarks/common/FileReaderIterator.h" #include "compute/VeloxBackend.h" -#include "compute/VeloxPlanConverter.h" #include "compute/VeloxRuntime.h" #include "config/GlutenConfig.h" #include "config/VeloxConfig.h" +#include "operators/reader/FileReaderIterator.h" +#include "operators/writer/VeloxArrowWriter.h" #include "shuffle/LocalPartitionWriter.h" #include "shuffle/VeloxShuffleWriter.h" #include "shuffle/rss/RssPartitionWriter.h" @@ -45,7 +45,6 @@ using namespace gluten; namespace { -DEFINE_bool(run_example, false, "Run the example and exit."); DEFINE_bool(print_result, true, "Print result for execution"); DEFINE_string(save_output, "", "Path to parquet file for saving the task output iterator"); DEFINE_bool(with_shuffle, false, "Add shuffle split at end."); @@ -388,7 +387,8 @@ auto BM_Generic = [](::benchmark::State& state, std::vector inputItersRaw; if (!dataFiles.empty()) { for (const auto& input : dataFiles) { - inputIters.push_back(getInputIteratorFromFileReader(input, readerType)); + inputIters.push_back(FileReaderIterator::getInputIteratorFromFileReader( + readerType, input, FLAGS_batch_size, runtime->memoryManager()->getLeafMemoryPool().get())); } std::transform( inputIters.begin(), @@ -417,10 +417,11 @@ auto BM_Generic = [](::benchmark::State& state, ArrowSchema cSchema; toArrowSchema(veloxPlan->outputType(), runtime->memoryManager()->getLeafMemoryPool().get(), &cSchema); GLUTEN_ASSIGN_OR_THROW(auto outputSchema, arrow::ImportSchema(&cSchema)); - ArrowWriter writer{FLAGS_save_output}; + auto writer = std::make_shared( + FLAGS_save_output, FLAGS_batch_size, runtime->memoryManager()->getLeafMemoryPool().get()); state.PauseTiming(); if (!FLAGS_save_output.empty()) { - GLUTEN_THROW_NOT_OK(writer.initWriter(*(outputSchema.get()))); + GLUTEN_THROW_NOT_OK(writer->initWriter(*(outputSchema.get()))); } state.ResumeTiming(); @@ -436,13 +437,13 @@ auto BM_Generic = [](::benchmark::State& state, LOG(WARNING) << maybeBatch.ValueOrDie()->ToString(); } if (!FLAGS_save_output.empty()) { - GLUTEN_THROW_NOT_OK(writer.writeInBatches(maybeBatch.ValueOrDie())); + GLUTEN_THROW_NOT_OK(writer->writeInBatches(maybeBatch.ValueOrDie())); } } state.PauseTiming(); if (!FLAGS_save_output.empty()) { - GLUTEN_THROW_NOT_OK(writer.closeWriter()); + GLUTEN_THROW_NOT_OK(writer->closeWriter()); } state.ResumeTiming(); } @@ -488,7 +489,8 @@ auto BM_ShuffleWriteRead = [](::benchmark::State& state, { ScopedTimer timer(&elapsedTime); for (auto _ : state) { - auto resultIter = getInputIteratorFromFileReader(inputFile, readerType); + auto resultIter = FileReaderIterator::getInputIteratorFromFileReader( + readerType, inputFile, FLAGS_batch_size, runtime->memoryManager()->getLeafMemoryPool().get()); runShuffle( runtime, listenerPtr, @@ -591,19 +593,7 @@ int main(int argc, char** argv) { std::vector splitFiles{}; std::vector dataFiles{}; - if (FLAGS_run_example) { - LOG(WARNING) << "Running example..."; - dataFiles.resize(2); - try { - substraitJsonFile = getGeneratedFilePath("example.json"); - dataFiles[0] = getGeneratedFilePath("example_orders"); - dataFiles[1] = getGeneratedFilePath("example_lineitem"); - } catch (const std::exception& e) { - LOG(ERROR) << "Failed to run example. " << e.what(); - ::benchmark::Shutdown(); - std::exit(EXIT_FAILURE); - } - } else if (FLAGS_run_shuffle) { + if (FLAGS_run_shuffle) { std::string errorMsg{}; if (FLAGS_data.empty()) { errorMsg = "Missing '--split' or '--data' option."; diff --git a/cpp/velox/benchmarks/common/OrcReaderIterator.h b/cpp/velox/benchmarks/common/OrcReaderIterator.h deleted file mode 100644 index f8c9f44b2008..000000000000 --- a/cpp/velox/benchmarks/common/OrcReaderIterator.h +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#pragma once - -#include -#include "benchmarks/common/FileReaderIterator.h" - -namespace gluten { - -class OrcReaderIterator : public FileReaderIterator { - public: - explicit OrcReaderIterator(const std::string& path) : FileReaderIterator(path) {} - - void createReader() override { - // Open File - auto input = arrow::io::ReadableFile::Open(path_); - GLUTEN_THROW_NOT_OK(input); - - // Open ORC File Reader - auto maybeReader = arrow::adapters::orc::ORCFileReader::Open(*input, arrow::default_memory_pool()); - GLUTEN_THROW_NOT_OK(maybeReader); - fileReader_.reset((*maybeReader).release()); - - // get record batch Reader - auto recordBatchReader = fileReader_->GetRecordBatchReader(4096, std::vector()); - GLUTEN_THROW_NOT_OK(recordBatchReader); - recordBatchReader_ = *recordBatchReader; - } - - std::shared_ptr getSchema() override { - auto schema = fileReader_->ReadSchema(); - GLUTEN_THROW_NOT_OK(schema); - return *schema; - } - - protected: - std::unique_ptr fileReader_; - std::shared_ptr recordBatchReader_; -}; - -class OrcStreamReaderIterator final : public OrcReaderIterator { - public: - explicit OrcStreamReaderIterator(const std::string& path) : OrcReaderIterator(path) { - createReader(); - } - - std::shared_ptr next() override { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); - DLOG(INFO) << "OrcStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0); - collectBatchTime_ += - std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); - if (batch == nullptr) { - return nullptr; - } - return convertBatch(std::make_shared(batch)); - } -}; - -class OrcBufferedReaderIterator final : public OrcReaderIterator { - public: - explicit OrcBufferedReaderIterator(const std::string& path) : OrcReaderIterator(path) { - createReader(); - collectBatches(); - iter_ = batches_.begin(); - DLOG(INFO) << "OrcBufferedReaderIterator open file: " << path; - DLOG(INFO) << "Number of input batches: " << std::to_string(batches_.size()); - if (iter_ != batches_.cend()) { - DLOG(INFO) << "columns: " << (*iter_)->num_columns(); - DLOG(INFO) << "rows: " << (*iter_)->num_rows(); - } - } - - std::shared_ptr next() override { - if (iter_ == batches_.cend()) { - return nullptr; - } - return convertBatch(std::make_shared(*iter_++)); - } - - private: - void collectBatches() { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(batches_, recordBatchReader_->ToRecordBatches()); - auto endTime = std::chrono::steady_clock::now(); - collectBatchTime_ += std::chrono::duration_cast(endTime - startTime).count(); - } - - arrow::RecordBatchVector batches_; - std::vector>::const_iterator iter_; -}; - -} // namespace gluten \ No newline at end of file diff --git a/cpp/velox/benchmarks/common/ParquetReaderIterator.h b/cpp/velox/benchmarks/common/ParquetReaderIterator.h deleted file mode 100644 index 20652ee27dc0..000000000000 --- a/cpp/velox/benchmarks/common/ParquetReaderIterator.h +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include "benchmarks/common/BenchmarkUtils.h" -#include "benchmarks/common/FileReaderIterator.h" -#include "utils/Macros.h" - -#include - -namespace gluten { - -class ParquetReaderIterator : public FileReaderIterator { - public: - explicit ParquetReaderIterator(const std::string& path) : FileReaderIterator(path) {} - - void createReader() { - parquet::ArrowReaderProperties properties = parquet::default_arrow_reader_properties(); - properties.set_batch_size(FLAGS_batch_size); - GLUTEN_THROW_NOT_OK(parquet::arrow::FileReader::Make( - arrow::default_memory_pool(), parquet::ParquetFileReader::OpenFile(path_), properties, &fileReader_)); - GLUTEN_THROW_NOT_OK( - fileReader_->GetRecordBatchReader(arrow::internal::Iota(fileReader_->num_row_groups()), &recordBatchReader_)); - - auto schema = recordBatchReader_->schema(); - LOG(INFO) << "schema:\n" << schema->ToString(); - } - - std::shared_ptr getSchema() override { - return recordBatchReader_->schema(); - } - - protected: - std::unique_ptr fileReader_; - std::shared_ptr recordBatchReader_; -}; - -class ParquetStreamReaderIterator final : public ParquetReaderIterator { - public: - explicit ParquetStreamReaderIterator(const std::string& path) : ParquetReaderIterator(path) { - createReader(); - DLOG(INFO) << "ParquetStreamReaderIterator open file: " << path; - } - - std::shared_ptr next() override { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); - DLOG(INFO) << "ParquetStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0); - collectBatchTime_ += - std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); - if (batch == nullptr) { - return nullptr; - } - return convertBatch(std::make_shared(batch)); - } -}; - -class ParquetBufferedReaderIterator final : public ParquetReaderIterator { - public: - explicit ParquetBufferedReaderIterator(const std::string& path) : ParquetReaderIterator(path) { - createReader(); - collectBatches(); - iter_ = batches_.begin(); - DLOG(INFO) << "ParquetBufferedReaderIterator open file: " << path; - DLOG(INFO) << "Number of input batches: " << std::to_string(batches_.size()); - if (iter_ != batches_.cend()) { - DLOG(INFO) << "columns: " << (*iter_)->num_columns(); - DLOG(INFO) << "rows: " << (*iter_)->num_rows(); - } - } - - std::shared_ptr next() override { - if (iter_ == batches_.cend()) { - return nullptr; - } - return convertBatch(std::make_shared(*iter_++)); - } - - private: - void collectBatches() { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(batches_, recordBatchReader_->ToRecordBatches()); - auto endTime = std::chrono::steady_clock::now(); - collectBatchTime_ += std::chrono::duration_cast(endTime - startTime).count(); - } - - arrow::RecordBatchVector batches_; - std::vector>::const_iterator iter_; -}; - -} // namespace gluten diff --git a/cpp/velox/benchmarks/exec/OrcConverter.cc b/cpp/velox/benchmarks/exec/OrcConverter.cc index b421ecca3b37..888cf27c35fe 100644 --- a/cpp/velox/benchmarks/exec/OrcConverter.cc +++ b/cpp/velox/benchmarks/exec/OrcConverter.cc @@ -16,7 +16,7 @@ */ #include -#include "benchmarks/common/ParquetReaderIterator.h" +#include "operators/reader/ParquetReaderIterator.h" namespace gluten { diff --git a/cpp/velox/compute/VeloxRuntime.cc b/cpp/velox/compute/VeloxRuntime.cc index 4c6b52e6fe04..20c3dec939a0 100644 --- a/cpp/velox/compute/VeloxRuntime.cc +++ b/cpp/velox/compute/VeloxRuntime.cc @@ -28,16 +28,14 @@ #include "compute/VeloxPlanConverter.h" #include "config/VeloxConfig.h" #include "operators/serializer/VeloxRowToColumnarConverter.h" -#include "shuffle/VeloxHashShuffleWriter.h" -#include "shuffle/VeloxRssSortShuffleWriter.h" +#include "operators/writer/VeloxArrowWriter.h" #include "shuffle/VeloxShuffleReader.h" +#include "shuffle/VeloxShuffleWriter.h" #include "utils/ConfigExtractor.h" #include "utils/VeloxArrowUtils.h" #ifdef ENABLE_HDFS - #include "operators/writer/VeloxParquetDataSourceHDFS.h" - #endif #ifdef ENABLE_S3 @@ -308,4 +306,12 @@ void VeloxRuntime::dumpConf(const std::string& path) { outFile.close(); } +std::shared_ptr VeloxRuntime::createArrowWriter(const std::string& path) { + int64_t batchSize = 4096; + if (auto it = confMap_.find(kSparkBatchSize); it != confMap_.end()) { + batchSize = std::atol(it->second.c_str()); + } + return std::make_shared(path, batchSize, memoryManager()->getLeafMemoryPool().get()); +} + } // namespace gluten diff --git a/cpp/velox/compute/VeloxRuntime.h b/cpp/velox/compute/VeloxRuntime.h index 846f740cb84c..798fa5bc72f7 100644 --- a/cpp/velox/compute/VeloxRuntime.h +++ b/cpp/velox/compute/VeloxRuntime.h @@ -76,6 +76,8 @@ class VeloxRuntime final : public Runtime { void dumpConf(const std::string& path) override; + std::shared_ptr createArrowWriter(const std::string& path) override; + std::shared_ptr createDataSource(const std::string& filePath, std::shared_ptr schema); std::shared_ptr getVeloxPlan() { diff --git a/cpp/velox/benchmarks/common/FileReaderIterator.cc b/cpp/velox/operators/reader/FileReaderIterator.cc similarity index 55% rename from cpp/velox/benchmarks/common/FileReaderIterator.cc rename to cpp/velox/operators/reader/FileReaderIterator.cc index 26985c7f03c4..d732adbf33c0 100644 --- a/cpp/velox/benchmarks/common/FileReaderIterator.cc +++ b/cpp/velox/operators/reader/FileReaderIterator.cc @@ -15,33 +15,38 @@ * limitations under the License. */ -#include "FileReaderIterator.h" -#include "benchmarks/common/ParquetReaderIterator.h" -#ifdef GLUTEN_ENABLE_ORC -#include "benchmarks/common/OrcReaderIterator.h" -#endif +#include "operators/reader/FileReaderIterator.h" +#include +#include "operators/reader/ParquetReaderIterator.h" -std::shared_ptr gluten::getInputIteratorFromFileReader( +namespace gluten { +namespace { +const std::string kParquetSuffix = ".parquet"; +} + +FileReaderIterator::FileReaderIterator(const std::string& path) : path_(path) {} + +int64_t FileReaderIterator::getCollectBatchTime() const { + return collectBatchTime_; +} + +std::shared_ptr FileReaderIterator::getInputIteratorFromFileReader( + FileReaderType readerType, const std::string& path, - gluten::FileReaderType readerType) { + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) { std::filesystem::path input{path}; auto suffix = input.extension().string(); if (suffix == kParquetSuffix) { if (readerType == FileReaderType::kStream) { - return std::make_shared(std::make_unique(path)); - } - if (readerType == FileReaderType::kBuffered) { - return std::make_shared(std::make_unique(path)); - } - } else if (suffix == kOrcSuffix) { -#ifdef GLUTEN_ENABLE_ORC - if (readerType == FileReaderType::kStream) { - return std::make_shared(std::make_unique(path)); + return std::make_shared( + std::make_unique(path, batchSize, pool)); } if (readerType == FileReaderType::kBuffered) { - return std::make_shared(std::make_unique(path)); + return std::make_shared( + std::make_unique(path, batchSize, pool)); } -#endif } throw new GlutenException("Unreachable."); } +} // namespace gluten diff --git a/cpp/velox/benchmarks/common/FileReaderIterator.h b/cpp/velox/operators/reader/FileReaderIterator.h similarity index 69% rename from cpp/velox/benchmarks/common/FileReaderIterator.h rename to cpp/velox/operators/reader/FileReaderIterator.h index 16db58ce4569..e782c2bf80f4 100644 --- a/cpp/velox/benchmarks/common/FileReaderIterator.h +++ b/cpp/velox/operators/reader/FileReaderIterator.h @@ -14,43 +14,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#pragma once -#include -#include -#include +#pragma once -#include "BenchmarkUtils.h" #include "compute/ResultIterator.h" -#include "memory/ColumnarBatch.h" #include "memory/ColumnarBatchIterator.h" +#include "velox/common/memory/MemoryPool.h" namespace gluten { - -static const std::string kOrcSuffix = ".orc"; -static const std::string kParquetSuffix = ".parquet"; - enum FileReaderType { kBuffered, kStream, kNone }; class FileReaderIterator : public ColumnarBatchIterator { public: - explicit FileReaderIterator(const std::string& path) : path_(path) {} + static std::shared_ptr getInputIteratorFromFileReader( + FileReaderType readerType, + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool); + + explicit FileReaderIterator(const std::string& path); virtual ~FileReaderIterator() = default; virtual std::shared_ptr getSchema() = 0; - int64_t getCollectBatchTime() const { - return collectBatchTime_; - } + int64_t getCollectBatchTime() const; protected: int64_t collectBatchTime_ = 0; std::string path_; }; -std::shared_ptr getInputIteratorFromFileReader( - const std::string& path, - FileReaderType readerType); - } // namespace gluten diff --git a/cpp/velox/operators/reader/ParquetReaderIterator.cc b/cpp/velox/operators/reader/ParquetReaderIterator.cc new file mode 100644 index 000000000000..3e61e1d8d936 --- /dev/null +++ b/cpp/velox/operators/reader/ParquetReaderIterator.cc @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "operators/reader/ParquetReaderIterator.h" +#include "memory/VeloxColumnarBatch.h" + +#include + +namespace gluten { + +ParquetReaderIterator::ParquetReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : FileReaderIterator(path), batchSize_(batchSize), pool_(pool) {} + +void ParquetReaderIterator::createReader() { + parquet::ArrowReaderProperties properties = parquet::default_arrow_reader_properties(); + properties.set_batch_size(batchSize_); + GLUTEN_THROW_NOT_OK(parquet::arrow::FileReader::Make( + arrow::default_memory_pool(), parquet::ParquetFileReader::OpenFile(path_), properties, &fileReader_)); + GLUTEN_THROW_NOT_OK( + fileReader_->GetRecordBatchReader(arrow::internal::Iota(fileReader_->num_row_groups()), &recordBatchReader_)); + + auto schema = recordBatchReader_->schema(); + DLOG(INFO) << "Schema:\n" << schema->ToString(); +} + +std::shared_ptr ParquetReaderIterator::getSchema() { + return recordBatchReader_->schema(); +} + +ParquetStreamReaderIterator::ParquetStreamReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : ParquetReaderIterator(path, batchSize, pool) { + createReader(); + DLOG(INFO) << "ParquetStreamReaderIterator open file: " << path; +} + +std::shared_ptr ParquetStreamReaderIterator::next() { + auto startTime = std::chrono::steady_clock::now(); + GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); + DLOG(INFO) << "ParquetStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0); + collectBatchTime_ += + std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); + if (batch == nullptr) { + return nullptr; + } + return VeloxColumnarBatch::from(pool_, std::make_shared(batch)); +} + +ParquetBufferedReaderIterator::ParquetBufferedReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : ParquetReaderIterator(path, batchSize, pool) { + createReader(); + collectBatches(); + iter_ = batches_.begin(); + DLOG(INFO) << "ParquetBufferedReaderIterator open file: " << path; + DLOG(INFO) << "Number of input batches: " << std::to_string(batches_.size()); + if (iter_ != batches_.cend()) { + DLOG(INFO) << "columns: " << (*iter_)->num_columns(); + DLOG(INFO) << "rows: " << (*iter_)->num_rows(); + } +} + +std::shared_ptr ParquetBufferedReaderIterator::next() { + if (iter_ == batches_.cend()) { + return nullptr; + } + return VeloxColumnarBatch::from(pool_, std::make_shared(*iter_++)); +} + +void ParquetBufferedReaderIterator::collectBatches() { + auto startTime = std::chrono::steady_clock::now(); + GLUTEN_ASSIGN_OR_THROW(batches_, recordBatchReader_->ToRecordBatches()); + auto endTime = std::chrono::steady_clock::now(); + collectBatchTime_ += std::chrono::duration_cast(endTime - startTime).count(); +} +} // namespace gluten \ No newline at end of file diff --git a/cpp/velox/operators/reader/ParquetReaderIterator.h b/cpp/velox/operators/reader/ParquetReaderIterator.h new file mode 100644 index 000000000000..f45fe5eb7721 --- /dev/null +++ b/cpp/velox/operators/reader/ParquetReaderIterator.h @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "operators/reader/FileReaderIterator.h" + +#include +#include + +namespace gluten { + +class ParquetReaderIterator : public FileReaderIterator { + public: + explicit ParquetReaderIterator(const std::string& path, int64_t batchSize, facebook::velox::memory::MemoryPool* pool); + + void createReader(); + + std::shared_ptr getSchema() override; + + protected: + std::unique_ptr<::parquet::arrow::FileReader> fileReader_; + std::shared_ptr recordBatchReader_; + int64_t batchSize_; + facebook::velox::memory::MemoryPool* pool_; +}; + +class ParquetStreamReaderIterator final : public ParquetReaderIterator { + public: + ParquetStreamReaderIterator(const std::string& path, int64_t batchSize, facebook::velox::memory::MemoryPool* pool); + + std::shared_ptr next() override; +}; + +class ParquetBufferedReaderIterator final : public ParquetReaderIterator { + public: + explicit ParquetBufferedReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool); + + std::shared_ptr next() override; + + private: + void collectBatches(); + + arrow::RecordBatchVector batches_; + std::vector>::const_iterator iter_; +}; + +} // namespace gluten diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc index acb14cf4de39..9c5d166a07da 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc @@ -17,10 +17,11 @@ #include "VeloxColumnarBatchSerializer.h" +#include + #include "memory/ArrowMemory.h" #include "memory/VeloxColumnarBatch.h" #include "velox/common/memory/Memory.h" -#include "velox/vector/ComplexVector.h" #include "velox/vector/FlatVector.h" #include "velox/vector/arrow/Bridge.h" diff --git a/cpp/velox/operators/writer/VeloxArrowWriter.cc b/cpp/velox/operators/writer/VeloxArrowWriter.cc new file mode 100644 index 000000000000..565602d95cc9 --- /dev/null +++ b/cpp/velox/operators/writer/VeloxArrowWriter.cc @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "operators/writer/VeloxArrowWriter.h" + +namespace gluten { + +VeloxArrowWriter::VeloxArrowWriter( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : ArrowWriter(path), batchSize_(batchSize), pool_(pool) {} + +std::shared_ptr VeloxArrowWriter::retrieveColumnarBatch() { + if (writer_ == nullptr) { + // No data to read. + return nullptr; + } + if (reader_ == nullptr) { + reader_ = std::make_unique(path_, batchSize_, pool_); + } + return reader_->next(); +} +} // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxArrowWriter.h b/cpp/velox/operators/writer/VeloxArrowWriter.h new file mode 100644 index 000000000000..8b7998628716 --- /dev/null +++ b/cpp/velox/operators/writer/VeloxArrowWriter.h @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "operators/reader/ParquetReaderIterator.h" +#include "operators/writer/ArrowWriter.h" + +namespace gluten { +class VeloxArrowWriter : public ArrowWriter { + public: + explicit VeloxArrowWriter(const std::string& path, int64_t batchSize, facebook::velox::memory::MemoryPool* pool); + + std::shared_ptr retrieveColumnarBatch() override; + + private: + int64_t batchSize_; + facebook::velox::memory::MemoryPool* pool_; + std::unique_ptr reader_{nullptr}; +}; +} // namespace gluten diff --git a/cpp/velox/tests/RuntimeTest.cc b/cpp/velox/tests/RuntimeTest.cc index 4b5bcdf81919..b4944d92054b 100644 --- a/cpp/velox/tests/RuntimeTest.cc +++ b/cpp/velox/tests/RuntimeTest.cc @@ -103,6 +103,10 @@ class DummyRuntime final : public Runtime { throw GlutenException("Not yet implemented"); } + std::shared_ptr createArrowWriter(const std::string& path) override { + throw GlutenException("Not yet implemented"); + } + private: class DummyResultIterator : public ColumnarBatchIterator { public: diff --git a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc index ffa6f032ac44..c00ab6a14844 100644 --- a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc @@ -19,12 +19,13 @@ #include "memory/ArrowMemoryPool.h" #include "memory/VeloxColumnarBatch.h" -#include "memory/VeloxMemoryManager.h" #include "operators/serializer/VeloxColumnarBatchSerializer.h" #include "utils/VeloxArrowUtils.h" #include "velox/vector/arrow/Bridge.h" #include "velox/vector/tests/utils/VectorTestBase.h" +#include + using namespace facebook::velox; namespace gluten { diff --git a/cpp/velox/utils/VeloxArrowUtils.cc b/cpp/velox/utils/VeloxArrowUtils.cc index 0349eb718b8e..f26b49a4768d 100644 --- a/cpp/velox/utils/VeloxArrowUtils.cc +++ b/cpp/velox/utils/VeloxArrowUtils.cc @@ -16,6 +16,9 @@ */ #include "utils/VeloxArrowUtils.h" + +#include + #include "memory/VeloxColumnarBatch.h" #include "utils/Common.h" #include "velox/vector/ComplexVector.h" diff --git a/docs/developers/MicroBenchmarks.md b/docs/developers/MicroBenchmarks.md index eedf5010b634..1483dc2cbaff 100644 --- a/docs/developers/MicroBenchmarks.md +++ b/docs/developers/MicroBenchmarks.md @@ -64,7 +64,7 @@ cd /path/to/gluten/cpp/build/velox/benchmarks --plan /home/sparkuser/github/apache/incubator-gluten/backends-velox/generated-native-benchmark/example.json \ --data /home/sparkuser/github/apache/incubator-gluten/backends-velox/generated-native-benchmark/example_orders/part-00000-1e66fb98-4dd6-47a6-8679-8625dbc437ee-c000.snappy.parquet,\ /home/sparkuser/github/apache/incubator-gluten/backends-velox/generated-native-benchmark/example_lineitem/part-00000-3ec19189-d20e-4240-85ae-88631d46b612-c000.snappy.parquet \ ---threads 1 --iterations 1 --noprint-result --benchmark_filter=InputFromBatchStream +--threads 1 --iterations 1 --noprint-result ``` The output should be like: @@ -118,12 +118,12 @@ cd /path/to/gluten/ First, get the Stage Id from spark UI for the stage you want to simulate. And then re-run the query with below configurations to dump the inputs to micro benchmark. -| Parameters | Description | Recommend Setting | -|---------------------------------------------|----------------------------------------------------------------------------------------------------------------|-----------------------| -| spark.gluten.sql.benchmark_task.stageId | Spark task stage id | target stage id | -| spark.gluten.sql.benchmark_task.partitionId | Spark task partition id, default value -1 means all the partition of this stage | 0 | -| spark.gluten.sql.benchmark_task.taskId | If not specify partition id, use spark task attempt id, default value -1 means all the partition of this stage | target task attemp id | -| spark.gluten.saveDir | Directory to save the inputs to micro benchmark, should exist and be empty. | /path/to/saveDir | +| Parameters | Description | Recommend Setting | +|---------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------| +| spark.gluten.sql.benchmark_task.taskId | Comma-separated string to specify the Task IDs to dump. If it's set, `spark.gluten.sql.benchmark_task.stageId` and `spark.gluten.sql.benchmark_task.partitionId` will be ignored. | Comma-separated string of task IDs. Empty by default. | +| spark.gluten.sql.benchmark_task.stageId | Spark stage ID. | Target stage ID | +| spark.gluten.sql.benchmark_task.partitionId | Comma-separated string to specify the Partition IDs in a stage to dump. Must be specified together with `spark.gluten.sql.benchmark_task.stageId`. Empty by default, meaning all partitions of this stage will be dumped. To identify the partition ID, navigate to the `Stage` tab in the Spark UI and locate it under the `Index` column. | Comma-separated string of partition IDs. Empty by default. | +| spark.gluten.saveDir | Directory to save the inputs to micro benchmark, should exist and be empty. | /path/to/saveDir | Check the files in `spark.gluten.saveDir`. If the simulated stage is a first stage, you will get 3 or 4 types of dumped file: diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala b/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala index dea0a4814028..f9bb7478e7d0 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala @@ -21,24 +21,32 @@ import org.apache.gluten.GlutenConfig import org.apache.spark.TaskContext object DebugUtil { - // if specify taskId, then only do that task partition - // if not specify stageId, then do nothing + // if taskId is specified and matches, then do that task + // if stageId is not specified or doesn't match, then do nothing // if specify stageId but no partitionId, then do all partitions for that stage // if specify stageId and partitionId, then only do that partition for that stage def saveInputToFile(): Boolean = { - if (TaskContext.get().taskAttemptId() == GlutenConfig.getConf.taskId) { - return true - } - if (TaskContext.get().stageId() != GlutenConfig.getConf.taskStageId) { - return false - } - if (GlutenConfig.getConf.taskPartitionId == -1) { - return true - } - if (TaskContext.getPartitionId() == GlutenConfig.getConf.taskPartitionId) { - return true - } + def taskIdMatches = + GlutenConfig.getConf.benchmarkTaskId.nonEmpty && + GlutenConfig.getConf.benchmarkTaskId + .split(",") + .map(_.toLong) + .contains(TaskContext.get().taskAttemptId()) + + def partitionIdMatches = + TaskContext.get().stageId() == GlutenConfig.getConf.benchmarkStageId && + (GlutenConfig.getConf.benchmarkPartitionId.isEmpty || + GlutenConfig.getConf.benchmarkPartitionId + .split(",") + .map(_.toInt) + .contains(TaskContext.get().partitionId())) - false + val saveInput = taskIdMatches || partitionIdMatches + if (saveInput) { + if (GlutenConfig.getConf.benchmarkSaveDir.isEmpty) { + throw new IllegalArgumentException(GlutenConfig.BENCHMARK_SAVE_DIR.key + " is not set.") + } + } + saveInput } } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index f756eb20a6d3..e0d06ce6fc0f 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -423,9 +423,10 @@ class GlutenConfig(conf: SQLConf) extends Logging { def debug: Boolean = conf.getConf(DEBUG_ENABLED) def debugKeepJniWorkspace: Boolean = conf.getConf(DEBUG_KEEP_JNI_WORKSPACE) def collectUtStats: Boolean = conf.getConf(UT_STATISTIC) - def taskStageId: Int = conf.getConf(BENCHMARK_TASK_STAGEID) - def taskPartitionId: Int = conf.getConf(BENCHMARK_TASK_PARTITIONID) - def taskId: Long = conf.getConf(BENCHMARK_TASK_TASK_ID) + def benchmarkStageId: Int = conf.getConf(BENCHMARK_TASK_STAGEID) + def benchmarkPartitionId: String = conf.getConf(BENCHMARK_TASK_PARTITIONID) + def benchmarkTaskId: String = conf.getConf(BENCHMARK_TASK_TASK_ID) + def benchmarkSaveDir: String = conf.getConf(BENCHMARK_SAVE_DIR) def textInputMaxBlockSize: Long = conf.getConf(TEXT_INPUT_ROW_MAX_BLOCK_SIZE) def textIputEmptyAsDefault: Boolean = conf.getConf(TEXT_INPUT_EMPTY_AS_DEFAULT) def enableParquetRowGroupMaxMinIndex: Boolean = @@ -1719,14 +1720,20 @@ object GlutenConfig { val BENCHMARK_TASK_PARTITIONID = buildConf("spark.gluten.sql.benchmark_task.partitionId") .internal() - .intConf - .createWithDefault(-1) + .stringConf + .createWithDefault("") val BENCHMARK_TASK_TASK_ID = buildConf("spark.gluten.sql.benchmark_task.taskId") .internal() - .longConf - .createWithDefault(-1L) + .stringConf + .createWithDefault("") + + val BENCHMARK_SAVE_DIR = + buildConf(GLUTEN_SAVE_DIR) + .internal() + .stringConf + .createWithDefault("") val NATIVE_WRITER_ENABLED = buildConf("spark.gluten.sql.native.writer.enabled") From 35de38f4ad1daa25a990659ef1243924e9964e27 Mon Sep 17 00:00:00 2001 From: Suraj Naik Date: Mon, 25 Nov 2024 08:37:53 +0530 Subject: [PATCH 138/211] [GLUTEN-7950][VL] Keep Core module's build flag consistent with Velox (#8027) --- cpp/core/CMakeLists.txt | 30 ++++++++++++++++++++++-------- ep/build-velox/src/get_velox.sh | 1 - 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 94484eab826c..88b0fd883e5c 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -22,10 +22,29 @@ include(FindPkgConfig) include(GNUInstallDirs) include(CheckCXXCompilerFlag) -# Only set arch=native for non-AppleClang compilers. -if(NOT CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=native") +set(CMAKE_CXX_FLAGS + "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") +if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") +endif() + +if(NOT DEFINED VELOX_HOME) + set(VELOX_HOME ${GLUTEN_HOME}/ep/build-velox/build/velox_ep) + message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") +endif() + +# Keep same compile option with Velox. +execute_process( + COMMAND + bash -c + "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" + OUTPUT_VARIABLE SCRIPT_CXX_FLAGS + RESULT_VARIABLE COMMAND_STATUS) +if(COMMAND_STATUS EQUAL "1") + message(FATAL_ERROR "Unable to determine compiler flags!") endif() +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") +message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") set(BOOST_MIN_VERSION "1.42.0") find_package(Boost REQUIRED) @@ -99,11 +118,6 @@ set_source_files_properties(${GLUTEN_PROTO_OUTPUT_FILES} PROPERTIES GENERATED TRUE) get_filename_component(GLUTEN_PROTO_DIR ${GLUTEN_PROTO_SRC_DIR}/ DIRECTORY) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") - -message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") - set(SPARK_COLUMNAR_PLUGIN_SRCS ${SUBSTRAIT_PROTO_SRCS} ${GLUTEN_PROTO_SRCS} diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index f50e244dffa6..d95f742a5515 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -189,7 +189,6 @@ function setup_linux { local LINUX_VERSION_ID=$(. /etc/os-release && echo ${VERSION_ID}) # apply patches - sed -i 's/-mavx2 -mfma -mavx -mf16c -mlzcnt -std=c++17/-march=native -std=c++17 -mno-avx512f/g' scripts/setup-helper-functions.sh sed -i 's/SUDO="${SUDO:-""}"/SUDO="${SUDO:-"sudo --preserve-env"}"/g' scripts/setup-helper-functions.sh if [[ "$LINUX_DISTRIBUTION" == "ubuntu" || "$LINUX_DISTRIBUTION" == "debian" || "$LINUX_DISTRIBUTION" == "pop" ]]; then process_setup_ubuntu From 0bd6584eab916e79028c6275ae72e4cb3a0364a8 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 25 Nov 2024 14:04:45 +0800 Subject: [PATCH 139/211] [VL] RAS: Remove alternative constraint sets passing to RAS planner (#8033) --- .../enumerated/EnumeratedTransform.scala | 11 +----- .../scala/org/apache/gluten/ras/Ras.scala | 17 +++------ .../org/apache/gluten/ras/RasPlanner.scala | 10 ++---- .../org/apache/gluten/ras/dp/DpPlanner.scala | 15 ++------ .../ras/exaustive/ExhaustivePlanner.scala | 10 ++---- .../org/apache/gluten/ras/PropertySuite.scala | 35 ------------------- 6 files changed, 13 insertions(+), 85 deletions(-) diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index 43c5f10bbde3..fad0ae386c04 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -21,7 +21,6 @@ import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv -import org.apache.gluten.extension.columnar.transition.ConventionReq import org.apache.gluten.extension.injector.Injector import org.apache.gluten.extension.util.AdaptiveContext import org.apache.gluten.logging.LogLevelUtil @@ -59,17 +58,9 @@ case class EnumeratedTransform(costModel: CostModel[SparkPlan], rules: Seq[RasRu private val reqConvention = Conv.any - private val altConventions = { - val rowBased: Conv = Conv.req(ConventionReq.row) - val backendBatchBased: Conv = Conv.req(ConventionReq.backendBatch) - Seq(rowBased, backendBatchBased) - } - override def apply(plan: SparkPlan): SparkPlan = { val constraintSet = PropertySet(List(reqConvention)) - val altConstraintSets = - altConventions.map(altConv => PropertySet(List(altConv))) - val planner = optimization.newPlanner(plan, constraintSet, altConstraintSets) + val planner = optimization.newPlanner(plan, constraintSet) val out = planner.plan() out } diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala index f705a2901be9..6cf15b0e8d6c 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala @@ -26,10 +26,7 @@ import scala.collection.mutable * https://github.com/apache/incubator-gluten/issues/5057. */ trait Optimization[T <: AnyRef] { - def newPlanner( - plan: T, - constraintSet: PropertySet[T], - altConstraintSets: Seq[PropertySet[T]]): RasPlanner[T] + def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] def anyPropSet(): PropertySet[T] def withNewConfig(confFunc: RasConfig => RasConfig): Optimization[T] } @@ -47,10 +44,7 @@ object Optimization { implicit class OptimizationImplicits[T <: AnyRef](opt: Optimization[T]) { def newPlanner(plan: T): RasPlanner[T] = { - opt.newPlanner(plan, opt.anyPropSet(), List.empty) - } - def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] = { - opt.newPlanner(plan, constraintSet, List.empty) + opt.newPlanner(plan, opt.anyPropSet()) } } } @@ -113,11 +107,8 @@ class Ras[T <: AnyRef] private ( } } - override def newPlanner( - plan: T, - constraintSet: PropertySet[T], - altConstraintSets: Seq[PropertySet[T]]): RasPlanner[T] = { - RasPlanner(this, altConstraintSets, constraintSet, plan) + override def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] = { + RasPlanner(this, constraintSet, plan) } override def anyPropSet(): PropertySet[T] = propertySetFactory().any() diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala index 327b980f38ec..626d2b1caf9c 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala @@ -32,16 +32,12 @@ trait RasPlanner[T <: AnyRef] { } object RasPlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { ras.config.plannerType match { case PlannerType.Exhaustive => - ExhaustivePlanner(ras, altConstraintSets, constraintSet, plan) + ExhaustivePlanner(ras, constraintSet, plan) case PlannerType.Dp => - DpPlanner(ras, altConstraintSets, constraintSet, plan) + DpPlanner(ras, constraintSet, plan) } } } diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala index 3f2590dff8a4..2b601720bfd5 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala @@ -26,11 +26,7 @@ import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.ras.rule.{EnforcerRuleSet, RuleApplier, Shape} // TODO: Branch and bound pruning. -private class DpPlanner[T <: AnyRef] private ( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T) +private class DpPlanner[T <: AnyRef] private (ras: Ras[T], constraintSet: PropertySet[T], plan: T) extends RasPlanner[T] { import DpPlanner._ @@ -43,7 +39,6 @@ private class DpPlanner[T <: AnyRef] private ( } private lazy val best: (Best[T], KnownCostPath[T]) = { - altConstraintSets.foreach(propSet => memo.memorize(plan, propSet)) val groupId = rootGroupId val memoTable = memo.table() val best = findBest(memoTable, groupId) @@ -70,12 +65,8 @@ private class DpPlanner[T <: AnyRef] private ( } object DpPlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { - new DpPlanner(ras, altConstraintSets: Seq[PropertySet[T]], constraintSet, plan) + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { + new DpPlanner(ras, constraintSet, plan) } // Visited flag. diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala index c4d3e4881c43..f0cb42cf662a 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala @@ -27,7 +27,6 @@ import org.apache.gluten.ras.rule.{EnforcerRuleSet, RuleApplier, Shape} private class ExhaustivePlanner[T <: AnyRef] private ( ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], constraintSet: PropertySet[T], plan: T) extends RasPlanner[T] { @@ -40,7 +39,6 @@ private class ExhaustivePlanner[T <: AnyRef] private ( } private lazy val best: (Best[T], KnownCostPath[T]) = { - altConstraintSets.foreach(propSet => memo.memorize(plan, propSet)) val groupId = rootGroupId explore() val memoState = memo.newState() @@ -72,12 +70,8 @@ private class ExhaustivePlanner[T <: AnyRef] private ( } object ExhaustivePlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { - new ExhaustivePlanner(ras, altConstraintSets, constraintSet, plan) + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { + new ExhaustivePlanner(ras, constraintSet, plan) } private class ExhaustiveExplorer[T <: AnyRef]( diff --git a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala index eb4babe069e5..06bb806f7d6b 100644 --- a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala +++ b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala @@ -250,41 +250,6 @@ abstract class PropertySuite extends AnyFunSuite { assert(out == TypedUnary(TypeA, 8, PassNodeType(5, TypedLeaf(TypeA, 10)))) } - test(s"Property convert - (A, B), alternative conventions") { - object ConvertEnforcerAndTypeAToTypeB extends RasRule[TestNode] { - override def shift(node: TestNode): Iterable[TestNode] = node match { - case TypeEnforcer(TypeB, _, TypedBinary(TypeA, 5, left, right)) => - List(TypedBinary(TypeB, 0, left, right)) - case _ => List.empty - } - override def shape(): Shape[TestNode] = Shapes.fixedHeight(2) - } - - val ras = - Ras[TestNode]( - PlanModelImpl, - CostModelImpl, - MetadataModelImpl, - propertyModel(zeroDepth), - ExplainImpl, - RasRule.Factory.reuse(List(ConvertEnforcerAndTypeAToTypeB))) - .withNewConfig(_ => conf) - val plan = - TypedBinary(TypeA, 5, TypedUnary(TypeA, 10, TypedLeaf(TypeA, 10)), TypedLeaf(TypeA, 10)) - val planner = ras.newPlanner( - plan, - PropertySet(Seq(TypeAny)), - List(PropertySet(Seq(TypeB)), PropertySet(Seq(TypeC)))) - val out = planner.plan() - assert( - out == TypedBinary( - TypeB, - 0, - TypeEnforcer(TypeB, 1, TypedUnary(TypeA, 10, TypedLeaf(TypeA, 10))), - TypeEnforcer(TypeB, 1, TypedLeaf(TypeA, 10)))) - assert(planner.newState().memoState().allGroups().size == 9) - } - test(s"Property convert - (A, B), Unary only has TypeA") { object ReplaceNonUnaryByTypeBRule extends RasRule[TestNode] { override def shift(node: TestNode): Iterable[TestNode] = { From e5b4b4ce081f99090ac532b3d16ea31b2de12aac Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 25 Nov 2024 14:29:54 +0800 Subject: [PATCH 140/211] [GLUTEN-6920][CORE] Move API `Backend#defaultBatchType` down to `BackendSettingsApi` in module gluten-substrait (#8016) --- .../backendsapi/clickhouse/CHBackend.scala | 17 +++++- .../backendsapi/clickhouse/CHRuleApi.scala | 3 +- .../backendsapi/velox/VeloxBackend.scala | 10 ++-- .../backendsapi/velox/VeloxRuleApi.scala | 5 +- .../ColumnarPartialProjectExec.scala | 2 - .../execution/VeloxResizeBatchesExec.scala | 1 - .../python/ColumnarArrowEvalPythonExec.scala | 3 +- .../execution/ArrowFileSourceScanExec.scala | 2 - .../sql/execution/BaseArrowScanExec.scala | 3 +- .../transition/VeloxTransitionSuite.scala | 46 ++++++++-------- .../org/apache/gluten/backend/Backend.scala | 5 +- .../execution/ColumnarToColumnarExec.scala | 21 ++++++-- .../gluten/extension/GlutenColumnarRule.scala | 9 ++-- .../enumerated/EnumeratedTransform.scala | 4 +- .../planner/cost/LongCostModel.scala | 9 ++-- .../planner/plan/GlutenPlanModel.scala | 19 ++++--- .../enumerated/planner/property/Conv.scala | 12 +++++ .../columnar/transition/Convention.scala | 52 ++++++++++++++++++- .../columnar/transition/ConventionFunc.scala | 52 ++++++++++++------- .../columnar/transition/ConventionReq.scala | 13 ++--- .../columnar/transition/Transition.scala | 19 ++++--- .../columnar/transition/Transitions.scala | 40 +++++++------- .../extension/injector/GlutenInjector.scala | 4 +- .../apache/gluten/iterator/Iterators.scala | 2 +- .../apache/spark/util/SparkVersionUtil.scala | 30 +++++++++++ .../backendsapi/BackendSettingsApi.scala | 5 ++ .../BasicPhysicalOperatorTransformer.scala | 2 - .../CartesianProductExecTransformer.scala | 1 - .../execution/ColumnarCoalesceExec.scala | 2 - .../execution/ColumnarToRowExecBase.scala | 11 ++-- .../execution/RowToColumnarExecBase.scala | 5 +- ...TakeOrderedAndProjectExecTransformer.scala | 1 - .../execution/WholeStageTransformer.scala | 2 - .../apache/gluten/extension/GlutenPlan.scala | 30 ++++++----- .../columnar/MiscColumnarRules.scala | 4 +- .../heuristic/ExpandFallbackPolicy.scala | 7 ++- .../transition/BackendTransitions.scala | 36 +++++++++++++ .../ColumnarBroadcastExchangeExec.scala | 2 - .../ColumnarCollapseTransformStages.scala | 5 +- .../ColumnarShuffleExchangeExec.scala | 1 - .../execution/ColumnarWriteFilesExec.scala | 14 +++-- .../GlutenWriterColumnarRules.scala | 6 ++- .../columnar/transition/TransitionSuite.scala | 15 +++--- .../transition/TransitionSuiteBase.scala | 11 ++-- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../benchmarks/ParquetReadBenchmark.scala | 3 +- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../AbstractFileSourceScanExec.scala | 7 ++- .../AbstractFileSourceScanExec.scala | 7 ++- .../AbstractFileSourceScanExec.scala | 12 ++--- .../AbstractFileSourceScanExec.scala | 12 ++--- 53 files changed, 401 insertions(+), 263 deletions(-) create mode 100644 gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 061ec9856e80..f6cacba42b9f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -24,7 +24,7 @@ import org.apache.gluten.columnarbatch.CHBatch import org.apache.gluten.execution.WriteFilesExecTransformer import org.apache.gluten.expression.WindowFunctionsBuilder import org.apache.gluten.extension.ValidationResult -import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat._ @@ -34,6 +34,8 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -45,10 +47,11 @@ import java.util.Locale import scala.util.control.Breaks.{break, breakable} class CHBackend extends SubstraitBackend { + import CHBackend._ override def name(): String = CHConf.BACKEND_NAME - override def defaultBatchType: Convention.BatchType = CHBatch override def buildInfo(): Backend.BuildInfo = Backend.BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN") + override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def iteratorApi(): IteratorApi = new CHIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new CHSparkPlanExecApi override def transformerApi(): TransformerApi = new CHTransformerApi @@ -59,7 +62,17 @@ class CHBackend extends SubstraitBackend { override def settings(): BackendSettingsApi = CHBackendSettings } +object CHBackend { + private class ConvFunc() extends ConventionFunc.Override { + override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + case a: AdaptiveSparkPlanExec if a.supportsColumnar => + CHBatch + } + } +} + object CHBackendSettings extends BackendSettingsApi with Logging { + override def primaryBatchType: Convention.BatchType = CHBatch private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD = "spark.gluten.sql.columnar.separate.scan.rdd.for.ch" private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD_DEFAULT = "false" diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 081e815588fa..f6d2b85d9d19 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -18,6 +18,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi +import org.apache.gluten.columnarbatch.CHBatch import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} @@ -114,7 +115,7 @@ object CHRuleApi { intercept( SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarTransformRules)( c.session))) - injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, CHBatch)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 03d5aa2549e5..e05fd92e3201 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.execution.{ColumnarCachedBatchSerializer, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} @@ -52,10 +53,9 @@ import scala.util.control.Breaks.breakable class VeloxBackend extends SubstraitBackend { import VeloxBackend._ override def name(): String = VeloxBackend.BACKEND_NAME - override def defaultBatchType: Convention.BatchType = VeloxBatch - override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def buildInfo(): Backend.BuildInfo = Backend.BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME) + override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def iteratorApi(): IteratorApi = new VeloxIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new VeloxSparkPlanExecApi override def transformerApi(): TransformerApi = new VeloxTransformerApi @@ -72,6 +72,8 @@ object VeloxBackend { private class ConvFunc() extends ConventionFunc.Override { override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + case a: AdaptiveSparkPlanExec if a.supportsColumnar => + VeloxBatch case i: InMemoryTableScanExec if i.supportsColumnar && i.relation.cacheBuilder.serializer .isInstanceOf[ColumnarCachedBatchSerializer] => @@ -81,13 +83,15 @@ object VeloxBackend { } object VeloxBackendSettings extends BackendSettingsApi { - val SHUFFLE_SUPPORTED_CODEC = Set("lz4", "zstd") val GLUTEN_VELOX_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".udfLibraryPaths" val GLUTEN_VELOX_DRIVER_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".driver.udfLibraryPaths" val GLUTEN_VELOX_INTERNAL_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".internal.udfLibraryPaths" val GLUTEN_VELOX_UDF_ALLOW_TYPE_CONVERSION = VeloxBackend.CONF_PREFIX + ".udfAllowTypeConversion" + /** The columnar-batch type this backend is by default using. */ + override def primaryBatchType: Convention.BatchType = VeloxBatch + override def validateScanExec( format: ReadFileFormat, fields: Array[StructField], diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index c5f46dae67dd..0a62a41baaf0 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -18,6 +18,7 @@ package org.apache.gluten.backendsapi.velox import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi +import org.apache.gluten.columnarbatch.VeloxBatch import org.apache.gluten.datasource.ArrowConvertorRule import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ @@ -101,7 +102,7 @@ object VeloxRuleApi { injector.injectPostTransform(_ => EliminateLocalSort) injector.injectPostTransform(_ => CollapseProjectExecTransformer) injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) - injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( @@ -187,7 +188,7 @@ object VeloxRuleApi { injector.injectPostTransform(_ => EliminateLocalSort) injector.injectPostTransform(_ => CollapseProjectExecTransformer) injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) - injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch)) injector.injectPostTransform( c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext())) SparkShimLoader.getSparkShims diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala index d993e399dbf4..84de41daa015 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala @@ -82,8 +82,6 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( replacedAliasUdf :: Nil } - final override val supportsColumnar: Boolean = true - private def validateExpression(expr: Expression): Boolean = { expr.deterministic && !expr.isInstanceOf[LambdaFunction] && expr.children .forall(validateExpression) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala index 995582024ba5..5283ab61e3b9 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala @@ -53,7 +53,6 @@ case class VeloxResizeBatchesExec( "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append / split batches") ) - override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala index f1f5eb906238..a2eee53660ce 100644 --- a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala @@ -214,9 +214,8 @@ case class ColumnarArrowEvalPythonExec( extends EvalPythonExec with GlutenPlan with KnownChildrenConventions { - override def supportsColumnar: Boolean = true - override protected def batchType0(): Convention.BatchType = ArrowJavaBatch + override def batchType(): Convention.BatchType = ArrowJavaBatch override def requiredChildrenConventions(): Seq[ConventionReq] = List( ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(ArrowJavaBatch))) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala index 85bc68223487..16b8fb0e9f6f 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala @@ -33,8 +33,6 @@ case class ArrowFileSourceScanExec(original: FileSourceScanExec) override def output: Seq[Attribute] = original.output - override def supportsColumnar: Boolean = original.supportsColumnar - override def doCanonicalize(): FileSourceScanExec = original.doCanonicalize() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala index 38a6d1803d0b..6617e8b138ba 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala @@ -21,8 +21,7 @@ import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.extension.columnar.transition.Convention trait BaseArrowScanExec extends GlutenPlan { - - final override protected def batchType0(): Convention.BatchType = { + final override def batchType(): Convention.BatchType = { ArrowBatches.ArrowJavaBatch } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala index d12faae0f73d..e14ffd43d82d 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala @@ -33,37 +33,37 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla C2R - outputs row") { val in = BatchLeaf(VanillaBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchLeaf(VanillaBatch))) } test("Vanilla C2R - requires row input") { val in = RowUnary(BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(VanillaBatch)))) } test("Vanilla R2C - requires vanilla input") { val in = BatchUnary(VanillaBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, RowToColumnarExec(RowLeaf())))) } test("ArrowNative C2R - outputs row") { val in = BatchLeaf(ArrowNativeBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch)))) } test("ArrowNative C2R - requires row input") { val in = RowUnary(BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch))))) } test("ArrowNative R2C - requires Arrow input") { val in = BatchUnary(ArrowNativeBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary(ArrowNativeBatch, RowToVeloxColumnarExec(RowLeaf()))))) @@ -71,7 +71,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowNative-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) // No explicit transition needed for ArrowNative-to-Velox. // FIXME: Add explicit transitions. // See https://github.com/apache/incubator-gluten/issues/7313. @@ -82,7 +82,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-ArrowNative C2C") { val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch))))) @@ -90,7 +90,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla-to-ArrowNative C2C") { val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary( @@ -100,7 +100,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowNative-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(ArrowNativeBatch))))) @@ -108,19 +108,19 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava C2R - outputs row") { val in = BatchLeaf(ArrowJavaBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchLeaf(ArrowJavaBatch))) } test("ArrowJava C2R - requires row input") { val in = RowUnary(BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(ArrowJavaBatch)))) } test("ArrowJava R2C - requires Arrow input") { val in = BatchUnary(ArrowJavaBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(ArrowJavaBatch, LoadArrowDataExec(RowToVeloxColumnarExec(RowLeaf()))))) @@ -128,7 +128,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == VeloxColumnarToRowExec( BatchUnary( @@ -138,7 +138,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-ArrowJava C2C") { val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(ArrowJavaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch))))) @@ -146,7 +146,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla-to-ArrowJava C2C") { val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary( @@ -156,37 +156,37 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch)))) } test("Velox C2R - outputs row") { val in = BatchLeaf(VeloxBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == VeloxColumnarToRowExec(BatchLeaf(VeloxBatch))) } test("Velox C2R - requires row input") { val in = RowUnary(BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(VeloxColumnarToRowExec(BatchLeaf(VeloxBatch)))) } test("Velox R2C - outputs Velox") { val in = RowLeaf() - val out = Transitions.insertTransitions(in, outputsColumnar = true) + val out = BackendTransitions.insert(in, outputsColumnar = true) assert(out == RowToVeloxColumnarExec(RowLeaf())) } test("Velox R2C - requires Velox input") { val in = BatchUnary(VeloxBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == VeloxColumnarToRowExec(BatchUnary(VeloxBatch, RowToVeloxColumnarExec(RowLeaf())))) } test("Vanilla-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == VeloxColumnarToRowExec( BatchUnary(VeloxBatch, RowToVeloxColumnarExec(ColumnarToRowExec(BatchLeaf(VanillaBatch)))))) @@ -194,7 +194,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec(BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch))))) } diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala index 346181e140ac..f406a6ac4d7d 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.backend -import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} +import org.apache.gluten.extension.columnar.transition.ConventionFunc import org.apache.gluten.extension.injector.Injector import org.apache.spark.SparkContext @@ -39,9 +39,6 @@ trait Backend { def onExecutorStart(pc: PluginContext): Unit = {} def onExecutorShutdown(): Unit = {} - /** The columnar-batch type this backend is by default using. */ - def defaultBatchType: Convention.BatchType - /** * Overrides [[org.apache.gluten.extension.columnar.transition.ConventionFunc]] Gluten is using to * determine the convention (its row-based processing / columnar-batch processing support) of a diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala index d0eacc1e4dac..7ca4b36b06d2 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala @@ -17,8 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.Convention.KnownBatchType -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.gluten.iterator.Iterators import org.apache.spark.rdd.RDD @@ -32,8 +30,9 @@ import java.util.concurrent.atomic.AtomicLong abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention.BatchType) extends ColumnarToColumnarTransition - with KnownBatchType - with KnownChildrenConventions { + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater + with ConventionReq.KnownChildrenConventions { def child: SparkPlan protected def mapIterator(in: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] @@ -47,8 +46,20 @@ abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert batches") ) - override def supportsColumnar: Boolean = true + final override val supportsColumnar: Boolean = { + batchType() != Convention.BatchType.None + } + override def batchType(): Convention.BatchType = to + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None + } + + override def rowType0(): Convention.RowType = { + Convention.RowType.None + } + override def requiredChildrenConventions(): Seq[ConventionReq] = List( ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(from))) diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala index 3378344253dd..5b440302a09b 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala @@ -31,11 +31,9 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.vectorized.ColumnarBatch object GlutenColumnarRule { - // Utilities to infer columnar rule's caller's property: // ApplyColumnarRulesAndInsertTransitions#outputsColumnar. - - case class DummyRowOutputExec(override val child: SparkPlan) extends UnaryExecNode { + private case class DummyRowOutputExec(override val child: SparkPlan) extends UnaryExecNode { override def supportsColumnar: Boolean = false override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = @@ -47,7 +45,7 @@ object GlutenColumnarRule { copy(child = newChild) } - case class DummyColumnarOutputExec(override val child: SparkPlan) extends UnaryExecNode { + private case class DummyColumnarOutputExec(override val child: SparkPlan) extends UnaryExecNode { override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = @@ -99,9 +97,8 @@ case class GlutenColumnarRule( "This should not happen. Please leave an issue at" + " https://github.com/apache/incubator-gluten.") } - val vanillaPlan = Transitions.insertTransitions(originalPlan, outputsColumnar) + val vanillaPlan = Transitions.insert(originalPlan, outputsColumnar) val applier = applierBuilder.apply(session) applier.apply(vanillaPlan, outputsColumnar) } - } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index fad0ae386c04..67399e25d412 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -56,10 +56,10 @@ case class EnumeratedTransform(costModel: CostModel[SparkPlan], rules: Seq[RasRu .create() } - private val reqConvention = Conv.any + private val convReq = Conv.any override def apply(plan: SparkPlan): SparkPlan = { - val constraintSet = PropertySet(List(reqConvention)) + val constraintSet = PropertySet(Seq(convReq)) val planner = optimization.newPlanner(plan, constraintSet) val out = planner.plan() out diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala index 1cfe132d84ed..393ac35de42f 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala @@ -69,19 +69,20 @@ object LongCostModel extends Logging { */ sealed trait Kind { import Kind._ - values.synchronized { + all.synchronized { val n = name() - if (values.contains(n)) { + if (all.contains(n)) { throw new GlutenException(s"Cost mode kind $n already registered") } - values += n -> this + all += n -> this } def name(): String } object Kind { - val values: mutable.Map[String, Kind] = mutable.Map() + private val all: mutable.Map[String, Kind] = mutable.Map() + def values(): Map[String, Kind] = all.toMap } /** diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala index fa92eacd4d47..568ea50396bd 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala @@ -19,7 +19,6 @@ package org.apache.gluten.extension.columnar.enumerated.planner.plan import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadata import org.apache.gluten.extension.columnar.enumerated.planner.property.{Conv, ConvDef} import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.Convention.{KnownBatchType, KnownRowType} import org.apache.gluten.ras.{Metadata, PlanModel} import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.sql.shims.SparkShimLoader @@ -43,17 +42,13 @@ object GlutenPlanModel { metadata: GlutenMetadata, constraintSet: PropertySet[SparkPlan]) extends LeafExecNode - with KnownBatchType - with KnownRowType { + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater { private val req: Conv.Req = constraintSet.get(ConvDef).asInstanceOf[Conv.Req] override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException() override def output: Seq[Attribute] = metadata.schema().output - override def supportsColumnar: Boolean = { - batchType != Convention.BatchType.None - } - override val batchType: Convention.BatchType = { val out = req.req.requiredBatchType match { case ConventionReq.BatchType.Any => Convention.BatchType.None @@ -62,13 +57,21 @@ object GlutenPlanModel { out } - override val rowType: Convention.RowType = { + final override val supportsColumnar: Boolean = { + batchType != Convention.BatchType.None + } + + override val rowType0: Convention.RowType = { val out = req.req.requiredRowType match { case ConventionReq.RowType.Any => Convention.RowType.None case ConventionReq.RowType.Is(r) => r } out } + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None + } } private object PlanModelImpl extends PlanModel[SparkPlan] { diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala index 831b212e1f96..7b2b801ac91e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala @@ -29,6 +29,18 @@ sealed trait Conv extends Property[SparkPlan] { } override def satisfies(other: Property[SparkPlan]): Boolean = { + // The following enforces strict type checking against `this` and `other` + // to make sure: + // + // 1. `this`, which came from user implementation of PropertyDef.getProperty, must be a `Prop` + // 2. `other` which came from user implementation of PropertyDef.getChildrenConstraints, + // must be a `Req` + // + // If the user implementation doesn't follow the criteria, cast error will be thrown. + // + // This can be a common practice to implement a safe Property for RAS. + // + // TODO: Add a similar case to RAS UTs. val req = other.asInstanceOf[Req] if (req.isAny) { return true diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala index 840b62fb672d..b57f3e0c0a29 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala @@ -16,10 +16,15 @@ */ package org.apache.gluten.extension.columnar.transition +import org.apache.gluten.exception.GlutenException + import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} +import org.apache.spark.util.SparkVersionUtil import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.mutable + /** * Convention of a query plan consists of the row data type and columnar data type it supports to * output. @@ -74,6 +79,7 @@ object Convention { } trait BatchType extends TransitionGraph.Vertex with Serializable { + import BatchType._ private val initialized: AtomicBoolean = new AtomicBoolean(false) final def ensureRegistered(): Unit = { @@ -84,7 +90,8 @@ object Convention { register() } - final private def register(): Unit = { + final private def register(): Unit = BatchType.synchronized { + assert(all.add(this)) Transition.graph.addVertex(this) registerTransitions() } @@ -117,6 +124,8 @@ object Convention { } object BatchType { + private val all: mutable.Set[BatchType] = mutable.Set() + def values(): Set[BatchType] = all.toSet // None indicates that the plan doesn't support batch-based processing. final case object None extends BatchType { override protected[this] def registerTransitions(): Unit = {} @@ -133,7 +142,46 @@ object Convention { def batchType(): BatchType } - trait KnownRowType { + sealed trait KnownRowType extends KnownRowType.SupportsRowBasedCompatible { def rowType(): RowType } + + object KnownRowType { + // To be compatible with Spark (version < 3.3) + sealed trait SupportsRowBasedCompatible { + def supportsRowBased(): Boolean = { + throw new GlutenException("Illegal state: The method is not expected to be called") + } + } + } + + trait KnownRowTypeForSpark33AndLater extends KnownRowType { + this: SparkPlan => + import KnownRowTypeForSpark33AndLater._ + + final override def rowType(): RowType = { + if (lteSpark32) { + // It's known that in Spark 3.2, one Spark plan node is considered either only having + // row-based support or only having columnar support at a time. + // Hence, if the plan supports columnar output, we'd disable its row-based support. + // The same for the opposite. + if (supportsColumnar) { + Convention.RowType.None + } else { + Convention.RowType.VanillaRow + } + } else { + rowType0() + } + } + + def rowType0(): RowType + } + + object KnownRowTypeForSpark33AndLater { + private val lteSpark32: Boolean = { + val v = SparkVersionUtil.majorMinorVersion() + SparkVersionUtil.compareMajorMinorVersion(v, (3, 2)) <= 0 + } + } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala index c3feefe9435a..5cb3d44a1562 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala @@ -21,7 +21,7 @@ import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildr import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnionExec} -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec @@ -89,17 +89,6 @@ object ConventionFunc { } case q: QueryStageExec => conventionOf0(q.plan) case r: ReusedExchangeExec => conventionOf0(r.child) - case a: AdaptiveSparkPlanExec => - val rowType = rowTypeOf(a) - val batchType = if (a.supportsColumnar) { - // By default, we execute columnar AQE with backend batch output. - // See org.apache.gluten.extension.columnar.transition.InsertTransitions.apply - Backend.get().defaultBatchType - } else { - Convention.BatchType.None - } - val conv = Convention.of(rowType, batchType) - conv case other => val conv = Convention.of(rowTypeOf(other), batchTypeOf(other)) conv @@ -119,12 +108,24 @@ object ConventionFunc { case _ => Convention.RowType.None } - assert( - out == Convention.RowType.None || plan.isInstanceOf[Convention.KnownRowType] || - SparkShimLoader.getSparkShims.supportsRowBased(plan)) + checkRowType(plan, out) out } + private def checkRowType(plan: SparkPlan, rowType: Convention.RowType): Unit = { + if (SparkShimLoader.getSparkShims.supportsRowBased(plan)) { + assert( + rowType != Convention.RowType.None, + s"Plan ${plan.nodeName} supports row-based execution, " + + s"however #rowTypeOf returns None") + } else { + assert( + rowType == Convention.RowType.None, + s"Plan ${plan.nodeName} doesn't support row-based " + + s"execution, however #rowTypeOf returns $rowType") + } + } + private def batchTypeOf(plan: SparkPlan): Convention.BatchType = { val out = o.batchTypeOf.applyOrElse(plan, batchTypeOf0) out @@ -139,10 +140,24 @@ object ConventionFunc { case _ => Convention.BatchType.None } - assert(out == Convention.BatchType.None || plan.supportsColumnar) + checkBatchType(plan, out) out } + private def checkBatchType(plan: SparkPlan, batchType: Convention.BatchType): Unit = { + if (plan.supportsColumnar) { + assert( + batchType != Convention.BatchType.None, + s"Plan ${plan.nodeName} supports columnar " + + s"execution, however #batchTypeOf returns None") + } else { + assert( + batchType == Convention.BatchType.None, + s"Plan ${plan.nodeName} doesn't support " + + s"columnar execution, however #batchTypeOf returns $batchType") + } + } + override def conventionReqOf(plan: SparkPlan): ConventionReq = { val req = o.conventionReqOf.applyOrElse(plan, conventionReqOf0) req @@ -169,14 +184,15 @@ object ConventionFunc { // To align with ApplyColumnarRulesAndInsertTransitions#insertTransitions ConventionReq.any case u: UnionExec => - // We force vanilla union to output row data to get best compatibility with vanilla Spark. + // We force vanilla union to output row data to get the best compatibility with vanilla + // Spark. // As a result it's a common practice to rewrite it with GlutenPlan for offloading. ConventionReq.of( ConventionReq.RowType.Is(Convention.RowType.VanillaRow), ConventionReq.BatchType.Any) case other => // In the normal case, children's convention should follow parent node's convention. - // Note, we don't have consider C2R / R2C here since they are already removed by + // Note, we don't have to consider C2R / R2C here since they are already removed by // RemoveTransitions. val thisConv = conventionOf0(other) thisConv.asReq() diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala index ce613bf7dbba..a081f214347b 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala @@ -16,8 +16,6 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend - import org.apache.spark.sql.execution.SparkPlan /** @@ -53,15 +51,14 @@ object ConventionReq { override val requiredBatchType: BatchType ) extends ConventionReq - val any: ConventionReq = Impl(RowType.Any, BatchType.Any) - val row: ConventionReq = Impl(RowType.Is(Convention.RowType.VanillaRow), BatchType.Any) - val vanillaBatch: ConventionReq = - Impl(RowType.Any, BatchType.Is(Convention.BatchType.VanillaBatch)) - lazy val backendBatch: ConventionReq = - Impl(RowType.Any, BatchType.Is(Backend.get().defaultBatchType)) + val any: ConventionReq = of(RowType.Any, BatchType.Any) + val row: ConventionReq = ofRow(RowType.Is(Convention.RowType.VanillaRow)) + val vanillaBatch: ConventionReq = ofBatch(BatchType.Is(Convention.BatchType.VanillaBatch)) def get(plan: SparkPlan): ConventionReq = ConventionFunc.create().conventionReqOf(plan) def of(rowType: RowType, batchType: BatchType): ConventionReq = Impl(rowType, batchType) + def ofRow(rowType: RowType): ConventionReq = Impl(rowType, BatchType.Any) + def ofBatch(batchType: BatchType): ConventionReq = Impl(RowType.Any, batchType) trait KnownChildrenConventions { def requiredChildrenConventions(): Seq[ConventionReq] diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala index ced9378ad69e..0a7f635b8bb0 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala @@ -101,22 +101,27 @@ object Transition { case (ConventionReq.RowType.Is(toRowType), ConventionReq.BatchType.Any) => from.rowType match { case Convention.RowType.None => + // Input query plan doesn't have recognizable row-based output, + // find columnar-to-row transition. graph.transitionOfOption(from.batchType, toRowType).getOrElse(orElse) - case fromRowType => + case fromRowType if toRowType == fromRowType => // We have only one single built-in row type. - assert(toRowType == fromRowType) Transition.empty + case _ => + throw new UnsupportedOperationException( + "Row-to-row transition is not yet supported") } case (ConventionReq.RowType.Any, ConventionReq.BatchType.Is(toBatchType)) => from.batchType match { case Convention.BatchType.None => + // Input query plan doesn't have recognizable columnar output, + // find row-to-columnar transition. graph.transitionOfOption(from.rowType, toBatchType).getOrElse(orElse) + case fromBatchType if toBatchType == fromBatchType => + Transition.empty case fromBatchType => - if (toBatchType == fromBatchType) { - Transition.empty - } else { - graph.transitionOfOption(fromBatchType, toBatchType).getOrElse(orElse) - } + // Find columnar-to-columnar transition. + graph.transitionOfOption(fromBatchType, toBatchType).getOrElse(orElse) } case (ConventionReq.RowType.Any, ConventionReq.BatchType.Any) => Transition.empty diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala index 2f2840b52b7f..10d50f453d6d 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala @@ -16,24 +16,22 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend +import org.apache.gluten.extension.columnar.transition.Convention.BatchType import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan import scala.annotation.tailrec -case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { +case class InsertTransitions(convReq: ConventionReq) extends Rule[SparkPlan] { private val convFunc = ConventionFunc.create() override def apply(plan: SparkPlan): SparkPlan = { // Remove all transitions at first. val removed = RemoveTransitions.apply(plan) val filled = fillWithTransitions(removed) - if (!outputsColumnar) { - return Transitions.toRowPlan(filled) - } - Transitions.toBackendBatchPlan(filled) + val out = Transitions.enforceReq(filled, convReq) + out } private def fillWithTransitions(plan: SparkPlan): SparkPlan = plan.transformUp { @@ -63,6 +61,17 @@ case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { } } +object InsertTransitions { + def create(outputsColumnar: Boolean, batchType: BatchType): InsertTransitions = { + val conventionReq = if (outputsColumnar) { + ConventionReq.ofBatch(ConventionReq.BatchType.Is(batchType)) + } else { + ConventionReq.row + } + InsertTransitions(conventionReq) + } +} + object RemoveTransitions extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { case p => removeForNode(p) } @@ -76,8 +85,8 @@ object RemoveTransitions extends Rule[SparkPlan] { } object Transitions { - def insertTransitions(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { - InsertTransitions(outputsColumnar).apply(plan) + def insert(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { + InsertTransitions.create(outputsColumnar, BatchType.VanillaBatch).apply(plan) } def toRowPlan(plan: SparkPlan): SparkPlan = { @@ -88,24 +97,13 @@ object Transitions { ConventionReq.BatchType.Any)) } - def toBackendBatchPlan(plan: SparkPlan): SparkPlan = { - val backendBatchType = Backend.get().defaultBatchType - val out = toBatchPlan(plan, backendBatchType) - out - } - - def toVanillaBatchPlan(plan: SparkPlan): SparkPlan = { - val out = toBatchPlan(plan, Convention.BatchType.VanillaBatch) - out - } - - private def toBatchPlan(plan: SparkPlan, toBatchType: Convention.BatchType): SparkPlan = { + def toBatchPlan(plan: SparkPlan, toBatchType: Convention.BatchType): SparkPlan = { enforceReq( plan, ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(toBatchType))) } - private def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { + def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { val convFunc = ConventionFunc.create() val removed = RemoveTransitions.removeForNode(plan) val transition = Transition diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala index 89314dfcf37f..efe584d44126 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala @@ -150,8 +150,8 @@ object GlutenInjector { private def findCostModel( registry: LongCostModel.Registry, aliasOrClass: String): CostModel[SparkPlan] = { - if (LongCostModel.Kind.values.contains(aliasOrClass)) { - val kind = LongCostModel.Kind.values(aliasOrClass) + if (LongCostModel.Kind.values().contains(aliasOrClass)) { + val kind = LongCostModel.Kind.values()(aliasOrClass) val model = registry.get(kind) return model } diff --git a/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala b/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala index 2de1c7b4ed80..ef8ca2c974cb 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala @@ -41,7 +41,7 @@ object Iterators { } def wrap[A](in: Iterator[A]): WrapperBuilder[A] = { - wrap(V1, in) + wrap(DEFAULT_VERSION, in) } def wrap[A](version: Version, in: Iterator[A]): WrapperBuilder[A] = { diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala new file mode 100644 index 000000000000..6864d3caa031 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util + +object SparkVersionUtil { + def majorMinorVersion(): (Int, Int) = { + VersionUtils.majorMinorVersion(org.apache.spark.SPARK_VERSION) + } + + // Returns X. X < 0 if one < other, x == 0 if one == other, x > 0 if one > other. + def compareMajorMinorVersion(one: (Int, Int), other: (Int, Int)): Int = { + val base = 1000 + assert(one._2 < base && other._2 < base) + one._1 * base + one._2 - (other._1 * base + other._2) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 506424b79cc3..1eb69da6e5f0 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -18,6 +18,7 @@ package org.apache.gluten.backendsapi import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -28,6 +29,10 @@ import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopF import org.apache.spark.sql.types.StructField trait BackendSettingsApi { + + /** The columnar-batch type this backend is by default using. */ + def primaryBatchType: Convention.BatchType + def validateScanExec( format: ReadFileFormat, fields: Array[StructField], diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index 76505da3e0cc..dbe667ebb2aa 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -263,8 +263,6 @@ case class ColumnarUnionExec(children: Seq[SparkPlan]) extends GlutenPlan { case _ => } - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = { children.map(_.output).transpose.map { attrs => diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala index 7f7e54e9c72c..28fb691896ff 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala @@ -45,7 +45,6 @@ import java.io.{IOException, ObjectOutputStream} */ case class ColumnarCartesianProductBridge(child: SparkPlan) extends UnaryExecNode with GlutenPlan { override def output: Seq[Attribute] = child.output - override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala index f40a7f8f07f6..3b13207c9350 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala @@ -30,8 +30,6 @@ case class ColumnarCoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala index fd86106bf367..fae3115981eb 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala @@ -18,7 +18,7 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.ConventionReq +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.spark.broadcast.Broadcast @@ -43,6 +43,10 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) final override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def batchType(): Convention.BatchType = Convention.BatchType.None + + override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow + override def doExecuteBroadcast[T](): Broadcast[T] = { // Require for explicit implementation, otherwise throw error. super.doExecuteBroadcast[T]() @@ -55,7 +59,8 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) } override def requiredChildrenConventions(): Seq[ConventionReq] = { - List(ConventionReq.backendBatch) + List( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) } - } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala index f4dd160b58b4..2a52616361a1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala @@ -18,6 +18,7 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.spark.broadcast import org.apache.spark.rdd.RDD @@ -45,6 +46,8 @@ abstract class RowToColumnarExecBase(child: SparkPlan) final override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def rowType0(): Convention.RowType = Convention.RowType.None + final override def doExecute(): RDD[InternalRow] = { child.execute() } @@ -54,8 +57,6 @@ abstract class RowToColumnarExecBase(child: SparkPlan) super.doExecuteBroadcast[T]() } - final override def supportsColumnar: Boolean = true - def doExecuteColumnarInternal(): RDD[ColumnarBatch] override def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala index c4e192d1837b..c960bda2490e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala @@ -43,7 +43,6 @@ case class TakeOrderedAndProjectExecTransformer( with GlutenPlan { override def outputPartitioning: Partitioning = SinglePartition override def outputOrdering: Seq[SortOrder] = sortOrder - override def supportsColumnar: Boolean = true override def output: Seq[Attribute] = { projectList.map(_.toAttribute) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index beb7fe5f99d2..e8a42883a54f 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -60,8 +60,6 @@ trait TransformSupport extends GlutenPlan { s"${this.getClass.getSimpleName} doesn't support doExecute") } - final override lazy val supportsColumnar: Boolean = true - /** * Returns all the RDDs of ColumnarBatch which generates the input rows. * diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala index a6ca9b1dcc5c..3639ac522f37 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala @@ -17,11 +17,11 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.expression.TransformerState import org.apache.gluten.extension.columnar.FallbackTag +import org.apache.gluten.extension.columnar.FallbackTag.{Appendable, Converter} import org.apache.gluten.extension.columnar.FallbackTags.add import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.extension.columnar.validator.Validator @@ -34,7 +34,6 @@ import org.apache.gluten.test.TestStats import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.SparkPlan -import FallbackTag.{Appendable, Converter} import com.google.common.collect.Lists sealed trait ValidationResult { @@ -83,7 +82,11 @@ object ValidationResult { } /** Every Gluten Operator should extend this trait. */ -trait GlutenPlan extends SparkPlan with Convention.KnownBatchType with LogLevelUtil { +trait GlutenPlan + extends SparkPlan + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater + with LogLevelUtil { protected lazy val enableNativeValidation = glutenConf.enableNativeValidation protected def glutenConf: GlutenConfig = GlutenConfig.getConf @@ -126,17 +129,20 @@ trait GlutenPlan extends SparkPlan with Convention.KnownBatchType with LogLevelU } } - final override def batchType(): Convention.BatchType = { - if (!supportsColumnar) { - return Convention.BatchType.None - } - val batchType = batchType0() - assert(batchType != Convention.BatchType.None) - batchType + final override val supportsColumnar: Boolean = { + batchType() != Convention.BatchType.None + } + + override def batchType(): Convention.BatchType = { + BackendsApiManager.getSettings.primaryBatchType + } + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None } - protected def batchType0(): Convention.BatchType = { - Backend.get().defaultBatchType + override def rowType0(): Convention.RowType = { + Convention.RowType.None } protected def doValidateInternal(): ValidationResult = ValidationResult.succeeded diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala index a199b5920ca4..e11c6139547a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.extension.columnar +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, Transitions} import org.apache.gluten.utils.PlanUtil @@ -106,7 +107,8 @@ object MiscColumnarRules { private def toColumnarBroadcastExchange( exchange: BroadcastExchangeExec): ColumnarBroadcastExchangeExec = { - val newChild = Transitions.toBackendBatchPlan(exchange.child) + val newChild = + Transitions.toBatchPlan(exchange.child, BackendsApiManager.getSettings.primaryBatchType) ColumnarBroadcastExchangeExec(exchange.mode, newChild) } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala index e1c8b6f41f44..44ed81f56581 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala @@ -20,7 +20,7 @@ import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} import org.apache.gluten.extension.columnar.FallbackTags.add -import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike, Transitions} +import org.apache.gluten.extension.columnar.transition.{BackendTransitions, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.catalyst.rules.Rule @@ -32,6 +32,9 @@ import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.exchange.Exchange + + + // format: off /** * Note, this rule should only fallback to row-based plan if there is no harm. @@ -226,7 +229,7 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP case _ => } - val planWithTransitions = Transitions.insertTransitions(originalPlan, outputsColumnar) + val planWithTransitions = BackendTransitions.insert(originalPlan, outputsColumnar) planWithTransitions } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala new file mode 100644 index 000000000000..86d4b40d55d8 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.transition + +import org.apache.gluten.backendsapi.BackendsApiManager + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +case class InsertBackendTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { + def apply(plan: SparkPlan): SparkPlan = { + InsertTransitions + .create(outputsColumnar, BackendsApiManager.getSettings.primaryBatchType) + .apply(plan) + } +} + +object BackendTransitions { + def insert(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { + InsertBackendTransitions(outputsColumnar)(plan) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala index d55733fe4e97..01a4380a148f 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala @@ -121,8 +121,6 @@ case class ColumnarBroadcastExchangeExec(mode: BroadcastMode, child: SparkPlan) @transient private val timeout: Long = SQLConf.get.broadcastTimeout - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index 5ea5e4159dda..9ec078e003da 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution._ import org.apache.gluten.extension.columnar.transition.Convention @@ -178,9 +177,9 @@ case class ColumnarInputAdapter(child: SparkPlan) extends InputAdapterGenerateTreeStringShim with Convention.KnownBatchType { override def output: Seq[Attribute] = child.output - override def supportsColumnar: Boolean = true + override val supportsColumnar: Boolean = true override def batchType(): Convention.BatchType = - Backend.get().defaultBatchType + BackendsApiManager.getSettings.primaryBatchType override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() override def outputPartitioning: Partitioning = child.outputPartitioning diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala index 4f62377b09e3..d4b33be292df 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala @@ -126,7 +126,6 @@ case class ColumnarShuffleExchangeExec( override def nodeName: String = "ColumnarExchange" - override def supportsColumnar: Boolean = true override def numMappers: Int = shuffleDependency.rdd.getNumPartitions override def numPartitions: Int = shuffleDependency.partitioner.numPartitions diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala index fcd82d8c1987..25d6c4ed61e7 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.Convention.{KnownRowType, RowType} +import org.apache.gluten.extension.columnar.transition.Convention.RowType import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.TaskContext @@ -45,18 +44,17 @@ abstract class ColumnarWriteFilesExec protected ( override val right: SparkPlan) extends BinaryExecNode with GlutenPlan - with KnownChildrenConventions - with KnownRowType + with ConventionReq.KnownChildrenConventions with ColumnarWriteFilesExec.ExecuteWriteCompatible { val child: SparkPlan = left override lazy val references: AttributeSet = AttributeSet.empty - override def supportsColumnar: Boolean = true - override def requiredChildrenConventions(): Seq[ConventionReq] = { - List(ConventionReq.backendBatch) + List( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) } /** @@ -69,7 +67,7 @@ abstract class ColumnarWriteFilesExec protected ( * * Since https://github.com/apache/incubator-gluten/pull/6745. */ - override def rowType(): RowType = { + override def rowType0(): RowType = { RowType.VanillaRow } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala index 917f6c03dfe0..fb42c55ba0a7 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala @@ -20,7 +20,7 @@ import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.gluten.execution.datasource.GlutenFormatFactory import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.Transitions +import org.apache.gluten.extension.columnar.transition.{Convention, Transitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -61,6 +61,8 @@ case class FakeRowAdaptor(child: SparkPlan) override def output: Seq[Attribute] = child.output + override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow + override protected def doExecute(): RDD[InternalRow] = { doExecuteColumnar().map(cb => new FakeRow(cb)) } @@ -74,7 +76,7 @@ case class FakeRowAdaptor(child: SparkPlan) if (child.supportsColumnar) { child.executeColumnar() } else { - val r2c = Transitions.toBackendBatchPlan(child) + val r2c = Transitions.toBatchPlan(child, BackendsApiManager.getSettings.primaryBatchType) r2c.executeColumnar() } } diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala index 9712bd2c219f..5daca9bede9e 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala @@ -32,7 +32,7 @@ class TransitionSuite extends SharedSparkSession { test("Trivial C2R") { val in = BatchLeaf(TypeA) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == BatchToRow(TypeA, BatchLeaf(TypeA))) } @@ -40,7 +40,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2R") { val in = RowUnary(BatchLeaf(TypeA)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == RowUnary(BatchToRow(TypeA, BatchLeaf(TypeA)))) } @@ -48,7 +48,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert R2C") { val in = BatchUnary(TypeA, RowLeaf()) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == BatchToRow(TypeA, BatchUnary(TypeA, RowToBatch(TypeA, RowLeaf())))) } @@ -56,7 +56,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2R2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeB)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert( out == BatchToRow( @@ -67,7 +67,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeC)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert( out == BatchToRow( @@ -79,7 +79,7 @@ class TransitionSuite extends SharedSparkSession { val in = BatchUnary(TypeA, BatchLeaf(TypeD)) assertThrows[GlutenException] { ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } } } @@ -116,8 +116,7 @@ object TransitionSuite extends TransitionSuiteBase { case class RowToBatch(toBatchType: Convention.BatchType, override val child: SparkPlan) extends RowToColumnarTransition with GlutenPlan { - override def supportsColumnar: Boolean = true - override protected def batchType0(): Convention.BatchType = toBatchType + override def batchType(): Convention.BatchType = toBatchType override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) override protected def doExecute(): RDD[InternalRow] = diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala index d82cc3aac9f1..43805b3d659b 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala @@ -24,22 +24,18 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.{BinaryExecNode, LeafExecNode, SparkPlan, UnaryExecNode} trait TransitionSuiteBase { - case class BatchLeaf(override val batchType0: Convention.BatchType) + case class BatchLeaf(override val batchType: Convention.BatchType) extends LeafExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = List.empty } - case class BatchUnary( - override val batchType0: Convention.BatchType, - override val child: SparkPlan) + case class BatchUnary(override val batchType: Convention.BatchType, override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) @@ -50,12 +46,11 @@ trait TransitionSuiteBase { } case class BatchBinary( - override val batchType0: Convention.BatchType, + override val batchType: Convention.BatchType, override val left: SparkPlan, override val right: SparkPlan) extends BinaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true override protected def withNewChildrenInternal( newLeft: SparkPlan, diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 7abe8228facf..7d4315e8d707 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.RemoveFallbackTagRule import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -44,7 +44,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -61,7 +61,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -79,7 +79,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -99,7 +99,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -119,7 +119,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -199,17 +199,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index d03619a2e07a..e8cc7898c2d4 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -43,7 +43,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -60,7 +60,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -78,7 +78,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -98,7 +98,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -118,7 +118,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -228,17 +228,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index d03619a2e07a..e8cc7898c2d4 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -43,7 +43,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -60,7 +60,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -78,7 +78,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -98,7 +98,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -118,7 +118,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -228,17 +228,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index 640b8cdc6f8f..cdfb63aa2074 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.benchmarks import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} import org.apache.gluten.extension.columnar.transition.Transitions import org.apache.gluten.jni.JniLibLoader @@ -125,7 +126,7 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { // generate ColumnarToRow val columnarToRowPlan = - Transitions.toBackendBatchPlan(newWholeStage) + Transitions.toBatchPlan(newWholeStage, BackendsApiManager.getSettings.primaryBatchType) val newWholeStageRDD = newWholeStage.executeColumnar() val newColumnarToRowRDD = columnarToRowPlan.execute() diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index f43652a7d4ee..1d45e8a672e3 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -44,7 +44,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -61,7 +61,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -79,7 +79,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -99,7 +99,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -119,7 +119,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -229,17 +229,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 2bdde3b4aaa0..a3bd5079b016 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -71,10 +71,9 @@ abstract class AbstractFileSourceScanExec( disableBucketedScan: Boolean = false) extends DataSourceScanExec { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 6b3d679fcdd4..c885f0cf44b3 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -75,10 +75,9 @@ abstract class AbstractFileSourceScanExec( lazy val metadataColumns: Seq[AttributeReference] = output.collect { case FileSourceMetadataAttribute(attr) => attr } - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 5e751861861a..53ea6f543a95 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -67,15 +67,9 @@ abstract class AbstractFileSourceScanExec( override val disableBucketedScan: Boolean = false) extends FileSourceScanLike { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - val conf = relation.sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 32cbd435b013..c8dbcc2fed4f 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -67,15 +67,9 @@ abstract class AbstractFileSourceScanExec( override val disableBucketedScan: Boolean = false) extends FileSourceScanLike { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - val conf = relation.sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { From e3682fdafb505af87ad1c1605908c5ec30463a9e Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Mon, 25 Nov 2024 16:22:40 +0800 Subject: [PATCH 141/211] [GLUTEN-8010][CORE] Don't generate native metrics if transformer don't generate relNode (#8011) --- .../org/apache/gluten/metrics/MetricsUtil.scala | 3 +++ .../BasicPhysicalOperatorTransformer.scala | 15 ++++++++++----- .../gluten/execution/ExpandExecTransformer.scala | 12 ++++++++---- .../gluten/execution/SortExecTransformer.scala | 12 ++++++++---- .../gluten/execution/WholeStageTransformer.scala | 3 +++ .../gluten/execution/WindowExecTransformer.scala | 12 ++++++++---- .../gluten/substrait/SubstraitContext.scala | 13 ------------- 7 files changed, 40 insertions(+), 30 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala index e1e0f7c11a09..7d81467e978f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala @@ -38,6 +38,9 @@ object MetricsUtil extends Logging { j.metricsUpdater(), // must put the buildPlan first Seq(treeifyMetricsUpdaters(j.buildPlan), treeifyMetricsUpdaters(j.streamedPlan))) + case t: TransformSupport if t.metricsUpdater() == MetricsUpdater.None => + assert(t.children.size == 1, "MetricsUpdater.None can only be used on unary operator") + treeifyMetricsUpdaters(t.children.head) case t: TransformSupport => MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters)) case _ => diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index dbe667ebb2aa..2830ef404c0e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -61,8 +61,13 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP case _ => false } - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = getRemainingCondition == null + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genFilterTransformerMetricsUpdater(metrics) + } def getRelNode( context: SubstraitContext, @@ -149,15 +154,15 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val remainingCondition = getRemainingCondition - val operatorId = context.nextOperatorId(this.nodeName) - if (remainingCondition == null) { + if (isNoop) { // The computing for this filter is not needed. - context.registerEmptyRelToOperator(operatorId) // Since some columns' nullability will be removed after this filter, we need to update the // outputAttributes of child context. return TransformContext(output, childCtx.root) } + + val operatorId = context.nextOperatorId(this.nodeName) + val remainingCondition = getRemainingCondition val currRel = getRelNode( context, remainingCondition, diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala index b600175b2826..c6936daaffe5 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala @@ -48,8 +48,13 @@ case class ExpandExecTransformer( AttributeSet.fromAttributeSets(projections.flatten.map(_.references)) } - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = projections == null || projections.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genExpandTransformerMetricsUpdater(metrics) + } // The GroupExpressions can output data with arbitrary partitioning, so set it // as UNKNOWN partitioning @@ -112,13 +117,12 @@ case class ExpandExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (projections == null || projections.isEmpty) { + if (isNoop) { // The computing for this Expand is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getRelNode(context, projections, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Expand Rel should be valid") diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala index c62a30b84632..6f9564e6d54f 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala @@ -44,8 +44,13 @@ case class SortExecTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genSortTransformerMetrics(sparkContext) - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = sortOrder == null || sortOrder.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genSortTransformerMetricsUpdater(metrics) + } override def output: Seq[Attribute] = child.output @@ -103,13 +108,12 @@ case class SortExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (sortOrder == null || sortOrder.isEmpty) { + if (isNoop) { // The computing for this project is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getRelNode(context, sortOrder, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Sort Rel should be valid") diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index e8a42883a54f..6414b67a8092 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -98,6 +98,9 @@ trait TransformSupport extends GlutenPlan { Seq(plan.executeColumnar()) } } + + // When true, it will not generate relNode, nor will it generate native metrics. + def isNoop: Boolean = false } trait LeafTransformSupport extends TransformSupport with LeafExecNode { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala index 7b9e2865f883..28d780992492 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala @@ -51,8 +51,13 @@ case class WindowExecTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetrics(sparkContext) - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = windowExpression == null || windowExpression.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetricsUpdater(metrics) + } override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) @@ -177,13 +182,12 @@ case class WindowExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (windowExpression == null || windowExpression.isEmpty) { + if (isNoop) { // The computing for this operator is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getWindowRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Rel should be valid") diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala b/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala index 79148d9f3093..1ceb2d4155ab 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala @@ -112,19 +112,6 @@ class SubstraitContext extends Serializable { id } - /** - * Register empty rel list to certain operator id. Used when the computing of a Spark transformer - * is omitted. - * @param operatorId - * operator id - */ - def registerEmptyRelToOperator(operatorId: JLong): Unit = { - if (!operatorToRelsMap.containsKey(operatorId)) { - val rels = new JArrayList[JLong]() - operatorToRelsMap.put(operatorId, rels) - } - } - /** * Return the registered map. * @return From cc25aa26a455b6708d3b9a0e9b416663d3e32531 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Mon, 25 Nov 2024 17:54:05 +0800 Subject: [PATCH 142/211] [VL] Bump jemalloc version and update relevant documents (#8035) --- cpp/CMake/Buildjemalloc.cmake | 4 +- docs/developers/HowTo.md | 7 ++++ .../ProfileMemoryOfGlutenWithVelox.md | 37 ++++++++++--------- docs/get-started/build-guide.md | 2 +- 4 files changed, 31 insertions(+), 19 deletions(-) diff --git a/cpp/CMake/Buildjemalloc.cmake b/cpp/CMake/Buildjemalloc.cmake index 5491bd829533..67a0cc5f1167 100644 --- a/cpp/CMake/Buildjemalloc.cmake +++ b/cpp/CMake/Buildjemalloc.cmake @@ -22,7 +22,7 @@ macro(build_jemalloc) if(DEFINED ENV{GLUTEN_JEMALLOC_URL}) set(JEMALLOC_SOURCE_URL "$ENV{GLUTEN_JEMALLOC_URL}") else() - set(JEMALLOC_BUILD_VERSION "5.2.1") + set(JEMALLOC_BUILD_VERSION "5.3.0") set(JEMALLOC_SOURCE_URL "https://github.com/jemalloc/jemalloc/releases/download/${JEMALLOC_BUILD_VERSION}/jemalloc-${JEMALLOC_BUILD_VERSION}.tar.bz2" "https://github.com/ursa-labs/thirdparty/releases/download/latest/jemalloc-${JEMALLOC_BUILD_VERSION}.tar.bz2" @@ -47,6 +47,8 @@ macro(build_jemalloc) # for dynamically linking. "--without-export" "--disable-cxx" "--disable-libdl" + # Enable heap profiling and leak detection functionality. + "--enable-prof" # For fixing an issue when loading native lib: cannot allocate memory in # static TLS block. "--disable-initial-exec-tls" diff --git a/docs/developers/HowTo.md b/docs/developers/HowTo.md index c4b1a03a36d8..22ad3e30efc7 100644 --- a/docs/developers/HowTo.md +++ b/docs/developers/HowTo.md @@ -122,6 +122,13 @@ gdb ${GLUTEN_HOME}/cpp/build/releases/libgluten.so 'core-Executor task l-2000883 ``` - the `core-Executor task l-2000883-1671542526` represents the core file name. +# How to use jemalloc for Gluten native engine + +Currently, we have no dedicated memory allocator implemented by jemalloc. User can set environment variable `LD_PRELOAD` for lib jemalloc +to let it override the corresponding C standard functions entirely. It may help alleviate OOM issues. + +`spark.executorEnv.LD_PREALOD=/path/to/libjemalloc.so` + # How to run TPC-H on Velox backend Now, both Parquet and DWRF format files are supported, related scripts and files are under the directory of `${GLUTEN_HOME}/backends-velox/workload/tpch`. diff --git a/docs/developers/ProfileMemoryOfGlutenWithVelox.md b/docs/developers/ProfileMemoryOfGlutenWithVelox.md index 480755cb99c5..1f57f809218c 100644 --- a/docs/developers/ProfileMemoryOfGlutenWithVelox.md +++ b/docs/developers/ProfileMemoryOfGlutenWithVelox.md @@ -5,13 +5,14 @@ nav_order: 8 has_children: true parent: /developer-overview/ --- -Gluten offloads most of computation to native engine. We can use [gperftools](https://github.com/gperftools/gperftools) or [jemalloc](https://github.com/jemalloc/jemalloc) to analyze the offheap memory and cpu profile. +Gluten offloads most of Spark SQL execution to native engine. We can use [gperftools](https://github.com/gperftools/gperftools) or [jemalloc](https://github.com/jemalloc/jemalloc) +to analyze the offheap memory and cpu profile. -# Profiling using gperftools +# Profile with gperftools `gperftools` is a collection of a high-performance multi-threaded malloc() implementation, plus some pretty nifty performance analysis -tools, see more: https://github.com/gperftools/gperftools/wiki +tools, see more: https://github.com/gperftools/gperftools/wiki. ## Build and install gperftools @@ -29,10 +30,10 @@ Then we can find the tcmalloc libraries in `$GPERFTOOLS_HOME/.lib`. ## Run Gluten with gperftools -Use `--file` or `spark.files` to upload tcmalloc library. +Configure `--files` or `spark.files` for Spark. ``` ---file /path/to/gperftools/libtcmalloc_and_profiler.so +--files /path/to/gperftools/libtcmalloc_and_profiler.so or spark.files /path/to/gperftools/libtcmalloc_and_profiler.so ``` @@ -48,14 +49,14 @@ spark.executorEnv.LD_PRELOAD ./libtcmalloc_and_profiler.so spark.executorEnv.HEAPPROFILE /tmp/gluten_heap_perf_${CONTAINER_ID} ``` -Finally, profiling files starting with `/tmp/gluten_heap_perf_${CONTAINER_ID}` will be generated in each spark executor. +Finally, profiling files prefixed with `/tmp/gluten_heap_perf_${CONTAINER_ID}` will be generated for each spark executor. -## Analyze output profiles +## Analyze profiling output -Prepare the required native libraries. We can extract the gluten and velox libraries from gluten bundle jar. (Maybe also need dependency libraries for non-static compilation) +Prepare the required native libraries. Assume static build is used for Gluten, there is no other shared dependency libs. ```bash -jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar libvelox.so libgluten.so +jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar relative/path/to/libvelox.so ralative/path/to/libgluten.so mv libvelox.so libgluten.so /path/to/gluten_lib_prefix ``` @@ -82,9 +83,11 @@ Result like: **\*\*** Get more help from https://github.com/gperftools/gperftools/wiki#documentation. -# Profiling using jemalloc +# Profile with jemalloc -`jemalloc` is a general purpose malloc(3) implementation that emphasizes fragmentation avoidance and scalable concurrency support. We can also use it to analyze Gluten performance. Getting Started with `jemalloc`: https://github.com/jemalloc/jemalloc/wiki/Getting-Started. +`jemalloc` is a general purpose malloc(3) implementation that emphasizes fragmentation +avoidance and scalable concurrency support. We can also use it to analyze Gluten performance. +Getting Started with `jemalloc`: https://github.com/jemalloc/jemalloc/wiki/Getting-Started. ## Build and install jemalloc @@ -99,10 +102,10 @@ Then we can find the jemalloc library in `$JEMALLOC_HOME/.lib`. ## Run Gluten with jemalloc -Use `--file` or `spark.files` to upload jemalloc library. +Configure `--files` or `spark.files` for Spark. ``` ---file /path/to/jemalloc/libjemalloc.so +--files /path/to/jemalloc/libjemalloc.so or spark.files /path/to/jemalloc/libjemalloc.so ``` @@ -114,14 +117,14 @@ spark.executorEnv.LD_PRELOAD ./libjemalloc.so spark.executorEnv.MALLOC_CONF prof:true,lg_prof_interval:30,prof_prefix:/tmp/gluten_heap_perf ``` -Finally, profiling files starting with `/tmp/gluten_heap_perf.${PID}` will be generated in each spark executor. +Finally, profiling files prefixed with `/tmp/gluten_heap_perf.${PID}` will be generated for each spark executor. -## Analyze output profiles +## Analyze profiling output -Prepare the required native libraries. We can extract the gluten and velox libraries from gluten bundle jar. (Maybe also need dependency libraries for non-static compilation) +Prepare the required native libraries. Assume static build is used for Gluten, so there is no other shared dependency libs. ```bash -jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar libvelox.so libgluten.so +jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar relative/path/to/libvelox.so relative/path/to/libgluten.so mv libvelox.so libgluten.so /path/to/gluten_lib_prefix ``` diff --git a/docs/get-started/build-guide.md b/docs/get-started/build-guide.md index 3c0c521e2181..cbaad979fb90 100644 --- a/docs/get-started/build-guide.md +++ b/docs/get-started/build-guide.md @@ -14,7 +14,7 @@ Please set them via `--`, e.g. `--build_type=Release`. | build_tests | Build gluten cpp tests. | OFF | | build_examples | Build udf example. | OFF | | build_benchmarks | Build gluten cpp benchmarks. | OFF | -| enable_jemalloc_stats | Print jemalloc stats for debugging. | OFF | +| enable_jemalloc_stats | Print jemalloc stats for debugging. | OFF | | build_protobuf | Build protobuf lib. | OFF | | enable_qat | Enable QAT for shuffle data de/compression. | OFF | | enable_iaa | Enable IAA for shuffle data de/compression. | OFF | From 9f69d217e6cd9342cc31fd3a9d20a8065e30890c Mon Sep 17 00:00:00 2001 From: Yuan Date: Tue, 26 Nov 2024 07:34:39 +0800 Subject: [PATCH 143/211] [MISC] Velox maintainers as triage member(collaborators) (#8037) This patch adds Velox maintainer Deepak and Pedro as triage member, so they could help to manage the issues related with Velox easier. Signed-off-by: Yuan Zhou --- .asf.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index 76429ad286de..dafb78888389 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -27,6 +27,9 @@ github: - spark-sql - vectorization - velox + collaborators: + - majetideepak + - pedroerp enabled_merge_buttons: squash: true merge: false From bcb768bde27688f0083861b566b787f07a2ffd40 Mon Sep 17 00:00:00 2001 From: Suraj Naik Date: Tue, 26 Nov 2024 08:54:51 +0530 Subject: [PATCH 144/211] [VL] Clean up duplicate CMake code for setting CMAKE_CXX_FLAGS (#8034) --- cpp/CMakeLists.txt | 21 ++++++++++++++++++++- cpp/core/CMakeLists.txt | 22 ---------------------- cpp/velox/CMakeLists.txt | 18 ------------------ 3 files changed, 20 insertions(+), 41 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 67fb9ec721ac..e0ab141a1f96 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -98,7 +98,9 @@ set(KNOWN_WARNINGS -Wno-error=unused-function \ -Wno-error=unused-variable \ -Wno-strict-aliasing \ - -Wno-ignored-qualifiers") + -Wno-ignored-qualifiers \ + -Wno-deprecated-declarations \ + -Wno-attributes") if("${CMAKE_CXX_COMPILER_ID}" STREQUAL "GNU") set(KNOWN_WARNINGS "-Wno-error=unused-but-set-variable \ @@ -141,8 +143,25 @@ if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin") add_compile_definitions(_GNU_SOURCE) endif() +if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") +endif() + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${KNOWN_WARNINGS}") +# Keep same compile option with Velox. +execute_process( + COMMAND + bash -c + "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" + OUTPUT_VARIABLE SCRIPT_CXX_FLAGS + RESULT_VARIABLE COMMAND_STATUS) +if(COMMAND_STATUS EQUAL "1") + message(FATAL_ERROR "Unable to determine compiler flags!") +endif() + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") + # # Dependencies # diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 88b0fd883e5c..5a5eeac35480 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -22,28 +22,6 @@ include(FindPkgConfig) include(GNUInstallDirs) include(CheckCXXCompilerFlag) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") -if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") -endif() - -if(NOT DEFINED VELOX_HOME) - set(VELOX_HOME ${GLUTEN_HOME}/ep/build-velox/build/velox_ep) - message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") -endif() - -# Keep same compile option with Velox. -execute_process( - COMMAND - bash -c - "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" - OUTPUT_VARIABLE SCRIPT_CXX_FLAGS - RESULT_VARIABLE COMMAND_STATUS) -if(COMMAND_STATUS EQUAL "1") - message(FATAL_ERROR "Unable to determine compiler flags!") -endif() -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") set(BOOST_MIN_VERSION "1.42.0") diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 9e110853eb6e..4503d5947d78 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -23,12 +23,6 @@ include(GNUInstallDirs) include(CheckCXXCompilerFlag) include(FindPackageHandleStandardArgs) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") -if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") -endif() - set(SYSTEM_LIB_PATH "/usr/lib" CACHE PATH "System Lib dir") @@ -62,18 +56,6 @@ if(NOT DEFINED VELOX_HOME) message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") endif() -# Keep same compile option with Velox. -execute_process( - COMMAND - bash -c - "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" - OUTPUT_VARIABLE SCRIPT_CXX_FLAGS - RESULT_VARIABLE COMMAND_STATUS) -if(COMMAND_STATUS EQUAL "1") - message(FATAL_ERROR "Unable to determine compiler flags!") -endif() -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") - message("Velox module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") # User can specify VELOX_BUILD_PATH, if Velox are built elsewhere. From 4be7bf17a6a031306759c1dfba95e4ef7ca9db1c Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 26 Nov 2024 11:37:26 +0800 Subject: [PATCH 145/211] [GLUTEN-7741][VL] Fix deprecated actions/upload-artifact version issue when building bundle package (#8017) V2 is deprecated: This request has been automatically failed because it uses a deprecated version of `actions/upload-artifact: v2`. Learn more: https://github.blog/changelog/2024-02-13-deprecation-notice-v1-and-v2-of-the-artifact-actions/ --- .github/workflows/build_bundle_package.yml | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build_bundle_package.yml b/.github/workflows/build_bundle_package.yml index a28720656651..9b94d302f962 100644 --- a/.github/workflows/build_bundle_package.yml +++ b/.github/workflows/build_bundle_package.yml @@ -54,13 +54,13 @@ jobs: cd $GITHUB_WORKSPACE/ bash dev/ci-velox-buildstatic-centos-7.sh - name: Upload native libs - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: path: ./cpp/build/releases/ name: velox-native-lib-${{github.sha}} retention-days: 1 - name: Upload Artifact Arrow Jar - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: path: /root/.m2/repository/org/apache/arrow/ name: velox-arrow-jar-centos-7-${{github.sha}} @@ -72,17 +72,19 @@ jobs: steps: - uses: actions/checkout@v2 - name: Download All Artifacts - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: velox-native-lib-${{github.sha}} path: ./cpp/build/releases - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: velox-arrow-jar-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - name: Setup java and maven run: | + sed -i 's/mirrorlist/#mirrorlist/g' /etc/yum.repos.d/CentOS-* && \ + sed -i 's|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g' /etc/yum.repos.d/CentOS-* && \ yum update -y && yum install -y java-1.8.0-openjdk-devel wget && \ wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz && \ tar -xvf apache-maven-3.8.8-bin.tar.gz && \ @@ -94,7 +96,7 @@ jobs: export PATH=${PATH}:${MAVEN_HOME}/bin && \ mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -Puniffle -DskipTests -Dmaven.source.skip - name: Upload bundle package - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: gluten-velox-bundle-package path: package/target/gluten-velox-bundle-*.jar From 1d1366a4229e0f0c9ead8ee24d7039ae8f99db65 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 26 Nov 2024 13:26:57 +0800 Subject: [PATCH 146/211] [VL] vcpkg: Broken libelf mirror (#8047) --- dev/vcpkg/ports/libelf/portfile.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/vcpkg/ports/libelf/portfile.cmake b/dev/vcpkg/ports/libelf/portfile.cmake index 8e9ea0c71fe1..d1b347d69352 100644 --- a/dev/vcpkg/ports/libelf/portfile.cmake +++ b/dev/vcpkg/ports/libelf/portfile.cmake @@ -1,5 +1,5 @@ vcpkg_download_distfile(ARCHIVE - URLS "https://ftp.osuosl.org/pub/blfs/conglomeration/libelf/libelf-0.8.13.tar.gz" + URLS "https://fossies.org/linux/misc/old/libelf-0.8.13.tar.gz" FILENAME "libelf-0.8.13.tar.gz" SHA512 d2a4ea8ccc0bbfecac38fa20fbd96aefa8e86f8af38691fb6991cd9c5a03f587475ecc2365fc89a4954c11a679d93460ee9a5890693112f6133719af3e6582fe ) From 4dfdfd77b52f2f98fa0cf32eca143b47e4bd11b5 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 26 Nov 2024 16:25:00 +0800 Subject: [PATCH 147/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_26) (#8042) Upstream Velox's New Commits: f33b40da0 by zhli1142015, feat(ABFS): Add dependency for ABFS Auth support (11633) 22392b175 by duanmeng, feat: Specifiy target tracing driver IDs (11560) 2b5e9f1a5 by Yenda Li, fix:Support hashing ipaddress (11640) 97bd72825 by Xiaoxuan Meng, feat: Add skewed partition balancer (11635) 97e47daf5 by Jacob Wujciak-Jens, build(ci): Fix scheduled.yml syntax (11652) e27867ca2 by Jia Ke, fix: Move the thread local parameter from method to member parameter in HdfsReadFile (11580) --- .github/workflows/velox_backend.yml | 22 ++++++++++++++-------- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index d7445d1a2752..56b43975acee 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -56,9 +56,8 @@ concurrency: jobs: build-native-lib-centos-7: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -68,10 +67,17 @@ jobs: ccache-centos7-release-default - name: Build Gluten native libraries run: | - df -a - cd $GITHUB_WORKSPACE/ - bash dev/ci-velox-buildstatic-centos-7.sh - ccache -s + docker pull apache/gluten:vcpkg-centos-7 + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + df -a + cd /work + export CCACHE_DIR=/work/.ccache + bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s + mkdir -p /work/.m2/repository/org/apache/arrow/ + cp -r /root/.m2/repository/org/apache/arrow/* /work/.m2/repository/org/apache/arrow/ + " + - name: "Save ccache" uses: actions/cache/save@v3 id: ccache @@ -85,7 +91,7 @@ jobs: - uses: actions/upload-artifact@v3 with: name: arrow-jars-centos-7-${{github.sha}} - path: /root/.m2/repository/org/apache/arrow/ + path: .m2/repository/org/apache/arrow/ run-tpc-test-ubuntu: needs: build-native-lib-centos-7 @@ -163,7 +169,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ "centos:7", "centos:8" ] + os: [ "centos:8" ] #TODO: add centos7 tests back with docker run spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] java: [ "java-8", "java-11", "java-17" ] # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index d95f742a5515..bcab2e31dbef 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_24 +VELOX_BRANCH=2024_11_26 VELOX_HOME="" OS=`uname -s` From efd2cbde3064631179dede8ad2b7a0e96d426d06 Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Tue, 26 Nov 2024 19:53:13 +0800 Subject: [PATCH 148/211] [GLUTEN-7896][CH]Fix to_date diff for time parser policy config (#7923) * fix pre-projection not take effect * Fix time_parser_plicy set legacy * fix * fix 11 * add test * fix ci test * Fix code bug * fix review * modify test --- ...enClickHouseTPCHSaltNullParquetSuite.scala | 18 ++- cpp-ch/local-engine/Common/CHUtil.cpp | 5 + cpp-ch/local-engine/Common/CHUtil.h | 1 + .../CommonScalarFunctionParser.cpp | 1 - .../scalar_function_parser/getTimestamp.cpp | 23 ++++ .../scalar_function_parser/getTimestamp.h | 106 ++++++++++++++++++ .../scalar_function_parser/unixTimestamp.cpp | 14 +-- .../org/apache/gluten/GlutenConfig.scala | 3 +- 8 files changed, 160 insertions(+), 11 deletions(-) create mode 100644 cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp create mode 100644 cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 5d7bcf324ae8..4a2b7040fa5e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -2192,7 +2192,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } } - test("GLUTEN-3135: Bug fix to_date") { + test("GLUTEN-3135/GLUTEN-7896: Bug fix to_date") { val create_table_sql = """ | create table test_tbl_3135(id bigint, data string) using parquet @@ -2209,13 +2209,27 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |(7, '1970-01-01 00:00:00'), |(8, '2024-3-2'), |(9, '2024-03-2'), - |(10, '2024-03') + |(10, '2024-03'), + |(11, '2024-03-02 11:22:33') |""".stripMargin spark.sql(create_table_sql) spark.sql(insert_data_sql) val select_sql = "select id, to_date(data) from test_tbl_3135" compareResultsAgainstVanillaSpark(select_sql, true, { _ => }) + + withSQLConf(("spark.sql.legacy.timeParserPolicy" -> "corrected")) { + compareResultsAgainstVanillaSpark( + "select id, to_date('2024-03-2 11:22:33', 'yyyy-MM-dd') from test_tbl_3135 where id = 11", + true, + { _ => }) + } + withSQLConf(("spark.sql.legacy.timeParserPolicy" -> "legacy")) { + compareResultsAgainstVanillaSpark( + "select id, to_date(data, 'yyyy-MM-dd') from test_tbl_3135 where id = 11", + true, + { _ => }) + } spark.sql("drop table test_tbl_3135") } diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 03df93c851e3..8fef52e50a68 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -762,6 +762,11 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ settings.set(key, toField(key, value)); LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Set settings key:{} value:{}", key, value); } + else if (key == TIMER_PARSER_POLICY) + { + settings.set(key, value); + LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Set settings key:{} value:{}", key, value); + } } /// Finally apply some fixed kvs to settings. diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index cff69090ee31..a5fb24f6afee 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -40,6 +40,7 @@ namespace local_engine static const String MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE = "mergetree.insert_without_local_storage"; static const String MERGETREE_MERGE_AFTER_INSERT = "mergetree.merge_after_insert"; static const std::string DECIMAL_OPERATIONS_ALLOW_PREC_LOSS = "spark.sql.decimalOperations.allowPrecisionLoss"; +static const std::string TIMER_PARSER_POLICY = "spark.sql.legacy.timeParserPolicy"; static const std::unordered_set BOOL_VALUE_SETTINGS{ MERGETREE_MERGE_AFTER_INSERT, MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS}; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp index d6584267455f..ec8b4e0d12bf 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp @@ -57,7 +57,6 @@ REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Not, not, not ); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Xor, xor, xor); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Cast, cast, CAST); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(GetTimestamp, get_timestamp, parseDateTime64InJodaSyntaxOrNull); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Quarter, quarter, toQuarter); // math functions diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp new file mode 100644 index 000000000000..4724f820099a --- /dev/null +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include + +namespace local_engine +{ + static FunctionParserRegister register_get_timestamp; +} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h new file mode 100644 index 000000000000..5e32e00569f1 --- /dev/null +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} +} + + +namespace local_engine +{ +class FunctionParserGetTimestamp : public FunctionParser +{ +public: + explicit FunctionParserGetTimestamp(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + ~FunctionParserGetTimestamp() override = default; + + static constexpr auto name = "get_timestamp"; + String getName() const override { return name; } + + const ActionsDAG::Node * parse( + const substrait::Expression_ScalarFunction & substrait_func, + ActionsDAG & actions_dag) const override + { + /* + spark function: get_timestamp(expr, fmt) + 1. If timeParserPolicy is LEGACY + 1) fmt has 0 'S', ch function = parseDateTime64InJodaSyntaxOrNull(substr(expr,1,length(fmt)), fmt); + 2) fmt has 'S' more than 0, make the fmt has 3 'S', ch function = parseDateTime64InJodaSyntaxOrNull(expr, fmt) + 2. Else ch function = parseDateTime64InJodaSyntaxOrNull(expr, fmt) + */ + auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); + if (parsed_args.size() != 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires exactly two arguments", getName()); + const auto * expr_arg = parsed_args[0]; + const auto * fmt_arg = parsed_args[1]; + + const auto & args = substrait_func.arguments(); + bool fmt_string_literal = args[1].value().has_literal(); + String fmt; + if (fmt_string_literal) + { + const auto & literal_fmt_expr = args[1].value().literal(); + fmt_string_literal = literal_fmt_expr.has_string(); + fmt = fmt_string_literal ? literal_fmt_expr.string() : ""; + } + if (!fmt_string_literal) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The second of function {} must be const String.", name); + + UInt32 s_count = std::count(fmt.begin(), fmt.end(), 'S'); + String time_parser_policy = getContext()->getSettingsRef().has(TIMER_PARSER_POLICY) ? toString(getContext()->getSettingsRef().get(TIMER_PARSER_POLICY)) : ""; + boost::to_lower(time_parser_policy); + if (time_parser_policy == "legacy") + { + if (s_count == 0) + { + const auto * index_begin_node = addColumnToActionsDAG(actions_dag, std::make_shared(), 1); + const auto * index_end_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt.size()); + const auto * substr_node = toFunctionNode(actions_dag, "substringUTF8", {expr_arg, index_begin_node, index_end_node}); + const auto * fmt_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt); + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {substr_node, fmt_node}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + else if (s_count < 3) + fmt += String(3 - s_count, 'S'); + else + fmt = fmt.substr(0, fmt.size() - (s_count - 3)); + + const auto * fmt_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt); + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_node}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + else + { + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_arg}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + } +}; +} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp index 622237da9707..33997734c5e4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp @@ -17,7 +17,7 @@ #include #include - +#include namespace DB { @@ -34,10 +34,10 @@ namespace local_engine { template -class FunctionParserUnixTimestamp : public FunctionParser +class FunctionParserUnixTimestamp : public FunctionParserGetTimestamp { public: - explicit FunctionParserUnixTimestamp(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + explicit FunctionParserUnixTimestamp(ParserContextPtr parser_context_) : FunctionParserGetTimestamp(parser_context_) {} ~FunctionParserUnixTimestamp() override = default; static constexpr auto name = Name::name; @@ -60,13 +60,13 @@ class FunctionParserUnixTimestamp : public FunctionParser const auto * expr_arg = parsed_args[0]; const auto * fmt_arg = parsed_args[1]; auto expr_type = removeNullable(expr_arg->result_type); + if (isString(expr_type)) + return FunctionParserGetTimestamp::parse(substrait_func, actions_dag); + const DateLUTImpl * date_lut = &DateLUT::instance(); const auto * time_zone_node = addColumnToActionsDAG(actions_dag, std::make_shared(), date_lut->getTimeZone()); - const DB::ActionsDAG::Node * result_node = nullptr; - if (isString(expr_type)) - result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_arg, time_zone_node}); - else if (isDateOrDate32(expr_type)) + if (isDateOrDate32(expr_type)) result_node = toFunctionNode(actions_dag, "sparkDateToUnixTimestamp", {expr_arg, time_zone_node}); else if (isDateTime(expr_type) || isDateTime64(expr_type)) result_node = toFunctionNode(actions_dag, "toUnixTimestamp", {expr_arg, time_zone_node}); diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index e0d06ce6fc0f..2ccdcae99b5c 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -805,7 +805,8 @@ object GlutenConfig { SPARK_OFFHEAP_ENABLED, SESSION_LOCAL_TIMEZONE.key, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key, - SPARK_REDACTION_REGEX + SPARK_REDACTION_REGEX, + LEGACY_TIME_PARSER_POLICY.key ) nativeConfMap.putAll(conf.filter(e => keys.contains(e._1)).asJava) From 8d8ee3d54af359c747cea6310553f2e66d5f2122 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Tue, 26 Nov 2024 22:33:32 +0800 Subject: [PATCH 149/211] [CH]Daily Update Clickhouse Version (20241118) (#7968) What changes were proposed in this pull request? Manual rebase Clickhouse repo How was this patch tested? unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- cpp-ch/clickhouse.version | 4 +- .../CompactObjectStorageDiskTransaction.cpp | 39 +++++++++++++++++-- .../CompactObjectStorageDiskTransaction.h | 34 ++++++++++++++-- .../Disks/ObjectStorages/GlutenDiskHDFS.cpp | 2 +- .../Disks/ObjectStorages/GlutenDiskS3.cpp | 2 +- .../Functions/SparkFunctionRoundHalfUp.h | 29 +++++++++++++- .../Operator/GraceAggregatingTransform.cpp | 25 +++++++----- .../Operator/GraceAggregatingTransform.h | 6 +-- .../Parser/RelParsers/CrossRelParser.cpp | 9 ++++- .../Parser/RelParsers/JoinRelParser.cpp | 13 ++++++- .../MergeTree/SparkStorageMergeTree.cpp | 9 ++++- 11 files changed, 141 insertions(+), 31 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index beda11ec0a99..62a70f06c1e1 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241111 -CH_COMMIT=3f7e46d4e9e +CH_BRANCH=rebase_ch/20241118 +CH_COMMIT=7f22fe487c88d3b988ea82a5c34882da23ea6289 diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp index 5b1fe63a09c5..148a43580b96 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp @@ -28,6 +28,36 @@ bool isMetaDataFile(const std::string & path) return !path.ends_with("bin"); } +TemporaryWriteBufferWrapper::TemporaryWriteBufferWrapper( + const String & file_name_, const std::shared_ptr & data_buffer_) + : WriteBufferFromFileBase(data_buffer_->buffer().size(), data_buffer_->buffer().begin(), 0) + , file_name(file_name_) + , data_buffer(data_buffer_) +{ +} +void TemporaryWriteBufferWrapper::preFinalize() +{ + next(); +} + +void TemporaryWriteBufferWrapper::finalizeImpl() +{ + next(); + data_buffer->finalizeImpl(); +} + +void TemporaryWriteBufferWrapper::cancelImpl() noexcept +{ + data_buffer->cancelImpl(); +} + +void TemporaryWriteBufferWrapper::nextImpl() +{ + data_buffer->position() = position(); + data_buffer->next(); + BufferBase::set(data_buffer->buffer().begin(), data_buffer->buffer().size(), data_buffer->offset()); +} + void CompactObjectStorageDiskTransaction::commit() { auto metadata_tx = disk.getMetadataStorage()->createTransaction(); @@ -52,9 +82,9 @@ void CompactObjectStorageDiskTransaction::commit() [&](auto & item) { DB::DiskObjectStorageMetadata metadata(object_storage->getCommonKeyPrefix(), item.first); - DB::ReadBufferFromFilePRead read(item.second->getAbsolutePath()); + auto read = item.second->read(); int file_size = 0; - while (int count = read.readBig(buffer.data(), buffer.size())) + while (int count = read->readBig(buffer.data(), buffer.size())) { file_size += count; out.write(buffer.data(), count); @@ -98,12 +128,13 @@ std::unique_ptr CompactObjectStorageDiskTransaction "Don't support write file in different dirs, path {}, prefix path: {}", path, prefix_path); - auto tmp = std::make_shared(tmp_data); + auto tmp = std::make_shared(tmp_data.get()); files.emplace_back(path, tmp); auto tx = disk.getMetadataStorage()->createTransaction(); tx->createDirectoryRecursive(std::filesystem::path(path).parent_path()); tx->createEmptyMetadataFile(path); tx->commit(); - return std::make_unique(tmp->getAbsolutePath(), buf_size); + + return std::make_unique(path, tmp); } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h index becb5371aad2..0f95ae01ec2b 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h @@ -32,15 +32,41 @@ extern const int NOT_IMPLEMENTED; namespace local_engine { +class TemporaryWriteBufferWrapper : public DB::WriteBufferFromFileBase +{ +public: + TemporaryWriteBufferWrapper(const String & file_name_, const std::shared_ptr & data_buffer_); + + void sync() override { data_buffer->nextImpl(); } + + void preFinalize() override; + +protected: + void finalizeImpl() override; + void cancelImpl() noexcept override; + +private: + void nextImpl() override; + +public: + std::string getFileName() const override + { + return file_name; + } + +private: + String file_name; + std::shared_ptr data_buffer; +}; + class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { public: static inline const String PART_DATA_FILE_NAME = "part_data.gluten"; static inline const String PART_META_FILE_NAME = "part_meta.gluten"; - explicit CompactObjectStorageDiskTransaction(DB::IDisk & disk_, const DB::DiskPtr tmp_) + explicit CompactObjectStorageDiskTransaction(DB::IDisk & disk_, const DB::TemporaryDataOnDiskScopePtr tmp_) : disk(disk_), tmp_data(tmp_) { - chassert(!tmp_->isRemote()); } void commit() override; @@ -170,8 +196,8 @@ class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { private: DB::IDisk & disk; - DB::DiskPtr tmp_data; - std::vector>> files; + DB::TemporaryDataOnDiskScopePtr tmp_data; + std::vector>> files; String prefix_path = ""; }; } diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp index bd005132b9b9..fed23d7eef61 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp @@ -29,7 +29,7 @@ using namespace DB; DiskTransactionPtr GlutenDiskHDFS::createTransaction() { - return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()->getVolume()->getDisk()); + return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()); } void GlutenDiskHDFS::createDirectory(const String & path) diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp index b2a6bb523d3a..a180ebd7eaf0 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp @@ -31,7 +31,7 @@ namespace local_engine DB::DiskTransactionPtr GlutenDiskS3::createTransaction() { - return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()->getVolume()->getDisk()); + return std::make_shared(*this, QueryContext::globalContext()->getSharedTempDataOnDisk()); } std::unique_ptr GlutenDiskS3::readFile( diff --git a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h index 3e219b51a695..2dfd25772ee7 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h @@ -127,6 +127,31 @@ class BaseFloatRoundingHalfUpComputation } }; +template <> +class BaseFloatRoundingHalfUpComputation +{ +public: + using ScalarType = BFloat16; + using VectorType = BFloat16; + static const size_t data_count = 1; + + static VectorType load(const ScalarType * in) { return *in; } + static VectorType load1(const ScalarType in) { return in; } + static VectorType store(ScalarType * out, ScalarType val) { return *out = val;} + static VectorType multiply(VectorType val, VectorType scale) { return val * scale; } + static VectorType divide(VectorType val, VectorType scale) { return val / scale; } + template + static VectorType apply(VectorType val) + { + return BFloat16(std::roundf(static_cast(val))); + } + + static VectorType prepare(size_t scale) + { + return load1(BFloat16(static_cast(scale))); + } +}; + /** Implementation of low-level round-off functions for floating-point values. */ @@ -167,7 +192,7 @@ struct FloatRoundingHalfUpImpl template ? Vectorize::No : Vectorize::Yes #else Vectorize::No #endif @@ -219,7 +244,7 @@ struct DispatcherRoundingHalfUp { template using FunctionRoundingImpl = std::conditional_t< - std::is_floating_point_v, + std::is_floating_point_v || std::is_same_v, FloatRoundingHalfUpImpl, IntegerRoundingImpl>; diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp index adf25d13f29d..63dc3c3457f6 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp @@ -44,7 +44,7 @@ GraceAggregatingTransform::GraceAggregatingTransform( , aggregate_columns(params_->params.aggregates_size) , no_pre_aggregated(no_pre_aggregated_) , final_output(final_output_) - , tmp_data_disk(std::make_unique(context_->getTempDataOnDisk())) + , tmp_data_disk(context_->getTempDataOnDisk()) { output_header = params->getHeader(); auto config = GraceMergingAggregateConfig::loadFromContext(context); @@ -302,10 +302,13 @@ void GraceAggregatingTransform::flushBuckets() flushBucket(i); } -static size_t flushBlocksInfoDisk(DB::TemporaryFileStream * file_stream, std::list & blocks) +static size_t flushBlocksInfoDisk(std::optional& file_stream, std::list & blocks) { size_t flush_bytes = 0; DB::Blocks tmp_blocks; + if (!file_stream) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "file_stream is empty"); + auto & tmp_stream = file_stream.value(); while (!blocks.empty()) { while (!blocks.empty()) @@ -322,11 +325,11 @@ static size_t flushBlocksInfoDisk(DB::TemporaryFileStream * file_stream, std::li flush_bytes += merged_block.bytes(); if (merged_block.rows()) { - file_stream->write(merged_block); + tmp_stream->write(merged_block); } } if (flush_bytes) - file_stream->flush(); + tmp_stream->flush(); return flush_bytes; } @@ -338,7 +341,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.original_blocks.empty()) { if (!file_stream.original_file_stream) - file_stream.original_file_stream = &tmp_data_disk->createStream(header); + file_stream.original_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk.get()); flush_bytes += flushBlocksInfoDisk(file_stream.original_file_stream, file_stream.original_blocks); } if (!file_stream.intermediate_blocks.empty()) @@ -346,7 +349,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.intermediate_file_stream) { auto intermediate_header = params->aggregator.getHeader(false); - file_stream.intermediate_file_stream = &tmp_data_disk->createStream(intermediate_header); + file_stream.intermediate_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk.get()); } flush_bytes += flushBlocksInfoDisk(file_stream.intermediate_file_stream, file_stream.intermediate_blocks); } @@ -373,9 +376,10 @@ std::unique_ptr GraceAggregatingTransform::prepareB if (buffer_file_stream.intermediate_file_stream) { buffer_file_stream.intermediate_file_stream->finishWriting(); + auto reader = buffer_file_stream.intermediate_file_stream->getReadStream(); while (true) { - auto block = buffer_file_stream.intermediate_file_stream->read(); + auto block = reader->read(); if (!block.rows()) break; read_bytes += block.bytes(); @@ -383,7 +387,7 @@ std::unique_ptr GraceAggregatingTransform::prepareB mergeOneBlock(block, false); block = {}; } - buffer_file_stream.intermediate_file_stream = nullptr; + buffer_file_stream.intermediate_file_stream.reset(); total_read_disk_time += watch.elapsedMilliseconds(); } if (!buffer_file_stream.intermediate_blocks.empty()) @@ -398,9 +402,10 @@ std::unique_ptr GraceAggregatingTransform::prepareB if (buffer_file_stream.original_file_stream) { buffer_file_stream.original_file_stream->finishWriting(); + auto reader = buffer_file_stream.original_file_stream->getReadStream(); while (true) { - auto block = buffer_file_stream.original_file_stream->read(); + auto block = reader->read(); if (!block.rows()) break; read_bytes += block.bytes(); @@ -408,7 +413,7 @@ std::unique_ptr GraceAggregatingTransform::prepareB mergeOneBlock(block, true); block = {}; } - buffer_file_stream.original_file_stream = nullptr; + buffer_file_stream.original_file_stream.reset(); total_read_disk_time += watch.elapsedMilliseconds(); } if (!buffer_file_stream.original_blocks.empty()) diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h index c2b787393a1b..612a58b3c943 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h @@ -59,7 +59,7 @@ class GraceAggregatingTransform : public DB::IProcessor DB::Aggregator::AggregateColumns aggregate_columns; DB::AggregatingTransformParamsPtr params; DB::ContextPtr context; - DB::TemporaryDataOnDiskPtr tmp_data_disk; + DB::TemporaryDataOnDiskScopePtr tmp_data_disk; DB::AggregatedDataVariantsPtr current_data_variants = nullptr; size_t current_bucket_index = 0; @@ -83,9 +83,9 @@ class GraceAggregatingTransform : public DB::IProcessor /// Only be used when there is no pre-aggregated step, store the original input blocks. std::list original_blocks; /// store the intermediate result blocks. - DB::TemporaryFileStream * intermediate_file_stream = nullptr; + std::optional intermediate_file_stream; /// Only be used when there is no pre-aggregated step - DB::TemporaryFileStream * original_file_stream = nullptr; + std::optional original_file_stream; size_t pending_bytes = 0; }; std::unordered_map buckets; diff --git a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp index 59d4c39f8737..5a6f229744fc 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp @@ -41,6 +41,7 @@ namespace DB namespace Setting { extern const SettingsUInt64 max_block_size; +extern const SettingsUInt64 min_joined_block_size_bytes; } namespace ErrorCodes { @@ -200,7 +201,13 @@ DB::QueryPlanPtr CrossRelParser::parseJoin(const substrait::CrossRel & join, DB: { JoinPtr hash_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty()); QueryPlanStepPtr join_step = std::make_unique( - left->getCurrentHeader(), right->getCurrentHeader(), hash_join, context->getSettingsRef()[Setting::max_block_size], 1, false); + left->getCurrentHeader(), + right->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + false); join_step->setStepDescription("CROSS_JOIN"); steps.emplace_back(join_step.get()); std::vector plans; diff --git a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp index 46f7926cf729..7493471697f1 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp @@ -46,6 +46,7 @@ namespace Setting { extern const SettingsJoinAlgorithm join_algorithm; extern const SettingsUInt64 max_block_size; +extern const SettingsUInt64 min_joined_block_size_bytes; } namespace ErrorCodes { @@ -315,7 +316,13 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q JoinPtr smj_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty(), -1); MultiEnum join_algorithm = context->getSettingsRef()[Setting::join_algorithm]; QueryPlanStepPtr join_step = std::make_unique( - left->getCurrentHeader(), right->getCurrentHeader(), smj_join, context->getSettingsRef()[Setting::max_block_size], 1, false); + left->getCurrentHeader(), + right->getCurrentHeader(), + smj_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + false); join_step->setStepDescription("SORT_MERGE_JOIN"); steps.emplace_back(join_step.get()); @@ -448,7 +455,7 @@ void JoinRelParser::collectJoinKeys( table_join.addDisjunct(); const auto & expr = join_rel.expression(); auto & join_clause = table_join.getClauses().back(); - std::list expressions_stack; + std::list expressions_stack; expressions_stack.push_back(&expr); while (!expressions_stack.empty()) { @@ -778,6 +785,7 @@ DB::QueryPlanPtr JoinRelParser::buildMultiOnClauseHashJoin( right_plan->getCurrentHeader(), hash_join, context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, false); join_step->setStepDescription("Multi join on clause hash join"); @@ -817,6 +825,7 @@ DB::QueryPlanPtr JoinRelParser::buildSingleOnClauseHashJoin( right_plan->getCurrentHeader(), hash_join, context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, false); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 0be7e0d892b7..17587e5200ef 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -74,7 +74,7 @@ void SparkStorageMergeTree::analysisPartsByRanges(DB::ReadFromMergeTree & source sum_ranges += part.ranges.size(); sum_marks += part.getMarksCount(); sum_rows += part.getRowsCount(); - total_marks_pk += part.data_part->index_granularity.getMarksCountWithoutFinal(); + total_marks_pk += part.data_part->index_granularity->getMarksCountWithoutFinal(); for (auto range : part.ranges) sum_marks_pk += range.getNumberOfMarks(); @@ -487,6 +487,12 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); auto txn = context->getCurrentTransaction(); + auto index_granularity_ptr = createMergeTreeIndexGranularity( + block.rows(), + block.bytes(), + *data.getSettings(), + new_data_part->index_granularity_info, + /*blocks_are_granules=*/false); auto out = std::make_unique( new_data_part, metadata_snapshot, @@ -494,6 +500,7 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( indices, MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()), compression_codec, + index_granularity_ptr, txn ? txn->tid : Tx::PrehistoricTID, false, false, From 2649fa7b34edf99268bff0d95d9d9621683f839c Mon Sep 17 00:00:00 2001 From: Yuan Date: Wed, 27 Nov 2024 09:06:50 +0800 Subject: [PATCH 150/211] [GLUTEN-8046][VL] CI: fix velox cache/bundle package script (#8051) This patch fixes the cache/build script due to nodejs version update in GHA This patch also adds centos7 tests back fixes #8046 Signed-off-by: Yuan Zhou --- .github/workflows/build_bundle_package.yml | 19 ++++-- .github/workflows/velox_backend.yml | 78 +++++++++++++++++++--- .github/workflows/velox_backend_cache.yml | 8 +-- 3 files changed, 86 insertions(+), 19 deletions(-) diff --git a/.github/workflows/build_bundle_package.yml b/.github/workflows/build_bundle_package.yml index 9b94d302f962..c7ddd8748b25 100644 --- a/.github/workflows/build_bundle_package.yml +++ b/.github/workflows/build_bundle_package.yml @@ -38,9 +38,8 @@ on: jobs: build-native-lib: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -50,9 +49,15 @@ jobs: ccache-centos7-release-default - name: Build Gluten velox third party run: | - df -a - cd $GITHUB_WORKSPACE/ - bash dev/ci-velox-buildstatic-centos-7.sh + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + df -a + cd /work + export CCACHE_DIR=/work/.ccache + bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s + mkdir -p /work/.m2/repository/org/apache/arrow/ + cp -r /root/.m2/repository/org/apache/arrow/* /work/.m2/repository/org/apache/arrow/ + " - name: Upload native libs uses: actions/upload-artifact@v3 with: @@ -65,10 +70,10 @@ jobs: path: /root/.m2/repository/org/apache/arrow/ name: velox-arrow-jar-centos-7-${{github.sha}} - build-bundle-package-centos7: + build-bundle-package-centos8: needs: build-native-lib runs-on: ubuntu-20.04 - container: centos:7 + container: centos:8 steps: - uses: actions/checkout@v2 - name: Download All Artifacts diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 56b43975acee..3c4bb3e2e97d 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -164,12 +164,12 @@ jobs: && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 - run-tpc-test-centos: + run-tpc-test-centos8: needs: build-native-lib-centos-7 strategy: fail-fast: false matrix: - os: [ "centos:8" ] #TODO: add centos7 tests back with docker run + os: [ "centos:8" ] spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] java: [ "java-8", "java-11", "java-17" ] # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 @@ -255,6 +255,68 @@ jobs: --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ --extra-conf=spark.gluten.ras.enabled=true + run-tpc-test-centos7: + needs: build-native-lib-centos-7 + strategy: + fail-fast: false + matrix: + spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] + java: [ "java-8" ] + # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 + runs-on: ubuntu-20.04 + steps: + - uses: actions/checkout@v4 + - name: Download All Native Artifacts + uses: actions/download-artifact@v3 + with: + name: velox-native-lib-centos-7-${{github.sha}} + path: ./cpp/build/releases/ + - name: Download All Arrow Jar Artifacts + uses: actions/download-artifact@v3 + with: + name: arrow-jars-centos-7-${{github.sha}} + path: .m2/repository/org/apache/arrow/ + - name: Build and run TPCH/DS tests + run: | + docker pull centos:7 + docker run -v $GITHUB_WORKSPACE:/work -v /$GITHUB_WORKSPACE/.m2:/root/.m2/ -w /work \ + -e matrix.java=${{ matrix.java }} -e matrix.spark=${{ matrix.spark }} \ + centos:7 \ + bash -c " + sed -i -e 's|mirrorlist=|#mirrorlist=|g' /etc/yum.repos.d/CentOS-* || true + sed -i -e 's|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g' /etc/yum.repos.d/CentOS-* || true + + # Setup java and maven + yum update -y && yum install -y java-1.8.0-openjdk-devel wget + + wget -nv https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz + tar -xvf apache-maven-3.8.8-bin.tar.gz && mv apache-maven-3.8.8 /usr/lib/maven + export PATH=${PATH}:/usr/lib/maven/bin + + # Set environment variables + export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk + + # Build gluten-it + mvn -ntp clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests + cd /work/tools/gluten-it + mvn -ntp clean install -P${{ matrix.spark }} -P${{ matrix.java }} + + # Run TPC-H / TPC-DS + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 + + # Run TPC-H / TPC-DS with RAS + cd /work/tools/gluten-it + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ + --extra-conf=spark.gluten.ras.enabled=true \ + && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ + --extra-conf=spark.gluten.ras.enabled=true + " + run-tpc-test-ubuntu-oom: needs: build-native-lib-centos-7 strategy: @@ -962,12 +1024,12 @@ jobs: df -a bash dev/ci-velox-buildshared-centos-8.sh ccache -s - - name: "Save ccache" - uses: actions/cache/save@v3 - id: ccache - with: - path: '${{ env.CCACHE_DIR }}' - key: ccache-centos8-release-default-${{github.sha}} + # - name: "Save ccache" + # uses: actions/cache/save@v3 + # id: ccache + # with: + # path: '${{ env.CCACHE_DIR }}' + # key: ccache-centos8-release-default-${{github.sha}} - name: Run CPP unit test run: | cd ./cpp/build && ctest -V diff --git a/.github/workflows/velox_backend_cache.yml b/.github/workflows/velox_backend_cache.yml index e969ffb7415e..d0b295b5f48e 100644 --- a/.github/workflows/velox_backend_cache.yml +++ b/.github/workflows/velox_backend_cache.yml @@ -31,9 +31,8 @@ concurrency: jobs: cache-native-lib-centos-7: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -43,8 +42,9 @@ jobs: ccache-centos7-release-default - name: Build Gluten native libraries run: | - df -a - bash dev/ci-velox-buildstatic-centos-7.sh + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + bash dev/ci-velox-buildstatic-centos-7.sh + " - name: Save Ccache uses: actions/cache/save@v3 id: ccache From bfafa576f6f7a333301788bffb434b3a5cc5ccf4 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Wed, 27 Nov 2024 11:24:23 +0800 Subject: [PATCH 151/211] [GLUTEN-7631][VL] Fall back lead/lag if input is foldable (#8038) --- .../org/apache/gluten/backendsapi/velox/VeloxBackend.scala | 4 +++- .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 3 --- .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 3 --- .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 3 --- .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 3 --- 5 files changed, 3 insertions(+), 13 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index e05fd92e3201..b1245687b0c2 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -400,7 +400,9 @@ object VeloxBackendSettings extends BackendSettingsApi { } windowExpression.windowFunction match { case _: RowNumber | _: Rank | _: CumeDist | _: DenseRank | _: PercentRank | - _: NthValue | _: NTile | _: Lag | _: Lead => + _: NthValue | _: NTile => + case l: Lag if !l.input.foldable => + case l: Lead if !l.input.foldable => case aggrExpr: AggregateExpression if !aggrExpr.aggregateFunction.isInstanceOf[ApproximatePercentile] && !aggrExpr.aggregateFunction.isInstanceOf[Percentile] => diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 363f9c85ed1d..276a8a0f7af2 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -318,9 +318,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet. enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenSortSuite] // Sort spill is not supported. .exclude("sorting does not crash for large inputs") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 1fc64577d2a6..08fab69c849b 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -886,9 +886,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet. enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenTakeOrderedAndProjectSuite] enableSuite[GlutenSessionExtensionSuite] enableSuite[TestFileSourceScanExecTransformer] diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 8c65afd1a3a2..df79b663bcbe 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -886,9 +886,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet.enableSuite[GlutenSQLWindowFunctionSuite] enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenTakeOrderedAndProjectSuite] enableSuite[GlutenSessionExtensionSuite] enableSuite[TestFileSourceScanExecTransformer] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 18c46a304282..cb2151fe4698 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -900,9 +900,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet. enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenTakeOrderedAndProjectSuite] enableSuite[GlutenSessionExtensionSuite] enableSuite[TestFileSourceScanExecTransformer] From 353169e5cd3541cf56ee8e5bdd01e73a5d157aaf Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 27 Nov 2024 13:37:06 +0800 Subject: [PATCH 152/211] [GLUTEN-6920][CORE] Redesign and move trait `GlutenPlan` to `gluten-core` (#8036) --- .../FallbackBroadcastHashJoinRules.scala | 8 +- .../GlutenClickHouseTPCDSMetricsSuite.scala | 2 +- .../GlutenClickHouseTPCHMetricsSuite.scala | 2 +- ...seTPCHColumnarShuffleParquetAQESuite.scala | 2 +- ...enClickHouseTPCHSaltNullParquetSuite.scala | 2 +- .../backendsapi/velox/VeloxRuleApi.scala | 1 - .../ColumnarPartialProjectExec.scala | 12 ++- .../execution/RowToVeloxColumnarExec.scala | 1 - .../execution/VeloxResizeBatchesExec.scala | 7 +- .../ArrowConvertorRule.scala | 3 +- .../python/ColumnarArrowEvalPythonExec.scala | 10 +- .../sql/execution/BaseArrowScanExec.scala | 4 +- .../gluten/execution/FallbackSuite.scala | 1 - .../execution/ColumnarToColumnarExec.scala | 14 +-- .../apache/gluten/execution/GlutenPlan.scala | 60 ++++++++++++ .../planner/plan/GlutenPlanModel.scala | 5 +- .../enumerated/planner/property/Conv.scala | 2 +- .../columnar/transition/Convention.scala | 26 ++--- .../columnar/transition/ConventionFunc.scala | 73 ++++++-------- .../columnar/transition/ConventionReq.scala | 6 +- .../columnar/transition/Transitions.scala | 8 +- .../BasicPhysicalOperatorTransformer.scala | 9 +- .../CartesianProductExecTransformer.scala | 5 +- .../execution/ColumnarCoalesceExec.scala | 9 +- .../execution/ColumnarToRowExecBase.scala | 19 ++-- .../execution/RowToColumnarExecBase.scala | 9 +- ...TakeOrderedAndProjectExecTransformer.scala | 7 +- .../execution/WholeStageTransformer.scala | 87 +++++++++++++++- .../apache/gluten/extension/GlutenPlan.scala | 98 ------------------- .../columnar/enumerated/RasOffload.scala | 6 +- .../columnar/enumerated/RemoveSort.scala | 3 +- .../heuristic/ExpandFallbackPolicy.scala | 4 +- .../offload/OffloadSingleNodeRules.scala | 4 +- .../columnar/validator/Validators.scala | 2 +- .../ColumnarBroadcastExchangeExec.scala | 9 +- .../ColumnarCollapseTransformStages.scala | 15 ++- .../ColumnarShuffleExchangeExec.scala | 10 +- .../ColumnarSubqueryBroadcastExec.scala | 7 +- .../execution/ColumnarWriteFilesExec.scala | 17 ++-- .../sql/execution/GlutenExplainUtils.scala | 3 +- .../execution/GlutenFallbackReporter.scala | 2 +- .../spark/sql/execution/GlutenImplicits.scala | 6 +- .../GlutenWriterColumnarRules.scala | 5 +- .../columnar/transition/TransitionSuite.scala | 4 +- .../transition/TransitionSuiteBase.scala | 5 +- .../org/apache/gluten/test/FallbackUtil.scala | 2 +- .../apache/spark/sql/GlutenQueryTest.scala | 2 +- .../execution/FallbackStrategiesSuite.scala | 11 ++- .../statistics/SparkFunctionStatistics.scala | 2 +- .../execution/FallbackStrategiesSuite.scala | 11 ++- .../statistics/SparkFunctionStatistics.scala | 2 +- .../execution/FallbackStrategiesSuite.scala | 11 ++- .../spark/sql/sources/GlutenInsertSuite.scala | 2 +- .../statistics/SparkFunctionStatistics.scala | 2 +- .../execution/FallbackStrategiesSuite.scala | 11 ++- .../spark/sql/sources/GlutenInsertSuite.scala | 2 +- .../statistics/SparkFunctionStatistics.scala | 2 +- .../GlutenExpressionDataTypesValidation.scala | 6 +- 58 files changed, 381 insertions(+), 279 deletions(-) rename backends-velox/src/main/scala/org/apache/gluten/{datasource => extension}/ArrowConvertorRule.scala (97%) create mode 100644 gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala index 4af2d0fc0c7b..6a788617a6c4 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala @@ -116,19 +116,19 @@ case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extend if (FallbackTags.nonEmpty(bnlj)) { ValidationResult.failed("broadcast join is already tagged as not transformable") } else { - val transformer = BackendsApiManager.getSparkPlanExecApiInstance + val bnljTransformer = BackendsApiManager.getSparkPlanExecApiInstance .genBroadcastNestedLoopJoinExecTransformer( bnlj.left, bnlj.right, bnlj.buildSide, bnlj.joinType, bnlj.condition) - val isTransformable = transformer.doValidate() - if (isTransformable.ok()) { + val isBnljTransformable = bnljTransformer.doValidate() + if (isBnljTransformable.ok()) { val exchangeTransformer = ColumnarBroadcastExchangeExec(mode, child) exchangeTransformer.doValidate() } else { - isTransformable + isBnljTransformable } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala index cbf1caf44e7f..02445270d42c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.metrics import org.apache.gluten.execution.{ColumnarNativeIterator, GlutenClickHouseTPCDSAbstractSuite, WholeStageTransformer} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala index 3cfb8cc4fc9c..015da0dfae10 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.metrics import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index 885e50b0468b..1c3e4de4c6ef 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -18,7 +18,7 @@ package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.Arm import org.apache.spark.SparkConf diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 4a2b7040fa5e..fbefc054fac7 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -19,7 +19,7 @@ package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.DataFrame diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 0a62a41baaf0..7841e6cd94b1 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -19,7 +19,6 @@ package org.apache.gluten.backendsapi.velox import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi import org.apache.gluten.columnarbatch.VeloxBatch -import org.apache.gluten.datasource.ArrowConvertorRule import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala index 84de41daa015..20744f531b5b 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala @@ -17,9 +17,11 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} import org.apache.gluten.expression.{ArrowProjection, ExpressionUtils} -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.sql.shims.SparkShimLoader @@ -31,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeRef import org.apache.spark.sql.execution.{ExplainUtils, ProjectExec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive.HiveUdfUtil -import org.apache.spark.sql.types.{BinaryType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, NullType, ShortType, StringType, TimestampType, YearMonthIntervalType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import scala.collection.mutable.ListBuffer @@ -51,7 +53,7 @@ import scala.collection.mutable.ListBuffer case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( replacedAliasUdf: Seq[Alias]) extends UnaryExecNode - with GlutenPlan { + with ValidatablePlan { private val projectAttributes: ListBuffer[Attribute] = ListBuffer() private val projectIndexInChild: ListBuffer[Int] = ListBuffer() @@ -73,6 +75,10 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( override def output: Seq[Attribute] = child.output ++ replacedAliasUdf.map(_.toAttribute) + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + final override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException( s"${this.getClass.getSimpleName} doesn't support doExecute") diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala index a853778484b1..9921ffbfab61 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala @@ -43,7 +43,6 @@ import org.apache.arrow.memory.ArrowBuf import scala.collection.mutable.ListBuffer case class RowToVeloxColumnarExec(child: SparkPlan) extends RowToColumnarExecBase(child = child) { - override def doExecuteColumnarInternal(): RDD[ColumnarBatch] = { val numInputRows = longMetric("numInputRows") val numOutputBatches = longMetric("numOutputBatches") diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala index 5283ab61e3b9..9ed687d33703 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala @@ -16,7 +16,8 @@ */ package org.apache.gluten.execution -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.iterator.Iterators import org.apache.gluten.utils.VeloxBatchResizer @@ -53,6 +54,10 @@ case class VeloxResizeBatchesExec( "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append / split batches") ) + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala similarity index 97% rename from backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala rename to backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala index b1b0b813f6c8..5e02cf54b0e5 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.datasource +package org.apache.gluten.extension import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.datasource.ArrowCSVFileFormat import org.apache.gluten.datasource.v2.ArrowCSVTable import org.apache.gluten.sql.shims.SparkShimLoader diff --git a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala index a2eee53660ce..548dec13beb0 100644 --- a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala @@ -19,9 +19,9 @@ package org.apache.spark.api.python import org.apache.gluten.columnarbatch.ArrowBatches.ArrowJavaBatch import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.exception.GlutenException -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions +import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildConvention import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.utils.PullOutProjectHelper @@ -213,11 +213,13 @@ case class ColumnarArrowEvalPythonExec( evalType: Int) extends EvalPythonExec with GlutenPlan - with KnownChildrenConventions { + with KnownChildConvention { override def batchType(): Convention.BatchType = ArrowJavaBatch - override def requiredChildrenConventions(): Seq[ConventionReq] = List( + override def rowType0(): Convention.RowType = Convention.RowType.None + + override def requiredChildConvention(): Seq[ConventionReq] = List( ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(ArrowJavaBatch))) override lazy val metrics = Map( diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala index 6617e8b138ba..1aacc1b95416 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.execution import org.apache.gluten.columnarbatch.ArrowBatches -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.transition.Convention trait BaseArrowScanExec extends GlutenPlan { final override def batchType(): Convention.BatchType = { ArrowBatches.ArrowJavaBatch } + + final override def rowType0(): Convention.RowType = Convention.RowType.None } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala index 0f7b70bf0032..0f94b8648e71 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala @@ -17,7 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.sql.execution.{ColumnarBroadcastExchangeExec, ColumnarShuffleExchangeExec, SparkPlan} diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala index 7ca4b36b06d2..e27b6a4e2f1e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala @@ -30,9 +30,7 @@ import java.util.concurrent.atomic.AtomicLong abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention.BatchType) extends ColumnarToColumnarTransition - with Convention.KnownBatchType - with Convention.KnownRowTypeForSpark33AndLater - with ConventionReq.KnownChildrenConventions { + with GlutenPlan { def child: SparkPlan protected def mapIterator(in: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] @@ -46,21 +44,13 @@ abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert batches") ) - final override val supportsColumnar: Boolean = { - batchType() != Convention.BatchType.None - } - override def batchType(): Convention.BatchType = to - final override val supportsRowBased: Boolean = { - rowType() != Convention.RowType.None - } - override def rowType0(): Convention.RowType = { Convention.RowType.None } - override def requiredChildrenConventions(): Seq[ConventionReq] = List( + override def requiredChildConvention(): Seq[ConventionReq] = List( ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(from))) override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala new file mode 100644 index 000000000000..460326d8d939 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution + +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} + +import org.apache.spark.sql.execution.SparkPlan + +trait GlutenPlan + extends SparkPlan + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater + with GlutenPlan.SupportsRowBasedCompatible + with ConventionReq.KnownChildConvention { + + final override val supportsColumnar: Boolean = { + batchType() != Convention.BatchType.None + } + + override def batchType(): Convention.BatchType + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None + } + + override def rowType0(): Convention.RowType + + override def requiredChildConvention(): Seq[ConventionReq] = { + // In the normal case, children's convention should follow parent node's convention. + val childReq = Convention.of(rowType(), batchType()).asReq() + Seq.tabulate(children.size)( + _ => { + childReq + }) + } +} + +object GlutenPlan { + // To be compatible with Spark (version < 3.3) + trait SupportsRowBasedCompatible { + def supportsRowBased(): Boolean = { + throw new GlutenException("Illegal state: The method is not expected to be called") + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala index 568ea50396bd..20a238a47d2c 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.extension.columnar.enumerated.planner.plan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadata import org.apache.gluten.extension.columnar.enumerated.planner.property.{Conv, ConvDef} import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} @@ -37,13 +38,15 @@ object GlutenPlanModel { PlanModelImpl } + // TODO: Make this inherit from GlutenPlan. case class GroupLeafExec( groupId: Int, metadata: GlutenMetadata, constraintSet: PropertySet[SparkPlan]) extends LeafExecNode with Convention.KnownBatchType - with Convention.KnownRowTypeForSpark33AndLater { + with Convention.KnownRowTypeForSpark33AndLater + with GlutenPlan.SupportsRowBasedCompatible { private val req: Conv.Req = constraintSet.get(ConvDef).asInstanceOf[Conv.Req] override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException() diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala index 7b2b801ac91e..cfb32e76446a 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala @@ -92,7 +92,7 @@ object ConvDef extends PropertyDef[SparkPlan, Conv] { override def getChildrenConstraints( constraint: Property[SparkPlan], plan: SparkPlan): Seq[Conv] = { - val out = List.tabulate(plan.children.size)(_ => Conv.req(ConventionReq.get(plan))) + val out = ConventionReq.get(plan).map(Conv.req) out } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala index b57f3e0c0a29..b9fbe023b27c 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala @@ -16,8 +16,6 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.exception.GlutenException - import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.util.SparkVersionUtil @@ -55,6 +53,19 @@ object Convention { } Convention.of(rowType(), batchType()) } + + def asReq(): ConventionReq = { + val rowTypeReq = conv.rowType match { + case Convention.RowType.None => ConventionReq.RowType.Any + case r => ConventionReq.RowType.Is(r) + } + + val batchTypeReq = conv.batchType match { + case Convention.BatchType.None => ConventionReq.BatchType.Any + case b => ConventionReq.BatchType.Is(b) + } + ConventionReq.of(rowTypeReq, batchTypeReq) + } } private case class Impl(override val rowType: RowType, override val batchType: BatchType) @@ -142,19 +153,10 @@ object Convention { def batchType(): BatchType } - sealed trait KnownRowType extends KnownRowType.SupportsRowBasedCompatible { + sealed trait KnownRowType { def rowType(): RowType } - object KnownRowType { - // To be compatible with Spark (version < 3.3) - sealed trait SupportsRowBasedCompatible { - def supportsRowBased(): Boolean = { - throw new GlutenException("Illegal state: The method is not expected to be called") - } - } - } - trait KnownRowTypeForSpark33AndLater extends KnownRowType { this: SparkPlan => import KnownRowTypeForSpark33AndLater._ diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala index 5cb3d44a1562..2be9271a1f53 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.transition import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions +import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildConvention import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnionExec} @@ -28,14 +28,14 @@ import org.apache.spark.sql.execution.exchange.ReusedExchangeExec /** ConventionFunc is a utility to derive [[Convention]] or [[ConventionReq]] from a query plan. */ sealed trait ConventionFunc { def conventionOf(plan: SparkPlan): Convention - def conventionReqOf(plan: SparkPlan): ConventionReq + def conventionReqOf(plan: SparkPlan): Seq[ConventionReq] } object ConventionFunc { trait Override { def rowTypeOf: PartialFunction[SparkPlan, Convention.RowType] = PartialFunction.empty def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = PartialFunction.empty - def conventionReqOf: PartialFunction[SparkPlan, ConventionReq] = PartialFunction.empty + def conventionReqOf: PartialFunction[SparkPlan, Seq[ConventionReq]] = PartialFunction.empty } object Override { @@ -72,7 +72,6 @@ object ConventionFunc { } private class BuiltinFunc(o: Override) extends ConventionFunc { - import BuiltinFunc._ override def conventionOf(plan: SparkPlan): Convention = { val conv = conventionOf0(plan) conv @@ -158,61 +157,45 @@ object ConventionFunc { } } - override def conventionReqOf(plan: SparkPlan): ConventionReq = { + override def conventionReqOf(plan: SparkPlan): Seq[ConventionReq] = { val req = o.conventionReqOf.applyOrElse(plan, conventionReqOf0) + assert(req.size == plan.children.size) req } - private def conventionReqOf0(plan: SparkPlan): ConventionReq = plan match { - case k: KnownChildrenConventions => - val reqs = k.requiredChildrenConventions().distinct - // This can be a temporary restriction. - assert( - reqs.size == 1, - "KnownChildrenConventions#requiredChildrenConventions should output the same element" + - " for all children") - reqs.head + private def conventionReqOf0(plan: SparkPlan): Seq[ConventionReq] = plan match { + case k: KnownChildConvention => + val reqs = k.requiredChildConvention() + reqs case RowToColumnarLike(_) => - ConventionReq.of( - ConventionReq.RowType.Is(Convention.RowType.VanillaRow), - ConventionReq.BatchType.Any) + Seq( + ConventionReq.of( + ConventionReq.RowType.Is(Convention.RowType.VanillaRow), + ConventionReq.BatchType.Any)) case ColumnarToRowExec(_) => - ConventionReq.of( - ConventionReq.RowType.Any, - ConventionReq.BatchType.Is(Convention.BatchType.VanillaBatch)) + Seq( + ConventionReq.of( + ConventionReq.RowType.Any, + ConventionReq.BatchType.Is(Convention.BatchType.VanillaBatch))) case write: DataWritingCommandExec if SparkShimLoader.getSparkShims.isPlannedV1Write(write) => // To align with ApplyColumnarRulesAndInsertTransitions#insertTransitions - ConventionReq.any + Seq(ConventionReq.any) case u: UnionExec => // We force vanilla union to output row data to get the best compatibility with vanilla // Spark. // As a result it's a common practice to rewrite it with GlutenPlan for offloading. - ConventionReq.of( - ConventionReq.RowType.Is(Convention.RowType.VanillaRow), - ConventionReq.BatchType.Any) + Seq.tabulate(u.children.size)( + _ => + ConventionReq.of( + ConventionReq.RowType.Is(Convention.RowType.VanillaRow), + ConventionReq.BatchType.Any)) case other => // In the normal case, children's convention should follow parent node's convention. - // Note, we don't have to consider C2R / R2C here since they are already removed by - // RemoveTransitions. - val thisConv = conventionOf0(other) - thisConv.asReq() - } - } - - private object BuiltinFunc { - implicit private class ConventionOps(conv: Convention) { - def asReq(): ConventionReq = { - val rowTypeReq = conv.rowType match { - case Convention.RowType.None => ConventionReq.RowType.Any - case r => ConventionReq.RowType.Is(r) - } - - val batchTypeReq = conv.batchType match { - case Convention.BatchType.None => ConventionReq.BatchType.Any - case b => ConventionReq.BatchType.Is(b) - } - ConventionReq.of(rowTypeReq, batchTypeReq) - } + val childReq = conventionOf0(other).asReq() + Seq.tabulate(other.children.size)( + _ => { + childReq + }) } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala index a081f214347b..86637f2d5a7e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala @@ -55,12 +55,12 @@ object ConventionReq { val row: ConventionReq = ofRow(RowType.Is(Convention.RowType.VanillaRow)) val vanillaBatch: ConventionReq = ofBatch(BatchType.Is(Convention.BatchType.VanillaBatch)) - def get(plan: SparkPlan): ConventionReq = ConventionFunc.create().conventionReqOf(plan) + def get(plan: SparkPlan): Seq[ConventionReq] = ConventionFunc.create().conventionReqOf(plan) def of(rowType: RowType, batchType: BatchType): ConventionReq = Impl(rowType, batchType) def ofRow(rowType: RowType): ConventionReq = Impl(rowType, BatchType.Any) def ofBatch(batchType: BatchType): ConventionReq = Impl(RowType.Any, batchType) - trait KnownChildrenConventions { - def requiredChildrenConventions(): Seq[ConventionReq] + trait KnownChildConvention { + def requiredChildConvention(): Seq[ConventionReq] } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala index 10d50f453d6d..cec09ee7a1ff 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala @@ -42,13 +42,13 @@ case class InsertTransitions(convReq: ConventionReq) extends Rule[SparkPlan] { if (node.children.isEmpty) { return node } - val convReq = convFunc.conventionReqOf(node) - val newChildren = node.children.map { - child => + val convReqs = convFunc.conventionReqOf(node) + val newChildren = node.children.zip(convReqs).map { + case (child, convReq) => val from = convFunc.conventionOf(child) if (from.isNone) { // For example, a union op with row child and columnar child at the same time, - // The plan is actually not executable and we cannot tell about its convention. + // The plan is actually not executable, and we cannot tell about its convention. child } else { val transition = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index 2830ef404c0e..f9755605cab2 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -19,7 +19,8 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.expression.{ConverterUtils, ExpressionConverter, ExpressionTransformer} -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.`type`.TypeBuilder import org.apache.gluten.substrait.SubstraitContext @@ -261,13 +262,17 @@ abstract class ProjectExecTransformerBase(val list: Seq[NamedExpression], val in } // An alternatives for UnionExec. -case class ColumnarUnionExec(children: Seq[SparkPlan]) extends GlutenPlan { +case class ColumnarUnionExec(children: Seq[SparkPlan]) extends ValidatablePlan { children.foreach { case w: WholeStageTransformer => w.setOutputSchemaForPlan(output) case _ => } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override def output: Seq[Attribute] = { children.map(_.output).transpose.map { attrs => diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala index 28fb691896ff..3e3169aa554e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala @@ -18,7 +18,8 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.expression.ExpressionConverter -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.rel.RelBuilder @@ -45,6 +46,8 @@ import java.io.{IOException, ObjectOutputStream} */ case class ColumnarCartesianProductBridge(child: SparkPlan) extends UnaryExecNode with GlutenPlan { override def output: Seq[Attribute] = child.output + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala index 3b13207c9350..107bf544cd6f 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala @@ -16,7 +16,8 @@ */ package org.apache.gluten.execution -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD @@ -28,7 +29,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class ColumnarCoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode - with GlutenPlan { + with ValidatablePlan { override def output: Seq[Attribute] = child.output @@ -36,6 +37,10 @@ case class ColumnarCoalesceExec(numPartitions: Int, child: SparkPlan) if (numPartitions == 1) SinglePartition else UnknownPartitioning(numPartitions) } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException() } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala index fae3115981eb..8bf1fbe44747 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala @@ -17,9 +17,8 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions +import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildConvention import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -30,8 +29,8 @@ import org.apache.spark.sql.execution.{ColumnarToRowTransition, SparkPlan} abstract class ColumnarToRowExecBase(child: SparkPlan) extends ColumnarToRowTransition - with GlutenPlan - with KnownChildrenConventions { + with KnownChildConvention + with ValidatablePlan { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics = @@ -47,6 +46,12 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow + override def requiredChildConvention(): Seq[ConventionReq] = { + List( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) + } + override def doExecuteBroadcast[T](): Broadcast[T] = { // Require for explicit implementation, otherwise throw error. super.doExecuteBroadcast[T]() @@ -57,10 +62,4 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) override def doExecute(): RDD[InternalRow] = { doExecuteInternal() } - - override def requiredChildrenConventions(): Seq[ConventionReq] = { - List( - ConventionReq.ofBatch( - ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) - } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala index 2a52616361a1..9f36b8fc672d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala @@ -17,8 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.spark.broadcast import org.apache.spark.rdd.RDD @@ -46,8 +45,14 @@ abstract class RowToColumnarExecBase(child: SparkPlan) final override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None + override def requiredChildConvention(): Seq[ConventionReq] = { + Seq(ConventionReq.row) + } + final override def doExecute(): RDD[InternalRow] = { child.execute() } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala index c960bda2490e..f19960ec1ca3 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala @@ -17,7 +17,8 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -40,9 +41,11 @@ case class TakeOrderedAndProjectExecTransformer( child: SparkPlan, offset: Int = 0) extends UnaryExecNode - with GlutenPlan { + with ValidatablePlan { override def outputPartitioning: Partitioning = SinglePartition override def outputOrdering: Seq[SortOrder] = sortOrder + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override def output: Seq[Attribute] = { projectList.map(_.toAttribute) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index 6414b67a8092..dbfc11c136db 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -18,14 +18,17 @@ package org.apache.gluten.execution import org.apache.gluten.{GlutenConfig, GlutenNumaBindingInfo} import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenException +import org.apache.gluten.exception.{GlutenException, GlutenNotSupportException} import org.apache.gluten.expression._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.metrics.{GlutenTimeMetric, MetricsUpdater} import org.apache.gluten.substrait.`type`.{TypeBuilder, TypeNode} import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.plan.{PlanBuilder, PlanNode} import org.apache.gluten.substrait.rel.{LocalFilesNode, RelNode, SplitInfo} +import org.apache.gluten.test.TestStats import org.apache.gluten.utils.SubstraitPlanPrinterUtil import org.apache.spark._ @@ -53,7 +56,74 @@ case class TransformContext(outputAttributes: Seq[Attribute], root: RelNode) case class WholeStageTransformContext(root: PlanNode, substraitContext: SubstraitContext = null) -trait TransformSupport extends GlutenPlan { +/** + * Base interface for a Gluten query plan that is also open to validation calls. + * + * Since https://github.com/apache/incubator-gluten/pull/2185. + */ +trait ValidatablePlan extends GlutenPlan with LogLevelUtil { + protected def glutenConf: GlutenConfig = GlutenConfig.getConf + + protected lazy val enableNativeValidation = glutenConf.enableNativeValidation + + /** + * Validate whether this SparkPlan supports to be transformed into substrait node in Native Code. + */ + final def doValidate(): ValidationResult = { + val schemaValidationResult = BackendsApiManager.getValidatorApiInstance + .doSchemaValidate(schema) + .map { + reason => + ValidationResult.failed(s"Found schema check failure for $schema, due to: $reason") + } + .getOrElse(ValidationResult.succeeded) + if (!schemaValidationResult.ok()) { + TestStats.addFallBackClassName(this.getClass.toString) + return schemaValidationResult + } + try { + TransformerState.enterValidation + val res = doValidateInternal() + if (!res.ok()) { + TestStats.addFallBackClassName(this.getClass.toString) + } + res + } catch { + case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => + if (!e.isInstanceOf[GlutenNotSupportException]) { + logDebug(s"Just a warning. This exception perhaps needs to be fixed.", e) + } + // FIXME: Use a validation-specific method to catch validation failures + TestStats.addFallBackClassName(this.getClass.toString) + logValidationMessage( + s"Validation failed with exception for plan: $nodeName, due to: ${e.getMessage}", + e) + ValidationResult.failed(e.getMessage) + } finally { + TransformerState.finishValidation + } + } + + protected def doValidateInternal(): ValidationResult = ValidationResult.succeeded + + private def logValidationMessage(msg: => String, e: Throwable): Unit = { + if (glutenConf.printStackOnValidationFailure) { + logOnLevel(glutenConf.validationLogLevel, msg, e) + } else { + logOnLevel(glutenConf.validationLogLevel, msg) + } + } +} + +/** Base interface for a query plan that can be interpreted to Substrait representation. */ +trait TransformSupport extends ValidatablePlan { + override def batchType(): Convention.BatchType = { + BackendsApiManager.getSettings.primaryBatchType + } + + override def rowType0(): Convention.RowType = { + Convention.RowType.None + } final override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException( @@ -68,6 +138,17 @@ trait TransformSupport extends GlutenPlan { */ def columnarInputRDDs: Seq[RDD[ColumnarBatch]] + // Since https://github.com/apache/incubator-gluten/pull/2185. + protected def doNativeValidation(context: SubstraitContext, node: RelNode): ValidationResult = { + if (node != null && glutenConf.enableNativeValidation) { + val planNode = PlanBuilder.makePlan(context, Lists.newArrayList(node)) + BackendsApiManager.getValidatorApiInstance + .doNativeValidateWithFailureReason(planNode) + } else { + ValidationResult.succeeded + } + } + final def transform(context: SubstraitContext): TransformContext = { if (isCanonicalizedPlan) { throw new IllegalStateException( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala index 3639ac522f37..b49917c59fcb 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala @@ -16,25 +16,12 @@ */ package org.apache.gluten.extension -import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.expression.TransformerState import org.apache.gluten.extension.columnar.FallbackTag import org.apache.gluten.extension.columnar.FallbackTag.{Appendable, Converter} import org.apache.gluten.extension.columnar.FallbackTags.add -import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.extension.columnar.validator.Validator -import org.apache.gluten.logging.LogLevelUtil -import org.apache.gluten.substrait.SubstraitContext -import org.apache.gluten.substrait.plan.PlanBuilder -import org.apache.gluten.substrait.rel.RelNode -import org.apache.gluten.test.TestStats import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.execution.SparkPlan - -import com.google.common.collect.Lists sealed trait ValidationResult { def ok(): Boolean @@ -80,88 +67,3 @@ object ValidationResult { } } } - -/** Every Gluten Operator should extend this trait. */ -trait GlutenPlan - extends SparkPlan - with Convention.KnownBatchType - with Convention.KnownRowTypeForSpark33AndLater - with LogLevelUtil { - protected lazy val enableNativeValidation = glutenConf.enableNativeValidation - - protected def glutenConf: GlutenConfig = GlutenConfig.getConf - - /** - * Validate whether this SparkPlan supports to be transformed into substrait node in Native Code. - */ - final def doValidate(): ValidationResult = { - val schemaValidationResult = BackendsApiManager.getValidatorApiInstance - .doSchemaValidate(schema) - .map { - reason => - ValidationResult.failed(s"Found schema check failure for $schema, due to: $reason") - } - .getOrElse(ValidationResult.succeeded) - if (!schemaValidationResult.ok()) { - TestStats.addFallBackClassName(this.getClass.toString) - return schemaValidationResult - } - try { - TransformerState.enterValidation - val res = doValidateInternal() - if (!res.ok()) { - TestStats.addFallBackClassName(this.getClass.toString) - } - res - } catch { - case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => - if (!e.isInstanceOf[GlutenNotSupportException]) { - logDebug(s"Just a warning. This exception perhaps needs to be fixed.", e) - } - // FIXME: Use a validation-specific method to catch validation failures - TestStats.addFallBackClassName(this.getClass.toString) - logValidationMessage( - s"Validation failed with exception for plan: $nodeName, due to: ${e.getMessage}", - e) - ValidationResult.failed(e.getMessage) - } finally { - TransformerState.finishValidation - } - } - - final override val supportsColumnar: Boolean = { - batchType() != Convention.BatchType.None - } - - override def batchType(): Convention.BatchType = { - BackendsApiManager.getSettings.primaryBatchType - } - - final override val supportsRowBased: Boolean = { - rowType() != Convention.RowType.None - } - - override def rowType0(): Convention.RowType = { - Convention.RowType.None - } - - protected def doValidateInternal(): ValidationResult = ValidationResult.succeeded - - protected def doNativeValidation(context: SubstraitContext, node: RelNode): ValidationResult = { - if (node != null && glutenConf.enableNativeValidation) { - val planNode = PlanBuilder.makePlan(context, Lists.newArrayList(node)) - BackendsApiManager.getValidatorApiInstance - .doNativeValidateWithFailureReason(planNode) - } else { - ValidationResult.succeeded - } - } - - private def logValidationMessage(msg: => String, e: Throwable): Unit = { - if (glutenConf.printStackOnValidationFailure) { - logOnLevel(glutenConf.validationLogLevel, msg, e) - } else { - logOnLevel(glutenConf.validationLogLevel, msg) - } - } -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala index 49401797f91a..982abc16c527 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.enumerated -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{GlutenPlan, ValidatablePlan} import org.apache.gluten.extension.columnar.offload.OffloadSingleNode import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode import org.apache.gluten.extension.columnar.validator.Validator @@ -119,7 +119,9 @@ object RasOffload { validator.validate(from) match { case Validator.Passed => val offloadedPlan = base.offload(from) - val offloadedNodes = offloadedPlan.collect[GlutenPlan] { case t: GlutenPlan => t } + val offloadedNodes = offloadedPlan.collect[ValidatablePlan] { + case t: ValidatablePlan => t + } val outComes = offloadedNodes.map(_.doValidate()).filter(!_.ok()) if (outComes.nonEmpty) { // 5. If native validation fails on at least one of the offloaded nodes, return diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala index 5b5d5e541eb7..fd562406162e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala @@ -16,8 +16,7 @@ */ package org.apache.gluten.extension.columnar.enumerated -import org.apache.gluten.execution.{HashAggregateExecBaseTransformer, ShuffledHashJoinExecTransformerBase, SortExecTransformer} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{GlutenPlan, HashAggregateExecBaseTransformer, ShuffledHashJoinExecTransformerBase, SortExecTransformer} import org.apache.gluten.ras.path.Pattern._ import org.apache.gluten.ras.path.Pattern.Matchers._ import org.apache.gluten.ras.rule.{RasRule, Shape} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala index 44ed81f56581..3418d3dddc99 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala @@ -17,12 +17,11 @@ package org.apache.gluten.extension.columnar.heuristic import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} import org.apache.gluten.extension.columnar.FallbackTags.add import org.apache.gluten.extension.columnar.transition.{BackendTransitions, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil - import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution._ @@ -35,6 +34,7 @@ import org.apache.spark.sql.execution.exchange.Exchange + // format: off /** * Note, this rule should only fallback to row-based plan if there is no harm. diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala index 0fee61acace9..fa698cd244cc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala @@ -20,7 +20,6 @@ import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.sql.shims.SparkShimLoader @@ -330,8 +329,7 @@ object OffloadOthers { child) case p if !p.isInstanceOf[GlutenPlan] => logDebug(s"Transformation for ${p.getClass} is currently not supported.") - val children = plan.children - p.withNewChildren(children) + p case other => other } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala index 7f41e810721b..7e7d732c29a2 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala @@ -19,7 +19,7 @@ package org.apache.gluten.extension.columnar.validator import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.{BackendsApiManager, BackendSettingsApi} import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.execution.{BasicScanExecTransformer, ColumnarCoalesceExec, ColumnarUnionExec, ExpandExecTransformer, HashAggregateExecBaseTransformer, LimitExecTransformer, ProjectExecTransformer, ScanTransformerFactory, SortExecTransformer, TakeOrderedAndProjectExecTransformer, WindowExecTransformer, WindowGroupLimitExecTransformer, WriteFilesExecTransformer} +import org.apache.gluten.execution._ import org.apache.gluten.expression.ExpressionUtils import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.extension.columnar.offload.OffloadJoin diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala index 01a4380a148f..1de490ad6165 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.ValidatablePlan +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.GlutenTimeMetric import org.apache.gluten.sql.shims.SparkShimLoader @@ -41,7 +42,7 @@ import scala.util.control.NonFatal case class ColumnarBroadcastExchangeExec(mode: BroadcastMode, child: SparkPlan) extends BroadcastExchangeLike - with GlutenPlan { + with ValidatablePlan { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics: Map[String, SQLMetric] = @@ -125,6 +126,10 @@ case class ColumnarBroadcastExchangeExec(mode: BroadcastMode, child: SparkPlan) override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override def doCanonicalize(): SparkPlan = { val canonicalized = BackendsApiManager.getSparkPlanExecApiInstance.doCanonicalizeForBroadcastMode(mode) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index 9ec078e003da..67895b439c2e 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution._ -import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.rel.RelBuilder @@ -173,13 +173,22 @@ case class ColumnarCollapseTransformStages( } } +// TODO: Make this inherit from GlutenPlan. case class ColumnarInputAdapter(child: SparkPlan) extends InputAdapterGenerateTreeStringShim - with Convention.KnownBatchType { + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater + with GlutenPlan.SupportsRowBasedCompatible + with ConventionReq.KnownChildConvention { override def output: Seq[Attribute] = child.output - override val supportsColumnar: Boolean = true + final override val supportsColumnar: Boolean = true + final override val supportsRowBased: Boolean = false + override def rowType0(): Convention.RowType = Convention.RowType.None override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def requiredChildConvention(): Seq[ConventionReq] = Seq( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() override def outputPartitioning: Partitioning = child.outputPartitioning diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala index d4b33be292df..007a93bd06a1 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.execution.ValidatablePlan +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark._ @@ -45,7 +47,7 @@ case class ColumnarShuffleExchangeExec( projectOutputAttributes: Seq[Attribute], advisoryPartitionSize: Option[Long] = None) extends ShuffleExchangeLike - with GlutenPlan { + with ValidatablePlan { private[sql] lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) @@ -148,6 +150,10 @@ case class ColumnarShuffleExchangeExec( super.stringArgs ++ Iterator(s"[shuffle_writer_type=$shuffleWriterType]") } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException() } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala index 2c1edd04bb4a..6275fbb3aa3c 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.GlutenTimeMetric import org.apache.spark.rdd.RDD @@ -107,6 +108,10 @@ case class ColumnarSubqueryBroadcastExec( relationFuture } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException( "SubqueryBroadcastExec does not support the execute() code path.") diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala index 25d6c4ed61e7..45f202637020 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenException -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.gluten.extension.columnar.transition.Convention.RowType -import org.apache.gluten.extension.columnar.transition.ConventionReq import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.TaskContext @@ -44,17 +44,19 @@ abstract class ColumnarWriteFilesExec protected ( override val right: SparkPlan) extends BinaryExecNode with GlutenPlan - with ConventionReq.KnownChildrenConventions with ColumnarWriteFilesExec.ExecuteWriteCompatible { val child: SparkPlan = left override lazy val references: AttributeSet = AttributeSet.empty - override def requiredChildrenConventions(): Seq[ConventionReq] = { - List( - ConventionReq.ofBatch( - ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) + override def requiredChildConvention(): Seq[ConventionReq] = { + val req = ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType)) + Seq.tabulate(2)( + _ => { + req + }) } /** @@ -67,6 +69,7 @@ abstract class ColumnarWriteFilesExec protected ( * * Since https://github.com/apache/incubator-gluten/pull/6745. */ + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType override def rowType0(): RowType = { RowType.VanillaRow } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala index d6167c931cd6..7a3335052423 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql.execution -import org.apache.gluten.execution.WholeStageTransformer -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{GlutenPlan, WholeStageTransformer} import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.utils.PlanUtil diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala index 481e16b0a5be..c28395941b0c 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.events.GlutenPlanFallbackEvent -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.logging.LogLevelUtil diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala index 435bf9239e8a..709673feab0b 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala @@ -16,16 +16,14 @@ */ package org.apache.spark.sql.execution -import org.apache.gluten.execution.WholeStageTransformer -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{GlutenPlan, WholeStageTransformer} import org.apache.gluten.utils.PlanUtil - import org.apache.spark.sql.{AnalysisException, Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, LogicalPlan} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.execution.ColumnarWriteFilesExec.NoopLeaf -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, AdaptiveSparkPlanExec, QueryStageExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} import org.apache.spark.sql.execution.datasources.WriteFilesExec diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala index fb42c55ba0a7..126417bf18a5 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala @@ -17,9 +17,8 @@ package org.apache.spark.sql.execution.datasources import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.execution.ColumnarToRowExecBase +import org.apache.gluten.execution.{ColumnarToRowExecBase, GlutenPlan} import org.apache.gluten.execution.datasource.GlutenFormatFactory -import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.extension.columnar.transition.{Convention, Transitions} import org.apache.spark.rdd.RDD @@ -61,6 +60,8 @@ case class FakeRowAdaptor(child: SparkPlan) override def output: Seq[Attribute] = child.output + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow override protected def doExecute(): RDD[InternalRow] = { diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala index 5daca9bede9e..fec36ac1acfa 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala @@ -17,8 +17,7 @@ package org.apache.gluten.extension.columnar.transition import org.apache.gluten.exception.GlutenException -import org.apache.gluten.execution.ColumnarToColumnarExec -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{ColumnarToColumnarExec, GlutenPlan} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -117,6 +116,7 @@ object TransitionSuite extends TransitionSuiteBase { extends RowToColumnarTransition with GlutenPlan { override def batchType(): Convention.BatchType = toBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) override protected def doExecute(): RDD[InternalRow] = diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala index 43805b3d659b..7ab9b5d0cfaf 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -27,6 +27,7 @@ trait TransitionSuiteBase { case class BatchLeaf(override val batchType: Convention.BatchType) extends LeafExecNode with GlutenPlan { + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() @@ -36,6 +37,7 @@ trait TransitionSuiteBase { case class BatchUnary(override val batchType: Convention.BatchType, override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) @@ -51,6 +53,7 @@ trait TransitionSuiteBase { override val right: SparkPlan) extends BinaryExecNode with GlutenPlan { + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def withNewChildrenInternal( newLeft: SparkPlan, diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala b/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala index 3d26dd16c4eb..2a0ecaf92ca9 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.test -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ diff --git a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala index 164083a8d8b0..8507233a57fc 100644 --- a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala +++ b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala @@ -21,8 +21,8 @@ package org.apache.spark.sql * 1. We need to modify the way org.apache.spark.sql.CHQueryTest#compare compares double */ import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.execution.TransformSupport -import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.SPARK_VERSION_SHORT diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 7d4315e8d707..754c29241864 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.RemoveFallbackTagRule import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -200,6 +201,8 @@ private object FallbackStrategiesSuite { // For replacing LeafOp. case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } @@ -208,6 +211,8 @@ private object FallbackStrategiesSuite { case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index 7a3c4d1056e7..e525eb1a9c01 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index e8cc7898c2d4..88e0ecf65a9a 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -229,6 +230,8 @@ private object FallbackStrategiesSuite { // For replacing LeafOp. case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } @@ -237,6 +240,8 @@ private object FallbackStrategiesSuite { case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index 40874cd69d30..34e97273d603 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index e8cc7898c2d4..88e0ecf65a9a 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -229,6 +230,8 @@ private object FallbackStrategiesSuite { // For replacing LeafOp. case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } @@ -237,6 +240,8 @@ private object FallbackStrategiesSuite { case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala index 74c4df197759..e6cc2937a43e 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources import org.apache.gluten.execution.{ProjectExecTransformer, SortExecTransformer} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.executor.OutputMetrics diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index e5e122c9065d..f7cc114859cd 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 1d45e8a672e3..8908047a3321 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -230,6 +231,8 @@ private object FallbackStrategiesSuite { // For replacing LeafOp. case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } @@ -238,6 +241,8 @@ private object FallbackStrategiesSuite { case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala index 1cb905e10abf..38e032aec33f 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.sources import org.apache.gluten.GlutenColumnarWriteTestSupport +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.execution.SortExecTransformer -import org.apache.gluten.extension.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.executor.OutputMetrics diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index e5e122c9065d..f7cc114859cd 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala b/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala index d2a9611471ab..2088c90c019a 100644 --- a/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala +++ b/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.{ProjectExecTransformer, WholeStageTransformerSuite} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{ProjectExecTransformer, TransformSupport, WholeStageTransformerSuite} import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.SparkConf @@ -100,7 +99,8 @@ class GlutenExpressionDataTypesValidation extends WholeStageTransformerSuite { case _ => throw new UnsupportedOperationException("Not supported type: " + t) } } - def generateGlutenProjectPlan(expr: Expression): GlutenPlan = { + + def generateGlutenProjectPlan(expr: Expression): TransformSupport = { val namedExpr = Seq(Alias(expr, "r")()) ProjectExecTransformer(namedExpr, DummyPlan()) } From 77e8e283b0913601d2a1cf270e674b3d3f57308b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 27 Nov 2024 14:13:29 +0800 Subject: [PATCH 153/211] [GLUTEN-3839][CH] Extend nested column pruning in vanilla spark (#7992) * support column pruning on generator * rename config name * fix style * fix style * support multiple fields * fix failed uts * fix building issues in spark3.2 * fix style * fix building * Update NormalFileWriter.cpp * Update GlutenConfig.scala * change as reeust --- .../backendsapi/clickhouse/CHRuleApi.scala | 2 +- .../extension/ExtendedColumnPruning.scala | 366 ++++++++++++++++++ ...xtendedGeneratorNestedColumnAliasing.scala | 126 ------ .../hive/GlutenClickHouseHiveTableSuite.scala | 128 +++++- .../Storages/Output/NormalFileWriter.cpp | 1 + .../org/apache/gluten/GlutenConfig.scala | 10 +- 6 files changed, 480 insertions(+), 153 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala delete mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index f6d2b85d9d19..02ec91496ec0 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -63,7 +63,7 @@ object CHRuleApi { injector.injectResolutionRule(spark => new RewriteToDateExpresstionRule(spark)) injector.injectResolutionRule(spark => new RewriteDateTimestampComparisonRule(spark)) injector.injectOptimizerRule(spark => new CommonSubexpressionEliminateRule(spark)) - injector.injectOptimizerRule(spark => new ExtendedGeneratorNestedColumnAliasing(spark)) + injector.injectOptimizerRule(spark => new ExtendedColumnPruning(spark)) injector.injectOptimizerRule(spark => CHAggregateFunctionRewriteRule(spark)) injector.injectOptimizerRule(_ => CountDistinctWithoutExpand) injector.injectOptimizerRule(_ => EqualToRewrite) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala new file mode 100644 index 000000000000..f2a0a549bc7e --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction +import org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing.canPruneGenerator +import org.apache.spark.sql.catalyst.optimizer.NestedColumnAliasing +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.AlwaysProcess +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import scala.collection.mutable + +object ExtendedGeneratorNestedColumnAliasing { + def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { + case pj @ Project(projectList, f @ Filter(condition, g: Generate)) + if canPruneGenerator(g.generator) && + GlutenConfig.getConf.enableExtendedColumnPruning && + (SQLConf.get.nestedPruningOnExpressions || SQLConf.get.nestedSchemaPruningEnabled) => + val attrToExtractValues = + getAttributeToExtractValues(projectList ++ g.generator.children :+ condition, Seq.empty) + if (attrToExtractValues.isEmpty) { + return None + } + + val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput) + var (attrToExtractValuesOnGenerator, attrToExtractValuesNotOnGenerator) = + attrToExtractValues.partition { + case (attr, _) => + attr.references.subsetOf(generatorOutputSet) + } + + val pushedThrough = rewritePlanWithAliases(pj, attrToExtractValuesNotOnGenerator) + + // We cannot push through if the child of generator is `MapType`. + g.generator.children.head.dataType match { + case _: MapType => return Some(pushedThrough) + case _ => + } + + if (!g.generator.isInstanceOf[ExplodeBase]) { + return Some(pushedThrough) + } + + // In spark3.2, we could not reuse [[NestedColumnAliasing.getAttributeToExtractValues]] + // which only accepts 2 arguments. Instead we redefine it in current file to avoid moving + // this rule to gluten-shims + attrToExtractValuesOnGenerator = getAttributeToExtractValues( + attrToExtractValuesOnGenerator.flatMap(_._2).toSeq, + Seq.empty, + collectNestedGetStructFields) + + val nestedFieldsOnGenerator = attrToExtractValuesOnGenerator.values.flatten.toSet + if (nestedFieldsOnGenerator.isEmpty) { + return Some(pushedThrough) + } + + // Multiple or single nested column accessors. + // E.g. df.select(explode($"items").as("item")).select($"item.a", $"item.b") + pushedThrough match { + case p2 @ Project(_, f2 @ Filter(_, g2: Generate)) => + val nestedFieldsOnGeneratorSeq = nestedFieldsOnGenerator.toSeq + val nestedFieldToOrdinal = nestedFieldsOnGeneratorSeq.zipWithIndex.toMap + val rewrittenG = g2.transformExpressions { + case e: ExplodeBase => + val extractors = nestedFieldsOnGeneratorSeq.map(replaceGenerator(e, _)) + val names = extractors.map { + case g: GetStructField => Literal(g.extractFieldName) + case ga: GetArrayStructFields => Literal(ga.field.name) + case other => + throw new IllegalStateException( + s"Unreasonable extractor " + + "after replaceGenerator: $other") + } + val zippedArray = ArraysZip(extractors, names) + e.withNewChildren(Seq(zippedArray)) + } + // As we change the child of the generator, its output data type must be updated. + val updatedGeneratorOutput = rewrittenG.generatorOutput + .zip( + rewrittenG.generator.elementSchema.map( + f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())) + .map { + case (oldAttr, newAttr) => + newAttr.withExprId(oldAttr.exprId).withName(oldAttr.name) + } + assert( + updatedGeneratorOutput.length == rewrittenG.generatorOutput.length, + "Updated generator output must have the same length " + + "with original generator output." + ) + val updatedGenerate = rewrittenG.copy(generatorOutput = updatedGeneratorOutput) + + // Replace nested column accessor with generator output. + val attrExprIdsOnGenerator = attrToExtractValuesOnGenerator.keys.map(_.exprId).toSet + val updatedFilter = f2.withNewChildren(Seq(updatedGenerate)).transformExpressions { + case f: GetStructField if nestedFieldsOnGenerator.contains(f) => + replaceGetStructField( + f, + updatedGenerate.output, + attrExprIdsOnGenerator, + nestedFieldToOrdinal) + } + + val updatedProject = p2.withNewChildren(Seq(updatedFilter)).transformExpressions { + case f: GetStructField if nestedFieldsOnGenerator.contains(f) => + replaceGetStructField( + f, + updatedFilter.output, + attrExprIdsOnGenerator, + nestedFieldToOrdinal) + } + + Some(updatedProject) + case other => + throw new IllegalStateException(s"Unreasonable plan after optimization: $other") + } + case _ => + None + } + + /** + * Returns two types of expressions: + * - Root references that are individually accessed + * - [[GetStructField]] or [[GetArrayStructFields]] on top of other [[ExtractValue]]s or special + * expressions. + */ + private def collectRootReferenceAndExtractValue(e: Expression): Seq[Expression] = e match { + case _: AttributeReference => Seq(e) + case GetStructField(_: ExtractValue | _: AttributeReference, _, _) => Seq(e) + case GetArrayStructFields( + _: MapValues | _: MapKeys | _: ExtractValue | _: AttributeReference, + _, + _, + _, + _) => + Seq(e) + case es if es.children.nonEmpty => es.children.flatMap(collectRootReferenceAndExtractValue) + case _ => Seq.empty + } + + /** + * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s. Nested field + * accessors of `exclusiveAttrs` are not considered in nested fields aliasing. + */ + private def getAttributeToExtractValues( + exprList: Seq[Expression], + exclusiveAttrs: Seq[Attribute], + extractor: (Expression) => Seq[Expression] = collectRootReferenceAndExtractValue) + : Map[Attribute, Seq[ExtractValue]] = { + + val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]() + val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]() + exprList.foreach { + e => + extractor(e).foreach { + // we can not alias the attr from lambda variable whose expr id is not available + case ev: ExtractValue if ev.find(_.isInstanceOf[NamedLambdaVariable]).isEmpty => + if (ev.references.size == 1) { + nestedFieldReferences.append(ev) + } + case ar: AttributeReference => otherRootReferences.append(ar) + case _ => // ignore + } + } + val exclusiveAttrSet = AttributeSet(exclusiveAttrs ++ otherRootReferences) + + // Remove cosmetic variations when we group extractors by their references + nestedFieldReferences + .filter(!_.references.subsetOf(exclusiveAttrSet)) + .groupBy(_.references.head.canonicalized.asInstanceOf[Attribute]) + .flatMap { + case (attr: Attribute, nestedFields: collection.Seq[ExtractValue]) => + // Check if `ExtractValue` expressions contain any aggregate functions in their tree. + // Those that do should not have an alias generated as it can lead to pushing the + // aggregate down into a projection. + def containsAggregateFunction(ev: ExtractValue): Boolean = + ev.find(_.isInstanceOf[AggregateFunction]).isDefined + + // Remove redundant [[ExtractValue]]s if they share the same parent nest field. + // For example, when `a.b` and `a.b.c` are in project list, we only need to alias `a.b`. + // Because `a.b` requires all of the inner fields of `b`, we cannot prune `a.b.c`. + val dedupNestedFields = nestedFields + .filter { + // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and + // [[GetStructField]] + case e @ (_: GetStructField | _: GetArrayStructFields) => + val child = e.children.head + nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty) + case _ => true + } + .distinct + // Discard [[ExtractValue]]s that contain aggregate functions. + .filterNot(containsAggregateFunction) + + // If all nested fields of `attr` are used, we don't need to introduce new aliases. + // By default, the [[ColumnPruning]] rule uses `attr` already. + // Note that we need to remove cosmetic variations first, so we only count a + // nested field once. + val numUsedNestedFields = dedupNestedFields + .map(_.canonicalized) + .distinct + .map(nestedField => totalFieldNum(nestedField.dataType)) + .sum + if (dedupNestedFields.nonEmpty && numUsedNestedFields < totalFieldNum(attr.dataType)) { + Some((attr, dedupNestedFields.toSeq)) + } else { + None + } + } + } + + /** + * Return total number of fields of this type. This is used as a threshold to use nested column + * pruning. It's okay to underestimate. If the number of reference is bigger than this, the parent + * reference is used instead of nested field references. + */ + private def totalFieldNum(dataType: DataType): Int = dataType match { + case StructType(fields) => fields.map(f => totalFieldNum(f.dataType)).sum + case ArrayType(elementType, _) => totalFieldNum(elementType) + case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType) + case _ => 1 // UDT and others + } + + private def replaceGetStructField( + g: GetStructField, + input: Seq[Attribute], + attrExprIdsOnGenerator: Set[ExprId], + nestedFieldToOrdinal: Map[ExtractValue, Int]): Expression = { + val attr = input.find(a => attrExprIdsOnGenerator.contains(a.exprId)) + attr match { + case Some(a) => + val ordinal = nestedFieldToOrdinal(g) + GetStructField(a, ordinal, g.name) + case None => g + } + } + + /** Replace the reference attribute of extractor expression with generator input. */ + private def replaceGenerator(generator: ExplodeBase, expr: Expression): Expression = { + expr match { + case a: Attribute if expr.references.contains(a) => + generator.child + case g: GetStructField => + // We cannot simply do a transformUp instead because if we replace the attribute + // `extractFieldName` could cause `ClassCastException` error. We need to get the + // field name before replacing down the attribute/other extractor. + val fieldName = g.extractFieldName + val newChild = replaceGenerator(generator, g.child) + ExtractValue(newChild, Literal(fieldName), SQLConf.get.resolver) + case other => + other.mapChildren(replaceGenerator(generator, _)) + } + } + + // This function collects all GetStructField*(attribute) from the passed in expression. + // GetStructField* means arbitrary levels of nesting. + private def collectNestedGetStructFields(e: Expression): Seq[Expression] = { + // The helper function returns a tuple of + // (nested GetStructField including the current level, all other nested GetStructField) + def helper(e: Expression): (Seq[Expression], Seq[Expression]) = e match { + case _: AttributeReference => (Seq(e), Seq.empty) + case gsf: GetStructField => + val child_res = helper(gsf.child) + (child_res._1.map(p => gsf.withNewChildren(Seq(p))), child_res._2) + case other => + val child_res = other.children.map(helper) + val child_res_combined = (child_res.flatMap(_._1), child_res.flatMap(_._2)) + (Seq.empty, child_res_combined._1 ++ child_res_combined._2) + } + + val res = helper(e) + (res._1 ++ res._2).filterNot(_.isInstanceOf[Attribute]) + } + + private def rewritePlanWithAliases( + plan: LogicalPlan, + attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = { + val attributeToExtractValuesAndAliases = + attributeToExtractValues.map { + case (attr, evSeq) => + val evAliasSeq = evSeq.map { + ev => + val fieldName = ev match { + case g: GetStructField => g.extractFieldName + case g: GetArrayStructFields => g.field.name + } + ev -> Alias(ev, s"_extract_$fieldName")() + } + + attr -> evAliasSeq + } + + val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.map { + case (field, alias) => field.canonicalized -> alias + }.toMap + + // A reference attribute can have multiple aliases for nested fields. + val attrToAliases = + AttributeMap(attributeToExtractValuesAndAliases.mapValues(_.map(_._2)).toSeq) + + plan match { + // Project(Filter(Generate)) + case Project(projectList, f @ Filter(condition, g: Generate)) => + val newProjectList = NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias) + val newCondition = getNewExpression(condition, nestedFieldToAlias) + val newGenerator = getNewExpression(g.generator, nestedFieldToAlias).asInstanceOf[Generator] + + val tmpG = NestedColumnAliasing + .replaceWithAliases(g, nestedFieldToAlias, attrToAliases) + .asInstanceOf[Generate] + val newG = Generate( + newGenerator, + tmpG.unrequiredChildIndex, + tmpG.outer, + tmpG.qualifier, + tmpG.generatorOutput, + tmpG.children.head) + val newF = Filter(newCondition, newG) + val newP = Project(newProjectList, newF) + newP + case _ => plan + } + } + + private def getNewExpression( + expr: Expression, + nestedFieldToAlias: Map[Expression, Alias]): Expression = { + expr.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) => + nestedFieldToAlias(f.canonicalized).toAttribute + } + } +} + +// ExtendedColumnPruning process Project(Filter(Generate)), +// which is ignored by vanilla spark in optimization rule: ColumnPruning +class ExtendedColumnPruning(spark: SparkSession) extends Rule[LogicalPlan] with Logging { + + override def apply(plan: LogicalPlan): LogicalPlan = + plan.transformWithPruning(AlwaysProcess.fn) { + case ExtendedGeneratorNestedColumnAliasing(rewrittenPlan) => rewrittenPlan + case p => p + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala deleted file mode 100644 index a97e625ae618..000000000000 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.gluten.extension - -import org.apache.gluten.GlutenConfig - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing.canPruneGenerator -import org.apache.spark.sql.catalyst.optimizer.NestedColumnAliasing -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.AlwaysProcess -import org.apache.spark.sql.internal.SQLConf - -// ExtendedGeneratorNestedColumnAliasing process Project(Filter(Generate)), -// which is ignored by vanilla spark in optimization rule: ColumnPruning -class ExtendedGeneratorNestedColumnAliasing(spark: SparkSession) - extends Rule[LogicalPlan] - with Logging { - - override def apply(plan: LogicalPlan): LogicalPlan = - plan.transformWithPruning(AlwaysProcess.fn) { - case pj @ Project(projectList, f @ Filter(condition, g: Generate)) - if canPruneGenerator(g.generator) && - GlutenConfig.getConf.enableExtendedGeneratorNestedColumnAliasing && - (SQLConf.get.nestedPruningOnExpressions || SQLConf.get.nestedSchemaPruningEnabled) => - val attrToExtractValues = NestedColumnAliasing.getAttributeToExtractValues( - projectList ++ g.generator.children :+ condition, - Seq.empty) - if (attrToExtractValues.isEmpty) { - pj - } else { - val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput) - val (_, attrToExtractValuesNotOnGenerator) = - attrToExtractValues.partition { - case (attr, _) => - attr.references.subsetOf(generatorOutputSet) - } - - val pushedThrough = rewritePlanWithAliases(pj, attrToExtractValuesNotOnGenerator) - pushedThrough - } - case p => - p - } - - private def rewritePlanWithAliases( - plan: LogicalPlan, - attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = { - val attributeToExtractValuesAndAliases = - attributeToExtractValues.map { - case (attr, evSeq) => - val evAliasSeq = evSeq.map { - ev => - val fieldName = ev match { - case g: GetStructField => g.extractFieldName - case g: GetArrayStructFields => g.field.name - } - ev -> Alias(ev, s"_extract_$fieldName")() - } - - attr -> evAliasSeq - } - - val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.map { - case (field, alias) => field.canonicalized -> alias - }.toMap - - // A reference attribute can have multiple aliases for nested fields. - val attrToAliases = - AttributeMap(attributeToExtractValuesAndAliases.mapValues(_.map(_._2)).toSeq) - - plan match { - // Project(Filter(Generate)) - case p @ Project(projectList, child) - if child - .isInstanceOf[Filter] && child.asInstanceOf[Filter].child.isInstanceOf[Generate] => - val f = child.asInstanceOf[Filter] - val g = f.child.asInstanceOf[Generate] - - val newProjectList = NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias) - val newCondition = getNewExpression(f.condition, nestedFieldToAlias) - val newGenerator = getNewExpression(g.generator, nestedFieldToAlias).asInstanceOf[Generator] - - val tmpG = NestedColumnAliasing - .replaceWithAliases(g, nestedFieldToAlias, attrToAliases) - .asInstanceOf[Generate] - val newG = Generate( - newGenerator, - tmpG.unrequiredChildIndex, - tmpG.outer, - tmpG.qualifier, - tmpG.generatorOutput, - tmpG.children.head) - val newF = Filter(newCondition, newG) - val newP = Project(newProjectList, newF) - newP - case _ => plan - } - } - - private def getNewExpression( - expr: Expression, - nestedFieldToAlias: Map[Expression, Alias]): Expression = { - expr.transform { - case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) => - nestedFieldToAlias(f.canonicalized).toAttribute - } - } -} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index 7e31e73040d4..14d6ff53649f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{ArrayType, StructType} import org.apache.hadoop.fs.Path @@ -1473,30 +1473,116 @@ class GlutenClickHouseHiveTableSuite | 'event_info', map('tab_type', '4', 'action', '12'))) """.stripMargin) - val df = - spark.sql(""" - | SELECT * FROM ( - | SELECT - | game_name, - | CASE WHEN - | event.event_info['tab_type'] IN (5) THEN '1' ELSE '0' END AS entrance - | FROM aj - | LATERAL VIEW explode(split(nvl(event.event_info['game_name'],'0'),',')) - | as game_name - | WHERE event.event_info['action'] IN (13) - |) WHERE game_name = 'xxx' + val sql = """ + | SELECT * FROM ( + | SELECT + | game_name, + | CASE WHEN + | event.event_info['tab_type'] IN (5) THEN '1' ELSE '0' END AS entrance + | FROM aj + | LATERAL VIEW explode(split(nvl(event.event_info['game_name'],'0'),',')) + | as game_name + | WHERE event.event_info['action'] IN (13) + |) WHERE game_name = 'xxx' + """.stripMargin + + compareResultsAgainstVanillaSpark( + sql, + compareResult = true, + df => { + val scan = df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + val fieldType = scan.schema.fields.head.dataType.asInstanceOf[StructType] + assert(fieldType.size == 1) + } + ) + + spark.sql("drop table if exists aj") + } + + test("Nested column pruning for Project(Filter(Generate)) on generator") { + def assertFieldSizeAfterPruning(sql: String, expectSize: Int): Unit = { + compareResultsAgainstVanillaSpark( + sql, + compareResult = true, + df => { + val scan = df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + + val fieldType = + scan.schema.fields.head.dataType + .asInstanceOf[ArrayType] + .elementType + .asInstanceOf[StructType] + assert(fieldType.size == expectSize) + } + ) + } + + spark.sql("drop table if exists ajog") + spark.sql( + """ + |CREATE TABLE if not exists ajog ( + | country STRING, + | events ARRAY, event_id:STRING, event_info:MAP>> + |) + |USING orc """.stripMargin) - val scan = df.queryExecution.executedPlan.collect { - case scan: FileSourceScanExecTransformer => scan - }.head + spark.sql(""" + |INSERT INTO ajog VALUES + | ('USA', array(named_struct('time', 1622547800, 'lng', -122, 'lat', 37, 'net', + | 'wifi', 'log_extra', map('key1', 'value1'), 'event_id', 'event1', + | 'event_info', map('tab_type', '5', 'action', '13')))), + | ('Canada', array(named_struct('time', 1622547801, 'lng', -79, 'lat', 43, 'net', + | '4g', 'log_extra', map('key2', 'value2'), 'event_id', 'event2', + | 'event_info', map('tab_type', '4', 'action', '12')))) + """.stripMargin) - val schema = scan.schema - assert(schema.size == 1) - val fieldType = schema.fields.head.dataType.asInstanceOf[StructType] - assert(fieldType.size == 1) + // Test nested column pruning on generator with single field extracted + val sql1 = """ + |select + |case when event.event_info['tab_type'] in (5) then '1' else '0' end as entrance + |from ajog + |lateral view explode(events) as event + |where event.event_info['action'] in (13) + """.stripMargin + assertFieldSizeAfterPruning(sql1, 1) + + // Test nested column pruning on generator with multiple field extracted, + // which resolves SPARK-34956 + val sql2 = """ + |select event.event_id, + |case when event.event_info['tab_type'] in (5) then '1' else '0' end as entrance + |from ajog + |lateral view explode(events) as event + |where event.event_info['action'] in (13) + """.stripMargin + assertFieldSizeAfterPruning(sql2, 2) + + // Test nested column pruning with two adjacent generate operator + val sql3 = """ + |SELECT + |abflag, + |event.event_info, + |event.log_extra + |FROM + |ajog + |LATERAL VIEW EXPLODE(events) AS event + |LATERAL VIEW EXPLODE(split(event.log_extra['key1'], ',')) AS abflag + |WHERE + |event.event_id = 'event1' + |AND event.event_info['tab_type'] IS NOT NULL + |AND event.event_info['tab_type'] != '' + |AND event.log_extra['key1'] = 'value1' + |LIMIT 100; + """.stripMargin + assertFieldSizeAfterPruning(sql3, 3) - spark.sql("drop table if exists aj") + spark.sql("drop table if exists ajog") } test("test hive table scan nested column pruning") { diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index d572b8538591..e9fd4f358a86 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -18,6 +18,7 @@ #include #include +#include namespace local_engine { diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 2ccdcae99b5c..c4c67f49a59b 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -112,8 +112,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableCountDistinctWithoutExpand: Boolean = conf.getConf(ENABLE_COUNT_DISTINCT_WITHOUT_EXPAND) - def enableExtendedGeneratorNestedColumnAliasing: Boolean = - conf.getConf(ENABLE_EXTENDED_GENERATOR_NESTED_COLUMN_ALIASING) + def enableExtendedColumnPruning: Boolean = + conf.getConf(ENABLE_EXTENDED_COLUMN_PRUNING) def veloxOrcScanEnabled: Boolean = conf.getConf(VELOX_ORC_SCAN_ENABLED) @@ -1980,10 +1980,10 @@ object GlutenConfig { .booleanConf .createWithDefault(false) - val ENABLE_EXTENDED_GENERATOR_NESTED_COLUMN_ALIASING = - buildConf("spark.gluten.sql.extendedGeneratorNestedColumnAliasing") + val ENABLE_EXTENDED_COLUMN_PRUNING = + buildConf("spark.gluten.sql.extendedColumnPruning.enabled") .internal() - .doc("Do nested column aliasing for Project(Filter(Generator))") + .doc("Do extended nested column pruning for cases ignored by vanilla Spark.") .booleanConf .createWithDefault(true) From f633870b41613bd04d1a63c587b7c1de4ed234d4 Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Wed, 27 Nov 2024 14:16:05 +0800 Subject: [PATCH 154/211] [GLUTEN-8039][VL] Native writer should respect table properties (#8040) --- .../VeloxParquetWriteForHiveSuite.scala | 37 +++++++++++++++++++ .../sql/hive/execution/HiveFileFormat.scala | 10 +++-- .../sql/hive/execution/HiveFileFormat.scala | 9 +++-- 3 files changed, 50 insertions(+), 6 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala index 8c2b98988ea2..5932f4e5a741 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala @@ -27,6 +27,11 @@ import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.util.QueryExecutionListener +import org.apache.spark.util.Utils + +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.hadoop.util.HadoopInputFile class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { private var _spark: SparkSession = _ @@ -185,4 +190,36 @@ class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { checkAnswer(spark.table("t"), Row(1)) } } + + test("native writer should respect table properties") { + Seq(true, false).foreach { + enableNativeWrite => + withSQLConf("spark.gluten.sql.native.writer.enabled" -> enableNativeWrite.toString) { + withTable("t") { + withSQLConf( + "spark.sql.hive.convertMetastoreParquet" -> "false", + "spark.sql.parquet.compression.codec" -> "gzip") { + checkNativeWrite( + "CREATE TABLE t STORED AS PARQUET TBLPROPERTIES ('parquet.compression'='zstd') " + + "AS SELECT 1 as c", + checkNative = enableNativeWrite) + val tableDir = new Path(s"${conf.getConf(StaticSQLConf.WAREHOUSE_PATH)}/t") + val configuration = spark.sessionState.newHadoopConf() + val files = tableDir + .getFileSystem(configuration) + .listStatus(tableDir) + .filterNot(_.getPath.getName.startsWith("\\.")) + assert(files.nonEmpty) + val in = HadoopInputFile.fromStatus(files.head, spark.sessionState.newHadoopConf()) + Utils.tryWithResource(ParquetFileReader.open(in)) { + reader => + val column = reader.getFooter.getBlocks.get(0).getColumns.get(0) + // native writer and vanilla spark hive writer should be consistent + "zstd".equalsIgnoreCase(column.getCodec.toString) + } + } + } + } + } + } } diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index c21c67f65481..821d8317d87d 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -43,6 +43,7 @@ import org.apache.hadoop.mapred.{JobConf, Reporter} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import scala.collection.JavaConverters._ + /* - * This class is copied from Spark 3.2 and modified for Gluten. \n * Gluten should make sure this class is loaded before the original class. @@ -101,19 +102,22 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) val outputFormat = fileSinkConf.tableInfo.getOutputFileFormatClassName if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") + val tableOptions = tableDesc.getProperties.asScala.toMap val isParquetFormat = nativeFormat == "parquet" val compressionCodec = if (fileSinkConf.compressed) { // hive related configurations fileSinkConf.compressCodec } else if (isParquetFormat) { - val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) + val parquetOptions = + new ParquetOptions(tableOptions, sparkSession.sessionState.conf) parquetOptions.compressionCodecClassName } else { - val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + val orcOptions = new OrcOptions(tableOptions, sparkSession.sessionState.conf) orcOptions.compressionCodec } - val nativeConf = GlutenFormatFactory(nativeFormat).nativeConf(options, compressionCodec) + val nativeConf = + GlutenFormatFactory(nativeFormat).nativeConf(tableOptions, compressionCodec) new OutputWriterFactory { private val jobConf = new SerializableJobConf(new JobConf(conf)) diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 6ed1b4d21536..61ad8fe72bc8 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -98,19 +98,22 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) val outputFormat = fileSinkConf.tableInfo.getOutputFileFormatClassName if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") + val tableOptions = tableDesc.getProperties.asScala.toMap val isParquetFormat = nativeFormat == "parquet" val compressionCodec = if (fileSinkConf.compressed) { // hive related configurations fileSinkConf.compressCodec } else if (isParquetFormat) { - val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) + val parquetOptions = + new ParquetOptions(tableOptions, sparkSession.sessionState.conf) parquetOptions.compressionCodecClassName } else { - val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + val orcOptions = new OrcOptions(tableOptions, sparkSession.sessionState.conf) orcOptions.compressionCodec } - val nativeConf = GlutenFormatFactory(nativeFormat).nativeConf(options, compressionCodec) + val nativeConf = + GlutenFormatFactory(nativeFormat).nativeConf(tableOptions, compressionCodec) new OutputWriterFactory { private val jobConf = new SerializableJobConf(new JobConf(conf)) From 8265fc045f0a711bbb824bc5df11cb40b4293056 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Wed, 27 Nov 2024 16:12:29 +0800 Subject: [PATCH 155/211] [VL] Enable locate function test (#4791) --- .../org/apache/gluten/execution/VeloxStringFunctionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala index 1bef5f991fac..f1e5a42bc0a3 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala @@ -268,7 +268,7 @@ class VeloxStringFunctionsSuite extends VeloxWholeStageTransformerSuite { s"from $LINEITEM_TABLE limit $LENGTH")(checkGlutenOperatorMatch[ProjectExecTransformer]) } - ignore("locate") { + test("locate") { runQueryAndCompare( s"select l_orderkey, locate(l_comment, 'a', 1) " + s"from $LINEITEM_TABLE limit $LENGTH")(checkGlutenOperatorMatch[ProjectExecTransformer]) From a2deffc1eb9ca1678076ae7e3807d85f63b81f40 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 27 Nov 2024 16:54:37 +0800 Subject: [PATCH 156/211] [GLUTEN-6920][VL] Following #8036, append some code cleanups (#8058) --- .../python/ColumnarArrowEvalPythonExec.scala | 6 ++--- .../execution/ColumnarToColumnarExec.scala | 5 ++-- .../apache/gluten/execution/GlutenPlan.scala | 27 ++++++++++++++++--- .../planner/plan/GlutenPlanModel.scala | 2 +- .../columnar/transition/Convention.scala | 6 ++--- .../columnar/transition/Transitions.scala | 10 ++----- .../execution/ColumnarToRowExecBase.scala | 2 -- .../ColumnarCollapseTransformStages.scala | 2 +- 8 files changed, 36 insertions(+), 24 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala index 548dec13beb0..4a5f43b7fdfa 100644 --- a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala @@ -21,7 +21,6 @@ import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.exception.GlutenException import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildConvention import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.utils.PullOutProjectHelper @@ -212,15 +211,14 @@ case class ColumnarArrowEvalPythonExec( child: SparkPlan, evalType: Int) extends EvalPythonExec - with GlutenPlan - with KnownChildConvention { + with GlutenPlan { override def batchType(): Convention.BatchType = ArrowJavaBatch override def rowType0(): Convention.RowType = Convention.RowType.None override def requiredChildConvention(): Seq[ConventionReq] = List( - ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(ArrowJavaBatch))) + ConventionReq.ofBatch(ConventionReq.BatchType.Is(ArrowJavaBatch))) override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala index e27b6a4e2f1e..ac1f8d683570 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala @@ -50,8 +50,9 @@ abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention Convention.RowType.None } - override def requiredChildConvention(): Seq[ConventionReq] = List( - ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(from))) + override def requiredChildConvention(): Seq[ConventionReq] = { + List(ConventionReq.ofBatch(ConventionReq.BatchType.Is(from))) + } override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala index 460326d8d939..2cd408f67c3f 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala @@ -21,10 +21,31 @@ import org.apache.gluten.extension.columnar.transition.{Convention, ConventionRe import org.apache.spark.sql.execution.SparkPlan +/** + * Base interface for Query plan that defined by backends. + * + * The following Spark APIs are marked final so forbidden from overriding: + * - supportsColumnar + * - supportsRowBased (Spark version >= 3.3) + * + * Instead, subclasses are expected to implement the following APIs: + * - batchType + * - rowType0 + * - requiredChildConvention (optional) + * + * With implementations of the APIs provided, Gluten query planner will be able to find and insert + * proper transitions between different plan nodes. + * + * Implementing `requiredChildConvention` is optional while the default implementation is a sequence + * of convention reqs that are exactly the same with the output convention. If it's not the case for + * some plan types, then the API should be overridden. For example, a typical row-to-columnar + * transition is at the same time a query plan node that requires for row input however produces + * columnar output. + */ trait GlutenPlan extends SparkPlan with Convention.KnownBatchType - with Convention.KnownRowTypeForSpark33AndLater + with Convention.KnownRowTypeForSpark33OrLater with GlutenPlan.SupportsRowBasedCompatible with ConventionReq.KnownChildConvention { @@ -32,12 +53,12 @@ trait GlutenPlan batchType() != Convention.BatchType.None } - override def batchType(): Convention.BatchType - final override val supportsRowBased: Boolean = { rowType() != Convention.RowType.None } + override def batchType(): Convention.BatchType + override def rowType0(): Convention.RowType override def requiredChildConvention(): Seq[ConventionReq] = { diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala index 20a238a47d2c..94059020efca 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala @@ -45,7 +45,7 @@ object GlutenPlanModel { constraintSet: PropertySet[SparkPlan]) extends LeafExecNode with Convention.KnownBatchType - with Convention.KnownRowTypeForSpark33AndLater + with Convention.KnownRowTypeForSpark33OrLater with GlutenPlan.SupportsRowBasedCompatible { private val req: Conv.Req = constraintSet.get(ConvDef).asInstanceOf[Conv.Req] diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala index b9fbe023b27c..0e5387559674 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala @@ -157,9 +157,9 @@ object Convention { def rowType(): RowType } - trait KnownRowTypeForSpark33AndLater extends KnownRowType { + trait KnownRowTypeForSpark33OrLater extends KnownRowType { this: SparkPlan => - import KnownRowTypeForSpark33AndLater._ + import KnownRowTypeForSpark33OrLater._ final override def rowType(): RowType = { if (lteSpark32) { @@ -180,7 +180,7 @@ object Convention { def rowType0(): RowType } - object KnownRowTypeForSpark33AndLater { + object KnownRowTypeForSpark33OrLater { private val lteSpark32: Boolean = { val v = SparkVersionUtil.majorMinorVersion() SparkVersionUtil.compareMajorMinorVersion(v, (3, 2)) <= 0 diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala index cec09ee7a1ff..297485d84419 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala @@ -90,17 +90,11 @@ object Transitions { } def toRowPlan(plan: SparkPlan): SparkPlan = { - enforceReq( - plan, - ConventionReq.of( - ConventionReq.RowType.Is(Convention.RowType.VanillaRow), - ConventionReq.BatchType.Any)) + enforceReq(plan, ConventionReq.row) } def toBatchPlan(plan: SparkPlan, toBatchType: Convention.BatchType): SparkPlan = { - enforceReq( - plan, - ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(toBatchType))) + enforceReq(plan, ConventionReq.ofBatch(ConventionReq.BatchType.Is(toBatchType))) } def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala index 8bf1fbe44747..5beaf49572ac 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala @@ -18,7 +18,6 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildConvention import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -29,7 +28,6 @@ import org.apache.spark.sql.execution.{ColumnarToRowTransition, SparkPlan} abstract class ColumnarToRowExecBase(child: SparkPlan) extends ColumnarToRowTransition - with KnownChildConvention with ValidatablePlan { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index 67895b439c2e..2bdc1cf4b371 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -177,7 +177,7 @@ case class ColumnarCollapseTransformStages( case class ColumnarInputAdapter(child: SparkPlan) extends InputAdapterGenerateTreeStringShim with Convention.KnownBatchType - with Convention.KnownRowTypeForSpark33AndLater + with Convention.KnownRowTypeForSpark33OrLater with GlutenPlan.SupportsRowBasedCompatible with ConventionReq.KnownChildConvention { override def output: Seq[Attribute] = child.output From 18f742a87b3be63e4fc62274650f87e5336f03e2 Mon Sep 17 00:00:00 2001 From: Yabin Ma Date: Wed, 27 Nov 2024 13:15:41 +0100 Subject: [PATCH 157/211] [GLUTEN-7977][VL] Include cstdint header explicitly (#8030) --- cpp/core/config/GlutenConfig.h | 1 + cpp/velox/shuffle/RadixSort.h | 1 + 2 files changed, 2 insertions(+) diff --git a/cpp/core/config/GlutenConfig.h b/cpp/core/config/GlutenConfig.h index 47e5558f280e..3dd2f77f9fae 100644 --- a/cpp/core/config/GlutenConfig.h +++ b/cpp/core/config/GlutenConfig.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include diff --git a/cpp/velox/shuffle/RadixSort.h b/cpp/velox/shuffle/RadixSort.h index 90921962a17f..69b87346c54c 100644 --- a/cpp/velox/shuffle/RadixSort.h +++ b/cpp/velox/shuffle/RadixSort.h @@ -16,6 +16,7 @@ */ #include #include +#include #include #include From a85889c6c5cf82edddddc685d06894554df856d9 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Wed, 27 Nov 2024 20:17:42 +0800 Subject: [PATCH 158/211] [GLUTEN-8061][VL] Fall back nth_value if input is foldable (#8062) --- .../org/apache/gluten/backendsapi/velox/VeloxBackend.scala | 4 ++-- .../org/apache/gluten/execution/MiscOperatorSuite.scala | 7 +++++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index b1245687b0c2..45ffd7765299 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -399,8 +399,8 @@ object VeloxBackendSettings extends BackendSettingsApi { case _ => } windowExpression.windowFunction match { - case _: RowNumber | _: Rank | _: CumeDist | _: DenseRank | _: PercentRank | - _: NthValue | _: NTile => + case _: RowNumber | _: Rank | _: CumeDist | _: DenseRank | _: PercentRank | _: NTile => + case nv: NthValue if !nv.input.foldable => case l: Lag if !l.input.foldable => case l: Lead if !l.input.foldable => case aggrExpr: AggregateExpression diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index 58c748d37bfa..5cb2b652604d 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -512,6 +512,13 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa checkGlutenOperatorMatch[WindowExecTransformer] } } + + // Foldable input of nth_value is not supported. + runQueryAndCompare( + "select l_suppkey, l_orderkey, nth_value(1, 2) over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { + checkSparkOperatorMatch[WindowExec] + } } } From 2bc0d7736d4379ac927b1dc77ff09c40d7756983 Mon Sep 17 00:00:00 2001 From: Yuan Date: Wed, 27 Nov 2024 22:08:47 +0800 Subject: [PATCH 159/211] [GLUTEN-8046][VL][CI] Fix ccache path (#8064) --- .github/workflows/velox_backend.yml | 1 + .github/workflows/velox_backend_cache.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 3c4bb3e2e97d..c3765d2f4f6d 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -72,6 +72,7 @@ jobs: df -a cd /work export CCACHE_DIR=/work/.ccache + mkdir -p /work/.ccache bash dev/ci-velox-buildstatic-centos-7.sh ccache -s mkdir -p /work/.m2/repository/org/apache/arrow/ diff --git a/.github/workflows/velox_backend_cache.yml b/.github/workflows/velox_backend_cache.yml index d0b295b5f48e..f590c6d19c4a 100644 --- a/.github/workflows/velox_backend_cache.yml +++ b/.github/workflows/velox_backend_cache.yml @@ -43,6 +43,7 @@ jobs: - name: Build Gluten native libraries run: | docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + mkdir -p /work/.ccache bash dev/ci-velox-buildstatic-centos-7.sh " - name: Save Ccache From 883d02641c82631491325f38e9484347b939a6a3 Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 28 Nov 2024 11:50:12 +0800 Subject: [PATCH 160/211] [GLUTEN-7905][CH] Implete window's `topk` by aggregation (#7976) What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Fixes: #7905 This PR will use aggregation to calculate window's topk automatically when the partition keys are low cardinality ones. How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- .../backendsapi/clickhouse/CHBackend.scala | 8 + .../backendsapi/clickhouse/CHRuleApi.scala | 1 + ...CHAggregateGroupLimitExecTransformer.scala | 174 ++++++ .../extension/ConvertWindowToAggregate.scala | 138 +++++ ...enClickHouseTPCHSaltNullParquetSuite.scala | 151 ++++++ .../GroupLimitFunctions.cpp | 303 +++++++++++ cpp-ch/local-engine/Common/AggregateUtil.cpp | 1 - .../local-engine/Common/ArrayJoinHelper.cpp | 175 ++++++ cpp-ch/local-engine/Common/ArrayJoinHelper.h | 39 ++ cpp-ch/local-engine/Common/CHUtil.cpp | 2 + cpp-ch/local-engine/Common/GlutenConfig.cpp | 11 +- cpp-ch/local-engine/Common/GlutenConfig.h | 22 +- cpp-ch/local-engine/Common/SortUtils.cpp | 105 ++++ cpp-ch/local-engine/Common/SortUtils.h | 33 ++ cpp-ch/local-engine/Operator/BranchStep.cpp | 265 +++++++++ cpp-ch/local-engine/Operator/BranchStep.h | 86 +++ .../local-engine/Operator/BranchTransform.cpp | 155 ++++++ .../local-engine/Operator/BranchTransform.h | 56 ++ .../Operator/WindowGroupLimitStep.cpp | 23 +- .../Parser/AdvancedParametersParseUtil.cpp | 1 + .../Parser/AdvancedParametersParseUtil.h | 1 + .../Parser/RelParsers/GroupLimitRelParser.cpp | 512 ++++++++++++++++++ .../Parser/RelParsers/GroupLimitRelParser.h | 88 +++ .../Parser/RelParsers/ProjectRelParser.cpp | 92 +--- .../Parser/RelParsers/ProjectRelParser.h | 12 - .../RelParsers/WindowGroupLimitRelParser.cpp | 92 ---- .../RelParsers/WindowGroupLimitRelParser.h | 46 -- 27 files changed, 2328 insertions(+), 264 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala create mode 100644 cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp create mode 100644 cpp-ch/local-engine/Common/ArrayJoinHelper.cpp create mode 100644 cpp-ch/local-engine/Common/ArrayJoinHelper.h create mode 100644 cpp-ch/local-engine/Common/SortUtils.cpp create mode 100644 cpp-ch/local-engine/Common/SortUtils.h create mode 100644 cpp-ch/local-engine/Operator/BranchStep.cpp create mode 100644 cpp-ch/local-engine/Operator/BranchStep.h create mode 100644 cpp-ch/local-engine/Operator/BranchTransform.cpp create mode 100644 cpp-ch/local-engine/Operator/BranchTransform.h create mode 100644 cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp create mode 100644 cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h delete mode 100644 cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.cpp delete mode 100644 cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.h diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index f6cacba42b9f..c6c8acf7050e 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -359,6 +359,14 @@ object CHBackendSettings extends BackendSettingsApi with Logging { ) } + // If the partition keys are high cardinality, the aggregation method is slower. + def enableConvertWindowGroupLimitToAggregate(): Boolean = { + SparkEnv.get.conf.getBoolean( + CHConf.runtimeConfig("enable_window_group_limit_to_aggregate"), + defaultValue = true + ) + } + override def enableNativeWriteFiles(): Boolean = { GlutenConfig.getConf.enableNativeWriter.getOrElse(false) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 02ec91496ec0..edf7a4802535 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -110,6 +110,7 @@ object CHRuleApi { injector.injectPostTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session)) injector.injectPostTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session)) injector.injectPostTransform(c => LazyAggregateExpandRule.apply(c.session)) + injector.injectPostTransform(c => ConverRowNumbertWindowToAggregateRule(c.session)) injector.injectPostTransform( c => intercept( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala new file mode 100644 index 000000000000..83bb33bfa225 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.exception.GlutenNotSupportException +import org.apache.gluten.expression._ +import org.apache.gluten.expression.{ConverterUtils, ExpressionConverter} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.metrics.MetricsUpdater +import org.apache.gluten.substrait.`type`.TypeBuilder +import org.apache.gluten.substrait.SubstraitContext +import org.apache.gluten.substrait.extensions.ExtensionBuilder +import org.apache.gluten.substrait.rel.{RelBuilder, RelNode} + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.execution.SparkPlan + +import com.google.protobuf.StringValue +import io.substrait.proto.SortField + +import scala.collection.JavaConverters._ + +case class CHAggregateGroupLimitExecTransformer( + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + rankLikeFunction: Expression, + resultAttributes: Seq[Attribute], + limit: Int, + child: SparkPlan) + extends UnaryTransformSupport { + + @transient override lazy val metrics = + BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetrics(sparkContext) + + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + copy(child = newChild) + + override def metricsUpdater(): MetricsUpdater = + BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetricsUpdater(metrics) + + override def output: Seq[Attribute] = resultAttributes + + override def requiredChildDistribution: Seq[Distribution] = { + if (partitionSpec.isEmpty) { + // Only show warning when the number of bytes is larger than 100 MiB? + logWarning( + "No Partition Defined for Window operation! Moving all data to a single " + + "partition, this can cause serious performance degradation.") + AllTuples :: Nil + } else ClusteredDistribution(partitionSpec) :: Nil + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + Seq(Nil) + } + + override def outputOrdering: Seq[SortOrder] = { + if (requiredChildOrdering.forall(_.isEmpty)) { + Nil + } else { + child.outputOrdering + } + } + + override def outputPartitioning: Partitioning = child.outputPartitioning + + def getWindowGroupLimitRel( + context: SubstraitContext, + originalInputAttributes: Seq[Attribute], + operatorId: Long, + input: RelNode, + validation: Boolean): RelNode = { + val args = context.registeredFunction + // Partition By Expressions + val partitionsExpressions = partitionSpec + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, attributeSeq = child.output) + .doTransform(args)) + .asJava + + // Sort By Expressions + val sortFieldList = + orderSpec.map { + order => + val builder = SortField.newBuilder() + val exprNode = ExpressionConverter + .replaceWithExpressionTransformer(order.child, attributeSeq = child.output) + .doTransform(args) + builder.setExpr(exprNode.toProtobuf) + builder.setDirectionValue(SortExecTransformer.transformSortDirection(order)) + builder.build() + }.asJava + if (!validation) { + val windowFunction = rankLikeFunction match { + case _: RowNumber => ExpressionNames.ROW_NUMBER + case _: Rank => ExpressionNames.RANK + case _: DenseRank => ExpressionNames.DENSE_RANK + case _ => throw new GlutenNotSupportException(s"Unknow window function $rankLikeFunction") + } + val parametersStr = new StringBuffer("WindowGroupLimitParameters:") + parametersStr + .append("window_function=") + .append(windowFunction) + .append("\n") + .append("is_aggregate_group_limit=true\n") + val message = StringValue.newBuilder().setValue(parametersStr.toString).build() + val extensionNode = ExtensionBuilder.makeAdvancedExtension( + BackendsApiManager.getTransformerApiInstance.packPBMessage(message), + null) + RelBuilder.makeWindowGroupLimitRel( + input, + partitionsExpressions, + sortFieldList, + limit, + extensionNode, + context, + operatorId) + } else { + // Use a extension node to send the input types through Substrait plan for validation. + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava + val extensionNode = ExtensionBuilder.makeAdvancedExtension( + BackendsApiManager.getTransformerApiInstance.packPBMessage( + TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) + + RelBuilder.makeWindowGroupLimitRel( + input, + partitionsExpressions, + sortFieldList, + limit, + extensionNode, + context, + operatorId) + } + } + + override protected def doValidateInternal(): ValidationResult = { + val substraitContext = new SubstraitContext + val operatorId = substraitContext.nextOperatorId(this.nodeName) + + val relNode = + getWindowGroupLimitRel(substraitContext, child.output, operatorId, null, validation = true) + + doNativeValidation(substraitContext, relNode) + } + + override protected def doTransform(context: SubstraitContext): TransformContext = { + val childCtx = child.asInstanceOf[TransformSupport].transform(context) + val operatorId = context.nextOperatorId(this.nodeName) + + val currRel = + getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false) + assert(currRel != null, "Window Group Limit Rel should be valid") + TransformContext(output, currRel) + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala new file mode 100644 index 000000000000..ad2b22ba6cc6 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.execution._ +import org.apache.gluten.expression.WindowFunctionsBuilder + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.rules.Rule +// import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types._ + +// When to find the first rows of partitions by window function, we can convert it to aggregate +// function. +case class ConverRowNumbertWindowToAggregateRule(spark: SparkSession) + extends Rule[SparkPlan] + with Logging { + + override def apply(plan: SparkPlan): SparkPlan = { + if (!CHBackendSettings.enableConvertWindowGroupLimitToAggregate) { + return plan + } + plan.transformUp { + case filter @ FilterExecTransformer( + condition, + window @ WindowExecTransformer( + windowExpressions, + partitionSpec, + orderSpec, + sort @ SortExecTransformer(_, _, _, _))) => + if ( + !isSupportedWindowFunction(windowExpressions) || !isTopKLimitFilter( + condition, + windowExpressions(0)) + ) { + logDebug( + s"xxx Not Supported case for converting window to aggregate. is topk limit: " + + s"${isTopKLimitFilter(condition, windowExpressions(0))}. is supported window " + + s"function: ${isSupportedWindowFunction(windowExpressions)}") + filter + } else { + val limit = getLimit(condition.asInstanceOf[BinaryComparison]) + if (limit < 1 || limit > 100) { + filter + } else { + val groupLimit = CHAggregateGroupLimitExecTransformer( + partitionSpec, + orderSpec, + extractWindowFunction(windowExpressions(0)), + sort.child.output ++ Seq(windowExpressions(0).toAttribute), + limit, + sort.child + ) + groupLimit + } + } + } + } + + def getLimit(e: BinaryComparison): Int = { + e match { + case _: EqualTo => evalIntLiteral(e.right).get + case _: LessThanOrEqual => evalIntLiteral(e.right).get + case _: LessThan => evalIntLiteral(e.right).get - 1 + case _ => throw new GlutenException(s"Unsupported comparison: $e") + } + } + + def evalIntLiteral(expr: Expression): Option[Int] = { + expr match { + case int @ Literal(value, IntegerType) => Some(value.asInstanceOf[Int]) + case long @ Literal(value, LongType) => Some(value.asInstanceOf[Long].toInt) + case _ => None + } + } + + def extractWindowFunction(windowExpression: NamedExpression): Expression = { + val aliasExpr = windowExpression.asInstanceOf[Alias] + WindowFunctionsBuilder.extractWindowExpression(aliasExpr.child).windowFunction + } + + def isSupportedWindowFunction(windowExpressions: Seq[NamedExpression]): Boolean = { + if (windowExpressions.length != 1) { + return false + } + val windowFunction = extractWindowFunction(windowExpressions(0)) + windowFunction match { + case _: RowNumber => true + case _ => false + } + } + + // If the filter condition is a constant limit condition, return the limit value. + def isTopKLimitFilter(condition: Expression, windowExpression: NamedExpression): Boolean = { + def isWindowFunctionResult( + condition: Expression, + windowExpression: NamedExpression): Boolean = { + condition match { + case attr: Attribute => + attr.semanticEquals(windowExpression.toAttribute) + case _ => false + } + } + if (!condition.isInstanceOf[BinaryComparison]) { + return false + } + val binaryComparison = condition.asInstanceOf[BinaryComparison] + val constLimit = evalIntLiteral(binaryComparison.right) + if (!isWindowFunctionResult(binaryComparison.left, windowExpression) || !constLimit.isDefined) { + return false + } + binaryComparison match { + case _: EqualTo => constLimit.get == 1 + case _: LessThanOrEqual | _: LessThan => true + case _ => false + } + } + +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index fbefc054fac7..f82acdc415b2 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -41,6 +41,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr override protected val tpchQueries: String = rootPath + "../../../../tools/gluten-it/common/src/main/resources/tpch-queries" override protected val queriesResults: String = rootPath + "queries-output" + val runtimeConfigPrefix = "spark.gluten.sql.columnar.backend.ch.runtime_config." override protected def sparkConf: SparkConf = { super.sparkConf @@ -3175,6 +3176,156 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr spark.sql("drop table if exists test_7647") } + test("GLUTEN-7905 get topk of window by aggregate") { + withSQLConf( + (runtimeConfigPrefix + "enable_window_group_limit_to_aggregate", "true"), + (runtimeConfigPrefix + "window.aggregate_topk_high_cardinality_threshold", "2.0")) { + def checkWindowGroupLimit(df: DataFrame): Unit = { + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: CHAggregateGroupLimitExecTransformer => e + case wgl: CHWindowGroupLimitExecTransformer => wgl + } + assert(expands.size >= 1) + } + spark.sql("create table test_win_top (a string, b int, c int) using parquet") + spark.sql(""" + |insert into test_win_top values + |('a', 3, 3), ('a', 1, 5), ('a', 2, 2), ('a', null, null), ('a', null, 1), + |('b', 1, 1), ('b', 2, 1), + |('c', 2, 3) + |""".stripMargin) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, + |row_number() over (partition by a order by b desc nulls first, c nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b desc, c nulls last) as r + |from test_win_top + |)where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls first, c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b , c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + spark.sql("drop table if exists test_win_top") + } + + } + + test("GLUTEN-7905 get topk of window by window") { + withSQLConf( + (runtimeConfigPrefix + "enable_window_group_limit_to_aggregate", "true"), + (runtimeConfigPrefix + "window.aggregate_topk_high_cardinality_threshold", "0.0")) { + def checkWindowGroupLimit(df: DataFrame): Unit = { + // for spark 3.5, CHWindowGroupLimitExecTransformer is in used + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: CHAggregateGroupLimitExecTransformer => e + case wgl: CHWindowGroupLimitExecTransformer => wgl + } + assert(expands.size >= 1) + } + spark.sql("drop table if exists test_win_top") + spark.sql("create table test_win_top (a string, b int, c int) using parquet") + spark.sql(""" + |insert into test_win_top values + |('a', 3, 3), ('a', 1, 5), ('a', 2, 2), ('a', null, null), ('a', null, 1), + |('b', 1, 1), ('b', 2, 1), + |('c', 2, 3) + |""".stripMargin) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, + |row_number() over (partition by a order by b desc nulls first, c nulls last) as r + |from test_win_top + |)where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b desc, c nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + | select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls first, c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b , c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + spark.sql("drop table if exists test_win_top") + } + + } + test("GLUTEN-7759: Fix bug of agg pre-project push down") { val table_create_sql = "create table test_tbl_7759(id bigint, name string, day string) using parquet" diff --git a/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp new file mode 100644 index 000000000000..137ae8a54489 --- /dev/null +++ b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include "base/defines.h" + +namespace DB::ErrorCodes +{ +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +} + +namespace local_engine +{ + +struct SortOrderField +{ + size_t pos = 0; + Int8 direction = 0; + Int8 nulls_direction = 0; +}; +using SortOrderFields = std::vector; + +struct RowNumGroupArraySortedData +{ +public: + using Data = DB::Tuple; + std::vector values; + + static bool compare(const Data & lhs, const Data & rhs, const SortOrderFields & sort_orders) + { + for (const auto & sort_order : sort_orders) + { + const auto & pos = sort_order.pos; + const auto & asc = sort_order.direction; + const auto & nulls_first = sort_order.nulls_direction; + bool l_is_null = lhs[pos].isNull(); + bool r_is_null = rhs[pos].isNull(); + if (l_is_null && r_is_null) + continue; + else if (l_is_null) + return nulls_first; + else if (r_is_null) + return !nulls_first; + else if (lhs[pos] < rhs[pos]) + return asc; + else if (lhs[pos] > rhs[pos]) + return !asc; + } + return false; + } + + ALWAYS_INLINE void heapReplaceTop(const SortOrderFields & sort_orders) + { + size_t size = values.size(); + if (size < 2) + return; + size_t child_index = 1; + if (size > 2 && compare(values[1], values[2], sort_orders)) + ++child_index; + + if (compare(values[child_index], values[0], sort_orders)) + return; + + size_t current_index = 0; + auto current = values[current_index]; + do + { + values[current_index] = values[child_index]; + current_index = child_index; + + child_index = 2 * child_index + 1; + + if (child_index >= size) + break; + + if ((child_index + 1) < size && compare(values[child_index], values[child_index + 1], sort_orders)) + ++child_index; + } while (!compare(values[child_index], current, sort_orders)); + + values[current_index] = current; + } + + ALWAYS_INLINE void addElement(const Data && data, const SortOrderFields & sort_orders, size_t max_elements) + { + if (values.size() >= max_elements) + { + if (!compare(data, values[0], sort_orders)) + return; + values[0] = data; + heapReplaceTop(sort_orders); + return; + } + values.emplace_back(std::move(data)); + auto cmp = [&sort_orders](const Data & a, const Data & b) { return compare(a, b, sort_orders); }; + std::push_heap(values.begin(), values.end(), cmp); + } + + ALWAYS_INLINE void sortAndLimit(size_t max_elements, const SortOrderFields & sort_orders) + { + ::sort(values.begin(), values.end(), [&sort_orders](const Data & a, const Data & b) { return compare(a, b, sort_orders); }); + if (values.size() > max_elements) + values.resize(max_elements); + } + + ALWAYS_INLINE void insertResultInto(DB::IColumn & to, size_t max_elements, const SortOrderFields & sort_orders) + { + auto & result_array = assert_cast(to); + auto & result_array_offsets = result_array.getOffsets(); + + sortAndLimit(max_elements, sort_orders); + + result_array_offsets.push_back(result_array_offsets.back() + values.size()); + + if (values.empty()) + return; + auto & result_array_data = result_array.getData(); + for (int i = 0, sz = static_cast(values.size()); i < sz; ++i) + { + auto & value = values[i]; + value.push_back(i + 1); + result_array_data.insert(value); + } + } +}; + +static DB::DataTypePtr getRowNumReultDataType(DB::DataTypePtr data_type) +{ + const auto * tuple_type = typeid_cast(data_type.get()); + if (!tuple_type) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Tuple type is expected, but got: {}", data_type->getName()); + DB::DataTypes element_types = tuple_type->getElements(); + std::vector element_names = tuple_type->getElementNames(); + element_types.push_back(std::make_shared()); + element_names.push_back("row_num"); + auto nested_tuple_type = std::make_shared(element_types, element_names); + return std::make_shared(nested_tuple_type); +} + +// usage: rowNumGroupArraySorted(1, "a asc nulls first, b desc nulls last")(tuple(a,b)) +class RowNumGroupArraySorted final : public DB::IAggregateFunctionDataHelper +{ +public: + explicit RowNumGroupArraySorted(DB::DataTypePtr data_type, const DB::Array & parameters_) + : DB::IAggregateFunctionDataHelper( + {data_type}, parameters_, getRowNumReultDataType(data_type)) + { + if (parameters_.size() != 2) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "{} needs two parameters: limit and order clause", getName()); + const auto * tuple_type = typeid_cast(data_type.get()); + if (!tuple_type) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Tuple type is expected, but got: {}", data_type->getName()); + + limit = parameters_[0].safeGet(); + + String order_by_clause = parameters_[1].safeGet(); + sort_order_fields = parseSortOrderFields(order_by_clause); + + serialization = data_type->getDefaultSerialization(); + } + + String getName() const override { return "rowNumGroupArraySorted"; } + + void add(DB::AggregateDataPtr __restrict place, const DB::IColumn ** columns, size_t row_num, DB::Arena * /*arena*/) const override + { + auto & data = this->data(place); + DB::Tuple data_tuple = (*columns[0])[row_num].safeGet(); + this->data(place).addElement(std::move(data_tuple), sort_order_fields, limit); + } + + void merge(DB::AggregateDataPtr __restrict place, DB::ConstAggregateDataPtr rhs, DB::Arena * /*arena*/) const override + { + auto & rhs_values = this->data(rhs).values; + for (auto & rhs_element : rhs_values) + this->data(place).addElement(std::move(rhs_element), sort_order_fields, limit); + } + + void serialize(DB::ConstAggregateDataPtr __restrict place, DB::WriteBuffer & buf, std::optional /* version */) const override + { + auto & values = this->data(place).values; + size_t size = values.size(); + DB::writeVarUInt(size, buf); + + for (const auto & value : values) + serialization->serializeBinary(value, buf, {}); + } + + void deserialize( + DB::AggregateDataPtr __restrict place, DB::ReadBuffer & buf, std::optional /* version */, DB::Arena *) const override + { + size_t size = 0; + DB::readVarUInt(size, buf); + + auto & values = this->data(place).values; + values.reserve(size); + for (size_t i = 0; i < size; ++i) + { + DB::Field data; + serialization->deserializeBinary(data, buf, {}); + values.emplace_back(data.safeGet()); + } + } + + void insertResultInto(DB::AggregateDataPtr __restrict place, DB::IColumn & to, DB::Arena * /*arena*/) const override + { + this->data(place).insertResultInto(to, limit, sort_order_fields); + } + + bool allocatesMemoryInArena() const override { return true; } + +private: + size_t limit = 0; + SortOrderFields sort_order_fields; + DB::SerializationPtr serialization; + + SortOrderFields parseSortOrderFields(const String & order_by_clause) const + { + DB::ParserOrderByExpressionList order_by_parser; + auto order_by_ast = DB::parseQuery(order_by_parser, order_by_clause, 1000, 1000, 1000); + SortOrderFields fields; + const auto expression_list_ast = assert_cast(order_by_ast.get()); + const auto & tuple_element_names = assert_cast(argument_types[0].get())->getElementNames(); + for (const auto & child : expression_list_ast->children) + { + const auto * order_by_element_ast = assert_cast(child.get()); + const auto * ident_ast = assert_cast(order_by_element_ast->children[0].get()); + const auto & ident_name = ident_ast->shortName(); + + + SortOrderField field; + field.direction = order_by_element_ast->direction == 1; + field.nulls_direction + = field.direction ? order_by_element_ast->nulls_direction == -1 : order_by_element_ast->nulls_direction == 1; + + auto name_pos = std::find(tuple_element_names.begin(), tuple_element_names.end(), ident_name); + if (name_pos == tuple_element_names.end()) + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, "Not found column {} in tuple {}", ident_name, argument_types[0]->getName()); + } + field.pos = std::distance(tuple_element_names.begin(), name_pos); + + fields.push_back(field); + } + return fields; + } +}; + + +DB::AggregateFunctionPtr createAggregateFunctionRowNumGroupArray( + const std::string & name, const DB::DataTypes & argument_types, const DB::Array & parameters, const DB::Settings *) +{ + if (argument_types.size() != 1 || !typeid_cast(argument_types[0].get())) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, " {} Nees only one tuple argument", name); + return std::make_shared(argument_types[0], parameters); +} + +void registerAggregateFunctionRowNumGroup(DB::AggregateFunctionFactory & factory) +{ + DB::AggregateFunctionProperties properties = {.returns_default_when_only_null = false, .is_order_dependent = false}; + + factory.registerFunction("rowNumGroupArraySorted", {createAggregateFunctionRowNumGroupArray, properties}); +} +} diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp index 2290747fa158..0707d18aa01b 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.cpp +++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp @@ -47,7 +47,6 @@ extern const SettingsBool enable_memory_bound_merging_of_aggregation_results; extern const SettingsUInt64 aggregation_in_order_max_block_bytes; extern const SettingsUInt64 group_by_two_level_threshold; extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; -extern const SettingsMaxThreads max_threads; extern const SettingsUInt64 max_block_size; } diff --git a/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp new file mode 100644 index 000000000000..acefad0aea2a --- /dev/null +++ b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "ArrayJoinHelper.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace +{ +extern const int LOGICAL_ERROR; +} + +namespace Setting +{ +extern const SettingsUInt64 max_block_size; +} +} + +namespace local_engine +{ +namespace ArrayJoinHelper +{ +const DB::ActionsDAG::Node * findArrayJoinNode(const DB::ActionsDAG & actions_dag) +{ + const DB::ActionsDAG::Node * array_join_node = nullptr; + const auto & nodes = actions_dag.getNodes(); + for (const auto & node : nodes) + { + if (node.type == DB::ActionsDAG::ActionType::ARRAY_JOIN) + { + if (array_join_node) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expect single ARRAY JOIN node in generate rel"); + array_join_node = &node; + } + } + return array_join_node; +} + +struct SplittedActionsDAGs +{ + DB::ActionsDAG before_array_join; /// Optional + DB::ActionsDAG array_join; + DB::ActionsDAG after_array_join; /// Optional +}; + +/// Split actions_dag of generate rel into 3 parts: before array join + during array join + after array join +static SplittedActionsDAGs splitActionsDAGInGenerate(const DB::ActionsDAG & actions_dag) +{ + SplittedActionsDAGs res; + + auto array_join_node = findArrayJoinNode(actions_dag); + std::unordered_set first_split_nodes(array_join_node->children.begin(), array_join_node->children.end()); + auto first_split_result = actions_dag.split(first_split_nodes); + res.before_array_join = std::move(first_split_result.first); + + array_join_node = findArrayJoinNode(first_split_result.second); + std::unordered_set second_split_nodes = {array_join_node}; + auto second_split_result = first_split_result.second.split(second_split_nodes); + res.array_join = std::move(second_split_result.first); + second_split_result.second.removeUnusedActions(); + res.after_array_join = std::move(second_split_result.second); + return res; +} + +DB::ActionsDAG applyArrayJoinOnOneColumn(const DB::Block & header, size_t column_index) +{ + auto arrayColumn = header.getByPosition(column_index); + if (!typeid_cast(arrayColumn.type.get())) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expect array column in array join"); + DB::ActionsDAG actions_dag(header.getColumnsWithTypeAndName()); + const auto * array_column_node = actions_dag.getInputs()[column_index]; + auto array_join_name = array_column_node->result_name; + const auto * array_join_node = &actions_dag.addArrayJoin(*array_column_node, array_join_name); + actions_dag.addOrReplaceInOutputs(*array_join_node); + return std::move(actions_dag); +} + + +std::vector +addArrayJoinStep(DB::ContextPtr context, DB::QueryPlan & plan, const DB::ActionsDAG & actions_dag, bool is_left) +{ + auto logger = getLogger("ArrayJoinHelper"); + std::vector steps; + if (findArrayJoinNode(actions_dag)) + { + /// If generator in generate rel is explode/posexplode, transform arrayJoin function to ARRAY JOIN STEP to apply max_block_size + /// which avoids OOM when several lateral view explode/posexplode is used in spark sqls + LOG_DEBUG(logger, "original actions_dag:{}", actions_dag.dumpDAG()); + auto splitted_actions_dags = splitActionsDAGInGenerate(actions_dag); + LOG_DEBUG(logger, "actions_dag before arrayJoin:{}", splitted_actions_dags.before_array_join.dumpDAG()); + LOG_DEBUG(logger, "actions_dag during arrayJoin:{}", splitted_actions_dags.array_join.dumpDAG()); + LOG_DEBUG(logger, "actions_dag after arrayJoin:{}", splitted_actions_dags.after_array_join.dumpDAG()); + + auto ignore_actions_dag = [](const DB::ActionsDAG & actions_dag_) -> bool + { + /* + We should ignore actions_dag like: + 0 : INPUT () (no column) String a + 1 : INPUT () (no column) String b + Output nodes: 0, 1 + */ + return actions_dag_.getOutputs().size() == actions_dag_.getNodes().size() + && actions_dag_.getInputs().size() == actions_dag_.getNodes().size(); + }; + + /// Pre-projection before array join + if (!ignore_actions_dag(splitted_actions_dags.before_array_join)) + { + auto step_before_array_join + = std::make_unique(plan.getCurrentHeader(), std::move(splitted_actions_dags.before_array_join)); + step_before_array_join->setStepDescription("Pre-projection In Generate"); + steps.emplace_back(step_before_array_join.get()); + plan.addStep(std::move(step_before_array_join)); + // LOG_DEBUG(logger, "plan1:{}", PlanUtil::explainPlan(*query_plan)); + } + + /// ARRAY JOIN + DB::Names array_joined_columns{findArrayJoinNode(splitted_actions_dags.array_join)->result_name}; + DB::ArrayJoin array_join; + array_join.columns = std::move(array_joined_columns); + array_join.is_left = is_left; + auto array_join_step = std::make_unique( + plan.getCurrentHeader(), std::move(array_join), false, context->getSettingsRef()[DB::Setting::max_block_size]); + array_join_step->setStepDescription("ARRAY JOIN In Generate"); + steps.emplace_back(array_join_step.get()); + plan.addStep(std::move(array_join_step)); + // LOG_DEBUG(logger, "plan2:{}", PlanUtil::explainPlan(*query_plan)); + + /// Post-projection after array join(Optional) + if (!ignore_actions_dag(splitted_actions_dags.after_array_join)) + { + auto step_after_array_join + = std::make_unique(plan.getCurrentHeader(), std::move(splitted_actions_dags.after_array_join)); + step_after_array_join->setStepDescription("Post-projection In Generate"); + steps.emplace_back(step_after_array_join.get()); + plan.addStep(std::move(step_after_array_join)); + // LOG_DEBUG(logger, "plan3:{}", PlanUtil::explainPlan(*query_plan)); + } + } + else + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expect array join node in actions_dag"); + } + + return steps; +} + + +} +} // namespace local_engine \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/ArrayJoinHelper.h b/cpp-ch/local-engine/Common/ArrayJoinHelper.h new file mode 100644 index 000000000000..a4b582b2f0c0 --- /dev/null +++ b/cpp-ch/local-engine/Common/ArrayJoinHelper.h @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once +#include + +namespace DB +{ +class IQueryPlanStep; +class QueryPlan; +} + +namespace local_engine +{ +namespace ArrayJoinHelper +{ +// apply array join on one column to flatten the array column +DB::ActionsDAG applyArrayJoinOnOneColumn(const DB::Block & header, size_t column_index); + +const DB::ActionsDAG::Node * findArrayJoinNode(const DB::ActionsDAG & actions_dag); + +// actions_dag is a actions dag that contains the array join node, if not, the plan will not be changed. +// return the steps that are added to the plan. +std::vector addArrayJoinStep(DB::ContextPtr context, DB::QueryPlan & plan, const DB::ActionsDAG & actions_dag, bool is_left); +} // namespace ArrayJoinHelper +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 8fef52e50a68..de1a26321801 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -887,6 +887,7 @@ void BackendInitializerUtil::applyGlobalConfigAndSettings(const DB::Context::Con extern void registerAggregateFunctionCombinatorPartialMerge(AggregateFunctionCombinatorFactory &); extern void registerAggregateFunctionsBloomFilter(AggregateFunctionFactory &); extern void registerAggregateFunctionSparkAvg(AggregateFunctionFactory &); +extern void registerAggregateFunctionRowNumGroup(AggregateFunctionFactory &); extern void registerFunctions(FunctionFactory &); void registerAllFunctions() @@ -897,6 +898,7 @@ void registerAllFunctions() auto & agg_factory = AggregateFunctionFactory::instance(); registerAggregateFunctionsBloomFilter(agg_factory); registerAggregateFunctionSparkAvg(agg_factory); + registerAggregateFunctionRowNumGroup(agg_factory); { /// register aggregate function combinators from local_engine auto & factory = AggregateFunctionCombinatorFactory::instance(); diff --git a/cpp-ch/local-engine/Common/GlutenConfig.cpp b/cpp-ch/local-engine/Common/GlutenConfig.cpp index 0cefbc383977..ce15a12f921a 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.cpp +++ b/cpp-ch/local-engine/Common/GlutenConfig.cpp @@ -141,4 +141,13 @@ MergeTreeCacheConfig MergeTreeCacheConfig::loadFromContext(const DB::ContextPtr config.enable_data_prefetch = context->getConfigRef().getBool(ENABLE_DATA_PREFETCH, config.enable_data_prefetch); return config; } -} \ No newline at end of file + +WindowConfig WindowConfig::loadFromContext(const DB::ContextPtr & context) +{ + WindowConfig config; + config.aggregate_topk_sample_rows = context->getConfigRef().getUInt64(WINDOW_AGGREGATE_TOPK_SAMPLE_ROWS, 5000); + config.aggregate_topk_high_cardinality_threshold + = context->getConfigRef().getDouble(WINDOW_AGGREGATE_TOPK_HIGH_CARDINALITY_THRESHOLD, 0.6); + return config; +} +} diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h index 8af83329b6b7..85839b70ecd2 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.h +++ b/cpp-ch/local-engine/Common/GlutenConfig.h @@ -56,9 +56,12 @@ struct GraceMergingAggregateConfig { inline static const String MAX_GRACE_AGGREGATE_MERGING_BUCKETS = "max_grace_aggregate_merging_buckets"; inline static const String THROW_ON_OVERFLOW_GRACE_AGGREGATE_MERGING_BUCKETS = "throw_on_overflow_grace_aggregate_merging_buckets"; - inline static const String AGGREGATED_KEYS_BEFORE_EXTEND_GRACE_AGGREGATE_MERGING_BUCKETS = "aggregated_keys_before_extend_grace_aggregate_merging_buckets"; - inline static const String MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET = "max_pending_flush_blocks_per_grace_aggregate_merging_bucket"; - inline static const String MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING = "max_allowed_memory_usage_ratio_for_aggregate_merging"; + inline static const String AGGREGATED_KEYS_BEFORE_EXTEND_GRACE_AGGREGATE_MERGING_BUCKETS + = "aggregated_keys_before_extend_grace_aggregate_merging_buckets"; + inline static const String MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET + = "max_pending_flush_blocks_per_grace_aggregate_merging_bucket"; + inline static const String MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING + = "max_allowed_memory_usage_ratio_for_aggregate_merging"; size_t max_grace_aggregate_merging_buckets = 32; bool throw_on_overflow_grace_aggregate_merging_buckets = false; @@ -73,7 +76,8 @@ struct StreamingAggregateConfig { inline static const String AGGREGATED_KEYS_BEFORE_STREAMING_AGGREGATING_EVICT = "aggregated_keys_before_streaming_aggregating_evict"; inline static const String MAX_MEMORY_USAGE_RATIO_FOR_STREAMING_AGGREGATING = "max_memory_usage_ratio_for_streaming_aggregating"; - inline static const String HIGH_CARDINALITY_THRESHOLD_FOR_STREAMING_AGGREGATING = "high_cardinality_threshold_for_streaming_aggregating"; + inline static const String HIGH_CARDINALITY_THRESHOLD_FOR_STREAMING_AGGREGATING + = "high_cardinality_threshold_for_streaming_aggregating"; inline static const String ENABLE_STREAMING_AGGREGATING = "enable_streaming_aggregating"; size_t aggregated_keys_before_streaming_aggregating_evict = 1024; @@ -154,6 +158,16 @@ struct MergeTreeCacheConfig static MergeTreeCacheConfig loadFromContext(const DB::ContextPtr & context); }; +struct WindowConfig +{ +public: + inline static const String WINDOW_AGGREGATE_TOPK_SAMPLE_ROWS = "window.aggregate_topk_sample_rows"; + inline static const String WINDOW_AGGREGATE_TOPK_HIGH_CARDINALITY_THRESHOLD = "window.aggregate_topk_high_cardinality_threshold"; + size_t aggregate_topk_sample_rows = 5000; + double aggregate_topk_high_cardinality_threshold = 0.6; + static WindowConfig loadFromContext(const DB::ContextPtr & context); +}; + namespace PathConfig { inline constexpr const char * USE_CURRENT_DIRECTORY_AS_TMP = "use_current_directory_as_tmp"; diff --git a/cpp-ch/local-engine/Common/SortUtils.cpp b/cpp-ch/local-engine/Common/SortUtils.cpp new file mode 100644 index 000000000000..1b18cc4bfaf5 --- /dev/null +++ b/cpp-ch/local-engine/Common/SortUtils.cpp @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "SortUtils.h" +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +} + +namespace local_engine +{ +DB::SortDescription parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & expressions) +{ + DB::SortDescription description; + for (const auto & expr : expressions) + if (expr.has_selection()) + { + auto pos = expr.selection().direct_reference().struct_field().field(); + const auto & col_name = header.getByPosition(pos).name; + description.push_back(DB::SortColumnDescription(col_name, 1, 1)); + } + else if (expr.has_literal()) + continue; + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow expression as sort field: {}", expr.DebugString()); + return description; +} + +DB::SortDescription parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields) +{ + static std::map> direction_map = {{1, {1, -1}}, {2, {1, 1}}, {3, {-1, 1}}, {4, {-1, -1}}}; + + DB::SortDescription sort_descr; + for (int i = 0, sz = sort_fields.size(); i < sz; ++i) + { + const auto & sort_field = sort_fields[i]; + /// There is no meaning to sort a const column. + if (sort_field.expr().has_literal()) + continue; + + if (!sort_field.expr().has_selection() || !sort_field.expr().selection().has_direct_reference() + || !sort_field.expr().selection().direct_reference().has_struct_field()) + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsupport sort field"); + } + auto field_pos = sort_field.expr().selection().direct_reference().struct_field().field(); + + auto direction_iter = direction_map.find(sort_field.direction()); + if (direction_iter == direction_map.end()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsuppor sort direction: {}", sort_field.direction()); + const auto & col_name = header.getByPosition(field_pos).name; + sort_descr.emplace_back(col_name, direction_iter->second.first, direction_iter->second.second); + } + return sort_descr; +} + +std::string +buildSQLLikeSortDescription(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields) +{ + static const std::unordered_map order_directions + = {{1, " asc nulls first"}, {2, " asc nulls last"}, {3, " desc nulls first"}, {4, " desc nulls last"}}; + size_t n = 0; + DB::WriteBufferFromOwnString ostr; + for (const auto & sort_field : sort_fields) + { + auto it = order_directions.find(sort_field.direction()); + if (it == order_directions.end()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow sort direction: {}", sort_field.direction()); + if (!sort_field.expr().has_selection()) + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, "Sort field must be a column reference. but got {}", sort_field.DebugString()); + } + auto ref = sort_field.expr().selection().direct_reference().struct_field().field(); + const auto & col_name = header.getByPosition(ref).name; + if (n) + ostr << String(","); + // the col_name may contain '#' which can may ch fail to parse. + ostr << "`" << col_name << "`" << it->second; + n += 1; + } + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Order by clasue: {}", ostr.str()); + return ostr.str(); +} +} diff --git a/cpp-ch/local-engine/Common/SortUtils.h b/cpp-ch/local-engine/Common/SortUtils.h new file mode 100644 index 000000000000..c460fa758b6d --- /dev/null +++ b/cpp-ch/local-engine/Common/SortUtils.h @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once +#include +#include +#include +#include + +namespace local_engine +{ +// convert expressions into sort description +DB::SortDescription +parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & expressions); +DB::SortDescription parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields); + +std::string +buildSQLLikeSortDescription(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields); +} diff --git a/cpp-ch/local-engine/Operator/BranchStep.cpp b/cpp-ch/local-engine/Operator/BranchStep.cpp new file mode 100644 index 000000000000..5e379ae9d4dc --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchStep.cpp @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "BranchStep.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace local_engine +{ + +class BranchHookSource : public DB::IProcessor +{ +public: + using Status = DB::IProcessor::Status; + BranchHookSource(const DB::Block & header_) : DB::IProcessor({}, {header_}) { inner_inputs.emplace_back(header_, this); } + ~BranchHookSource() override = default; + + String getName() const override { return "BranchHookSource"; } + + Status prepare() override; + void work() override; + void enableInputs() { inputs.swap(inner_inputs); } + +private: + DB::InputPorts inner_inputs; + bool has_output = false; + DB::Chunk output_chunk; + bool has_input = false; + DB::Chunk input_chunk; +}; + +BranchHookSource::Status BranchHookSource::prepare() +{ + auto & output = outputs.front(); + auto & input = inputs.front(); + if (output.isFinished()) + { + input.close(); + return Status::Finished; + } + if (has_output) + { + if (output.canPush()) + { + output.push(std::move(output_chunk)); + has_output = false; + } + return Status::PortFull; + } + if (has_input) + return Status::Ready; + if (input.isFinished()) + { + output.finish(); + return Status::Finished; + } + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + input_chunk = input.pull(true); + has_input = true; + return Status::Ready; +} + +void BranchHookSource::work() +{ + if (has_input) + { + output_chunk = std::move(input_chunk); + has_output = true; + has_input = false; + } +} + + +static DB::ITransformingStep::Traits getTraits() +{ + return DB::ITransformingStep::Traits{ + { + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = false, + }}; +} + +class ResizeStep : public DB::ITransformingStep +{ +public: + explicit ResizeStep(const DB::Block & header_, size_t num_streams_) + : DB::ITransformingStep(header_, header_, getTraits()), num_streams(num_streams_) + { + } + ~ResizeStep() override = default; + + String getName() const override { return "UniteBranchesStep"; } + + void transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings &) override + { + LOG_ERROR(getLogger("ResizeStep"), "xxx num_streams: {}", num_streams); + pipeline.resize(num_streams); + } + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override + { + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); + } + +private: + size_t num_streams; + void updateOutputHeader() override {}; +}; + +DB::QueryPlanPtr BranchStepHelper::createSubPlan(const DB::Block & header, size_t num_streams) +{ + auto source = std::make_unique(DB::Pipe(std::make_shared(header))); + source->setStepDescription("Hook node connected to one branch output"); + auto plan = std::make_unique(); + plan->addStep(std::move(source)); + + if (num_streams > 1) + { + auto resize_step = std::make_unique(plan->getCurrentHeader(), num_streams); + plan->addStep(std::move(resize_step)); + } + return std::move(plan); +} + +StaticBranchStep::StaticBranchStep( + DB::ContextPtr context_, const DB::Block & header_, size_t branches_, size_t sample_rows_, BranchSelector selector_) + : DB::ITransformingStep(header_, header_, getTraits()) + , context(context_) + , header(header_) + , branches(branches_) + , max_sample_rows(sample_rows_) + , selector(selector_) +{ +} + +void StaticBranchStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & settings) +{ + auto build_transform = [&](DB::OutputPortRawPtrs child_outputs) -> DB::Processors + { + DB::Processors new_processors; + for (auto & output : child_outputs) + { + if (!output) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Output port is null"); + auto branch_transform = std::make_shared(header, max_sample_rows, branches, selector); + DB::connect(*output, branch_transform->getInputs().front()); + new_processors.push_back(branch_transform); + } + return new_processors; + }; + pipeline.resize(1); + pipeline.transform(build_transform); +} + +void StaticBranchStep::describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const +{ + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); +} + +void StaticBranchStep::updateOutputHeader() +{ +} + +UniteBranchesStep::UniteBranchesStep(const DB::Block & header_, std::vector && branch_plans_, size_t num_streams_) + : DB::ITransformingStep(header_, branch_plans_[0]->getCurrentHeader(), getTraits()), header(header_) +{ + branch_plans.swap(branch_plans_); + size_t branches = branch_plans.size(); + num_streams = num_streams_; +} + +void UniteBranchesStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings &) +{ + auto add_transform = [&](DB::OutputPortRawPtrs child_outputs) -> DB::Processors + { + DB::Processors new_processors; + size_t branch_index = 0; + if (child_outputs.size() != branch_plans.size()) + { + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Output port's size({}) is not equal to branches size({})", + child_outputs.size(), + branch_plans.size()); + } + for (auto output : child_outputs) + { + auto & branch_plan = branch_plans[branch_index]; + DB::QueryPlanOptimizationSettings optimization_settings; + DB::BuildQueryPipelineSettings build_settings; + DB::QueryPlanResourceHolder resource_holder; + + auto pipeline_builder = branch_plan->buildQueryPipeline(optimization_settings, build_settings); + auto pipe = DB::QueryPipelineBuilder::getPipe(std::move(*pipeline_builder), resource_holder); + DB::ProcessorPtr source_node = nullptr; + auto processors = DB::Pipe::detachProcessors(std::move(pipe)); + for (auto processor : processors) + { + if (auto * source = typeid_cast(processor.get())) + { + if (source->getInputs().empty()) + { + if (source_node) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There is multi source in branch plan"); + source->enableInputs(); + source_node = processor; + } + } + } + if (!source_node) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot find source node in branch plan"); + if (source_node->getInputs().empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Source node has no input"); + DB::connect(*output, source_node->getInputs().front()); + new_processors.insert(new_processors.end(), processors.begin(), processors.end()); + branch_index++; + } + return new_processors; + }; + pipeline.transform(add_transform); + pipeline.resize(1); + if (num_streams > 1) + pipeline.resize(num_streams); +} + +void UniteBranchesStep::describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const +{ + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); +} +} diff --git a/cpp-ch/local-engine/Operator/BranchStep.h b/cpp-ch/local-engine/Operator/BranchStep.h new file mode 100644 index 000000000000..ddbd4c6fbb70 --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchStep.h @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include "Processors/Port.h" +#include "Processors/QueryPlan/QueryPlan.h" + +namespace local_engine +{ + +class BranchStepHelper +{ +public: + // Create a new query plan that would be used to build sub branch query plan. + static DB::QueryPlanPtr createSubPlan(const DB::Block & header, size_t num_streams); +}; + +// Use to branch the query plan. +class StaticBranchStep : public DB::ITransformingStep +{ +public: + using BranchSelector = std::function &)>; + explicit StaticBranchStep( + DB::ContextPtr context_, const DB::Block & header, size_t branches, size_t sample_rows, BranchSelector selector); + ~StaticBranchStep() override = default; + + String getName() const override { return "StaticBranchStep"; } + + // This will resize the num_streams to 1. You may need to resize after this. + void transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & settings) override; + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override; + +protected: + void updateOutputHeader() override; + +private: + DB::ContextPtr context; + DB::Block header; + size_t max_sample_rows; + size_t branches; + BranchSelector selector; +}; + + +// It should be better to build execution branches on QueryPlan. +class UniteBranchesStep : public DB::ITransformingStep +{ +public: + explicit UniteBranchesStep(const DB::Block & header_, std::vector && branch_plans_, size_t num_streams_); + ~UniteBranchesStep() override = default; + + String getName() const override { return "UniteBranchesStep"; } + + void transformPipeline(DB::QueryPipelineBuilder & pipelines, const DB::BuildQueryPipelineSettings &) override; + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override; + +private: + DB::Block header; + std::vector branch_plans; + size_t num_streams; + + void updateOutputHeader() override { output_header = header; }; +}; + +} diff --git a/cpp-ch/local-engine/Operator/BranchTransform.cpp b/cpp-ch/local-engine/Operator/BranchTransform.cpp new file mode 100644 index 000000000000..f923f4ac4b41 --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchTransform.cpp @@ -0,0 +1,155 @@ + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "BranchTransform.h" +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + +namespace local_engine +{ +static DB::OutputPorts buildOutputPorts(const DB::Block & header, size_t branches) +{ + DB::OutputPorts output_ports; + for (size_t i = 0; i < branches; ++i) + output_ports.emplace_back(header); + return output_ports; +} +StaticBranchTransform::StaticBranchTransform(const DB::Block & header_, size_t sample_rows_, size_t branches_, BranchSelector selector_) + : DB::IProcessor({header_}, buildOutputPorts(header_, branches_)), max_sample_rows(sample_rows_), selector(selector_) +{ +} + +static bool existFinishedOutput(const DB::OutputPorts & output_ports) +{ + for (const auto & output_port : output_ports) + if (output_port.isFinished()) + return true; + return false; +} + +StaticBranchTransform::Status StaticBranchTransform::prepare() +{ + auto & input = inputs.front(); + if ((selected_output_port && selected_output_port->isFinished()) || (!selected_output_port && existFinishedOutput(outputs))) + { + input.close(); + return Status::Finished; + } + + if (has_output) + { + assert(selected_output_port != nullptr); + if (selected_output_port->canPush()) + { + selected_output_port->push(std::move(output_chunk)); + has_output = false; + } + return Status::PortFull; + } + + if (has_input || (selected_output_port && !sample_chunks.empty())) + { + // to clear the pending chunks + return Status::Ready; + } + + if (input.isFinished()) + { + if (!sample_chunks.empty()) + { + // to clear the pending chunks + return Status::Ready; + } + else + { + if (selected_output_port) + selected_output_port->finish(); + else + for (auto & output_port : outputs) + output_port.finish(); + return Status::Finished; + } + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + input_chunk = input.pull(true); + has_input = true; + return Status::Ready; +} + +void StaticBranchTransform::work() +{ + if (selected_output_port) + { + if (!sample_chunks.empty()) + { + assert(!has_input); + has_output = true; + output_chunk.swap(sample_chunks.front()); + sample_chunks.pop_front(); + } + else + { + assert(has_input); + has_input = false; + has_output = true; + output_chunk.swap(input_chunk); + } + } + else if (has_input) + { + sample_rows += input_chunk.getNumRows(); + sample_chunks.emplace_back(std::move(input_chunk)); + if (sample_rows >= max_sample_rows) + setupOutputPort(); + has_input = false; + } + else if (!sample_chunks.empty()) + { + if (!selected_output_port) + setupOutputPort(); + output_chunk.swap(sample_chunks.front()); + sample_chunks.pop_front(); + has_output = true; + } +} + +void StaticBranchTransform::setupOutputPort() +{ + size_t branch_index = selector(sample_chunks); + LOG_DEBUG(getLogger("StaticBranchTransform"), "Select output port: {}", branch_index); + if (branch_index >= outputs.size()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Branch index {} is out of range(0, {})", branch_index, outputs.size()); + auto it = outputs.begin(); + std::advance(it, branch_index); + selected_output_port = &(*it); + // close other output ports + for (auto oit = outputs.begin(); oit != outputs.end(); ++oit) + if (oit != it) + oit->finish(); +} +} // namespace local_engine diff --git a/cpp-ch/local-engine/Operator/BranchTransform.h b/cpp-ch/local-engine/Operator/BranchTransform.h new file mode 100644 index 000000000000..f5284b5ae968 --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchTransform.h @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once +#include +#include +#include + +namespace local_engine +{ + +// This is designed for adaptive execution. It has multiple outputs, each indicates for a execution branches. +// It accepts a branch selector, this selector will analysis the input data, and select one of the output port +// as the final only output port. Other output ports will be closed. +// The output port cannot be changed once it's selected. +class StaticBranchTransform : public DB::IProcessor +{ +public: + using BranchSelector = std::function &)>; + using Status = DB::IProcessor::Status; + StaticBranchTransform(const DB::Block & header_, size_t sample_rows_, size_t branches_, BranchSelector selector_); + + String getName() const override { return "StaticBranchTransform"; } + + Status prepare() override; + void work() override; + +private: + size_t max_sample_rows; + BranchSelector selector; + DB::OutputPort * selected_output_port = nullptr; + std::list sample_chunks; + size_t sample_rows = 0; + bool has_input = false; + bool has_output = false; + DB::Chunk input_chunk; + DB::Chunk output_chunk; + + void setupOutputPort(); +}; + +}; diff --git a/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp b/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp index f25e3f22ac65..d2264e24dc13 100644 --- a/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp +++ b/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp @@ -37,7 +37,6 @@ enum class WindowGroupLimitFunction DenseRank }; - template class WindowGroupLimitTransform : public DB::IProcessor { @@ -50,7 +49,6 @@ class WindowGroupLimitTransform : public DB::IProcessor , partition_columns(partition_columns_) , sort_columns(sort_columns_) , limit(limit_) - { } ~WindowGroupLimitTransform() override = default; @@ -95,9 +93,7 @@ class WindowGroupLimitTransform : public DB::IProcessor void work() override { if (!has_input) [[unlikely]] - { return; - } DB::Block block = header.cloneWithColumns(input_chunk.getColumns()); size_t partition_start_row = 0; size_t chunk_rows = input_chunk.getNumRows(); @@ -151,7 +147,6 @@ class WindowGroupLimitTransform : public DB::IProcessor DB::Columns partition_start_row_columns; DB::Columns peer_group_start_row_columns; - size_t advanceNextPartition(const DB::Chunk & chunk, size_t start_offset) { if (partition_start_row_columns.empty()) @@ -159,12 +154,8 @@ class WindowGroupLimitTransform : public DB::IProcessor size_t max_row = chunk.getNumRows(); for (size_t i = start_offset; i < max_row; ++i) - { if (!isRowEqual(partition_columns, partition_start_row_columns, 0, chunk.getColumns(), i)) - { return i; - } - } return max_row; } @@ -199,7 +190,6 @@ class WindowGroupLimitTransform : public DB::IProcessor if (current_row_rank_value > limit) return; - size_t chunk_rows = chunk.getNumRows(); auto has_peer_group_ended = [&](size_t offset, size_t partition_end_offset, size_t chunk_rows_) { return offset < partition_end_offset || end_offset < chunk_rows_; }; @@ -241,6 +231,7 @@ class WindowGroupLimitTransform : public DB::IProcessor size_t limit_remained = limit - current_row_rank_value + 1; rows = rows > limit_remained ? limit_remained : rows; insertResultValue(chunk, start_offset, rows); + current_row_rank_value += rows; } else @@ -249,8 +240,8 @@ class WindowGroupLimitTransform : public DB::IProcessor while (peer_group_start_offset < end_offset && current_row_rank_value <= limit) { auto next_peer_group_start_offset = advanceNextPeerGroup(chunk, peer_group_start_offset, end_offset); - - insertResultValue(chunk, peer_group_start_offset, next_peer_group_start_offset - peer_group_start_offset); + size_t group_rows = next_peer_group_start_offset - peer_group_start_offset; + insertResultValue(chunk, peer_group_start_offset, group_rows); try_end_peer_group(peer_group_start_offset, next_peer_group_start_offset, end_offset, chunk_rows); peer_group_start_offset = next_peer_group_start_offset; } @@ -261,12 +252,8 @@ class WindowGroupLimitTransform : public DB::IProcessor if (!rows) return; if (output_columns.empty()) - { for (const auto & col : chunk.getColumns()) - { output_columns.push_back(col->cloneEmpty()); - } - } size_t i = 0; for (const auto & col : chunk.getColumns()) { @@ -279,12 +266,8 @@ class WindowGroupLimitTransform : public DB::IProcessor if (peer_group_start_row_columns.empty()) peer_group_start_row_columns = extractOneRowColumns(chunk, start_offset); for (size_t i = start_offset; i < partition_end_offset; ++i) - { if (!isRowEqual(sort_columns, peer_group_start_row_columns, 0, chunk.getColumns(), i)) - { return i; - } - } return partition_end_offset; } }; diff --git a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp index f2cc84082d77..49bbe02c551c 100644 --- a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp +++ b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp @@ -164,6 +164,7 @@ WindowGroupOptimizationInfo WindowGroupOptimizationInfo::parse(const String & ad auto kkvs = convertToKVs(advance); auto & kvs = kkvs["WindowGroupLimitParameters"]; tryAssign(kvs, "window_function", info.window_function); + tryAssign(kvs, "is_aggregate_group_limit", info.is_aggregate_group_limit); return info; } } diff --git a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h index 3028639bb37c..795577328f99 100644 --- a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h +++ b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h @@ -49,6 +49,7 @@ struct AggregateOptimizationInfo struct WindowGroupOptimizationInfo { String window_function; + bool is_aggregate_group_limit = false; static WindowGroupOptimizationInfo parse(const String & advnace); }; } diff --git a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp new file mode 100644 index 000000000000..06f68e8ae218 --- /dev/null +++ b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp @@ -0,0 +1,512 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "GroupLimitRelParser.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace DB +{ +namespace Setting +{ +extern const SettingsMaxThreads max_threads; + +} +} + +namespace local_engine +{ +GroupLimitRelParser::GroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) +{ +} + +DB::QueryPlanPtr +GroupLimitRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) +{ + const auto win_rel_def = rel.windowgrouplimit(); + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def.advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + if (optimization_info.is_aggregate_group_limit) + { + AggregateGroupLimitRelParser aggregate_group_limit_parser(parser_context); + auto plan = aggregate_group_limit_parser.parse(std::move(current_plan_), rel, rel_stack_); + steps = aggregate_group_limit_parser.getSteps(); + return std::move(plan); + } + else + { + WindowGroupLimitRelParser window_parser(parser_context); + auto plan = window_parser.parse(std::move(current_plan_), rel, rel_stack_); + steps = window_parser.getSteps(); + return std::move(plan); + } +} + +static std::vector parsePartitionFields(const google::protobuf::RepeatedPtrField & expressions) +{ + std::vector fields; + for (const auto & expr : expressions) + if (expr.has_selection()) + fields.push_back(static_cast(expr.selection().direct_reference().struct_field().field())); + else if (expr.has_literal()) + continue; + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow expression: {}", expr.DebugString()); + return fields; +} + +std::vector parseSortFields(const google::protobuf::RepeatedPtrField & sort_fields) +{ + std::vector fields; + for (const auto sort_field : sort_fields) + if (sort_field.expr().has_literal()) + continue; + else if (sort_field.expr().has_selection()) + fields.push_back(static_cast(sort_field.expr().selection().direct_reference().struct_field().field())); + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknown expression: {}", sort_field.expr().DebugString()); + return fields; +} + + +WindowGroupLimitRelParser::WindowGroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) +{ +} + +DB::QueryPlanPtr +WindowGroupLimitRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) +{ + const auto win_rel_def = rel.windowgrouplimit(); + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def.advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + window_function_name = optimization_info.window_function; + + current_plan = std::move(current_plan_); + + auto partition_fields = parsePartitionFields(win_rel_def.partition_expressions()); + auto sort_fields = parseSortFields(win_rel_def.sorts()); + size_t limit = static_cast(win_rel_def.limit()); + + auto window_group_limit_step = std::make_unique( + current_plan->getCurrentHeader(), window_function_name, partition_fields, sort_fields, limit); + window_group_limit_step->setStepDescription("Window group limit"); + steps.emplace_back(window_group_limit_step.get()); + current_plan->addStep(std::move(window_group_limit_step)); + + return std::move(current_plan); +} + +AggregateGroupLimitRelParser::AggregateGroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) +{ +} + +// used to decide which branch +size_t selectBranchOnPartitionKeysCardinality( + const std::vector & partition_keys, double high_card_threshold, const std::list & chunks) +{ + size_t total_rows = 0; + std::unordered_set ids; + for (const auto & chunk : chunks) + { + total_rows += chunk.getNumRows(); + DB::WeakHash32 hash(chunk.getNumRows()); + const auto & cols = chunk.getColumns(); + for (auto i : partition_keys) + hash.update(cols[i]->getWeakHash32()); + const auto & data = hash.getData(); + for (size_t n = 0, sz = chunk.getNumRows(); n < sz; ++n) + ids.insert(data[n]); + } + LOG_DEBUG( + getLogger("AggregateGroupLimitRelParser"), + "Approximate distinct keys {}, total rows: {}, thrshold: {}", + ids.size(), + total_rows, + high_card_threshold); + return ids.size() * 1.0 / (total_rows + 1) <= high_card_threshold ? 0 : 1; +} + +DB::QueryPlanPtr AggregateGroupLimitRelParser::parse( + DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) +{ + // calculate window's topk by aggregation. + // 1. add a pre-projecttion. Make two tuple arguments for the aggregation function. One is the required columns for the output, the other + // is the required columns for sorting. + // 2. Collect the sorting directions for each sorting field, Let them as the aggregation function's parameters. + // 3. Add a aggregation step. + // 4. Add a post-projecttion. Explode the aggregation function's result, since the result is an array. + + current_plan = std::move(current_plan_); + input_header = current_plan->getCurrentHeader(); + win_rel_def = &rel.windowgrouplimit(); + + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def->advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + limit = static_cast(win_rel_def->limit()); + aggregate_function_name = getAggregateFunctionName(optimization_info.window_function); + + if (limit < 1) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid limit: {}", limit); + + auto win_config = WindowConfig::loadFromContext(getContext()); + auto high_card_threshold = win_config.aggregate_topk_high_cardinality_threshold; + + // Aggregation doesn't perform well on high cardinality keys. We make two execution pathes here. + // - if the partition keys are low cardinality, run it by aggregation + // - if the partition keys are high cardinality, run it by window. + auto partition_fields = parsePartitionFields(win_rel_def->partition_expressions()); + auto branch_in_header = current_plan->getCurrentHeader(); + auto branch_step = std::make_unique( + getContext(), + branch_in_header, + 2, + win_config.aggregate_topk_sample_rows, + [partition_fields, high_card_threshold](const std::list & chunks) -> size_t + { return selectBranchOnPartitionKeysCardinality(partition_fields, high_card_threshold, chunks); }); + branch_step->setStepDescription("Window TopK"); + steps.push_back(branch_step.get()); + current_plan->addStep(std::move(branch_step)); + + // If all partition keys are low cardinality keys, use aggregattion to get topk of each partition + auto aggregation_plan = BranchStepHelper::createSubPlan(branch_in_header, 1); + prePrejectionForAggregateArguments(*aggregation_plan); + addGroupLmitAggregationStep(*aggregation_plan); + postProjectionForExplodingArrays(*aggregation_plan); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Aggregate topk plan:\n{}", PlanUtil::explainPlan(*aggregation_plan)); + + auto window_plan = BranchStepHelper::createSubPlan(branch_in_header, 1); + addSortStep(*window_plan); + addWindowLimitStep(*window_plan); + auto convert_actions_dag = DB::ActionsDAG::makeConvertingActions( + window_plan->getCurrentHeader().getColumnsWithTypeAndName(), + aggregation_plan->getCurrentHeader().getColumnsWithTypeAndName(), + DB::ActionsDAG::MatchColumnsMode::Position); + auto convert_step = std::make_unique(window_plan->getCurrentHeader(), std::move(convert_actions_dag)); + convert_step->setStepDescription("Rename rank column name"); + window_plan->addStep(std::move(convert_step)); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Window topk plan:\n{}", PlanUtil::explainPlan(*window_plan)); + + std::vector branch_plans; + branch_plans.emplace_back(std::move(aggregation_plan)); + branch_plans.emplace_back(std::move(window_plan)); + auto unite_branches_step = std::make_unique(branch_in_header, std::move(branch_plans), 1); + unite_branches_step->setStepDescription("Unite TopK branches"); + steps.push_back(unite_branches_step.get()); + + current_plan->addStep(std::move(unite_branches_step)); + return std::move(current_plan); +} + +String AggregateGroupLimitRelParser::getAggregateFunctionName(const String & window_function_name) +{ + if (window_function_name == "row_number") + return "rowNumGroupArraySorted"; + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unsupported window function: {}", window_function_name); +} + +// Build one tuple column as the aggregate function's arguments +void AggregateGroupLimitRelParser::prePrejectionForAggregateArguments(DB::QueryPlan & plan) +{ + auto projection_actions = std::make_shared(input_header.getColumnsWithTypeAndName()); + + + auto partition_fields = parsePartitionFields(win_rel_def->partition_expressions()); + std::set unique_partition_fields(partition_fields.begin(), partition_fields.end()); + DB::NameSet required_column_names; + auto build_tuple = [&](const DB::DataTypes & data_types, + const Strings & names, + const DB::ActionsDAG::NodeRawConstPtrs & elements, + const String & name_prefix, + String & result_name) + { + result_name = expression_parser->getUniqueName(name_prefix); + auto tuple = expression_parser->toFunctionNode(*projection_actions, "tuple", elements, result_name); + auto tuple_type = std::make_shared(data_types, names); + DB::ActionsDAG::NodeRawConstPtrs cast_args; + cast_args.push_back(tuple); + cast_args.push_back( + expression_parser->addConstColumn(*projection_actions, std::make_shared(), tuple_type->getName())); + tuple = expression_parser->toFunctionNode(*projection_actions, "CAST", cast_args, result_name); + projection_actions->addOrReplaceInOutputs(*tuple); + required_column_names.insert(tuple->result_name); + }; + + DB::DataTypes aggregate_data_tuple_types; + Strings aggregate_data_tuple_names; + DB::ActionsDAG::NodeRawConstPtrs aggregate_data_tuple_nodes; + for (size_t i = 0; i < input_header.columns(); ++i) + { + const auto & col = input_header.getByPosition(i); + if (unique_partition_fields.count(i)) + { + required_column_names.insert(col.name); + aggregate_grouping_keys.push_back(col.name); + } + else + { + aggregate_data_tuple_types.push_back(col.type); + aggregate_data_tuple_names.push_back(col.name); + aggregate_data_tuple_nodes.push_back(projection_actions->getInputs()[i]); + } + } + build_tuple( + aggregate_data_tuple_types, + aggregate_data_tuple_names, + aggregate_data_tuple_nodes, + "aggregate_data_tuple", + aggregate_tuple_column_name); + + projection_actions->removeUnusedActions(required_column_names); + LOG_DEBUG( + getLogger("AggregateGroupLimitRelParser"), + "Projection for building tuples for aggregate function:\n{}", + projection_actions->dumpDAG()); + + auto expression_step = std::make_unique(input_header, std::move(*projection_actions)); + expression_step->setStepDescription("Pre-projection for aggregate group limit arguments"); + plan.addStep(std::move(expression_step)); +} + +DB::AggregateDescription AggregateGroupLimitRelParser::buildAggregateDescription(DB::QueryPlan & plan) +{ + DB::AggregateDescription agg_desc; + agg_desc.column_name = aggregate_tuple_column_name; + agg_desc.argument_names = {aggregate_tuple_column_name}; + DB::Array parameters; + parameters.push_back(static_cast(limit)); + auto sort_directions = buildSQLLikeSortDescription(input_header, win_rel_def->sorts()); + parameters.push_back(sort_directions); + + auto header = plan.getCurrentHeader(); + DB::DataTypes arg_types; + arg_types.push_back(header.getByName(aggregate_tuple_column_name).type); + + DB::AggregateFunctionProperties properties; + agg_desc.function = getAggregateFunction(aggregate_function_name, arg_types, properties, parameters); + return agg_desc; +} + +void AggregateGroupLimitRelParser::addGroupLmitAggregationStep(DB::QueryPlan & plan) +{ + const auto & settings = getContext()->getSettingsRef(); + DB::AggregateDescriptions agg_descs = {buildAggregateDescription(plan)}; + auto params = AggregatorParamsHelper::buildParams( + getContext(), aggregate_grouping_keys, agg_descs, AggregatorParamsHelper::Mode::INIT_TO_COMPLETED); + auto agg_step = std::make_unique(getContext(), plan.getCurrentHeader(), params, true); + plan.addStep(std::move(agg_step)); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Plan after add group limit:\n{}", PlanUtil::explainPlan(plan)); +} + +void AggregateGroupLimitRelParser::postProjectionForExplodingArrays(DB::QueryPlan & plan) +{ + auto header = plan.getCurrentHeader(); + + /// flatten the array column. + auto agg_result_index = header.columns() - 1; + auto array_join_actions_dag = ArrayJoinHelper::applyArrayJoinOnOneColumn(header, agg_result_index); + auto new_steps = ArrayJoinHelper::addArrayJoinStep(getContext(), plan, array_join_actions_dag, false); + + auto array_join_output_header = plan.getCurrentHeader(); + DB::ActionsDAG flatten_actions_dag(array_join_output_header.getColumnsWithTypeAndName()); + DB::Names flatten_output_column_names; + for (size_t i = 0; i < array_join_output_header.columns() - 1; ++i) + { + const auto & col = array_join_output_header.getByPosition(i); + flatten_output_column_names.push_back(col.name); + } + auto last_column = array_join_output_header.getByPosition(array_join_output_header.columns() - 1); + const auto * tuple_column = typeid_cast(last_column.column.get()); + const auto * tuple_datatype = typeid_cast(last_column.type.get()); + const auto & field_names = tuple_datatype->getElementNames(); + DB::DataTypePtr tuple_index_type = std::make_shared(); + const auto * tuple_node = flatten_actions_dag.getInputs().back(); + for (size_t i = 0; i < field_names.size(); ++i) + { + DB::ActionsDAG::NodeRawConstPtrs tuple_index_args; + tuple_index_args.push_back(tuple_node); + tuple_index_args.push_back(expression_parser->addConstColumn(flatten_actions_dag, tuple_index_type, i + 1)); + const auto * field_node = expression_parser->toFunctionNode(flatten_actions_dag, "tupleElement", tuple_index_args, field_names[i]); + flatten_actions_dag.addOrReplaceInOutputs(*field_node); + flatten_output_column_names.push_back(field_node->result_name); + } + flatten_actions_dag.removeUnusedActions(flatten_output_column_names); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Actions dag for untupling aggregate result:\n{}", flatten_actions_dag.dumpDAG()); + auto flatten_expression_step = std::make_unique(plan.getCurrentHeader(), std::move(flatten_actions_dag)); + flatten_expression_step->setStepDescription("Untuple the aggregation result"); + plan.addStep(std::move(flatten_expression_step)); + + auto flatten_tuple_output_header = plan.getCurrentHeader(); + auto window_result_column = flatten_tuple_output_header.getByPosition(flatten_tuple_output_header.columns() - 1); + /// The result column is put at the end of the header. + auto output_header = input_header; + output_header.insert(window_result_column); + auto adjust_pos_actions_dag = DB::ActionsDAG::makeConvertingActions( + flatten_tuple_output_header.getColumnsWithTypeAndName(), + output_header.getColumnsWithTypeAndName(), + DB::ActionsDAG::MatchColumnsMode::Name); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Actions dag for replacing columns:\n{}", adjust_pos_actions_dag.dumpDAG()); + auto adjust_pos_expression_step = std::make_unique(flatten_tuple_output_header, std::move(adjust_pos_actions_dag)); + adjust_pos_expression_step->setStepDescription("Adjust position of the output columns"); + plan.addStep(std::move(adjust_pos_expression_step)); +} + +void AggregateGroupLimitRelParser::addSortStep(DB::QueryPlan & plan) +{ + auto header = plan.getCurrentHeader(); + DB::SortDescription full_sort_descr; + auto partition_fields = parsePartitionFields(win_rel_def->partition_expressions()); + for (auto field : partition_fields) + { + const auto & col = header.getByPosition(field); + full_sort_descr.emplace_back(col.name, 1, -1); + } + auto sort_desrc = SortRelParser::parseSortDescription(win_rel_def->sorts(), header); + full_sort_descr.insert(full_sort_descr.end(), sort_desrc.begin(), sort_desrc.end()); + + DB::SortingStep::Settings settings(*getContext()); + auto config = MemoryConfig::loadFromContext(getContext()); + double spill_mem_ratio = config.spill_mem_ratio; + settings.worth_external_sort = [spill_mem_ratio]() -> bool { return currentThreadGroupMemoryUsageRatio() > spill_mem_ratio; }; + auto sorting_step = std::make_unique(plan.getCurrentHeader(), full_sort_descr, 0, settings); + sorting_step->setStepDescription("Sorting step"); + plan.addStep(std::move(sorting_step)); +} + +static DB::WindowFrame buildWindowFrame(const std::string & ch_function_name) +{ + DB::WindowFrame frame; + // default window frame is [unbounded preceding, current row] + if (ch_function_name == "row_number") + { + frame.type = DB::WindowFrame::FrameType::ROWS; + frame.begin_type = DB::WindowFrame::BoundaryType::Offset; + frame.begin_offset = 1; + } + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow window function: {}", ch_function_name); + return frame; +} + +static DB::WindowFunctionDescription buildWindowFunctionDescription(const std::string & ch_function_name) +{ + DB::WindowFunctionDescription description; + if (ch_function_name == "row_number") + { + description.column_name = ch_function_name; + description.function_node = nullptr; + DB::AggregateFunctionProperties agg_props; + auto agg_func = RelParser::getAggregateFunction(ch_function_name, {}, agg_props, {}); + description.aggregate_function = agg_func; + } + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow window function: {}", ch_function_name); + return description; +} + + +// TODO: WindowGroupLimitStep has bad performance, need to improve it. So we still use window + filter here. +void AggregateGroupLimitRelParser::addWindowLimitStep(DB::QueryPlan & plan) +{ + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def->advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + auto window_function_name = optimization_info.window_function; + + auto in_header = plan.getCurrentHeader(); + DB::WindowDescription win_descr; + win_descr.frame = buildWindowFrame(window_function_name); + win_descr.partition_by = parseSortFields(in_header, win_rel_def->partition_expressions()); + win_descr.order_by = parseSortFields(in_header, win_rel_def->sorts()); + win_descr.full_sort_description = win_descr.partition_by; + win_descr.full_sort_description.insert(win_descr.full_sort_description.end(), win_descr.order_by.begin(), win_descr.order_by.end()); + DB::WriteBufferFromOwnString ss; + ss << "partition by " << DB::dumpSortDescription(win_descr.partition_by); + ss << " order by " << DB::dumpSortDescription(win_descr.order_by); + ss << " " << win_descr.frame.toString(); + win_descr.window_name = ss.str(); + + auto win_func_description = buildWindowFunctionDescription(window_function_name); + win_descr.window_functions.push_back(win_func_description); + + auto win_step = std::make_unique(in_header, win_descr, win_descr.window_functions, false); + win_step->setStepDescription("Window (" + win_descr.window_name + ")"); + plan.addStep(std::move(win_step)); + + auto win_result_header = plan.getCurrentHeader(); + DB::ActionsDAG limit_actions_dag(win_result_header.getColumnsWithTypeAndName()); + const auto * rank_value_node = limit_actions_dag.getInputs().back(); + const auto * limit_value_node = expression_parser->addConstColumn(limit_actions_dag, std::make_shared(), limit); + const auto * cmp_node = expression_parser->toFunctionNode(limit_actions_dag, "lessOrEquals", {rank_value_node, limit_value_node}); + auto cmp_column_name = cmp_node->result_name; + limit_actions_dag.addOrReplaceInOutputs(*cmp_node); + auto filter_step = std::make_unique(win_result_header, std::move(limit_actions_dag), cmp_column_name, true); + plan.addStep(std::move(filter_step)); +} + +void registerWindowGroupLimitRelParser(RelParserFactory & factory) +{ + auto builder = [](ParserContextPtr parser_context) { return std::make_shared(parser_context); }; + factory.registerBuilder(substrait::Rel::RelTypeCase::kWindowGroupLimit, builder); +} +} diff --git a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h new file mode 100644 index 000000000000..b9f3aa6631c3 --- /dev/null +++ b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once +#include +#include +#include +#include +#include +#include "Analyzer/IQueryTreeNode.h" +#include "substrait/algebra.pb.h" + +namespace local_engine +{ + +class GroupLimitRelParser : public RelParser +{ +public: + explicit GroupLimitRelParser(ParserContextPtr parser_context_); + ~GroupLimitRelParser() override = default; + DB::QueryPlanPtr + parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; + std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } +}; + +/// Similar to WindowRelParser. Some differences +/// 1. cannot support aggregate functions. only support window functions: row_number, rank, dense_rank +/// 2. row_number, rank and dense_rank are mapped to new variants +/// 3. the output columns don't contain window function results +class WindowGroupLimitRelParser : public RelParser +{ +public: + explicit WindowGroupLimitRelParser(ParserContextPtr parser_context_); + ~WindowGroupLimitRelParser() override = default; + DB::QueryPlanPtr + parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; + std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } + +private: + DB::QueryPlanPtr current_plan; + String window_function_name; +}; + +class AggregateGroupLimitRelParser : public RelParser +{ +public: + explicit AggregateGroupLimitRelParser(ParserContextPtr parser_context_); + ~AggregateGroupLimitRelParser() override = default; + DB::QueryPlanPtr + parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; + std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } + +private: + DB::QueryPlanPtr current_plan; + const substrait::WindowGroupLimitRel * win_rel_def; + String aggregate_function_name; + size_t limit = 0; + DB::Block input_header; + // DB::Block output_header; + DB::Names aggregate_grouping_keys; + String aggregate_tuple_column_name; + + String getAggregateFunctionName(const String & window_function_name); + + void prePrejectionForAggregateArguments(DB::QueryPlan & plan); + + void addGroupLmitAggregationStep(DB::QueryPlan & plan); + String parseSortDirections(const google::protobuf::RepeatedPtrField & sort_fields); + DB::AggregateDescription buildAggregateDescription(DB::QueryPlan & plan); + void postProjectionForExplodingArrays(DB::QueryPlan & plan); + + void addSortStep(DB::QueryPlan & plan); + void addWindowLimitStep(DB::QueryPlan & plan); +}; +} diff --git a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp index 20af6f83fc59..21e2dba44fa6 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB { @@ -85,40 +86,6 @@ ProjectRelParser::parseProject(DB::QueryPlanPtr query_plan, const substrait::Rel } } -const DB::ActionsDAG::Node * ProjectRelParser::findArrayJoinNode(const ActionsDAG & actions_dag) -{ - const ActionsDAG::Node * array_join_node = nullptr; - const auto & nodes = actions_dag.getNodes(); - for (const auto & node : nodes) - { - if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) - { - if (array_join_node) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect single ARRAY JOIN node in generate rel"); - array_join_node = &node; - } - } - return array_join_node; -} - -ProjectRelParser::SplittedActionsDAGs ProjectRelParser::splitActionsDAGInGenerate(const ActionsDAG & actions_dag) -{ - SplittedActionsDAGs res; - - auto array_join_node = findArrayJoinNode(actions_dag); - std::unordered_set first_split_nodes(array_join_node->children.begin(), array_join_node->children.end()); - auto first_split_result = actions_dag.split(first_split_nodes); - res.before_array_join = std::move(first_split_result.first); - - array_join_node = findArrayJoinNode(first_split_result.second); - std::unordered_set second_split_nodes = {array_join_node}; - auto second_split_result = first_split_result.second.split(second_split_nodes); - res.array_join = std::move(second_split_result.first); - second_split_result.second.removeUnusedActions(); - res.after_array_join = std::move(second_split_result.second); - return res; -} - bool ProjectRelParser::isReplicateRows(substrait::GenerateRel rel) { auto signature = expression_parser->getFunctionNameInSignature(rel.generator().scalar_function()); @@ -164,7 +131,7 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re auto header = query_plan->getCurrentHeader(); auto actions_dag = expressionsToActionsDAG(expressions, header); - if (!findArrayJoinNode(actions_dag)) + if (!ArrayJoinHelper::findArrayJoinNode(actions_dag)) { /// If generator in generate rel is not explode/posexplode, e.g. json_tuple auto expression_step = std::make_unique(query_plan->getCurrentHeader(), std::move(actions_dag)); @@ -174,59 +141,8 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re } else { - /// If generator in generate rel is explode/posexplode, transform arrayJoin function to ARRAY JOIN STEP to apply max_block_size - /// which avoids OOM when several lateral view explode/posexplode is used in spark sqls - LOG_DEBUG(logger, "original actions_dag:{}", actions_dag.dumpDAG()); - auto splitted_actions_dags = splitActionsDAGInGenerate(actions_dag); - LOG_DEBUG(logger, "actions_dag before arrayJoin:{}", splitted_actions_dags.before_array_join.dumpDAG()); - LOG_DEBUG(logger, "actions_dag during arrayJoin:{}", splitted_actions_dags.array_join.dumpDAG()); - LOG_DEBUG(logger, "actions_dag after arrayJoin:{}", splitted_actions_dags.after_array_join.dumpDAG()); - - auto ignore_actions_dag = [](const ActionsDAG & actions_dag_) -> bool - { - /* - We should ignore actions_dag like: - 0 : INPUT () (no column) String a - 1 : INPUT () (no column) String b - Output nodes: 0, 1 - */ - return actions_dag_.getOutputs().size() == actions_dag_.getNodes().size() - && actions_dag_.getInputs().size() == actions_dag_.getNodes().size(); - }; - - /// Pre-projection before array join - if (!ignore_actions_dag(splitted_actions_dags.before_array_join)) - { - auto step_before_array_join - = std::make_unique(query_plan->getCurrentHeader(), std::move(splitted_actions_dags.before_array_join)); - step_before_array_join->setStepDescription("Pre-projection In Generate"); - steps.emplace_back(step_before_array_join.get()); - query_plan->addStep(std::move(step_before_array_join)); - // LOG_DEBUG(logger, "plan1:{}", PlanUtil::explainPlan(*query_plan)); - } - - /// ARRAY JOIN - Names array_joined_columns{findArrayJoinNode(splitted_actions_dags.array_join)->result_name}; - ArrayJoin array_join; - array_join.columns = std::move(array_joined_columns); - array_join.is_left = generate_rel.outer(); - auto array_join_step = std::make_unique( - query_plan->getCurrentHeader(), std::move(array_join), false, getContext()->getSettingsRef()[Setting::max_block_size]); - array_join_step->setStepDescription("ARRAY JOIN In Generate"); - steps.emplace_back(array_join_step.get()); - query_plan->addStep(std::move(array_join_step)); - // LOG_DEBUG(logger, "plan2:{}", PlanUtil::explainPlan(*query_plan)); - - /// Post-projection after array join(Optional) - if (!ignore_actions_dag(splitted_actions_dags.after_array_join)) - { - auto step_after_array_join - = std::make_unique(query_plan->getCurrentHeader(), std::move(splitted_actions_dags.after_array_join)); - step_after_array_join->setStepDescription("Post-projection In Generate"); - steps.emplace_back(step_after_array_join.get()); - query_plan->addStep(std::move(step_after_array_join)); - // LOG_DEBUG(logger, "plan3:{}", PlanUtil::explainPlan(*query_plan)); - } + auto new_steps = ArrayJoinHelper::addArrayJoinStep(getContext(), *query_plan, actions_dag, generate_rel.outer()); + steps.insert(steps.end(), new_steps.begin(), new_steps.end()); } return query_plan; diff --git a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h index ce7a6faa70c0..cb6cf84f729d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h @@ -26,13 +26,6 @@ namespace local_engine class ProjectRelParser : public RelParser { public: - struct SplittedActionsDAGs - { - ActionsDAG before_array_join; /// Optional - ActionsDAG array_join; - ActionsDAG after_array_join; /// Optional - }; - explicit ProjectRelParser(ParserContextPtr parser_context_); ~ProjectRelParser() override = default; @@ -45,11 +38,6 @@ class ProjectRelParser : public RelParser DB::QueryPlanPtr parseProject(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); DB::QueryPlanPtr parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); - static const DB::ActionsDAG::Node * findArrayJoinNode(const ActionsDAG & actions_dag); - - /// Split actions_dag of generate rel into 3 parts: before array join + during array join + after array join - static SplittedActionsDAGs splitActionsDAGInGenerate(const ActionsDAG & actions_dag); - bool isReplicateRows(substrait::GenerateRel rel); DB::QueryPlanPtr parseReplicateRows(QueryPlanPtr query_plan, substrait::GenerateRel generate_rel); diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.cpp deleted file mode 100644 index e82d68c1d115..000000000000 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.cpp +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include "WindowGroupLimitRelParser.h" -#include -#include -#include -#include -#include - -namespace DB::ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -namespace local_engine -{ -WindowGroupLimitRelParser::WindowGroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) -{ -} - -DB::QueryPlanPtr -WindowGroupLimitRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) -{ - const auto win_rel_def = rel.windowgrouplimit(); - google::protobuf::StringValue optimize_info_str; - optimize_info_str.ParseFromString(win_rel_def.advanced_extension().optimization().value()); - auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); - window_function_name = optimization_info.window_function; - - current_plan = std::move(current_plan_); - - auto partition_fields = parsePartitoinFields(win_rel_def.partition_expressions()); - auto sort_fields = parseSortFields(win_rel_def.sorts()); - size_t limit = static_cast(win_rel_def.limit()); - - auto window_group_limit_step = std::make_unique( - current_plan->getCurrentHeader(), window_function_name, partition_fields, sort_fields, limit); - window_group_limit_step->setStepDescription("Window group limit"); - steps.emplace_back(window_group_limit_step.get()); - current_plan->addStep(std::move(window_group_limit_step)); - - return std::move(current_plan); -} - -std::vector -WindowGroupLimitRelParser::parsePartitoinFields(const google::protobuf::RepeatedPtrField & expressions) -{ - std::vector fields; - for (const auto & expr : expressions) - if (expr.has_selection()) - fields.push_back(static_cast(expr.selection().direct_reference().struct_field().field())); - else if (expr.has_literal()) - continue; - else - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow expression: {}", expr.DebugString()); - return fields; -} - -std::vector WindowGroupLimitRelParser::parseSortFields(const google::protobuf::RepeatedPtrField & sort_fields) -{ - std::vector fields; - for (const auto sort_field : sort_fields) - if (sort_field.expr().has_literal()) - continue; - else if (sort_field.expr().has_selection()) - fields.push_back(static_cast(sort_field.expr().selection().direct_reference().struct_field().field())); - else - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknown expression: {}", sort_field.expr().DebugString()); - return fields; -} - -void registerWindowGroupLimitRelParser(RelParserFactory & factory) -{ - auto builder = [](ParserContextPtr parser_context) { return std::make_shared(parser_context); }; - factory.registerBuilder(substrait::Rel::RelTypeCase::kWindowGroupLimit, builder); -} -} diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.h deleted file mode 100644 index 573b7fad7cf9..000000000000 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.h +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#pragma once -#include -#include -#include -#include -#include - -namespace local_engine -{ -/// Similar to WindowRelParser. Some differences -/// 1. cannot support aggregate functions. only support window functions: row_number, rank, dense_rank -/// 2. row_number, rank and dense_rank are mapped to new variants -/// 3. the output columns don't contain window function results -class WindowGroupLimitRelParser : public RelParser -{ -public: - explicit WindowGroupLimitRelParser(ParserContextPtr parser_context_); - ~WindowGroupLimitRelParser() override = default; - DB::QueryPlanPtr - parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; - std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } - -private: - DB::QueryPlanPtr current_plan; - String window_function_name; - - std::vector parsePartitoinFields(const google::protobuf::RepeatedPtrField & expressions); - std::vector parseSortFields(const google::protobuf::RepeatedPtrField & sort_fields); -}; -} From d9cdf616f7eb7836fd65ed5e48784c3a05c4afc6 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 28 Nov 2024 12:01:23 +0800 Subject: [PATCH 161/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_27) (#8057) Upstream Velox's New Commits: e80bf12e3 by Xiaoxuan Meng, fix: Fix the driver block hanging issue in serialized execution mode (11647) ea3656e53 by Xuedong Luan, fix: Remove unnecessary method in HdfsReadFile.h (11662) 4e3311a7e by Krishna Pai, fix(CI): Disabling Conbench uploads due to security sev (11667) 21576daae by Jialiang Tan, Back out "Combine low selectivity vectors generated by the hash join filter" (11659) 1e2d30ca9 by rui-mo, fix: Add testInt128Range for filters (11645) 8f6d89762 by Yenda Li, feat: support options to trace directories (11631) c395c553b by Yenda Li, feat: Update throttler to support network signal (11611) f0136845e by Yenda Li, feat: Support API for setting a spill directory callback in task (11572) 045290b32 by Satadru Pan, refactor: Add new callback to log plan fragment from TaskListener (11459) 300098113 by Zhaokuo, fix(regexp_replace): Move regex preprocessing to functions/lib for Spark reuse and fix backslash handling (10981) --- .github/workflows/velox_backend_cache.yml | 1 + ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/velox_backend_cache.yml b/.github/workflows/velox_backend_cache.yml index f590c6d19c4a..3a6a8fadca0e 100644 --- a/.github/workflows/velox_backend_cache.yml +++ b/.github/workflows/velox_backend_cache.yml @@ -43,6 +43,7 @@ jobs: - name: Build Gluten native libraries run: | docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + export CCACHE_DIR=/work/.ccache mkdir -p /work/.ccache bash dev/ci-velox-buildstatic-centos-7.sh " diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index bcab2e31dbef..01740f138fa4 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_26 +VELOX_BRANCH=2024_11_27 VELOX_HOME="" OS=`uname -s` From 6bedcfa28af4a16c82e79d06d5d08c09a109fcb0 Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 28 Nov 2024 16:44:26 +0800 Subject: [PATCH 162/211] [GLUTEN-8073][CH] Replace some deprecated methods about sort (#8079) What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Fixes: #8073 How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- .../GroupLimitFunctions.cpp | 4 +- cpp-ch/local-engine/Operator/BranchStep.h | 4 +- .../Parser/RelParsers/GroupLimitRelParser.cpp | 15 ++---- .../Parser/RelParsers/GroupLimitRelParser.h | 5 +- .../RelParsers/SortParsingUtils.cpp} | 4 +- .../RelParsers/SortParsingUtils.h} | 0 .../Parser/RelParsers/SortRelParser.cpp | 43 +---------------- .../Parser/RelParsers/SortRelParser.h | 2 - .../Parser/RelParsers/WindowRelParser.cpp | 47 ++----------------- .../Parser/RelParsers/WindowRelParser.h | 1 - cpp-ch/local-engine/tests/gtest_ch_join.cpp | 4 +- 11 files changed, 18 insertions(+), 111 deletions(-) rename cpp-ch/local-engine/{Common/SortUtils.cpp => Parser/RelParsers/SortParsingUtils.cpp} (99%) rename cpp-ch/local-engine/{Common/SortUtils.h => Parser/RelParsers/SortParsingUtils.h} (100%) diff --git a/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp index 137ae8a54489..a9be46908bb9 100644 --- a/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp +++ b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include #include #include #include @@ -41,7 +40,6 @@ #include #include -#include "base/defines.h" namespace DB::ErrorCodes { @@ -182,7 +180,7 @@ class RowNumGroupArraySorted final : public DB::IAggregateFunctionDataHelper( - {data_type}, parameters_, getRowNumReultDataType(data_type)) + {data_type}, parameters_, getRowNumReultDataType(data_type)) { if (parameters_.size() != 2) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "{} needs two parameters: limit and order clause", getName()); diff --git a/cpp-ch/local-engine/Operator/BranchStep.h b/cpp-ch/local-engine/Operator/BranchStep.h index ddbd4c6fbb70..fd2203aae80a 100644 --- a/cpp-ch/local-engine/Operator/BranchStep.h +++ b/cpp-ch/local-engine/Operator/BranchStep.h @@ -21,10 +21,10 @@ #include #include #include +#include #include #include -#include "Processors/Port.h" -#include "Processors/QueryPlan/QueryPlan.h" +#include namespace local_engine { diff --git a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp index 06f68e8ae218..e39878cae8f7 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp @@ -17,7 +17,6 @@ #include "GroupLimitRelParser.h" #include -#include #include #include #include @@ -34,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -54,7 +54,6 @@ #include #include #include -#include #include namespace DB::ErrorCodes @@ -414,15 +413,9 @@ void AggregateGroupLimitRelParser::postProjectionForExplodingArrays(DB::QueryPla void AggregateGroupLimitRelParser::addSortStep(DB::QueryPlan & plan) { auto header = plan.getCurrentHeader(); - DB::SortDescription full_sort_descr; - auto partition_fields = parsePartitionFields(win_rel_def->partition_expressions()); - for (auto field : partition_fields) - { - const auto & col = header.getByPosition(field); - full_sort_descr.emplace_back(col.name, 1, -1); - } - auto sort_desrc = SortRelParser::parseSortDescription(win_rel_def->sorts(), header); - full_sort_descr.insert(full_sort_descr.end(), sort_desrc.begin(), sort_desrc.end()); + auto full_sort_descr = parseSortFields(header, win_rel_def->partition_expressions()); + auto sort_descr = parseSortFields(header, win_rel_def->sorts()); + full_sort_descr.insert(full_sort_descr.end(), sort_descr.begin(), sort_descr.end()); DB::SortingStep::Settings settings(*getContext()); auto config = MemoryConfig::loadFromContext(getContext()); diff --git a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h index b9f3aa6631c3..6f570ba17934 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h @@ -18,10 +18,7 @@ #include #include #include -#include -#include -#include "Analyzer/IQueryTreeNode.h" -#include "substrait/algebra.pb.h" +#include namespace local_engine { diff --git a/cpp-ch/local-engine/Common/SortUtils.cpp b/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.cpp similarity index 99% rename from cpp-ch/local-engine/Common/SortUtils.cpp rename to cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.cpp index 1b18cc4bfaf5..70cacf0633a2 100644 --- a/cpp-ch/local-engine/Common/SortUtils.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.cpp @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "SortUtils.h" +#include "SortParsingUtils.h" #include #include #include @@ -37,7 +37,7 @@ DB::SortDescription parseSortFields(const DB::Block & header, const google::prot { auto pos = expr.selection().direct_reference().struct_field().field(); const auto & col_name = header.getByPosition(pos).name; - description.push_back(DB::SortColumnDescription(col_name, 1, 1)); + description.push_back(DB::SortColumnDescription(col_name, 1, -1)); } else if (expr.has_literal()) continue; diff --git a/cpp-ch/local-engine/Common/SortUtils.h b/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.h similarity index 100% rename from cpp-ch/local-engine/Common/SortUtils.h rename to cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.h diff --git a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp index 1ed4f2565dea..a8023a98185d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp @@ -16,6 +16,7 @@ */ #include "SortRelParser.h" +#include #include #include #include @@ -41,7 +42,7 @@ SortRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, st { size_t limit = parseLimit(rel_stack_); const auto & sort_rel = rel.sort(); - auto sort_descr = parseSortDescription(sort_rel.sorts(), query_plan->getCurrentHeader()); + auto sort_descr = parseSortFields(query_plan->getCurrentHeader(), sort_rel.sorts()); SortingStep::Settings settings(*getContext()); auto config = MemoryConfig::loadFromContext(getContext()); double spill_mem_ratio = config.spill_mem_ratio; @@ -53,46 +54,6 @@ SortRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, st return query_plan; } -DB::SortDescription -SortRelParser::parseSortDescription(const google::protobuf::RepeatedPtrField & sort_fields, const DB::Block & header) -{ - static std::map> direction_map = {{1, {1, -1}}, {2, {1, 1}}, {3, {-1, 1}}, {4, {-1, -1}}}; - - DB::SortDescription sort_descr; - for (int i = 0, sz = sort_fields.size(); i < sz; ++i) - { - const auto & sort_field = sort_fields[i]; - /// There is no meaning to sort a const column. - if (sort_field.expr().has_literal()) - continue; - - if (!sort_field.expr().has_selection() || !sort_field.expr().selection().has_direct_reference() - || !sort_field.expr().selection().direct_reference().has_struct_field()) - { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsupport sort field"); - } - auto field_pos = sort_field.expr().selection().direct_reference().struct_field().field(); - - auto direction_iter = direction_map.find(sort_field.direction()); - if (direction_iter == direction_map.end()) - { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsuppor sort direction: {}", sort_field.direction()); - } - if (header.columns()) - { - const auto & col_name = header.getByPosition(field_pos).name; - sort_descr.emplace_back(col_name, direction_iter->second.first, direction_iter->second.second); - sort_descr.back().column_name = col_name; - } - else - { - const auto & col_name = header.getByPosition(field_pos).name; - sort_descr.emplace_back(col_name, direction_iter->second.first, direction_iter->second.second); - } - } - return sort_descr; -} - size_t SortRelParser::parseLimit(std::list & rel_stack_) { if (rel_stack_.empty()) diff --git a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h index 7e6119095c75..27cd4497d19d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h @@ -30,8 +30,6 @@ class SortRelParser : public RelParser DB::QueryPlanPtr parse(DB::QueryPlanPtr query_plan, const substrait::Rel & sort_rel, std::list & rel_stack_) override; - static DB::SortDescription - parseSortDescription(const google::protobuf::RepeatedPtrField & sort_fields, const DB::Block & header); std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.sort().input(); } diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp index d52f2543c852..d3ed22cbb827 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -97,8 +98,8 @@ DB::WindowDescription WindowRelParser::parseWindowDescription(const WindowInfo & { DB::WindowDescription win_descr; win_descr.frame = parseWindowFrame(win_info); - win_descr.partition_by = parsePartitionBy(win_info.partition_exprs); - win_descr.order_by = SortRelParser::parseSortDescription(win_info.sort_fields, current_plan->getCurrentHeader()); + win_descr.partition_by = parseSortFields(current_plan->getCurrentHeader(), win_info.partition_exprs); + win_descr.order_by = parseSortFields(current_plan->getCurrentHeader(), win_info.sort_fields); win_descr.full_sort_description = win_descr.partition_by; win_descr.full_sort_description.insert(win_descr.full_sort_description.end(), win_descr.order_by.begin(), win_descr.order_by.end()); @@ -177,17 +178,11 @@ WindowRelParser::parseWindowFrameType(const std::string & function_name, const s frame_type = window_function.window_type(); if (frame_type == substrait::ROWS) - { return DB::WindowFrame::FrameType::ROWS; - } else if (frame_type == substrait::RANGE) - { return DB::WindowFrame::FrameType::RANGE; - } else - { throw DB::Exception(DB::ErrorCodes::UNKNOWN_TYPE, "Unknow window frame type:{}", frame_type); - } } void WindowRelParser::parseBoundType( @@ -206,13 +201,9 @@ void WindowRelParser::parseBoundType( bound_type = DB::WindowFrame::BoundaryType::Offset; preceding_direction = preceding.offset() >= 0; if (preceding.offset() < 0) - { offset = 0 - preceding.offset(); - } else - { offset = preceding.offset(); - } } else if (bound.has_following()) { @@ -220,13 +211,9 @@ void WindowRelParser::parseBoundType( bound_type = DB::WindowFrame::BoundaryType::Offset; preceding_direction = following.offset() < 0; if (following.offset() < 0) - { offset = 0 - following.offset(); - } else - { offset = following.offset(); - } } else if (bound.has_current_row()) { @@ -252,31 +239,6 @@ void WindowRelParser::parseBoundType( } } -DB::SortDescription WindowRelParser::parsePartitionBy(const google::protobuf::RepeatedPtrField & expressions) -{ - DB::Block header = current_plan->getCurrentHeader(); - DB::SortDescription sort_descr; - for (const auto & expr : expressions) - { - if (expr.has_selection()) - { - auto pos = expr.selection().direct_reference().struct_field().field(); - auto col_name = header.getByPosition(pos).name; - sort_descr.push_back(DB::SortColumnDescription(col_name, 1, 1)); - } - else if (expr.has_literal()) - { - // literal is a special case, see in #2586 - continue; - } - else - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknow partition argument type: {}", expr.DebugString()); - } - } - return sort_descr; -} - WindowFunctionDescription WindowRelParser::parseWindowFunctionDescription( const String & ch_function_name, const substrait::Expression::WindowFunction & window_function, @@ -370,8 +332,7 @@ void WindowRelParser::tryAddProjectionAfterWindow() { ActionsDAG convert_action = ActionsDAG::makeConvertingActions( current_header.getColumnsWithTypeAndName(), output_header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Name); - QueryPlanStepPtr convert_step - = std::make_unique(current_plan->getCurrentHeader(), std::move(convert_action)); + QueryPlanStepPtr convert_step = std::make_unique(current_plan->getCurrentHeader(), std::move(convert_action)); convert_step->setStepDescription("Convert window Output"); steps.emplace_back(convert_step.get()); current_plan->addStep(std::move(convert_step)); diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h index da058af857a9..fccdb1c6f4a6 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h @@ -78,7 +78,6 @@ class WindowRelParser : public RelParser DB::WindowFrame::BoundaryType & bound_type, Field & offset, bool & preceding); - DB::SortDescription parsePartitionBy(const google::protobuf::RepeatedPtrField & expressions); DB::WindowFunctionDescription parseWindowFunctionDescription( const String & ch_function_name, const substrait::Expression::WindowFunction & window_function, diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index 52120cede088..02d43124745e 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -123,7 +123,7 @@ TEST(TestJoin, simple) auto hash_join = std::make_shared(join, right_plan.getCurrentHeader()); QueryPlanStepPtr join_step - = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 1, false); + = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 8192, 1, false); std::cerr << "join step:" << join_step->getOutputHeader().dumpStructure() << std::endl; @@ -145,4 +145,4 @@ TEST(TestJoin, simple) auto res = pipeline->getHeader().cloneEmpty(); executor.pull(res); debug::headBlock(res); -} \ No newline at end of file +} From 72b8810be57cc986a52aa83df50e31281a803733 Mon Sep 17 00:00:00 2001 From: Lingfeng Zhang Date: Thu, 28 Nov 2024 18:58:39 +0800 Subject: [PATCH 163/211] [GLUTEN-7860][CORE] In shuffle writer, replace MemoryMappedFile to avoid OOM (#7861) --- cpp/core/config/GlutenConfig.h | 1 + cpp/core/jni/JniWrapper.cc | 7 ++ cpp/core/shuffle/LocalPartitionWriter.cc | 5 +- cpp/core/shuffle/Options.h | 3 + cpp/core/shuffle/Payload.cc | 5 + cpp/core/shuffle/Spill.cc | 5 +- cpp/core/shuffle/Spill.h | 9 +- cpp/core/shuffle/Utils.cc | 112 ++++++++++++++++++ cpp/core/shuffle/Utils.h | 35 ++++++ .../org/apache/gluten/GlutenConfig.scala | 12 +- 10 files changed, 183 insertions(+), 11 deletions(-) diff --git a/cpp/core/config/GlutenConfig.h b/cpp/core/config/GlutenConfig.h index 3dd2f77f9fae..5a61b27a80b6 100644 --- a/cpp/core/config/GlutenConfig.h +++ b/cpp/core/config/GlutenConfig.h @@ -73,6 +73,7 @@ const std::string kSparkRedactionRegex = "spark.redaction.regex"; const std::string kSparkRedactionString = "*********(redacted)"; const std::string kSparkLegacyTimeParserPolicy = "spark.sql.legacy.timeParserPolicy"; +const std::string kShuffleFileBufferSize = "spark.shuffle.file.buffer"; std::unordered_map parseConfMap(JNIEnv* env, const uint8_t* planData, const int32_t planDataLength); diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 963440f6fc16..794ca6b88f48 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -824,6 +824,13 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe partitionWriterOptions.codecBackend = getCodecBackend(env, codecBackendJstr); partitionWriterOptions.compressionMode = getCompressionMode(env, compressionModeJstr); } + const auto& conf = ctx->getConfMap(); + { + auto it = conf.find(kShuffleFileBufferSize); + if (it != conf.end()) { + partitionWriterOptions.shuffleFileBufferSize = static_cast(stoi(it->second)); + } + } std::unique_ptr partitionWriter; diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index b7bfa19304fd..6692097986c2 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -389,9 +389,9 @@ arrow::Result> LocalPartitionWriter::op std::shared_ptr fout; ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(file)); if (options_.bufferedWrite) { - // The 16k bytes is a temporary allocation and will be freed with file close. + // The `shuffleFileBufferSize` bytes is a temporary allocation and will be freed with file close. // Use default memory pool and count treat the memory as executor memory overhead to avoid unnecessary spill. - return arrow::io::BufferedOutputStream::Create(16384, arrow::default_memory_pool(), fout); + return arrow::io::BufferedOutputStream::Create(options_.shuffleFileBufferSize, arrow::default_memory_pool(), fout); } return fout; } @@ -420,6 +420,7 @@ arrow::Status LocalPartitionWriter::mergeSpills(uint32_t partitionId) { auto spillIter = spills_.begin(); while (spillIter != spills_.end()) { ARROW_ASSIGN_OR_RAISE(auto st, dataFileOs_->Tell()); + (*spillIter)->openForRead(options_.shuffleFileBufferSize); // Read if partition exists in the spilled file and write to the final file. while (auto payload = (*spillIter)->nextPayload(partitionId)) { // May trigger spill during compression. diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h index 6a9e0ec4b340..3a1efdc2ae90 100644 --- a/cpp/core/shuffle/Options.h +++ b/cpp/core/shuffle/Options.h @@ -39,6 +39,7 @@ static constexpr bool kEnableBufferedWrite = true; static constexpr bool kDefaultUseRadixSort = true; static constexpr int32_t kDefaultSortBufferSize = 4096; static constexpr int64_t kDefaultReadBufferSize = 1 << 20; +static constexpr int64_t kDefaultShuffleFileBufferSize = 32 << 10; enum ShuffleWriterType { kHashShuffle, kSortShuffle, kRssSortShuffle }; enum PartitionWriterType { kLocal, kRss }; @@ -86,6 +87,8 @@ struct PartitionWriterOptions { int64_t pushBufferMaxSize = kDefaultPushMemoryThreshold; int64_t sortBufferMaxSize = kDefaultSortBufferThreshold; + + int64_t shuffleFileBufferSize = kDefaultShuffleFileBufferSize; }; struct ShuffleWriterMetrics { diff --git a/cpp/core/shuffle/Payload.cc b/cpp/core/shuffle/Payload.cc index 55f3a4339677..ddf4a409661e 100644 --- a/cpp/core/shuffle/Payload.cc +++ b/cpp/core/shuffle/Payload.cc @@ -481,6 +481,9 @@ arrow::Result> UncompressedDiskBlockPayload::read } arrow::Status UncompressedDiskBlockPayload::serialize(arrow::io::OutputStream* outputStream) { + ARROW_RETURN_IF( + inputStream_ == nullptr, arrow::Status::Invalid("inputStream_ is uninitialized before calling serialize().")); + if (codec_ == nullptr || type_ == Payload::kUncompressed) { ARROW_ASSIGN_OR_RAISE(auto block, inputStream_->Read(rawSize_)); RETURN_NOT_OK(outputStream->Write(block)); @@ -545,6 +548,8 @@ CompressedDiskBlockPayload::CompressedDiskBlockPayload( : Payload(Type::kCompressed, numRows, isValidityBuffer), inputStream_(inputStream), rawSize_(rawSize) {} arrow::Status CompressedDiskBlockPayload::serialize(arrow::io::OutputStream* outputStream) { + ARROW_RETURN_IF( + inputStream_ == nullptr, arrow::Status::Invalid("inputStream_ is uninitialized before calling serialize().")); ScopedTimer timer(&writeTime_); ARROW_ASSIGN_OR_RAISE(auto block, inputStream_->Read(rawSize_)); RETURN_NOT_OK(outputStream->Write(block)); diff --git a/cpp/core/shuffle/Spill.cc b/cpp/core/shuffle/Spill.cc index d8b9bc7ebf99..8cc3a9d05ea7 100644 --- a/cpp/core/shuffle/Spill.cc +++ b/cpp/core/shuffle/Spill.cc @@ -34,7 +34,6 @@ bool Spill::hasNextPayload(uint32_t partitionId) { } std::unique_ptr Spill::nextPayload(uint32_t partitionId) { - openSpillFile(); if (!hasNextPayload(partitionId)) { return nullptr; } @@ -71,9 +70,9 @@ void Spill::insertPayload( } } -void Spill::openSpillFile() { +void Spill::openForRead(uint64_t shuffleFileBufferSize) { if (!is_) { - GLUTEN_ASSIGN_OR_THROW(is_, arrow::io::MemoryMappedFile::Open(spillFile_, arrow::io::FileMode::READ)); + GLUTEN_ASSIGN_OR_THROW(is_, MmapFileStream::open(spillFile_, shuffleFileBufferSize)); rawIs_ = is_.get(); } } diff --git a/cpp/core/shuffle/Spill.h b/cpp/core/shuffle/Spill.h index c82a60f562b4..fd692537c570 100644 --- a/cpp/core/shuffle/Spill.h +++ b/cpp/core/shuffle/Spill.h @@ -37,6 +37,8 @@ class Spill final { SpillType type() const; + void openForRead(uint64_t shuffleFileBufferSize); + bool hasNextPayload(uint32_t partitionId); std::unique_ptr nextPayload(uint32_t partitionId); @@ -69,15 +71,12 @@ class Spill final { }; SpillType type_; - std::shared_ptr is_; + std::shared_ptr is_; std::list partitionPayloads_{}; - std::shared_ptr inputStream_{}; std::string spillFile_; int64_t spillTime_; int64_t compressTime_; - arrow::io::InputStream* rawIs_; - - void openSpillFile(); + arrow::io::InputStream* rawIs_{nullptr}; }; } // namespace gluten \ No newline at end of file diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index a11b6b09aa3d..457702c0c9df 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -16,8 +16,11 @@ */ #include "shuffle/Utils.h" +#include #include #include +#include +#include #include #include #include @@ -151,6 +154,14 @@ arrow::Status getLengthBufferAndValueBufferStream( *compressedLengthPtr = actualLength; return arrow::Status::OK(); } + +uint64_t roundUpToPageSize(uint64_t value) { + static auto pageSize = static_cast(arrow::internal::GetPageSize()); + static auto pageMask = ~(pageSize - 1); + DCHECK_GT(pageSize, 0); + DCHECK_EQ(pageMask & pageSize, pageSize); + return (value + pageSize - 1) & pageMask; +} } // namespace arrow::Result> makeCompressedRecordBatch( @@ -212,6 +223,107 @@ arrow::Result> makeUncompressedRecordBatch( } return arrow::RecordBatch::Make(writeSchema, 1, {arrays}); } + +MmapFileStream::MmapFileStream(arrow::internal::FileDescriptor fd, uint8_t* data, int64_t size, uint64_t prefetchSize) + : prefetchSize_(roundUpToPageSize(prefetchSize)), fd_(std::move(fd)), data_(data), size_(size){}; + +arrow::Result> MmapFileStream::open(const std::string& path, uint64_t prefetchSize) { + ARROW_ASSIGN_OR_RAISE(auto fileName, arrow::internal::PlatformFilename::FromString(path)); + + ARROW_ASSIGN_OR_RAISE(auto fd, arrow::internal::FileOpenReadable(fileName)); + ARROW_ASSIGN_OR_RAISE(auto size, arrow::internal::FileGetSize(fd.fd())); + + ARROW_RETURN_IF(size == 0, arrow::Status::Invalid("Cannot mmap an empty file: ", path)); + + void* result = mmap(nullptr, size, PROT_READ, MAP_PRIVATE, fd.fd(), 0); + if (result == MAP_FAILED) { + return arrow::Status::IOError("Memory mapping file failed: ", ::arrow::internal::ErrnoMessage(errno)); + } + + return std::make_shared(std::move(fd), static_cast(result), size, prefetchSize); +} + +arrow::Result MmapFileStream::actualReadSize(int64_t nbytes) { + if (nbytes < 0 || pos_ > size_) { + return arrow::Status::IOError("Read out of range. Offset: ", pos_, " Size: ", nbytes, " File Size: ", size_); + } + return std::min(size_ - pos_, nbytes); +} + +bool MmapFileStream::closed() const { + return data_ == nullptr; +}; + +void MmapFileStream::advance(int64_t length) { + // Dont need data before pos + auto purgeLength = (pos_ - posRetain_) / prefetchSize_ * prefetchSize_; + if (purgeLength > 0) { + int ret = madvise(data_ + posRetain_, purgeLength, MADV_DONTNEED); + if (ret != 0) { + LOG(WARNING) << "fadvise failed " << ::arrow::internal::ErrnoMessage(errno); + } + posRetain_ += purgeLength; + } + + pos_ += length; +} + +void MmapFileStream::willNeed(int64_t length) { + // Skip if already fetched + if (pos_ + length <= posFetch_) { + return; + } + + // Round up to multiple of prefetchSize + auto fetchLen = ((length + prefetchSize_ - 1) / prefetchSize_) * prefetchSize_; + fetchLen = std::min(size_ - pos_, fetchLen); + int ret = madvise(data_ + posFetch_, fetchLen, MADV_WILLNEED); + if (ret != 0) { + LOG(WARNING) << "madvise willneed failed: " << ::arrow::internal::ErrnoMessage(errno); + } + + posFetch_ += fetchLen; +} + +arrow::Status MmapFileStream::Close() { + if (data_ != nullptr) { + int result = munmap(data_, size_); + if (result != 0) { + LOG(WARNING) << "munmap failed"; + } + data_ = nullptr; + } + + return fd_.Close(); +} + +arrow::Result MmapFileStream::Tell() const { + return pos_; +} + +arrow::Result MmapFileStream::Read(int64_t nbytes, void* out) { + ARROW_ASSIGN_OR_RAISE(nbytes, actualReadSize(nbytes)); + + if (nbytes > 0) { + memcpy(out, data_ + pos_, nbytes); + advance(nbytes); + } + + return nbytes; +} + +arrow::Result> MmapFileStream::Read(int64_t nbytes) { + ARROW_ASSIGN_OR_RAISE(nbytes, actualReadSize(nbytes)); + + if (nbytes > 0) { + auto buffer = std::make_shared(data_ + pos_, nbytes); + willNeed(nbytes); + advance(nbytes); + return buffer; + } else { + return std::make_shared(nullptr, 0); + } +} } // namespace gluten std::string gluten::getShuffleSpillDir(const std::string& configuredDir, int32_t subDirId) { diff --git a/cpp/core/shuffle/Utils.h b/cpp/core/shuffle/Utils.h index 64b9292d9d36..2e5ff58b6ebf 100644 --- a/cpp/core/shuffle/Utils.h +++ b/cpp/core/shuffle/Utils.h @@ -70,4 +70,39 @@ arrow::Result> makeUncompressedRecordBatch( std::shared_ptr zeroLengthNullBuffer(); +// MmapFileStream is used to optimize sequential file reading. It uses madvise +// to prefetch and release memory timely. +class MmapFileStream : public arrow::io::InputStream { + public: + MmapFileStream(arrow::internal::FileDescriptor fd, uint8_t* data, int64_t size, uint64_t prefetchSize); + + static arrow::Result> open(const std::string& path, uint64_t prefetchSize = 0); + + arrow::Result Tell() const override; + + arrow::Status Close() override; + + arrow::Result Read(int64_t nbytes, void* out) override; + + arrow::Result> Read(int64_t nbytes) override; + + bool closed() const override; + + private: + arrow::Result actualReadSize(int64_t nbytes); + + void advance(int64_t length); + + void willNeed(int64_t length); + + // Page-aligned prefetch size + const int64_t prefetchSize_; + arrow::internal::FileDescriptor fd_; + uint8_t* data_ = nullptr; + int64_t size_; + int64_t pos_ = 0; + int64_t posFetch_ = 0; + int64_t posRetain_ = 0; +}; + } // namespace gluten diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index c4c67f49a59b..b63a6bfecca7 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -17,7 +17,7 @@ package org.apache.gluten import org.apache.spark.internal.Logging -import org.apache.spark.network.util.ByteUnit +import org.apache.spark.network.util.{ByteUnit, JavaUtils} import org.apache.spark.sql.internal.SQLConf import com.google.common.collect.ImmutableList @@ -568,6 +568,7 @@ object GlutenConfig { val SPARK_OFFHEAP_SIZE_KEY = "spark.memory.offHeap.size" val SPARK_OFFHEAP_ENABLED = "spark.memory.offHeap.enabled" val SPARK_REDACTION_REGEX = "spark.redaction.regex" + val SPARK_SHUFFLE_FILE_BUFFER = "spark.shuffle.file.buffer" // For Soft Affinity Scheduling // Enable Soft Affinity Scheduling, default value is false @@ -736,6 +737,15 @@ object GlutenConfig { ) keyWithDefault.forEach(e => nativeConfMap.put(e._1, conf.getOrElse(e._1, e._2))) + conf + .get(SPARK_SHUFFLE_FILE_BUFFER) + .foreach( + v => + nativeConfMap + .put( + SPARK_SHUFFLE_FILE_BUFFER, + (JavaUtils.byteStringAs(v, ByteUnit.KiB) * 1024).toString)) + // Backend's dynamic session conf only. val confPrefix = prefixOf(backendName) conf From e56da333dc33de90540efa6dc63efa70c3c71ce8 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 28 Nov 2024 21:20:17 +0800 Subject: [PATCH 164/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_28) (#8067) Upstream Velox's New Commits: 7e4656f24 by Xiaoxuan Meng, fix: Back out "fix: Fix the driver block hanging issue in serialized execution mode" (11681) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 01740f138fa4..6c7fabab88a6 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_27 +VELOX_BRANCH=2024_11_28 VELOX_HOME="" OS=`uname -s` From f9f4ddc5bba04800c9d80521fa4c50986775be16 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 29 Nov 2024 16:09:37 +0800 Subject: [PATCH 165/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_29) (#8086) Upstream Velox's New Commits: 2e5dadcef by Jimmy Lu, fix: Delta update corner cases (11682) e8fe5b967 by dependabot[bot], build(deps): Bump pypa/cibuildwheel from 2.12.1 to 2.22.0 (11643) 1ce3c7ad2 by Xiaoxuan Meng, fix: Fix the memory reclaim bytes for hash join (11642) 4a4b4a030 by Sergey Pershin, Rename "max-coalesced-distance-bytes" and add session property. (11671) 0d572d23a by David Reveman, build(ci): Add breeze build and test job (11637) --- cpp/velox/compute/VeloxBackend.cc | 4 ++-- cpp/velox/config/VeloxConfig.h | 2 +- docs/Configuration.md | 2 +- ep/build-velox/src/get_velox.sh | 2 +- .../src/main/scala/org/apache/gluten/GlutenConfig.scala | 8 ++++---- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index 62cc36163d33..c453b9981fab 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -273,8 +273,8 @@ void VeloxBackend::initConnector() { connectorConfMap[velox::connector::hive::HiveConfig::kMaxCoalescedBytes] = backendConf_->get(kMaxCoalescedBytes, "67108864"); // 64M - connectorConfMap[velox::connector::hive::HiveConfig::kMaxCoalescedDistanceBytes] = - backendConf_->get(kMaxCoalescedDistanceBytes, "1048576"); // 1M + connectorConfMap[velox::connector::hive::HiveConfig::kMaxCoalescedDistance] = + backendConf_->get(kMaxCoalescedDistance, "512KB"); // 512KB connectorConfMap[velox::connector::hive::HiveConfig::kPrefetchRowGroups] = backendConf_->get(kPrefetchRowGroups, "1"); connectorConfMap[velox::connector::hive::HiveConfig::kLoadQuantum] = diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index cb70dc62780b..dc234152a362 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -118,7 +118,7 @@ const std::string kDirectorySizeGuess = "spark.gluten.sql.columnar.backend.velox const std::string kFilePreloadThreshold = "spark.gluten.sql.columnar.backend.velox.filePreloadThreshold"; const std::string kPrefetchRowGroups = "spark.gluten.sql.columnar.backend.velox.prefetchRowGroups"; const std::string kLoadQuantum = "spark.gluten.sql.columnar.backend.velox.loadQuantum"; -const std::string kMaxCoalescedDistanceBytes = "spark.gluten.sql.columnar.backend.velox.maxCoalescedDistanceBytes"; +const std::string kMaxCoalescedDistance = "spark.gluten.sql.columnar.backend.velox.maxCoalescedDistance"; const std::string kMaxCoalescedBytes = "spark.gluten.sql.columnar.backend.velox.maxCoalescedBytes"; const std::string kCachePrefetchMinPct = "spark.gluten.sql.columnar.backend.velox.cachePrefetchMinPct"; diff --git a/docs/Configuration.md b/docs/Configuration.md index 0c44341c363c..cb8efe802e2a 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -89,7 +89,7 @@ The following configurations are related to Velox settings. | spark.gluten.sql.columnar.backend.velox.filePreloadThreshold | Set the file preload threshold for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.prefetchRowGroups | Set the prefetch row groups for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.loadQuantum | Set the load quantum for velox file scan. | | -| spark.gluten.sql.columnar.backend.velox.maxCoalescedDistanceBytes | Set the max coalesced distance bytes for velox file scan. | | +| spark.gluten.sql.columnar.backend.velox.maxCoalescedDistance | Set the max coalesced distance for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.maxCoalescedBytes | Set the max coalesced bytes for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.cachePrefetchMinPct | Set prefetch cache min pct for velox file scan. | | | spark.gluten.velox.awsSdkLogLevel | Log granularity of AWS C++ SDK in velox. | FATAL | diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 6c7fabab88a6..1d3a84d858a8 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_28 +VELOX_BRANCH=2024_11_29 VELOX_HOME="" OS=`uname -s` diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index b63a6bfecca7..f643ad7eed0e 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -456,7 +456,7 @@ class GlutenConfig(conf: SQLConf) extends Logging { conf.getConf(PREFETCH_ROW_GROUPS) def loadQuantum: Long = conf.getConf(LOAD_QUANTUM) - def maxCoalescedDistanceBytes: Long = + def maxCoalescedDistance: String = conf.getConf(MAX_COALESCED_DISTANCE_BYTES) def maxCoalescedBytes: Long = conf.getConf(MAX_COALESCED_BYTES) @@ -2095,11 +2095,11 @@ object GlutenConfig { .createWithDefaultString("256MB") val MAX_COALESCED_DISTANCE_BYTES = - buildStaticConf("spark.gluten.sql.columnar.backend.velox.maxCoalescedDistanceBytes") + buildStaticConf("spark.gluten.sql.columnar.backend.velox.maxCoalescedDistance") .internal() .doc(" Set the max coalesced distance bytes for velox file scan") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("1MB") + .stringConf + .createWithDefaultString("512KB") val MAX_COALESCED_BYTES = buildStaticConf("spark.gluten.sql.columnar.backend.velox.maxCoalescedBytes") From ea0bcd5b7027f86e7625f9b77f1c5542697fb2a0 Mon Sep 17 00:00:00 2001 From: Zhichao Zhang Date: Fri, 29 Nov 2024 16:38:05 +0800 Subject: [PATCH 166/211] [GLUTEN-8094][CH][Part-1] Support reading data from the iceberg with CH backend (#8095) * [GLUTEN-8094][CH][Part-1] Support reading data from the iceberg with CH backend Support reading data from the iceberg with CH backend - basic iceberg scan transformer - read from the iceberg with the copy-on-write mode --- backends-clickhouse/pom.xml | 64 ++ .../clickhouse/CHTransformerApi.scala | 5 + .../iceberg/ClickHouseIcebergSuite.scala | 640 ++++++++++++++++++ ...seTPCHColumnarShuffleParquetAQESuite.scala | 26 +- .../Storages/SubstraitSource/FormatFile.cpp | 4 +- .../source/GlutenIcebergSourceUtil.scala | 8 +- .../gluten/backendsapi/TransformerApi.scala | 3 + 7 files changed, 744 insertions(+), 6 deletions(-) create mode 100644 backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index c616266bf218..3a4a9422479b 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -13,6 +13,70 @@ jar Gluten Backends ClickHouse + + + iceberg + + false + + + + org.apache.gluten + gluten-iceberg + ${project.version} + + + org.apache.iceberg + iceberg-spark-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + provided + + + org.apache.iceberg + iceberg-spark-runtime-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + test + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-iceberg-sources + generate-sources + + add-source + + + + ${project.basedir}/src/main-iceberg/scala + ${project.basedir}/src/main-iceberg/java + + + + + add-iceberg-test-sources + generate-test-sources + + add-test-source + + + + ${project.basedir}/src/test-iceberg/scala + ${project.basedir}/src/test-iceberg/java + + + + + + + + + + org.apache.gluten diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala index 438c6cd3dfb0..0be8cf2c25bf 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala @@ -42,6 +42,7 @@ import org.apache.spark.util.collection.BitSet import com.google.common.collect.Lists import com.google.protobuf.{Any, Message} +import org.apache.hadoop.fs.Path import java.util @@ -269,4 +270,8 @@ class CHTransformerApi extends TransformerApi with Logging { } packPBMessage(write.build()) } + + /** use Hadoop Path class to encode the file path */ + override def encodeFilePathIfNeed(filePath: String): String = + (new Path(filePath)).toUri.toASCIIString } diff --git a/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala b/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala new file mode 100644 index 000000000000..676061235ed0 --- /dev/null +++ b/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala @@ -0,0 +1,640 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution.iceberg + +import org.apache.gluten.execution.{GlutenClickHouseWholeStageTransformerSuite, IcebergScanTransformer} +import org.apache.gluten.GlutenConfig +import org.apache.spark.SparkConf +import org.apache.spark.sql.Row + +class ClickHouseIcebergSuite extends GlutenClickHouseWholeStageTransformerSuite { + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.sql.adaptive.enabled", "true") + .set("spark.sql.shuffle.partitions", "2") + .set("spark.memory.offHeap.size", "2g") + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set("spark.sql.autoBroadcastJoinThreshold", "-1") + .set( + "spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") + .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkCatalog") + .set("spark.sql.catalog.spark_catalog.type", "hadoop") + .set("spark.sql.catalog.spark_catalog.warehouse", s"file://$basePath/tpch-data-iceberg") + } + + test("iceberg transformer exists") { + withTable("iceberg_tb") { + spark.sql( + """ + |create table iceberg_tb using iceberg as + |(select 1 as col1, 2 as col2, 3 as col3) + |""".stripMargin) + + runQueryAndCompare( + """ + |select * from iceberg_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + testWithSpecifiedSparkVersion( + "iceberg bucketed join", Array("3.3", "3.5")) { + val leftTable = "p_str_tb" + val rightTable = "p_int_tb" + withTable(leftTable, rightTable) { + // Partition key of string type. + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $leftTable(id int, name string, p string) + |using iceberg + |partitioned by (bucket(4, id)); + |""".stripMargin) + spark.sql( + s""" + |insert into table $leftTable values + |(4, 'a5', 'p4'), + |(1, 'a1', 'p1'), + |(2, 'a3', 'p2'), + |(1, 'a2', 'p1'), + |(3, 'a4', 'p3'); + |""".stripMargin + ) + } + + // Partition key of integer type. + withSQLConf( + GlutenConfig.GLUTEN_ENABLED_KEY -> "false" + ) { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $rightTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id)); + |""".stripMargin) + spark.sql( + s""" + |insert into table $rightTable values + |(3, 'b4', 23), + |(1, 'b2', 21), + |(4, 'b5', 24), + |(2, 'b3', 22), + |(1, 'b1', 21); + |""".stripMargin + ) + } + + withSQLConf( + "spark.sql.sources.v2.bucketing.enabled" -> "true", + "spark.sql.requireAllClusterKeysForCoPartition" -> "false", + "spark.sql.adaptive.enabled" -> "false", + "spark.sql.iceberg.planning.preserve-data-grouping" -> "true", + "spark.sql.autoBroadcastJoinThreshold" -> "-1", + "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true" + ) { + runQueryAndCompare( + s""" + |select s.id, s.name, i.name, i.p + | from $leftTable s inner join $rightTable i + | on s.id = i.id; + |""".stripMargin) { + df => { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[IcebergScanTransformer] + }) == 2) + getExecutedPlan(df).map { + case plan if plan.isInstanceOf[IcebergScanTransformer] => + assert( + plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) + case _ => // do nothing + } + checkLengthAndPlan(df, 7) + } + } + } + } + } + + testWithSpecifiedSparkVersion( + "iceberg bucketed join with partition", Array("3.3", "3.5")) { + val leftTable = "p_str_tb" + val rightTable = "p_int_tb" + withTable(leftTable, rightTable) { + // Partition key of string type. + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $leftTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $leftTable values + |(4, 'a5', 2), + |(1, 'a1', 1), + |(2, 'a3', 1), + |(1, 'a2', 1), + |(3, 'a4', 2); + |""".stripMargin + ) + } + + // Partition key of integer type. + withSQLConf( + GlutenConfig.GLUTEN_ENABLED_KEY -> "false" + ) { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $rightTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $rightTable values + |(3, 'b4', 2), + |(1, 'b2', 1), + |(4, 'b5', 2), + |(2, 'b3', 1), + |(1, 'b1', 1); + |""".stripMargin + ) + } + + withSQLConf( + "spark.sql.sources.v2.bucketing.enabled" -> "true", + "spark.sql.requireAllClusterKeysForCoPartition" -> "false", + "spark.sql.adaptive.enabled" -> "false", + "spark.sql.iceberg.planning.preserve-data-grouping" -> "true", + "spark.sql.autoBroadcastJoinThreshold" -> "-1", + "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true" + ) { + runQueryAndCompare( + s""" + |select s.id, s.name, i.name, i.p + | from $leftTable s inner join $rightTable i + | on s.id = i.id and s.p = i.p; + |""".stripMargin) { + df => { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[IcebergScanTransformer] + }) == 2) + getExecutedPlan(df).map { + case plan if plan.isInstanceOf[IcebergScanTransformer] => + assert( + plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) + case _ => // do nothing + } + checkLengthAndPlan(df, 7) + } + } + } + } + } + + testWithSpecifiedSparkVersion( + "iceberg bucketed join with partition filter", Array("3.3", "3.5")) { + val leftTable = "p_str_tb" + val rightTable = "p_int_tb" + withTable(leftTable, rightTable) { + // Partition key of string type. + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $leftTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $leftTable values + |(4, 'a5', 2), + |(1, 'a1', 1), + |(2, 'a3', 1), + |(1, 'a2', 1), + |(3, 'a4', 2); + |""".stripMargin + ) + } + + // Partition key of integer type. + withSQLConf( + GlutenConfig.GLUTEN_ENABLED_KEY -> "false" + ) { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $rightTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $rightTable values + |(3, 'b4', 2), + |(1, 'b2', 1), + |(4, 'b5', 2), + |(2, 'b3', 1), + |(1, 'b1', 1); + |""".stripMargin + ) + } + + withSQLConf( + "spark.sql.sources.v2.bucketing.enabled" -> "true", + "spark.sql.requireAllClusterKeysForCoPartition" -> "false", + "spark.sql.adaptive.enabled" -> "false", + "spark.sql.iceberg.planning.preserve-data-grouping" -> "true", + "spark.sql.autoBroadcastJoinThreshold" -> "-1", + "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true" + ) { + runQueryAndCompare( + s""" + |select s.id, s.name, i.name, i.p + | from $leftTable s inner join $rightTable i + | on s.id = i.id + | where s.p = 1 and i.p = 1; + |""".stripMargin) { + df => { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[IcebergScanTransformer] + }) == 2) + getExecutedPlan(df).map { + case plan if plan.isInstanceOf[IcebergScanTransformer] => + assert( + plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 1) + case _ => // do nothing + } + checkLengthAndPlan(df, 5) + } + } + } + } + } + + testWithSpecifiedSparkVersion("iceberg: time travel") { + withTable("iceberg_tm") { + spark.sql( + s""" + |create table iceberg_tm (id int, name string) using iceberg + |""".stripMargin) + spark.sql( + s""" + |insert into iceberg_tm values (1, "v1"), (2, "v2") + |""".stripMargin) + spark.sql( + s""" + |insert into iceberg_tm values (3, "v3"), (4, "v4") + |""".stripMargin) + + val df = + spark.sql("select snapshot_id from default.iceberg_tm.snapshots where parent_id is null") + val value = df.collectAsList().get(0).getAs[Long](0); + spark.sql(s"call system.set_current_snapshot('default.iceberg_tm',$value)"); + val data = runQueryAndCompare("select * from iceberg_tm") { _ => } + checkLengthAndPlan(data, 2) + checkAnswer(data, Row(1, "v1") :: Row(2, "v2") :: Nil) + } + } + + test("iceberg: partition filters") { + withTable("iceberg_pf") { + spark.sql( + s""" + |create table iceberg_pf (id int, name string) + | using iceberg partitioned by (name) + |""".stripMargin) + spark.sql( + s""" + |insert into iceberg_pf values (1, "v1"), (2, "v2"), (3, "v1"), (4, "v2") + |""".stripMargin) + val df1 = runQueryAndCompare("select * from iceberg_pf where name = 'v1'") { _ => } + checkLengthAndPlan(df1, 2) + checkAnswer(df1, Row(1, "v1") :: Row(3, "v1") :: Nil) + } + } + + test("iceberg read mor table - delete and update") { + withTable("iceberg_mor_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2' + |) + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'), + | (4, 'a4', 'p1'), (5, 'a5', 'p2'), (6, 'a6', 'p1'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + // Delete row again. + spark.sql( + """ + |delete from iceberg_mor_tb where id = 6; + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + // TODO: support merge-on-read mode + ignore("iceberg read mor table - delete and update with merge-on-read mode") { + withTable("iceberg_mor_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2', + | 'write.delete.mode' = 'merge-on-read', + | 'write.update.mode' = 'merge-on-read', + | 'write.merge.mode' = 'merge-on-read' + |) + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'), + | (4, 'a4', 'p1'), (5, 'a5', 'p2'), (6, 'a6', 'p1'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + // Delete row again. + spark.sql( + """ + |delete from iceberg_mor_tb where id = 6; + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + test("iceberg read mor table - merge into") { + withTable("iceberg_mor_tb", "merge_into_source_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2' + |) + |partitioned by (p); + |""".stripMargin) + spark.sql( + """ + |create table merge_into_source_tb ( + | id int, + | name string, + | p string + |) using iceberg; + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'); + |""".stripMargin) + spark.sql( + """ + |insert into table merge_into_source_tb + |values (1, 'a1_1', 'p2'), (2, 'a2_1', 'p2'), (3, 'a3_1', 'p1'), + | (4, 'a4', 'p2'), (5, 'a5', 'p1'), (6, 'a6', 'p2'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + + // Merge into. + spark.sql( + """ + |merge into iceberg_mor_tb t + |using (select * from merge_into_source_tb) s + |on t.id = s.id + |when matched then + | update set t.name = s.name, t.p = s.p + |when not matched then + | insert (id, name, p) values (s.id, s.name, s.p); + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + // TODO: support merge-on-read mode + ignore("iceberg read mor table - merge into with merge-on-read mode") { + withTable("iceberg_mor_tb", "merge_into_source_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2', + | 'write.delete.mode' = 'merge-on-read', + | 'write.update.mode' = 'merge-on-read', + | 'write.merge.mode' = 'merge-on-read' + |) + |partitioned by (p); + |""".stripMargin) + spark.sql( + """ + |create table merge_into_source_tb ( + | id int, + | name string, + | p string + |) using iceberg; + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'); + |""".stripMargin) + spark.sql( + """ + |insert into table merge_into_source_tb + |values (1, 'a1_1', 'p2'), (2, 'a2_1', 'p2'), (3, 'a3_1', 'p1'), + | (4, 'a4', 'p2'), (5, 'a5', 'p1'), (6, 'a6', 'p2'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + + // Merge into. + spark.sql( + """ + |merge into iceberg_mor_tb t + |using (select * from merge_into_source_tb) s + |on t.id = s.id + |when matched then + | update set t.name = s.name, t.p = s.p + |when not matched then + | insert (id, name, p) values (s.id, s.name, s.p); + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + // Spark configuration spark.sql.iceberg.handle-timestamp-without-timezone is not supported + // in Spark 3.4 + // TODO: there is a bug when using timestamp type as the partition column + testWithSpecifiedSparkVersion("iceberg partition type - timestamp", Array("")) { + Seq("true", "false").foreach { + flag => + withSQLConf( + "spark.sql.iceberg.handle-timestamp-without-timezone" -> flag, + "spark.sql.iceberg.use-timestamp-without-timezone-in-new-tables" -> flag) { + withTable("part_by_timestamp") { + spark.sql( + """ + |create table part_by_timestamp ( + | p timestamp + |) using iceberg + |tblproperties ( + | 'format-version' = '1' + |) + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table part_by_timestamp + |values (TIMESTAMP '2022-01-01 00:01:20'); + |""".stripMargin) + val df = spark.sql("select * from part_by_timestamp") + checkAnswer(df, Row(java.sql.Timestamp.valueOf("2022-01-01 00:01:20")) :: Nil) + } + } + } + } +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index 1c3e4de4c6ef..ad9cb854d97b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -18,7 +18,6 @@ package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig import org.apache.gluten.execution._ -import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.Arm import org.apache.spark.SparkConf @@ -116,6 +115,31 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite } } + // TODO: there is a bug when using timestamp type as the partition column + ignore("test timestamp as partition column") { + spark.sql(""" + |create table part_by_timestamp ( + | a int, + | b timestamp, + | c string, + | p timestamp + |) using parquet + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql(""" + |insert into table part_by_timestamp + |values + |(1, TIMESTAMP '2022-01-01 00:01:20', '2022-01-01 00:01:20', + |TIMESTAMP '2022-01-01 00:01:20'); + |""".stripMargin) + compareResultsAgainstVanillaSpark( + "select a, b, to_timestamp(c), p from part_by_timestamp", + compareResult = true, + customCheck = { _ => }) + } + test("TPCH Q2") { runTPCHQuery(2) { df => diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp index 3c68c70e666d..ee54a8ff5e9a 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp @@ -87,7 +87,7 @@ FormatFilePtr FormatFileUtil::createFile( DB::ContextPtr context, ReadBufferBuilderPtr read_buffer_builder, const substrait::ReadRel::LocalFiles::FileOrFiles & file) { #if USE_PARQUET - if (file.has_parquet()) + if (file.has_parquet() || (file.has_iceberg() && file.iceberg().has_parquet())) { auto config = ExecutorConfig::loadFromContext(context); return std::make_shared(context, file, read_buffer_builder, config.use_local_format); @@ -95,7 +95,7 @@ FormatFilePtr FormatFileUtil::createFile( #endif #if USE_ORC - if (file.has_orc()) + if (file.has_orc() || (file.has_iceberg() && file.iceberg().has_orc())) return std::make_shared(context, file, read_buffer_builder); #endif diff --git a/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala b/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala index ad8222cff5f5..a7451355b00f 100644 --- a/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala +++ b/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala @@ -16,6 +16,7 @@ */ package org.apache.iceberg.spark.source +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.substrait.rel.{IcebergLocalFilesBuilder, SplitInfo} import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat @@ -25,7 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.connector.read.{InputPartition, Scan} import org.apache.spark.sql.types.StructType -import org.apache.iceberg.{CombinedScanTask, DeleteFile, FileFormat, FileScanTask, ScanTask, Schema} +import org.apache.iceberg._ import org.apache.iceberg.spark.SparkSchemaUtil import java.lang.{Long => JLong} @@ -50,8 +51,9 @@ object GlutenIcebergSourceUtil { val tasks = partition.taskGroup[ScanTask]().tasks().asScala asFileScanTask(tasks.toList).foreach { task => - val filePath = task.file().path().toString - paths.add(filePath) + paths.add( + BackendsApiManager.getTransformerApiInstance + .encodeFilePathIfNeed(task.file().path().toString)) starts.add(task.start()) lengths.add(task.length()) partitionColumns.add(getPartitionColumns(task, readPartitionSchema)) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala index 9642f63b3be3..69cea9c5470d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala @@ -76,4 +76,7 @@ trait TransformerApi { def invalidateSQLExecutionResource(executionId: String): Unit = {} def genWriteParameters(fileFormat: FileFormat, writeOptions: Map[String, String]): Any + + /** use Hadoop Path class to encode the file path */ + def encodeFilePathIfNeed(filePath: String): String = filePath } From 738b226264a7030a844f6852b97a8d72c26abd5e Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Fri, 29 Nov 2024 16:42:33 +0800 Subject: [PATCH 167/211] [GLUTEN-8074][CH] Fix adjust output constant column (#8076) * [GLUTEN-8074][CH] Fix adjust output constant column --- .../GlutenClickHouseDecimalSuite.scala | 12 +++++++++ .../Parser/SerializedPlanParser.cpp | 25 +++++++++++++------ 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala index 2663a8fb81fb..9988d5372943 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala @@ -404,6 +404,18 @@ class GlutenClickHouseDecimalSuite ) } + test("GLUTEN-8074 Fix adjust output constant column") { + val sql = + s""" + |select bround(1002.5786, 3), + | bround(-10.8, 0), + | bround(13.888888888889, 5) + |from $decimalTable + |WHERE bround(cast(decimal_field as decimal(30, 2)), 1) > 0 LIMIT 2; + |""".stripMargin + runQueryAndCompare(sql)(_ => {}) + } + test("fix decimal32 with negative value") { val schema = StructType.apply( Seq( diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 42541d9d9204..95086121d4cb 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -160,20 +160,29 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con else { need_final_project = true; - bool need_const = origin_column.column && isColumnConst(*origin_column.column); - ColumnWithTypeAndName final_column( - need_const ? final_type->createColumnConst(0, assert_cast(*origin_column.column).getField()) - : final_type->createColumn(), - final_type, - origin_column.name); - final_columns.emplace_back(std::move(final_column)); + if (need_const) + { + const DB::ContextPtr context = DB::CurrentThread::get().getQueryContext(); + const FunctionOverloadResolverPtr & cast_resolver = FunctionFactory::instance().get("CAST", context); + const DataTypePtr string_type = std::make_shared(); + ColumnWithTypeAndName to_type_column = {string_type->createColumnConst(1, final_type->getName()), string_type, "__cast_const__"}; + FunctionBasePtr cast_function = cast_resolver->build({origin_column, to_type_column}); + ColumnPtr const_col = ColumnConst::create(cast_function->execute({origin_column, to_type_column}, final_type, 1), 1); + ColumnWithTypeAndName final_column(const_col, final_type, origin_column.name); + final_columns.emplace_back(std::move(final_column)); + } + else + { + ColumnWithTypeAndName final_column(final_type->createColumn(), final_type, origin_column.name); + final_columns.emplace_back(std::move(final_column)); + } } } if (need_final_project) { - ActionsDAG final_project = ActionsDAG::makeConvertingActions(origin_columns, final_columns, ActionsDAG::MatchColumnsMode::Position); + ActionsDAG final_project = ActionsDAG::makeConvertingActions(origin_columns, final_columns, ActionsDAG::MatchColumnsMode::Position, true); QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentHeader(), std::move(final_project)); final_project_step->setStepDescription("Project for output schema"); From 5055e8e5b4c7a05ae71067337926a4a33413a025 Mon Sep 17 00:00:00 2001 From: lgbo Date: Fri, 29 Nov 2024 17:18:11 +0800 Subject: [PATCH 168/211] [GLUTEN-8096][CH] Invalid header for disk tmp file (#8100) [CH] Invalid header for disk tmp file --- .../execution/GlutenClickHouseTPCHBucketSuite.scala | 3 +++ .../tpcds/GlutenClickHouseTPCDSParquetSuite.scala | 3 +++ .../tpch/GlutenClickHouseTPCHParquetBucketSuite.scala | 3 +++ cpp-ch/local-engine/Common/GlutenConfig.cpp | 1 + cpp-ch/local-engine/Common/GlutenConfig.h | 2 ++ .../local-engine/Operator/GraceAggregatingTransform.cpp | 8 +++++--- cpp-ch/local-engine/Operator/GraceAggregatingTransform.h | 1 + 7 files changed, 18 insertions(+), 3 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala index 6a09bf494217..7bccb6dfb598 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala @@ -47,6 +47,9 @@ class GlutenClickHouseTPCHBucketSuite .set("spark.sql.autoBroadcastJoinThreshold", "-1") // for test bucket join .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") + .set( + "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_grace_aggregate_spill_test", + "true") } override protected val createNullableTables = true diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala index aa7c6b0f560c..0ba7de90c670 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala @@ -38,6 +38,9 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui .set("spark.memory.offHeap.size", "4g") .set("spark.gluten.sql.validation.logLevel", "ERROR") .set("spark.gluten.sql.validation.printStackOnFailure", "true") + .set( + "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_grace_aggregate_spill_test", + "true") } executeTPCDSTest(false) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala index 59b5834e64a1..a257e2ed5094 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala @@ -53,6 +53,9 @@ class GlutenClickHouseTPCHParquetBucketSuite .set("spark.sql.autoBroadcastJoinThreshold", "-1") // for test bucket join .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") + .set( + "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_grace_aggregate_spill_test", + "true") } override protected val createNullableTables = true diff --git a/cpp-ch/local-engine/Common/GlutenConfig.cpp b/cpp-ch/local-engine/Common/GlutenConfig.cpp index ce15a12f921a..75bb41f10d98 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.cpp +++ b/cpp-ch/local-engine/Common/GlutenConfig.cpp @@ -73,6 +73,7 @@ GraceMergingAggregateConfig GraceMergingAggregateConfig::loadFromContext(const D = context->getConfigRef().getUInt64(MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET, 1_MiB); config.max_allowed_memory_usage_ratio_for_aggregate_merging = context->getConfigRef().getDouble(MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING, 0.9); + config.enable_spill_test = context->getConfigRef().getBool(ENABLE_SPILL_TEST, false); return config; } diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h index 85839b70ecd2..07952383fa10 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.h +++ b/cpp-ch/local-engine/Common/GlutenConfig.h @@ -62,12 +62,14 @@ struct GraceMergingAggregateConfig = "max_pending_flush_blocks_per_grace_aggregate_merging_bucket"; inline static const String MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING = "max_allowed_memory_usage_ratio_for_aggregate_merging"; + inline static const String ENABLE_SPILL_TEST = "enable_grace_aggregate_spill_test"; size_t max_grace_aggregate_merging_buckets = 32; bool throw_on_overflow_grace_aggregate_merging_buckets = false; size_t aggregated_keys_before_extend_grace_aggregate_merging_buckets = 8192; size_t max_pending_flush_blocks_per_grace_aggregate_merging_bucket = 1_MiB; double max_allowed_memory_usage_ratio_for_aggregate_merging = 0.9; + bool enable_spill_test = false; static GraceMergingAggregateConfig loadFromContext(const DB::ContextPtr & context); }; diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp index 63dc3c3457f6..0dd1ac358466 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp @@ -56,7 +56,9 @@ GraceAggregatingTransform::GraceAggregatingTransform( max_allowed_memory_usage_ratio = config.max_allowed_memory_usage_ratio_for_aggregate_merging; // bucket 0 is for in-memory data, it's just a placeholder. buckets.emplace(0, BufferFileStream()); - + enable_spill_test = config.enable_spill_test; + if (enable_spill_test) + buckets.emplace(1, BufferFileStream()); current_data_variants = std::make_shared(); } @@ -289,7 +291,7 @@ void GraceAggregatingTransform::addBlockIntoFileBucket(size_t bucket_index, cons file_stream.original_blocks.push_back(block); else file_stream.intermediate_blocks.push_back(block); - if (file_stream.pending_bytes > max_pending_flush_blocks_per_bucket) + if (file_stream.pending_bytes > max_pending_flush_blocks_per_bucket || (file_stream.pending_bytes && enable_spill_test)) { flushBucket(bucket_index); file_stream.pending_bytes = 0; @@ -349,7 +351,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.intermediate_file_stream) { auto intermediate_header = params->aggregator.getHeader(false); - file_stream.intermediate_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk.get()); + file_stream.intermediate_file_stream = DB::TemporaryBlockStreamHolder(intermediate_header, tmp_data_disk.get()); } flush_bytes += flushBlocksInfoDisk(file_stream.intermediate_file_stream, file_stream.intermediate_blocks); } diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h index 612a58b3c943..c34d7714cfd4 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h @@ -116,6 +116,7 @@ class GraceAggregatingTransform : public DB::IProcessor DB::BlocksList current_final_blocks; std::unique_ptr block_converter = nullptr; bool no_more_keys = false; + bool enable_spill_test = false; double per_key_memory_usage = 0; From 504815fdaec5ee3779e68a00daa5f7e3f51741f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 29 Nov 2024 17:41:53 +0800 Subject: [PATCH 169/211] [GLUTEN-8021][CH] Fix ORC read/write mismatch and parquet read failure when column with complex types contains null (#8023) * fix issue https://github.com/apache/incubator-gluten/issues/8022 and https://github.com/apache/incubator-gluten/issues/8021 * fix bugs * fix issue https://github.com/apache/incubator-gluten/issues/8032 * update ch version * fix failed uts * fix failure in spark3.5 * adjust style --- ...lutenClickHouseNativeWriteTableSuite.scala | 61 ++++++++- cpp-ch/clickhouse.version | 2 +- cpp-ch/local-engine/Common/CHUtil.cpp | 1 + .../Storages/Output/NormalFileWriter.cpp | 123 ++++++++++++++++++ 4 files changed, 185 insertions(+), 2 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala index d0d03eaf1edb..03d27f33b1e1 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala @@ -903,7 +903,7 @@ class GlutenClickHouseNativeWriteTableSuite } test("GLUTEN-2584: fix native write and read mismatch about complex types") { - def table(format: String): String = s"t_$format" + def table(format: String): String = s"t_2584_$format" def create(format: String, table_name: Option[String] = None): String = s"""CREATE TABLE ${table_name.getOrElse(table(format))}( | id INT, @@ -934,4 +934,63 @@ class GlutenClickHouseNativeWriteTableSuite } ) } + + test( + "GLUTEN-8021/8022/8032: fix orc read/write mismatch and parquet" + + "read exception when written complex column contains null") { + def table(format: String): String = s"t_8021_$format" + def create(format: String, table_name: Option[String] = None): String = + s"""CREATE TABLE ${table_name.getOrElse(table(format))}( + |id int, + |x int, + |y int, + |mp map, + |arr array, + |tup struct, + |arr_mp array>, + |mp_arr map>, + |tup_arr struct>, + |tup_map struct> + |) stored as $format""".stripMargin + def insert(format: String, table_name: Option[String] = None): String = + s"""INSERT OVERWRITE TABLE ${table_name.getOrElse(table(format))} + |SELECT + | id, x, y, + | str_to_map(concat('x:', x, ',y:', y)) AS mp, + | IF(id % 4 = 0, NULL, array(x, y)) AS arr, + | IF(id % 4 = 1, NULL, struct(x, y)) AS tup, + | IF(id % 4 = 2, NULL, array(str_to_map(concat('x:', x, ',y:', y)))) AS arr_mp, + | IF(id % 4 = 3, NULL, map('x', array(x), 'y', array(y))) AS mp_arr, + | IF(id % 4 = 0, NULL, named_struct('a', array(x, y))) AS tup_arr, + | IF(id % 4 = 1, NULL, named_struct('m', + | str_to_map(concat('x:', x, ',y:', y)))) AS tup_map + |FROM ( + | SELECT + | id, + | IF(id % 3 = 1, NULL, id + 1) AS x, + | IF(id % 3 = 1, NULL, id + 2) AS y + | FROM range(100) + |) AS data_source;""".stripMargin + + // TODO fix it in spark3.5 + if (!isSparkVersionGE("3.5")) { + nativeWrite2( + format => (table(format), create(format), insert(format)), + (table_name, format) => { + val vanilla_table = s"${table_name}_v" + val vanilla_create = create(format, Some(vanilla_table)) + vanillaWrite { + withDestinationTable(vanilla_table, Option(vanilla_create)) { + checkInsertQuery(insert(format, Some(vanilla_table)), checkNative = false) + } + } + val rowsFromOriginTable = + spark.sql(s"select * from $vanilla_table").collect() + val dfFromWriteTable = + spark.sql(s"select * from $table_name") + checkAnswer(dfFromWriteTable, rowsFromOriginTable) + } + ) + } + } } diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 62a70f06c1e1..003f11113345 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence CH_BRANCH=rebase_ch/20241118 -CH_COMMIT=7f22fe487c88d3b988ea82a5c34882da23ea6289 +CH_COMMIT=a5944dfb7b3 diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index de1a26321801..88c5303c50e4 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -710,6 +710,7 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ settings.set("max_parsing_threads", 1); settings.set("max_download_threads", 1); settings.set("input_format_parquet_enable_row_group_prefetch", false); + settings.set("output_format_parquet_use_custom_encoder", false); /// update per https://github.com/ClickHouse/ClickHouse/pull/71539 /// if true, we can't get correct metrics for the query diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index e9fd4f358a86..e5a2d89f26c8 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -19,13 +19,134 @@ #include #include #include +#include +#include +#include namespace local_engine { +using namespace DB; + const std::string SubstraitFileSink::NO_PARTITION_ID{"__NO_PARTITION_ID__"}; const std::string SparkPartitionedBaseSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; +/// For Nullable(Map(K, V)) or Nullable(Array(T)), if the i-th row is null, we must make sure its nested data is empty. +/// It is for ORC/Parquet writing compatiability. For more details, refer to +/// https://github.com/apache/incubator-gluten/issues/8022 and https://github.com/apache/incubator-gluten/issues/8021 +static ColumnPtr truncateNestedDataIfNull(const ColumnPtr & column) +{ + if (const auto * col_const = checkAndGetColumn(column.get())) + { + size_t s = col_const->size(); + auto new_data = truncateNestedDataIfNull(col_const->getDataColumnPtr()); + return ColumnConst::create(std::move(new_data), s); + } + else if (const auto * col_array = checkAndGetColumn(column.get())) + { + auto new_data = truncateNestedDataIfNull(col_array->getDataPtr()); + return ColumnArray::create(std::move(new_data), col_array->getOffsetsPtr()); + } + else if (const auto * col_map = checkAndGetColumn(column.get())) + { + auto new_nested = truncateNestedDataIfNull(col_map->getNestedColumnPtr()); + return ColumnMap::create(std::move(new_nested)); + } + else if (const auto * col_tuple = checkAndGetColumn(column.get())) + { + Columns new_columns; + for (size_t i = 0; i < col_tuple->tupleSize(); ++i) + new_columns.emplace_back(truncateNestedDataIfNull(col_tuple->getColumnPtr(i))); + return ColumnTuple::create(std::move(new_columns)); + } + else if (const auto * col_nullable = checkAndGetColumn(column.get())) + { + const auto & null_map = col_nullable->getNullMapData(); + auto nested = truncateNestedDataIfNull(col_nullable->getNestedColumnPtr()); + const auto * nested_array = checkAndGetColumn(nested.get()); + const auto * nested_map = checkAndGetColumn(nested.get()); + const auto * nested_tuple = checkAndGetColumn(nested.get()); + + if (!memoryIsZero(null_map.data(), 0, null_map.size()) && (nested_array || nested_map || nested_tuple)) + { + /// Process Nullable(Array) or Nullable(Map) + if (nested_array || nested_map) + { + if (!nested_array) + nested_array = checkAndGetColumn(&nested_map->getNestedColumn()); + + const auto & offsets = nested_array->getOffsets(); + size_t total_data_size = 0; + for (size_t i = 0; i < null_map.size(); ++i) + total_data_size += (offsets[i] - offsets[i - 1]) * (!null_map[i]); + + auto new_nested_array = nested_array->cloneEmpty(); + new_nested_array->reserve(nested_array->size()); + auto & new_nested_array_data = assert_cast(*new_nested_array).getData(); + new_nested_array_data.reserve(total_data_size); + + for (size_t i = 0; i < null_map.size(); ++i) + if (null_map[i]) + new_nested_array->insertDefault(); + else + new_nested_array->insertFrom(*nested_array, i); + + if (nested_map) + { + auto new_nested_map = ColumnMap::create(std::move(new_nested_array)); + return ColumnNullable::create(std::move(new_nested_map), col_nullable->getNullMapColumnPtr()); + } + else + { + return ColumnNullable::create(std::move(new_nested_array), col_nullable->getNullMapColumnPtr()); + } + } + else + { + /// Process Nullable(Tuple) + const auto & nested_columns = nested_tuple->getColumns(); + Columns new_nested_columns(nested_columns.size()); + for (size_t i = 0; i < nested_columns.size(); ++i) + { + const auto & nested_column = nested_columns[i]; + TypeIndex type_index = nested_column->getDataType(); + if (const auto * nullable_nested_column = checkAndGetColumn(nested_column.get())) + type_index = nullable_nested_column->getNestedColumnPtr()->getDataType(); + + bool should_truncate = type_index == TypeIndex::Array || type_index == TypeIndex::Map || type_index == TypeIndex::Tuple; + if (should_truncate) + { + auto new_nested_column = nested_column->cloneEmpty(); + new_nested_column->reserve(nested_column->size()); + for (size_t j = 0; j < null_map.size(); ++j) + { + if (null_map[j]) + new_nested_column->insertDefault(); + else + new_nested_column->insertFrom(*nested_column, j); + } + new_nested_columns[i] = std::move(new_nested_column); + } + else + { + new_nested_columns[i] = nested_column; + } + } + + auto new_nested_tuple = ColumnTuple::create(std::move(new_nested_columns)); + return ColumnNullable::create(std::move(new_nested_tuple), col_nullable->getNullMapColumnPtr()); + } + } + else + { + auto new_nested = truncateNestedDataIfNull(nested); + return ColumnNullable::create(std::move(new_nested), col_nullable->getNullMapColumnPtr()); + } + } + else + return column; +} + NormalFileWriter::NormalFileWriter(const OutputFormatFilePtr & file_, const DB::ContextPtr & context_) : file(file_), context(context_) { } @@ -51,6 +172,8 @@ void NormalFileWriter::write(DB::Block & block) continue; const auto & preferred_column = preferred_schema.getByPosition(index++); + /// Make sure nested array or map data is empty when the row is null in Nullable(Map(K, V)) or Nullable(Array(T)). + column.column = truncateNestedDataIfNull(column.column); column.column = DB::castColumn(column, preferred_column.type); column.name = preferred_column.name; column.type = preferred_column.type; From b6e7b87399d9d27c7165fd2d789de79866014d8f Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Fri, 29 Nov 2024 18:09:56 +0800 Subject: [PATCH 170/211] [GLUTEN-8095][CH] package with iceberg profile (#8106) [CH] package with iceberg profile --- ep/build-clickhouse/src/package.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-clickhouse/src/package.sh b/ep/build-clickhouse/src/package.sh index 0d544a983fee..2583727b212e 100755 --- a/ep/build-clickhouse/src/package.sh +++ b/ep/build-clickhouse/src/package.sh @@ -90,7 +90,7 @@ function build_gluten_by_spark_version() { sv=$(echo "$spark_profile" | tr -d '.') echo "build gluten with spark ${spark_profile}, scala ${scala_version}" - mvn clean install -Pbackends-clickhouse -Pspark-"${spark_profile}" -Pscala-"${scala_version}" -Pceleborn -DskipTests -Dcheckstyle.skip + mvn clean install -Pbackends-clickhouse -Pspark-"${spark_profile}" -Pscala-"${scala_version}" -Pceleborn -Piceberg -DskipTests -Dcheckstyle.skip cp "${GLUTEN_SOURCE}"/backends-clickhouse/target/gluten-*-spark-"${spark_profile}"-jar-with-dependencies.jar "${PACKAGE_DIR_PATH}"/jars/spark"${sv}"/gluten.jar cp "${GLUTEN_SOURCE}"/gluten-celeborn/clickhouse/target/gluten-celeborn-clickhouse-"${PROJECT_VERSION}"-jar-with-dependencies.jar "${PACKAGE_DIR_PATH}"/jars/spark"${sv}" delta_version=$(mvn -q -Dexec.executable="echo" -Dexec.args='${delta.version}' -Pspark-"${spark_profile}" --non-recursive exec:exec) From 4cafdf1d2746402e6df1293474d3756553c71505 Mon Sep 17 00:00:00 2001 From: Ruilei Ma Date: Fri, 29 Nov 2024 20:16:49 +0800 Subject: [PATCH 171/211] [GLUTEN-8103][DOC] Fix TPC-H/DS queries link (#8104) --- tools/workload/tpcds/README.md | 2 +- tools/workload/tpch/README.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tools/workload/tpcds/README.md b/tools/workload/tpcds/README.md index efe3083d0cfe..8091054a0fb5 100644 --- a/tools/workload/tpcds/README.md +++ b/tools/workload/tpcds/README.md @@ -29,5 +29,5 @@ val dbgenDir = "/PATH/TO/TPCDS_DBGEN" // location of dbgen Currently, Gluten with Velox can support Parquet file format and three compression codec including snappy, gzip, zstd. ## Test Queries -We provide the test queries in [TPC-DS Queries](../../../gluten-core/src/test/resources/tpcds-queries). +We provide the test queries in [TPC-DS Queries](../../../tools/gluten-it/common/src/main/resources/tpcds-queries). We also provide a Scala script in [Run TPC-DS](./run_tpcds) directory about how to run TPC-DS queries. diff --git a/tools/workload/tpch/README.md b/tools/workload/tpch/README.md index 65a7deb09f99..4180df60f8b7 100644 --- a/tools/workload/tpch/README.md +++ b/tools/workload/tpch/README.md @@ -48,6 +48,6 @@ val dwrf_file_path = "/PATH/TO/TPCH_DWRF_PATH" ``` ## Test Queries -We provide the test queries in [TPC-H queries](../../../gluten-core/src/test/resources/tpch-queries). +We provide the test queries in [TPC-H queries](../../../tools/gluten-it/common/src/main/resources/tpch-queries). We also provide a scala script in [Run TPC-H](./run_tpch/) directory about how to run TPC-H queries. Please note if you are using DWRF test, please remember to set the file format to DWRF in the code. From b2706cba5a9efa95d249a32523bd03eb3b3290fe Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Fri, 29 Nov 2024 18:56:01 -0600 Subject: [PATCH 172/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241129) (#8087) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241129) * Fix Build AND UT Due to [Added cache for primary index](https://github.com/ClickHouse/ClickHouse/pull/72102) * Fix Build and UT due to [no auto write buffer finalization in destructors](https://github.com/ClickHouse/ClickHouse/pull/68800) - Make LocalPartitionWriter::evictPartitions called, e.g. set(GlutenConfig.COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD.key, (1024*1024).toString) * Fix Build due to [Save several minutes of build time](https://github.com/ClickHouse/ClickHouse/pull/72046) * Fix Benchmark Build due to [Scatter blocks in hash join without copying](https://github.com/ClickHouse/ClickHouse/pull/67782) (cherry picked from commit 8d566d6a8b8785e4072ffd6f774eb83b07ac3d8d) * Fix Benchmark Build * Fix endless loop due to https://github.com/ClickHouse/ClickHouse/pull/70598 * [Refactor #8100] using CHConf.setCHConfig() * fix style --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- .../GlutenClickHouseTPCHBucketSuite.scala | 5 ++-- .../GlutenClickHouseTPCDSParquetSuite.scala | 5 ++-- ...seTPCHColumnarShuffleParquetAQESuite.scala | 1 + ...utenClickHouseTPCHParquetBucketSuite.scala | 5 ++-- cpp-ch/clickhouse.version | 4 +-- cpp-ch/local-engine/Common/CHUtil.cpp | 26 ++++++++++++++++--- .../Common/GlutenSignalHandler.cpp | 4 +-- .../Functions/FunctionGreatestLeast.h | 9 ++++--- .../Functions/SparkFunctionArrayJoin.cpp | 13 ++++++---- .../Functions/SparkFunctionArraysOverlap.cpp | 11 ++++---- .../SparkFunctionCheckDecimalOverflow.cpp | 1 + .../SparkFunctionDecimalBinaryArithmetic.cpp | 3 +-- .../Functions/SparkFunctionMakeDecimal.cpp | 4 +-- .../Functions/SparkFunctionMapToString.h | 7 ++--- .../Functions/SparkFunctionSplitByRegexp.cpp | 2 +- .../Parser/SerializedPlanParser.cpp | 11 +++++--- .../scalar_function_parser/arrayExcept.cpp | 1 + .../scalar_function_parser/arrayRemove.cpp | 2 +- .../scalar_function_parser/arrayRepeat.cpp | 3 ++- .../scalar_function_parser/bitLength.cpp | 5 ++-- .../scalar_function_parser/elementAt.cpp | 2 ++ .../Parser/scalar_function_parser/length.cpp | 5 ++-- .../Parser/scalar_function_parser/locate.cpp | 3 ++- .../scalar_function_parser/octetLength.cpp | 5 ++-- .../local-engine/Shuffle/PartitionWriter.cpp | 6 +++++ .../local-engine/Shuffle/SelectorBuilder.cpp | 1 + cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp | 17 +++++------- cpp-ch/local-engine/Shuffle/ShuffleWriter.h | 2 +- .../Shuffle/SparkExchangeSink.cpp | 20 +++++++------- .../MergeTree/MergeSparkMergeTreeTask.cpp | 8 +++++- .../MergeTree/MergeSparkMergeTreeTask.h | 3 ++- .../Storages/MergeTree/MetaDataHelper.cpp | 1 + .../MergeTree/SparkStorageMergeTree.cpp | 1 - .../Storages/Output/NormalFileWriter.cpp | 4 +++ .../Storages/Output/NormalFileWriter.h | 4 +-- .../Storages/Output/OutputFormatFile.h | 7 ++++- .../tests/benchmark_cast_float_function.cpp | 4 +-- .../tests/benchmark_local_engine.cpp | 2 +- .../tests/benchmark_spark_floor_function.cpp | 8 +++--- .../tests/benchmark_to_datetime_function.cpp | 4 +-- .../benchmark_unix_timestamp_function.cpp | 8 +++--- cpp-ch/local-engine/tests/gluten_test_util.h | 2 +- .../local-engine/tests/gtest_ch_functions.cpp | 10 +++---- .../local-engine/tests/gtest_local_engine.cpp | 1 + 44 files changed, 151 insertions(+), 99 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala index 7bccb6dfb598..90e09e75f1ff 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala @@ -40,6 +40,7 @@ class GlutenClickHouseTPCHBucketSuite override protected val queriesResults: String = rootPath + "bucket-queries-output" override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") @@ -47,9 +48,7 @@ class GlutenClickHouseTPCHBucketSuite .set("spark.sql.autoBroadcastJoinThreshold", "-1") // for test bucket join .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_grace_aggregate_spill_test", - "true") + .setCHConfig("enable_grace_aggregate_spill_test", "true") } override protected val createNullableTables = true diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala index 0ba7de90c670..f0025cf30cad 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala @@ -30,6 +30,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ super.sparkConf .set("spark.shuffle.manager", "sort") .set("spark.io.compression.codec", "snappy") @@ -38,9 +39,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui .set("spark.memory.offHeap.size", "4g") .set("spark.gluten.sql.validation.logLevel", "ERROR") .set("spark.gluten.sql.validation.printStackOnFailure", "true") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_grace_aggregate_spill_test", - "true") + .setCHConfig("enable_grace_aggregate_spill_test", "true") } executeTPCDSTest(false) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index ad9cb854d97b..b4186fee66aa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -51,6 +51,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") .setCHConfig("enable_streaming_aggregating", true) + .set(GlutenConfig.COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD.key, (1024 * 1024).toString) } override protected def createTPCHNotNullTables(): Unit = { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala index a257e2ed5094..7a927bf23a49 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala @@ -46,6 +46,7 @@ class GlutenClickHouseTPCHParquetBucketSuite protected val bucketTableDataPath: String = basePath + "/tpch-parquet-bucket" override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") @@ -53,9 +54,7 @@ class GlutenClickHouseTPCHParquetBucketSuite .set("spark.sql.autoBroadcastJoinThreshold", "-1") // for test bucket join .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") - .set( - "spark.gluten.sql.columnar.backend.ch.runtime_config.enable_grace_aggregate_spill_test", - "true") + .setCHConfig("enable_grace_aggregate_spill_test", "true") } override protected val createNullableTables = true diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 003f11113345..edb13fdc5715 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241118 -CH_COMMIT=a5944dfb7b3 +CH_BRANCH=rebase_ch/20241129 +CH_COMMIT=101ba3f944d1 diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 88c5303c50e4..310b39d3e594 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -76,6 +76,12 @@ namespace DB { +namespace ServerSetting +{ +extern const ServerSettingsString primary_index_cache_policy; +extern const ServerSettingsUInt64 primary_index_cache_size; +extern const ServerSettingsDouble primary_index_cache_size_ratio; +} namespace Setting { extern const SettingsUInt64 prefer_external_sort_block_bytes; @@ -712,11 +718,13 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ settings.set("input_format_parquet_enable_row_group_prefetch", false); settings.set("output_format_parquet_use_custom_encoder", false); - /// update per https://github.com/ClickHouse/ClickHouse/pull/71539 + /// Set false after https://github.com/ClickHouse/ClickHouse/pull/71539 /// if true, we can't get correct metrics for the query settings[Setting::query_plan_merge_filters] = false; + /// We now set BuildQueryPipelineSettings according to config. - settings[Setting::compile_expressions] = true; + // TODO: FIXME. Set false after https://github.com/ClickHouse/ClickHouse/pull/70598. + settings[Setting::compile_expressions] = false; settings[Setting::short_circuit_function_evaluation] = ShortCircuitFunctionEvaluation::DISABLE; /// @@ -820,6 +828,10 @@ void BackendInitializerUtil::initContexts(DB::Context::ConfigurationPtr config) /// Make sure global_context and shared_context are constructed only once. if (auto global_context = QueryContext::globalMutableContext(); !global_context) { + ServerSettings server_settings; + server_settings.loadSettingsFromConfig(*config); + + auto log = getLogger("CHUtil"); global_context = QueryContext::createGlobal(); global_context->makeGlobalContext(); global_context->setConfig(config); @@ -844,10 +856,16 @@ void BackendInitializerUtil::initContexts(DB::Context::ConfigurationPtr config) size_t mark_cache_size = config->getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); double mark_cache_size_ratio = config->getDouble("mark_cache_size_ratio", DEFAULT_MARK_CACHE_SIZE_RATIO); if (!mark_cache_size) - LOG_ERROR(&Poco::Logger::get("CHUtil"), "Too low mark cache size will lead to severe performance degradation."); - + LOG_ERROR(log, "Mark cache is disabled, it will lead to severe performance degradation."); + LOG_INFO(log, "mark cache size to {}.", formatReadableSizeWithBinarySuffix(mark_cache_size)); global_context->setMarkCache(mark_cache_policy, mark_cache_size, mark_cache_size_ratio); + String primary_index_cache_policy = server_settings[ServerSetting::primary_index_cache_policy]; + size_t primary_index_cache_size = server_settings[ServerSetting::primary_index_cache_size]; + double primary_index_cache_size_ratio = server_settings[ServerSetting::primary_index_cache_size_ratio]; + LOG_INFO(log, "Primary index cache size to {}.", formatReadableSizeWithBinarySuffix(primary_index_cache_size)); + global_context->setPrimaryIndexCache(primary_index_cache_policy, primary_index_cache_size, primary_index_cache_size_ratio); + String index_uncompressed_cache_policy = config->getString("index_uncompressed_cache_policy", DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY); size_t index_uncompressed_cache_size diff --git a/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp b/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp index 44c43fcb65aa..712d8ddcf5cc 100644 --- a/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp +++ b/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp @@ -104,7 +104,7 @@ static void writeSignalIDtoSignalPipe(int sig) char buf[signal_pipe_buf_size]; WriteBufferFromFileDescriptor out(writeFD(), signal_pipe_buf_size, buf); writeBinary(sig, out); - out.next(); + out.finalize(); errno = saved_errno; } @@ -251,9 +251,7 @@ class SignalListener : public Poco::Runnable query = thread_ptr->getQueryForLog(); if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) - { CurrentThread::attachInternalTextLogsQueue(logs_queue, LogsLevel::trace); - } } std::string signal_description = "Unknown signal"; diff --git a/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h b/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h index 6930c1d75b79..e9b66df84ef0 100644 --- a/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h +++ b/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h @@ -14,9 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include +#pragma once #include +#include +#include +#include +#include namespace DB { @@ -64,7 +67,7 @@ class FunctionGreatestestLeast : public DB::FunctionLeastGreatestGeneric else { auto cmp_result = converted_columns[arg]->compareAt(row_num, row_num, *converted_columns[best_arg], 1); - if (cmp_result < 0) + if (cmp_result < 0) best_arg = arg; } } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp index 4c2847d9f92a..bf65b253479b 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp @@ -14,13 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include -#include +#include +#include #include -#include -#include +#include +#include #include +#include +#include +#include +#include using namespace DB; diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp index e43b52823175..ea841632a984 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include +#include #include -#include -#include -#include +#include +#include #include +#include +#include using namespace DB; @@ -92,7 +93,7 @@ class SparkFunctionArraysOverlap : public IFunction { res_data[i] = 1; null_map_data[i] = 0; - break; + break; } } } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp index c75d25b6ef80..8b5a7eff65db 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp index 830fc0e65287..f89943fc7a3d 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -26,8 +27,6 @@ #include #include #include -#include -#include #include namespace DB diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp index 795e2b0be329..f136f587c539 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp @@ -15,12 +15,12 @@ * limitations under the License. */ #include +#include #include #include #include #include -#include "SparkFunctionCheckDecimalOverflow.h" - +#include namespace DB { diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h index 3f8a0c97dc07..5541245244a7 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h @@ -14,17 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - +#pragma once #include +#include #include #include -#include #include #include -#include #include +#include #include #include +#include #include namespace DB diff --git a/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp b/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp index 66f37c62033f..1868c40c0f31 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp @@ -20,8 +20,8 @@ #include #include #include +#include #include -#include #include #include diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 95086121d4cb..820a99ad3bfb 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -160,15 +161,17 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con else { need_final_project = true; - bool need_const = origin_column.column && isColumnConst(*origin_column.column); - if (need_const) + if (origin_column.column && isColumnConst(*origin_column.column)) { + /// For const column, we need to cast it individually. Otherwise, the const column will be converted to full column in + /// ActionsDAG::makeConvertingActions. + /// Note: creating fianl_column with Field of origin_column will cause Exception in some case. const DB::ContextPtr context = DB::CurrentThread::get().getQueryContext(); const FunctionOverloadResolverPtr & cast_resolver = FunctionFactory::instance().get("CAST", context); const DataTypePtr string_type = std::make_shared(); ColumnWithTypeAndName to_type_column = {string_type->createColumnConst(1, final_type->getName()), string_type, "__cast_const__"}; FunctionBasePtr cast_function = cast_resolver->build({origin_column, to_type_column}); - ColumnPtr const_col = ColumnConst::create(cast_function->execute({origin_column, to_type_column}, final_type, 1), 1); + ColumnPtr const_col = ColumnConst::create(cast_function->execute({origin_column, to_type_column}, final_type, 1, false), 1); ColumnWithTypeAndName final_column(const_col, final_type, origin_column.name); final_columns.emplace_back(std::move(final_column)); } @@ -310,7 +313,7 @@ DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPl BuildQueryPipelineSettings build_settings = BuildQueryPipelineSettings::fromContext(context); build_settings.process_list_element = query_status; build_settings.progress_callback = nullptr; - return query_plan.buildQueryPipeline(optimization_settings,build_settings); + return query_plan.buildQueryPipeline(optimization_settings, build_settings); } std::unique_ptr SerializedPlanParser::createExecutor(const std::string_view plan) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp index 8857fbbf5df6..a9a0f305a0ac 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp @@ -15,6 +15,7 @@ * limitations under the License. */ #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp index f45544cfa4d1..27bd9f84a9e6 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp @@ -15,11 +15,11 @@ * limitations under the License. */ #include +#include #include #include #include #include - namespace DB { namespace ErrorCodes diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp index 581fd6f66589..04f4f64e7bd2 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp @@ -16,6 +16,7 @@ * limitations under the License. */ #include +#include #include #include #include @@ -56,7 +57,7 @@ class FunctionParserArrayRepeat : public FunctionParser const auto * const_zero_node = addColumnToActionsDAG(actions_dag, n_not_null_arg->result_type, {0}); const auto * greatest_node = toFunctionNode(actions_dag, "greatest", {n_not_null_arg, const_zero_node}); const auto * range_node = toFunctionNode(actions_dag, "range", {greatest_node}); - const auto & range_type = assert_cast(*removeNullable(range_node->result_type)); + const auto & range_type = assert_cast(*removeNullable(range_node->result_type)); // Create lambda function x -> elem ActionsDAG lambda_actions_dag; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp index ebd89f8fa8e8..d663052a8ea7 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp @@ -15,8 +15,9 @@ * limitations under the License. */ -#include +#include #include +#include namespace DB { @@ -57,7 +58,7 @@ class FunctionParserBitLength : public FunctionParser const auto * const_eight_node = addColumnToActionsDAG(actions_dag, std::make_shared(), 8); const auto * result_node = toFunctionNode(actions_dag, "multiply", {octet_length_node, const_eight_node}); - return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag);; + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp index 1586870541b6..80797dedcab5 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp @@ -16,7 +16,9 @@ */ #include +#include #include +#include #include namespace DB diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp index 34a7348b9ac6..b4e4ad119fdc 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp @@ -15,9 +15,10 @@ * limitations under the License. */ -#include +#include #include #include +#include namespace DB { @@ -71,7 +72,7 @@ class FunctionParserLength : public FunctionParser else result_node = toFunctionNode(actions_dag, "char_length", {new_arg}); - return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag);; + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp index c4f0234957af..a9ad4d9aee4a 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp @@ -15,10 +15,11 @@ * limitations under the License. */ +#include +#include #include #include #include -#include namespace DB { diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp index 1dd61587e011..d57d8d661b50 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp @@ -15,8 +15,9 @@ * limitations under the License. */ -#include +#include #include +#include namespace DB { @@ -52,7 +53,7 @@ class FunctionParserOctetLength : public FunctionParser new_arg = toFunctionNode(actions_dag, "CAST", {arg, string_type_node}); } const auto * octet_length_node = toFunctionNode(actions_dag, "octet_length", {new_arg}); - return convertNodeTypeIfNeeded(substrait_func, octet_length_node, actions_dag);; + return convertNodeTypeIfNeeded(substrait_func, octet_length_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index aa9e15b9f036..43459f20c5b3 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -172,6 +172,8 @@ size_t LocalPartitionWriter::evictPartitions() split_result->total_compress_time += compressed_output.getCompressTime(); split_result->total_write_time += compressed_output.getWriteTime(); split_result->total_serialize_time += serialization_time_watch.elapsedNanoseconds(); + compressed_output.finalize(); + output.finalize(); }; Stopwatch spill_time_watch; @@ -342,6 +344,8 @@ size_t MemorySortLocalPartitionWriter::evictPartitions() split_result->total_compress_time += compressed_output.getCompressTime(); split_result->total_io_time += compressed_output.getWriteTime(); split_result->total_serialize_time += serialization_time_watch.elapsedNanoseconds(); + compressed_output.finalize(); + output.finalize(); }; Stopwatch spill_time_watch; @@ -428,6 +432,8 @@ size_t MemorySortCelebornPartitionWriter::evictPartitions() split_result->total_compress_time += compressed_output.getCompressTime(); split_result->total_io_time += compressed_output.getWriteTime(); split_result->total_serialize_time += serialization_time_watch.elapsedNanoseconds(); + compressed_output.finalize(); + output.finalize(); }; Stopwatch spill_time_watch; diff --git a/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp b/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp index 02baa4a9c09c..ab4cfc18c89d 100644 --- a/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp +++ b/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp @@ -17,6 +17,7 @@ #include "SelectorBuilder.h" #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp b/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp index 8aa624ff9979..7167dabfad55 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp @@ -53,23 +53,20 @@ void ShuffleWriter::write(const Block & block) native_writer->write(block); } } -void ShuffleWriter::flush() +void ShuffleWriter::flush() const { if (native_writer) - { native_writer->flush(); - } } + ShuffleWriter::~ShuffleWriter() { if (native_writer) - { native_writer->flush(); - if (compression_enable) - { - compressed_out->finalize(); - } - write_buffer->finalize(); - } + + if (compression_enable) + compressed_out->finalize(); + + write_buffer->finalize(); } } diff --git a/cpp-ch/local-engine/Shuffle/ShuffleWriter.h b/cpp-ch/local-engine/Shuffle/ShuffleWriter.h index 541e93e0347c..94886210c1d2 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleWriter.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleWriter.h @@ -27,7 +27,7 @@ class ShuffleWriter jobject output_stream, jbyteArray buffer, const std::string & codecStr, jint level, bool enable_compression, size_t customize_buffer_size); virtual ~ShuffleWriter(); void write(const DB::Block & block); - void flush(); + void flush() const; private: std::unique_ptr compressed_out; diff --git a/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp b/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp index a78d615be62b..c40b474e7a8b 100644 --- a/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp +++ b/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp @@ -16,15 +16,15 @@ */ #include "SparkExchangeSink.h" +#include #include +#include #include -#include +#include +#include #include +#include #include -#include -#include -#include -#include namespace DB @@ -74,7 +74,7 @@ void SparkExchangeSink::consume(Chunk chunk) void SparkExchangeSink::onFinish() { Stopwatch wall_time; - if (!dynamic_cast(partition_writer.get())) + if (!dynamic_cast(partition_writer.get())) { partition_writer->evictPartitions(); } @@ -222,8 +222,7 @@ void SparkExchangeManager::finish() std::vector extra_datas; for (const auto & writer : partition_writers) { - LocalPartitionWriter * local_partition_writer = dynamic_cast(writer.get()); - if (local_partition_writer) + if (LocalPartitionWriter * local_partition_writer = dynamic_cast(writer.get())) { extra_datas.emplace_back(local_partition_writer->getExtraData()); } @@ -232,12 +231,13 @@ void SparkExchangeManager::finish() chassert(extra_datas.size() == partition_writers.size()); WriteBufferFromFile output(options.data_file, options.io_buffer_size); split_result.partition_lengths = mergeSpills(output, infos, extra_datas); + output.finalize(); } split_result.wall_time += wall_time.elapsedNanoseconds(); } -void checkPartitionLengths(const std::vector & partition_length,size_t partition_num) +void checkPartitionLengths(const std::vector & partition_length, size_t partition_num) { if (partition_num != partition_length.size()) { @@ -284,7 +284,7 @@ void SparkExchangeManager::mergeSplitResult() std::vector SparkExchangeManager::gatherAllSpillInfo() const { std::vector res; - for (const auto& writer : partition_writers) + for (const auto & writer : partition_writers) { if (Spillable * spillable = dynamic_cast(writer.get())) { diff --git a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp index cecb6308745c..ee6930e4de51 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp @@ -15,11 +15,11 @@ * limitations under the License. */ #include "MergeSparkMergeTreeTask.h" -#include #include #include #include +#include #include #include #include @@ -94,6 +94,12 @@ bool MergeSparkMergeTreeTask::executeStep() } +void MergeSparkMergeTreeTask::cancel() noexcept +{ + if (merge_task) + merge_task->cancel(); +} + void MergeSparkMergeTreeTask::prepare() { future_part = merge_mutate_entry->future_part; diff --git a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h index ac167da3fb49..60b3328f0d1b 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h +++ b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h @@ -65,6 +65,7 @@ class MergeSparkMergeTreeTask : public IExecutableTask txn_holder = std::move(txn_holder_); txn = std::move(txn_); } + void cancel() noexcept override; private: void prepare(); @@ -116,7 +117,7 @@ class MergeSparkMergeTreeTask : public IExecutableTask using MergeSparkMergeTreeTaskPtr = std::shared_ptr; -[[ maybe_unused ]] static void executeHere(MergeSparkMergeTreeTaskPtr task) +[[maybe_unused]] static void executeHere(MergeSparkMergeTreeTaskPtr task) { while (task->executeStep()) {} } diff --git a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp index 958421022ba2..84dbc3a8d3bb 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp @@ -190,6 +190,7 @@ void restoreMetaData( auto item_path = part_path / item.first; auto out = metadata_disk->writeFile(item_path); out->write(item.second.data(), item.second.size()); + out->finalize(); } }; thread_pool.scheduleOrThrow(job); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 17587e5200ef..5669489f5477 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -504,7 +504,6 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( txn ? txn->tid : Tx::PrehistoricTID, false, false, - false, context->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index e5a2d89f26c8..ad2e3abf7b52 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -191,7 +191,11 @@ void NormalFileWriter::close() /// When insert into a table with empty dataset, NormalFileWriter::consume would be never called. /// So we need to skip when writer is nullptr. if (writer) + { writer->finish(); + assert(output_format); + output_format->finalizeOutput(); + } } OutputFormatFilePtr createOutputFormatFile( diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index d55703741845..8cfe079d92c5 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -293,9 +293,7 @@ class SubstraitFileSink final : public DB::SinkToStorage { if (output_format_) [[unlikely]] { - output_format_->output->finalize(); - output_format_->output->flush(); - output_format_->write_buffer->finalize(); + output_format_->finalizeOutput(); assert(delta_stats_.row_count > 0); if (stats_) stats_->collectStats(relative_path_, partition_id_, delta_stats_); diff --git a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h index e94923f77a43..915f9a7e7efa 100644 --- a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h @@ -29,9 +29,14 @@ class OutputFormatFile public: struct OutputFormat { - public: DB::OutputFormatPtr output; std::unique_ptr write_buffer; + void finalizeOutput() const + { + output->finalize(); + output->flush(); + write_buffer->finalize(); + } }; using OutputFormatPtr = std::shared_ptr; diff --git a/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp b/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp index 4ef9b5771af8..a50bcf170eff 100644 --- a/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp @@ -52,7 +52,7 @@ static void BM_CHCastFloatToInt(benchmark::State & state) args.emplace_back(type_name_col); auto executable = function->build(args); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_SparkCastFloatToInt(benchmark::State & state) @@ -63,7 +63,7 @@ static void BM_SparkCastFloatToInt(benchmark::State & state) Block block = createDataBlock(30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } BENCHMARK(BM_CHCastFloatToInt)->Unit(benchmark::kMillisecond)->Iterations(100); diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index 13e74abaee1f..eacfb1781b26 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -846,7 +846,7 @@ QueryPlanPtr joinPlan(QueryPlanPtr left, QueryPlanPtr right, String left_key, St auto hash_join = std::make_shared(join, right->getCurrentHeader()); QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 1, false); + = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 0, 1, false); std::vector plans; plans.emplace_back(std::move(left)); diff --git a/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp b/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp index ef961f21cbb6..a672fdee350a 100644 --- a/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp @@ -66,7 +66,7 @@ static void BM_CHFloorFunction_For_Int64(benchmark::State & state) auto executable = function->build(int64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows()); + auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows(), false); benchmark::DoNotOptimize(result); } } @@ -80,7 +80,7 @@ static void BM_CHFloorFunction_For_Float64(benchmark::State & state) auto executable = function->build(float64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows()); + auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows(), false); benchmark::DoNotOptimize(result); } } @@ -94,7 +94,7 @@ static void BM_SparkFloorFunction_For_Int64(benchmark::State & state) auto executable = function->build(int64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows()); + auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows(), false); benchmark::DoNotOptimize(result); } } @@ -108,7 +108,7 @@ static void BM_SparkFloorFunction_For_Float64(benchmark::State & state) auto executable = function->build(float64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows()); + auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows(), false); benchmark::DoNotOptimize(result); } } diff --git a/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp b/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp index c72125163351..49f9dde989e9 100644 --- a/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp @@ -45,7 +45,7 @@ static void BM_CHParseDateTime64(benchmark::State & state) Block block = createDataBlock(30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } @@ -57,7 +57,7 @@ static void BM_SparkParseDateTime64(benchmark::State & state) Block block = createDataBlock(30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } BENCHMARK(BM_CHParseDateTime64)->Unit(benchmark::kMillisecond)->Iterations(50); diff --git a/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp b/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp index e7abfda7a2b2..a7dc3ffa2b91 100644 --- a/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp @@ -49,7 +49,7 @@ static void BM_CHUnixTimestamp_For_Date32(benchmark::State & state) Block block = createDataBlock("Date32", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_CHUnixTimestamp_For_Date(benchmark::State & state) @@ -60,7 +60,7 @@ static void BM_CHUnixTimestamp_For_Date(benchmark::State & state) Block block = createDataBlock("Date", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_SparkUnixTimestamp_For_Date32(benchmark::State & state) @@ -71,7 +71,7 @@ static void BM_SparkUnixTimestamp_For_Date32(benchmark::State & state) Block block = createDataBlock("Date32", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_SparkUnixTimestamp_For_Date(benchmark::State & state) @@ -82,7 +82,7 @@ static void BM_SparkUnixTimestamp_For_Date(benchmark::State & state) Block block = createDataBlock("Date", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } BENCHMARK(BM_CHUnixTimestamp_For_Date32)->Unit(benchmark::kMillisecond)->Iterations(100); diff --git a/cpp-ch/local-engine/tests/gluten_test_util.h b/cpp-ch/local-engine/tests/gluten_test_util.h index 799a6d7967dc..a61612662961 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.h +++ b/cpp-ch/local-engine/tests/gluten_test_util.h @@ -16,9 +16,9 @@ */ #pragma once +#include "testConfig.h" #include -#include #include #include #include diff --git a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp index e905bc1787fa..3b91e0799404 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp @@ -47,7 +47,7 @@ TEST(TestFuntion, Hash) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); std::cerr << "output:\n"; debug::headColumn(result); ASSERT_EQ(result->getUInt(0), result->getUInt(1)); @@ -89,7 +89,7 @@ TEST(TestFunction, In) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); std::cerr << "output:\n"; debug::headColumn(result); ASSERT_EQ(result->getUInt(3), 0); @@ -133,7 +133,7 @@ TEST(TestFunction, NotIn1) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); std::cerr << "output:\n"; debug::headColumn(result); ASSERT_EQ(result->getUInt(3), 1); @@ -176,14 +176,14 @@ TEST(TestFunction, NotIn2) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); auto function_not = factory.get("not", local_engine::QueryContext::globalContext()); auto type_bool = DataTypeFactory::instance().get("UInt8"); ColumnsWithTypeAndName columns2 = {ColumnWithTypeAndName(result, type_bool, "string0")}; Block block2(columns2); auto executable2 = function_not->build(block2.getColumnsWithTypeAndName()); - auto result2 = executable2->execute(block2.getColumnsWithTypeAndName(), executable2->getResultType(), block2.rows()); + auto result2 = executable2->execute(block2.getColumnsWithTypeAndName(), executable2->getResultType(), block2.rows(), false); std::cerr << "output:\n"; debug::headColumn(result2); ASSERT_EQ(result2->getUInt(3), 1); diff --git a/cpp-ch/local-engine/tests/gtest_local_engine.cpp b/cpp-ch/local-engine/tests/gtest_local_engine.cpp index 5f9b6f280e58..06e94e051b86 100644 --- a/cpp-ch/local-engine/tests/gtest_local_engine.cpp +++ b/cpp-ch/local-engine/tests/gtest_local_engine.cpp @@ -67,6 +67,7 @@ TEST(ReadBufferFromFile, seekBackwards) WriteBufferFromFile out(tmp_file->path()); for (size_t i = 0; i < N; ++i) writeIntBinary(i, out); + out.finalize(); } ReadBufferFromFile in(tmp_file->path(), BUF_SIZE); From 920bd8cb7803fc82c2e49cca26ebeb482fcfc9f2 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Sat, 30 Nov 2024 17:29:11 +0800 Subject: [PATCH 173/211] [CH] Support separate debug symbols from so file #8083 What changes were proposed in this pull request? support separate debug symbols How was this patch tested? manual tests image (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- cpp-ch/local-engine/CMakeLists.txt | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/cpp-ch/local-engine/CMakeLists.txt b/cpp-ch/local-engine/CMakeLists.txt index 4392b55008bf..4b1c64363646 100644 --- a/cpp-ch/local-engine/CMakeLists.txt +++ b/cpp-ch/local-engine/CMakeLists.txt @@ -174,6 +174,26 @@ else() set(LOCALENGINE_SHARED_LIB_NAME "libch.so") endif() +option(ENABLE_SEPARATE_SYMBOLS "support separate debug symbols from so" OFF) +if(ENABLE_SEPARATE_SYMBOLS) + set(SYMBOL_OUTPUT_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/debug_symbols) + file(MAKE_DIRECTORY ${SYMBOL_OUTPUT_DIRECTORY}) + function(separate_symbols target) + add_custom_command( + TARGET ${target} + POST_BUILD + COMMAND ${CMAKE_OBJCOPY} --only-keep-debug $ + ${SYMBOL_OUTPUT_DIRECTORY}/$.debug + COMMAND ${CMAKE_OBJCOPY} --strip-debug $ + COMMAND + ${CMAKE_OBJCOPY} + --add-gnu-debuglink=${SYMBOL_OUTPUT_DIRECTORY}/$.debug + $ + COMMENT "Separating debug symbols for target: ${target}") + endfunction() + separate_symbols(${LOCALENGINE_SHARED_LIB}) +endif() + add_custom_command( OUTPUT ${LOCALENGINE_SHARED_LIB_NAME} COMMAND ${CMAKE_COMMAND} -E rename $ From 31e1b7449fba2ab6938d92f5cf013b2e82df5446 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Sat, 30 Nov 2024 07:34:46 -0600 Subject: [PATCH 174/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241130) (#8112) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241130) * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/71406 * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/72460 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 +- cpp-ch/local-engine/Common/AggregateUtil.cpp | 39 ++++++++++--------- cpp-ch/local-engine/Common/AggregateUtil.h | 2 +- .../Parser/RelParsers/CrossRelParser.cpp | 11 +++++- .../Parser/RelParsers/JoinRelParser.cpp | 18 +++++++-- cpp-ch/local-engine/tests/gtest_ch_join.cpp | 6 ++- 6 files changed, 52 insertions(+), 28 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index edb13fdc5715..565220d7867d 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241129 -CH_COMMIT=101ba3f944d1 +CH_BRANCH=rebase_ch/20241130 +CH_COMMIT=d5d38588bd3 diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp index 0707d18aa01b..4a768eb17b6c 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.cpp +++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp @@ -48,6 +48,9 @@ extern const SettingsUInt64 aggregation_in_order_max_block_bytes; extern const SettingsUInt64 group_by_two_level_threshold; extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; extern const SettingsUInt64 max_block_size; +extern const SettingsBool compile_aggregate_expressions; +extern const SettingsUInt64 min_count_to_compile_aggregate_expression; +extern const SettingsBool enable_software_prefetch_in_aggregation; } template @@ -186,7 +189,7 @@ DB::Block AggregateDataBlockConverter::next() } DB::Aggregator::Params AggregatorParamsHelper::buildParams( - DB::ContextPtr context, + const DB::ContextPtr & context, const DB::Names & grouping_keys, const DB::AggregateDescriptions & agg_descriptions, Mode mode, @@ -194,7 +197,7 @@ DB::Aggregator::Params AggregatorParamsHelper::buildParams( { const auto & settings = context->getSettingsRef(); size_t max_rows_to_group_by = mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::max_rows_to_group_by]); - DB::OverflowMode group_by_overflow_mode = settings[DB::Setting::group_by_overflow_mode]; + size_t group_by_two_level_threshold = algorithm == Algorithm::GlutenGraceAggregate ? static_cast(settings[DB::Setting::group_by_two_level_threshold]) : 0; size_t group_by_two_level_threshold_bytes = algorithm == Algorithm::GlutenGraceAggregate @@ -207,39 +210,39 @@ DB::Aggregator::Params AggregatorParamsHelper::buildParams( ? false : (mode == Mode::PARTIAL_TO_FINISHED ? false : static_cast(settings[DB::Setting::empty_result_for_aggregation_by_empty_set])); DB::TemporaryDataOnDiskScopePtr tmp_data_scope = algorithm == Algorithm::GlutenGraceAggregate ? nullptr : context->getTempDataOnDisk(); - size_t max_threads = settings[DB::Setting::max_threads]; + size_t min_free_disk_space = algorithm == Algorithm::GlutenGraceAggregate ? 0 : static_cast(settings[DB::Setting::min_free_disk_space_for_temporary_data]); - bool compile_aggregate_expressions = mode == Mode::PARTIAL_TO_FINISHED ? false : true; - size_t min_count_to_compile_aggregate_expression = mode == Mode::PARTIAL_TO_FINISHED ? 0 : 3; + bool compile_aggregate_expressions = mode == Mode::PARTIAL_TO_FINISHED ? false : settings[DB::Setting::compile_aggregate_expressions]; + size_t min_count_to_compile_aggregate_expression = mode == Mode::PARTIAL_TO_FINISHED ? 0 : settings[DB::Setting::min_count_to_compile_aggregate_expression]; size_t max_block_size = PODArrayUtil::adjustMemoryEfficientSize(settings[DB::Setting::max_block_size]); - bool enable_prefetch = mode == Mode::PARTIAL_TO_FINISHED ? false : true; + bool enable_prefetch = mode != Mode::PARTIAL_TO_FINISHED; bool only_merge = mode == Mode::PARTIAL_TO_FINISHED; bool optimize_group_by_constant_keys = mode == Mode::PARTIAL_TO_FINISHED ? false : settings[DB::Setting::optimize_group_by_constant_keys]; - double min_hit_rate_to_use_consecutive_keys_optimization = settings[DB::Setting::min_hit_rate_to_use_consecutive_keys_optimization]; + + DB::Settings aggregate_settings{settings}; + aggregate_settings[DB::Setting::max_rows_to_group_by] = max_rows_to_group_by; + aggregate_settings[DB::Setting::max_bytes_before_external_group_by] = max_bytes_before_external_group_by; + aggregate_settings[DB::Setting::min_free_disk_space_for_temporary_data] = min_free_disk_space; + aggregate_settings[DB::Setting::compile_aggregate_expressions] = compile_aggregate_expressions; + aggregate_settings[DB::Setting::min_count_to_compile_aggregate_expression] = min_count_to_compile_aggregate_expression; + aggregate_settings[DB::Setting::max_block_size] = max_block_size; + aggregate_settings[DB::Setting::enable_software_prefetch_in_aggregation] = enable_prefetch; + aggregate_settings[DB::Setting::optimize_group_by_constant_keys] = optimize_group_by_constant_keys; DB::Aggregator::Params params( + aggregate_settings, grouping_keys, agg_descriptions, false, - max_rows_to_group_by, - group_by_overflow_mode, group_by_two_level_threshold, group_by_two_level_threshold_bytes, - max_bytes_before_external_group_by, empty_result_for_aggregation_by_empty_set, tmp_data_scope, - max_threads, - min_free_disk_space, - compile_aggregate_expressions, - min_count_to_compile_aggregate_expression, - max_block_size, - enable_prefetch, only_merge, - optimize_group_by_constant_keys, - min_hit_rate_to_use_consecutive_keys_optimization, {}); + return params; } diff --git a/cpp-ch/local-engine/Common/AggregateUtil.h b/cpp-ch/local-engine/Common/AggregateUtil.h index 380e1ea35539..8fd36987ac8c 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.h +++ b/cpp-ch/local-engine/Common/AggregateUtil.h @@ -71,7 +71,7 @@ class AggregatorParamsHelper // for using grace aggregating, never enable ch spill, otherwise there will be data lost. static DB::Aggregator::Params buildParams( - DB::ContextPtr context, + const DB::ContextPtr & context, const DB::Names & grouping_keys, const DB::AggregateDescriptions & agg_descriptions, Mode mode, diff --git a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp index 5a6f229744fc..ae0a50d2d3f2 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp @@ -207,7 +207,9 @@ DB::QueryPlanPtr CrossRelParser::parseJoin(const substrait::CrossRel & join, DB: context->getSettingsRef()[Setting::max_block_size], context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, - false); + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("CROSS_JOIN"); steps.emplace_back(join_step.get()); std::vector plans; @@ -254,7 +256,12 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left NameSet left_columns_set; for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); - table_join.setColumnsFromJoinedTable(right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); + + table_join.setColumnsFromJoinedTable( + right.getCurrentHeader().getNamesAndTypesList(), + left_columns_set, + getUniqueName("right") + ".", + left.getCurrentHeader().getNamesAndTypesList()); // fix right table key duplicate NamesWithAliases right_table_alias; diff --git a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp index 7493471697f1..6a5f9bc9378e 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp @@ -322,7 +322,9 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q context->getSettingsRef()[Setting::max_block_size], context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, - false); + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("SORT_MERGE_JOIN"); steps.emplace_back(join_step.get()); @@ -390,7 +392,11 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, NameSet left_columns_set; for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); - table_join.setColumnsFromJoinedTable(right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); + table_join.setColumnsFromJoinedTable( + right.getCurrentHeader().getNamesAndTypesList(), + left_columns_set, + getUniqueName("right") + ".", + left.getCurrentHeader().getNamesAndTypesList()); // fix right table key duplicate NamesWithAliases right_table_alias; @@ -787,7 +793,9 @@ DB::QueryPlanPtr JoinRelParser::buildMultiOnClauseHashJoin( context->getSettingsRef()[Setting::max_block_size], context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, - false); + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("Multi join on clause hash join"); steps.emplace_back(join_step.get()); std::vector plans; @@ -827,7 +835,9 @@ DB::QueryPlanPtr JoinRelParser::buildSingleOnClauseHashJoin( context->getSettingsRef()[Setting::max_block_size], context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, - false); + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("HASH_JOIN"); steps.emplace_back(join_step.get()); diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index 02d43124745e..5df5eaff8c43 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -97,6 +97,10 @@ TEST(TestJoin, simple) for (const auto & column : join->columnsFromJoinedTable()) join->addJoinedColumn(column); + auto columns_from_left_table = left_plan.getCurrentHeader().getNamesAndTypesList(); + for (auto & column_from_joined_table : columns_from_left_table) + join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); + auto left_keys = left.getNamesAndTypesList(); join->addJoinedColumnsAndCorrectTypes(left_keys, true); std::cerr << "after join:\n"; @@ -123,7 +127,7 @@ TEST(TestJoin, simple) auto hash_join = std::make_shared(join, right_plan.getCurrentHeader()); QueryPlanStepPtr join_step - = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 8192, 1, false); + = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 8192, 1, NameSet{}, false, false); std::cerr << "join step:" << join_step->getOutputHeader().dumpStructure() << std::endl; From 6d7824ece7340d407f8e318e4f829f2a4209877c Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 2 Dec 2024 07:19:45 +0800 Subject: [PATCH 175/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_30) (#8111) Upstream Velox's New Commits: ac5c15eb7 by dependabot[bot], Bump pypa/gh-action-pypi-publish from 1.6.4 to 1.12.2 (11494) af20b93b1 by zuyu, fix: Build runner test only when VELOX_BUILD_TESTING enabled (11669) 281fb0471 by Xiaoxuan Meng, fix: Remove the fdallocate in ssd file space allocation (11690) feb5b757f by Jimmy Lu, fix: Unsigned index underflow in trimWhiteSpace (11689) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 1d3a84d858a8..3e907e92360e 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_29 +VELOX_BRANCH=2024_11_30 VELOX_HOME="" OS=`uname -s` From d75e90f5c57be3bf345ed861c52d474550001282 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Mon, 2 Dec 2024 09:44:46 +0800 Subject: [PATCH 176/211] [GLUTEN-8080][CH]Support function transform_keys/transform_values (#8085) * support function transfrom_keys/transform_values * fix velox failed ut * revert all files * support transform_keys/values * add uts --- .../gluten/utils/CHExpressionUtil.scala | 2 - .../GlutenFunctionValidateSuite.scala | 12 +++ cpp-ch/local-engine/Parser/FunctionParser.cpp | 2 - .../mapHighOrderFunctions.cpp | 92 +++++++++++++++++++ .../clickhouse/ClickHouseTestSettings.scala | 4 + .../clickhouse/ClickHouseTestSettings.scala | 4 + .../clickhouse/ClickHouseTestSettings.scala | 4 + .../clickhouse/ClickHouseTestSettings.scala | 4 + 8 files changed, 120 insertions(+), 4 deletions(-) create mode 100644 cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala index f6d18d7a2228..1dd815b6d78d 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala @@ -203,8 +203,6 @@ object CHExpressionUtil { TO_UTC_TIMESTAMP -> UtcTimestampValidator(), FROM_UTC_TIMESTAMP -> UtcTimestampValidator(), STACK -> DefaultValidator(), - TRANSFORM_KEYS -> DefaultValidator(), - TRANSFORM_VALUES -> DefaultValidator(), RAISE_ERROR -> DefaultValidator(), WIDTH_BUCKET -> DefaultValidator() ) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala index dbe8852290aa..39b5421f5d68 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala @@ -860,4 +860,16 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val sql = "select cast(id % 2 = 1 as string) from range(10)" compareResultsAgainstVanillaSpark(sql, true, { _ => }) } + + test("Test transform_keys/transform_values") { + val sql = """ + |select + | transform_keys(map_from_arrays(array(id+1, id+2, id+3), + | array(1, id+2, 3)), (k, v) -> k + 1), + | transform_values(map_from_arrays(array(id+1, id+2, id+3), + | array(1, id+2, 3)), (k, v) -> v + 1) + |from range(10) + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } } diff --git a/cpp-ch/local-engine/Parser/FunctionParser.cpp b/cpp-ch/local-engine/Parser/FunctionParser.cpp index 7e794dabec64..6ea5148ea406 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/FunctionParser.cpp @@ -181,9 +181,7 @@ FunctionParserPtr FunctionParserFactory::get(const String & name, ParserContextP { auto res = tryGet(name, ctx); if (!res) - { throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown function parser {}", name); - } return res; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp new file mode 100644 index 000000000000..e559980f8548 --- /dev/null +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "DataTypes/DataTypeMap.h" + +namespace DB::ErrorCodes +{ + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +namespace local_engine +{ + +template +class FunctionParserMapTransformImpl : public FunctionParser +{ +public: + static constexpr auto name = transform_keys ? "transform_keys" : "transform_values"; + String getName() const override { return name; } + + explicit FunctionParserMapTransformImpl(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + ~FunctionParserMapTransformImpl() override = default; + + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override + { + /// Parse spark transform_keys(map, func) as CH mapFromArrays(arrayMap(func, cast(map as array)), mapValues(map)) + /// Parse spark transform_values(map, func) as CH mapFromArrays(mapKeys(map), arrayMap(func, cast(map as array))) + auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); + if (parsed_args.size() != 2) + throw DB::Exception(DB::ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "{} function must have three arguments", getName()); + + auto lambda_args = collectLambdaArguments(parser_context, substrait_func.arguments()[1].value().scalar_function()); + if (lambda_args.size() != 2) + throw DB::Exception( + DB::ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "The lambda function in {} must have two arguments", getName()); + + const auto * map_node = parsed_args[0]; + const auto * func_node = parsed_args[1]; + const auto & map_type = map_node->result_type; + auto array_type = checkAndGetDataType(removeNullable(map_type).get())->getNestedType(); + if (map_type->isNullable()) + array_type = std::make_shared(array_type); + const auto * array_node = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, map_node, array_type); + const auto * transformed_node = toFunctionNode(actions_dag, "arrayMap", {func_node, array_node}); + + const DB::ActionsDAG::Node * result_node = nullptr; + if constexpr (transform_keys) + { + const auto * nontransformed_node = toFunctionNode(actions_dag, "mapValues", {parsed_args[0]}); + result_node = toFunctionNode(actions_dag, "mapFromArrays", {transformed_node, nontransformed_node}); + } + else + { + const auto * nontransformed_node = toFunctionNode(actions_dag, "mapKeys", {parsed_args[0]}); + result_node = toFunctionNode(actions_dag, "mapFromArrays", {nontransformed_node, transformed_node}); + } + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } +}; + +using FunctionParserTransformKeys = FunctionParserMapTransformImpl; +using FunctionParserTransformValues = FunctionParserMapTransformImpl; + +static FunctionParserRegister register_transform_keys; +static FunctionParserRegister register_transform_values; +} \ No newline at end of file diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 2eb5bd11ffbe..71e32bdccf7a 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -166,6 +166,10 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") + .exclude("transform keys function - primitive data types") + .exclude("transform keys function - Invalid lambda functions and exceptions") + .exclude("transform values function - test primitive data types") + .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index a7bf5d4da903..ce09d0f59580 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -184,6 +184,10 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") + .exclude("transform keys function - primitive data types") + .exclude("transform keys function - Invalid lambda functions and exceptions") + .exclude("transform values function - test primitive data types") + .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index b7e3905740fb..71e2f6375e8f 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -186,6 +186,10 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") + .exclude("transform keys function - primitive data types") + .exclude("transform keys function - Invalid lambda functions and exceptions") + .exclude("transform values function - test primitive data types") + .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 8ce145735dc3..f08b66972c04 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -186,6 +186,10 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") + .exclude("transform keys function - primitive data types") + .exclude("transform keys function - Invalid lambda functions and exceptions") + .exclude("transform values function - test primitive data types") + .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( From fedbc72c9153f75c83f1717360b48f4e40f4e7fd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Mon, 2 Dec 2024 13:06:22 +0800 Subject: [PATCH 177/211] =?UTF-8?q?Revert=20"[GLUTEN-8080][CH]Support=20fu?= =?UTF-8?q?nction=20transform=5Fkeys/transform=5Fvalues=20(#8=E2=80=A6"=20?= =?UTF-8?q?(#8121)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit d75e90f5c57be3bf345ed861c52d474550001282. --- .../gluten/utils/CHExpressionUtil.scala | 2 + .../GlutenFunctionValidateSuite.scala | 12 --- cpp-ch/local-engine/Parser/FunctionParser.cpp | 2 + .../mapHighOrderFunctions.cpp | 92 ------------------- .../clickhouse/ClickHouseTestSettings.scala | 4 - .../clickhouse/ClickHouseTestSettings.scala | 4 - .../clickhouse/ClickHouseTestSettings.scala | 4 - .../clickhouse/ClickHouseTestSettings.scala | 4 - 8 files changed, 4 insertions(+), 120 deletions(-) delete mode 100644 cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala index 1dd815b6d78d..f6d18d7a2228 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHExpressionUtil.scala @@ -203,6 +203,8 @@ object CHExpressionUtil { TO_UTC_TIMESTAMP -> UtcTimestampValidator(), FROM_UTC_TIMESTAMP -> UtcTimestampValidator(), STACK -> DefaultValidator(), + TRANSFORM_KEYS -> DefaultValidator(), + TRANSFORM_VALUES -> DefaultValidator(), RAISE_ERROR -> DefaultValidator(), WIDTH_BUCKET -> DefaultValidator() ) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala index 39b5421f5d68..dbe8852290aa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala @@ -860,16 +860,4 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val sql = "select cast(id % 2 = 1 as string) from range(10)" compareResultsAgainstVanillaSpark(sql, true, { _ => }) } - - test("Test transform_keys/transform_values") { - val sql = """ - |select - | transform_keys(map_from_arrays(array(id+1, id+2, id+3), - | array(1, id+2, 3)), (k, v) -> k + 1), - | transform_values(map_from_arrays(array(id+1, id+2, id+3), - | array(1, id+2, 3)), (k, v) -> v + 1) - |from range(10) - |""".stripMargin - compareResultsAgainstVanillaSpark(sql, true, { _ => }) - } } diff --git a/cpp-ch/local-engine/Parser/FunctionParser.cpp b/cpp-ch/local-engine/Parser/FunctionParser.cpp index 6ea5148ea406..7e794dabec64 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/FunctionParser.cpp @@ -181,7 +181,9 @@ FunctionParserPtr FunctionParserFactory::get(const String & name, ParserContextP { auto res = tryGet(name, ctx); if (!res) + { throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown function parser {}", name); + } return res; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp deleted file mode 100644 index e559980f8548..000000000000 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/mapHighOrderFunctions.cpp +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "DataTypes/DataTypeMap.h" - -namespace DB::ErrorCodes -{ - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; -} - -namespace local_engine -{ - -template -class FunctionParserMapTransformImpl : public FunctionParser -{ -public: - static constexpr auto name = transform_keys ? "transform_keys" : "transform_values"; - String getName() const override { return name; } - - explicit FunctionParserMapTransformImpl(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} - ~FunctionParserMapTransformImpl() override = default; - - const DB::ActionsDAG::Node * - parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override - { - /// Parse spark transform_keys(map, func) as CH mapFromArrays(arrayMap(func, cast(map as array)), mapValues(map)) - /// Parse spark transform_values(map, func) as CH mapFromArrays(mapKeys(map), arrayMap(func, cast(map as array))) - auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); - if (parsed_args.size() != 2) - throw DB::Exception(DB::ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "{} function must have three arguments", getName()); - - auto lambda_args = collectLambdaArguments(parser_context, substrait_func.arguments()[1].value().scalar_function()); - if (lambda_args.size() != 2) - throw DB::Exception( - DB::ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "The lambda function in {} must have two arguments", getName()); - - const auto * map_node = parsed_args[0]; - const auto * func_node = parsed_args[1]; - const auto & map_type = map_node->result_type; - auto array_type = checkAndGetDataType(removeNullable(map_type).get())->getNestedType(); - if (map_type->isNullable()) - array_type = std::make_shared(array_type); - const auto * array_node = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, map_node, array_type); - const auto * transformed_node = toFunctionNode(actions_dag, "arrayMap", {func_node, array_node}); - - const DB::ActionsDAG::Node * result_node = nullptr; - if constexpr (transform_keys) - { - const auto * nontransformed_node = toFunctionNode(actions_dag, "mapValues", {parsed_args[0]}); - result_node = toFunctionNode(actions_dag, "mapFromArrays", {transformed_node, nontransformed_node}); - } - else - { - const auto * nontransformed_node = toFunctionNode(actions_dag, "mapKeys", {parsed_args[0]}); - result_node = toFunctionNode(actions_dag, "mapFromArrays", {nontransformed_node, transformed_node}); - } - return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); - } -}; - -using FunctionParserTransformKeys = FunctionParserMapTransformImpl; -using FunctionParserTransformValues = FunctionParserMapTransformImpl; - -static FunctionParserRegister register_transform_keys; -static FunctionParserRegister register_transform_values; -} \ No newline at end of file diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 71e32bdccf7a..2eb5bd11ffbe 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -166,10 +166,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") - .exclude("transform keys function - primitive data types") - .exclude("transform keys function - Invalid lambda functions and exceptions") - .exclude("transform values function - test primitive data types") - .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index ce09d0f59580..a7bf5d4da903 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -184,10 +184,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") - .exclude("transform keys function - primitive data types") - .exclude("transform keys function - Invalid lambda functions and exceptions") - .exclude("transform values function - test primitive data types") - .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 71e2f6375e8f..b7e3905740fb 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -186,10 +186,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") - .exclude("transform keys function - primitive data types") - .exclude("transform keys function - Invalid lambda functions and exceptions") - .exclude("transform values function - test primitive data types") - .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index f08b66972c04..8ce145735dc3 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -186,10 +186,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("aggregate function - array for non-primitive type") .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") .exclude("SPARK-24734: Fix containsNull of Concat for array type") - .exclude("transform keys function - primitive data types") - .exclude("transform keys function - Invalid lambda functions and exceptions") - .exclude("transform values function - test primitive data types") - .exclude("transform values function - test empty") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenDataFrameJoinSuite].exclude( From 9f8acc9be8ede26be820619282ae87bfb45918fb Mon Sep 17 00:00:00 2001 From: lgbo Date: Mon, 2 Dec 2024 13:42:02 +0800 Subject: [PATCH 178/211] in case DB::Setting::max_bytes_ratio_before_external_group_by is enable by default (#8120) --- cpp-ch/local-engine/Common/AggregateUtil.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp index 4a768eb17b6c..a6fedcc535af 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.cpp +++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp @@ -35,6 +35,7 @@ extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; namespace Setting { +extern const SettingsDouble max_bytes_ratio_before_external_group_by; extern const SettingsUInt64 max_bytes_before_external_group_by; extern const SettingsBool optimize_group_by_constant_keys; extern const SettingsUInt64 min_free_disk_space_for_temporary_data; @@ -203,6 +204,9 @@ DB::Aggregator::Params AggregatorParamsHelper::buildParams( size_t group_by_two_level_threshold_bytes = algorithm == Algorithm::GlutenGraceAggregate ? 0 : (mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::group_by_two_level_threshold_bytes])); + double max_bytes_ratio_before_external_group_by = algorithm == Algorithm::GlutenGraceAggregate + ? 0.0 + : (mode == Mode::PARTIAL_TO_FINISHED ? 0.0 : settings[DB::Setting::max_bytes_ratio_before_external_group_by]); size_t max_bytes_before_external_group_by = algorithm == Algorithm::GlutenGraceAggregate ? 0 : (mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::max_bytes_before_external_group_by])); @@ -215,7 +219,8 @@ DB::Aggregator::Params AggregatorParamsHelper::buildParams( ? 0 : static_cast(settings[DB::Setting::min_free_disk_space_for_temporary_data]); bool compile_aggregate_expressions = mode == Mode::PARTIAL_TO_FINISHED ? false : settings[DB::Setting::compile_aggregate_expressions]; - size_t min_count_to_compile_aggregate_expression = mode == Mode::PARTIAL_TO_FINISHED ? 0 : settings[DB::Setting::min_count_to_compile_aggregate_expression]; + size_t min_count_to_compile_aggregate_expression + = mode == Mode::PARTIAL_TO_FINISHED ? 0 : settings[DB::Setting::min_count_to_compile_aggregate_expression]; size_t max_block_size = PODArrayUtil::adjustMemoryEfficientSize(settings[DB::Setting::max_block_size]); bool enable_prefetch = mode != Mode::PARTIAL_TO_FINISHED; bool only_merge = mode == Mode::PARTIAL_TO_FINISHED; @@ -224,6 +229,7 @@ DB::Aggregator::Params AggregatorParamsHelper::buildParams( DB::Settings aggregate_settings{settings}; aggregate_settings[DB::Setting::max_rows_to_group_by] = max_rows_to_group_by; + aggregate_settings[DB::Setting::max_bytes_ratio_before_external_group_by] = max_bytes_ratio_before_external_group_by; aggregate_settings[DB::Setting::max_bytes_before_external_group_by] = max_bytes_before_external_group_by; aggregate_settings[DB::Setting::min_free_disk_space_for_temporary_data] = min_free_disk_space; aggregate_settings[DB::Setting::compile_aggregate_expressions] = compile_aggregate_expressions; From 9059ff685f37367af8d84c6cc26bae05ec2a225f Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 3 Dec 2024 09:36:15 +0800 Subject: [PATCH 179/211] [GLUTEN-8060][CORE] GlutenShuffleManager as a registry of shuffle managers (#8084) --- .../spark/shuffle/GlutenShuffleManager.scala | 71 ++++ .../org/apache/spark/shuffle/LookupKey.scala | 27 ++ .../spark/shuffle/ShuffleManagerLookup.scala | 38 +++ .../shuffle/ShuffleManagerRegistry.scala | 94 ++++++ .../spark/shuffle/ShuffleManagerRouter.scala | 137 ++++++++ .../shuffle/GlutenShuffleManagerSuite.scala | 315 ++++++++++++++++++ 6 files changed, 682 insertions(+) create mode 100644 gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala create mode 100644 gluten-core/src/main/scala/org/apache/spark/shuffle/LookupKey.scala create mode 100644 gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerLookup.scala create mode 100644 gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala create mode 100644 gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRouter.scala create mode 100644 gluten-core/src/test/scala/org/apache/spark/shuffle/GlutenShuffleManagerSuite.scala diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala new file mode 100644 index 000000000000..d38781675baf --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.{ShuffleDependency, SparkConf, TaskContext} +import org.apache.spark.annotation.Experimental + +/** + * Shuffle manager that routes shuffle API calls to different shuffle managers registered by + * different backends. + * + * A SPIP may cause refactoring of this class in the future: + * https://issues.apache.org/jira/browse/SPARK-45792 + */ +@Experimental +class GlutenShuffleManager(conf: SparkConf, isDriver: Boolean) extends ShuffleManager { + private val routerBuilder = ShuffleManagerRegistry.get().newRouterBuilder(conf, isDriver) + + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + routerBuilder.getOrBuild().registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + routerBuilder.getOrBuild().getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + routerBuilder + .getOrBuild() + .getReader(handle, startMapIndex, endMapIndex, startPartition, endPartition, context, metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + routerBuilder.getOrBuild().unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = { + routerBuilder.getOrBuild().shuffleBlockResolver + } + + override def stop(): Unit = { + routerBuilder.getOrBuild().stop() + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/LookupKey.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/LookupKey.scala new file mode 100644 index 000000000000..502dd92efe80 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/LookupKey.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.ShuffleDependency + +/** + * Required during shuffle manager registration to determine whether the shuffle manager should be + * used for the particular shuffle dependency. + */ +trait LookupKey { + def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerLookup.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerLookup.scala new file mode 100644 index 000000000000..8b060c98184f --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerLookup.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.ShuffleDependency + +private class ShuffleManagerLookup(all: Seq[(LookupKey, ShuffleManager)]) { + private val allReversed = all.reverse + + def findShuffleManager[K, V, C](dependency: ShuffleDependency[K, V, C]): ShuffleManager = { + this.synchronized { + // The latest shuffle manager registered will be looked up earlier. + allReversed.find(_._1.accepts(dependency)).map(_._2).getOrElse { + throw new IllegalStateException(s"No ShuffleManager found for $dependency") + } + } + } + + def all(): Seq[ShuffleManager] = { + this.synchronized { + all.map(_._2) + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala new file mode 100644 index 000000000000..4310054caa51 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils + +import scala.collection.mutable + +class ShuffleManagerRegistry private[ShuffleManagerRegistry] { + import ShuffleManagerRegistry._ + private val all: mutable.Buffer[(LookupKey, String)] = mutable.Buffer() + private val routerBuilders: mutable.Buffer[RouterBuilder] = mutable.Buffer() + private val classDeDup: mutable.Set[String] = mutable.Set() + + def register(lookupKey: LookupKey, shuffleManagerClass: String): Unit = { + val clazz = Utils.classForName(shuffleManagerClass) + require( + !clazz.isAssignableFrom(classOf[GlutenShuffleManager]), + "It's not allowed to register GlutenShuffleManager recursively") + require( + classOf[ShuffleManager].isAssignableFrom(clazz), + s"Shuffle manager class to register is not an implementation of Spark ShuffleManager: " + + s"$shuffleManagerClass" + ) + require( + !classDeDup.contains(shuffleManagerClass), + s"Shuffle manager class already registered: $shuffleManagerClass") + this.synchronized { + classDeDup += shuffleManagerClass + all += lookupKey -> shuffleManagerClass + // Invalidate all shuffle managers cached in each alive router builder instances. + // Then, once the router builder is accessed, a new router will be forced to create. + routerBuilders.foreach(_.invalidateCache()) + } + } + + // Visible for testing + private[shuffle] def clear(): Unit = { + this.synchronized { + classDeDup.clear() + all.clear() + routerBuilders.foreach(_.invalidateCache()) + } + } + + private[shuffle] def newRouterBuilder(conf: SparkConf, isDriver: Boolean): RouterBuilder = + this.synchronized { + val out = new RouterBuilder(this, conf, isDriver) + routerBuilders += out + out + } +} + +object ShuffleManagerRegistry { + private val instance = new ShuffleManagerRegistry() + + def get(): ShuffleManagerRegistry = instance + + class RouterBuilder(registry: ShuffleManagerRegistry, conf: SparkConf, isDriver: Boolean) { + private var router: Option[ShuffleManagerRouter] = None + + private[ShuffleManagerRegistry] def invalidateCache(): Unit = synchronized { + router = None + } + + private[shuffle] def getOrBuild(): ShuffleManagerRouter = synchronized { + if (router.isEmpty) { + val instances = registry.all.map(key => key._1 -> instantiate(key._2, conf, isDriver)) + router = Some(new ShuffleManagerRouter(new ShuffleManagerLookup(instances.toSeq))) + } + router.get + } + + private def instantiate(clazz: String, conf: SparkConf, isDriver: Boolean): ShuffleManager = { + Utils + .instantiateSerializerOrShuffleManager[ShuffleManager](clazz, conf, isDriver) + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRouter.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRouter.scala new file mode 100644 index 000000000000..80aa9d804785 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRouter.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle +import org.apache.spark.{ShuffleDependency, TaskContext} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.MergedBlockMeta +import org.apache.spark.storage.{BlockId, ShuffleBlockBatchId, ShuffleBlockId, ShuffleMergedBlockId} + +/** The internal shuffle manager instance used by GlutenShuffleManager. */ +private class ShuffleManagerRouter(lookup: ShuffleManagerLookup) extends ShuffleManager { + import ShuffleManagerRouter._ + private val cache = new Cache() + private val resolver = new BlockResolver(cache) + + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + val manager = lookup.findShuffleManager(dependency) + cache.store(shuffleId, manager).registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + cache.get(handle.shuffleId).getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + cache + .get(handle.shuffleId) + .getReader(handle, startMapIndex, endMapIndex, startPartition, endPartition, context, metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + cache.remove(shuffleId).unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = resolver + + override def stop(): Unit = { + assert(cache.size() == 0) + lookup.all().reverse.foreach(_.stop()) + } +} + +private object ShuffleManagerRouter { + private class Cache { + private val cache: java.util.Map[Int, ShuffleManager] = + new java.util.concurrent.ConcurrentHashMap() + + def store(shuffleId: Int, manager: ShuffleManager): ShuffleManager = { + cache.compute( + shuffleId, + (id, m) => { + assert(m == null, s"Shuffle manager was already cached for shuffle id: $id") + manager + }) + } + + def get(shuffleId: Int): ShuffleManager = { + val manager = cache.get(shuffleId) + assert(manager != null, s"Shuffle manager not registered for shuffle id: $shuffleId") + manager + } + + def remove(shuffleId: Int): ShuffleManager = { + val manager = cache.remove(shuffleId) + assert(manager != null, s"Shuffle manager not registered for shuffle id: $shuffleId") + manager + } + + def size(): Int = { + cache.size() + } + + def clear(): Unit = { + cache.clear() + } + } + + private class BlockResolver(cache: Cache) extends ShuffleBlockResolver { + override def getBlockData(blockId: BlockId, dirs: Option[Array[String]]): ManagedBuffer = { + val shuffleId = blockId match { + case id: ShuffleBlockId => + id.shuffleId + case batchId: ShuffleBlockBatchId => + batchId.shuffleId + case _ => + throw new IllegalArgumentException( + "GlutenShuffleManager: Unsupported shuffle block id: " + blockId) + } + cache.get(shuffleId).shuffleBlockResolver.getBlockData(blockId, dirs) + } + + override def getMergedBlockData( + blockId: ShuffleMergedBlockId, + dirs: Option[Array[String]]): Seq[ManagedBuffer] = { + val shuffleId = blockId.shuffleId + cache.get(shuffleId).shuffleBlockResolver.getMergedBlockData(blockId, dirs) + } + + override def getMergedBlockMeta( + blockId: ShuffleMergedBlockId, + dirs: Option[Array[String]]): MergedBlockMeta = { + val shuffleId = blockId.shuffleId + cache.get(shuffleId).shuffleBlockResolver.getMergedBlockMeta(blockId, dirs) + } + + override def stop(): Unit = { + throw new UnsupportedOperationException( + s"BlockResolver ${getClass.getSimpleName} doesn't need to be explicitly stopped") + } + } +} diff --git a/gluten-core/src/test/scala/org/apache/spark/shuffle/GlutenShuffleManagerSuite.scala b/gluten-core/src/test/scala/org/apache/spark/shuffle/GlutenShuffleManagerSuite.scala new file mode 100644 index 000000000000..640fc0ab0796 --- /dev/null +++ b/gluten-core/src/test/scala/org/apache/spark/shuffle/GlutenShuffleManagerSuite.scala @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.{Partitioner, ShuffleDependency, SparkConf, TaskContext} +import org.apache.spark.internal.config.SHUFFLE_MANAGER +import org.apache.spark.rdd.EmptyRDD +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.sql.test.SharedSparkSession + +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable + +class GlutenShuffleManagerSuite extends SharedSparkSession { + import GlutenShuffleManagerSuite._ + override protected def sparkConf: SparkConf = { + super.sparkConf + .set(SHUFFLE_MANAGER.key, classOf[GlutenShuffleManager].getName) + } + + override protected def afterEach(): Unit = { + val registry = ShuffleManagerRegistry.get() + registry.clear() + counter1.clear() + counter2.clear() + } + + test("register one") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("stop") == 0) + gm.stop() + assert(counter1.count("stop") == 1) + gm.stop() + gm.stop() + assert(counter1.count("stop") == 3) + } + + test("register two") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager2].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 0) + // The statement calls #registerShuffle internally. + val dep = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep.shuffleId) + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 1) + + assert(counter1.count("stop") == 0) + assert(counter2.count("stop") == 0) + gm.stop() + assert(counter1.count("stop") == 1) + assert(counter2.count("stop") == 1) + } + + test("register two - disordered registration") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 0) + val dep1 = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep1.shuffleId) + assert(counter1.count("registerShuffle") == 1) + assert(counter2.count("registerShuffle") == 0) + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager2].getName) + + // The statement calls #registerShuffle internally. + val dep2 = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep2.shuffleId) + assert(counter1.count("registerShuffle") == 1) + assert(counter2.count("registerShuffle") == 1) + + assert(counter1.count("stop") == 0) + assert(counter2.count("stop") == 0) + gm.stop() + assert(counter1.count("stop") == 1) + assert(counter2.count("stop") == 1) + } + + test("register two - with empty key") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = false + }, + classOf[ShuffleManager2].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 0) + // The statement calls #registerShuffle internally. + val dep = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep.shuffleId) + assert(counter1.count("registerShuffle") == 1) + assert(counter2.count("registerShuffle") == 0) + } + + test("register recursively") { + val registry = ShuffleManagerRegistry.get() + + assertThrows[IllegalArgumentException]( + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[GlutenShuffleManager].getName)) + } + + test("register duplicated") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + assertThrows[IllegalArgumentException]( + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName)) + } +} + +object GlutenShuffleManagerSuite { + private val counter1 = new InvocationCounter + private val counter2 = new InvocationCounter + + class ShuffleManager1(conf: SparkConf) extends ShuffleManager { + private val delegate = new SortShuffleManager(conf) + private val counter = counter1 + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + counter.increment("registerShuffle") + delegate.registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + counter.increment("getWriter") + delegate.getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + counter.increment("getReader") + delegate.getReader( + handle, + startMapIndex, + endMapIndex, + startPartition, + endPartition, + context, + metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + counter.increment("unregisterShuffle") + delegate.unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = { + counter.increment("shuffleBlockResolver") + delegate.shuffleBlockResolver + } + + override def stop(): Unit = { + counter.increment("stop") + delegate.stop() + } + } + + class ShuffleManager2(conf: SparkConf, isDriver: Boolean) extends ShuffleManager { + private val delegate = new SortShuffleManager(conf) + private val counter = counter2 + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + counter.increment("registerShuffle") + delegate.registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + counter.increment("getWriter") + delegate.getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + counter.increment("getReader") + delegate.getReader( + handle, + startMapIndex, + endMapIndex, + startPartition, + endPartition, + context, + metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + counter.increment("unregisterShuffle") + delegate.unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = { + counter.increment("shuffleBlockResolver") + delegate.shuffleBlockResolver + } + + override def stop(): Unit = { + counter.increment("stop") + delegate.stop() + } + } + + private class InvocationCounter { + private val counter: mutable.Map[String, AtomicInteger] = mutable.Map() + + def increment(name: String): Unit = synchronized { + counter.getOrElseUpdate(name, new AtomicInteger()).incrementAndGet() + } + + def count(name: String): Int = { + counter.getOrElse(name, new AtomicInteger()).get() + } + + def clear(): Unit = { + counter.clear() + } + } + + private object DummyPartitioner extends Partitioner { + override def numPartitions: Int = 0 + override def getPartition(key: Any): Int = 0 + } +} From 6dd91bac6cc155810beb34415a6dee1a03069f6d Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 3 Dec 2024 09:40:46 +0800 Subject: [PATCH 180/211] [GLUTEN-7745][VL] Incorporate SQL Union operator into Velox execution pipeline (#7842) --- .../backendsapi/clickhouse/CHMetricsApi.scala | 8 + .../apache/gluten/utils/GlutenURLDecoder.java | 2 +- .../backendsapi/velox/VeloxListenerApi.scala | 3 +- .../backendsapi/velox/VeloxMetricsApi.scala | 11 ++ .../backendsapi/velox/VeloxRuleApi.scala | 2 + .../apache/gluten/metrics/MetricsUtil.scala | 9 +- .../gluten/metrics/UnionMetricsUpdater.scala | 35 ++++ .../gluten/execution/MiscOperatorSuite.scala | 28 +++- .../VeloxOrcDataTypeValidationSuite.scala | 5 +- .../VeloxParquetDataTypeValidationSuite.scala | 5 +- cpp/velox/compute/WholeStageResultIterator.cc | 44 ++++- cpp/velox/substrait/SubstraitToVeloxPlan.cc | 46 ++++++ cpp/velox/substrait/SubstraitToVeloxPlan.h | 3 + .../SubstraitToVeloxPlanValidator.cc | 153 ++++++++++++++---- .../substrait/SubstraitToVeloxPlanValidator.h | 15 +- .../gluten/substrait/rel/RelBuilder.java | 17 ++ .../gluten/substrait/rel/SetRelNode.java | 62 +++++++ .../gluten/backendsapi/MetricsApi.scala | 4 + .../BasicPhysicalOperatorTransformer.scala | 3 +- .../execution/UnionExecTransformer.scala | 99 ++++++++++++ .../columnar/UnionTransformerRule.scala | 61 +++++++ .../WholeStageTransformerSuite.scala | 35 +++- .../org/apache/gluten/GlutenConfig.scala | 9 ++ 23 files changed, 604 insertions(+), 55 deletions(-) create mode 100644 backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala create mode 100644 gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/SetRelNode.java create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/execution/UnionExecTransformer.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/UnionTransformerRule.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala index 73b2d0f21101..a0576a807b98 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala @@ -450,6 +450,14 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { s"SampleTransformer metrics update is not supported in CH backend") } + override def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = + throw new UnsupportedOperationException( + "UnionExecTransformer metrics update is not supported in CH backend") + + override def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater = + throw new UnsupportedOperationException( + "UnionExecTransformer metrics update is not supported in CH backend") + def genWriteFilesTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map( "physicalWrittenBytes" -> SQLMetrics.createMetric(sparkContext, "number of written bytes"), diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java b/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java index 9228a2f860ae..856ddf159730 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java @@ -31,7 +31,7 @@ public class GlutenURLDecoder { *

    Note: The World Wide Web Consortium * Recommendation states that UTF-8 should be used. Not doing so may introduce - * incompatibilites. + * incompatibilities. * * @param s the String to decode * @param enc The name of a supported character diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index d29d3029709e..3a82abe61833 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -31,6 +31,7 @@ import org.apache.spark.{HdfsConfGenerator, SparkConf, SparkContext} import org.apache.spark.api.plugin.PluginContext import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit +import org.apache.spark.sql.execution.ColumnarCachedBatchSerializer import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules import org.apache.spark.sql.execution.datasources.velox.{VeloxParquetWriterInjects, VeloxRowSplitter} import org.apache.spark.sql.expression.UDFResolver @@ -75,7 +76,7 @@ class VeloxListenerApi extends ListenerApi with Logging { if (conf.getBoolean(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, defaultValue = false)) { conf.set( StaticSQLConf.SPARK_CACHE_SERIALIZER.key, - "org.apache.spark.sql.execution.ColumnarCachedBatchSerializer") + classOf[ColumnarCachedBatchSerializer].getName) } // Static initializers for driver. diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala index e70e1d13bdfe..934b680382ea 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala @@ -582,4 +582,15 @@ class VeloxMetricsApi extends MetricsApi with Logging { override def genSampleTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater = new SampleMetricsUpdater(metrics) + + override def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map( + "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), + "inputVectors" -> SQLMetrics.createMetric(sparkContext, "number of input vectors"), + "inputBytes" -> SQLMetrics.createSizeMetric(sparkContext, "number of input bytes"), + "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of union"), + "cpuCount" -> SQLMetrics.createMetric(sparkContext, "cpu wall time count") + ) + + override def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater = + new UnionMetricsUpdater(metrics) } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 7841e6cd94b1..7337be573710 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -92,6 +92,7 @@ object VeloxRuleApi { c => HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads)) // Legacy: Post-transform rules. + injector.injectPostTransform(_ => UnionTransformerRule()) injector.injectPostTransform(c => PartialProjectRule.apply(c.session)) injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectPostTransform(c => RewriteTransformer.apply(c.session)) @@ -178,6 +179,7 @@ object VeloxRuleApi { // Gluten RAS: Post rules. injector.injectPostTransform(_ => RemoveTransitions) + injector.injectPostTransform(_ => UnionTransformerRule()) injector.injectPostTransform(c => PartialProjectRule.apply(c.session)) injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectPostTransform(c => RewriteTransformer.apply(c.session)) diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala index cd50d0b8e20c..b8ef1620f905 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala @@ -58,7 +58,8 @@ object MetricsUtil extends Logging { assert(t.children.size == 1, "MetricsUpdater.None can only be used on unary operator") treeifyMetricsUpdaters(t.children.head) case t: TransformSupport => - MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters)) + // Reversed children order to match the traversal code. + MetricsUpdaterTree(t.metricsUpdater(), t.children.reverse.map(treeifyMetricsUpdaters)) case _ => MetricsUpdaterTree(MetricsUpdater.Terminate, Seq()) } @@ -233,6 +234,12 @@ object MetricsUtil extends Logging { operatorMetrics, metrics.getSingleMetrics, joinParamsMap.get(operatorIdx)) + case u: UnionMetricsUpdater => + // JoinRel outputs two suites of metrics respectively for hash build and hash probe. + // Therefore, fetch one more suite of metrics here. + operatorMetrics.add(metrics.getOperatorMetrics(curMetricsIdx)) + curMetricsIdx -= 1 + u.updateUnionMetrics(operatorMetrics) case hau: HashAggregateMetricsUpdater => hau.updateAggregationMetrics(operatorMetrics, aggParamsMap.get(operatorIdx)) case lu: LimitMetricsUpdater => diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala new file mode 100644 index 000000000000..9e91cf368c0a --- /dev/null +++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.metrics + +import org.apache.spark.sql.execution.metric.SQLMetric + +class UnionMetricsUpdater(val metrics: Map[String, SQLMetric]) extends MetricsUpdater { + override def updateNativeMetrics(opMetrics: IOperatorMetrics): Unit = { + throw new UnsupportedOperationException() + } + + def updateUnionMetrics(unionMetrics: java.util.ArrayList[OperatorMetrics]): Unit = { + // Union was interpreted to LocalExchange + LocalPartition. Use metrics from LocalExchange. + val localExchangeMetrics = unionMetrics.get(0) + metrics("numInputRows") += localExchangeMetrics.inputRows + metrics("inputVectors") += localExchangeMetrics.inputVectors + metrics("inputBytes") += localExchangeMetrics.inputBytes + metrics("cpuCount") += localExchangeMetrics.cpuCount + metrics("wallNanos") += localExchangeMetrics.wallNanos + } +} diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index 5cb2b652604d..8063a5d12207 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -537,11 +537,37 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa |""".stripMargin) { df => { - getExecutedPlan(df).exists(plan => plan.find(_.isInstanceOf[ColumnarUnionExec]).isDefined) + assert( + getExecutedPlan(df).exists( + plan => plan.find(_.isInstanceOf[ColumnarUnionExec]).isDefined)) } } } + test("union_all two tables with known partitioning") { + withSQLConf(GlutenConfig.NATIVE_UNION_ENABLED.key -> "true") { + compareDfResultsAgainstVanillaSpark( + () => { + val df1 = spark.sql("select l_orderkey as orderkey from lineitem") + val df2 = spark.sql("select o_orderkey as orderkey from orders") + df1.repartition(5).union(df2.repartition(5)) + }, + compareResult = true, + checkGlutenOperatorMatch[UnionExecTransformer] + ) + + compareDfResultsAgainstVanillaSpark( + () => { + val df1 = spark.sql("select l_orderkey as orderkey from lineitem") + val df2 = spark.sql("select o_orderkey as orderkey from orders") + df1.repartition(5).union(df2.repartition(6)) + }, + compareResult = true, + checkGlutenOperatorMatch[ColumnarUnionExec] + ) + } + } + test("union_all three tables") { runQueryAndCompare(""" |select count(orderkey) from ( diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala index 24e04f2dfce3..6ac59ba4fa6b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala @@ -255,7 +255,10 @@ class VeloxOrcDataTypeValidationSuite extends VeloxWholeStageTransformerSuite { |""".stripMargin) { df => { - assert(getExecutedPlan(df).exists(plan => plan.isInstanceOf[ColumnarUnionExec])) + assert( + getExecutedPlan(df).exists( + plan => + plan.isInstanceOf[ColumnarUnionExec] || plan.isInstanceOf[UnionExecTransformer])) } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala index 57ca448fec79..cb5614f39669 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala @@ -254,7 +254,10 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit |""".stripMargin) { df => { - assert(getExecutedPlan(df).exists(plan => plan.isInstanceOf[ColumnarUnionExec])) + assert( + getExecutedPlan(df).exists( + plan => + plan.isInstanceOf[ColumnarUnionExec] || plan.isInstanceOf[UnionExecTransformer])) } } diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index b6ecbd959f09..411c6c563646 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -91,7 +91,7 @@ WholeStageResultIterator::WholeStageResultIterator( std::move(queryCtx), velox::exec::Task::ExecutionMode::kSerial); if (!task_->supportSerialExecutionMode()) { - throw std::runtime_error("Task doesn't support single thread execution: " + planNode->toString()); + throw std::runtime_error("Task doesn't support single threaded execution: " + planNode->toString()); } auto fileSystem = velox::filesystems::getFileSystem(spillDir, nullptr); GLUTEN_CHECK(fileSystem != nullptr, "File System for spilling is null!"); @@ -248,15 +248,47 @@ void WholeStageResultIterator::getOrderedNodeIds( const std::shared_ptr& planNode, std::vector& nodeIds) { bool isProjectNode = (std::dynamic_pointer_cast(planNode) != nullptr); + bool isLocalExchangeNode = (std::dynamic_pointer_cast(planNode) != nullptr); + bool isUnionNode = isLocalExchangeNode && + std::dynamic_pointer_cast(planNode)->type() == + velox::core::LocalPartitionNode::Type::kGather; const auto& sourceNodes = planNode->sources(); - for (const auto& sourceNode : sourceNodes) { + if (isProjectNode) { + GLUTEN_CHECK(sourceNodes.size() == 1, "Illegal state"); + const auto sourceNode = sourceNodes.at(0); // Filter over Project are mapped into FilterProject operator in Velox. // Metrics are all applied on Project node, and the metrics for Filter node // do not exist. - if (isProjectNode && std::dynamic_pointer_cast(sourceNode)) { + if (std::dynamic_pointer_cast(sourceNode)) { omittedNodeIds_.insert(sourceNode->id()); } getOrderedNodeIds(sourceNode, nodeIds); + nodeIds.emplace_back(planNode->id()); + return; + } + + if (isUnionNode) { + // FIXME: The whole metrics system in gluten-substrait is magic. Passing metrics trees through JNI with a trivial + // array is possible but requires for a solid design. Apparently we haven't had it. All the code requires complete + // rework. + // Union was interpreted as LocalPartition + LocalExchange + 2 fake projects as children in Velox. So we only fetch + // metrics from the root node. + std::vector> unionChildren{}; + for (const auto& source : planNode->sources()) { + const auto projectedChild = std::dynamic_pointer_cast(source); + GLUTEN_CHECK(projectedChild != nullptr, "Illegal state"); + const auto projectSources = projectedChild->sources(); + GLUTEN_CHECK(projectSources.size() == 1, "Illegal state"); + const auto projectSource = projectSources.at(0); + getOrderedNodeIds(projectSource, nodeIds); + } + nodeIds.emplace_back(planNode->id()); + return; + } + + for (const auto& sourceNode : sourceNodes) { + // Post-order traversal. + getOrderedNodeIds(sourceNode, nodeIds); } nodeIds.emplace_back(planNode->id()); } @@ -350,9 +382,9 @@ void WholeStageResultIterator::collectMetrics() { continue; } - const auto& status = planStats.at(nodeId); - // Add each operator status into metrics. - for (const auto& entry : status.operatorStats) { + const auto& stats = planStats.at(nodeId); + // Add each operator stats into metrics. + for (const auto& entry : stats.operatorStats) { const auto& second = entry.second; metrics_->get(Metrics::kInputRows)[metricIndex] = second->inputRows; metrics_->get(Metrics::kInputVectors)[metricIndex] = second->inputVectors; diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 1efa7338796d..3ceccca4a3de 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -1043,6 +1043,50 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan( childNode); } +core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::SetRel& setRel) { + switch (setRel.op()) { + case ::substrait::SetRel_SetOp::SetRel_SetOp_SET_OP_UNION_ALL: { + std::vector children; + for (int32_t i = 0; i < setRel.inputs_size(); ++i) { + const auto& input = setRel.inputs(i); + children.push_back(toVeloxPlan(input)); + } + GLUTEN_CHECK(!children.empty(), "At least one source is required for Velox LocalPartition"); + + // Velox doesn't allow different field names in schemas of LocalPartitionNode's children. + // Add project nodes to unify the schemas. + const RowTypePtr outRowType = asRowType(children[0]->outputType()); + std::vector outNames; + for (int32_t colIdx = 0; colIdx < outRowType->size(); ++colIdx) { + const auto name = outRowType->childAt(colIdx)->name(); + outNames.push_back(name); + } + + std::vector projectedChildren; + for (int32_t i = 0; i < children.size(); ++i) { + const auto& child = children[i]; + const RowTypePtr& childRowType = child->outputType(); + std::vector expressions; + for (int32_t colIdx = 0; colIdx < outNames.size(); ++colIdx) { + const auto fa = + std::make_shared(childRowType->childAt(colIdx), childRowType->nameOf(colIdx)); + const auto cast = std::make_shared(outRowType->childAt(colIdx), fa, false); + expressions.push_back(cast); + } + auto project = std::make_shared(nextPlanNodeId(), outNames, expressions, child); + projectedChildren.push_back(project); + } + return std::make_shared( + nextPlanNodeId(), + core::LocalPartitionNode::Type::kGather, + std::make_shared(), + projectedChildren); + } + default: + throw GlutenException("Unsupported SetRel op: " + std::to_string(setRel.op())); + } +} + core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::SortRel& sortRel) { auto childNode = convertSingleInput<::substrait::SortRel>(sortRel); auto [sortingKeys, sortingOrders] = processSortField(sortRel.sorts(), childNode->outputType()); @@ -1298,6 +1342,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: return toVeloxPlan(rel.write()); } else if (rel.has_windowgrouplimit()) { return toVeloxPlan(rel.windowgrouplimit()); + } else if (rel.has_set()) { + return toVeloxPlan(rel.set()); } else { VELOX_NYI("Substrait conversion not supported for Rel."); } diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.h b/cpp/velox/substrait/SubstraitToVeloxPlan.h index 51e50ce34767..6121923df787 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.h @@ -84,6 +84,9 @@ class SubstraitToVeloxPlanConverter { /// Used to convert Substrait WindowGroupLimitRel into Velox PlanNode. core::PlanNodePtr toVeloxPlan(const ::substrait::WindowGroupLimitRel& windowGroupLimitRel); + /// Used to convert Substrait SetRel into Velox PlanNode. + core::PlanNodePtr toVeloxPlan(const ::substrait::SetRel& setRel); + /// Used to convert Substrait JoinRel into Velox PlanNode. core::PlanNodePtr toVeloxPlan(const ::substrait::JoinRel& joinRel); diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc index 3b74caf8ba5a..9325fed3217c 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc @@ -22,7 +22,6 @@ #include "TypeUtils.h" #include "udf/UdfLoader.h" #include "utils/Common.h" -#include "velox/core/ExpressionEvaluator.h" #include "velox/exec/Aggregate.h" #include "velox/expression/Expr.h" #include "velox/expression/SignatureBinder.h" @@ -30,7 +29,7 @@ namespace gluten { namespace { -static const char* extractFileName(const char* file) { +const char* extractFileName(const char* file) { return strrchr(file, '/') ? strrchr(file, '/') + 1 : file; } @@ -53,13 +52,13 @@ static const char* extractFileName(const char* file) { __FUNCTION__, \ reason)) -static const std::unordered_set kRegexFunctions = { +const std::unordered_set kRegexFunctions = { "regexp_extract", "regexp_extract_all", "regexp_replace", "rlike"}; -static const std::unordered_set kBlackList = { +const std::unordered_set kBlackList = { "split_part", "factorial", "concat_ws", @@ -70,32 +69,59 @@ static const std::unordered_set kBlackList = { "approx_percentile", "get_array_struct_fields", "map_from_arrays"}; - } // namespace -bool SubstraitToVeloxPlanValidator::validateInputTypes( +bool SubstraitToVeloxPlanValidator::parseVeloxType( const ::substrait::extensions::AdvancedExtension& extension, - std::vector& types) { + TypePtr& out) { + ::substrait::Type substraitType; // The input type is wrapped in enhancement. if (!extension.has_enhancement()) { LOG_VALIDATION_MSG("Input type is not wrapped in enhancement."); return false; } const auto& enhancement = extension.enhancement(); - ::substrait::Type inputType; - if (!enhancement.UnpackTo(&inputType)) { + if (!enhancement.UnpackTo(&substraitType)) { LOG_VALIDATION_MSG("Enhancement can't be unpacked to inputType."); return false; } - if (!inputType.has_struct_()) { - LOG_VALIDATION_MSG("Input type has no struct."); + + out = SubstraitParser::parseType(substraitType); + return true; +} + +bool SubstraitToVeloxPlanValidator::flattenVeloxType1(const TypePtr& type, std::vector& out) { + if (type->kind() != TypeKind::ROW) { + LOG_VALIDATION_MSG("Type is not a RowType."); + return false; + } + auto rowType = std::dynamic_pointer_cast(type); + if (!rowType) { + LOG_VALIDATION_MSG("Failed to cast to RowType."); return false; } + for (const auto& field : rowType->children()) { + out.emplace_back(field); + } + return true; +} - // Get the input types. - const auto& sTypes = inputType.struct_().types(); - for (const auto& sType : sTypes) { - types.emplace_back(SubstraitParser::parseType(sType)); +bool SubstraitToVeloxPlanValidator::flattenVeloxType2(const TypePtr& type, std::vector>& out) { + if (type->kind() != TypeKind::ROW) { + LOG_VALIDATION_MSG("Type is not a RowType."); + return false; + } + auto rowType = std::dynamic_pointer_cast(type); + if (!rowType) { + LOG_VALIDATION_MSG("Failed to cast to RowType."); + return false; + } + for (const auto& field : rowType->children()) { + std::vector inner; + if (!flattenVeloxType1(field, inner)) { + return false; + } + out.emplace_back(inner); } return true; } @@ -341,10 +367,11 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WriteRel& writeR } // Validate input data type. + TypePtr inputRowType; std::vector types; if (writeRel.has_named_table()) { const auto& extension = writeRel.named_table().advanced_extension(); - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input type validation in WriteRel."); return false; } @@ -380,12 +407,12 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WriteRel& writeR } bool SubstraitToVeloxPlanValidator::validate(const ::substrait::FetchRel& fetchRel) { - RowTypePtr rowType = nullptr; // Get and validate the input types from extension. if (fetchRel.has_advanced_extension()) { const auto& extension = fetchRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Unsupported input types in FetchRel."); return false; } @@ -396,7 +423,6 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::FetchRel& fetchR for (auto colIdx = 0; colIdx < types.size(); colIdx++) { names.emplace_back(SubstraitParser::makeNodeName(inputPlanNodeId, colIdx)); } - rowType = std::make_shared(std::move(names), std::move(types)); } if (fetchRel.offset() < 0 || fetchRel.count() < 0) { @@ -412,8 +438,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::TopNRel& topNRel // Get and validate the input types from extension. if (topNRel.has_advanced_extension()) { const auto& extension = topNRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Unsupported input types in TopNRel."); return false; } @@ -457,8 +484,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::GenerateRel& gen return false; } const auto& extension = generateRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in GenerateRel."); return false; } @@ -487,8 +515,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::ExpandRel& expan // Get and validate the input types from extension. if (expandRel.has_advanced_extension()) { const auto& extension = expandRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Unsupported input types in ExpandRel."); return false; } @@ -571,8 +600,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowRel& windo return false; } const auto& extension = windowRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in WindowRel."); return false; } @@ -680,7 +710,7 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowRel& windo LOG_VALIDATION_MSG("in windowRel, the sorting key in Sort Operator only support field."); return false; } - exec::ExprSet exprSet({std::move(expression)}, execCtx_); + exec::ExprSet exprSet1({std::move(expression)}, execCtx_); } } @@ -699,8 +729,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowGroupLimit return false; } const auto& extension = windowGroupLimitRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in WindowGroupLimitRel."); return false; } @@ -750,13 +781,61 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowGroupLimit LOG_VALIDATION_MSG("in windowGroupLimitRel, the sorting key in Sort Operator only support field."); return false; } - exec::ExprSet exprSet({std::move(expression)}, execCtx_); + exec::ExprSet exprSet1({std::move(expression)}, execCtx_); } } return true; } +bool SubstraitToVeloxPlanValidator::validate(const ::substrait::SetRel& setRel) { + switch (setRel.op()) { + case ::substrait::SetRel_SetOp::SetRel_SetOp_SET_OP_UNION_ALL: { + for (int32_t i = 0; i < setRel.inputs_size(); ++i) { + const auto& input = setRel.inputs(i); + if (!validate(input)) { + LOG_VALIDATION_MSG("ProjectRel input"); + return false; + } + } + if (!setRel.has_advanced_extension()) { + LOG_VALIDATION_MSG("Input types are expected in SetRel."); + return false; + } + const auto& extension = setRel.advanced_extension(); + TypePtr inputRowType; + std::vector> childrenTypes; + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType2(inputRowType, childrenTypes)) { + LOG_VALIDATION_MSG("Validation failed for input types in SetRel."); + return false; + } + std::vector childrenRowTypes; + for (auto i = 0; i < childrenTypes.size(); ++i) { + auto& types = childrenTypes.at(i); + std::vector names; + names.reserve(types.size()); + for (auto colIdx = 0; colIdx < types.size(); colIdx++) { + names.emplace_back(SubstraitParser::makeNodeName(i, colIdx)); + } + childrenRowTypes.push_back(std::make_shared(std::move(names), std::move(types))); + } + + for (auto i = 1; i < childrenRowTypes.size(); ++i) { + if (!(childrenRowTypes[i]->equivalent(*childrenRowTypes[0]))) { + LOG_VALIDATION_MSG( + "All sources of the Set operation must have the same output type: " + childrenRowTypes[i]->toString() + + " vs. " + childrenRowTypes[0]->toString()); + return false; + } + } + return true; + } + default: + LOG_VALIDATION_MSG("Unsupported SetRel op: " + std::to_string(setRel.op())); + return false; + } +} + bool SubstraitToVeloxPlanValidator::validate(const ::substrait::SortRel& sortRel) { if (sortRel.has_input() && !validate(sortRel.input())) { return false; @@ -769,8 +848,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::SortRel& sortRel } const auto& extension = sortRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in SortRel."); return false; } @@ -822,8 +902,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::ProjectRel& proj return false; } const auto& extension = projectRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in ProjectRel."); return false; } @@ -865,8 +946,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::FilterRel& filte return false; } const auto& extension = filterRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in FilterRel."); return false; } @@ -938,8 +1020,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::JoinRel& joinRel } const auto& extension = joinRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in JoinRel."); return false; } @@ -991,8 +1074,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::CrossRel& crossR } const auto& extension = crossRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { logValidateMsg("Native validation failed due to: Validation failed for input types in CrossRel"); return false; } @@ -1070,11 +1154,13 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::AggregateRel& ag // Validate input types. if (aggRel.has_advanced_extension()) { + TypePtr inputRowType; std::vector types; const auto& extension = aggRel.advanced_extension(); // Aggregate always has advanced extension for streaming aggregate optimization, // but only some of them have enhancement for validation. - if (extension.has_enhancement() && !validateInputTypes(extension, types)) { + if (extension.has_enhancement() && + (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types))) { LOG_VALIDATION_MSG("Validation failed for input types in AggregateRel."); return false; } @@ -1266,7 +1352,10 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::Rel& rel) { return validate(rel.write()); } else if (rel.has_windowgrouplimit()) { return validate(rel.windowgrouplimit()); + } else if (rel.has_set()) { + return validate(rel.set()); } else { + LOG_VALIDATION_MSG("Unsupported relation type: " + rel.GetTypeName()); return false; } } diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h index 1fe174928fd9..0c8d882ca031 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h @@ -61,6 +61,9 @@ class SubstraitToVeloxPlanValidator { /// Used to validate whether the computing of this WindowGroupLimit is supported. bool validate(const ::substrait::WindowGroupLimitRel& windowGroupLimitRel); + /// Used to validate whether the computing of this Set is supported. + bool validate(const ::substrait::SetRel& setRel); + /// Used to validate whether the computing of this Aggregation is supported. bool validate(const ::substrait::AggregateRel& aggRel); @@ -103,9 +106,17 @@ class SubstraitToVeloxPlanValidator { std::vector validateLog_; - /// Used to get types from advanced extension and validate them. - bool validateInputTypes(const ::substrait::extensions::AdvancedExtension& extension, std::vector& types); + /// Used to get types from advanced extension and validate them, then convert to a Velox type that has arbitrary + /// levels of nesting. + bool parseVeloxType(const ::substrait::extensions::AdvancedExtension& extension, TypePtr& out); + + /// Flattens a Velox type with single level of nesting into a std::vector of child types. + bool flattenVeloxType1(const TypePtr& type, std::vector& out); + + /// Flattens a Velox type with two level of nesting into a dual-nested std::vector of child types. + bool flattenVeloxType2(const TypePtr& type, std::vector>& out); + /// Validate aggregate rel. bool validateAggRelFunctionType(const ::substrait::AggregateRel& substraitAgg); /// Validate the round scalar function. diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java index def1dca0a028..7d1931180847 100644 --- a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java @@ -27,6 +27,7 @@ import io.substrait.proto.CrossRel; import io.substrait.proto.JoinRel; +import io.substrait.proto.SetRel; import io.substrait.proto.SortField; import org.apache.spark.sql.catalyst.expressions.Attribute; @@ -317,4 +318,20 @@ public static RelNode makeGenerateRel( context.registerRelToOperator(operatorId); return new GenerateRelNode(input, generator, childOutput, extensionNode, outer); } + + public static RelNode makeSetRel( + List inputs, SetRel.SetOp setOp, SubstraitContext context, Long operatorId) { + context.registerRelToOperator(operatorId); + return new SetRelNode(inputs, setOp); + } + + public static RelNode makeSetRel( + List inputs, + SetRel.SetOp setOp, + AdvancedExtensionNode extensionNode, + SubstraitContext context, + Long operatorId) { + context.registerRelToOperator(operatorId); + return new SetRelNode(inputs, setOp, extensionNode); + } } diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/SetRelNode.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/SetRelNode.java new file mode 100644 index 000000000000..ddcfb1701dd1 --- /dev/null +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/SetRelNode.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.substrait.rel; + +import org.apache.gluten.substrait.extensions.AdvancedExtensionNode; + +import io.substrait.proto.Rel; +import io.substrait.proto.RelCommon; +import io.substrait.proto.SetRel; + +import java.io.Serializable; +import java.util.List; + +public class SetRelNode implements RelNode, Serializable { + private final List inputs; + private final SetRel.SetOp setOp; + private final AdvancedExtensionNode extensionNode; + + public SetRelNode(List inputs, SetRel.SetOp setOp, AdvancedExtensionNode extensionNode) { + this.inputs = inputs; + this.setOp = setOp; + this.extensionNode = extensionNode; + } + + public SetRelNode(List inputs, SetRel.SetOp setOp) { + this(inputs, setOp, null); + } + + @Override + public Rel toProtobuf() { + final RelCommon.Builder relCommonBuilder = RelCommon.newBuilder(); + relCommonBuilder.setDirect(RelCommon.Direct.newBuilder()); + final SetRel.Builder setBuilder = SetRel.newBuilder(); + setBuilder.setCommon(relCommonBuilder.build()); + if (inputs != null) { + for (RelNode input : inputs) { + setBuilder.addInputs(input.toProtobuf()); + } + } + setBuilder.setOp(setOp); + if (extensionNode != null) { + setBuilder.setAdvancedExtension(extensionNode.toProtobuf()); + } + final Rel.Builder builder = Rel.newBuilder(); + builder.setSet(setBuilder.build()); + return builder.build(); + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala index c67d4b5f8876..453cfab4e487 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala @@ -126,6 +126,10 @@ trait MetricsApi extends Serializable { def genSampleTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater + def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] + + def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater + def genColumnarInMemoryTableMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index f9755605cab2..ac8e610956dc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -261,10 +261,11 @@ abstract class ProjectExecTransformerBase(val list: Seq[NamedExpression], val in } } -// An alternatives for UnionExec. +// An alternative for UnionExec. case class ColumnarUnionExec(children: Seq[SparkPlan]) extends ValidatablePlan { children.foreach { case w: WholeStageTransformer => + // FIXME: Avoid such practice for plan immutability. w.setOutputSchemaForPlan(output) case _ => } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/UnionExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/UnionExecTransformer.scala new file mode 100644 index 000000000000..d27558746a40 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/UnionExecTransformer.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.expression.ConverterUtils +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.metrics.MetricsUpdater +import org.apache.gluten.substrait.`type`.TypeBuilder +import org.apache.gluten.substrait.SubstraitContext +import org.apache.gluten.substrait.extensions.ExtensionBuilder +import org.apache.gluten.substrait.rel.{RelBuilder, RelNode} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.{SparkPlan, UnionExec} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +import io.substrait.proto.SetRel.SetOp + +import scala.collection.JavaConverters._ + +/** Transformer for UnionExec. Note: Spark's UnionExec represents a SQL UNION ALL. */ +case class UnionExecTransformer(children: Seq[SparkPlan]) extends TransformSupport { + private val union = UnionExec(children) + + // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. + @transient override lazy val metrics: Map[String, SQLMetric] = + BackendsApiManager.getMetricsApiInstance.genUnionTransformerMetrics(sparkContext) + + override def output: Seq[Attribute] = union.output + + override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = children.flatMap(getColumnarInputRDDs) + + override def metricsUpdater(): MetricsUpdater = + BackendsApiManager.getMetricsApiInstance.genUnionTransformerMetricsUpdater(metrics) + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = + copy(children = newChildren) + + override protected def doValidateInternal(): ValidationResult = { + val context = new SubstraitContext + val operatorId = context.nextOperatorId(this.nodeName) + val relNode = getRelNode(context, operatorId, children.map(_.output), null, true) + doNativeValidation(context, relNode) + } + + override protected def doTransform(context: SubstraitContext): TransformContext = { + val childrenCtx = children.map(_.asInstanceOf[TransformSupport].transform(context)) + val operatorId = context.nextOperatorId(this.nodeName) + val relNode = + getRelNode(context, operatorId, children.map(_.output), childrenCtx.map(_.root), false) + TransformContext(output, relNode) + } + + private def getRelNode( + context: SubstraitContext, + operatorId: Long, + inputAttributes: Seq[Seq[Attribute]], + inputs: Seq[RelNode], + validation: Boolean): RelNode = { + if (validation) { + // Use the second level of nesting to represent N way inputs. + val inputTypeNodes = + inputAttributes.map( + attributes => + attributes.map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)).asJava) + val extensionNode = ExtensionBuilder.makeAdvancedExtension( + BackendsApiManager.getTransformerApiInstance.packPBMessage( + TypeBuilder + .makeStruct( + false, + inputTypeNodes.map(nodes => TypeBuilder.makeStruct(false, nodes)).asJava) + .toProtobuf)) + return RelBuilder.makeSetRel( + inputs.asJava, + SetOp.SET_OP_UNION_ALL, + extensionNode, + context, + operatorId) + } + RelBuilder.makeSetRel(inputs.asJava, SetOp.SET_OP_UNION_ALL, context, operatorId) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/UnionTransformerRule.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/UnionTransformerRule.scala new file mode 100644 index 000000000000..f0eea08018dd --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/UnionTransformerRule.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.execution.{ColumnarUnionExec, UnionExecTransformer} + +import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +/** + * Replace ColumnarUnionExec with UnionExecTransformer if possible. + * + * The rule is not included in [[org.apache.gluten.extension.columnar.heuristic.HeuristicTransform]] + * or [[org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform]] because it relies on + * children's output partitioning to be fully provided. + */ +case class UnionTransformerRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + if (!GlutenConfig.getConf.enableNativeUnion) { + return plan + } + plan.transformUp { + case plan: ColumnarUnionExec => + val transformer = UnionExecTransformer(plan.children) + if (sameNumPartitions(plan.children) && validate(transformer)) { + transformer + } else { + plan + } + } + } + + private def sameNumPartitions(plans: Seq[SparkPlan]): Boolean = { + val partitioning = plans.map(_.outputPartitioning) + if (partitioning.exists(p => p.isInstanceOf[UnknownPartitioning])) { + return false + } + val numPartitions = plans.map(_.outputPartitioning.numPartitions) + numPartitions.forall(_ == numPartitions.head) + } + + private def validate(union: UnionExecTransformer): Boolean = { + union.doValidate().ok() + } +} diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala index fd250834d078..08081fadb5f9 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala @@ -176,25 +176,39 @@ abstract class WholeStageTransformerSuite result } + protected def compareResultsAgainstVanillaSpark( + sql: String, + compareResult: Boolean = true, + customCheck: DataFrame => Unit, + noFallBack: Boolean = true, + cache: Boolean = false): DataFrame = { + compareDfResultsAgainstVanillaSpark( + () => spark.sql(sql), + compareResult, + customCheck, + noFallBack, + cache) + } + /** * run a query with native engine as well as vanilla spark then compare the result set for * correctness check */ - protected def compareResultsAgainstVanillaSpark( - sqlStr: String, + protected def compareDfResultsAgainstVanillaSpark( + dataframe: () => DataFrame, compareResult: Boolean = true, customCheck: DataFrame => Unit, noFallBack: Boolean = true, cache: Boolean = false): DataFrame = { var expected: Seq[Row] = null withSQLConf(vanillaSparkConfs(): _*) { - val df = spark.sql(sqlStr) + val df = dataframe() expected = df.collect() } - // By default we will fallabck complex type scan but here we should allow + // By default, we will fallback complex type scan but here we should allow // to test support of complex type spark.conf.set("spark.gluten.sql.complexType.scan.fallback.enabled", "false"); - val df = spark.sql(sqlStr) + val df = dataframe() if (cache) { df.cache() } @@ -239,7 +253,12 @@ abstract class WholeStageTransformerSuite noFallBack: Boolean = true, cache: Boolean = false)(customCheck: DataFrame => Unit): DataFrame = { - compareResultsAgainstVanillaSpark(sqlStr, compareResult, customCheck, noFallBack, cache) + compareDfResultsAgainstVanillaSpark( + () => spark.sql(sqlStr), + compareResult, + customCheck, + noFallBack, + cache) } /** @@ -256,8 +275,8 @@ abstract class WholeStageTransformerSuite customCheck: DataFrame => Unit, noFallBack: Boolean = true, compareResult: Boolean = true): Unit = - compareResultsAgainstVanillaSpark( - tpchSQL(queryNum, tpchQueries), + compareDfResultsAgainstVanillaSpark( + () => spark.sql(tpchSQL(queryNum, tpchQueries)), compareResult, customCheck, noFallBack) diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index f643ad7eed0e..9ae4c0ce90eb 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -86,6 +86,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableColumnarUnion: Boolean = conf.getConf(COLUMNAR_UNION_ENABLED) + def enableNativeUnion: Boolean = conf.getConf(NATIVE_UNION_ENABLED) + def enableColumnarExpand: Boolean = conf.getConf(COLUMNAR_EXPAND_ENABLED) def enableColumnarBroadcastExchange: Boolean = conf.getConf(COLUMNAR_BROADCAST_EXCHANGE_ENABLED) @@ -1022,6 +1024,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val NATIVE_UNION_ENABLED = + buildConf("spark.gluten.sql.native.union") + .internal() + .doc("Enable or disable native union where computation is completely offloaded to backend.") + .booleanConf + .createWithDefault(false) + val COLUMNAR_EXPAND_ENABLED = buildConf("spark.gluten.sql.columnar.expand") .internal() From 11d658ec4541fcf2c470c64e745a0c8c79ecfb32 Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Tue, 3 Dec 2024 11:44:49 +0800 Subject: [PATCH 181/211] remove least and greatest function (#8091) --- .../Functions/FunctionGreatestLeast.h | 80 ------------------- .../Functions/SparkFunctionGreatest.cpp | 38 --------- .../Functions/SparkFunctionLeast.cpp | 38 --------- .../CommonScalarFunctionParser.cpp | 4 +- .../clickhouse/ClickHouseTestSettings.scala | 2 - .../clickhouse/ClickHouseTestSettings.scala | 2 - .../clickhouse/ClickHouseTestSettings.scala | 2 - .../clickhouse/ClickHouseTestSettings.scala | 2 - 8 files changed, 2 insertions(+), 166 deletions(-) delete mode 100644 cpp-ch/local-engine/Functions/FunctionGreatestLeast.h delete mode 100644 cpp-ch/local-engine/Functions/SparkFunctionGreatest.cpp delete mode 100644 cpp-ch/local-engine/Functions/SparkFunctionLeast.cpp diff --git a/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h b/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h deleted file mode 100644 index e9b66df84ef0..000000000000 --- a/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#pragma once -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} -} -namespace local_engine -{ -template -class FunctionGreatestestLeast : public DB::FunctionLeastGreatestGeneric -{ -public: - bool useDefaultImplementationForNulls() const override { return false; } - virtual String getName() const = 0; - -private: - DB::DataTypePtr getReturnTypeImpl(const DB::DataTypes & types) const override - { - if (types.empty()) - throw DB::Exception(DB::ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} cannot be called without arguments", getName()); - return makeNullable(getLeastSupertype(types)); - } - - DB::ColumnPtr executeImpl(const DB::ColumnsWithTypeAndName & arguments, const DB::DataTypePtr & result_type, size_t input_rows_count) const override - { - size_t num_arguments = arguments.size(); - DB::Columns converted_columns(num_arguments); - for (size_t arg = 0; arg < num_arguments; ++arg) - converted_columns[arg] = castColumn(arguments[arg], result_type)->convertToFullColumnIfConst(); - auto result_column = result_type->createColumn(); - result_column->reserve(input_rows_count); - for (size_t row_num = 0; row_num < input_rows_count; ++row_num) - { - size_t best_arg = 0; - for (size_t arg = 1; arg < num_arguments; ++arg) - { - if constexpr (kind == DB::LeastGreatest::Greatest) - { - auto cmp_result = converted_columns[arg]->compareAt(row_num, row_num, *converted_columns[best_arg], -1); - if (cmp_result > 0) - best_arg = arg; - } - else - { - auto cmp_result = converted_columns[arg]->compareAt(row_num, row_num, *converted_columns[best_arg], 1); - if (cmp_result < 0) - best_arg = arg; - } - } - result_column->insertFrom(*converted_columns[best_arg], row_num); - } - return result_column; - } -}; - -} diff --git a/cpp-ch/local-engine/Functions/SparkFunctionGreatest.cpp b/cpp-ch/local-engine/Functions/SparkFunctionGreatest.cpp deleted file mode 100644 index 920fe1b9c9cc..000000000000 --- a/cpp-ch/local-engine/Functions/SparkFunctionGreatest.cpp +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#include - -namespace local_engine -{ -class SparkFunctionGreatest : public FunctionGreatestestLeast -{ -public: - static constexpr auto name = "sparkGreatest"; - static DB::FunctionPtr create(DB::ContextPtr) { return std::make_shared(); } - SparkFunctionGreatest() = default; - ~SparkFunctionGreatest() override = default; - String getName() const override - { - return name; - } -}; - -REGISTER_FUNCTION(SparkGreatest) -{ - factory.registerFunction(); -} -} diff --git a/cpp-ch/local-engine/Functions/SparkFunctionLeast.cpp b/cpp-ch/local-engine/Functions/SparkFunctionLeast.cpp deleted file mode 100644 index 70aafdf07209..000000000000 --- a/cpp-ch/local-engine/Functions/SparkFunctionLeast.cpp +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#include - -namespace local_engine -{ -class SparkFunctionLeast : public FunctionGreatestestLeast -{ -public: - static constexpr auto name = "sparkLeast"; - static DB::FunctionPtr create(DB::ContextPtr) { return std::make_shared(); } - SparkFunctionLeast() = default; - ~SparkFunctionLeast() override = default; - String getName() const override - { - return name; - } -}; - -REGISTER_FUNCTION(SparkLeast) -{ - factory.registerFunction(); -} -} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp index ec8b4e0d12bf..e4a56194c171 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp @@ -98,8 +98,8 @@ REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Unhex, unhex, unhex); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Hypot, hypot, hypot); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Sign, sign, sign); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Radians, radians, radians); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Greatest, greatest, sparkGreatest); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Least, least, sparkLeast); +REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Greatest, greatest, greatest); +REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Least, least, least); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Rand, rand, randCanonical); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Bin, bin, sparkBin); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Rint, rint, sparkRint); diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 2eb5bd11ffbe..36d5b5177c6b 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -554,8 +554,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index a7bf5d4da903..b9bf4e1ac40f 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -570,8 +570,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index b7e3905740fb..a407c5d68247 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -513,8 +513,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 8ce145735dc3..9c22af0434af 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -513,8 +513,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") From 4b0a1c10939fa1314e5d892173dd8ed9afb3441e Mon Sep 17 00:00:00 2001 From: Zhen Li <10524738+zhli1142015@users.noreply.github.com> Date: Tue, 3 Dec 2024 11:54:43 +0800 Subject: [PATCH 182/211] [VL] Enable Spark JsonExpressionsSuite (#8099) --- .../gluten/utils/velox/VeloxTestSettings.scala | 16 ++++++++++++++++ .../gluten/utils/velox/VeloxTestSettings.scala | 18 +++++++++++++++++- .../gluten/utils/velox/VeloxTestSettings.scala | 18 +++++++++++++++++- .../gluten/utils/velox/VeloxTestSettings.scala | 18 +++++++++++++++++- 4 files changed, 67 insertions(+), 3 deletions(-) diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 276a8a0f7af2..15495270a189 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -364,6 +364,22 @@ class VeloxTestSettings extends BackendTestSettings { // Rewrite the following two tests in GlutenDatasetSuite. .exclude("dropDuplicates: columns with same column name") .exclude("groupBy.as") + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // Velox does not support single quotes in get_json_object function. .exclude("function get_json_object - support single quotes") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 08fab69c849b..407b9c8b95cc 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -20,7 +20,7 @@ import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} import org.apache.spark.GlutenSortShuffleSuite import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{GlutenAnsiCastSuiteWithAnsiModeOff, GlutenAnsiCastSuiteWithAnsiModeOn, GlutenArithmeticExpressionSuite, GlutenBitwiseExpressionsSuite, GlutenCastSuite, GlutenCastSuiteWithAnsiModeOn, GlutenCollectionExpressionsSuite, GlutenComplexTypeSuite, GlutenConditionalExpressionSuite, GlutenDateExpressionsSuite, GlutenDecimalExpressionSuite, GlutenDecimalPrecisionSuite, GlutenHashExpressionsSuite, GlutenHigherOrderFunctionsSuite, GlutenIntervalExpressionsSuite, GlutenLiteralExpressionSuite, GlutenMathExpressionsSuite, GlutenMiscExpressionsSuite, GlutenNondeterministicSuite, GlutenNullExpressionsSuite, GlutenPredicateSuite, GlutenRandomSuite, GlutenRegexpExpressionsSuite, GlutenSortOrderExpressionsSuite, GlutenStringExpressionsSuite, GlutenTryCastSuite} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector._ import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution._ @@ -142,6 +142,22 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenHashExpressionsSuite] enableSuite[GlutenHigherOrderFunctionsSuite] enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // Velox does not support single quotes in get_json_object function. .exclude("function get_json_object - support single quotes") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index df79b663bcbe..dbb01fbe7067 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -20,7 +20,7 @@ import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} import org.apache.spark.GlutenSortShuffleSuite import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{GlutenArithmeticExpressionSuite, GlutenBitwiseExpressionsSuite, GlutenCastSuite, GlutenCollectionExpressionsSuite, GlutenComplexTypeSuite, GlutenConditionalExpressionSuite, GlutenDateExpressionsSuite, GlutenDecimalExpressionSuite, GlutenDecimalPrecisionSuite, GlutenHashExpressionsSuite, GlutenHigherOrderFunctionsSuite, GlutenIntervalExpressionsSuite, GlutenLiteralExpressionSuite, GlutenMathExpressionsSuite, GlutenMiscExpressionsSuite, GlutenNondeterministicSuite, GlutenNullExpressionsSuite, GlutenPredicateSuite, GlutenRandomSuite, GlutenRegexpExpressionsSuite, GlutenSortOrderExpressionsSuite, GlutenStringExpressionsSuite, GlutenTryEvalSuite} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector.{GlutenDataSourceV2DataFrameSessionCatalogSuite, GlutenDataSourceV2DataFrameSuite, GlutenDataSourceV2FunctionSuite, GlutenDataSourceV2SQLSessionCatalogSuite, GlutenDataSourceV2SQLSuiteV1Filter, GlutenDataSourceV2SQLSuiteV2Filter, GlutenDataSourceV2Suite, GlutenDeleteFromTableSuite, GlutenDeltaBasedDeleteFromTableSuite, GlutenFileDataSourceV2FallBackSuite, GlutenGroupBasedDeleteFromTableSuite, GlutenKeyGroupedPartitioningSuite, GlutenLocalScanSuite, GlutenMetadataColumnSuite, GlutenSupportsCatalogOptionsSuite, GlutenTableCapabilityCheckSuite, GlutenWriteDistributionAndOrderingSuite} import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution.{FallbackStrategiesSuite, GlutenBroadcastExchangeSuite, GlutenCoalesceShufflePartitionsSuite, GlutenExchangeSuite, GlutenLocalBroadcastExchangeSuite, GlutenReplaceHashWithSortAggSuite, GlutenReuseExchangeAndSubquerySuite, GlutenSameResultSuite, GlutenSortSuite, GlutenSQLAggregateFunctionSuite, GlutenSQLWindowFunctionSuite, GlutenTakeOrderedAndProjectSuite} @@ -120,6 +120,22 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenHashExpressionsSuite] enableSuite[GlutenHigherOrderFunctionsSuite] enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // Velox does not support single quotes in get_json_object function. .exclude("function get_json_object - support single quotes") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index cb2151fe4698..f5a1a076956e 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -20,7 +20,7 @@ import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} import org.apache.spark.GlutenSortShuffleSuite import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{GlutenArithmeticExpressionSuite, GlutenBitwiseExpressionsSuite, GlutenCastSuite, GlutenCollectionExpressionsSuite, GlutenComplexTypeSuite, GlutenConditionalExpressionSuite, GlutenDateExpressionsSuite, GlutenDecimalExpressionSuite, GlutenDecimalPrecisionSuite, GlutenHashExpressionsSuite, GlutenHigherOrderFunctionsSuite, GlutenIntervalExpressionsSuite, GlutenLiteralExpressionSuite, GlutenMathExpressionsSuite, GlutenMiscExpressionsSuite, GlutenNondeterministicSuite, GlutenNullExpressionsSuite, GlutenPredicateSuite, GlutenRandomSuite, GlutenRegexpExpressionsSuite, GlutenSortOrderExpressionsSuite, GlutenStringExpressionsSuite, GlutenTryEvalSuite} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector._ import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution._ @@ -121,6 +121,22 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenHashExpressionsSuite] enableSuite[GlutenHigherOrderFunctionsSuite] enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // * in get_json_object expression not supported in velox .exclude("SPARK-42782: Hive compatibility check for get_json_object") From 2346584a72cae67494a073a4f0141386bad723db Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Tue, 3 Dec 2024 15:57:26 +0800 Subject: [PATCH 183/211] [GLUTEN-7028][CH][Part-11] Support write parquet files with bucket (#8052) * [GLUTEN-7028][CH] Support write parquet files with bucket * [GLUTEN-7028][CH] Fix comment --- .../execution/FileDeltaColumnarWrite.scala | 6 +- .../backendsapi/clickhouse/CHBackend.scala | 11 +- .../clickhouse/CHIteratorApi.scala | 13 +-- .../backendsapi/clickhouse/CHRuleApi.scala | 1 + .../clickhouse/RuntimeSettings.scala | 6 - .../extension/WriteFilesWithBucketValue.scala | 76 ++++++++++++ .../spark/sql/execution/CHColumnarWrite.scala | 46 ++++++-- ...lutenClickHouseNativeWriteTableSuite.scala | 15 +-- .../Parser/RelParsers/WriteRelParser.cpp | 14 +-- .../Parser/RelParsers/WriteRelParser.h | 1 - .../Storages/MergeTree/SparkMergeTreeSink.h | 6 + .../Storages/Output/NormalFileWriter.cpp | 4 +- .../Storages/Output/NormalFileWriter.h | 110 +++++++++++++++--- .../Storages/Output/OutputFormatFile.cpp | 1 - .../tests/gtest_write_pipeline.cpp | 6 +- 15 files changed, 242 insertions(+), 74 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala index 784614152f63..bf6b0c0074dc 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala @@ -108,13 +108,15 @@ case class FileDeltaColumnarWrite( * {{{ * part-00000-7d672b28-c079-4b00-bb0a-196c15112918-c000.snappy.parquet * => - * part-00000-{}.snappy.parquet + * part-00000-{id}.snappy.parquet * }}} */ val guidPattern = """.*-([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12})(?:-c(\d+)\..*)?$""".r val fileNamePattern = - guidPattern.replaceAllIn(writeFileName, m => writeFileName.replace(m.group(1), "{}")) + guidPattern.replaceAllIn( + writeFileName, + m => writeFileName.replace(m.group(1), FileNamePlaceHolder.ID)) logDebug(s"Native staging write path: $writePath and with pattern: $fileNamePattern") val settings = diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index c6c8acf7050e..e5eb91b69bdb 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -246,20 +246,11 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } } - def validateBucketSpec(): Option[String] = { - if (bucketSpec.nonEmpty) { - Some("Unsupported native write: bucket write is not supported.") - } else { - None - } - } - validateCompressionCodec() .orElse(validateFileFormat()) .orElse(validateFieldMetadata()) .orElse(validateDateTypes()) - .orElse(validateWriteFilesOptions()) - .orElse(validateBucketSpec()) match { + .orElse(validateWriteFilesOptions()) match { case Some(reason) => ValidationResult.failed(reason) case _ => ValidationResult.succeeded } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala index ff268b95d8de..878e27a5b8c2 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala @@ -26,7 +26,7 @@ import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.plan.PlanNode import org.apache.gluten.substrait.rel._ import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat -import org.apache.gluten.vectorized.{BatchIterator, CHNativeExpressionEvaluator, CloseableCHColumnBatchIterator, NativeExpressionEvaluator} +import org.apache.gluten.vectorized.{BatchIterator, CHNativeExpressionEvaluator, CloseableCHColumnBatchIterator} import org.apache.spark.{InterruptibleIterator, SparkConf, TaskContext} import org.apache.spark.affinity.CHAffinity @@ -322,17 +322,6 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { createNativeIterator(splitInfoByteArray, wsPlan, materializeInput, inputIterators)) } - /** - * This function used to inject the staging write path before initializing the native plan.Only - * used in a pipeline model (spark 3.5) for writing parquet or orc files. - */ - override def injectWriteFilesTempPath(path: String, fileName: String): Unit = { - val settings = - Map( - RuntimeSettings.TASK_WRITE_TMP_DIR.key -> path, - RuntimeSettings.TASK_WRITE_FILENAME.key -> fileName) - NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) - } } class CollectMetricIterator( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index edf7a4802535..98cfa0e7547b 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -79,6 +79,7 @@ object CHRuleApi { injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) injector.injectPreTransform(c => FallbackBroadcastHashJoin.apply(c.session)) injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session)) + injector.injectPreTransform(_ => WriteFilesWithBucketValue) // Legacy: The legacy transform rule. val validatorBuilder: GlutenConfig => Validator = conf => diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala index b59bb32392df..c2747cf1eb53 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala @@ -35,12 +35,6 @@ object RuntimeSettings { .stringConf .createWithDefault("") - val TASK_WRITE_FILENAME = - buildConf(runtimeSettings("gluten.task_write_filename")) - .doc("The temporary file name for writing data") - .stringConf - .createWithDefault("") - val TASK_WRITE_FILENAME_PATTERN = buildConf(runtimeSettings("gluten.task_write_filename_pattern")) .doc("The pattern to generate file name for writing delta parquet in spark 3.5") diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala new file mode 100644 index 000000000000..8ab78dcff9bc --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.sql.catalyst.expressions.{Alias, BitwiseAnd, Expression, HiveHash, Literal, Pmod} +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.WriteFilesExec + +/** + * Wrap with bucket value to specify the bucket file name in native write. Native writer will remove + * this value in the final output. + */ +object WriteFilesWithBucketValue extends Rule[SparkPlan] { + + val optionForHiveCompatibleBucketWrite = "__hive_compatible_bucketed_table_insertion__" + + override def apply(plan: SparkPlan): SparkPlan = { + if ( + GlutenConfig.getConf.enableGluten + && GlutenConfig.getConf.enableNativeWriter.getOrElse(false) + ) { + plan.transformDown { + case writeFiles: WriteFilesExec if writeFiles.bucketSpec.isDefined => + val bucketIdExp = getWriterBucketIdExp(writeFiles) + val wrapBucketValue = ProjectExec( + writeFiles.child.output :+ Alias(bucketIdExp, "__bucket_value__")(), + writeFiles.child) + writeFiles.copy(child = wrapBucketValue) + } + } else { + plan + } + } + + private def getWriterBucketIdExp(writeFilesExec: WriteFilesExec): Expression = { + val partitionColumns = writeFilesExec.partitionColumns + val outputColumns = writeFilesExec.child.output + val dataColumns = outputColumns.filterNot(partitionColumns.contains) + val bucketSpec = writeFilesExec.bucketSpec.get + val bucketColumns = bucketSpec.bucketColumnNames.map(c => dataColumns.find(_.name == c).get) + if (writeFilesExec.options.getOrElse(optionForHiveCompatibleBucketWrite, "false") == "true") { + val hashId = BitwiseAnd(HiveHash(bucketColumns), Literal(Int.MaxValue)) + Pmod(hashId, Literal(bucketSpec.numBuckets)) + // The bucket file name prefix is following Hive, Presto and Trino conversion, so this + // makes sure Hive bucketed table written by Spark, can be read by other SQL engines. + // + // Hive: `org.apache.hadoop.hive.ql.exec.Utilities#getBucketIdFromFile()`. + // Trino: `io.trino.plugin.hive.BackgroundHiveSplitLoader#BUCKET_PATTERNS`. + + } else { + // Spark bucketed table: use `HashPartitioning.partitionIdExpression` as bucket id + // expression, so that we can guarantee the data distribution is same between shuffle and + // bucketed data source, which enables us to only shuffle one side when join a bucketed + // table and a normal one. + HashPartitioning(bucketColumns, bucketSpec.numBuckets).partitionIdExpression + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala index 6c7877cc020b..1342e250430e 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala @@ -16,7 +16,8 @@ */ package org.apache.spark.sql.execution -import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings +import org.apache.gluten.vectorized.NativeExpressionEvaluator import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -25,11 +26,11 @@ import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker -import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, BasicWriteTaskStats, ExecutedWriteSummary, PartitioningUtils, WriteJobDescription, WriteTaskResult, WriteTaskStatsTracker} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.util.Utils import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobID, OutputCommitter, TaskAttemptContext, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl @@ -102,6 +103,12 @@ object CreateFileNameSpec { } } +// More details in local_engine::FileNameGenerator in NormalFileWriter.cpp +object FileNamePlaceHolder { + val ID = "{id}" + val BUCKET = "{bucket}" +} + /** [[HadoopMapReduceAdapter]] for [[HadoopMapReduceCommitProtocol]]. */ case class HadoopMapReduceAdapter(sparkCommitter: HadoopMapReduceCommitProtocol) { private lazy val committer: OutputCommitter = { @@ -132,12 +139,26 @@ case class HadoopMapReduceAdapter(sparkCommitter: HadoopMapReduceCommitProtocol) GetFilename.invoke(sparkCommitter, taskContext, spec).asInstanceOf[String] } - def getTaskAttemptTempPathAndFilename( + def getTaskAttemptTempPathAndFilePattern( taskContext: TaskAttemptContext, description: WriteJobDescription): (String, String) = { val stageDir = newTaskAttemptTempPath(description.path) - val filename = getFilename(taskContext, CreateFileNameSpec(taskContext, description)) - (stageDir, filename) + + if (isBucketWrite(description)) { + val filePart = getFilename(taskContext, FileNameSpec("", "")) + val fileSuffix = CreateFileNameSpec(taskContext, description).suffix + (stageDir, s"${filePart}_${FileNamePlaceHolder.BUCKET}$fileSuffix") + } else { + val filename = getFilename(taskContext, CreateFileNameSpec(taskContext, description)) + (stageDir, filename) + } + } + + private def isBucketWrite(desc: WriteJobDescription): Boolean = { + // In Spark 3.2, bucketSpec is not defined, instead, it uses bucketIdExpression. + val bucketSpecField: Field = desc.getClass.getDeclaredField("bucketSpec") + bucketSpecField.setAccessible(true) + bucketSpecField.get(desc).asInstanceOf[Option[_]].isDefined } } @@ -234,10 +255,15 @@ case class HadoopMapReduceCommitProtocolWrite( * initializing the native plan and collect native write files metrics for each backend. */ override def doSetupNativeTask(): Unit = { - val (writePath, writeFileName) = - adapter.getTaskAttemptTempPathAndFilename(taskAttemptContext, description) - logDebug(s"Native staging write path: $writePath and file name: $writeFileName") - BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath, writeFileName) + val (writePath, writeFilePattern) = + adapter.getTaskAttemptTempPathAndFilePattern(taskAttemptContext, description) + logDebug(s"Native staging write path: $writePath and file pattern: $writeFilePattern") + + val settings = + Map( + RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath, + RuntimeSettings.TASK_WRITE_FILENAME_PATTERN.key -> writeFilePattern) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) } def doCollectNativeResult(stats: Seq[InternalRow]): Option[WriteTaskResult] = { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala index 03d27f33b1e1..16ed302a02f4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala @@ -553,7 +553,7 @@ class GlutenClickHouseNativeWriteTableSuite // spark write does not support bucketed table // https://issues.apache.org/jira/browse/SPARK-19256 val table_name = table_name_template.format(format) - writeAndCheckRead(origin_table, table_name, fields_.keys.toSeq, isSparkVersionLE("3.3")) { + writeAndCheckRead(origin_table, table_name, fields_.keys.toSeq) { fields => spark .table("origin_table") @@ -589,8 +589,9 @@ class GlutenClickHouseNativeWriteTableSuite ("byte_field", "byte"), ("boolean_field", "boolean"), ("decimal_field", "decimal(23,12)"), - ("date_field", "date"), - ("timestamp_field", "timestamp") + ("date_field", "date") + // ("timestamp_field", "timestamp") + // FIXME https://github.com/apache/incubator-gluten/issues/8053 ) val origin_table = "origin_table" withSource(genTestData(), origin_table) { @@ -598,7 +599,7 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) val testFields = fields.keys.toSeq - writeAndCheckRead(origin_table, table_name, testFields, isSparkVersionLE("3.3")) { + writeAndCheckRead(origin_table, table_name, testFields) { fields => spark .table(origin_table) @@ -658,7 +659,7 @@ class GlutenClickHouseNativeWriteTableSuite nativeWrite { format => val table_name = table_name_template.format(format) - writeAndCheckRead(origin_table, table_name, fields.keys.toSeq, isSparkVersionLE("3.3")) { + writeAndCheckRead(origin_table, table_name, fields.keys.toSeq) { fields => spark .table("origin_table") @@ -762,7 +763,7 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) { + withNativeWriteCheck(checkNative = true) { spark .range(10000000) .selectExpr("id", "cast('2020-01-01' as date) as p") @@ -798,7 +799,7 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) { + withNativeWriteCheck(checkNative = true) { spark .range(30000) .selectExpr("id", "cast(null as string) as p") diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp index 2dacb3991858..a76b4d398d97 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp @@ -56,10 +56,11 @@ DB::ProcessorPtr make_sink( const std::string & format_hint, const std::shared_ptr & stats) { - if (partition_by.empty()) + bool no_bucketed = !SparkPartitionedBaseSink::isBucketedWrite(input_header); + if (partition_by.empty() && no_bucketed) { return std::make_shared( - context, base_path, "", generator.generate(), format_hint, input_header, stats, DeltaStats{input_header.columns()}); + context, base_path, "", false, generator.generate(), format_hint, input_header, stats, DeltaStats{input_header.columns()}); } return std::make_shared( @@ -184,13 +185,10 @@ void addNormalFileWriterSinkTransform( if (write_settings.task_write_tmp_dir.empty()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject temp directory."); - if (write_settings.task_write_filename.empty() && write_settings.task_write_filename_pattern.empty()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name or file name pattern."); + if (write_settings.task_write_filename_pattern.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file pattern."); - FileNameGenerator generator{ - .pattern = write_settings.task_write_filename.empty(), - .filename_or_pattern - = write_settings.task_write_filename.empty() ? write_settings.task_write_filename_pattern : write_settings.task_write_filename}; + FileNameGenerator generator(write_settings.task_write_filename_pattern); auto stats = WriteStats::create(output, partitionCols); diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h index 0c9bc11f1f10..01e0dabaaa7d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h @@ -44,7 +44,6 @@ DB::Names collect_partition_cols(const DB::Block & header, const substrait::Name #define WRITE_RELATED_SETTINGS(M, ALIAS) \ M(String, task_write_tmp_dir, , "The temporary directory for writing data") \ - M(String, task_write_filename, , "The filename for writing data") \ M(String, task_write_filename_pattern, , "The pattern to generate file name for writing delta parquet in spark 3.5") DECLARE_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h index 38f574ea9872..b551d86d1d0c 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h @@ -278,6 +278,12 @@ class SparkMergeTreePartitionedFileSink final : public SparkPartitionedBaseSink return SparkMergeTreeSink::create( table, write_settings, context_->getGlobalContext(), {std::dynamic_pointer_cast(stats_)}); } + + // TODO implement with bucket + DB::SinkPtr createSinkForPartition(const String & partition_id, const String & bucket) override + { + return createSinkForPartition(partition_id); + } }; } diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index ad2e3abf7b52..2d70380a8959 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -30,6 +30,8 @@ using namespace DB; const std::string SubstraitFileSink::NO_PARTITION_ID{"__NO_PARTITION_ID__"}; const std::string SparkPartitionedBaseSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; +const std::string SparkPartitionedBaseSink::BUCKET_COLUMN_NAME{"__bucket_value__"}; +const std::vector FileNameGenerator::SUPPORT_PLACEHOLDERS{"{id}", "{bucket}"}; /// For Nullable(Map(K, V)) or Nullable(Array(T)), if the i-th row is null, we must make sure its nested data is empty. /// It is for ORC/Parquet writing compatiability. For more details, refer to @@ -168,7 +170,7 @@ void NormalFileWriter::write(DB::Block & block) const auto & preferred_schema = file->getPreferredSchema(); for (auto & column : block) { - if (column.name.starts_with("__bucket_value__")) + if (column.name.starts_with(SparkPartitionedBaseSink::BUCKET_COLUMN_NAME)) continue; const auto & preferred_column = preferred_schema.getByPosition(index++); diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index 8cfe079d92c5..998f8d624721 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -230,20 +230,57 @@ class WriteStats : public WriteStatsBase struct FileNameGenerator { - const bool pattern; - const std::string filename_or_pattern; + // Align with org.apache.spark.sql.execution.FileNamePlaceHolder + static const std::vector SUPPORT_PLACEHOLDERS; + // Align with placeholders above + const std::vector need_to_replace; + const std::string file_pattern; + + FileNameGenerator(const std::string & file_pattern) + : file_pattern(file_pattern), need_to_replace(compute_need_to_replace(file_pattern)) + { + } + + std::vector compute_need_to_replace(const std::string & file_pattern) + { + std::vector result; + for(const std::string& placeholder: SUPPORT_PLACEHOLDERS) + { + if (file_pattern.find(placeholder) != std::string::npos) + result.push_back(true); + else + result.push_back(false); + } + return result; + } + + std::string generate(const std::string & bucket = "") const + { + std::string result = file_pattern; + if (need_to_replace[0]) // {id} + result = pattern_format(SUPPORT_PLACEHOLDERS[0], toString(DB::UUIDHelpers::generateV4())); + if (need_to_replace[1]) // {bucket} + result = pattern_format(SUPPORT_PLACEHOLDERS[1], bucket); + return result; + } - std::string generate() const + std::string pattern_format(const std::string & arg, const std::string & replacement) const { - if (pattern) - return fmt::vformat(filename_or_pattern, fmt::make_format_args(toString(DB::UUIDHelpers::generateV4()))); - return filename_or_pattern; + std::string format_str = file_pattern; + size_t pos = format_str.find(arg); + while (pos != std::string::npos) + { + format_str.replace(pos, arg.length(), replacement); + pos = format_str.find(arg, pos + arg.length()); + } + return format_str; } }; class SubstraitFileSink final : public DB::SinkToStorage { const std::string partition_id_; + const bool bucketed_write_; const std::string relative_path_; OutputFormatFilePtr format_file_; OutputFormatFile::OutputFormatPtr output_format_; @@ -265,6 +302,7 @@ class SubstraitFileSink final : public DB::SinkToStorage const DB::ContextPtr & context, const std::string & base_path, const std::string & partition_id, + const bool bucketed_write, const std::string & relative, const std::string & format_hint, const DB::Block & header, @@ -272,6 +310,7 @@ class SubstraitFileSink final : public DB::SinkToStorage const DeltaStats & delta_stats) : SinkToStorage(header) , partition_id_(partition_id.empty() ? NO_PARTITION_ID : partition_id) + , bucketed_write_(bucketed_write) , relative_path_(relative) , format_file_(createOutputFormatFile(context, makeAbsoluteFilename(base_path, partition_id, relative), header, format_hint)) , stats_(std::dynamic_pointer_cast(stats)) @@ -287,7 +326,18 @@ class SubstraitFileSink final : public DB::SinkToStorage delta_stats_.update(chunk); if (!output_format_) [[unlikely]] output_format_ = format_file_->createOutputFormat(); - output_format_->output->write(materializeBlock(getHeader().cloneWithColumns(chunk.detachColumns()))); + + const DB::Block & input_header = getHeader(); + if (bucketed_write_) + { + chunk.erase(input_header.columns() - 1); + const DB::ColumnsWithTypeAndName & cols = input_header.getColumnsWithTypeAndName(); + DB::ColumnsWithTypeAndName without_bucket_cols(cols.begin(), cols.end() - 1); + DB::Block without_bucket_header = DB::Block(without_bucket_cols); + output_format_->output->write(materializeBlock(without_bucket_header.cloneWithColumns(chunk.detachColumns()))); + } + else + output_format_->output->write(materializeBlock(input_header.cloneWithColumns(chunk.detachColumns()))); } void onFinish() override { @@ -303,11 +353,19 @@ class SubstraitFileSink final : public DB::SinkToStorage class SparkPartitionedBaseSink : public DB::PartitionedSink { - static const std::string DEFAULT_PARTITION_NAME; public: + static const std::string DEFAULT_PARTITION_NAME; + static const std::string BUCKET_COLUMN_NAME; + + static bool isBucketedWrite(const DB::Block & input_header) + { + return input_header.has(BUCKET_COLUMN_NAME) && + input_header.getPositionByName(BUCKET_COLUMN_NAME) == input_header.columns() - 1; + } + /// visible for UTs - static DB::ASTPtr make_partition_expression(const DB::Names & partition_columns) + static DB::ASTPtr make_partition_expression(const DB::Names & partition_columns, const DB::Block & input_header) { /// Parse the following expression into ASTs /// cancat('/col_name=', 'toString(col_name)') @@ -327,13 +385,35 @@ class SparkPartitionedBaseSink : public DB::PartitionedSink makeASTFunction("toString", DB::ASTs{column_ast}), std::make_shared(DEFAULT_PARTITION_NAME)}; arguments.emplace_back(makeASTFunction("ifNull", std::move(if_null_args))); } + if (isBucketedWrite(input_header)) + { + DB::ASTs args {std::make_shared("%05d"), std::make_shared(BUCKET_COLUMN_NAME)}; + arguments.emplace_back(DB::makeASTFunction("printf", std::move(args))); + } + assert(!arguments.empty()); + if (arguments.size() == 1) + return arguments[0]; return DB::makeASTFunction("concat", std::move(arguments)); } + DB::SinkPtr createSinkForPartition(const String & partition_id) override + { + if (bucketed_write_) + { + std::string bucket_val = partition_id.substr(partition_id.length() - 5, 5); + std::string real_partition_id = partition_id.substr(0, partition_id.length() - 5); + return createSinkForPartition(real_partition_id, bucket_val); + } + return createSinkForPartition(partition_id, ""); + } + + virtual DB::SinkPtr createSinkForPartition(const String & partition_id, const String & bucket) = 0; + protected: DB::ContextPtr context_; std::shared_ptr stats_; DeltaStats empty_delta_stats_; + bool bucketed_write_; public: SparkPartitionedBaseSink( @@ -341,9 +421,10 @@ class SparkPartitionedBaseSink : public DB::PartitionedSink const DB::Names & partition_by, const DB::Block & input_header, const std::shared_ptr & stats) - : PartitionedSink(make_partition_expression(partition_by), context, input_header) + : PartitionedSink(make_partition_expression(partition_by, input_header), context, input_header) , context_(context) , stats_(stats) + , bucketed_write_(isBucketedWrite(input_header)) , empty_delta_stats_(DeltaStats::create(input_header, partition_by)) { } @@ -353,6 +434,7 @@ class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink { const std::string base_path_; const FileNameGenerator generator_; + const DB::Block input_header_; const DB::Block sample_block_; const std::string format_hint_; @@ -370,18 +452,20 @@ class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink , base_path_(base_path) , generator_(generator) , sample_block_(sample_block) + , input_header_(input_header) , format_hint_(format_hint) { } - DB::SinkPtr createSinkForPartition(const String & partition_id) override + DB::SinkPtr createSinkForPartition(const String & partition_id, const String & bucket) override { assert(stats_); - std::string filename = generator_.generate(); + bool bucketed_write = !bucket.empty(); + std::string filename = bucketed_write ? generator_.generate(bucket) : generator_.generate(); const auto partition_path = fmt::format("{}/{}", partition_id, filename); validatePartitionKey(partition_path, true); return std::make_shared( - context_, base_path_, partition_id, filename, format_hint_, sample_block_, stats_, empty_delta_stats_); + context_, base_path_, partition_id, bucketed_write, filename, format_hint_, sample_block_, stats_, empty_delta_stats_); } String getName() const override { return "SubstraitPartitionedFileSink"; } }; diff --git a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp index 194d997ddf82..d5ed4309439c 100644 --- a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp @@ -61,7 +61,6 @@ Block OutputFormatFile::createHeaderWithPreferredSchema(const Block & header) ColumnWithTypeAndName column(preferred_column.type->createColumn(), preferred_column.type, preferred_column.name); columns.emplace_back(std::move(column)); } - assert(preferred_schema.columns() == index); return {std::move(columns)}; } diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp index 00f2da20c522..b764f62f542e 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -206,14 +206,14 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) TEST(WritePipeline, ComputePartitionedExpression) { const auto context = DB::Context::createCopy(QueryContext::globalContext()); - - auto partition_by = SubstraitPartitionedFileSink::make_partition_expression({"s_nationkey", "name"}); + + Block sample_block{{STRING(), "name"}, {UINT(), "s_nationkey"}}; + auto partition_by = SubstraitPartitionedFileSink::make_partition_expression({"s_nationkey", "name"}, sample_block); // auto partition_by = printColumn("s_nationkey"); ASTs arguments(1, partition_by); ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); - Block sample_block{{STRING(), "name"}, {UINT(), "s_nationkey"}}; auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); auto partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); From ca528c577a4ce8f9fa91678de219ac9353b308a5 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Tue, 3 Dec 2024 04:54:35 -0600 Subject: [PATCH 184/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241203) (#8125) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241203) * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/72651 * fix in clang19 * Fix UT due to https://github.com/ClickHouse/ClickHouse/pull/72651 * remove last '/' for HDFS_METADATA_PATH --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- .../GlutenClickHouseWholeStageTransformerSuite.scala | 4 ++-- .../GlutenClickHouseMergeTreeCacheDataSuite.scala | 10 +++++----- cpp-ch/clickhouse.version | 4 ++-- .../Functions/SparkFunctionDecimalBinaryArithmetic.cpp | 2 +- .../Storages/MergeTree/SparkStorageMergeTree.cpp | 1 + .../Storages/MergeTree/StorageMergeTreeFactory.h | 7 +++---- 6 files changed, 14 insertions(+), 14 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index 6ed106e2c75a..d7b277edfac5 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -42,14 +42,14 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu } val SPARK_DIR_NAME: String = sparkVersion.replace(".", "-") - val S3_METADATA_PATH = s"/tmp/metadata/s3/$SPARK_DIR_NAME/" + val S3_METADATA_PATH = s"/tmp/metadata/s3/$SPARK_DIR_NAME" val S3_CACHE_PATH = s"/tmp/s3_cache/$SPARK_DIR_NAME/" val S3_ENDPOINT = "s3://127.0.0.1:9000/" val MINIO_ENDPOINT: String = S3_ENDPOINT.replace("s3", "http") val BUCKET_NAME: String = SPARK_DIR_NAME val WHOLE_PATH: String = MINIO_ENDPOINT + BUCKET_NAME + "/" - val HDFS_METADATA_PATH = s"/tmp/metadata/hdfs/$SPARK_DIR_NAME/" + val HDFS_METADATA_PATH = s"/tmp/metadata/hdfs/$SPARK_DIR_NAME" val HDFS_CACHE_PATH = s"/tmp/hdfs_cache/$SPARK_DIR_NAME/" val HDFS_URL_ENDPOINT = "hdfs://127.0.0.1:8020" val HDFS_URL = s"$HDFS_URL_ENDPOINT/$SPARK_DIR_NAME" diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala index 14a8c68bdb51..a01c708adabe 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala @@ -134,7 +134,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) assertResult(22)(metaPath.list().length) assert(countFiles(dataPath) > initial_cache_files) @@ -238,7 +238,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) eventually(timeout(60.seconds), interval(2.seconds)) { assertResult(22)(metaPath.list().length) @@ -346,7 +346,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) assertResult(22)(metaPath.list().length) assert(countFiles(dataPath) > initial_cache_files) @@ -439,7 +439,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite val dataPath = new File(HDFS_CACHE_PATH) val initial_cache_files = countFiles(dataPath) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") val res1 = spark.sql(s"cache data select * from lineitem_mergetree_hdfs").collect() assertResult(true)(res1(0).getBoolean(0)) assertResult(1)(metaPath.list().length) @@ -539,7 +539,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) assertResult(22)(metaPath.list().length) assert(countFiles(dataPath) > initial_cache_files) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 565220d7867d..c3fda4bd2f55 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241130 -CH_COMMIT=d5d38588bd3 +CH_BRANCH=rebase_ch/20241203 +CH_COMMIT=473be0b9c50 diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp index f89943fc7a3d..8b79bae90d0c 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp @@ -487,7 +487,7 @@ class SparkFunctionDecimalBinaryArithmetic final : public IFunction right_generic, removeNullable(arguments[2].type).get(), [&](const auto & left, const auto & right, const auto & result) { - return (res = SparkDecimalBinaryOperation::template executeDecimal(arguments, left, right, result)) + return (res = SparkDecimalBinaryOperation::template executeDecimal<>(arguments, left, right, result)) != nullptr; }); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 5669489f5477..cbb40ae89dc0 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -502,6 +502,7 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( compression_codec, index_granularity_ptr, txn ? txn->tid : Tx::PrehistoricTID, + block.bytes(), false, false, context->getWriteSettings()); diff --git a/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h index 2c76fe51eda4..f6bc6cda4f81 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h +++ b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h @@ -30,8 +30,8 @@ class DataPartStorageHolder { public: DataPartStorageHolder(const DataPartPtr& data_part, const SparkStorageMergeTreePtr& storage) - : data_part_(data_part), - storage_(storage) + : storage_(storage), + data_part_(data_part) { } @@ -48,12 +48,11 @@ class DataPartStorageHolder ~DataPartStorageHolder() { storage_->removePartFromMemory(*data_part_); - // std::cerr << fmt::format("clean part {}", data_part_->name) << std::endl; } private: - DataPartPtr data_part_; SparkStorageMergeTreePtr storage_; + DataPartPtr data_part_; }; using DataPartStorageHolderPtr = std::shared_ptr; From 51ef77ea4ff329d143f8b620bef07ffa8658a0d8 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Tue, 3 Dec 2024 19:23:13 +0800 Subject: [PATCH 185/211] [GLUTEN-8046][VL] Fix GHA checkout issue on centos-7 for weekly build job (#8129) --- .github/workflows/velox_weekly.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/velox_weekly.yml b/.github/workflows/velox_weekly.yml index 666f0b033b62..fa8273d51a2f 100644 --- a/.github/workflows/velox_weekly.yml +++ b/.github/workflows/velox_weekly.yml @@ -38,7 +38,6 @@ jobs: runs-on: ubuntu-20.04 container: ${{ matrix.os }} steps: - - uses: actions/checkout@v2 - name: Update mirror list run: | sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true @@ -72,7 +71,7 @@ jobs: mv apache-maven-3.8.8 /usr/lib/maven && \ export MAVEN_HOME=/usr/lib/maven && \ export PATH=${PATH}:${MAVEN_HOME}/bin && \ - cd $GITHUB_WORKSPACE/ && \ + git clone -b main --depth=1 https://github.com/apache/incubator-gluten.git && cd incubator-gluten/ ./dev/package.sh build-on-ubuntu: From 61ece8e568642a15b4b97ed9aa95dcda8c874f35 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 3 Dec 2024 21:54:16 +0800 Subject: [PATCH 186/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_12_03) (#8124) Upstream Velox's New Commits: 480f989d8 by Jimmy Lu, feat: Allow fixed random seed in approx_percentile for debug purpose (11697) 036ea7d03 by Jimmy Lu, fix: Buckify local runner and fix flaky LocalRunnerTest (11694) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 3e907e92360e..62e7e33f823b 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_30 +VELOX_BRANCH=2024_12_03 VELOX_HOME="" OS=`uname -s` From d7331bea9a5e6ecce2e1a6448a57511c19eb8142 Mon Sep 17 00:00:00 2001 From: exmy Date: Wed, 4 Dec 2024 11:53:28 +0800 Subject: [PATCH 187/211] [GLUTEN-8130][CH] Use the actual user insted of yarn user to read hdfs file (#8131) --- .../Storages/SubstraitSource/ReadBufferBuilder.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index c1796f3e3cda..daaf6482dd87 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -258,9 +258,16 @@ class HDFSFileReadBufferBuilder : public ReadBufferBuilder /// Get hdfs_uri Poco::URI uri(file_info.uri_file()); auto hdfs_file_path = uri.getPath(); - std::string hdfs_uri = "hdfs://" + uri.getHost(); - if (uri.getPort()) - hdfs_uri += ":" + std::to_string(uri.getPort()); + + std::string new_file_uri = uri.toString(); + if (uri.getUserInfo().empty() && BackendInitializerUtil::spark_user.has_value()) + { + uri.setUserInfo(*BackendInitializerUtil::spark_user); + new_file_uri = uri.toString(); + } + + auto begin_of_path = new_file_uri.find('/', new_file_uri.find("//") + 2); + auto hdfs_uri = new_file_uri.substr(0, begin_of_path); std::optional file_size; std::optional modified_time; From cc6dd509b1570c5d0e463828879005ded9b50662 Mon Sep 17 00:00:00 2001 From: zhaokuo Date: Wed, 4 Dec 2024 15:24:36 +0800 Subject: [PATCH 188/211] [VL] Enhance VeloxHashShuffleWriter partition buffer size estimation by incorporating complex type columns (#8089) --- cpp/velox/shuffle/VeloxHashShuffleWriter.cc | 6 ++++++ cpp/velox/shuffle/VeloxHashShuffleWriter.h | 1 + 2 files changed, 7 insertions(+) diff --git a/cpp/velox/shuffle/VeloxHashShuffleWriter.cc b/cpp/velox/shuffle/VeloxHashShuffleWriter.cc index f04473614291..4cd6630fc3ff 100644 --- a/cpp/velox/shuffle/VeloxHashShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxHashShuffleWriter.cc @@ -725,7 +725,9 @@ arrow::Status VeloxHashShuffleWriter::splitComplexType(const facebook::velox::Ro for (auto& pid : partitionUsed_) { if (rowIndexs[pid].size() != 0) { + auto old = arenas_[pid]->size(); complexTypeData_[pid]->append(rowVector, folly::Range(rowIndexs[pid].data(), rowIndexs[pid].size())); + complexTotalSizeBytes_ += arenas_[pid]->size() - old; } } @@ -853,6 +855,10 @@ uint32_t VeloxHashShuffleWriter::calculatePartitionBufferSize(const facebook::ve VS_PRINT_VECTOR_MAPPING(binaryArrayAvgBytesPerRow); + if (totalInputNumRows_ > 0) { + bytesPerRow += complexTotalSizeBytes_ / totalInputNumRows_; + } + VS_PRINTLF(bytesPerRow); memLimit += cachedPayloadSize(); diff --git a/cpp/velox/shuffle/VeloxHashShuffleWriter.h b/cpp/velox/shuffle/VeloxHashShuffleWriter.h index 121eaf116c00..4ee12a1550d4 100644 --- a/cpp/velox/shuffle/VeloxHashShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxHashShuffleWriter.h @@ -355,6 +355,7 @@ class VeloxHashShuffleWriter : public VeloxShuffleWriter { // Updated for each input RowVector. uint64_t totalInputNumRows_ = 0; std::vector binaryArrayTotalSizeBytes_; + size_t complexTotalSizeBytes_ = 0; // True if input column has null in any processed input RowVector. // In the order of fixed-width columns + binary columns. From ff945f99452cf471122255fe5f549e836a86aee4 Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Wed, 4 Dec 2024 16:06:33 +0800 Subject: [PATCH 189/211] [GLUTEN-7518] Remove build_protobuf option from build-guide (#8140) --- docs/get-started/build-guide.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/get-started/build-guide.md b/docs/get-started/build-guide.md index cbaad979fb90..c68a5f463317 100644 --- a/docs/get-started/build-guide.md +++ b/docs/get-started/build-guide.md @@ -15,7 +15,6 @@ Please set them via `--`, e.g. `--build_type=Release`. | build_examples | Build udf example. | OFF | | build_benchmarks | Build gluten cpp benchmarks. | OFF | | enable_jemalloc_stats | Print jemalloc stats for debugging. | OFF | -| build_protobuf | Build protobuf lib. | OFF | | enable_qat | Enable QAT for shuffle data de/compression. | OFF | | enable_iaa | Enable IAA for shuffle data de/compression. | OFF | | enable_hbm | Enable HBM allocator. | OFF | From 6ffab3a3fe903bd26b06ca98992edee97c7ffdf7 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Wed, 4 Dec 2024 20:08:32 +0800 Subject: [PATCH 190/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_12_04) (#8137) Upstream Velox's New Commits: 2c57445fd by zuyu, refactor: Use const ref in Filter (11725) f209751d2 by Wei He, refactor: Extract the definition of Vector::Options::TimestampPrecision to Utils.h (11722) 86d6f3389 by aditi-pandit, refactor: Change C style casts to C++ style (Part 2) (11684) f89b68d27 by Kevin Wilfong, fix: Throw on negative numbers in url_decode (11734) 1cab68067 by Kostas Xirogiannopoulos, feat(planbuilder): Enable passing compressionKind via tableWrite (11724) a94e87065 by Sergey Pershin, fix: Parsing of fractions of a second in parse_datetime() (11723) a0bbea2f3 by zhli1142015, feat(ABFS): Support SAS and OAuth config (11623) 4dd6499f0 by Minhan Cao, build: Removed GTest::gtest_main from CMakeLists.txt for velox_simple_aggregate_test (11668) fc5aa37fe by Jialiang Tan, fix: Fix HashJoinTest.buildReclaimedMemoryReport (11721) 2dae23fcb by yingsu00, misc: Add isBlockedTiming to PlanNodeStats (11361) 0bb7e64c4 by Yang Zhang, refactor(sparksql): Speed up sparksql compilation by splitting function registrations (11565) 46fd360d2 by rui-mo, fix: Use input directly in createLongDecimalVector if aligned (11648) db0c3e697 by Chengcheng Jin, fix: Fix Stream not close because it holds in TreeOfLoser (11660) c542ebc21 by mohsaka, refactor: Clean up, match presto, and optimize ipprefix string -> ipprefix (11670) 0a685b123 by Pavel Solodovnikov, fix: Some minor fixes for `RawVector` (11693) 2872a16c9 by Jacob Wujciak-Jens, build(ci): Enforce `build(ci)` prefix on dependabot PRs (11714) 9bbe4cbff by Raymond Wu, feat(column selector): allow row type (11700) debdf42e6 by Pavel Solodovnikov, build: Make benchmarks respect `VELOX_ENABLE_BENCHMARKS` cmake option (11692) d838d0744 by Yenda Li, fix: MergeExchange hangs at shutdown after abort/cancel (11718) 0dccaeabb by Masha Basmanova, refactor: Reduce dependencies in LocalRunnerTestBase.h (11717) 3e20a7afd by Masha Basmanova, feat: Allow to control level of detail in MultiFragmentPlan::toString (11715) 331ca86a6 by Masha Basmanova, refactor: Move MultiFragmentPlan::toString into its own .cpp file (11713) 557b23827 by Masha Basmanova, refactor: Remove Task dependency from MultiFragmentPlan.h (11712) a4d093454 by duanmeng, feat: Print operator stats in query replayer (11699) a969af8c5 by Wei He, fix: Fix flaky velox_local_runner_test due to memory pool name collision (11683) 6d08f3b5a by Pedro Eugenio Rocha Pedreira, fix(merge-join): Produce output before advancing key comparison (11605) 6ff029eea by Joe Giardino, Velox: fix complex vector memory tracking (11630) d9aaa6afd by Xiaoxuan Meng, misc: Remove legacy code in task and driver (11696) --- cpp/velox/CMakeLists.txt | 2 ++ .../functions/RegistrationAllFunctions.cc | 2 +- .../operators/plannodes/RowVectorStream.h | 21 +++++++++++++++++-- cpp/velox/substrait/SubstraitToVeloxPlan.cc | 7 ------- .../tests/VeloxSubstraitRoundTripTest.cc | 1 - dev/vcpkg/vcpkg.json | 3 ++- ep/build-velox/src/get_velox.sh | 2 +- 7 files changed, 25 insertions(+), 13 deletions(-) diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 4503d5947d78..17b824b6190f 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -132,6 +132,7 @@ macro(find_azure) set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") find_package(azure-storage-blobs-cpp CONFIG REQUIRED) find_package(azure-storage-files-datalake-cpp CONFIG REQUIRED) + find_package(azure-identity-cpp CONFIG REQUIRED) set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() @@ -337,6 +338,7 @@ if(ENABLE_ABFS) find_azure() target_link_libraries(velox PUBLIC Azure::azure-storage-blobs) target_link_libraries(velox PUBLIC Azure::azure-storage-files-datalake) + target_link_libraries(velox PUBLIC Azure::azure-identity) endif() if(BUILD_EXAMPLES) diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.cc b/cpp/velox/operators/functions/RegistrationAllFunctions.cc index ea9d3f4202f7..06147d6d43e5 100644 --- a/cpp/velox/operators/functions/RegistrationAllFunctions.cc +++ b/cpp/velox/operators/functions/RegistrationAllFunctions.cc @@ -29,8 +29,8 @@ #include "velox/functions/sparksql/DecimalArithmetic.h" #include "velox/functions/sparksql/Hash.h" #include "velox/functions/sparksql/Rand.h" -#include "velox/functions/sparksql/Register.h" #include "velox/functions/sparksql/aggregates/Register.h" +#include "velox/functions/sparksql/registration/Register.h" #include "velox/functions/sparksql/window/WindowFunctionsRegistration.h" using namespace facebook; diff --git a/cpp/velox/operators/plannodes/RowVectorStream.h b/cpp/velox/operators/plannodes/RowVectorStream.h index e5a469afee3d..fcf2ffd15f81 100644 --- a/cpp/velox/operators/plannodes/RowVectorStream.h +++ b/cpp/velox/operators/plannodes/RowVectorStream.h @@ -21,6 +21,7 @@ #include "memory/VeloxColumnarBatch.h" #include "velox/exec/Driver.h" #include "velox/exec/Operator.h" +#include "velox/exec/Task.h" namespace gluten { class RowVectorStream { @@ -46,8 +47,16 @@ class RowVectorStream { // As of now, non-zero running threads usually happens when: // 1. Task A spills task B; // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again. - facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); + // facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); + auto driver = driverCtx_->driver; + if (driver->task()->enterSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { + VELOX_FAIL("Terminate detected when entering suspended section"); + } hasNext = iterator_->hasNext(); + if (driver->task()->leaveSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { + LOG(WARNING) << "Terminate detected when leaving suspended section for driver " << driver->driverCtx()->driverId + << " from task " << driver->task()->taskId(); + } } if (!hasNext) { finished_ = true; @@ -64,8 +73,16 @@ class RowVectorStream { { // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current // driver to make the current task open to spilling. - facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); + // facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); + auto driver = driverCtx_->driver; + if (driver->task()->enterSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { + VELOX_FAIL("Terminate detected when entering suspended section"); + } cb = iterator_->next(); + if (driver->task()->leaveSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { + LOG(WARNING) << "Terminate detected when leaving suspended section for driver " << driver->driverCtx()->driverId + << " from task " << driver->task()->taskId(); + } } const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, cb); auto vp = vb->getRowVector(); diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 3ceccca4a3de..e0cc5a184b8e 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -140,13 +140,6 @@ RowTypePtr getJoinOutputType( VELOX_FAIL("Output should include left or right columns."); } -// Returns the path vector used to create Subfield. -std::vector> getPath(const std::string& field) { - std::vector> path; - path.push_back(std::make_unique(field)); - return path; -} - } // namespace core::PlanNodePtr SubstraitToVeloxPlanConverter::processEmit( diff --git a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc index 68e79c80f5b9..804ba6c413e1 100644 --- a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc +++ b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc @@ -27,7 +27,6 @@ #include "substrait/SubstraitToVeloxPlan.h" #include "substrait/VeloxToSubstraitPlan.h" -#include "velox/functions/sparksql/Register.h" #include "velox/vector/tests/utils/VectorTestBase.h" #include "substrait/VariantToVectorConverter.h" diff --git a/dev/vcpkg/vcpkg.json b/dev/vcpkg/vcpkg.json index 66ba246f530a..c0123cfbe998 100644 --- a/dev/vcpkg/vcpkg.json +++ b/dev/vcpkg/vcpkg.json @@ -90,7 +90,8 @@ "description": "Velox ABFS Support", "dependencies": [ "azure-storage-blobs-cpp", - "azure-storage-files-datalake-cpp" + "azure-storage-files-datalake-cpp", + "azure-identity-cpp" ] }, "duckdb": { diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 62e7e33f823b..962a3b415bd4 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_12_03 +VELOX_BRANCH=2024_12_04 VELOX_HOME="" OS=`uname -s` From 6dd65c025886db558de0ab0eabaa70cb100e6f05 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Wed, 4 Dec 2024 06:44:11 -0600 Subject: [PATCH 191/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241204) (#8135) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241204) * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/72715 * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/65691 * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/72722 * Fix gtest due to #8052 * Fix benchmark due to https://github.com/ClickHouse/ClickHouse/pull/72460 * Add SPARK_DIR_NAME for fixing unstable ut --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- .../execution/tpch/GlutenClickHouseHDFSSuite.scala | 2 +- cpp-ch/clickhouse.version | 4 ++-- .../AggregateFunctions/GroupLimitFunctions.cpp | 7 +------ cpp-ch/local-engine/Functions/SparkFunctionFloor.h | 4 ++++ cpp-ch/local-engine/Shuffle/PartitionWriter.cpp | 1 + cpp-ch/local-engine/Shuffle/ShuffleReader.h | 1 + .../Storages/MergeTree/SparkStorageMergeTree.cpp | 1 + .../Storages/SubstraitSource/ReadBufferBuilder.cpp | 13 ++++++------- .../Storages/SubstraitSource/ReadBufferBuilder.h | 2 +- .../local-engine/tests/benchmark_local_engine.cpp | 2 +- cpp-ch/local-engine/tests/gtest_write_pipeline.cpp | 6 +++--- 11 files changed, 22 insertions(+), 21 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala index aa77f79bbac1..d79b25a7bd09 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala @@ -144,7 +144,7 @@ class GlutenClickHouseHDFSSuite test("GLUTEN-7542: Fix cache refresh") { withSQLConf("spark.sql.hive.manageFilesourcePartitions" -> "false") { - val filePath = s"$tablesPath/issue_7542/" + val filePath = s"$tablesPath/$SPARK_DIR_NAME/issue_7542/" val targetDirs = new Path(filePath) val fs = targetDirs.getFileSystem(spark.sessionState.newHadoopConf()) fs.mkdirs(targetDirs) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index c3fda4bd2f55..999c8c957298 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241203 -CH_COMMIT=473be0b9c50 +CH_BRANCH=rebase_ch/20241204 +CH_COMMIT=7b72d0dc991 diff --git a/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp index a9be46908bb9..67c9e8380aa3 100644 --- a/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp +++ b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp @@ -16,8 +16,6 @@ */ #include #include -#include -#include #include #include #include @@ -28,17 +26,14 @@ #include #include #include +#include #include #include #include #include -#include #include #include -#include #include - -#include #include namespace DB::ErrorCodes diff --git a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h index 4f655c615b9f..c9058a564450 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h @@ -23,6 +23,10 @@ #include #include +#if USE_MULTITARGET_CODE +#include +#endif + using namespace DB; namespace local_engine diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index 43459f20c5b3..158f1a9cfa0e 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Shuffle/ShuffleReader.h b/cpp-ch/local-engine/Shuffle/ShuffleReader.h index 3281a3c844f3..721a157849a7 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleReader.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleReader.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index cbb40ae89dc0..c1a95a6be02c 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -17,6 +17,7 @@ #include "SparkStorageMergeTree.h" #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index daaf6482dd87..3dd0d1127f10 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -616,21 +616,20 @@ ConcurrentLRU> S3FileReadBufferBuil class AzureBlobReadBuffer : public ReadBufferBuilder { public: - explicit AzureBlobReadBuffer(DB::ContextPtr context_) : ReadBufferBuilder(context_) { } + explicit AzureBlobReadBuffer(const DB::ContextPtr & context_) : ReadBufferBuilder(context_) { } ~AzureBlobReadBuffer() override = default; std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) override { Poco::URI file_uri(file_info.uri_file()); - std::unique_ptr read_buffer; - read_buffer = std::make_unique(getClient(), file_uri.getPath(), DB::ReadSettings(), 5, 5); - return read_buffer; + return std::make_unique(getClient(), file_uri.getPath(), DB::ReadSettings(), 5, 5); } private: - std::shared_ptr shared_client; - std::shared_ptr getClient() + std::shared_ptr shared_client; + + std::shared_ptr getClient() { if (shared_client) return shared_client; @@ -687,7 +686,7 @@ DB::ReadSettings ReadBufferBuilder::getReadSettings() const return read_settings; } -ReadBufferBuilder::ReadBufferBuilder(DB::ContextPtr context_) : context(context_) +ReadBufferBuilder::ReadBufferBuilder(const DB::ContextPtr & context_) : context(context_) { } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h index e76e79a9b0ba..81a4c60bfcb1 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h @@ -32,7 +32,7 @@ namespace local_engine class ReadBufferBuilder { public: - explicit ReadBufferBuilder(DB::ContextPtr context_); + explicit ReadBufferBuilder(const DB::ContextPtr & context_); virtual ~ReadBufferBuilder() = default; diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index eacfb1781b26..20ab61265e5b 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -846,7 +846,7 @@ QueryPlanPtr joinPlan(QueryPlanPtr left, QueryPlanPtr right, String left_key, St auto hash_join = std::make_shared(join, right->getCurrentHeader()); QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 0, 1, false); + = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 8192, 1, NameSet{}, false, false); std::vector plans; plans.emplace_back(std::move(left)); diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp index b764f62f542e..288cd6f0ec6a 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -121,7 +121,7 @@ TEST(WritePipeline, SubstraitFileSink) const auto context = DB::Context::createCopy(QueryContext::globalContext()); GlutenWriteSettings settings{ .task_write_tmp_dir = "file:///tmp/test_table/test", - .task_write_filename = "data.parquet", + .task_write_filename_pattern = "data.parquet", }; settings.set(context); @@ -155,7 +155,7 @@ TEST(WritePipeline, SubstraitFileSink) std::cerr << debug::verticalShowString(x, 10, 50) << std::endl; EXPECT_EQ(1, x.rows()); const auto & col_a = *(x.getColumns()[0]); - EXPECT_EQ(settings.task_write_filename, col_a.getDataAt(0)); + EXPECT_EQ(settings.task_write_filename_pattern, col_a.getDataAt(0)); const auto & col_b = *(x.getColumns()[1]); EXPECT_EQ(SubstraitFileSink::NO_PARTITION_ID, col_b.getDataAt(0)); const auto & col_c = *(x.getColumns()[2]); @@ -169,7 +169,7 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) const auto context = DB::Context::createCopy(QueryContext::globalContext()); GlutenWriteSettings settings{ .task_write_tmp_dir = "file:///tmp/test_table/test_partition", - .task_write_filename = "data.parquet", + .task_write_filename_pattern = "data.parquet", }; settings.set(context); From 57a713a95e5ab9c79a8a23289d6fd714740444c9 Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Thu, 5 Dec 2024 08:26:58 +0800 Subject: [PATCH 192/211] [CORE] Add Gluten Project Improvement Proposals (GPIP) doc (#8133) This PR aims to add Gluten Project Improvement Proposals doc. Gluten is growing rapidly, and many major optimizations are expected for it. To follow the Apache way, we should have a specification for major optimizations, and this documentation refers to Spark's SPIP. --- docs/developers/improvement-proposals.md | 100 +++++++++++++++++++++++ 1 file changed, 100 insertions(+) create mode 100644 docs/developers/improvement-proposals.md diff --git a/docs/developers/improvement-proposals.md b/docs/developers/improvement-proposals.md new file mode 100644 index 000000000000..1b7c47bbf222 --- /dev/null +++ b/docs/developers/improvement-proposals.md @@ -0,0 +1,100 @@ +--- +layout: page +title: Gluten Project Improvement Proposals (GPIP) +nav_order: 10 +parent: Developer Overview +--- + +# Gluten Project Improvement Proposals (GPIP) + +The Gluten Project Improvement Proposals doc references [the Spark SPIP documentation](https://spark.apache.org/improvement-proposals.html). + +The purpose of a GPIP is to inform and involve the user community in major improvements to the Gluten codebase throughout the development process to increase the likelihood that user needs are met. + +GPIPs should be used for significant user-facing or cutting-edge changes, not small incremental improvements. + +If your proposal meets the definition of GPIP, we recommend you to create a GPIP, which will facilitate the advancement and discussion of the proposal, but it is not mandatory, and we welcome any contribution and community participation. + +## What is a GPIP? + +A GPIP is similar to a product requirement document commonly used in product management. + +A GPIP: + +- Is a ticket labeled “GPIP” proposing a major improvement or change to Gluten +- Follows the template defined below +- Includes discussions on the ticket and dev@ list about the proposal + +## Who? + +Any **community member** can help by discussing whether a GPIP is likely to meet their needs and propose GPIPs. + +**Contributors** can help by discussing whether a GPIP is likely to be technically feasible. + +**Committers** can help by discussing whether a GPIP aligns with long-term project goals, and by shepherding GPIPs. + +**GPIP Author** is any community member who authors a GPIP and is committed to pushing the change through the entire process. GPIP authorship can be transferred. + +**GPIP Shepherd** is a PMC member who is committed to shepherding the proposed change throughout the entire process. Although the shepherd can delegate or work with other committers in the development process, the shepherd is ultimately responsible for the success or failure of the GPIP. Responsibilities of the shepherd include, but are not limited to: + +- Be the advocate for the proposed change +- Help push forward on design and achieve consensus among key stakeholders +- Review code changes, making sure the change follows project standards +- Get feedback from users and iterate on the design & implementation +- Uphold the quality of the changes, including verifying whether the changes satisfy the goal of the GPIP and are absent of critical bugs before releasing them + +## GPIP Process +### Proposing a GPIP + +Anyone may propose a GPIP, using the document template below. Please only submit a GPIP if you are willing to help, at least with discussion. + +After a GPIP is created, the author should email dev@gluten.apache.org to notify the community of the GPIP, and discussions should ensue on the ticket. + +If a GPIP is too small or incremental and should have been done through the normal ticket process, a committer should remove the GPIP label. + +### GPIP Document Template + +A GPIP document is a short document with a few questions, inspired by the Heilmeier Catechism: + +- Q1. What are you trying to do? Articulate your objectives using absolutely no jargon. + +- Q2. What problem is this proposal NOT designed to solve? + +- Q3. How is it done today, and what are the limits of current practice? + +- Q4. What is new in your approach, and why do you think it will be successful? + +- Q5. Who cares? If you are successful, what difference will it make? + +- Q6. What are the risks? + +- Q7. How long will it take? + +- Q8. What are the mid-term and final “exams” to check for success? + +- Appendix A. Proposed API Changes. Optional section defining APIs changes, if any. Backward and forward compatibility must be taken into account. + +- Appendix B. Optional Design Sketch: How are the goals going to be accomplished? Give sufficient technical detail to allow a contributor to judge whether it's likely to be feasible. Note that this is not a full design document. + +- Appendix C. Optional Rejected Designs: What alternatives were considered? Why were they rejected? If no alternatives have been considered, the problem needs more thought. + +### Discussing a GPIP + +All discussions of a GPIP should take place in a public forum, preferably the discussion attached to the ticket. Any discussion that happen offline should be made available online for the public via meeting notes summarizing the discussions. + +During this discussion, one or more shepherds should be identified among PMC members. + +Once the discussion settles, the shepherd(s) should call for a vote on the GPIP moving forward on the dev@ list. The vote should be open for at least 72 hours and follows the typical Apache vote process and passes upon consensus (at least 3 +1 votes from PMC members and no -1 votes from PMC members). dev@ should be notified of the vote result. + +If there does not exist at least one PMC member that is committed to shepherding the change within a month, the GPIP is rejected. + +If a committer does not think a GPIP aligns with long-term project goals, or is not practical at the point of proposal, the committer should -1 the GPIP explicitly and give technical justifications. + + +### Implementing a GPIP + +Implementation should take place via the contribution guidelines. Changes that require GPIPs typically also require design documents to be written and reviewed. + +### GPIP Community collaboration + +The gluten community has always been open to contributions and participation of all kinds, and feel free to contact the community if you have any questions about the GPIP process. From e52dea1d90556dc924f631e02ae5fc0198df5fcd Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Thu, 5 Dec 2024 13:36:19 +0800 Subject: [PATCH 193/211] [VL] Add back RAII style Velox driver suspension into RowVectorStream (#8149) --- .../operators/plannodes/RowVectorStream.h | 41 +++++++++++-------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/cpp/velox/operators/plannodes/RowVectorStream.h b/cpp/velox/operators/plannodes/RowVectorStream.h index fcf2ffd15f81..63ddd0abe00d 100644 --- a/cpp/velox/operators/plannodes/RowVectorStream.h +++ b/cpp/velox/operators/plannodes/RowVectorStream.h @@ -23,6 +23,27 @@ #include "velox/exec/Operator.h" #include "velox/exec/Task.h" +namespace { +class SuspendedSection { + public: + explicit SuspendedSection(facebook::velox::exec::Driver* driver) : driver_(driver) { + if (driver_->task()->enterSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { + VELOX_FAIL("Terminate detected when entering suspended section"); + } + } + + virtual ~SuspendedSection() { + if (driver_->task()->leaveSuspended(driver_->state()) != facebook::velox::exec::StopReason::kNone) { + LOG(WARNING) << "Terminate detected when leaving suspended section for driver " << driver_->driverCtx()->driverId + << " from task " << driver_->task()->taskId(); + } + } + + private: + facebook::velox::exec::Driver* const driver_; +}; +} // namespace + namespace gluten { class RowVectorStream { public: @@ -47,16 +68,8 @@ class RowVectorStream { // As of now, non-zero running threads usually happens when: // 1. Task A spills task B; // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again. - // facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); - auto driver = driverCtx_->driver; - if (driver->task()->enterSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { - VELOX_FAIL("Terminate detected when entering suspended section"); - } + SuspendedSection ss(driverCtx_->driver); hasNext = iterator_->hasNext(); - if (driver->task()->leaveSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { - LOG(WARNING) << "Terminate detected when leaving suspended section for driver " << driver->driverCtx()->driverId - << " from task " << driver->task()->taskId(); - } } if (!hasNext) { finished_ = true; @@ -73,16 +86,8 @@ class RowVectorStream { { // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current // driver to make the current task open to spilling. - // facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); - auto driver = driverCtx_->driver; - if (driver->task()->enterSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { - VELOX_FAIL("Terminate detected when entering suspended section"); - } + SuspendedSection ss(driverCtx_->driver); cb = iterator_->next(); - if (driver->task()->leaveSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { - LOG(WARNING) << "Terminate detected when leaving suspended section for driver " << driver->driverCtx()->driverId - << " from task " << driver->task()->taskId(); - } } const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, cb); auto vp = vb->getRowVector(); From bb348e8b9b74eb7c15a57613bfefb657a539ded0 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Thu, 5 Dec 2024 15:31:30 +0800 Subject: [PATCH 194/211] [VL] Change C style casts to C++ style (#8153) Follows facebookincubator/velox@28c319e to change C style casts to C++ style. --- cpp/core/jni/JniWrapper.cc | 4 ++-- cpp/core/utils/Metrics.h | 4 ++-- cpp/velox/compute/WholeStageResultIterator.cc | 4 ++-- cpp/velox/substrait/SubstraitToVeloxPlan.cc | 16 ++++++++++++---- cpp/velox/tests/VeloxSubstraitRoundTripTest.cc | 14 +++++++------- 5 files changed, 25 insertions(+), 17 deletions(-) diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 794ca6b88f48..b7bdae81750a 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -475,7 +475,7 @@ JNIEXPORT jobject JNICALL Java_org_apache_gluten_metrics_IteratorMetricsJniWrapp } jlongArray longArray[Metrics::kNum]; - for (auto i = (int)Metrics::kBegin; i != (int)Metrics::kEnd; ++i) { + for (auto i = static_cast(Metrics::kBegin); i != static_cast(Metrics::kEnd); ++i) { longArray[i] = env->NewLongArray(numMetrics); if (metrics) { env->SetLongArrayRegion(longArray[i], 0, numMetrics, metrics->get((Metrics::TYPE)i)); @@ -801,7 +801,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe .bufferSize = bufferSize, .bufferReallocThreshold = reallocThreshold, .partitioning = toPartitioning(jStringToCString(env, partitioningNameJstr)), - .taskAttemptId = (int64_t)taskAttemptId, + .taskAttemptId = static_cast(taskAttemptId), .startPartitionId = startPartitionId, .shuffleWriterType = ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterTypeJstr)), .sortBufferInitialSize = sortBufferInitialSize, diff --git a/cpp/core/utils/Metrics.h b/cpp/core/utils/Metrics.h index 698e97f3dce6..8da13bc91635 100644 --- a/cpp/core/utils/Metrics.h +++ b/cpp/core/utils/Metrics.h @@ -95,8 +95,8 @@ struct Metrics { Metrics& operator=(Metrics&&) = delete; long* get(TYPE type) { - assert((int)type >= (int)kBegin && (int)type < (int)kEnd); - auto offset = ((int)type - (int)kBegin) * numMetrics; + assert(static_cast(type) >= static_cast(kBegin) && static_cast(type) < static_cast(kEnd)); + auto offset = (static_cast(type) - static_cast(kBegin)) * numMetrics; return &arrayRawPtr[offset]; } }; diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index 411c6c563646..e5463b96fd25 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -478,9 +478,9 @@ std::unordered_map WholeStageResultIterator::getQueryC // partial aggregation memory config auto offHeapMemory = veloxCfg_->get(kSparkTaskOffHeapMemory, facebook::velox::memory::kMaxMemory); auto maxPartialAggregationMemory = - (long)(veloxCfg_->get(kMaxPartialAggregationMemoryRatio, 0.1) * offHeapMemory); + static_cast((veloxCfg_->get(kMaxPartialAggregationMemoryRatio, 0.1) * offHeapMemory)); auto maxExtendedPartialAggregationMemory = - (long)(veloxCfg_->get(kMaxExtendedPartialAggregationMemoryRatio, 0.15) * offHeapMemory); + static_cast((veloxCfg_->get(kMaxExtendedPartialAggregationMemoryRatio, 0.15) * offHeapMemory)); configs[velox::core::QueryConfig::kMaxPartialAggregationMemory] = std::to_string(maxPartialAggregationMemory); configs[velox::core::QueryConfig::kMaxExtendedPartialAggregationMemory] = std::to_string(maxExtendedPartialAggregationMemory); diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index e0cc5a184b8e..1e1ab8971ef1 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -1023,7 +1023,11 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan( // Handle if all sorting keys are also used as partition keys. return std::make_shared( - nextPlanNodeId(), partitionKeys, rowNumberColumnName, (int32_t)windowGroupLimitRel.limit(), childNode); + nextPlanNodeId(), + partitionKeys, + rowNumberColumnName, + static_cast(windowGroupLimitRel.limit()), + childNode); } return std::make_shared( @@ -1032,7 +1036,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan( sortingKeys, sortingOrders, rowNumberColumnName, - (int32_t)windowGroupLimitRel.limit(), + static_cast(windowGroupLimitRel.limit()), childNode); } @@ -1122,14 +1126,18 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::FetchRel& fetchRel) { auto childNode = convertSingleInput<::substrait::FetchRel>(fetchRel); return std::make_shared( - nextPlanNodeId(), (int32_t)fetchRel.offset(), (int32_t)fetchRel.count(), false /*isPartial*/, childNode); + nextPlanNodeId(), + static_cast(fetchRel.offset()), + static_cast(fetchRel.count()), + false /*isPartial*/, + childNode); } core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::TopNRel& topNRel) { auto childNode = convertSingleInput<::substrait::TopNRel>(topNRel); auto [sortingKeys, sortingOrders] = processSortField(topNRel.sorts(), childNode->outputType()); return std::make_shared( - nextPlanNodeId(), sortingKeys, sortingOrders, (int32_t)topNRel.n(), false /*isPartial*/, childNode); + nextPlanNodeId(), sortingKeys, sortingOrders, static_cast(topNRel.n()), false /*isPartial*/, childNode); } core::PlanNodePtr SubstraitToVeloxPlanConverter::constructValueStreamNode( diff --git a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc index 804ba6c413e1..b675bf4a281e 100644 --- a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc +++ b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc @@ -366,13 +366,13 @@ TEST_F(VeloxSubstraitRoundTripTest, notNullLiteral) { .addNode([&](std::string id, core::PlanNodePtr input) { std::vector projectNames = {"a", "b", "c", "d", "e", "f", "g", "h"}; std::vector projectExpressions = { - makeConstantExpr(BOOLEAN(), (bool)1), - makeConstantExpr(TINYINT(), (int8_t)23), - makeConstantExpr(SMALLINT(), (int16_t)45), - makeConstantExpr(INTEGER(), (int32_t)678), - makeConstantExpr(BIGINT(), (int64_t)910), - makeConstantExpr(REAL(), (float)1.23), - makeConstantExpr(DOUBLE(), (double)4.56), + makeConstantExpr(BOOLEAN(), static_cast(1)), + makeConstantExpr(TINYINT(), static_cast(23)), + makeConstantExpr(SMALLINT(), static_cast(45)), + makeConstantExpr(INTEGER(), static_cast(678)), + makeConstantExpr(BIGINT(), static_cast(910)), + makeConstantExpr(REAL(), static_cast(1.23)), + makeConstantExpr(DOUBLE(), static_cast(4.56)), makeConstantExpr(VARCHAR(), "789")}; return std::make_shared( id, std::move(projectNames), std::move(projectExpressions), input); From c448ea55ba7582e94760151d50e7973b071161f1 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Thu, 5 Dec 2024 15:49:54 +0800 Subject: [PATCH 195/211] [DOC] Fix typos in documentation (#8155) --- docs/developers/CppCodingStyle.md | 10 +++++----- docs/developers/MicroBenchmarks.md | 2 +- docs/developers/NewToGluten.md | 4 ++-- docs/developers/VeloxUDF.md | 4 ++-- docs/get-started/ClickHouse.md | 2 +- docs/get-started/Velox.md | 2 +- docs/get-started/VeloxGCS.md | 4 ++-- docs/velox-backend-limitations.md | 2 +- 8 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/developers/CppCodingStyle.md b/docs/developers/CppCodingStyle.md index 42101882a9e5..2cea8fe8feb0 100644 --- a/docs/developers/CppCodingStyle.md +++ b/docs/developers/CppCodingStyle.md @@ -73,7 +73,7 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt * Obvious exception: files used for defining various misc functions. * If a header file has a corresponding source file, they should have the same file name with different suffix, such as `a.h vs a.cc`. -* If a function is declared in the file `a.h`, ensure it's defined in the corrosponding +* If a function is declared in the file `a.h`, ensure it's defined in the corresponding source file `a.cc`, do not define it in other files. * No deep source directory for CPP files, not do it as JAVA. * Include header files should satisfy the following rules. @@ -95,7 +95,7 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt * Ensure that public inheritance represent the relation of `is-a`. * Ensure that private inheritance represent the relation of `implements-with`. * Don't make a function `virtual` without reason. -* Ensure the polymorphic base class has a `virtual` deconstructor. +* Ensure the polymorphic base class has a `virtual` destructor. * Use `override` to make overriding explicit and to make the compiler work. * Use `const` to mark the member function read-only as far as possible. * When you try to define a `copy constructor` or a `operator=` for a class, remember @@ -113,8 +113,8 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt the function name. such as. use `get(size_t index)` instead of `getByIndex`. * A function should focus on a single logic operation. * A function should do as the name meaning. - * do everything converd by the function name - * don't do anything not convered by the function name + * do everything covered by the function name + * don't do anything not covered by the function name ## Variable @@ -158,7 +158,7 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt * `unique_ptr` represents ownership, but not share ownership. `unique_ptr` is equivalent to RAII, release the resource when the object is destructed. * `shared_ptr` represents shared ownership by use-count. It is more expensive - that `unqiue_ptr`. + that `unique_ptr`. * `weak_ptr` models temporary ownership. It is useful in breaking reference cycles formed by objects managed by `shared_ptr`. * Use `unique_ptr` or `shared_ptr` to represent ownership. diff --git a/docs/developers/MicroBenchmarks.md b/docs/developers/MicroBenchmarks.md index 1483dc2cbaff..c59d6be3a631 100644 --- a/docs/developers/MicroBenchmarks.md +++ b/docs/developers/MicroBenchmarks.md @@ -15,7 +15,7 @@ comparing with directly debugging in a Spark job. Developers can use it to creat workloads, debug in native process, profile the hotspot and do optimizations. To simulate a first stage, you need to dump the Substrait plan and input split info into two JSON -files. The input URIs of the splits should be exising file locations, which can be either local or +files. The input URIs of the splits should be existing file locations, which can be either local or HDFS paths. To simulate a middle stage, in addition to the JSON file, you also need to save the input data of diff --git a/docs/developers/NewToGluten.md b/docs/developers/NewToGluten.md index 347213a41824..b790d8740a9e 100644 --- a/docs/developers/NewToGluten.md +++ b/docs/developers/NewToGluten.md @@ -55,7 +55,7 @@ spark.executor.extraJavaOptions=-Dio.netty.tryReflectionSetAccessible=true ## Maven 3.6.3 or above -[Maven Dowload Page](https://maven.apache.org/docs/history.html) +[Maven Download Page](https://maven.apache.org/docs/history.html) And then set the environment setting. ## GCC 11 or above @@ -247,7 +247,7 @@ Then you can create breakpoint and debug in `Run and Debug` section. ### Velox debug For some velox tests such as `ParquetReaderTest`, tests need to read the parquet file in `/velox/dwio/parquet/tests/examples`, -you should let the screen on `ParquetReaderTest.cpp`, then click `Start Debuging`, otherwise `No such file or directory` exception will be raised. +you should let the screen on `ParquetReaderTest.cpp`, then click `Start Debugging`, otherwise `No such file or directory` exception will be raised. ## Useful notes diff --git a/docs/developers/VeloxUDF.md b/docs/developers/VeloxUDF.md index 4cbdcfa99292..a8b6a86aeb55 100644 --- a/docs/developers/VeloxUDF.md +++ b/docs/developers/VeloxUDF.md @@ -38,7 +38,7 @@ The following steps demonstrate how to set up a UDF library project: - `registerUdf()`: This function is called to register the UDF to Velox function registry. - This is where users should register functions by calling `facebook::velox::exec::registerVecotorFunction` or other Velox APIs. + This is where users should register functions by calling `facebook::velox::exec::registerVectorFunction` or other Velox APIs. - The interface functions are mapped to marcos in [Udf.h](../../cpp/velox/udf/Udf.h). @@ -215,7 +215,7 @@ pip3 install pyspark==$SPARK_VERSION cython pip3 install pandas pyarrow ``` -Gluten provides a config to control enable `ColumnarArrowEvalPython` or not, with `true` as defalt. +Gluten provides a config to control enable `ColumnarArrowEvalPython` or not, with `true` as default. ``` spark.gluten.sql.columnar.arrowUdf diff --git a/docs/get-started/ClickHouse.md b/docs/get-started/ClickHouse.md index 2f6a912f4362..287747418294 100644 --- a/docs/get-started/ClickHouse.md +++ b/docs/get-started/ClickHouse.md @@ -118,7 +118,7 @@ cd $gluten_root/cpp-ch/local-engine/docker `build_directory` is a directory used as to store the intermediate files from compiling. It will use `current_dir>`/build as the default value if you don't provide it. -`output_directory` is used as to store the finaly output `libch.so`. The default value is `current_dir`/output if you don't provide it. +`output_directory` is used as to store the finally output `libch.so`. The default value is `current_dir`/output if you don't provide it. 3. (Option 1) Use CLion diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index bfaa0200c785..3b3ab07a8dea 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -217,7 +217,7 @@ cd /path/to/gluten ./dev/buildbundle-veloxbe.sh --enable_s3=ON ``` -Currently there are several ways to asscess S3 in Spark. Please refer [Velox S3](VeloxS3.md) part for more detailed configurations +Currently there are several ways to access S3 in Spark. Please refer [Velox S3](VeloxS3.md) part for more detailed configurations ## Celeborn support diff --git a/docs/get-started/VeloxGCS.md b/docs/get-started/VeloxGCS.md index 453bee117c1e..09e0a927cab4 100644 --- a/docs/get-started/VeloxGCS.md +++ b/docs/get-started/VeloxGCS.md @@ -24,7 +24,7 @@ After these steps, no specific configuration is required for Gluten, since the a For workloads that need to be fully automated, manually authorizing can be problematic. For such cases it is better to use a json file with the credentials. This is described in the [instructions to configure a service account]https://cloud.google.com/sdk/docs/authorizing#service-account. -Such json file with the credetials can be passed to Gluten: +Such json file with the credentials can be passed to Gluten: ```sh spark.hadoop.fs.gs.auth.type SERVICE_ACCOUNT_JSON_KEYFILE @@ -49,5 +49,5 @@ spark.hadoop.fs.gs.http.max.retry // number of times to keep retrying unless a n For cases when a transient server error is detected, GCS can be configured to keep retrying until the retry loop exceeds a prescribed duration. ```sh -spark.hadoop.fs.gs.http.max.retry-time // a string representing the time keep retring (10s, 1m, etc). +spark.hadoop.fs.gs.http.max.retry-time // a string representing the time keep retrying (10s, 1m, etc). ``` \ No newline at end of file diff --git a/docs/velox-backend-limitations.md b/docs/velox-backend-limitations.md index e05b589524d5..161b1bfc713a 100644 --- a/docs/velox-backend-limitations.md +++ b/docs/velox-backend-limitations.md @@ -147,7 +147,7 @@ Gluten's. ### Spill -`OutOfMemoryExcetpion` may still be triggered within current implementation of spill-to-disk feature, when shuffle partitions is set to a large number. When this case happens, please try to reduce the partition number to get rid of the OOM. +`OutOfMemoryException` may still be triggered within current implementation of spill-to-disk feature, when shuffle partitions is set to a large number. When this case happens, please try to reduce the partition number to get rid of the OOM. ### Unsupported Data type support in ParquetScan From f704f092855d664def191935c03d3c284b97f218 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Thu, 5 Dec 2024 16:04:39 +0800 Subject: [PATCH 196/211] [GLUTEN-7143][VL] RAS: Remove experimental flags for RAS (#8154) Part of #7143 (not to close it) RAS is considered production-ready now as all Spark / Gluten UTs, TPC-H, TPC-DS ITs have passed for a period of time. More documentations needed but we can remove the experimental flags to state it's ready for use. The patch removes the flags from doc and code. --- docs/Configuration.md | 2 +- .../extension/columnar/enumerated/EnumeratedApplier.scala | 7 ++----- .../src/main/scala/org/apache/gluten/GlutenConfig.scala | 7 ++++--- 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/docs/Configuration.md b/docs/Configuration.md index cb8efe802e2a..e217be45ff80 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -23,7 +23,7 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.shuffle.manager | To turn on Gluten Columnar Shuffle Plugin | org.apache.spark.shuffle.sort.ColumnarShuffleManager | | spark.gluten.enabled | Enable Gluten, default is true. Just an experimental property. Recommend to enable/disable Gluten through the setting for `spark.plugins`. | true | | spark.gluten.memory.isolation | (Experimental) Enable isolated memory mode. If true, Gluten controls the maximum off-heap memory can be used by each task to X, X = executor memory / max task slots. It's recommended to set true if Gluten serves concurrent queries within a single session, since not all memory Gluten allocated is guaranteed to be spillable. In the case, the feature should be enabled to avoid OOM. Note when true, setting spark.memory.storageFraction to a lower value is suggested since storage memory is considered non-usable by Gluten. | false | -| spark.gluten.ras.enabled | Experimental: Enables RAS (relation algebra selector) during physical planning to generate more efficient query plan. Note, this feature is still in development and may not bring performance profits. | false | +| spark.gluten.ras.enabled | Enables RAS (relation algebra selector) during physical planning to generate more efficient query plan. Note, this feature doesn't bring performance profits by default. Try exploring option `spark.gluten.ras.costModel` for advanced usage. | false | | spark.gluten.sql.columnar.maxBatchSize | Number of rows to be processed in each batch. Default value is 4096. | 4096 | | spark.gluten.sql.columnar.scanOnly | When enabled, this config will overwrite all other operators' enabling, and only Scan and Filter pushdown will be offloaded to native. | false | | spark.gluten.sql.columnar.batchscan | Enable or Disable Columnar BatchScan, default is true | true | diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala index ff822d1c0a1f..04cd70656ef6 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala @@ -21,7 +21,6 @@ import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.util.AdaptiveContext import org.apache.gluten.logging.LogLevelUtil -import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule @@ -31,11 +30,9 @@ import org.apache.spark.sql.execution.SparkPlan * Columnar rule applier that optimizes, implements Spark plan into Gluten plan by enumerating on * all the possibilities of executable Gluten plans, then choose the best plan among them. * - * NOTE: This is still working in progress. We still have a bunch of heuristic rules in this - * implementation's rule list. Future work will include removing them from the list then - * implementing them in EnumeratedTransform. + * NOTE: We still have a bunch of heuristic rules in this implementation's rule list. Future work + * will include removing them from the list then implementing them in EnumeratedTransform. */ -@Experimental class EnumeratedApplier( session: SparkSession, ruleBuilders: Seq[ColumnarRuleCall => Rule[SparkPlan]]) diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 9ae4c0ce90eb..4f243f03fb78 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -1416,9 +1416,10 @@ object GlutenConfig { val RAS_ENABLED = buildConf("spark.gluten.ras.enabled") .doc( - "Experimental: Enables RAS (relational algebra selector) during physical " + - "planning to generate more efficient query plan. Note, this feature is still in " + - "development and may not bring performance profits.") + "Enables RAS (relational algebra selector) during physical " + + "planning to generate more efficient query plan. Note, this feature doesn't bring " + + "performance profits by default. Try exploring option `spark.gluten.ras.costModel` " + + "for advanced usage.") .booleanConf .createWithDefault(false) From a7f64b980345574316d943ddccd8869926efeee6 Mon Sep 17 00:00:00 2001 From: Shuai li Date: Thu, 5 Dec 2024 16:35:22 +0800 Subject: [PATCH 197/211] [GLUTEN-8148][CH] Fix corr with NaN (#8150) What changes were proposed in this pull request? (Fixes: #8148) How was this patch tested? Test by ut --- .../GlutenClickhouseFunctionSuite.scala | 14 +++++++++++++ .../Parser/AggregateFunctionParser.cpp | 21 +++++++++++++++++++ .../Parser/AggregateFunctionParser.h | 3 +++ 3 files changed, 38 insertions(+) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index e35f6bf65bf3..2437ffd03550 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -401,4 +401,18 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { } } + test("GLUTEN-8148: Fix corr with NaN") { + withTable("corr_nan") { + sql("create table if not exists corr_nan (x double, y double) using parquet") + sql("insert into corr_nan values(0,1)") + compareResultsAgainstVanillaSpark( + """ + |select corr(x,y), corr(y,x) from corr_nan + """.stripMargin, + true, + { _ => } + ) + } + } + } diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp index 42c4230e4a9d..eb05b26dcf94 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp @@ -195,6 +195,8 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( actions_dag.addOrReplaceInOutputs(*func_node); } + func_node = convertNanToNullIfNeed(func_info, func_node, actions_dag); + if (output_type.has_decimal()) { String checkDecimalOverflowSparkOrNull = "checkDecimalOverflowSparkOrNull"; @@ -209,6 +211,25 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( return func_node; } +const DB::ActionsDAG::Node * AggregateFunctionParser::convertNanToNullIfNeed( + const CommonFunctionInfo & func_info, const DB::ActionsDAG::Node * func_node, DB::ActionsDAG & actions_dag) const +{ + if (getCHFunctionName(func_info) != "corr" || !func_node->result_type->isNullable()) + return func_node; + + /// result is nullable. + /// if result is NaN, convert it to NULL. + auto is_nan_func_node = toFunctionNode(actions_dag, "isNaN", getUniqueName("isNaN"), {func_node}); + auto nullable_col = func_node->result_type->createColumn(); + nullable_col->insertDefault(); + const auto * null_node + = &actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(nullable_col), func_node->result_type, getUniqueName("null"))); + DB::ActionsDAG::NodeRawConstPtrs convert_nan_func_args = {is_nan_func_node, null_node, func_node}; + func_node = toFunctionNode(actions_dag, "if", func_node->result_name, convert_nan_func_args); + actions_dag.addOrReplaceInOutputs(*func_node); + return func_node; +} + AggregateFunctionParserFactory & AggregateFunctionParserFactory::instance() { static AggregateFunctionParserFactory factory; diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.h b/cpp-ch/local-engine/Parser/AggregateFunctionParser.h index 02b09fc256f4..a41b3e3ad956 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.h +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.h @@ -144,6 +144,9 @@ class AggregateFunctionParser std::pair parseLiteral(const substrait::Expression_Literal & literal) const; + const DB::ActionsDAG::Node * convertNanToNullIfNeed( + const CommonFunctionInfo & func_info, const DB::ActionsDAG::Node * func_node, DB::ActionsDAG & actions_dag) const; + ParserContextPtr parser_context; std::unique_ptr expression_parser; Poco::Logger * logger = &Poco::Logger::get("AggregateFunctionParserFactory"); From bae4dbcbed6cd257a8c7f329f7924d3ef84df3ee Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Thu, 5 Dec 2024 03:44:48 -0600 Subject: [PATCH 198/211] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241205) (#8152) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20241205) * Fix Build due to https://github.com/ClickHouse/ClickHouse/pull/72747 * make LiteralParser::parse static --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 +- cpp-ch/local-engine/Common/AggregateUtil.cpp | 22 +++++-- .../Parser/AggregateFunctionParser.cpp | 2 +- .../local-engine/Parser/ExpressionParser.cpp | 32 ++++------- cpp-ch/local-engine/Parser/ExpressionParser.h | 12 ++-- cpp-ch/local-engine/Parser/FunctionParser.cpp | 2 +- .../scalar_function_parser/arrayExcept.cpp | 27 ++++----- .../arrayHighOrderFunctions.cpp | 4 +- .../scalar_function_parser/arrayRemove.cpp | 10 ++-- .../scalar_function_parser/arrayRepeat.cpp | 10 ++-- .../scalar_function_parser/lambdaFunction.cpp | 57 ++++++++----------- .../local-engine/tests/gtest_ch_functions.cpp | 9 ++- 12 files changed, 90 insertions(+), 101 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 999c8c957298..7c94e253c7b7 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241204 -CH_COMMIT=7b72d0dc991 +CH_BRANCH=rebase_ch/20241205 +CH_COMMIT=7b8b1c0b554 diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp index a6fedcc535af..36b26e94bd17 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.cpp +++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp @@ -237,19 +237,29 @@ DB::Aggregator::Params AggregatorParamsHelper::buildParams( aggregate_settings[DB::Setting::max_block_size] = max_block_size; aggregate_settings[DB::Setting::enable_software_prefetch_in_aggregation] = enable_prefetch; aggregate_settings[DB::Setting::optimize_group_by_constant_keys] = optimize_group_by_constant_keys; - DB::Aggregator::Params params( - aggregate_settings, + return DB::Aggregator::Params{ grouping_keys, agg_descriptions, - false, + /*overflow_row*/ false, + aggregate_settings[DB::Setting::max_rows_to_group_by], + aggregate_settings[DB::Setting::group_by_overflow_mode], group_by_two_level_threshold, group_by_two_level_threshold_bytes, + DB::Aggregator::Params::getMaxBytesBeforeExternalGroupBy( + aggregate_settings[DB::Setting::max_bytes_before_external_group_by], + aggregate_settings[DB::Setting::max_bytes_ratio_before_external_group_by]), empty_result_for_aggregation_by_empty_set, tmp_data_scope, + aggregate_settings[DB::Setting::max_threads], + aggregate_settings[DB::Setting::min_free_disk_space_for_temporary_data], + aggregate_settings[DB::Setting::compile_aggregate_expressions], + aggregate_settings[DB::Setting::min_count_to_compile_aggregate_expression], + aggregate_settings[DB::Setting::max_block_size], + aggregate_settings[DB::Setting::enable_software_prefetch_in_aggregation], only_merge, - {}); - - return params; + aggregate_settings[DB::Setting::optimize_group_by_constant_keys], + aggregate_settings[DB::Setting::min_hit_rate_to_use_consecutive_keys_optimization], + {}}; } diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp index eb05b26dcf94..5452972c6f7e 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp @@ -76,7 +76,7 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::parseExpression(DB::Action std::pair AggregateFunctionParser::parseLiteral(const substrait::Expression_Literal & literal) const { - return LiteralParser().parse(literal); + return LiteralParser::parse(literal); } DB::ActionsDAG::NodeRawConstPtrs diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.cpp b/cpp-ch/local-engine/Parser/ExpressionParser.cpp index 9899e90ddcda..e7de46483eda 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.cpp +++ b/cpp-ch/local-engine/Parser/ExpressionParser.cpp @@ -15,7 +15,7 @@ * limitations under the License. */ #include "ExpressionParser.h" -#include +#include #include #include #include @@ -31,17 +31,13 @@ #include #include #include -#include #include #include #include #include -#include #include #include -#include #include -#include "SerializedPlanParser.h" namespace DB { @@ -55,7 +51,7 @@ extern const int BAD_ARGUMENTS; namespace local_engine { -std::pair LiteralParser::parse(const substrait::Expression_Literal & literal) const +std::pair LiteralParser::parse(const substrait::Expression_Literal & literal) { DB::DataTypePtr type; DB::Field field; @@ -274,7 +270,7 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ case substrait::Expression::RexTypeCase::kLiteral: { DB::DataTypePtr type; DB::Field field; - std::tie(type, field) = LiteralParser().parse(rel.literal()); + std::tie(type, field) = LiteralParser::parse(rel.literal()); return addConstColumn(actions_dag, type, field); } @@ -333,7 +329,8 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ // ISSUE-7389: spark cast(map to string) has different behavior with CH cast(map to string) auto map_input_type = std::static_pointer_cast(denull_input_type); args.emplace_back(addConstColumn(actions_dag, map_input_type->getKeyType(), map_input_type->getKeyType()->getDefault())); - args.emplace_back(addConstColumn(actions_dag, map_input_type->getValueType(), map_input_type->getValueType()->getDefault())); + args.emplace_back( + addConstColumn(actions_dag, map_input_type->getValueType(), map_input_type->getValueType()->getDefault())); result_node = toFunctionNode(actions_dag, "sparkCastMapToString", args); } else if (isString(denull_input_type) && substrait_type.has_bool_()) @@ -411,7 +408,7 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ args.emplace_back(parseExpression(actions_dag, rel.singular_or_list().value())); bool nullable = false; - int options_len = static_cast(options.size()); + int options_len = options.size(); for (int i = 0; i < options_len; ++i) { if (!options[i].has_literal()) @@ -421,15 +418,14 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ } DB::DataTypePtr elem_type; - LiteralParser literal_parser; - std::tie(elem_type, std::ignore) = literal_parser.parse(options[0].literal()); + std::tie(elem_type, std::ignore) = LiteralParser::parse(options[0].literal()); elem_type = wrapNullableType(nullable, elem_type); DB::MutableColumnPtr elem_column = elem_type->createColumn(); elem_column->reserve(options_len); for (int i = 0; i < options_len; ++i) { - auto type_and_field = LiteralParser().parse(options[i].literal()); + auto type_and_field = LiteralParser::parse(options[i].literal()); auto option_type = wrapNullableType(nullable, type_and_field.first); if (!elem_type->equals(*option_type)) throw DB::Exception( @@ -440,16 +436,12 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ elem_column->insert(type_and_field.second); } - - DB::MutableColumns elem_columns; - elem_columns.emplace_back(std::move(elem_column)); - auto name = getUniqueName("__set"); - DB::Block elem_block; - elem_block.insert(DB::ColumnWithTypeAndName(nullptr, elem_type, name)); - elem_block.setColumns(std::move(elem_columns)); + ColumnWithTypeAndName elem_block{std::move(elem_column), elem_type, name}; - auto future_set = std::make_shared(elem_block, context->queryContext()->getSettingsRef()); + PreparedSets prepared_sets; + FutureSet::Hash emptyKey; + auto future_set = prepared_sets.addFromTuple(emptyKey, {elem_block}, context->queryContext()->getSettingsRef()); auto arg = DB::ColumnSet::create(1, std::move(future_set)); args.emplace_back(&actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(arg), std::make_shared(), name))); diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.h b/cpp-ch/local-engine/Parser/ExpressionParser.h index 95c66146ab3d..06a80d756e3f 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.h +++ b/cpp-ch/local-engine/Parser/ExpressionParser.h @@ -16,17 +16,16 @@ */ #pragma once +#include #include #include #include #include -#include -#include "SerializedPlanParser.h" namespace local_engine { -struct ParserContext; +class ParserContext; class SerializedPlanParser; class LiteralParser @@ -34,17 +33,17 @@ class LiteralParser public: /// Parse a substrait literal into a CH field /// returns are the type and field value. - std::pair parse(const substrait::Expression_Literal & literal) const; + static std::pair parse(const substrait::Expression_Literal & literal); }; class ExpressionParser { public: - ExpressionParser(std::shared_ptr context_) : context(context_) { } + ExpressionParser(const std::shared_ptr & context_) : context(context_) { } ~ExpressionParser() = default; - /// Append a counter suffix to name + /// Append a counter-suffix to name String getUniqueName(const String & name) const; const DB::ActionsDAG::Node * addConstColumn(DB::ActionsDAG & actions_dag, const DB::DataTypePtr type, const DB::Field & field) const; @@ -84,6 +83,5 @@ class ExpressionParser const substrait::Expression_ScalarFunction & func, DB::ActionsDAG & actions_dag, bool position, bool & is_map) const; DB::ActionsDAG::NodeRawConstPtrs parseJsonTuple(const substrait::Expression_ScalarFunction & func, DB::ActionsDAG & actions_dag) const; - }; } diff --git a/cpp-ch/local-engine/Parser/FunctionParser.cpp b/cpp-ch/local-engine/Parser/FunctionParser.cpp index 7e794dabec64..581ab65f6114 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/FunctionParser.cpp @@ -92,7 +92,7 @@ const DB::ActionsDAG::Node * FunctionParser::parseExpression(DB::ActionsDAG & ac std::pair FunctionParser::parseLiteral(const substrait::Expression_Literal & literal) const { - return LiteralParser().parse(literal); + return LiteralParser::parse(literal); } ActionsDAG::NodeRawConstPtrs diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp index a9a0f305a0ac..935aad6244bd 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp @@ -69,16 +69,14 @@ class FunctionParserArrayExcept : public FunctionParser const auto * lambda_output = toFunctionNode(lambda_actions_dag, "not", {has_in_lambda}); lambda_actions_dag.getOutputs().push_back(lambda_output); lambda_actions_dag.removeUnusedActions(Names(1, lambda_output->result_name)); - - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names{arr2_in_lambda->result_name}; NamesAndTypesList lambda_arguments_names_and_types; lambda_arguments_names_and_types.emplace_back(x_in_lambda->result_name, x_in_lambda->result_type); - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, @@ -97,13 +95,16 @@ class FunctionParserArrayExcept : public FunctionParser const auto * arr2_is_null_node = toFunctionNode(actions_dag, "isNull", {arr2_arg}); const auto * null_array_node = addColumnToActionsDAG(actions_dag, std::make_shared(array_distinct_node->result_type), {}); - const auto * multi_if_node = toFunctionNode(actions_dag, "multiIf", { - arr1_is_null_node, - null_array_node, - arr2_is_null_node, - null_array_node, - array_distinct_node, - }); + const auto * multi_if_node = toFunctionNode( + actions_dag, + "multiIf", + { + arr1_is_null_node, + null_array_node, + arr2_is_null_node, + null_array_node, + array_distinct_node, + }); return convertNodeTypeIfNeeded(substrait_func, multi_if_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp index 7ab829585b79..1d418ba0733d 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp @@ -218,14 +218,14 @@ class FunctionParserArraySort : public FunctionParser const auto var_expr = expr.scalar_function().arguments()[0].value(); if (!var_expr.has_literal()) return false; - auto [_, name] = LiteralParser().parse(var_expr.literal()); + auto [_, name] = LiteralParser::parse(var_expr.literal()); return var == name.safeGet(); }; auto is_int_value = [&](const substrait::Expression & expr, Int32 val) { if (!expr.has_literal()) return false; - auto [_, x] = LiteralParser().parse(expr.literal()); + auto [_, x] = LiteralParser::parse(expr.literal()); return val == x.safeGet(); }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp index 27bd9f84a9e6..9bc67111f0d6 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp @@ -66,16 +66,14 @@ class FunctionParserArrayRemove : public FunctionParser const auto * lambda_output = if_null_in_lambda; lambda_actions_dag.getOutputs().push_back(lambda_output); lambda_actions_dag.removeUnusedActions(Names(1, lambda_output->result_name)); - - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names{elem_in_lambda->result_name}; NamesAndTypesList lambda_arguments_names_and_types; lambda_arguments_names_and_types.emplace_back(x_in_lambda->result_name, x_in_lambda->result_type); - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp index 04f4f64e7bd2..e7df0250c870 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp @@ -66,16 +66,14 @@ class FunctionParserArrayRepeat : public FunctionParser const auto * lambda_output = elem_in_lambda; lambda_actions_dag.getOutputs().push_back(lambda_output); lambda_actions_dag.removeUnusedActions(Names(1, lambda_output->result_name)); - - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names{elem_in_lambda->result_name}; NamesAndTypesList lambda_arguments_names_and_types; lambda_arguments_names_and_types.emplace_back(x_in_lambda->result_name, x_in_lambda->result_type); - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp index 2d61633e80ce..19e3bfca056f 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp @@ -17,10 +17,9 @@ #include #include #include +#include #include #include -#include -#include #include #include #include @@ -29,7 +28,7 @@ namespace DB::ErrorCodes { - extern const int LOGICAL_ERROR; +extern const int LOGICAL_ERROR; } namespace local_engine @@ -44,12 +43,10 @@ DB::NamesAndTypesList collectLambdaArguments(ParserContextPtr parser_context_, c if (arg.value().has_scalar_function() && parser_context_->getFunctionNameInSignature(arg.value().scalar_function().function_reference()) == "namedlambdavariable") { - auto [_, col_name_field] = LiteralParser().parse(arg.value().scalar_function().arguments()[0].value().literal()); + auto [_, col_name_field] = LiteralParser::parse(arg.value().scalar_function().arguments()[0].value().literal()); String col_name = col_name_field.safeGet(); if (collected_names.contains(col_name)) - { continue; - } collected_names.insert(col_name); auto type = TypeParser::parseType(arg.value().scalar_function().output_type()); lambda_arguments.emplace_back(col_name, type); @@ -63,7 +60,7 @@ class FunctionParserLambda : public FunctionParser { public: static constexpr auto name = "lambdafunction"; - explicit FunctionParserLambda(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + explicit FunctionParserLambda(ParserContextPtr parser_context_) : FunctionParser(parser_context_) { } ~FunctionParserLambda() override = default; String getName() const override { return name; } @@ -73,15 +70,14 @@ class FunctionParserLambda : public FunctionParser throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "getCHFunctionName is not implemented for LambdaFunction"); } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { /// Some special cases, for example, `transform(arr, x -> concat(arr, array(x)))` refers to /// a column `arr` out of it directly. We need a `arr` as an input column for `lambda_actions_dag` DB::NamesAndTypesList parent_header; for (const auto * output_node : actions_dag.getOutputs()) - { parent_header.emplace_back(output_node->result_name, output_node->result_type); - } ActionsDAG lambda_actions_dag{parent_header}; /// The first argument is the lambda function body, followings are the lambda arguments which is @@ -90,28 +86,21 @@ class FunctionParserLambda : public FunctionParser /// this outside lambda function's arguments. For an example, transform(number, x -> transform(letter, y -> struct(x, y))). /// Before parsing the lambda function body, we add lambda function arguments int actions dag at first. for (size_t i = 1; i < substrait_func.arguments().size(); ++i) - { (void)parseExpression(lambda_actions_dag, substrait_func.arguments()[i].value()); - } const auto & substrait_lambda_body = substrait_func.arguments()[0].value(); const auto * lambda_body_node = parseExpression(lambda_actions_dag, substrait_lambda_body); lambda_actions_dag.getOutputs().push_back(lambda_body_node); lambda_actions_dag.removeUnusedActions(Names(1, lambda_body_node->result_name)); - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names; - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); DB::ActionsDAG::NodeRawConstPtrs lambda_children; auto lambda_function_args = collectLambdaArguments(parser_context, substrait_func); - const auto & lambda_actions_inputs = lambda_actions->getActionsDAG().getInputs(); + const auto & lambda_actions_inputs = lambda_actions_dag.getInputs(); std::unordered_map parent_nodes; for (const auto & node : actions_dag.getNodes()) - { parent_nodes[node.result_name] = &node; - } for (const auto & required_column_name : required_column_names) { if (std::find_if( @@ -125,13 +114,13 @@ class FunctionParserLambda : public FunctionParser lambda_actions_inputs.end(), [&required_column_name](const auto & node) { return node->result_name == required_column_name; }); if (it == lambda_actions_inputs.end()) - { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Required column not found: {}", required_column_name); - } auto parent_node_it = parent_nodes.find(required_column_name); if (parent_node_it == parent_nodes.end()) { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Not found column {} in actions dag:\n{}", + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Not found column {} in actions dag:\n{}", required_column_name, actions_dag.dumpDAG()); } @@ -141,9 +130,10 @@ class FunctionParserLambda : public FunctionParser captured_column_names.push_back(required_column_name); } } - + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_function_args, lambda_body_node->result_type, @@ -153,10 +143,10 @@ class FunctionParserLambda : public FunctionParser const auto * result = &actions_dag.addFunction(function_capture, lambda_children, lambda_body_node->result_name); return result; } + protected: - DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAG & actions_dag) const override + DB::ActionsDAG::NodeRawConstPtrs + parseFunctionArguments(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "parseFunctionArguments is not implemented for LambdaFunction"); } @@ -177,7 +167,7 @@ class NamedLambdaVariable : public FunctionParser { public: static constexpr auto name = "namedlambdavariable"; - explicit NamedLambdaVariable(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + explicit NamedLambdaVariable(ParserContextPtr parser_context_) : FunctionParser(parser_context_) { } ~NamedLambdaVariable() override = default; String getName() const override { return name; } @@ -187,7 +177,8 @@ class NamedLambdaVariable : public FunctionParser throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "getCHFunctionName is not implemented for NamedLambdaVariable"); } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto [_, col_name_field] = parseLiteral(substrait_func.arguments()[0].value().literal()); String col_name = col_name_field.safeGet(); @@ -196,15 +187,13 @@ class NamedLambdaVariable : public FunctionParser const auto & inputs = actions_dag.getInputs(); auto it = std::find_if(inputs.begin(), inputs.end(), [&col_name](const auto * node) { return node->result_name == col_name; }); if (it == inputs.end()) - { return &(actions_dag.addInput(col_name, type)); - } return *it; } + protected: - DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAG & actions_dag) const override + DB::ActionsDAG::NodeRawConstPtrs + parseFunctionArguments(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "parseFunctionArguments is not implemented for NamedLambdaVariable"); } diff --git a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp index 3b91e0799404..c9bff195c5d4 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp @@ -79,7 +79,8 @@ TEST(TestFunction, In) set->setHeader(col1_set_block.getColumnsWithTypeAndName()); set->insertFromBlock(col1_set_block.getColumnsWithTypeAndName()); set->finishInsert(); - auto future_set = std::make_shared(std::move(set)); + PreparedSets::Hash empty; + auto future_set = std::make_shared(empty, std::move(set)); //TODO: WHY? after https://github.com/ClickHouse/ClickHouse/pull/63723 we need pass 4 instead of 1 auto arg = ColumnSet::create(4, future_set); @@ -122,7 +123,8 @@ TEST(TestFunction, NotIn1) set->setHeader(col1_set_block.getColumnsWithTypeAndName()); set->insertFromBlock(col1_set_block.getColumnsWithTypeAndName()); set->finishInsert(); - auto future_set = std::make_shared(std::move(set)); + PreparedSets::Hash empty; + auto future_set = std::make_shared(empty, std::move(set)); //TODO: WHY? after https://github.com/ClickHouse/ClickHouse/pull/63723 we need pass 4 instead of 1 auto arg = ColumnSet::create(4, future_set); @@ -165,7 +167,8 @@ TEST(TestFunction, NotIn2) set->setHeader(col1_set_block.getColumnsWithTypeAndName()); set->insertFromBlock(col1_set_block.getColumnsWithTypeAndName()); set->finishInsert(); - auto future_set = std::make_shared(std::move(set)); + PreparedSets::Hash empty; + auto future_set = std::make_shared(empty, std::move(set)); //TODO: WHY? after https://github.com/ClickHouse/ClickHouse/pull/63723 we need pass 4 instead of 1 auto arg = ColumnSet::create(4, future_set); From 98b15c4f2d11919fa9e8019754f175490465f1c0 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 5 Dec 2024 21:02:48 +0800 Subject: [PATCH 199/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_12_05) (#8147) Upstream Velox's New Commits: 03f08942f by duanmeng, fix: Flaky HashJoinReplayerTest.partialDriverIds (11742) 28c319e6e by aditi-pandit, refactor: Change C style casts to C++ style (Part 1) (11680) bf857d92a by Richard Barnes, Remove unused-variable in velox/common/base/tests/IndexedPriorityQueueTest.cpp +53 (11737) d456517ec by yingsu00, misc: Improve ExchangeBenchmark (11543) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 962a3b415bd4..5cd5716f79ed 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_12_04 +VELOX_BRANCH=2024_12_05 VELOX_HOME="" OS=`uname -s` From 731c5b5826a24ef789a3bbaa9f16cc91f562422c Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Thu, 5 Dec 2024 22:32:16 +0800 Subject: [PATCH 200/211] [VL] Minor fix for cpp code style (part 1) (#8157) --- cpp/CMakeLists.txt | 3 - cpp/velox/CMakeLists.txt | 16 ++-- cpp/velox/compute/WholeStageResultIterator.h | 2 +- cpp/velox/jni/JniFileSystem.h | 6 +- cpp/velox/memory/VeloxMemoryManager.cc | 1 + .../functions/RegistrationAllFunctions.cc | 2 + .../functions/RowConstructorWithNull.cc | 2 + .../functions/RowConstructorWithNull.h | 2 + .../operators/functions/RowFunctionWithNull.h | 6 +- .../SparkExprToSubfieldFilterParser.cc | 1 + .../SparkExprToSubfieldFilterParser.h | 1 + .../operators/plannodes/RowVectorStream.h | 7 +- .../operators/reader/FileReaderIterator.cc | 1 + .../operators/reader/FileReaderIterator.h | 1 + .../operators/reader/ParquetReaderIterator.cc | 3 +- .../VeloxColumnarBatchSerializer.cc | 4 +- .../serializer/VeloxRowToColumnarConverter.cc | 2 + .../operators/writer/VeloxArrowWriter.cc | 1 + cpp/velox/operators/writer/VeloxArrowWriter.h | 2 + .../writer/VeloxParquetDataSource.cc | 1 - .../writer/VeloxParquetDataSourceABFS.h | 1 + .../writer/VeloxParquetDataSourceGCS.h | 2 + .../writer/VeloxParquetDataSourceHDFS.h | 1 + .../writer/VeloxParquetDataSourceS3.h | 1 + cpp/velox/shuffle/GlutenByteStream.h | 2 + cpp/velox/shuffle/RadixSort.h | 84 +++++++++---------- cpp/velox/shuffle/VeloxShuffleWriter.cc | 2 + cpp/velox/shuffle/VeloxSortShuffleWriter.cc | 4 +- cpp/velox/shuffle/VeloxSortShuffleWriter.h | 1 + .../SubstraitToVeloxPlanValidator.cc | 2 +- cpp/velox/substrait/TypeUtils.h | 1 + .../substrait/VariantToVectorConverter.cc | 1 + cpp/velox/substrait/VeloxToSubstraitPlan.cc | 1 - cpp/velox/tests/BufferOutputStreamTest.cc | 2 + cpp/velox/tests/FunctionTest.cc | 2 + cpp/velox/tests/MemoryManagerTest.cc | 2 + cpp/velox/tests/SparkFunctionTest.cc | 1 + .../Substrait2VeloxPlanConversionTest.cc | 2 + .../tests/Substrait2VeloxPlanValidatorTest.cc | 2 + ...Substrait2VeloxValuesNodeConversionTest.cc | 2 + cpp/velox/tests/VeloxBatchResizerTest.cc | 1 + .../tests/VeloxColumnarBatchSerializerTest.cc | 1 + cpp/velox/tests/VeloxColumnarBatchTest.cc | 1 + cpp/velox/tests/VeloxColumnarToRowTest.cc | 2 + cpp/velox/tests/VeloxRowToColumnarTest.cc | 2 + cpp/velox/tests/VeloxShuffleWriterTest.cc | 2 +- .../tests/VeloxSubstraitRoundTripTest.cc | 1 + cpp/velox/tests/VeloxToSubstraitTypeTest.cc | 1 + cpp/velox/udf/Udaf.h | 1 + cpp/velox/udf/UdfLoader.h | 1 + cpp/velox/udf/examples/MyUDAF.cc | 1 + cpp/velox/udf/examples/MyUDF.cc | 2 + cpp/velox/utils/HdfsUtils.cc | 3 + cpp/velox/utils/VeloxBatchResizer.cc | 3 +- cpp/velox/utils/VeloxBatchResizer.h | 2 + cpp/velox/utils/tests/MemoryPoolUtils.cc | 3 +- cpp/velox/utils/tests/MemoryPoolUtils.h | 8 +- 57 files changed, 133 insertions(+), 82 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index e0ab141a1f96..5cc1d4ee4b19 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -236,10 +236,7 @@ if(ENABLE_IAA) add_definitions(-DGLUTEN_ENABLE_IAA) endif() -# # Subdirectories -# - add_subdirectory(core) if(BUILD_VELOX_BACKEND) diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 17b824b6190f..4dc5ec0aff43 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -140,12 +140,12 @@ endmacro() set(VELOX_SRCS compute/VeloxBackend.cc compute/VeloxRuntime.cc - compute/WholeStageResultIterator.cc compute/VeloxPlanConverter.cc + compute/WholeStageResultIterator.cc compute/iceberg/IcebergPlanConverter.cc - jni/VeloxJniWrapper.cc jni/JniFileSystem.cc jni/JniUdf.cc + jni/VeloxJniWrapper.cc memory/BufferOutputStream.cc memory/VeloxColumnarBatch.cc memory/VeloxMemoryManager.cc @@ -154,30 +154,30 @@ set(VELOX_SRCS operators/functions/SparkExprToSubfieldFilterParser.cc operators/reader/FileReaderIterator.cc operators/reader/ParquetReaderIterator.cc - operators/serializer/VeloxColumnarToRowConverter.cc operators/serializer/VeloxColumnarBatchSerializer.cc + operators/serializer/VeloxColumnarToRowConverter.cc operators/serializer/VeloxRowToColumnarConverter.cc operators/writer/VeloxArrowWriter.cc operators/writer/VeloxParquetDataSource.cc + shuffle/VeloxHashShuffleWriter.cc + shuffle/VeloxRssSortShuffleWriter.cc shuffle/VeloxShuffleReader.cc shuffle/VeloxShuffleWriter.cc - shuffle/VeloxHashShuffleWriter.cc shuffle/VeloxSortShuffleWriter.cc - shuffle/VeloxRssSortShuffleWriter.cc + substrait/SubstraitExtensionCollector.cc substrait/SubstraitParser.cc substrait/SubstraitToVeloxExpr.cc substrait/SubstraitToVeloxPlan.cc substrait/SubstraitToVeloxPlanValidator.cc substrait/VariantToVectorConverter.cc - substrait/SubstraitExtensionCollector.cc substrait/VeloxSubstraitSignature.cc substrait/VeloxToSubstraitExpr.cc substrait/VeloxToSubstraitPlan.cc substrait/VeloxToSubstraitType.cc udf/UdfLoader.cc - utils/VeloxArrowUtils.cc - utils/ConfigExtractor.cc utils/Common.cc + utils/ConfigExtractor.cc + utils/VeloxArrowUtils.cc utils/VeloxBatchResizer.cc) if(ENABLE_S3) diff --git a/cpp/velox/compute/WholeStageResultIterator.h b/cpp/velox/compute/WholeStageResultIterator.h index 9b1f2b728802..d0dd47bfd1e3 100644 --- a/cpp/velox/compute/WholeStageResultIterator.h +++ b/cpp/velox/compute/WholeStageResultIterator.h @@ -120,7 +120,7 @@ class WholeStageResultIterator : public ColumnarBatchIterator { /// All the children plan node ids with postorder traversal. std::vector orderedNodeIds_; - /// Node ids should be ommited in metrics. + /// Node ids should be omitted in metrics. std::unordered_set omittedNodeIds_; std::vector scanNodeIds_; std::vector> scanInfos_; diff --git a/cpp/velox/jni/JniFileSystem.h b/cpp/velox/jni/JniFileSystem.h index 9313e7af7b09..11c0f27b369e 100644 --- a/cpp/velox/jni/JniFileSystem.h +++ b/cpp/velox/jni/JniFileSystem.h @@ -24,9 +24,9 @@ namespace gluten { -// Register JNI-or-local (or JVM-over-local, as long as it describes what happens here) -// file system. maxFileSize is necessary (!= 0) because we use this size to decide -// whether a new file can fit in JVM heap, otherwise we write it via local fs directly. +// Register JNI-or-local (or JVM-over-local, as long as it describes what happens here) file system. maxFileSize is +// necessary (!= 0) because we use this size to decide whether a new file can fit in JVM heap, otherwise we write it via +// local fs directly. void registerJolFileSystem(uint64_t maxFileSize); void initVeloxJniFileSystem(JNIEnv* env); diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 63a61ae2ddda..5e6fdd08a2db 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -59,6 +59,7 @@ T getConfig( return defaultValue; } } // namespace + /// We assume in a single Spark task. No thread-safety should be guaranteed. class ListenableArbitrator : public velox::memory::MemoryArbitrator { public: diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.cc b/cpp/velox/operators/functions/RegistrationAllFunctions.cc index 06147d6d43e5..5d46dbdcdd0e 100644 --- a/cpp/velox/operators/functions/RegistrationAllFunctions.cc +++ b/cpp/velox/operators/functions/RegistrationAllFunctions.cc @@ -43,8 +43,10 @@ void registerPrestoVectorFunctions() { VELOX_REGISTER_VECTOR_FUNCTION(udf_transform_values, "transform_values"); } } // namespace facebook::velox::functions + namespace gluten { namespace { + void registerFunctionOverwrite() { velox::functions::registerUnaryNumeric({"round"}); velox::registerFunction({"round"}); diff --git a/cpp/velox/operators/functions/RowConstructorWithNull.cc b/cpp/velox/operators/functions/RowConstructorWithNull.cc index e8b8a288360b..4d052e0df279 100644 --- a/cpp/velox/operators/functions/RowConstructorWithNull.cc +++ b/cpp/velox/operators/functions/RowConstructorWithNull.cc @@ -19,6 +19,7 @@ #include "velox/expression/VectorFunction.h" namespace gluten { + facebook::velox::TypePtr RowConstructorWithNullCallToSpecialForm::resolveType( const std::vector& argTypes) { auto numInput = argTypes.size(); @@ -52,4 +53,5 @@ facebook::velox::exec::ExprPtr RowConstructorWithNullCallToSpecialForm::construc return std::make_shared( type, std::move(compiledChildren), function, metadata, name, trackCpuUsage); } + } // namespace gluten diff --git a/cpp/velox/operators/functions/RowConstructorWithNull.h b/cpp/velox/operators/functions/RowConstructorWithNull.h index 66b745e3ed9b..2565ba42f382 100644 --- a/cpp/velox/operators/functions/RowConstructorWithNull.h +++ b/cpp/velox/operators/functions/RowConstructorWithNull.h @@ -21,6 +21,7 @@ #include "velox/expression/SpecialForm.h" namespace gluten { + class RowConstructorWithNullCallToSpecialForm : public facebook::velox::exec::FunctionCallToSpecialForm { public: RowConstructorWithNullCallToSpecialForm(const std::string& rowFunctionName) { @@ -49,4 +50,5 @@ class RowConstructorWithNullCallToSpecialForm : public facebook::velox::exec::Fu private: std::string rowFunctionName; }; + } // namespace gluten diff --git a/cpp/velox/operators/functions/RowFunctionWithNull.h b/cpp/velox/operators/functions/RowFunctionWithNull.h index 4131fb472ddd..6477449fb0dc 100644 --- a/cpp/velox/operators/functions/RowFunctionWithNull.h +++ b/cpp/velox/operators/functions/RowFunctionWithNull.h @@ -22,10 +22,8 @@ namespace gluten { -/** - * @tparam allNull If true, set struct as null when all of arguments are all, else will - * set it null when one of its arguments is null. - */ +///@tparam allNull If true, set struct as null when all of arguments are all, else will +/// set it null when one of its arguments is null. template class RowFunctionWithNull final : public facebook::velox::exec::VectorFunction { public: diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc index 8ad537d8168a..26614690e141 100644 --- a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc @@ -100,4 +100,5 @@ std::unique_ptr SparkExprToSubfieldFilterParser::leafCallToSubfi } return nullptr; } + } // namespace gluten diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h index d050091c0793..11e1b401a299 100644 --- a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h @@ -34,4 +34,5 @@ class SparkExprToSubfieldFilterParser : public facebook::velox::exec::ExprToSubf // Compared to the upstream 'toSubfield', the push-down of filter on nested field is disabled. bool toSparkSubfield(const facebook::velox::core::ITypedExpr* field, facebook::velox::common::Subfield& subfield); }; + } // namespace gluten diff --git a/cpp/velox/operators/plannodes/RowVectorStream.h b/cpp/velox/operators/plannodes/RowVectorStream.h index 63ddd0abe00d..d503eb2d14c3 100644 --- a/cpp/velox/operators/plannodes/RowVectorStream.h +++ b/cpp/velox/operators/plannodes/RowVectorStream.h @@ -24,6 +24,7 @@ #include "velox/exec/Task.h" namespace { + class SuspendedSection { public: explicit SuspendedSection(facebook::velox::exec::Driver* driver) : driver_(driver) { @@ -42,9 +43,11 @@ class SuspendedSection { private: facebook::velox::exec::Driver* const driver_; }; + } // namespace namespace gluten { + class RowVectorStream { public: explicit RowVectorStream( @@ -67,7 +70,7 @@ class RowVectorStream { // possibility that this spill call hangs. See https://github.com/apache/incubator-gluten/issues/7243. // As of now, non-zero running threads usually happens when: // 1. Task A spills task B; - // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again. + // 2. Task A tries to grow buffers created by task B, during which spill is requested on task A again. SuspendedSection ss(driverCtx_->driver); hasNext = iterator_->hasNext(); } @@ -77,7 +80,7 @@ class RowVectorStream { return hasNext; } - // Convert arrow batch to rowvector and use new output columns + // Convert arrow batch to row vector and use new output columns facebook::velox::RowVectorPtr next() { if (finished_) { return nullptr; diff --git a/cpp/velox/operators/reader/FileReaderIterator.cc b/cpp/velox/operators/reader/FileReaderIterator.cc index d732adbf33c0..49d52f5ffd06 100644 --- a/cpp/velox/operators/reader/FileReaderIterator.cc +++ b/cpp/velox/operators/reader/FileReaderIterator.cc @@ -49,4 +49,5 @@ std::shared_ptr FileReaderIterator::getInputIteratorFrom } throw new GlutenException("Unreachable."); } + } // namespace gluten diff --git a/cpp/velox/operators/reader/FileReaderIterator.h b/cpp/velox/operators/reader/FileReaderIterator.h index e782c2bf80f4..708120603e42 100644 --- a/cpp/velox/operators/reader/FileReaderIterator.h +++ b/cpp/velox/operators/reader/FileReaderIterator.h @@ -22,6 +22,7 @@ #include "velox/common/memory/MemoryPool.h" namespace gluten { + enum FileReaderType { kBuffered, kStream, kNone }; class FileReaderIterator : public ColumnarBatchIterator { diff --git a/cpp/velox/operators/reader/ParquetReaderIterator.cc b/cpp/velox/operators/reader/ParquetReaderIterator.cc index 3e61e1d8d936..014bccbfce9c 100644 --- a/cpp/velox/operators/reader/ParquetReaderIterator.cc +++ b/cpp/velox/operators/reader/ParquetReaderIterator.cc @@ -94,4 +94,5 @@ void ParquetBufferedReaderIterator::collectBatches() { auto endTime = std::chrono::steady_clock::now(); collectBatchTime_ += std::chrono::duration_cast(endTime - startTime).count(); } -} // namespace gluten \ No newline at end of file + +} // namespace gluten diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc index 9c5d166a07da..7b858cce21b2 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc @@ -30,14 +30,15 @@ using namespace facebook::velox; namespace gluten { - namespace { + std::unique_ptr toByteStream(uint8_t* data, int32_t size) { std::vector byteRanges; byteRanges.push_back(ByteRange{data, size, 0}); auto byteStream = std::make_unique(byteRanges); return byteStream; } + } // namespace VeloxColumnarBatchSerializer::VeloxColumnarBatchSerializer( @@ -89,4 +90,5 @@ std::shared_ptr VeloxColumnarBatchSerializer::deserialize(uint8_t serde_->deserialize(byteStream.get(), veloxPool_.get(), rowType_, &result, &options_); return std::make_shared(result); } + } // namespace gluten diff --git a/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc b/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc index 5dec7db23af4..19a2bbafd2c2 100644 --- a/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc +++ b/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc @@ -255,6 +255,7 @@ bool supporteType(const RowTypePtr rowType) { } } // namespace + VeloxRowToColumnarConverter::VeloxRowToColumnarConverter( struct ArrowSchema* cSchema, std::shared_ptr memoryPool) @@ -301,4 +302,5 @@ VeloxRowToColumnarConverter::convertPrimitive(int64_t numRows, int64_t* rowLengt auto rowVector = std::make_shared(pool_.get(), rowType_, BufferPtr(nullptr), numRows, std::move(columns)); return std::make_shared(rowVector); } + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxArrowWriter.cc b/cpp/velox/operators/writer/VeloxArrowWriter.cc index 565602d95cc9..e01c0aee9fe2 100644 --- a/cpp/velox/operators/writer/VeloxArrowWriter.cc +++ b/cpp/velox/operators/writer/VeloxArrowWriter.cc @@ -35,4 +35,5 @@ std::shared_ptr VeloxArrowWriter::retrieveColumnarBatch() { } return reader_->next(); } + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxArrowWriter.h b/cpp/velox/operators/writer/VeloxArrowWriter.h index 8b7998628716..e0a0861b3bc2 100644 --- a/cpp/velox/operators/writer/VeloxArrowWriter.h +++ b/cpp/velox/operators/writer/VeloxArrowWriter.h @@ -21,6 +21,7 @@ #include "operators/writer/ArrowWriter.h" namespace gluten { + class VeloxArrowWriter : public ArrowWriter { public: explicit VeloxArrowWriter(const std::string& path, int64_t batchSize, facebook::velox::memory::MemoryPool* pool); @@ -32,4 +33,5 @@ class VeloxArrowWriter : public ArrowWriter { facebook::velox::memory::MemoryPool* pool_; std::unique_ptr reader_{nullptr}; }; + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSource.cc b/cpp/velox/operators/writer/VeloxParquetDataSource.cc index aeec1b4c82d7..07f49c06ed1f 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSource.cc +++ b/cpp/velox/operators/writer/VeloxParquetDataSource.cc @@ -38,7 +38,6 @@ using namespace facebook::velox::common; using namespace facebook::velox::filesystems; namespace gluten { - namespace { const int32_t kGzipWindowBits4k = 12; } diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h index 8a417e1c22d2..f8f6e5878cff 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h @@ -51,4 +51,5 @@ class VeloxParquetDataSourceABFS final : public VeloxParquetDataSource { abfsFileSystem->openFileForWrite(filePath_, {{}, sinkPool_.get()}), filePath_); } }; + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h index 579a425d1833..80c224194c25 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h @@ -32,6 +32,7 @@ #include "velox/dwio/common/Options.h" namespace gluten { + class VeloxParquetDataSourceGCS final : public VeloxParquetDataSource { public: VeloxParquetDataSourceGCS( @@ -48,4 +49,5 @@ class VeloxParquetDataSourceGCS final : public VeloxParquetDataSource { gcsFileSystem->openFileForWrite(filePath_, {{}, sinkPool_.get()}), filePath_); } }; + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h index 053b3da2ff7f..5f61d9145f46 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h @@ -48,4 +48,5 @@ class VeloxParquetDataSourceHDFS final : public VeloxParquetDataSource { sink_ = dwio::common::FileSink::create(filePath_, {.connectorProperties = hiveConf, .pool = sinkPool_.get()}); } }; + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h b/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h index 3082f82a9193..788eda19e5d2 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h @@ -48,4 +48,5 @@ class VeloxParquetDataSourceS3 final : public VeloxParquetDataSource { sink_ = dwio::common::FileSink::create(filePath_, {.connectorProperties = hiveConf, .pool = sinkPool_.get()}); } }; + } // namespace gluten diff --git a/cpp/velox/shuffle/GlutenByteStream.h b/cpp/velox/shuffle/GlutenByteStream.h index 78ea7b905adc..cc63b730ebca 100644 --- a/cpp/velox/shuffle/GlutenByteStream.h +++ b/cpp/velox/shuffle/GlutenByteStream.h @@ -19,6 +19,7 @@ #include "velox/common/memory/ByteStream.h" namespace facebook::velox { + class GlutenByteInputStream : public ByteInputStream { protected: /// TODO Remove after refactoring SpillInput. @@ -264,4 +265,5 @@ inline int128_t GlutenByteInputStream::read() { readBytes(reinterpret_cast(&value), sizeof(value)); return value; } + } // namespace facebook::velox diff --git a/cpp/velox/shuffle/RadixSort.h b/cpp/velox/shuffle/RadixSort.h index 69b87346c54c..aa078ccd06e0 100644 --- a/cpp/velox/shuffle/RadixSort.h +++ b/cpp/velox/shuffle/RadixSort.h @@ -27,21 +27,19 @@ namespace gluten { // https://github.com/apache/spark/blob/308669fc301916837bacb7c3ec1ecef93190c094/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java#L25 class RadixSort { public: - /** - * Sorts a given array of longs using least-significant-digit radix sort. This routine assumes - * you have extra space at the end of the array at least equal to the number of records. The - * sort is destructive and may relocate the data positioned within the array. - * - * @param array array of long elements followed by at least that many empty slots. - * @param numRecords number of data records in the array. - * @param startByteIndex the first byte (in range [0, 7]) to sort each long by, counting from the - * least significant byte. - * @param endByteIndex the last byte (in range [0, 7]) to sort each long by, counting from the - * least significant byte. Must be greater than startByteIndex. - * - * @return The starting index of the sorted data within the given array. We return this instead - * of always copying the data back to position zero for efficiency. - */ + // Sorts a given array of longs using least-significant-digit radix sort. This routine assumes + // you have extra space at the end of the array at least equal to the number of records. The + // sort is destructive and may relocate the data positioned within the array. + // + // @param array array of long elements followed by at least that many empty slots. + // @param numRecords number of data records in the array. + // @param startByteIndex the first byte (in range [0, 7]) to sort each long by, counting from the + // least significant byte. + // @param endByteIndex the last byte (in range [0, 7]) to sort each long by, counting from the + // least significant byte. Must be greater than startByteIndex. + // + // @return The starting index of the sorted data within the given array. We return this instead + // of always copying the data back to position zero for efficiency. static int32_t sort(uint64_t* array, size_t size, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { assert(startByteIndex >= 0 && "startByteIndex should >= 0"); assert(endByteIndex <= 7 && "endByteIndex should <= 7"); @@ -66,17 +64,15 @@ class RadixSort { } private: - /** - * Performs a partial sort by copying data into destination offsets for each byte value at the - * specified byte offset. - * - * @param array array to partially sort. - * @param numRecords number of data records in the array. - * @param counts counts for each byte value. This routine destructively modifies this array. - * @param byteIdx the byte in a long to sort at, counting from the least significant byte. - * @param inIndex the starting index in the array where input data is located. - * @param outIndex the starting index where sorted output data should be written. - */ + // Performs a partial sort by copying data into destination offsets for each byte value at the + // specified byte offset. + // + // @param array array to partially sort. + // @param numRecords number of data records in the array. + // @param counts counts for each byte value. This routine destructively modifies this array. + // @param byteIdx the byte in a long to sort at, counting from the least significant byte. + // @param inIndex the starting index in the array where input data is located. + // @param outIndex the starting index where sorted output data should be written. static void sortAtByte( uint64_t* array, int64_t numRecords, @@ -94,17 +90,15 @@ class RadixSort { } } - /** - * Computes a value histogram for each byte in the given array. - * - * @param array array to count records in. - * @param numRecords number of data records in the array. - * @param startByteIndex the first byte to compute counts for (the prior are skipped). - * @param endByteIndex the last byte to compute counts for. - * - * @return a vector of eight 256-element count arrays, one for each byte starting from the least - * significant byte. If the byte does not need sorting the vector entry will be empty. - */ + // Computes a value histogram for each byte in the given array. + // + // @param array array to count records in. + // @param numRecords number of data records in the array. + // @param startByteIndex the first byte to compute counts for (the prior are skipped). + // @param endByteIndex the last byte to compute counts for. + // + // @return a vector of eight 256-element count arrays, one for each byte starting from the least + // significant byte. If the byte does not need sorting the vector entry will be empty. static std::vector> getCounts(uint64_t* array, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { std::vector> counts; @@ -134,15 +128,13 @@ class RadixSort { return counts; } - /** - * Transforms counts into the proper output offsets for the sort type. - * - * @param counts counts for each byte value. This routine destructively modifies this vector. - * @param numRecords number of data records in the original data array. - * @param outputOffset output offset in bytes from the base array object. - * - * @return the input counts vector. - */ + // Transforms counts into the proper output offsets for the sort type. + // + // @param counts counts for each byte value. This routine destructively modifies this vector. + // @param numRecords number of data records in the original data array. + // @param outputOffset output offset in bytes from the base array object. + // + // @return the input counts vector. static std::vector& transformCountsToOffsets(std::vector& counts, int64_t outputOffset) { assert(counts.size() == 256); diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.cc b/cpp/velox/shuffle/VeloxShuffleWriter.cc index 69fd6801df54..06b59cf3ebdd 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxShuffleWriter.cc @@ -21,6 +21,7 @@ #include "shuffle/VeloxSortShuffleWriter.h" namespace gluten { + arrow::Result> VeloxShuffleWriter::create( ShuffleWriterType type, uint32_t numPartitions, @@ -43,4 +44,5 @@ arrow::Result> VeloxShuffleWriter::create( return arrow::Status::Invalid("Unsupported shuffle writer type: ", std::to_string(type)); } } + } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc index ab37c0be74ad..793130a58adc 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc @@ -25,8 +25,8 @@ #include "utils/Timer.h" namespace gluten { - namespace { + constexpr uint32_t kMaskLower27Bits = (1 << 27) - 1; constexpr uint64_t kMaskLower40Bits = (1UL << 40) - 1; constexpr uint32_t kPartitionIdStartByteIndex = 5; @@ -44,6 +44,7 @@ uint32_t extractPartitionId(uint64_t compactRowId) { std::pair extractPageNumberAndOffset(uint64_t compactRowId) { return {(compactRowId & kMaskLower40Bits) >> 27, compactRowId & kMaskLower27Bits}; } + } // namespace arrow::Result> VeloxSortShuffleWriter::create( @@ -449,4 +450,5 @@ void VeloxSortShuffleWriter::updateSpillMetrics(const std::unique_ptrrawSize(); } } + } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.h b/cpp/velox/shuffle/VeloxSortShuffleWriter.h index 5b8cff452d56..97c040ba5930 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.h @@ -131,4 +131,5 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { int64_t sortTime_{0}; bool stopped_{false}; }; + } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc index 9325fed3217c..153db7097002 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc @@ -27,8 +27,8 @@ #include "velox/expression/SignatureBinder.h" namespace gluten { - namespace { + const char* extractFileName(const char* file) { return strrchr(file, '/') ? strrchr(file, '/') + 1 : file; } diff --git a/cpp/velox/substrait/TypeUtils.h b/cpp/velox/substrait/TypeUtils.h index b2aaf725789d..20dd33031d71 100644 --- a/cpp/velox/substrait/TypeUtils.h +++ b/cpp/velox/substrait/TypeUtils.h @@ -22,6 +22,7 @@ using namespace facebook::velox; namespace gluten { + #ifndef RANGETRAITS_H #define RANGETRAITS_H diff --git a/cpp/velox/substrait/VariantToVectorConverter.cc b/cpp/velox/substrait/VariantToVectorConverter.cc index f8c9c11f0f37..f33633b91e50 100644 --- a/cpp/velox/substrait/VariantToVectorConverter.cc +++ b/cpp/velox/substrait/VariantToVectorConverter.cc @@ -67,4 +67,5 @@ VectorPtr setVectorFromVariantsByKind( VectorPtr setVectorFromVariants(const TypePtr& type, const std::vector& values, memory::MemoryPool* pool) { return VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH(setVectorFromVariantsByKind, type->kind(), values, type, pool); } + } // namespace gluten diff --git a/cpp/velox/substrait/VeloxToSubstraitPlan.cc b/cpp/velox/substrait/VeloxToSubstraitPlan.cc index 19259f81f629..44fbb01b39a6 100644 --- a/cpp/velox/substrait/VeloxToSubstraitPlan.cc +++ b/cpp/velox/substrait/VeloxToSubstraitPlan.cc @@ -20,7 +20,6 @@ #include "utils/Exception.h" namespace gluten { - namespace { struct AggregateCompanion { diff --git a/cpp/velox/tests/BufferOutputStreamTest.cc b/cpp/velox/tests/BufferOutputStreamTest.cc index 0e16f8c87769..b9ea62fd7b56 100644 --- a/cpp/velox/tests/BufferOutputStreamTest.cc +++ b/cpp/velox/tests/BufferOutputStreamTest.cc @@ -24,6 +24,7 @@ using namespace facebook::velox; namespace gluten { + class BufferOutputStreamTest : public ::testing::Test, public test::VectorTestBase { protected: // Velox requires the mem manager to be instanced. @@ -69,4 +70,5 @@ TEST_F(BufferOutputStreamTest, outputStream) { // We expect dropping the stream frees the backing memory. EXPECT_EQ(0, veloxPool_->usedBytes()); } + } // namespace gluten diff --git a/cpp/velox/tests/FunctionTest.cc b/cpp/velox/tests/FunctionTest.cc index b55b64ba9811..c149b2db38fc 100644 --- a/cpp/velox/tests/FunctionTest.cc +++ b/cpp/velox/tests/FunctionTest.cc @@ -34,6 +34,7 @@ using namespace facebook::velox; using namespace facebook::velox::test; namespace gluten { + class FunctionTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { @@ -212,4 +213,5 @@ TEST_F(FunctionTest, sigToTypes) { ASSERT_TRUE(types[0]->childAt(2)->isDecimal()); ASSERT_EQ(types[0]->childAt(3)->kind(), TypeKind::BOOLEAN); } + } // namespace gluten diff --git a/cpp/velox/tests/MemoryManagerTest.cc b/cpp/velox/tests/MemoryManagerTest.cc index 8794f9d3af2a..74bf2c58ef82 100644 --- a/cpp/velox/tests/MemoryManagerTest.cc +++ b/cpp/velox/tests/MemoryManagerTest.cc @@ -21,6 +21,7 @@ #include "velox/common/base/tests/GTestUtils.h" namespace gluten { + using namespace facebook::velox; class MockAllocationListener : public gluten::AllocationListener { @@ -398,4 +399,5 @@ TEST_F(MultiMemoryManagerTest, spill) { ASSERT_EQ(tmm.currentBytes(), 0); } + } // namespace gluten diff --git a/cpp/velox/tests/SparkFunctionTest.cc b/cpp/velox/tests/SparkFunctionTest.cc index 2105b155e3db..ae5e7c48d8ab 100644 --- a/cpp/velox/tests/SparkFunctionTest.cc +++ b/cpp/velox/tests/SparkFunctionTest.cc @@ -22,6 +22,7 @@ using namespace facebook::velox::functions::sparksql::test; using namespace facebook::velox; + class SparkFunctionTest : public SparkFunctionBaseTest { public: SparkFunctionTest() { diff --git a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc index 3193f53bfbf1..cccc619a86ff 100644 --- a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc +++ b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc @@ -35,6 +35,7 @@ using namespace facebook::velox::connector::hive; using namespace facebook::velox::exec; namespace gluten { + class Substrait2VeloxPlanConversionTest : public exec::test::HiveConnectorTestBase { protected: std::vector> makeSplits( @@ -275,4 +276,5 @@ TEST_F(Substrait2VeloxPlanConversionTest, filterUpper) { "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, remaining filter: (and(isnotnull(\"key\"),lessthan(\"key\",3)))] -> n0_0:INTEGER\n", planNode->toString(true, true)); } + } // namespace gluten diff --git a/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc b/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc index 0a957f038f8d..3f90c865df16 100644 --- a/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc +++ b/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc @@ -35,6 +35,7 @@ using namespace facebook::velox::connector::hive; using namespace facebook::velox::exec; namespace gluten { + class Substrait2VeloxPlanValidatorTest : public exec::test::HiveConnectorTestBase { protected: bool validatePlan(std::string file) { @@ -64,4 +65,5 @@ TEST_F(Substrait2VeloxPlanValidatorTest, group) { ASSERT_FALSE(validatePlan(substraitPlan)); } + } // namespace gluten diff --git a/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc b/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc index 75099db95976..b5a131790a64 100644 --- a/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc +++ b/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc @@ -32,6 +32,7 @@ using namespace facebook::velox::exec; using namespace facebook::velox::exec::test; namespace gluten { + class Substrait2VeloxValuesNodeConversionTest : public OperatorTestBase {}; // SELECT * FROM tmp @@ -57,4 +58,5 @@ TEST_F(Substrait2VeloxValuesNodeConversionTest, valuesNode) { createDuckDbTable({expectedData}); assertQuery(veloxPlan, "SELECT * FROM tmp"); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxBatchResizerTest.cc b/cpp/velox/tests/VeloxBatchResizerTest.cc index aecd52f927cc..fc741e015468 100644 --- a/cpp/velox/tests/VeloxBatchResizerTest.cc +++ b/cpp/velox/tests/VeloxBatchResizerTest.cc @@ -82,4 +82,5 @@ TEST_F(VeloxBatchResizerTest, sanity) { checkResize(100, 200, {5, 900, 700}, {5, 200, 200, 200, 200, 100, 200, 200, 200, 100}); ASSERT_ANY_THROW(checkResize(0, 0, {}, {})); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc index c00ab6a14844..333bb047f900 100644 --- a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc @@ -29,6 +29,7 @@ using namespace facebook::velox; namespace gluten { + class VeloxColumnarBatchSerializerTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { diff --git a/cpp/velox/tests/VeloxColumnarBatchTest.cc b/cpp/velox/tests/VeloxColumnarBatchTest.cc index ba66afb40fdf..3ad975751539 100644 --- a/cpp/velox/tests/VeloxColumnarBatchTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchTest.cc @@ -61,4 +61,5 @@ TEST_F(VeloxColumnarBatchTest, flattenTruncatedVector) { auto batchOfMap = std::make_shared(inputOfMap); ASSERT_NO_THROW(batchOfMap->getFlattenedRowVector()); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxColumnarToRowTest.cc b/cpp/velox/tests/VeloxColumnarToRowTest.cc index 3adacdda9d51..85bdf5b9ecda 100644 --- a/cpp/velox/tests/VeloxColumnarToRowTest.cc +++ b/cpp/velox/tests/VeloxColumnarToRowTest.cc @@ -27,6 +27,7 @@ using namespace facebook; using namespace facebook::velox; namespace gluten { + class VeloxColumnarToRowTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { @@ -92,4 +93,5 @@ TEST_F(VeloxColumnarToRowTest, Buffer_int64_int64_with_null) { }; testRowBufferAddr(vector, expectArr, sizeof(expectArr)); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxRowToColumnarTest.cc b/cpp/velox/tests/VeloxRowToColumnarTest.cc index 0d11dd4acbc9..937b8b80b6cd 100644 --- a/cpp/velox/tests/VeloxRowToColumnarTest.cc +++ b/cpp/velox/tests/VeloxRowToColumnarTest.cc @@ -26,6 +26,7 @@ using namespace facebook; using namespace facebook::velox; namespace gluten { + class VeloxRowToColumnarTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { @@ -141,4 +142,5 @@ TEST_F(VeloxRowToColumnarTest, timestamp) { }); testRowVectorEqual(vector); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxShuffleWriterTest.cc b/cpp/velox/tests/VeloxShuffleWriterTest.cc index ddcc30ff1ce9..e760a469b1e7 100644 --- a/cpp/velox/tests/VeloxShuffleWriterTest.cc +++ b/cpp/velox/tests/VeloxShuffleWriterTest.cc @@ -35,7 +35,6 @@ using namespace arrow; using namespace arrow::ipc; namespace gluten { - namespace { facebook::velox::RowVectorPtr takeRows( @@ -728,4 +727,5 @@ INSTANTIATE_TEST_SUITE_P( VeloxShuffleWriteParam, RangePartitioningShuffleWriter, ::testing::ValuesIn(kShuffleWriteTestParams)); + } // namespace gluten diff --git a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc index b675bf4a281e..a1cd59c6c05a 100644 --- a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc +++ b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc @@ -37,6 +37,7 @@ using namespace facebook::velox::test; using namespace facebook::velox::exec::test; namespace gluten { + class VeloxSubstraitRoundTripTest : public OperatorTestBase { protected: /// Makes a vector of INTEGER type with 'size' RowVectorPtr. diff --git a/cpp/velox/tests/VeloxToSubstraitTypeTest.cc b/cpp/velox/tests/VeloxToSubstraitTypeTest.cc index bc9e9df05c05..ea8ff790b26b 100644 --- a/cpp/velox/tests/VeloxToSubstraitTypeTest.cc +++ b/cpp/velox/tests/VeloxToSubstraitTypeTest.cc @@ -61,4 +61,5 @@ TEST_F(VeloxToSubstraitTypeTest, basic) { testTypeConversion(ROW({"a", "b", "c"}, {BIGINT(), ROW({"x", "y"}, {BOOLEAN(), VARCHAR()}), REAL()})); testTypeConversion(ROW({}, {})); } + } // namespace gluten diff --git a/cpp/velox/udf/Udaf.h b/cpp/velox/udf/Udaf.h index 4555bdfdf8a3..f421ecbc31e8 100644 --- a/cpp/velox/udf/Udaf.h +++ b/cpp/velox/udf/Udaf.h @@ -39,4 +39,5 @@ struct UdafEntry { #define GLUTEN_REGISTER_UDAF registerUdf #define DEFINE_REGISTER_UDAF extern "C" void GLUTEN_REGISTER_UDAF() + } // namespace gluten diff --git a/cpp/velox/udf/UdfLoader.h b/cpp/velox/udf/UdfLoader.h index 51264e67cc4d..8a28c2a35145 100644 --- a/cpp/velox/udf/UdfLoader.h +++ b/cpp/velox/udf/UdfLoader.h @@ -94,4 +94,5 @@ class UdfLoader { std::unordered_set> signatures_; std::unordered_set names_; }; + } // namespace gluten diff --git a/cpp/velox/udf/examples/MyUDAF.cc b/cpp/velox/udf/examples/MyUDAF.cc index 516404b55c3f..816880d2e71b 100644 --- a/cpp/velox/udf/examples/MyUDAF.cc +++ b/cpp/velox/udf/examples/MyUDAF.cc @@ -173,6 +173,7 @@ class MyAvgRegisterer final : public gluten::UdafRegisterer { const char* myAvgIntermediateType_ = "struct"; }; + } // namespace myavg std::vector>& globalRegisters() { diff --git a/cpp/velox/udf/examples/MyUDF.cc b/cpp/velox/udf/examples/MyUDF.cc index 783699614fbd..260629fdf946 100644 --- a/cpp/velox/udf/examples/MyUDF.cc +++ b/cpp/velox/udf/examples/MyUDF.cc @@ -33,6 +33,7 @@ static const char* kDate = "date"; static const char* kVarChar = "varchar"; namespace hivestringstring { + template struct HiveStringStringFunction { VELOX_DEFINE_FUNCTION_TYPES(T); @@ -67,6 +68,7 @@ class HiveStringStringRegisterer final : public gluten::UdfRegisterer { const std::string name_ = "org.apache.spark.sql.hive.execution.UDFStringString"; const char* arg_[2] = {kVarChar, kVarChar}; }; + } // namespace hivestringstring std::vector>& globalRegisters() { diff --git a/cpp/velox/utils/HdfsUtils.cc b/cpp/velox/utils/HdfsUtils.cc index ec395f817cdf..f03ca9c17dbb 100644 --- a/cpp/velox/utils/HdfsUtils.cc +++ b/cpp/velox/utils/HdfsUtils.cc @@ -23,6 +23,7 @@ namespace gluten { namespace { + struct Credential { const std::string userName; const std::string allTokens; @@ -34,6 +35,7 @@ struct Credential { return !(rhs == *this); } }; + } // namespace void updateHdfsTokens(const facebook::velox::config::ConfigBase* veloxCfg) { @@ -63,4 +65,5 @@ void updateHdfsTokens(const facebook::velox::config::ConfigBase* veloxCfg) { hdfsSetTokenForDefaultUser(token.data()); activeCredential.emplace(newCredential); } + } // namespace gluten diff --git a/cpp/velox/utils/VeloxBatchResizer.cc b/cpp/velox/utils/VeloxBatchResizer.cc index 56429299464a..851af9c83cd9 100644 --- a/cpp/velox/utils/VeloxBatchResizer.cc +++ b/cpp/velox/utils/VeloxBatchResizer.cc @@ -18,8 +18,8 @@ #include "VeloxBatchResizer.h" namespace gluten { - namespace { + class SliceRowVector : public ColumnarBatchIterator { public: SliceRowVector(int32_t maxOutputBatchSize, facebook::velox::RowVectorPtr in) @@ -115,4 +115,5 @@ std::shared_ptr VeloxBatchResizer::next() { int64_t VeloxBatchResizer::spillFixedSize(int64_t size) { return in_->spillFixedSize(size); } + } // namespace gluten diff --git a/cpp/velox/utils/VeloxBatchResizer.h b/cpp/velox/utils/VeloxBatchResizer.h index 522baf4982e4..d3d1539dd7f8 100644 --- a/cpp/velox/utils/VeloxBatchResizer.h +++ b/cpp/velox/utils/VeloxBatchResizer.h @@ -22,6 +22,7 @@ #include "velox/vector/ComplexVector.h" namespace gluten { + class VeloxBatchResizer : public ColumnarBatchIterator { public: VeloxBatchResizer( @@ -42,4 +43,5 @@ class VeloxBatchResizer : public ColumnarBatchIterator { std::unique_ptr next_ = nullptr; }; + } // namespace gluten diff --git a/cpp/velox/utils/tests/MemoryPoolUtils.cc b/cpp/velox/utils/tests/MemoryPoolUtils.cc index 5a0ae03b1496..2d4e19511a6e 100644 --- a/cpp/velox/utils/tests/MemoryPoolUtils.cc +++ b/cpp/velox/utils/tests/MemoryPoolUtils.cc @@ -152,4 +152,5 @@ arrow::Status SelfEvictedMemoryPool::ensureCapacity(int64_t size) { } return arrow::Status::OK(); } -} // namespace gluten \ No newline at end of file + +} // namespace gluten diff --git a/cpp/velox/utils/tests/MemoryPoolUtils.h b/cpp/velox/utils/tests/MemoryPoolUtils.h index 68e279008497..5fdf880be678 100644 --- a/cpp/velox/utils/tests/MemoryPoolUtils.h +++ b/cpp/velox/utils/tests/MemoryPoolUtils.h @@ -24,9 +24,7 @@ namespace gluten { -/** - * arrow::MemoryPool instance with limited capacity, used by tests and benchmarks - */ +// arrow::MemoryPool instance with limited capacity, used by tests and benchmarks. class LimitedMemoryPool final : public arrow::MemoryPool { public: explicit LimitedMemoryPool() : capacity_(std::numeric_limits::max()) {} @@ -54,9 +52,7 @@ class LimitedMemoryPool final : public arrow::MemoryPool { arrow::internal::MemoryPoolStats stats_; }; -/** - * arrow::MemoryPool instance with limited capacity and can be evictable on OOM, used by tests and benchmarks - */ +// arrow::MemoryPool instance with limited capacity and can be evictable on OOM, used by tests and benchmarks. class SelfEvictedMemoryPool : public arrow::MemoryPool { public: explicit SelfEvictedMemoryPool(arrow::MemoryPool* pool, bool failIfOOM = true) : pool_(pool), failIfOOM_(failIfOOM) {} From f96105de853ad5855b59953f4932c38b2860b05c Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Fri, 6 Dec 2024 09:14:35 +0800 Subject: [PATCH 201/211] [CORE] Simplify code of offload scan (#8144) --- .../backendsapi/clickhouse/CHBackend.scala | 2 + .../clickhouse/CHSparkPlanExecApi.scala | 13 ++--- .../backendsapi/BackendSettingsApi.scala | 2 + .../gluten/backendsapi/SparkPlanExecApi.scala | 5 +- .../execution/BatchScanExecTransformer.scala | 4 ++ .../CartesianProductExecTransformer.scala | 8 +++ .../execution/ScanTransformerFactory.scala | 52 ++++--------------- .../offload/OffloadSingleNodeRules.scala | 50 ++++-------------- .../columnar/validator/Validators.scala | 31 +++-------- .../hive/HiveTableScanExecTransformer.scala | 23 +------- 10 files changed, 47 insertions(+), 143 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index e5eb91b69bdb..823ed747005c 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -364,6 +364,8 @@ object CHBackendSettings extends BackendSettingsApi with Logging { override def supportCartesianProductExec(): Boolean = true + override def supportCartesianProductExecWithCondition(): Boolean = false + override def supportHashBuildJoinTypeOnLeft: JoinType => Boolean = { t => if (super.supportHashBuildJoinTypeOnLeft(t)) { diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index dde03d4ad087..c2f91fa15214 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -363,15 +363,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { left: SparkPlan, right: SparkPlan, condition: Option[Expression]): CartesianProductExecTransformer = - if (!condition.isEmpty) { - throw new GlutenNotSupportException( - "CartesianProductExecTransformer with condition is not supported in ch backend.") - } else { - CartesianProductExecTransformer( - ColumnarCartesianProductBridge(left), - ColumnarCartesianProductBridge(right), - condition) - } + CartesianProductExecTransformer( + ColumnarCartesianProductBridge(left), + ColumnarCartesianProductBridge(right), + condition) override def genBroadcastNestedLoopJoinExecTransformer( left: SparkPlan, diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 1eb69da6e5f0..0d5b6b36da85 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -121,6 +121,8 @@ trait BackendSettingsApi { def supportCartesianProductExec(): Boolean = false + def supportCartesianProductExecWithCondition(): Boolean = true + def supportBroadcastNestedLoopJoinExec(): Boolean = true def supportSampleExec(): Boolean = false diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala index 667c0bdc25a9..ec032af92d96 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution.joins.BuildSideRelation import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.ArrowEvalPythonExec import org.apache.spark.sql.execution.window._ -import org.apache.spark.sql.hive.{HiveTableScanExecTransformer, HiveUDFTransformer} +import org.apache.spark.sql.hive.HiveUDFTransformer import org.apache.spark.sql.types.{DecimalType, LongType, NullType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -64,9 +64,6 @@ trait SparkPlanExecApi { */ def genFilterExecTransformer(condition: Expression, child: SparkPlan): FilterExecTransformerBase - def genHiveTableScanExecTransformer(plan: SparkPlan): HiveTableScanExecTransformer = - HiveTableScanExecTransformer(plan) - def genProjectExecTransformer( projectList: Seq[NamedExpression], child: SparkPlan): ProjectExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala index e1a1be8e29b5..4f603e102443 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala @@ -134,6 +134,10 @@ abstract class BatchScanExecTransformerBase( } override def doValidateInternal(): ValidationResult = { + if (!ScanTransformerFactory.supportedBatchScan(scan)) { + return ValidationResult.failed(s"Unsupported scan $scan") + } + if (pushedAggregate.nonEmpty) { return ValidationResult.failed(s"Unsupported aggregation push down for $scan.") } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala index 3e3169aa554e..9e2f12bcf8ed 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala @@ -111,6 +111,14 @@ case class CartesianProductExecTransformer( } override protected def doValidateInternal(): ValidationResult = { + if ( + !BackendsApiManager.getSettings.supportCartesianProductExecWithCondition() && + condition.nonEmpty + ) { + return ValidationResult.failed( + "CartesianProductExecTransformer with condition is not supported in this backend.") + } + if (!BackendsApiManager.getSettings.supportCartesianProductExec()) { return ValidationResult.failed("Cartesian product is not supported in this backend") } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala index 52dad6da3773..dfdf2d2f34e1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala @@ -16,12 +16,10 @@ */ package org.apache.gluten.execution -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.connector.read.Scan -import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} import java.util.ServiceLoader @@ -58,8 +56,7 @@ object ScanTransformerFactory { } } - private def lookupBatchScanTransformer( - batchScanExec: BatchScanExec): BatchScanExecTransformerBase = { + def createBatchScanTransformer(batchScanExec: BatchScanExec): BatchScanExecTransformerBase = { val scan = batchScanExec.scan lookupDataSourceScanTransformer(scan.getClass.getName) match { case Some(clz) => @@ -69,46 +66,16 @@ object ScanTransformerFactory { .asInstanceOf[DataSourceScanTransformerRegister] .createDataSourceV2Transformer(batchScanExec) case _ => - scan match { - case _: FileScan => - BatchScanExecTransformer( - batchScanExec.output, - batchScanExec.scan, - batchScanExec.runtimeFilters, - table = SparkShimLoader.getSparkShims.getBatchScanExecTable(batchScanExec) - ) - case _ => - throw new GlutenNotSupportException(s"Unsupported scan $scan") - } - } - } - - def createBatchScanTransformer( - batchScan: BatchScanExec, - validation: Boolean = false): SparkPlan = { - if (supportedBatchScan(batchScan.scan)) { - val transformer = lookupBatchScanTransformer(batchScan) - if (!validation) { - val validationResult = transformer.doValidate() - if (validationResult.ok()) { - transformer - } else { - FallbackTags.add(batchScan, validationResult.reason()) - batchScan - } - } else { - transformer - } - } else { - if (validation) { - throw new GlutenNotSupportException(s"Unsupported scan ${batchScan.scan}") - } - FallbackTags.add(batchScan, "The scan in BatchScanExec is not supported.") - batchScan + BatchScanExecTransformer( + batchScanExec.output, + batchScanExec.scan, + batchScanExec.runtimeFilters, + table = SparkShimLoader.getSparkShims.getBatchScanExecTable(batchScanExec) + ) } } - private def supportedBatchScan(scan: Scan): Boolean = scan match { + def supportedBatchScan(scan: Scan): Boolean = scan match { case _: FileScan => true case _ => lookupDataSourceScanTransformer(scan.getClass.getName).nonEmpty } @@ -132,5 +99,4 @@ object ScanTransformerFactory { ) Option(clz) } - } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala index fa698cd244cc..7dc40faa4315 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala @@ -18,7 +18,6 @@ package org.apache.gluten.extension.columnar.offload import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.execution._ import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.logging.LogLevelUtil @@ -178,7 +177,7 @@ object OffloadJoin { // Other transformations. case class OffloadOthers() extends OffloadSingleNode with LogLevelUtil { import OffloadOthers._ - private val replace = new ReplaceSingleNode() + private val replace = new ReplaceSingleNode override def offload(plan: SparkPlan): SparkPlan = replace.doReplace(plan) } @@ -190,7 +189,7 @@ object OffloadOthers { // Do not look up on children on the input node in this rule. Otherwise // it may break RAS which would group all the possible input nodes to // search for validate candidates. - private class ReplaceSingleNode() extends LogLevelUtil with Logging { + private class ReplaceSingleNode extends LogLevelUtil with Logging { def doReplace(p: SparkPlan): SparkPlan = { val plan = p @@ -199,11 +198,15 @@ object OffloadOthers { } plan match { case plan: BatchScanExec => - applyScanTransformer(plan) + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") + ScanTransformerFactory.createBatchScanTransformer(plan) case plan: FileSourceScanExec => - applyScanTransformer(plan) + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") + ScanTransformerFactory.createFileSourceScanTransformer(plan) case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - applyScanTransformer(plan) + // TODO: Add DynamicPartitionPruningHiveScanSuite.scala + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") + HiveTableScanExecTransformer(plan) case plan: CoalesceExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") ColumnarCoalesceExec(plan.numPartitions, plan.child) @@ -333,40 +336,5 @@ object OffloadOthers { case other => other } } - - /** - * Apply scan transformer for file source and batch source, - * 1. create new filter and scan transformer, 2. validate, tag new scan as unsupported if - * failed, 3. return new source. - */ - private def applyScanTransformer(plan: SparkPlan): SparkPlan = plan match { - case plan: FileSourceScanExec => - val transformer = ScanTransformerFactory.createFileSourceScanTransformer(plan) - val validationResult = transformer.doValidate() - if (validationResult.ok()) { - logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - transformer - } else { - logDebug(s"Columnar Processing for ${plan.getClass} is currently unsupported.") - FallbackTags.add(plan, validationResult.reason()) - plan - } - case plan: BatchScanExec => - ScanTransformerFactory.createBatchScanTransformer(plan) - case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - // TODO: Add DynamicPartitionPruningHiveScanSuite.scala - val hiveTableScanExecTransformer = - BackendsApiManager.getSparkPlanExecApiInstance.genHiveTableScanExecTransformer(plan) - val validateResult = hiveTableScanExecTransformer.doValidate() - if (validateResult.ok()) { - logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - return hiveTableScanExecTransformer - } - logDebug(s"Columnar Processing for ${plan.getClass} is currently unsupported.") - FallbackTags.add(plan, validateResult.reason()) - plan - case other => - throw new GlutenNotSupportException(s"${other.getClass.toString} is not supported.") - } } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala index 7e7d732c29a2..d246167bd7c8 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala @@ -18,7 +18,6 @@ package org.apache.gluten.extension.columnar.validator import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.{BackendsApiManager, BackendSettingsApi} -import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.execution._ import org.apache.gluten.expression.ExpressionUtils import org.apache.gluten.extension.columnar.FallbackTags @@ -95,7 +94,7 @@ object Validators { * native validation failed. */ def fallbackByNativeValidation(): Validator.Builder = { - builder.add(new FallbackByNativeValidation()) + builder.add(new FallbackByNativeValidation) } } @@ -223,34 +222,16 @@ object Validators { } } - private class FallbackByNativeValidation() extends Validator with Logging { - override def validate(plan: SparkPlan): Validator.OutCome = { - try { - validate0(plan) - } catch { - case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => - if (!e.isInstanceOf[GlutenNotSupportException]) { - logDebug("Just a warning. This exception perhaps needs to be fixed.", e) - } - fail( - s"${e.getMessage}, original Spark plan is " + - s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") - } - } - - private def validate0(plan: SparkPlan): Validator.OutCome = plan match { + private class FallbackByNativeValidation extends Validator with Logging { + override def validate(plan: SparkPlan): Validator.OutCome = plan match { case plan: BatchScanExec => - val transformer = - ScanTransformerFactory - .createBatchScanTransformer(plan, validation = true) - .asInstanceOf[BasicScanExecTransformer] + val transformer = ScanTransformerFactory.createBatchScanTransformer(plan) transformer.doValidate().toValidatorOutcome() case plan: FileSourceScanExec => - val transformer = - ScanTransformerFactory.createFileSourceScanTransformer(plan) + val transformer = ScanTransformerFactory.createFileSourceScanTransformer(plan) transformer.doValidate().toValidatorOutcome() case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - HiveTableScanExecTransformer.validate(plan).toValidatorOutcome() + HiveTableScanExecTransformer(plan).doValidate().toValidatorOutcome() case plan: ProjectExec => val transformer = ProjectExecTransformer(plan.projectList, plan.child) transformer.doValidate().toValidatorOutcome() diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala index f701c76b1813..6911672376a9 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.ValidationResult import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat @@ -181,8 +180,8 @@ case class HiveTableScanExecTransformer( object HiveTableScanExecTransformer { - val NULL_VALUE: Char = 0x00 - val DEFAULT_FIELD_DELIMITER: Char = 0x01 + private val NULL_VALUE: Char = 0x00 + private val DEFAULT_FIELD_DELIMITER: Char = 0x01 val TEXT_INPUT_FORMAT_CLASS: Class[TextInputFormat] = Utils.classForName("org.apache.hadoop.mapred.TextInputFormat") val ORC_INPUT_FORMAT_CLASS: Class[OrcInputFormat] = @@ -193,24 +192,6 @@ object HiveTableScanExecTransformer { plan.isInstanceOf[HiveTableScanExec] } - def copyWith(plan: SparkPlan, newPartitionFilters: Seq[Expression]): SparkPlan = { - val hiveTableScanExec = plan.asInstanceOf[HiveTableScanExec] - hiveTableScanExec.copy(partitionPruningPred = newPartitionFilters)(sparkSession = - hiveTableScanExec.session) - } - - def validate(plan: SparkPlan): ValidationResult = { - plan match { - case hiveTableScan: HiveTableScanExec => - val hiveTableScanTransformer = new HiveTableScanExecTransformer( - hiveTableScan.requestedAttributes, - hiveTableScan.relation, - hiveTableScan.partitionPruningPred)(hiveTableScan.session) - hiveTableScanTransformer.doValidate() - case _ => ValidationResult.failed("Is not a Hive scan") - } - } - def apply(plan: SparkPlan): HiveTableScanExecTransformer = { plan match { case hiveTableScan: HiveTableScanExec => From e34914d4a152abdf6fde45b0855badf4c2170865 Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Fri, 6 Dec 2024 09:49:10 +0800 Subject: [PATCH 202/211] remove decimal divide (#8160) --- .../Functions/SparkFunctionDecimalDivide.cpp | 28 --- .../Functions/SparkFunctionDecimalDivide.h | 176 ------------------ 2 files changed, 204 deletions(-) delete mode 100644 cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.cpp delete mode 100644 cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.h diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.cpp b/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.cpp deleted file mode 100644 index 0de3b757e5fd..000000000000 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.cpp +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include - -namespace local_engine -{ - -REGISTER_FUNCTION(SparkFunctionDecimalDivide) -{ - factory.registerFunction>(); -} - -} diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.h b/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.h deleted file mode 100644 index 1b93e77d9400..000000000000 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.h +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include -#include -#include -#include -#include -#include - -using namespace DB; - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} -} - -namespace local_engine -{ -struct DivideDecimalsImpl -{ - static constexpr auto name = "sparkDivideDecimal"; - template - static inline Decimal256 - execute(FirstType a, SecondType b, UInt16 scale_a, UInt16 scale_b, UInt16 result_scale) - { - if (a.value == 0 || b.value == 0) - return Decimal256(0); - - Int256 sign_a = a.value < 0 ? -1 : 1; - Int256 sign_b = b.value < 0 ? -1 : 1; - - std::vector a_digits = DecimalOpHelpers::toDigits(a.value * sign_a); - - while (scale_a < scale_b + result_scale) - { - a_digits.push_back(0); - ++scale_a; - } - - while (scale_a > scale_b + result_scale && !a_digits.empty()) - { - a_digits.pop_back(); - --scale_a; - } - - if (a_digits.empty()) - return Decimal256(0); - - std::vector divided = DecimalOpHelpers::divide(a_digits, b.value * sign_b); - - if (divided.size() > DecimalUtils::max_precision) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow: result bigger that Decimal256"); - return Decimal256(sign_a * sign_b * DecimalOpHelpers::fromDigits(divided)); - } -}; - -template -class SparkFunctionDecimalDivide : public FunctionsDecimalArithmetics -{ -public: - static constexpr auto name = Transform::name; - static DB::FunctionPtr create(DB::ContextPtr) { return std::make_shared(); } - SparkFunctionDecimalDivide() = default; - ~SparkFunctionDecimalDivide() override = default; - String getName() const override { return name; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - return makeNullable(FunctionsDecimalArithmetics::getReturnTypeImpl(arguments)); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows) const override - { - if (arguments.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} must have 2 arugments.", name); - - ColumnPtr res_col = nullptr; - MutableColumnPtr null_map_col = ColumnUInt8::create(input_rows, 0); - auto getNonNullableColumn = [&](const ColumnPtr & col) -> const ColumnPtr - { - if (col->isNullable()) - { - auto * nullable_col = checkAndGetColumn(col.get()); - return nullable_col->getNestedColumnPtr(); - } - else - return col; - }; - - ColumnWithTypeAndName new_arg0 {getNonNullableColumn(arguments[0].column), removeNullable(arguments[0].type), arguments[0].name}; - ColumnWithTypeAndName new_arg1 {getNonNullableColumn(arguments[1].column), removeNullable(arguments[1].type), arguments[0].name}; - ColumnsWithTypeAndName new_args {new_arg0, new_arg1}; - bool arg_type_valid = true; - - if (isDecimal(new_arg1.type)) - { - using Types = TypeList; - arg_type_valid = castTypeToEither(Types{}, new_arg1.type.get(), [&](const auto & right_) - { - using R = typename std::decay_t::FieldType; - const ColumnDecimal * const_col_right = checkAndGetColumnConstData>(new_arg1.column.get()); - if (const_col_right && const_col_right->getElement(0).value == 0) - { - null_map_col = ColumnUInt8::create(input_rows, 1); - res_col = ColumnDecimal::create(input_rows, 0); - } - else - res_col = FunctionsDecimalArithmetics::executeImpl(new_args, removeNullable(result_type), input_rows); - - if (!const_col_right) - { - const ColumnDecimal * col_right = assert_cast *>(new_arg1.column.get()); - PaddedPODArray & null_map = assert_cast*>(null_map_col.get())->getData(); - for (size_t i = 0; i < col_right->size(); ++i) - null_map[i] = (col_right->getElement(i).value == 0 || arguments[1].column->isNullAt(i)); - } - return true; - }); - } - else if (isNumber(new_arg1.type)) - { - using Types = TypeList; - arg_type_valid = castTypeToEither(Types{}, new_arg1.type.get(), [&](const auto & right_) - { - using R = typename std::decay_t::FieldType; - const ColumnVector * const_col_right = checkAndGetColumnConstData>(new_arg1.column.get()); - if (const_col_right && const_col_right->getElement(0) == 0) - { - null_map_col = ColumnUInt8::create(input_rows, 1); - res_col = ColumnDecimal::create(input_rows, 0); - } - else - res_col = FunctionsDecimalArithmetics::executeImpl(new_args, removeNullable(result_type), input_rows); - - if (!const_col_right) - { - const ColumnVector * col_right = assert_cast *>(new_arg1.column.get()); - PaddedPODArray & null_map = assert_cast*>(null_map_col.get())->getData(); - for (size_t i = 0; i < col_right->size(); ++i) - null_map[i] = (col_right->getElement(i) == 0 || arguments[1].column->isNullAt(i)); - } - return true; - }); - } - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s arguments type must be numbeic", name); - - if (!arg_type_valid) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s arguments type is not valid.", name); - - return ColumnNullable::create(res_col, std::move(null_map_col)); - } - -}; -} From 8c652f15253ac380a7f5af9d6e07c709aad7ed06 Mon Sep 17 00:00:00 2001 From: Jin Chengcheng Date: Fri, 6 Dec 2024 15:08:09 +0800 Subject: [PATCH 203/211] [GLUTEN-7900][VL] Enable prefix sort config in spill (#7904) --- cpp/velox/compute/WholeStageResultIterator.cc | 2 ++ cpp/velox/config/VeloxConfig.h | 1 + 2 files changed, 3 insertions(+) diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index e5463b96fd25..d2f825dcfa8e 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -518,6 +518,8 @@ std::unordered_map WholeStageResultIterator::getQueryC std::to_string(veloxCfg_->get(kSpillableReservationGrowthPct, 25)); configs[velox::core::QueryConfig::kSpillCompressionKind] = veloxCfg_->get(kSpillCompressionKind, "lz4"); + configs[velox::core::QueryConfig::kSpillPrefixSortEnabled] = + veloxCfg_->get(kSpillPrefixSortEnabled, "false"); configs[velox::core::QueryConfig::kSparkBloomFilterExpectedNumItems] = std::to_string(veloxCfg_->get(kBloomFilterExpectedNumItems, 1000000)); configs[velox::core::QueryConfig::kSparkBloomFilterNumBits] = diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index dc234152a362..4ae82f263d37 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -44,6 +44,7 @@ const uint64_t kMaxSpillFileSizeDefault = 1L * 1024 * 1024 * 1024; const std::string kSpillableReservationGrowthPct = "spark.gluten.sql.columnar.backend.velox.spillableReservationGrowthPct"; const std::string kSpillCompressionKind = "spark.io.compression.codec"; +const std::string kSpillPrefixSortEnabled = "spark.gluten.sql.columnar.backend.velox.spillPrefixsortEnabled"; const std::string kMaxPartialAggregationMemoryRatio = "spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio"; const std::string kMaxExtendedPartialAggregationMemoryRatio = From e3322a6563c7bbaaeaa173b03fca7d151be37ac0 Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Fri, 6 Dec 2024 15:20:19 +0800 Subject: [PATCH 204/211] [CORE] Bump celeborn to 0.5.2 (#8054) --- .github/workflows/velox_backend.yml | 4 ++-- pom.xml | 2 +- tools/gluten-it/pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index c3765d2f4f6d..ce195c4e1b52 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -549,7 +549,7 @@ jobs: fail-fast: false matrix: spark: [ "spark-3.2" ] - celeborn: [ "celeborn-0.5.1", "celeborn-0.4.2", "celeborn-0.3.2-incubating" ] + celeborn: [ "celeborn-0.5.2", "celeborn-0.4.2", "celeborn-0.3.2-incubating" ] runs-on: ubuntu-20.04 container: ubuntu:22.04 steps: @@ -582,7 +582,7 @@ jobs: EXTRA_PROFILE="" if [ "${{ matrix.celeborn }}" = "celeborn-0.4.2" ]; then EXTRA_PROFILE="-Pceleborn-0.4" - elif [ "${{ matrix.celeborn }}" = "celeborn-0.5.1" ]; then + elif [ "${{ matrix.celeborn }}" = "celeborn-0.5.2" ]; then EXTRA_PROFILE="-Pceleborn-0.5" fi echo "EXTRA_PROFILE: ${EXTRA_PROFILE}" diff --git a/pom.xml b/pom.xml index 8a2968b56905..a2cccda1e45e 100644 --- a/pom.xml +++ b/pom.xml @@ -65,7 +65,7 @@ delta-core 2.4.0 24 - 0.5.1 + 0.5.2 0.9.0 15.0.0 15.0.0-gluten diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index bad4d6087f11..1973a7d99890 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -177,7 +177,7 @@ celeborn-0.5 - 0.5.1 + 0.5.2 From 232ce55443a7ea1daa1285a8a402e50f71334418 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 6 Dec 2024 16:29:58 +0800 Subject: [PATCH 205/211] [GLUTEN-6920][CORE][VL] New APIs and refactors to allow different backends / components to be registered and used (#8143) This is the major API layer change for #6920. The PR introduces a new concept Component int Gluten and adjusts the role of current concept Backend. After the change, Backend API will become a special variant of Component API. Details are as below: Dependency Component should have at least one parent. Each parent can either be another Component or Backend. Backend doesn't have any parents. Component and Backend instances are used as exactly the same manner in Gluten. Gluten treats them all as regular entities that inject custom code into Gluten. Having Component's parents defined, Gluten will figure out a fixed order / priority of using the components. For exmaple: A component's parent's loading APIs, namely onDriverStart / onExecutorStart, will be guaranteed to call EARLIER than the same APIs of the child. A component's parent's rule injection API, namely injectRules, will be guaranteed to call LATER than the same API of the child. For legacy reason, only one SubstraitBackend will be allowed to register at the same time. Which means, CH backend and Velox backend are still not allowed to load at the same time. When they are both loaded, error will be thrown. See code. In subsequent PRs of #6920, we will: Make iceberg / hudi / delta implement component API, remove the previous rule injection points used by their code. Make uniffle / celeborn implement component API, remove the previous rule injection points by used their code. Add an example backend that can work together with Velox backend. Non-goals: This doesn't tend to make Velox backend and CH backend work together. --- .../CHNativeExpressionEvaluator.java | 9 +- .../backendsapi/clickhouse/CHBackend.scala | 6 +- .../columnarbatch/VeloxColumnarBatches.java | 15 +- .../metrics/IteratorMetricsJniWrapper.java | 4 +- .../gluten/utils/VeloxBatchResizer.java | 9 +- .../apache/gluten/utils/VeloxBloomFilter.java | 4 +- .../backendsapi/velox/VeloxBackend.scala | 7 +- .../backendsapi/velox/VeloxIteratorApi.scala | 8 +- .../backendsapi/velox/VeloxListenerApi.scala | 2 +- .../velox/VeloxTransformerApi.scala | 6 +- .../backendsapi/velox/VeloxValidatorApi.scala | 4 +- .../datasource/VeloxDataSourceUtil.scala | 3 +- .../ColumnarPartialProjectExec.scala | 3 +- .../execution/RowToVeloxColumnarExec.scala | 3 +- .../execution/VeloxColumnarToRowExec.scala | 5 +- .../vectorized/ColumnarBatchSerializer.scala | 6 +- .../spark/shuffle/ColumnarShuffleWriter.scala | 5 +- .../spark/sql/execution/BroadcastUtils.scala | 8 +- .../execution/ColumnarBuildSideRelation.scala | 9 +- .../ColumnarCachedBatchSerializer.scala | 17 +- .../velox/VeloxFormatWriterInjects.scala | 10 +- .../spark/sql/execution/utils/ExecUtil.scala | 6 +- .../columnarbatch/ColumnarBatchTest.java | 8 +- cpp/core/compute/Runtime.h | 56 +++- cpp/core/jni/JniWrapper.cc | 131 ++++++--- cpp/core/utils/Registry.h | 6 +- cpp/velox/jni/VeloxJniWrapper.cc | 8 +- .../ColumnarBatchJniWrapper.java | 20 +- .../gluten/columnarbatch/ColumnarBatches.java | 18 +- .../gluten/columnarbatch/IndicatorVector.java | 2 +- .../columnarbatch/IndicatorVectorBase.java | 11 +- .../gluten/init/NativeBackendInitializer.java | 28 +- .../vectorized/ColumnarBatchInIterator.java | 6 +- .../vectorized/NativePlanEvaluator.java | 14 +- .../gluten/memory/NativeMemoryManager.scala | 22 +- .../org/apache/gluten/runtime/Runtime.scala | 16 +- .../org/apache/gluten/runtime/Runtimes.scala | 9 +- ...CelebornColumnarShuffleWriterFactory.scala | 4 +- ...VeloxCelebornColumnarBatchSerializer.scala | 4 +- .../VeloxCelebornColumnarShuffleWriter.scala | 8 +- .../org/apache/gluten/GlutenPlugin.scala | 26 +- .../org/apache/gluten/backend/Backend.scala | 57 +--- .../org/apache/gluten/backend/Component.scala | 251 ++++++++++++++++++ .../org/apache/gluten/backend/package.scala | 47 ++++ .../extension/GlutenSessionExtensions.scala | 5 +- .../enumerated/EnumeratedTransform.scala | 5 +- .../heuristic/HeuristicTransform.scala | 5 +- .../columnar/transition/ConventionFunc.scala | 21 +- .../spark/shuffle/GlutenShuffleManager.scala | 4 + .../shuffle/ShuffleManagerRegistry.scala | 5 +- .../org/apache/spark/util/SparkTestUtil.scala | 23 ++ .../gluten/backend/ComponentSuite.scala | 100 +++++++ .../backendsapi/BackendsApiManager.scala | 6 +- .../VeloxUniffleColumnarShuffleWriter.java | 6 +- 54 files changed, 822 insertions(+), 259 deletions(-) create mode 100644 gluten-core/src/main/scala/org/apache/gluten/backend/Component.scala create mode 100644 gluten-core/src/main/scala/org/apache/gluten/backend/package.scala create mode 100644 gluten-core/src/main/scala/org/apache/spark/util/SparkTestUtil.scala create mode 100644 gluten-core/src/test/scala/org/apache/gluten/backend/ComponentSuite.scala diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java index 370f434a3c88..2099ddbbf545 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java @@ -17,7 +17,6 @@ package org.apache.gluten.vectorized; import org.apache.gluten.GlutenConfig; -import org.apache.gluten.backend.Backend; import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.execution.ColumnarNativeIterator; import org.apache.gluten.memory.CHThreadGroup; @@ -35,11 +34,12 @@ private CHNativeExpressionEvaluator() {} // Used to initialize the native computing. public static void initNative(scala.collection.Map conf) { Map nativeConfMap = - GlutenConfig.getNativeBackendConf(Backend.get().name(), conf); + GlutenConfig.getNativeBackendConf(BackendsApiManager.getBackendName(), conf); // Get the customer config from SparkConf for each backend BackendsApiManager.getTransformerApiInstance() - .postProcessNativeConfig(nativeConfMap, GlutenConfig.prefixOf(Backend.get().name())); + .postProcessNativeConfig( + nativeConfMap, GlutenConfig.prefixOf(BackendsApiManager.getBackendName())); nativeInitNative(ConfigUtil.serialize(nativeConfMap)); } @@ -54,7 +54,8 @@ public static boolean doValidate(byte[] subPlan) { } private static Map getNativeBackendConf() { - return GlutenConfig.getNativeBackendConf(Backend.get().name(), SQLConf.get().getAllConfs()); + return GlutenConfig.getNativeBackendConf( + BackendsApiManager.getBackendName(), SQLConf.get().getAllConfs()); } // Used by WholeStageTransform to create the native computing pipeline and diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 823ed747005c..83a92db51897 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -18,7 +18,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenBuildInfo._ import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component.BuildInfo import org.apache.gluten.backendsapi._ import org.apache.gluten.columnarbatch.CHBatch import org.apache.gluten.execution.WriteFilesExecTransformer @@ -49,8 +49,8 @@ import scala.util.control.Breaks.{break, breakable} class CHBackend extends SubstraitBackend { import CHBackend._ override def name(): String = CHConf.BACKEND_NAME - override def buildInfo(): Backend.BuildInfo = - Backend.BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN") + override def buildInfo(): BuildInfo = + BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN") override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def iteratorApi(): IteratorApi = new CHIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new CHSparkPlanExecApi diff --git a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java index e2035455fd74..db2d08e31435 100644 --- a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java +++ b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.columnarbatch; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtime; import org.apache.gluten.runtime.Runtimes; @@ -59,8 +60,10 @@ public static ColumnarBatch toVeloxBatch(ColumnarBatch input) { return input; } Preconditions.checkArgument(!isVeloxBatch(input)); - final Runtime runtime = Runtimes.contextInstance("VeloxColumnarBatches#toVeloxBatch"); - final long handle = ColumnarBatches.getNativeHandle(input); + final Runtime runtime = + Runtimes.contextInstance( + BackendsApiManager.getBackendName(), "VeloxColumnarBatches#toVeloxBatch"); + final long handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), input); final long outHandle = VeloxColumnarBatchJniWrapper.create(runtime).from(handle); final ColumnarBatch output = ColumnarBatches.create(outHandle); @@ -88,9 +91,13 @@ public static ColumnarBatch toVeloxBatch(ColumnarBatch input) { * Otherwise {@link UnsupportedOperationException} will be thrown. */ public static ColumnarBatch compose(ColumnarBatch... batches) { - final Runtime runtime = Runtimes.contextInstance("VeloxColumnarBatches#compose"); + final Runtime runtime = + Runtimes.contextInstance( + BackendsApiManager.getBackendName(), "VeloxColumnarBatches#compose"); final long[] handles = - Arrays.stream(batches).mapToLong(ColumnarBatches::getNativeHandle).toArray(); + Arrays.stream(batches) + .mapToLong(b -> ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), b)) + .toArray(); final long handle = VeloxColumnarBatchJniWrapper.create(runtime).compose(handles); return ColumnarBatches.create(handle); } diff --git a/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java b/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java index 7250fde0fe40..d03f8816adfc 100644 --- a/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java +++ b/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.metrics; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtime; import org.apache.gluten.runtime.RuntimeAware; import org.apache.gluten.runtime.Runtimes; @@ -29,7 +30,8 @@ private IteratorMetricsJniWrapper(Runtime runtime) { } public static IteratorMetricsJniWrapper create() { - final Runtime runtime = Runtimes.contextInstance("IteratorMetrics"); + final Runtime runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName(), "IteratorMetrics"); return new IteratorMetricsJniWrapper(runtime); } diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java index 2a6dcb43a052..73bfec08cf48 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.utils; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtime; import org.apache.gluten.runtime.Runtimes; import org.apache.gluten.vectorized.ColumnarBatchInIterator; @@ -28,10 +29,14 @@ public final class VeloxBatchResizer { public static ColumnarBatchOutIterator create( int minOutputBatchSize, int maxOutputBatchSize, Iterator in) { - final Runtime runtime = Runtimes.contextInstance("VeloxBatchResizer"); + final Runtime runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName(), "VeloxBatchResizer"); long outHandle = VeloxBatchResizerJniWrapper.create(runtime) - .create(minOutputBatchSize, maxOutputBatchSize, new ColumnarBatchInIterator(in)); + .create( + minOutputBatchSize, + maxOutputBatchSize, + new ColumnarBatchInIterator(BackendsApiManager.getBackendName(), in)); return new ColumnarBatchOutIterator(runtime, outHandle); } } diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java index 10179d63edb3..cdf2b195cd0c 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.utils; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtimes; import org.apache.commons.io.IOUtils; @@ -30,7 +31,8 @@ public class VeloxBloomFilter extends BloomFilter { private final VeloxBloomFilterJniWrapper jni = - VeloxBloomFilterJniWrapper.create(Runtimes.contextInstance("VeloxBloomFilter")); + VeloxBloomFilterJniWrapper.create( + Runtimes.contextInstance(BackendsApiManager.getBackendName(), "VeloxBloomFilter")); private final long handle; private VeloxBloomFilter(byte[] data) { diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 45ffd7765299..6bf4b6a4e256 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -18,7 +18,7 @@ package org.apache.gluten.backendsapi.velox import org.apache.gluten.GlutenBuildInfo._ import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component.BuildInfo import org.apache.gluten.backendsapi._ import org.apache.gluten.columnarbatch.VeloxBatch import org.apache.gluten.exception.GlutenNotSupportException @@ -52,9 +52,10 @@ import scala.util.control.Breaks.breakable class VeloxBackend extends SubstraitBackend { import VeloxBackend._ + override def name(): String = VeloxBackend.BACKEND_NAME - override def buildInfo(): Backend.BuildInfo = - Backend.BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME) + override def buildInfo(): BuildInfo = + BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME) override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def iteratorApi(): IteratorApi = new VeloxIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new VeloxSparkPlanExecApi diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala index 061daaac0fad..26bc108c15d9 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala @@ -183,9 +183,9 @@ class VeloxIteratorApi extends IteratorApi with Logging { val columnarNativeIterators = new JArrayList[ColumnarBatchInIterator](inputIterators.map { - iter => new ColumnarBatchInIterator(iter.asJava) + iter => new ColumnarBatchInIterator(BackendsApiManager.getBackendName, iter.asJava) }.asJava) - val transKernel = NativePlanEvaluator.create() + val transKernel = NativePlanEvaluator.create(BackendsApiManager.getBackendName) val splitInfoByteArray = inputPartition .asInstanceOf[GlutenPartition] @@ -235,10 +235,10 @@ class VeloxIteratorApi extends IteratorApi with Logging { ExecutorManager.tryTaskSet(numaBindingInfo) - val transKernel = NativePlanEvaluator.create() + val transKernel = NativePlanEvaluator.create(BackendsApiManager.getBackendName) val columnarNativeIterator = new JArrayList[ColumnarBatchInIterator](inputIterators.map { - iter => new ColumnarBatchInIterator(iter.asJava) + iter => new ColumnarBatchInIterator(BackendsApiManager.getBackendName, iter.asJava) }.asJava) val spillDirPath = SparkDirectoryUtil .get() diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index 3a82abe61833..175e34177a5d 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -164,7 +164,7 @@ class VeloxListenerApi extends ListenerApi with Logging { if (isDriver && !inLocalMode(conf)) { parsed += (GlutenConfig.COLUMNAR_VELOX_CACHE_ENABLED.key -> "false") } - NativeBackendInitializer.initializeBackend(parsed) + NativeBackendInitializer.forBackend(VeloxBackend.BACKEND_NAME).initialize(parsed) // Inject backend-specific implementations to override spark classes. GlutenFormatFactory.register(new VeloxParquetWriterInjects) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala index 1687f24ce35e..c6d2bc065879 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.backendsapi.velox -import org.apache.gluten.backendsapi.TransformerApi +import org.apache.gluten.backendsapi.{BackendsApiManager, TransformerApi} import org.apache.gluten.execution.WriteFilesExecTransformer import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.runtime.Runtimes @@ -87,7 +87,9 @@ class VeloxTransformerApi extends TransformerApi with Logging { override def getNativePlanString(substraitPlan: Array[Byte], details: Boolean): String = { TaskResources.runUnsafe { val jniWrapper = PlanEvaluatorJniWrapper.create( - Runtimes.contextInstance("VeloxTransformerApi#getNativePlanString")) + Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "VeloxTransformerApi#getNativePlanString")) jniWrapper.nativePlanString(substraitPlan, details) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala index ddf77e5fa3d4..9b3001366198 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.backendsapi.velox -import org.apache.gluten.backendsapi.ValidatorApi +import org.apache.gluten.backendsapi.{BackendsApiManager, ValidatorApi} import org.apache.gluten.extension.ValidationResult import org.apache.gluten.substrait.plan.PlanNode import org.apache.gluten.validate.NativePlanValidationInfo @@ -38,7 +38,7 @@ class VeloxValidatorApi extends ValidatorApi { override def doNativeValidateWithFailureReason(plan: PlanNode): ValidationResult = { TaskResources.runUnsafe { - val validator = NativePlanEvaluator.create() + val validator = NativePlanEvaluator.create(BackendsApiManager.getBackendName) asValidationResult(validator.doNativeValidateWithFailureReason(plan.toProtobuf.toByteArray)) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala b/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala index fe0d0eb0f8f2..7ea67feb1f8d 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.datasource +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.runtime.Runtimes import org.apache.gluten.utils.ArrowAbiUtil @@ -38,7 +39,7 @@ object VeloxDataSourceUtil { def readSchema(file: FileStatus): Option[StructType] = { val allocator = ArrowBufferAllocators.contextInstance() - val runtime = Runtimes.contextInstance("VeloxWriter") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxWriter") val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime) val dsHandle = datasourceJniWrapper.init(file.getPath.toString, -1, new util.HashMap[String, String]()) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala index 20744f531b5b..576f3a2cb205 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala @@ -188,7 +188,8 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( Iterator.empty } else { val start = System.currentTimeMillis() - val childData = ColumnarBatches.select(batch, projectIndexInChild.toArray) + val childData = ColumnarBatches + .select(BackendsApiManager.getBackendName, batch, projectIndexInChild.toArray) val projectedBatch = getProjectedBatchArrow(childData, c2a, a2c) val batchIterator = projectedBatch.map { diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala index 9921ffbfab61..9cdcf854db8b 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala @@ -17,6 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators @@ -121,7 +122,7 @@ object RowToVeloxColumnarExec { val arrowSchema = SparkArrowUtil.toArrowSchema(schema, SQLConf.get.sessionLocalTimeZone) - val runtime = Runtimes.contextInstance("RowToColumnar") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "RowToColumnar") val jniWrapper = NativeRowToColumnarJniWrapper.create(runtime) val arrowAllocator = ArrowBufferAllocators.contextInstance() val cSchema = ArrowSchema.allocateNew(arrowAllocator) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala index 8aedeb87cb28..4c0f79538c0f 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.execution +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.extension.ValidationResult @@ -122,7 +123,7 @@ object VeloxColumnarToRowExec { return Iterator.empty } - val runtime = Runtimes.contextInstance("ColumnarToRow") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ColumnarToRow") // TODO: Pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast. val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime) val c2rId = jniWrapper.nativeColumnarToRowInit() @@ -156,7 +157,7 @@ object VeloxColumnarToRowExec { val cols = batch.numCols() val rows = batch.numRows() val beforeConvert = System.currentTimeMillis() - val batchHandle = ColumnarBatches.getNativeHandle(batch) + val batchHandle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch) var info = jniWrapper.nativeColumnarToRowConvert(c2rId, batchHandle, 0) diff --git a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala index 88215e36bb77..cd035e3202d2 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala @@ -17,6 +17,7 @@ package org.apache.gluten.vectorized import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.iterator.ClosableIterator import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.runtime.Runtimes @@ -99,7 +100,7 @@ private class ColumnarBatchSerializerInstance( GlutenConfig.getConf.columnarShuffleCodecBackend.orNull val batchSize = GlutenConfig.getConf.maxBatchSize val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize - val runtime = Runtimes.contextInstance("ShuffleReader") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleReader") val jniWrapper = ShuffleReaderJniWrapper.create(runtime) val shuffleReaderHandle = jniWrapper.make( cSchema.memoryAddress(), @@ -135,7 +136,8 @@ private class ColumnarBatchSerializerInstance( extends DeserializationStream with TaskResource { private val byteIn: JniByteInputStream = JniByteInputStreams.create(in) - private val runtime = Runtimes.contextInstance("ShuffleReader") + private val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleReader") private val wrappedOut: ClosableIterator = new ColumnarBatchOutIterator( runtime, ShuffleReaderJniWrapper diff --git a/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala b/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala index eaf9d99a9ecc..e9f821512e16 100644 --- a/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala +++ b/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.memory.memtarget.{MemoryTarget, Spiller, Spillers} import org.apache.gluten.runtime.Runtimes @@ -99,7 +100,7 @@ class ColumnarShuffleWriter[K, V]( private val reallocThreshold = GlutenConfig.getConf.columnarShuffleReallocThreshold - private val runtime = Runtimes.contextInstance("ShuffleWriter") + private val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleWriter") private val jniWrapper = ShuffleWriterJniWrapper.create(runtime) @@ -135,7 +136,7 @@ class ColumnarShuffleWriter[K, V]( logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols") } else { val rows = cb.numRows() - val handle = ColumnarBatches.getNativeHandle(cb) + val handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, cb) if (nativeShuffleWriter == -1L) { nativeShuffleWriter = jniWrapper.make( dep.nativePartitioning.getShortName, diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala index 5bc6b7c56da5..11a8cc980904 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.runtime.Runtimes import org.apache.gluten.sql.shims.SparkShimLoader @@ -152,11 +153,14 @@ object BroadcastUtils { if (filtered.isEmpty) { return ColumnarBatchSerializeResult.EMPTY } - val handleArray = filtered.map(ColumnarBatches.getNativeHandle) + val handleArray = + filtered.map(b => ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, b)) val serializeResult = try { ColumnarBatchSerializerJniWrapper - .create(Runtimes.contextInstance("BroadcastUtils#serializeStream")) + .create( + Runtimes + .contextInstance(BackendsApiManager.getBackendName, "BroadcastUtils#serializeStream")) .serialize(handleArray) } finally { filtered.foreach(ColumnarBatches.release) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala index d5c848dd9700..fa3d348967d5 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators @@ -40,7 +41,8 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra extends BuildSideRelation { override def deserialized: Iterator[ColumnarBatch] = { - val runtime = Runtimes.contextInstance("BuildSideRelation#deserialized") + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#deserialized") val jniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime) val serializeHandle: Long = { val allocator = ArrowBufferAllocators.contextInstance() @@ -86,7 +88,8 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra * was called in Spark Driver, should manage resources carefully. */ override def transform(key: Expression): Array[InternalRow] = TaskResources.runUnsafe { - val runtime = Runtimes.contextInstance("BuildSideRelation#transform") + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#transform") // This transformation happens in Spark driver, thus resources can not be managed automatically. val serializerJniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime) val serializeHandle = { @@ -150,7 +153,7 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra var info = jniWrapper.nativeColumnarToRowConvert( c2rId, - ColumnarBatches.getNativeHandle(batch), + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch), 0) batch.close() diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala index 6fd346f64cbd..64ad105c7f7c 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala @@ -176,8 +176,12 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe val batch = it.next() val results = ColumnarBatchSerializerJniWrapper - .create(Runtimes.contextInstance("ColumnarCachedBatchSerializer#serialize")) - .serialize(Array(ColumnarBatches.getNativeHandle(batch))) + .create( + Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "ColumnarCachedBatchSerializer#serialize")) + .serialize( + Array(ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch))) CachedColumnarBatch( results.getNumRows.toInt, results.getSerialized.length, @@ -201,7 +205,9 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe val timezoneId = SQLConf.get.sessionLocalTimeZone input.mapPartitions { it => - val runtime = Runtimes.contextInstance("ColumnarCachedBatchSerializer#read") + val runtime = Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "ColumnarCachedBatchSerializer#read") val jniWrapper = ColumnarBatchSerializerJniWrapper .create(runtime) val schema = SparkArrowUtil.toArrowSchema(localSchema, timezoneId) @@ -224,7 +230,10 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe val batch = ColumnarBatches.create(batchHandle) if (shouldSelectAttributes) { try { - ColumnarBatches.select(batch, requestedColumnIndices.toArray) + ColumnarBatches.select( + BackendsApiManager.getBackendName, + batch, + requestedColumnIndices.toArray) } finally { batch.close() } diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala index cd5f442bc765..c9651557fa70 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.velox +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.datasource.{VeloxDataSourceJniWrapper, VeloxDataSourceUtil} import org.apache.gluten.exception.GlutenException @@ -57,7 +58,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { SparkArrowUtil.toArrowSchema(dataSchema, SQLConf.get.sessionLocalTimeZone) val cSchema = ArrowSchema.allocateNew(ArrowBufferAllocators.contextInstance()) var dsHandle = -1L - val runtime = Runtimes.contextInstance("VeloxWriter") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxWriter") val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime) val allocator = ArrowBufferAllocators.contextInstance() try { @@ -77,7 +78,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { ColumnarBatches.retain(batch) val batchHandle = { ColumnarBatches.checkOffloaded(batch) - ColumnarBatches.getNativeHandle(batch) + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch) } datasourceJniWrapper.writeBatch(dsHandle, batchHandle) batch.close() @@ -108,8 +109,9 @@ class VeloxRowSplitter extends GlutenRowSplitter { partitionColIndice: Array[Int], hasBucket: Boolean, reserve_partition_columns: Boolean = false): BlockStripes = { - val handler = ColumnarBatches.getNativeHandle(row.batch) - val runtime = Runtimes.contextInstance("VeloxPartitionWriter") + val handler = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, row.batch) + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxPartitionWriter") val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime) val originalColumns: Array[Int] = Array.range(0, row.batch.numCols()) val dataColIndice = originalColumns.filterNot(partitionColIndice.contains(_)) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala index 32bac0204519..e3a84d4f074d 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.utils +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators @@ -41,10 +42,11 @@ import org.apache.spark.util.MutablePair object ExecUtil { def convertColumnarToRow(batch: ColumnarBatch): Iterator[InternalRow] = { - val runtime = Runtimes.contextInstance("ExecUtil#ColumnarToRow") + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "ExecUtil#ColumnarToRow") val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime) var info: NativeColumnarToRowInfo = null - val batchHandle = ColumnarBatches.getNativeHandle(batch) + val batchHandle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch) val c2rHandle = jniWrapper.nativeColumnarToRowInit() info = jniWrapper.nativeColumnarToRowConvert(c2rHandle, batchHandle, 0) diff --git a/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java b/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java index 54803aa1930f..f02caf8f2d1b 100644 --- a/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java +++ b/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.columnarbatch; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.execution.RowToVeloxColumnarExec; import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators; import org.apache.gluten.test.VeloxBackendTestBase; @@ -114,9 +115,12 @@ public void testCreateByHandle() { final ColumnarBatch offloaded = ColumnarBatches.offload(ArrowBufferAllocators.contextInstance(), batch); Assert.assertEquals(1, ColumnarBatches.getRefCnt(offloaded)); - final long handle = ColumnarBatches.getNativeHandle(offloaded); + final long handle = + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), offloaded); final ColumnarBatch created = ColumnarBatches.create(handle); - Assert.assertEquals(handle, ColumnarBatches.getNativeHandle(created)); + Assert.assertEquals( + handle, + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), created)); Assert.assertEquals(1, ColumnarBatches.getRefCnt(offloaded)); Assert.assertEquals(1, ColumnarBatches.getRefCnt(created)); ColumnarBatches.retain(created); diff --git a/cpp/core/compute/Runtime.h b/cpp/core/compute/Runtime.h index 3090652b8148..16acada54fb3 100644 --- a/cpp/core/compute/Runtime.h +++ b/cpp/core/compute/Runtime.h @@ -80,11 +80,17 @@ class Runtime : public std::enable_shared_from_this { return kind_; } - virtual void parsePlan(const uint8_t* data, int32_t size, std::optional dumpFile) = 0; + virtual void parsePlan(const uint8_t* data, int32_t size, std::optional dumpFile) { + throw GlutenException("Not implemented"); + } - virtual void parseSplitInfo(const uint8_t* data, int32_t size, std::optional dumpFile) = 0; + virtual void parseSplitInfo(const uint8_t* data, int32_t size, std::optional dumpFile) { + throw GlutenException("Not implemented"); + } - virtual std::string planString(bool details, const std::unordered_map& sessionConf) = 0; + virtual std::string planString(bool details, const std::unordered_map& sessionConf) { + throw GlutenException("Not implemented"); + } // Just for benchmark ::substrait::Plan& getPlan() { @@ -94,11 +100,17 @@ class Runtime : public std::enable_shared_from_this { virtual std::shared_ptr createResultIterator( const std::string& spillDir, const std::vector>& inputs, - const std::unordered_map& sessionConf) = 0; + const std::unordered_map& sessionConf) { + throw GlutenException("Not implemented"); + } - virtual std::shared_ptr createOrGetEmptySchemaBatch(int32_t numRows) = 0; + virtual std::shared_ptr createOrGetEmptySchemaBatch(int32_t numRows) { + throw GlutenException("Not implemented"); + } - virtual std::shared_ptr select(std::shared_ptr, const std::vector&) = 0; + virtual std::shared_ptr select(std::shared_ptr, const std::vector&) { + throw GlutenException("Not implemented"); + } virtual MemoryManager* memoryManager() { return memoryManager_; @@ -106,26 +118,42 @@ class Runtime : public std::enable_shared_from_this { /// This function is used to create certain converter from the format used by /// the backend to Spark unsafe row. - virtual std::shared_ptr createColumnar2RowConverter(int64_t column2RowMemThreshold) = 0; + virtual std::shared_ptr createColumnar2RowConverter(int64_t column2RowMemThreshold) { + throw GlutenException("Not implemented"); + } - virtual std::shared_ptr createRow2ColumnarConverter(struct ArrowSchema* cSchema) = 0; + virtual std::shared_ptr createRow2ColumnarConverter(struct ArrowSchema* cSchema) { + throw GlutenException("Not implemented"); + } virtual std::shared_ptr createShuffleWriter( int numPartitions, std::unique_ptr partitionWriter, - ShuffleWriterOptions options) = 0; + ShuffleWriterOptions options) { + throw GlutenException("Not implemented"); + } - virtual Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) = 0; + virtual Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) { + throw GlutenException("Not implemented"); + } virtual std::shared_ptr createShuffleReader( std::shared_ptr schema, - ShuffleReaderOptions options) = 0; + ShuffleReaderOptions options) { + throw GlutenException("Not implemented"); + } - virtual std::unique_ptr createColumnarBatchSerializer(struct ArrowSchema* cSchema) = 0; + virtual std::unique_ptr createColumnarBatchSerializer(struct ArrowSchema* cSchema) { + throw GlutenException("Not implemented"); + } - virtual void dumpConf(const std::string& path) = 0; + virtual void dumpConf(const std::string& path) { + throw GlutenException("Not implemented"); + } - virtual std::shared_ptr createArrowWriter(const std::string& path) = 0; + virtual std::shared_ptr createArrowWriter(const std::string& path) { + throw GlutenException("Not implemented"); + }; const std::unordered_map& getConfMap() { return confMap_; diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index b7bdae81750a..f75b16b46126 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -41,32 +41,33 @@ using namespace gluten; -static jclass javaReservationListenerClass; +namespace { +jclass javaReservationListenerClass; -static jmethodID reserveMemoryMethod; -static jmethodID unreserveMemoryMethod; +jmethodID reserveMemoryMethod; +jmethodID unreserveMemoryMethod; -static jclass byteArrayClass; +jclass byteArrayClass; -static jclass jniByteInputStreamClass; -static jmethodID jniByteInputStreamRead; -static jmethodID jniByteInputStreamTell; -static jmethodID jniByteInputStreamClose; +jclass jniByteInputStreamClass; +jmethodID jniByteInputStreamRead; +jmethodID jniByteInputStreamTell; +jmethodID jniByteInputStreamClose; -static jclass splitResultClass; -static jmethodID splitResultConstructor; +jclass splitResultClass; +jmethodID splitResultConstructor; -static jclass columnarBatchSerializeResultClass; -static jmethodID columnarBatchSerializeResultConstructor; +jclass columnarBatchSerializeResultClass; +jmethodID columnarBatchSerializeResultConstructor; -static jclass metricsBuilderClass; -static jmethodID metricsBuilderConstructor; -static jclass nativeColumnarToRowInfoClass; -static jmethodID nativeColumnarToRowInfoConstructor; +jclass metricsBuilderClass; +jmethodID metricsBuilderConstructor; +jclass nativeColumnarToRowInfoClass; +jmethodID nativeColumnarToRowInfoConstructor; -static jclass shuffleReaderMetricsClass; -static jmethodID shuffleReaderMetricsSetDecompressTime; -static jmethodID shuffleReaderMetricsSetDeserializeTime; +jclass shuffleReaderMetricsClass; +jmethodID shuffleReaderMetricsSetDecompressTime; +jmethodID shuffleReaderMetricsSetDeserializeTime; class JavaInputStreamAdaptor final : public arrow::io::InputStream { public: @@ -140,6 +141,61 @@ class JavaInputStreamAdaptor final : public arrow::io::InputStream { bool closed_ = false; }; +/// Internal backend consists of empty implementations of Runtime API and MemoryManager API. +/// The backend is used for saving contextual objects only. +/// +/// It's also possible to extend the implementation for handling Arrow-based requests either in the future. +inline static const std::string kInternalBackendKind{"internal"}; + +class InternalMemoryManager : public MemoryManager { + public: + InternalMemoryManager(const std::string& kind) : MemoryManager(kind) {} + + arrow::MemoryPool* getArrowMemoryPool() override { + throw GlutenException("Not implemented"); + } + + const MemoryUsageStats collectMemoryUsageStats() const override { + return MemoryUsageStats(); + } + + const int64_t shrink(int64_t size) override { + return 0; + } + + void hold() override {} +}; + +class InternalRuntime : public Runtime { + public: + InternalRuntime( + const std::string& kind, + MemoryManager* memoryManager, + const std::unordered_map& confMap) + : Runtime(kind, memoryManager, confMap) {} +}; + +MemoryManager* internalMemoryManagerFactory(const std::string& kind, std::unique_ptr listener) { + return new InternalMemoryManager(kind); +} + +void internalMemoryManagerReleaser(MemoryManager* memoryManager) { + delete memoryManager; +} + +Runtime* internalRuntimeFactory( + const std::string& kind, + MemoryManager* memoryManager, + const std::unordered_map& sessionConf) { + return new InternalRuntime(kind, memoryManager, sessionConf); +} + +void internalRuntimeReleaser(Runtime* runtime) { + delete runtime; +} + +} // namespace + #ifdef __cplusplus extern "C" { #endif @@ -152,6 +208,9 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) { getJniCommonState()->ensureInitialized(env); getJniErrorState()->ensureInitialized(env); + MemoryManager::registerFactory(kInternalBackendKind, internalMemoryManagerFactory, internalMemoryManagerReleaser); + Runtime::registerFactory(kInternalBackendKind, internalRuntimeFactory, internalRuntimeReleaser); + byteArrayClass = createGlobalClassReferenceOrError(env, "[B"); jniByteInputStreamClass = createGlobalClassReferenceOrError(env, "Lorg/apache/gluten/vectorized/JniByteInputStream;"); @@ -275,11 +334,11 @@ JNIEXPORT jbyteArray JNICALL Java_org_apache_gluten_memory_NativeMemoryManagerJn const MemoryUsageStats& stats = memoryManager->collectMemoryUsageStats(); auto size = stats.ByteSizeLong(); jbyteArray out = env->NewByteArray(size); - uint8_t buffer[size]; + std::vector buffer(size); GLUTEN_CHECK( - stats.SerializeToArray(reinterpret_cast(buffer), size), + stats.SerializeToArray(reinterpret_cast(buffer.data()), size), "Serialization failed when collecting memory usage stats"); - env->SetByteArrayRegion(out, 0, size, reinterpret_cast(buffer)); + env->SetByteArrayRegion(out, 0, size, reinterpret_cast(buffer.data())); return out; JNI_METHOD_END(nullptr) } @@ -650,7 +709,7 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_vectorized_NativeRowToColumnarJniW JNIEXPORT jstring JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_getType( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -660,7 +719,7 @@ JNIEXPORT jstring JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniW JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_numBytes( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -670,7 +729,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_numColumns( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -680,7 +739,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_numRows( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -690,7 +749,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_exportToArrow( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle, jlong cSchema, jlong cArray) { @@ -703,6 +762,15 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrap JNI_METHOD_END() } +JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_close( // NOLINT + JNIEnv* env, + jclass, + jlong batchHandle) { + JNI_METHOD_START + ObjectStore::release(batchHandle); + JNI_METHOD_END() +} + JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_createWithArrowArray( // NOLINT JNIEnv* env, jobject wrapper, @@ -752,15 +820,6 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNI_METHOD_END(kInvalidObjectHandle) } -JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_close( // NOLINT - JNIEnv* env, - jobject wrapper, - jlong batchHandle) { - JNI_METHOD_START - ObjectStore::release(batchHandle); - JNI_METHOD_END() -} - // Shuffle JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrapper_nativeMake( // NOLINT JNIEnv* env, diff --git a/cpp/core/utils/Registry.h b/cpp/core/utils/Registry.h index e50eb6763d72..a325a05e5a85 100644 --- a/cpp/core/utils/Registry.h +++ b/cpp/core/utils/Registry.h @@ -29,19 +29,19 @@ class Registry { public: void registerObj(const std::string& kind, T t) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) == map_.end(), "Already registered for " + kind); + GLUTEN_CHECK(map_.find(kind) == map_.end(), "Required object already registered for " + kind); map_[kind] = std::move(t); } T& get(const std::string& kind) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) != map_.end(), "Not registered for " + kind); + GLUTEN_CHECK(map_.find(kind) != map_.end(), "Required object not registered for " + kind); return map_[kind]; } bool unregisterObj(const std::string& kind) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) != map_.end(), "Not registered for " + kind); + GLUTEN_CHECK(map_.find(kind) != map_.end(), "Required object not registered for " + kind); return map_.erase(kind); } diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index 6ea60d651a73..9d6ad157ff3a 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -41,13 +41,15 @@ using namespace gluten; using namespace facebook; +namespace { +jclass blockStripesClass; +jmethodID blockStripesConstructor; +} // namespace + #ifdef __cplusplus extern "C" { #endif -static jclass blockStripesClass; -static jmethodID blockStripesConstructor; - jint JNI_OnLoad(JavaVM* vm, void*) { JNIEnv* env; if (vm->GetEnv(reinterpret_cast(&env), jniVersion) != JNI_OK) { diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java index 94312a2cf53e..d69e84c3a1fc 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java @@ -30,24 +30,26 @@ public static ColumnarBatchJniWrapper create(Runtime runtime) { return new ColumnarBatchJniWrapper(runtime); } - public native long createWithArrowArray(long cSchema, long cArray); + // Static methods. + public static native String getType(long batch); - public native long getForEmptySchema(int numRows); + public static native long numColumns(long batch); + + public static native long numRows(long batch); - public native String getType(long batch); + public static native long numBytes(long batch); - public native long numColumns(long batch); + public static native void exportToArrow(long batch, long cSchema, long cArray); - public native long numRows(long batch); + public static native void close(long batch); - public native long numBytes(long batch); + // Member methods in which native code relies on the backend's runtime API implementation. + public native long createWithArrowArray(long cSchema, long cArray); - public native void exportToArrow(long batch, long cSchema, long cArray); + public native long getForEmptySchema(int numRows); public native long select(long batch, int[] columnIndices); - public native void close(long batch); - @Override public long rtHandle() { return runtime.getHandle(); diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java index 04236884a1a2..3914fb155ec4 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java @@ -45,6 +45,7 @@ import scala.collection.JavaConverters; public final class ColumnarBatches { + private static final String INTERNAL_BACKEND_KIND = "internal"; private ColumnarBatches() {} @@ -108,8 +109,8 @@ static boolean isZeroColumnBatch(ColumnarBatch batch) { * This method will always return a velox based ColumnarBatch. This method will close the input * column batch. */ - public static ColumnarBatch select(ColumnarBatch batch, int[] columnIndices) { - final Runtime runtime = Runtimes.contextInstance("ColumnarBatches#select"); + public static ColumnarBatch select(String backendName, ColumnarBatch batch, int[] columnIndices) { + final Runtime runtime = Runtimes.contextInstance(backendName, "ColumnarBatches#select"); switch (identifyBatchType(batch)) { case LIGHT: final IndicatorVector iv = getIndicatorVector(batch); @@ -188,8 +189,8 @@ public static ColumnarBatch load(BufferAllocator allocator, ColumnarBatch input) ArrowArray cArray = ArrowArray.allocateNew(allocator); ArrowSchema arrowSchema = ArrowSchema.allocateNew(allocator); CDataDictionaryProvider provider = new CDataDictionaryProvider()) { - ColumnarBatchJniWrapper.create(Runtimes.contextInstance("ColumnarBatches#load")) - .exportToArrow(iv.handle(), cSchema.memoryAddress(), cArray.memoryAddress()); + ColumnarBatchJniWrapper.exportToArrow( + iv.handle(), cSchema.memoryAddress(), cArray.memoryAddress()); Data.exportSchema( allocator, ArrowUtil.toArrowSchema(cSchema, allocator, provider), provider, arrowSchema); @@ -229,7 +230,10 @@ public static ColumnarBatch offload(BufferAllocator allocator, ColumnarBatch inp if (input.numCols() == 0) { throw new IllegalArgumentException("batch with zero columns cannot be offloaded"); } - final Runtime runtime = Runtimes.contextInstance("ColumnarBatches#offload"); + // Batch-offloading doesn't involve any backend-specific native code. Use the internal + // backend to store native batch references only. + final Runtime runtime = + Runtimes.contextInstance(INTERNAL_BACKEND_KIND, "ColumnarBatches#offload"); try (ArrowArray cArray = ArrowArray.allocateNew(allocator); ArrowSchema cSchema = ArrowSchema.allocateNew(allocator)) { ArrowAbiUtil.exportFromSparkColumnarBatch(allocator, input, cSchema, cArray); @@ -383,11 +387,11 @@ private static IndicatorVector getIndicatorVector(ColumnarBatch input) { return (IndicatorVector) input.column(0); } - public static long getNativeHandle(ColumnarBatch batch) { + public static long getNativeHandle(String backendName, ColumnarBatch batch) { if (isZeroColumnBatch(batch)) { final ColumnarBatchJniWrapper jniWrapper = ColumnarBatchJniWrapper.create( - Runtimes.contextInstance("ColumnarBatches#getNativeHandle")); + Runtimes.contextInstance(backendName, "ColumnarBatches#getNativeHandle")); return jniWrapper.getForEmptySchema(batch.numRows()); } return getIndicatorVector(batch).handle(); diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java index 3f09a3619b3a..251925d035e6 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java @@ -54,7 +54,7 @@ void release() { } if (refCnt.decrementAndGet() == 0) { pool.remove(handle); - jniWrapper.close(handle); + ColumnarBatchJniWrapper.close(handle); } } } diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java index 700eb3cadeee..e0e266ca0490 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java @@ -16,8 +16,6 @@ */ package org.apache.gluten.columnarbatch; -import org.apache.gluten.runtime.Runtimes; - import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.vectorized.ColumnVector; @@ -26,26 +24,23 @@ import org.apache.spark.unsafe.types.UTF8String; public abstract class IndicatorVectorBase extends ColumnVector { - protected final ColumnarBatchJniWrapper jniWrapper; protected final long handle; protected IndicatorVectorBase(long handle) { super(DataTypes.NullType); - this.jniWrapper = - ColumnarBatchJniWrapper.create(Runtimes.contextInstance("IndicatorVectorBase#init")); this.handle = handle; } public String getType() { - return jniWrapper.getType(handle); + return ColumnarBatchJniWrapper.getType(handle); } public long getNumColumns() { - return jniWrapper.numColumns(handle); + return ColumnarBatchJniWrapper.numColumns(handle); } public long getNumRows() { - return jniWrapper.numRows(handle); + return ColumnarBatchJniWrapper.numRows(handle); } abstract long refCnt(); diff --git a/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java b/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java index f85187fae695..fe1ebc763415 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java @@ -17,7 +17,6 @@ package org.apache.gluten.init; import org.apache.gluten.GlutenConfig; -import org.apache.gluten.backend.Backend; import org.apache.gluten.utils.ConfigUtil; import org.apache.spark.util.SparkShutdownManagerUtil; @@ -25,6 +24,7 @@ import org.slf4j.LoggerFactory; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import scala.runtime.BoxedUnit; @@ -32,7 +32,18 @@ // Initialize native backend before calling any native methods from Java side. public final class NativeBackendInitializer { private static final Logger LOG = LoggerFactory.getLogger(NativeBackendInitializer.class); - private static final AtomicBoolean initialized = new AtomicBoolean(false); + private static final Map instances = new ConcurrentHashMap<>(); + + private final AtomicBoolean initialized = new AtomicBoolean(false); + private final String backendName; + + private NativeBackendInitializer(String backendName) { + this.backendName = backendName; + } + + public static NativeBackendInitializer forBackend(String backendName) { + return instances.computeIfAbsent(backendName, k -> new NativeBackendInitializer(backendName)); + } // Spark DriverPlugin/ExecutorPlugin will only invoke NativeBackendInitializer#initializeBackend // method once in its init method. @@ -41,7 +52,7 @@ public final class NativeBackendInitializer { // In local mode, NativeBackendInitializer#initializeBackend will be invoked twice in same // thread, driver first then executor, initialized flag ensure only invoke initializeBackend once, // so there are no race condition here. - public static void initializeBackend(scala.collection.Map conf) { + public void initialize(scala.collection.Map conf) { if (!initialized.compareAndSet(false, true)) { // Already called. return; @@ -54,10 +65,9 @@ public static void initializeBackend(scala.collection.Map conf) }); } - private static void initialize0(scala.collection.Map conf) { + private void initialize0(scala.collection.Map conf) { try { - Map nativeConfMap = - GlutenConfig.getNativeBackendConf(Backend.get().name(), conf); + Map nativeConfMap = GlutenConfig.getNativeBackendConf(backendName, conf); initialize(ConfigUtil.serialize(nativeConfMap)); } catch (Exception e) { LOG.error("Failed to call native backend's initialize method", e); @@ -65,9 +75,7 @@ private static void initialize0(scala.collection.Map conf) { } } - private static native void initialize(byte[] configPlan); - - private static native void shutdown(); + private native void initialize(byte[] configPlan); - private NativeBackendInitializer() {} + private native void shutdown(); } diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java index f95324fad991..378927bec433 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java @@ -23,9 +23,11 @@ import java.util.Iterator; public class ColumnarBatchInIterator { + private final String backendName; private final Iterator delegated; - public ColumnarBatchInIterator(Iterator delegated) { + public ColumnarBatchInIterator(String backendName, Iterator delegated) { + this.backendName = backendName; this.delegated = delegated; } @@ -38,6 +40,6 @@ public boolean hasNext() { public long next() { final ColumnarBatch next = delegated.next(); ColumnarBatches.checkOffloaded(next); - return ColumnarBatches.getNativeHandle(next); + return ColumnarBatches.getNativeHandle(backendName, next); } } diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java index 1c03c415ed10..bfa059b5aeb2 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java @@ -32,17 +32,19 @@ public class NativePlanEvaluator { private static final AtomicInteger id = new AtomicInteger(0); - private final Runtime runtime = - Runtimes.contextInstance(String.format("NativePlanEvaluator-%d", id.getAndIncrement())); + private final Runtime runtime; private final PlanEvaluatorJniWrapper jniWrapper; - private NativePlanEvaluator() { - jniWrapper = PlanEvaluatorJniWrapper.create(runtime); + private NativePlanEvaluator(Runtime runtime) { + this.runtime = runtime; + this.jniWrapper = PlanEvaluatorJniWrapper.create(runtime); } - public static NativePlanEvaluator create() { - return new NativePlanEvaluator(); + public static NativePlanEvaluator create(String backendName) { + return new NativePlanEvaluator( + Runtimes.contextInstance( + backendName, String.format("NativePlanEvaluator-%d", id.getAndIncrement()))); } public NativePlanValidationInfo doNativeValidateWithFailureReason(byte[] subPlan) { diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala b/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala index 409c4297fac2..e65d3c4d9589 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala @@ -17,7 +17,6 @@ package org.apache.gluten.memory import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.exception.GlutenException import org.apache.gluten.memory.listener.ReservationListeners import org.apache.gluten.memory.memtarget.{KnownNameAndStats, MemoryTarget, Spiller, Spillers} @@ -42,18 +41,19 @@ trait NativeMemoryManager { } object NativeMemoryManager { - private class Impl(name: String) extends NativeMemoryManager with TaskResource { + private class Impl(backendName: String, name: String) + extends NativeMemoryManager + with TaskResource { private val LOGGER = LoggerFactory.getLogger(classOf[NativeMemoryManager]) private val spillers = Spillers.appendable() private val mutableStats: mutable.Map[String, MemoryUsageStatsBuilder] = mutable.Map() private val rl = ReservationListeners.create(name, spillers, mutableStats.asJava) private val handle = NativeMemoryManagerJniWrapper.create( - Backend.get().name(), + backendName, rl, ConfigUtil.serialize( - GlutenConfig.getNativeSessionConf( - Backend.get().name(), - GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) + GlutenConfig + .getNativeSessionConf(backendName, GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) ) spillers.append(new Spiller() { override def spill(self: MemoryTarget, phase: Spiller.Phase, size: Long): Long = { @@ -109,11 +109,15 @@ object NativeMemoryManager { )) } } - override def priority(): Int = 0 + override def priority(): Int = { + // Memory managers should be released after all runtimes are released. + // So lower the priority to 0. + 0 + } override def resourceName(): String = "nmm" } - def apply(name: String): NativeMemoryManager = { - TaskResources.addAnonymousResource(new Impl(name)) + def apply(backendName: String, name: String): NativeMemoryManager = { + TaskResources.addAnonymousResource(new Impl(backendName, name)) } } diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala index efd4928cfdee..8741c12474e4 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala @@ -17,7 +17,6 @@ package org.apache.gluten.runtime import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.exception.GlutenException import org.apache.gluten.memory.NativeMemoryManager import org.apache.gluten.utils.ConfigUtil @@ -35,21 +34,20 @@ trait Runtime { } object Runtime { - private[runtime] def apply(name: String): Runtime with TaskResource = { - new RuntimeImpl(name) + private[runtime] def apply(backendName: String, name: String): Runtime with TaskResource = { + new RuntimeImpl(backendName, name) } - private class RuntimeImpl(name: String) extends Runtime with TaskResource { + private class RuntimeImpl(backendName: String, name: String) extends Runtime with TaskResource { private val LOGGER = LoggerFactory.getLogger(classOf[Runtime]) - private val nmm: NativeMemoryManager = NativeMemoryManager(name) + private val nmm: NativeMemoryManager = NativeMemoryManager(backendName, name) private val handle = RuntimeJniWrapper.createRuntime( - Backend.get().name(), + backendName, nmm.getHandle(), ConfigUtil.serialize( - GlutenConfig.getNativeSessionConf( - Backend.get().name(), - GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) + GlutenConfig + .getNativeSessionConf(backendName, GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) ) private val released: AtomicBoolean = new AtomicBoolean(false) diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala index bfb2465b12b9..1aca275d5ce9 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala @@ -21,15 +21,16 @@ import org.apache.spark.task.{TaskResource, TaskResources} object Runtimes { /** Get or create the runtime which bound with Spark TaskContext. */ - def contextInstance(name: String): Runtime = { + def contextInstance(backendName: String, name: String): Runtime = { if (!TaskResources.inSparkTask()) { throw new IllegalStateException("This method must be called in a Spark task.") } - TaskResources.addResourceIfNotRegistered(name, () => create(name)) + val resourceName = String.format("%s:%s", backendName, name) + TaskResources.addResourceIfNotRegistered(resourceName, () => create(backendName, name)) } - private def create(name: String): Runtime with TaskResource = { - Runtime(name) + private def create(backendName: String, name: String): Runtime with TaskResource = { + Runtime(backendName, name) } } diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala index 42dcd9bed567..e4652c541512 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.shuffle -import org.apache.gluten.backend.Backend +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.spark.TaskContext import org.apache.spark.shuffle.celeborn.CelebornShuffleHandle @@ -26,7 +26,7 @@ import org.apache.celeborn.client.ShuffleClient import org.apache.celeborn.common.CelebornConf class CHCelebornColumnarShuffleWriterFactory extends CelebornShuffleWriterFactory { - override def backendName(): String = Backend.get().name() + override def backendName(): String = BackendsApiManager.getBackendName override def createShuffleWriterInstance[K, V]( shuffleId: Int, diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala index d5f20c8dea00..a4a97d43de08 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig import org.apache.gluten.GlutenConfig.{GLUTEN_RSS_SORT_SHUFFLE_WRITER, GLUTEN_SORT_SHUFFLE_WRITER} +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.runtime.Runtimes import org.apache.gluten.utils.ArrowAbiUtil @@ -65,7 +66,8 @@ private class CelebornColumnarBatchSerializerInstance( extends SerializerInstance with Logging { - private val runtime = Runtimes.contextInstance("CelebornShuffleReader") + private val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "CelebornShuffleReader") private val shuffleReaderHandle = { val allocator: BufferAllocator = ArrowBufferAllocators diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala index 1a0bc475d327..165d68785d4b 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.memory.memtarget.{MemoryTarget, Spiller, Spillers} import org.apache.gluten.runtime.Runtimes @@ -51,7 +52,8 @@ class VeloxCelebornColumnarShuffleWriter[K, V]( writeMetrics) { private val isSort = !GlutenConfig.GLUTEN_HASH_SHUFFLE_WRITER.equals(shuffleWriterType) - private val runtime = Runtimes.contextInstance("CelebornShuffleWriter") + private val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "CelebornShuffleWriter") private val jniWrapper = ShuffleWriterJniWrapper.create(runtime) @@ -75,7 +77,7 @@ class VeloxCelebornColumnarShuffleWriter[K, V]( logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols") } else { initShuffleWriter(cb) - val handle = ColumnarBatches.getNativeHandle(cb) + val handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, cb) val startTime = System.nanoTime() jniWrapper.write(nativeShuffleWriter, cb.numRows, handle, availableOffHeapPerTask()) dep.metrics("shuffleWallTime").add(System.nanoTime() - startTime) @@ -131,7 +133,7 @@ class VeloxCelebornColumnarShuffleWriter[K, V]( clientPushBufferMaxSize, clientPushSortMemoryThreshold, celebornPartitionPusher, - ColumnarBatches.getNativeHandle(columnarBatch), + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, columnarBatch), context.taskAttemptId(), GlutenShuffleUtils.getStartPartitionId(dep.nativePartitioning, context.partitionId), "celeborn", diff --git a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala index e52c53d1d558..03d16c41c72b 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala @@ -18,7 +18,7 @@ package org.apache.gluten import org.apache.gluten.GlutenBuildInfo._ import org.apache.gluten.GlutenConfig._ -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component import org.apache.gluten.events.GlutenBuildInfoEvent import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.GlutenSessionExtensions @@ -68,7 +68,7 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { setPredefinedConfigs(conf) // Initialize Backend. - Backend.get().onDriverStart(sc, pluginContext) + Component.sorted().foreach(_.onDriverStart(sc, pluginContext)) Collections.emptyMap() } @@ -84,12 +84,10 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { } override def shutdown(): Unit = { - Backend.get().onDriverShutdown() + Component.sorted().reverse.foreach(_.onDriverShutdown()) } private def postBuildInfoEvent(sc: SparkContext): Unit = { - val buildInfo = Backend.get().buildInfo() - // export gluten version to property to spark System.setProperty("gluten.version", VERSION) @@ -105,10 +103,16 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { glutenBuildInfo.put("Gluten Revision Time", REVISION_TIME) glutenBuildInfo.put("Gluten Build Time", BUILD_DATE) glutenBuildInfo.put("Gluten Repo URL", REPO_URL) - glutenBuildInfo.put("Backend", buildInfo.name) - glutenBuildInfo.put("Backend Branch", buildInfo.branch) - glutenBuildInfo.put("Backend Revision", buildInfo.revision) - glutenBuildInfo.put("Backend Revision Time", buildInfo.revisionTime) + + Component.sorted().foreach { + comp => + val buildInfo = comp.buildInfo() + glutenBuildInfo.put("Component", buildInfo.name) + glutenBuildInfo.put("Component Branch", buildInfo.branch) + glutenBuildInfo.put("Component Revision", buildInfo.revision) + glutenBuildInfo.put("Component Revision Time", buildInfo.revisionTime) + } + val infoMap = glutenBuildInfo.toMap val loggingInfo = infoMap.toSeq .sortBy(_._1) @@ -254,12 +258,12 @@ private[gluten] class GlutenExecutorPlugin extends ExecutorPlugin { /** Initialize the executor plugin. */ override def init(ctx: PluginContext, extraConf: util.Map[String, String]): Unit = { // Initialize Backend. - Backend.get().onExecutorStart(ctx) + Component.sorted().foreach(_.onExecutorStart(ctx)) } /** Clean up and terminate this plugin. For example: close the native engine. */ override def shutdown(): Unit = { - Backend.get().onExecutorShutdown() + Component.sorted().reverse.foreach(_.onExecutorShutdown()) super.shutdown() } diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala index f406a6ac4d7d..02a2a44349c9 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala @@ -16,60 +16,11 @@ */ package org.apache.gluten.backend -import org.apache.gluten.extension.columnar.transition.ConventionFunc -import org.apache.gluten.extension.injector.Injector - -import org.apache.spark.SparkContext -import org.apache.spark.api.plugin.PluginContext - -import java.util.ServiceLoader - -import scala.collection.JavaConverters - -trait Backend { - import Backend._ - - /** Base information. */ - def name(): String - def buildInfo(): BuildInfo - - /** Spark listeners. */ - def onDriverStart(sc: SparkContext, pc: PluginContext): Unit = {} - def onDriverShutdown(): Unit = {} - def onExecutorStart(pc: PluginContext): Unit = {} - def onExecutorShutdown(): Unit = {} +trait Backend extends Component { /** - * Overrides [[org.apache.gluten.extension.columnar.transition.ConventionFunc]] Gluten is using to - * determine the convention (its row-based processing / columnar-batch processing support) of a - * plan with a user-defined function that accepts a plan then returns convention type it outputs, - * and input conventions it requires. + * Backends don't have dependencies. They are all considered root components in the component DAG + * and will be loaded at the beginning. */ - def convFuncOverride(): ConventionFunc.Override = ConventionFunc.Override.Empty - - /** Query planner rules. */ - def injectRules(injector: Injector): Unit -} - -object Backend { - private val backend: Backend = { - val discoveredBackends = - JavaConverters.iterableAsScalaIterable(ServiceLoader.load(classOf[Backend])).toList - discoveredBackends match { - case Nil => - throw new IllegalStateException("Backend implementation not discovered from JVM classpath") - case head :: Nil => - head - case backends => - val backendNames = backends.map(_.name()) - throw new IllegalStateException( - s"More than one Backend implementation discovered from JVM classpath: $backendNames") - } - } - - def get(): Backend = { - backend - } - - case class BuildInfo(name: String, branch: String, revision: String, revisionTime: String) + final override def dependencies(): Seq[Class[_ <: Component]] = Nil } diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/Component.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/Component.scala new file mode 100644 index 000000000000..8670bede87e5 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/Component.scala @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.backend + +import org.apache.gluten.extension.columnar.transition.ConventionFunc +import org.apache.gluten.extension.injector.Injector + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.plugin.PluginContext + +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} + +import scala.collection.mutable + +/** + * The base API to inject user-defined logic to Gluten. To register a component, its implementations + * should be placed to Gluten's classpath with a Java service file. Gluten will discover all the + * component implementations then register them at the booting time. + * + * Experimental: This is not expected to be used in production yet. Use [[Backend]] instead. + */ +@Experimental +trait Component { + import Component._ + + private val uid = nextUid.getAndIncrement() + private val isRegistered = new AtomicBoolean(false) + + def ensureRegistered(): Unit = { + if (!isRegistered.compareAndSet(false, true)) { + return + } + graph.add(this) + dependencies().foreach(req => graph.declareDependency(this, req)) + } + + /** Base information. */ + def name(): String + def buildInfo(): BuildInfo + def dependencies(): Seq[Class[_ <: Component]] + + /** Spark listeners. */ + def onDriverStart(sc: SparkContext, pc: PluginContext): Unit = {} + def onDriverShutdown(): Unit = {} + def onExecutorStart(pc: PluginContext): Unit = {} + def onExecutorShutdown(): Unit = {} + + /** + * Overrides [[org.apache.gluten.extension.columnar.transition.ConventionFunc]] Gluten is using to + * determine the convention (its row-based processing / columnar-batch processing support) of a + * plan with a user-defined function that accepts a plan then returns convention type it outputs, + * and input conventions it requires. + */ + def convFuncOverride(): ConventionFunc.Override = ConventionFunc.Override.Empty + + /** Query planner rules. */ + def injectRules(injector: Injector): Unit +} + +object Component { + private val nextUid = new AtomicInteger() + private val graph: Graph = new Graph() + + // format: off + /** + * Apply topology sort on all registered components in graph to get an ordered list of + * components. The root nodes will be on the head side of the list, while leaf nodes + * will be on the tail side of the list. + * + * Say if component-A depends on component-B while component-C requires nothing, then the + * output order will be one of the following: + * + * 1. [component-B, component-A, component-C] + * 2. [component-C, component-B, component-A] + * 3. [component-B, component-C, component-A] + * + * By all means component B will be placed before component A because of the declared + * dependency from component A to component B. + * + * @throws UnsupportedOperationException When cycles in dependency graph are found. + */ + // format: on + def sorted(): Seq[Component] = { + ensureAllComponentsRegistered() + graph.sorted() + } + + private[backend] def sortedUnsafe(): Seq[Component] = { + graph.sorted() + } + + private class Registry { + private val lookupByUid: mutable.Map[Int, Component] = mutable.Map() + private val lookupByClass: mutable.Map[Class[_ <: Component], Component] = mutable.Map() + + def register(comp: Component): Unit = synchronized { + val uid = comp.uid + val clazz = comp.getClass + require(!lookupByUid.contains(uid), s"Component UID $uid already registered: ${comp.name()}") + require( + !lookupByClass.contains(clazz), + s"Component class $clazz already registered: ${comp.name()}") + lookupByUid += uid -> comp + lookupByClass += clazz -> comp + } + + def isUidRegistered(uid: Int): Boolean = synchronized { + lookupByUid.contains(uid) + } + + def isClassRegistered(clazz: Class[_ <: Component]): Boolean = synchronized { + lookupByClass.contains(clazz) + } + + def findByClass(clazz: Class[_ <: Component]): Component = synchronized { + require(lookupByClass.contains(clazz)) + lookupByClass(clazz) + } + + def findByUid(uid: Int): Component = synchronized { + require(lookupByUid.contains(uid)) + lookupByUid(uid) + } + + def allUids(): Seq[Int] = synchronized { + return lookupByUid.keys.toSeq + } + } + + private class Graph { + import Graph._ + private val registry: Registry = new Registry() + private val dependencies: mutable.Buffer[(Int, Class[_ <: Component])] = mutable.Buffer() + + private var sortedComponents: Option[Seq[Component]] = None + + def add(comp: Component): Unit = synchronized { + require( + !registry.isUidRegistered(comp.uid), + s"Component UID ${comp.uid} already registered: ${comp.name()}") + require( + !registry.isClassRegistered(comp.getClass), + s"Component class ${comp.getClass} already registered: ${comp.name()}") + registry.register(comp) + sortedComponents = None + } + + def declareDependency(comp: Component, dependencyCompClass: Class[_ <: Component]): Unit = + synchronized { + require(registry.isUidRegistered(comp.uid)) + require(registry.isClassRegistered(comp.getClass)) + dependencies += comp.uid -> dependencyCompClass + sortedComponents = None + } + + private def newLookup(): mutable.Map[Int, Node] = { + val lookup: mutable.Map[Int, Node] = mutable.Map() + + registry.allUids().foreach { + uid => + require(!lookup.contains(uid)) + val n = new Node(uid) + lookup += uid -> n + } + + dependencies.foreach { + case (uid, dependencyCompClass) => + val dependencyUid = registry.findByClass(dependencyCompClass).uid + require(uid != dependencyUid) + require(lookup.contains(uid)) + require(lookup.contains(dependencyUid)) + val n = lookup(uid) + val r = lookup(dependencyUid) + require(!n.parents.contains(r.uid)) + require(!r.children.contains(n.uid)) + n.parents += r.uid -> r + r.children += n.uid -> n + } + + lookup + } + + def sorted(): Seq[Component] = synchronized { + if (sortedComponents.isDefined) { + return sortedComponents.get + } + + val lookup: mutable.Map[Int, Node] = newLookup() + + val out = mutable.Buffer[Component]() + val uidToNumParents = lookup.map { case (uid, node) => uid -> node.parents.size } + val removalQueue = mutable.Queue[Int]() + + // 1. Find out all nodes with zero parents then enqueue them. + uidToNumParents.filter(_._2 == 0).foreach(kv => removalQueue.enqueue(kv._1)) + + // 2. Loop to dequeue and remove nodes from the uid-to-num-parents map. + while (removalQueue.nonEmpty) { + val parentUid = removalQueue.dequeue() + val node = lookup(parentUid) + out += registry.findByUid(parentUid) + node.children.keys.foreach { + childUid => + uidToNumParents += childUid -> (uidToNumParents(childUid) - 1) + val updatedNumParents = uidToNumParents(childUid) + assert(updatedNumParents >= 0) + if (updatedNumParents == 0) { + removalQueue.enqueue(childUid) + } + } + } + + // 3. If there are still outstanding nodes (those are with more non-zero parents) in the + // uid-to-num-parents map, then it means at least one cycle is found. Report error if so. + if (uidToNumParents.exists(_._2 != 0)) { + val cycleNodes = uidToNumParents.filter(_._2 != 0).keys.map(registry.findByUid) + val cycleNodeNames = cycleNodes.map(_.name()).mkString(", ") + throw new UnsupportedOperationException( + s"Cycle detected in the component graph: $cycleNodeNames") + } + + // 4. Return the ordered nodes. + sortedComponents = Some(out.toSeq) + sortedComponents.get + } + } + + private object Graph { + class Node(val uid: Int) { + val parents: mutable.Map[Int, Node] = mutable.Map() + val children: mutable.Map[Int, Node] = mutable.Map() + } + } + + case class BuildInfo(name: String, branch: String, revision: String, revisionTime: String) +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/package.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/package.scala new file mode 100644 index 000000000000..a9981719a333 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/package.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten + +import org.apache.spark.internal.Logging + +import java.util.ServiceLoader +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +package object backend extends Logging { + private[backend] val allComponentsLoaded: AtomicBoolean = new AtomicBoolean(false) + + private[backend] def ensureAllComponentsRegistered(): Unit = { + if (!allComponentsLoaded.compareAndSet(false, true)) { + return + } + + // Load all components in classpath. + val discoveredBackends = ServiceLoader.load(classOf[Backend]).asScala + val discoveredComponents = ServiceLoader.load(classOf[Component]).asScala + val all = discoveredBackends ++ discoveredComponents + + // Register all components. + all.foreach(_.ensureRegistered()) + + // Output log so user could view the component loading order. + // Call #sortedUnsafe than on #sorted to avoid unnecessary recursion. + val components = Component.sortedUnsafe() + logInfo(s"Components registered within order: ${components.map(_.name()).mkString(", ")}") + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala index addcad8dd05c..794f38365a0e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component import org.apache.gluten.extension.injector.Injector import org.apache.spark.internal.Logging @@ -47,7 +47,8 @@ private[gluten] class GlutenSessionExtensions logDebug(s"Gluten is disabled by variable: glutenEnabledForThread: $glutenEnabledForThread") disabled } - Backend.get().injectRules(injector) + // Components should override Backend's rules. Hence, reversed injection order is applied. + Component.sorted().reverse.foreach(_.injectRules(injector)) injector.inject() } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index 67399e25d412..f1a325bc4379 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.enumerated -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization @@ -74,7 +74,8 @@ object EnumeratedTransform { def static(): EnumeratedTransform = { val exts = new SparkSessionExtensions() val dummyInjector = new Injector(exts) - Backend.get().injectRules(dummyInjector) + // Components should override Backend's rules. Hence, reversed injection order is applied. + Component.sorted().reverse.foreach(_.injectRules(dummyInjector)) val session = SparkSession.getActiveSession.getOrElse( throw new GlutenException( "HeuristicTransform#static can only be called when an active Spark session exists")) diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala index aa13cd958cb7..e53c4cbf80e0 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.heuristic -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.offload.OffloadSingleNode @@ -88,7 +88,8 @@ object HeuristicTransform { def static(): HeuristicTransform = { val exts = new SparkSessionExtensions() val dummyInjector = new Injector(exts) - Backend.get().injectRules(dummyInjector) + // Components should override Backend's rules. Hence, reversed injection order is applied. + Component.sorted().reverse.foreach(_.injectRules(dummyInjector)) val session = SparkSession.getActiveSession.getOrElse( throw new GlutenException( "HeuristicTransform#static can only be called when an active Spark session exists")) diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala index 2be9271a1f53..bb894c2af0e1 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildConvention import org.apache.gluten.sql.shims.SparkShimLoader @@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnionExec} import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.util.SparkTestUtil /** ConventionFunc is a utility to derive [[Convention]] or [[ConventionReq]] from a query plan. */ sealed trait ConventionFunc { @@ -45,8 +46,9 @@ object ConventionFunc { // For testing, to make things work without a backend loaded. private var ignoreBackend: Boolean = false - // Visible for testing + // Visible for testing. def ignoreBackend[T](body: => T): T = synchronized { + assert(SparkTestUtil.isTesting) assert(!ignoreBackend) ignoreBackend = true try { @@ -68,7 +70,20 @@ object ConventionFunc { return Override.Empty } } - Backend.get().convFuncOverride() + // Components should override Backend's convention function. Hence, reversed injection order + // is applied. + val overrides = Component.sorted().reverse.map(_.convFuncOverride()) + new Override { + override val rowTypeOf: PartialFunction[SparkPlan, Convention.RowType] = { + overrides.map(_.rowTypeOf).reduce((l, r) => l.orElse(r)) + } + override val batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + overrides.map(_.batchTypeOf).reduce((l, r) => l.orElse(r)) + } + override val conventionReqOf: PartialFunction[SparkPlan, Seq[ConventionReq]] = { + overrides.map(_.conventionReqOf).reduce((l, r) => l.orElse(r)) + } + } } private class BuiltinFunc(o: Override) extends ConventionFunc { diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala index d38781675baf..aa3f4e095861 100644 --- a/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala @@ -25,6 +25,10 @@ import org.apache.spark.annotation.Experimental * * A SPIP may cause refactoring of this class in the future: * https://issues.apache.org/jira/browse/SPARK-45792 + * + * Experimental: This is not expected to be used in production yet. Use backend shuffle manager + * (e.g., ColumnarShuffleManager or other RSS shuffle manager provided in Gluten's code + * base)instead. */ @Experimental class GlutenShuffleManager(conf: SparkConf, isDriver: Boolean) extends ShuffleManager { diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala index 4310054caa51..5b621a755d2b 100644 --- a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle import org.apache.spark.SparkConf -import org.apache.spark.util.Utils +import org.apache.spark.util.{SparkTestUtil, Utils} import scala.collection.mutable @@ -49,8 +49,9 @@ class ShuffleManagerRegistry private[ShuffleManagerRegistry] { } } - // Visible for testing + // Visible for testing. private[shuffle] def clear(): Unit = { + assert(SparkTestUtil.isTesting) this.synchronized { classDeDup.clear() all.clear() diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkTestUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkTestUtil.scala new file mode 100644 index 000000000000..4fc09cf17c99 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkTestUtil.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util + +object SparkTestUtil { + def isTesting: Boolean = { + Utils.isTesting + } +} diff --git a/gluten-core/src/test/scala/org/apache/gluten/backend/ComponentSuite.scala b/gluten-core/src/test/scala/org/apache/gluten/backend/ComponentSuite.scala new file mode 100644 index 000000000000..a6f8bf2a0cbd --- /dev/null +++ b/gluten-core/src/test/scala/org/apache/gluten/backend/ComponentSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.backend + +import org.apache.gluten.extension.injector.Injector + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite + +class ComponentSuite extends AnyFunSuite with BeforeAndAfterAll { + import ComponentSuite._ + + private val d = new DummyComponentD() + d.ensureRegistered() + private val b = new DummyBackendB() + b.ensureRegistered() + private val a = new DummyBackendA() + a.ensureRegistered() + private val c = new DummyComponentC() + c.ensureRegistered() + private val e = new DummyComponentE() + e.ensureRegistered() + + test("Load order - sanity") { + val possibleOrders = + Set( + Seq(a, b, c, d, e), + Seq(a, b, d, c, e), + Seq(b, a, c, d, e), + Seq(b, a, d, c, e) + ) + + assert(possibleOrders.contains(Component.sorted())) + } + + test("Register again") { + assertThrows[IllegalArgumentException] { + new DummyBackendA().ensureRegistered() + } + } +} + +object ComponentSuite { + private class DummyBackendA extends Backend { + override def name(): String = "dummy-backend-a" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_BACKEND_A", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyBackendB extends Backend { + override def name(): String = "dummy-backend-b" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_BACKEND_B", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyComponentC extends Component { + override def dependencies(): Seq[Class[_ <: Component]] = classOf[DummyBackendA] :: Nil + + override def name(): String = "dummy-component-c" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_COMPONENT_C", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyComponentD extends Component { + override def dependencies(): Seq[Class[_ <: Component]] = + Seq(classOf[DummyBackendA], classOf[DummyBackendB]) + + override def name(): String = "dummy-component-d" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_COMPONENT_D", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyComponentE extends Component { + override def dependencies(): Seq[Class[_ <: Component]] = + Seq(classOf[DummyBackendA], classOf[DummyComponentD]) + + override def name(): String = "dummy-component-e" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_COMPONENT_E", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala index 942058cc54d9..ab8ab3688916 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala @@ -16,14 +16,16 @@ */ package org.apache.gluten.backendsapi -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component object BackendsApiManager { private lazy val backend: SubstraitBackend = initializeInternal() /** Initialize all backends api. */ private def initializeInternal(): SubstraitBackend = { - Backend.get().asInstanceOf[SubstraitBackend] + val loadedSubstraitBackends = Component.sorted().filter(_.isInstanceOf[SubstraitBackend]) + assert(loadedSubstraitBackends.size == 1, "More than one Substrait backends are loaded") + loadedSubstraitBackends.head.asInstanceOf[SubstraitBackend] } /** diff --git a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java index 84fac8ace697..d32da8d93b31 100644 --- a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java +++ b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.writer; import org.apache.gluten.GlutenConfig; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.columnarbatch.ColumnarBatches; import org.apache.gluten.memory.memtarget.MemoryTarget; import org.apache.gluten.memory.memtarget.Spiller; @@ -70,7 +71,8 @@ public class VeloxUniffleColumnarShuffleWriter extends RssShuffleWriter> records) { if (cb.numRows() == 0 || cb.numCols() == 0) { LOG.info("Skip ColumnarBatch of 0 rows or 0 cols"); } else { - long handle = ColumnarBatches.getNativeHandle(cb); + long handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), cb); if (nativeShuffleWriter == -1) { nativeShuffleWriter = jniWrapper.makeForRSS( From 5002106d28a3ab782c5286438277fc15cd5686a5 Mon Sep 17 00:00:00 2001 From: lgbo Date: Fri, 6 Dec 2024 16:50:59 +0800 Subject: [PATCH 206/211] [GLUTEN-8142][CH] Duplicated columns in group by (#8164) * fixed: duplicated cols in group by * do not change hash shuffle --- .../backendsapi/clickhouse/CHRuleApi.scala | 1 + .../clickhouse/CHSparkPlanExecApi.scala | 4 +- .../extension/RemoveDuplicatedColumns.scala | 126 ++++++++++++++++++ .../execution/GlutenClickHouseTPCHSuite.scala | 20 +++ 4 files changed, 149 insertions(+), 2 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 98cfa0e7547b..141778688967 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -118,6 +118,7 @@ object CHRuleApi { SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarTransformRules)( c.session))) injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, CHBatch)) + injector.injectPostTransform(c => RemoveDuplicatedColumns.apply(c.session)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index c2f91fa15214..de0680df10ab 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -164,11 +164,11 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { resultExpressions) CHHashAggregateExecTransformer( requiredChildDistributionExpressions, - groupingExpressions.distinct, + groupingExpressions, aggregateExpressions, aggregateAttributes, initialInputBufferOffset, - replacedResultExpressions.distinct, + replacedResultExpressions, child ) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala new file mode 100644 index 000000000000..7f378b5a41a0 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension + +import org.apache.gluten.execution._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.CHColumnarToRowExec + +/* + * CH doesn't support will for duplicate columns in the a block. + * Most of the cases that introduce duplicate columns are from group by. + */ +case class RemoveDuplicatedColumns(session: SparkSession) extends Rule[SparkPlan] with Logging { + override def apply(plan: SparkPlan): SparkPlan = { + visitPlan(plan) + } + + private def visitPlan(plan: SparkPlan): SparkPlan = { + plan match { + case c2r @ CHColumnarToRowExec(hashAgg: CHHashAggregateExecTransformer) => + // This is a special case. Use the result from aggregation as the input of sink. + // We need to make the schema same as the input of sink. + val newChildren = hashAgg.children.map(visitPlan) + val newHashAgg = uniqueHashAggregateColumns(hashAgg) + .withNewChildren(newChildren) + .asInstanceOf[CHHashAggregateExecTransformer] + if (newHashAgg.resultExpressions.length != hashAgg.resultExpressions.length) { + val project = ProjectExecTransformer(hashAgg.resultExpressions, newHashAgg) + c2r.copy(child = project) + } else { + c2r.copy(child = newHashAgg) + } + case hashAgg: CHHashAggregateExecTransformer => + val newChildren = hashAgg.children.map(visitPlan) + val newHashAgg = uniqueHashAggregateColumns(hashAgg) + newHashAgg.withNewChildren(newChildren) + case _ => + plan.withNewChildren(plan.children.map(visitPlan)) + } + } + + private def unwrapAliasNamedExpression(e: NamedExpression): NamedExpression = { + e match { + case a: Alias => + if (a.child.isInstanceOf[NamedExpression]) { + a.child.asInstanceOf[NamedExpression] + } else { + a + } + case _ => e + } + } + private def unwrapAliasExpression(e: Expression): Expression = { + e match { + case a: Alias => + if (a.child.isInstanceOf[Expression]) { + a.child.asInstanceOf[Expression] + } else { + a + } + case _ => e + } + } + + private def uniqueNamedExpressions( + groupingExpressions: Seq[NamedExpression]): Seq[NamedExpression] = { + var addedExpression = Seq[NamedExpression]() + groupingExpressions.foreach { + e => + val unwrapped = unwrapAliasNamedExpression(e) + if ( + !addedExpression.exists(_.semanticEquals(unwrapped)) && !unwrapped.isInstanceOf[Literal] + ) { + addedExpression = addedExpression :+ unwrapped + } + } + addedExpression + } + + private def uniqueExpressions(expressions: Seq[Expression]): Seq[Expression] = { + var addedExpression = Seq[Expression]() + expressions.foreach { + e => + val unwrapped = unwrapAliasExpression(e) + if ( + !addedExpression.exists(_.semanticEquals(unwrapped)) && !unwrapped.isInstanceOf[Literal] + ) { + addedExpression = addedExpression :+ unwrapped + } + } + addedExpression + } + + private def uniqueHashAggregateColumns( + hashAgg: CHHashAggregateExecTransformer): CHHashAggregateExecTransformer = { + val newGroupingExpressions = uniqueNamedExpressions(hashAgg.groupingExpressions) + val newResultExpressions = uniqueNamedExpressions(hashAgg.resultExpressions) + if (newResultExpressions.length != hashAgg.resultExpressions.length) { + hashAgg + .copy( + groupingExpressions = newGroupingExpressions, + resultExpressions = newResultExpressions) + } else { + hashAgg + } + } +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala index bbe51ef3894c..65a01dea3073 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala @@ -570,5 +570,25 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { ", split(concat('a|b|c', cast(id as string)), '|') from range(10)" compareResultsAgainstVanillaSpark(sql, true, { _ => }) } + test("GLUTEN-8142 duplicated columns in group by") { + sql("create table test_8142 (day string, rtime int, uid string, owner string) using parquet") + sql("insert into test_8142 values ('2024-09-01', 123, 'user1', 'owner1')") + sql("insert into test_8142 values ('2024-09-01', 123, 'user1', 'owner1')") + sql("insert into test_8142 values ('2024-09-02', 567, 'user2', 'owner2')") + compareResultsAgainstVanillaSpark( + """ + |select days, rtime, uid, owner, day1 + |from ( + | select day1 as days, rtime, uid, owner, day1 + | from ( + | select distinct coalesce(day, "today") as day1, rtime, uid, owner + | from test_8142 where day = '2024-09-01' + | )) group by days, rtime, uid, owner, day1 + |""".stripMargin, + true, + { _ => } + ) + sql("drop table test_8142") + } } // scalastyle:off line.size.limit From 36f92a529450b247903ef57d07b7f243d31ee302 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 6 Dec 2024 18:41:26 +0800 Subject: [PATCH 207/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_12_06) (#8162) Upstream Velox's New Commits: 3dd572fe4 by Jimmy Lu, fix: Allow association with a delta file with no corresponding row in case of empty base file (11746) 37a5ffbb1 by Dark Knight, fix: Revert PR 1137 (11743) 627adacd7 by Xiaoxuan Meng, feat: Fix the task hanging under serialized execution mode (11747) e40259fc1 by Xiaoxuan Meng, feat: Add auto scale writer support (11702) 83cfd4f7c by Wenbin Lin, Revert D66724539: refactor: Use KeepAlive instead of Executor* 09fbd548e by Pedro Eugenio Rocha Pedreira, refactor: Use KeepAlive instead of Executor* (11732) 8b4663df4 by David Reveman, feat: Add key/value support to radix sort in breeze (11733) eb49cea62 by Hongze Zhang, test: A test case to guard against output order of LocalPartiton(kGather) in serial execution mode (11726) --- cpp/velox/substrait/SubstraitToVeloxPlan.cc | 1 + ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 1e1ab8971ef1..5870c4ef9f35 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -1076,6 +1076,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: return std::make_shared( nextPlanNodeId(), core::LocalPartitionNode::Type::kGather, + false, std::make_shared(), projectedChildren); } diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 5cd5716f79ed..65337932f412 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_12_05 +VELOX_BRANCH=2024_12_06 VELOX_HOME="" OS=`uname -s` From e61a2a8735c64440e9e9ce7c09849486a149a4db Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 9 Dec 2024 09:52:10 +0800 Subject: [PATCH 208/211] [GLUTEN-6887][VL] Daily Update Velox Version (2024_12_07) (#8171) Upstream Velox's New Commits: 2c5384ed4 by Jimmy Lu, fix: Incorrect result when hash probe dynamic filters push down through right join (11768) c11a46102 by Masha Basmanova, feat: Add Type::toSummaryString API (11776) 12ae85b56 by aditi-pandit, refactor: Change C style casts to C++ style (Part 3) (11686) 422fcee45 by zuyu, refactor(parquet): Int96 timestamp column reader (11719) e983aaccc by Deepak Majeti, refactor(parquet): Move arrow levelComparison to common (11711) 3ead2f4de by Xiaoxuan Meng, fix: Fix flaky TaskPauseTest.resumeFuture test (11773) 939c102f0 by Jialiang Tan, feat: Add priority based memory reclaim framework (11598) 7672dd541 by Yenda Li, Fix initialization order of kSpecialforms (11770) 54ad56dbc by Huameng (Michael) Jiang, refactor(nimble selective reader): Extract common deduplicated reader helper class (11766) 19533e433 by Jialiang Tan, fix: Make TopNRowNumber reclaimable when loading lazy (11764) 9226a863e by Huameng (Michael) Jiang, test(nimble selective reader): Add small read size tests for selective reader complex types (11767) 24b41e5fd by Jacob Wujciak-Jens, build(cmake): Clean up various issues (11751) 72054031f by Yenda Li, Add ipprefix cast operators for ipaddr [2/n] (11481) e7fbaea86 by Kevin Wilfong, fix: Throw on white space in percent encoded values in url_decode (11749) 71d1eca08 by Daniel Bauer, fix: Benchmark flag max_coalesced_distance_bytes type (11758) 99590e9d4 by Jialiang Tan, feat: Enable RowContainer column stats by default (11731) 28f43bf4f by Masha Basmanova, feat: Add MultiFragmentPlan::toSummaryString method (11756) d97bae061 by Masha Basmanova, feat: Add method to print plan summary (11750) --- cpp/velox/tests/MemoryManagerTest.cc | 3 ++- ep/build-velox/src/get_velox.sh | 2 +- ep/build-velox/src/modify_velox.patch | 9 ++++----- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cpp/velox/tests/MemoryManagerTest.cc b/cpp/velox/tests/MemoryManagerTest.cc index 74bf2c58ef82..85df9eed49e4 100644 --- a/cpp/velox/tests/MemoryManagerTest.cc +++ b/cpp/velox/tests/MemoryManagerTest.cc @@ -274,7 +274,8 @@ void MockSparkTaskMemoryManager::release(uint64_t bytes) { class MockMemoryReclaimer : public facebook::velox::memory::MemoryReclaimer { public: - explicit MockMemoryReclaimer(std::vector& buffs, int32_t size) : buffs_(buffs), size_(size) {} + explicit MockMemoryReclaimer(std::vector& buffs, int32_t size) + : facebook::velox::memory::MemoryReclaimer(0), buffs_(buffs), size_(size) {} bool reclaimableBytes(const memory::MemoryPool& pool, uint64_t& reclaimableBytes) const override { uint64_t total = 0; diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 65337932f412..e880db163505 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_12_06 +VELOX_BRANCH=2024_12_07 VELOX_HOME="" OS=`uname -s` diff --git a/ep/build-velox/src/modify_velox.patch b/ep/build-velox/src/modify_velox.patch index bf801ead83ef..6938deddef99 100644 --- a/ep/build-velox/src/modify_velox.patch +++ b/ep/build-velox/src/modify_velox.patch @@ -10,16 +10,15 @@ index ddd232050..b6ed2432b 100644 + FetchContent_MakeAvailable(simdjson) diff --git a/CMakeLists.txt b/CMakeLists.txt -index 7fd99b6dc..e7e03a800 100644 +index 5929473ae..e4e28c4f1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt -@@ -428,7 +428,7 @@ velox_resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) - # for reference. find_package(range-v3) +@@ -422,7 +422,7 @@ velox_resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) velox_set_source(gflags) + -velox_resolve_dependency(gflags COMPONENTS ${VELOX_GFLAGS_TYPE}) +velox_resolve_dependency(gflags) + if(NOT TARGET gflags::gflags) # This is a bit convoluted, but we want to be able to use gflags::gflags as a - # target even when velox is built as a subproject which uses - \ No newline at end of file From ebcba49cd9bd36c858459870d6b54556f2936c49 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Mon, 9 Dec 2024 13:51:08 +0800 Subject: [PATCH 209/211] [CORE] Add nativeFilters info for simpleString of scan (#8169) --- .../execution/BatchScanExecTransformer.scala | 10 ++++++++++ .../execution/FileSourceScanExecTransformer.scala | 15 +++++++++++++++ .../datasources/v2/AbstractBatchScanExec.scala | 9 +-------- 3 files changed, 26 insertions(+), 8 deletions(-) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala index 4f603e102443..d229117aa4e1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala @@ -27,6 +27,7 @@ import org.apache.gluten.utils.FileIndexUtil import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read.{InputPartition, Scan} import org.apache.spark.sql.execution.datasources.v2.{BatchScanExecShim, FileScan} @@ -169,4 +170,13 @@ abstract class BatchScanExecTransformerBase( case "ClickHouseScan" => ReadFileFormat.MergeTreeReadFormat case _ => ReadFileFormat.UnknownFormat } + + override def simpleString(maxFields: Int): String = { + val truncatedOutputString = truncatedString(output, "[", ", ", "]", maxFields) + val runtimeFiltersString = s"RuntimeFilters: ${runtimeFilters.mkString("[", ",", "]")}" + val nativeFiltersString = s"NativeFilters: ${filterExprs().mkString("[", ",", "]")}" + val result = s"$nodeName$truncatedOutputString ${scan.description()}" + + s" $runtimeFiltersString $nativeFiltersString" + redact(result) + } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala index d2f8237b6969..7f3c6d4f9f47 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala @@ -26,6 +26,7 @@ import org.apache.gluten.utils.FileIndexUtil import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, PlanExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.execution.FileSourceScanExecShim import org.apache.spark.sql.execution.datasources.HadoopFsRelation @@ -33,6 +34,8 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType import org.apache.spark.util.collection.BitSet +import org.apache.commons.lang3.StringUtils + case class FileSourceScanExecTransformer( @transient override val relation: HadoopFsRelation, override val output: Seq[Attribute], @@ -190,6 +193,18 @@ abstract class FileSourceScanExecTransformerBase( case "CSVFileFormat" => ReadFileFormat.TextReadFormat case _ => ReadFileFormat.UnknownFormat } + + override def simpleString(maxFields: Int): String = { + val metadataEntries = metadata.toSeq.sorted.map { + case (key, value) => + key + ": " + StringUtils.abbreviate(redact(value), maxMetadataValueLength) + } + val metadataStr = truncatedString(metadataEntries, " ", ", ", "", maxFields) + val nativeFiltersString = s"NativeFilters: ${filterExprs().mkString("[", ",", "]")}" + redact( + s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr" + + s" $nativeFiltersString") + } } object FileSourceScanExecTransformerBase { diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala index e3e9659ed33a..3313c3c76842 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, Partitioning, SinglePartition} -import org.apache.spark.sql.catalyst.util.{truncatedString, InternalRowComparableWrapper} +import org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.internal.SQLConf @@ -252,13 +252,6 @@ abstract class AbstractBatchScanExec( rdd } - override def simpleString(maxFields: Int): String = { - val truncatedOutputString = truncatedString(output, "[", ", ", "]", maxFields) - val runtimeFiltersString = s"RuntimeFilters: ${runtimeFilters.mkString("[", ",", "]")}" - val result = s"$nodeName$truncatedOutputString ${scan.description()} $runtimeFiltersString" - redact(result) - } - override def nodeName: String = { s"BatchScanTransformer ${table.name()}".trim } From 152be37c1669ef4b98e63e42b9653e23499ce633 Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Mon, 9 Dec 2024 14:08:34 +0800 Subject: [PATCH 210/211] [CORE][UNIFFLE] Bump uniffle 0.9.1 (#8166) * Bump uniffle 0.9.1 * address comment * address comment --- .github/workflows/velox_backend.yml | 23 ++++++++++------------- docs/get-started/Velox.md | 2 +- pom.xml | 2 +- tools/gluten-it/pom.xml | 2 +- 4 files changed, 13 insertions(+), 16 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index ce195c4e1b52..5feb5fa56855 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -491,6 +491,8 @@ jobs: fail-fast: false matrix: spark: [ "spark-3.2" ] + uniffle: [ "0.9.1" ] + hadoop: [ "2.8.5" ] runs-on: ubuntu-20.04 container: centos:8 steps: @@ -513,21 +515,16 @@ jobs: run: | yum update -y && yum install -y java-1.8.0-openjdk-devel wget git $SETUP install_maven - - name: Build for Uniffle 0.9.0 + - name: Install Uniffle ${{ matrix.uniffle }} run: | export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \ cd /opt && \ - git clone -b v0.9.0 https://github.com/apache/incubator-uniffle.git && \ - cd incubator-uniffle && \ - $MVN_CMD clean install -Phadoop2.8,spark3 -DskipTests - cd /opt && \ - ${WGET_CMD} https://archive.apache.org/dist/incubator/uniffle/0.9.0/apache-uniffle-0.9.0-incubating-bin.tar.gz && \ - tar xzf apache-uniffle-0.9.0-incubating-bin.tar.gz -C /opt/ && mv /opt/rss-0.9.0-hadoop2.8 /opt/uniffle && \ - ${WGET_CMD} https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz && \ - tar xzf hadoop-2.8.5.tar.gz -C /opt/ - rm -rf /opt/incubator-uniffle + ${WGET_CMD} https://archive.apache.org/dist/incubator/uniffle/${{ matrix.uniffle }}/apache-uniffle-${{ matrix.uniffle }}-incubating-bin.tar.gz && \ + mkdir /opt/uniffle && tar xzf apache-uniffle-${{ matrix.uniffle }}-incubating-bin.tar.gz -C /opt/uniffle --strip-components=1 && \ + ${WGET_CMD} https://archive.apache.org/dist/hadoop/common/hadoop-${{ matrix.hadoop }}/hadoop-${{ matrix.hadoop }}.tar.gz && \ + tar xzf hadoop-${{ matrix.hadoop }}.tar.gz -C /opt/ && \ cd /opt/uniffle && mkdir shuffle_data && \ - bash -c "echo -e 'XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-2.8.5' > ./bin/rss-env.sh" && \ + bash -c "echo -e 'XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-${{ matrix.hadoop }}' > ./bin/rss-env.sh" && \ bash -c "echo -e 'rss.coordinator.shuffle.nodes.max 1\nrss.rpc.server.port 19999' > ./conf/coordinator.conf" && \ bash -c "echo -e 'rss.server.app.expired.withoutHeartbeat 7200000\nrss.server.heartbeat.delay 3000\nrss.rpc.server.port 19997\nrss.rpc.server.type GRPC_NETTY\nrss.jetty.http.port 19996\nrss.server.netty.port 19995\nrss.storage.basePath /opt/uniffle/shuffle_data\nrss.storage.type MEMORY_LOCALFILE\nrss.coordinator.quorum localhost:19999\nrss.server.flush.thread.alive 10\nrss.server.single.buffer.flush.threshold 64m' > ./conf/server.conf" && \ bash ./bin/start-coordinator.sh && bash ./bin/start-shuffle-server.sh @@ -535,11 +532,11 @@ jobs: run: | cd $GITHUB_WORKSPACE/ && \ $MVN_CMD clean install -P${{ matrix.spark }} -Pbackends-velox -Puniffle -DskipTests - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle-0.9.0 + - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle-${{ matrix.uniffle }} run: | export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \ cd $GITHUB_WORKSPACE/tools/gluten-it && \ - $MVN_CMD clean install -Pspark-3.2 -Puniffle && \ + $MVN_CMD clean install -P${{ matrix.spark }} -Puniffle && \ GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ --local --preset=velox-with-uniffle --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index 3b3ab07a8dea..a0af0cc75d02 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -272,7 +272,7 @@ spark.dynamicAllocation.enabled false ## Uniffle support -Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.9.0`. +Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.9.1`. First refer to this URL(https://uniffle.apache.org/docs/intro) to get start with uniffle. diff --git a/pom.xml b/pom.xml index a2cccda1e45e..9b27a6839712 100644 --- a/pom.xml +++ b/pom.xml @@ -66,7 +66,7 @@ 2.4.0 24 0.5.2 - 0.9.0 + 0.9.1 15.0.0 15.0.0-gluten arrow-memory-unsafe diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 1973a7d99890..eb2c3d87e761 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -22,7 +22,7 @@ 2.12 3 0.3.2-incubating - 0.9.0 + 0.9.1 1.3.0-SNAPSHOT 32.0.1-jre 1.1 From 15f4cde02bf965e70bae5079c40aae78ad80e45a Mon Sep 17 00:00:00 2001 From: Kaifei Yi Date: Mon, 9 Dec 2024 14:24:23 +0800 Subject: [PATCH 211/211] [GLUTEN-8115][CORE] Refine the BuildSideRelation transform to support all scenarios (#8116) --- .../apache/gluten/utils/PlanNodesUtil.scala | 49 +++++++++---------- .../joins/ClickHouseBuildSideRelation.scala | 17 +++++-- .../velox/VeloxSparkPlanExecApi.scala | 2 +- .../spark/sql/execution/BroadcastUtils.scala | 6 ++- .../execution/ColumnarBuildSideRelation.scala | 38 ++++++++------ .../ColumnarSubqueryBroadcastExec.scala | 17 ++++++- .../execution/joins/BuildSideRelation.scala | 11 ++++- 7 files changed, 90 insertions(+), 50 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala index 9dcb7ee3c41d..d6511f7a4a29 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala @@ -22,13 +22,17 @@ import org.apache.gluten.substrait.expression.ExpressionNode import org.apache.gluten.substrait.plan.{PlanBuilder, PlanNode} import org.apache.gluten.substrait.rel.RelBuilder -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Expression} import com.google.common.collect.Lists +import java.util + +import scala.collection.JavaConverters._ + object PlanNodesUtil { - def genProjectionsPlanNode(key: Expression, output: Seq[Attribute]): PlanNode = { + def genProjectionsPlanNode(key: Seq[Expression], output: Seq[Attribute]): PlanNode = { val context = new SubstraitContext var operatorId = context.nextOperatorId("ClickHouseBuildSideRelationReadIter") @@ -36,41 +40,36 @@ object PlanNodesUtil { val nameList = ConverterUtils.collectAttributeNamesWithExprId(output) val readRel = RelBuilder.makeReadRelForInputIterator(typeList, nameList, context, operatorId) - // replace attribute to BoundRefernce according to the output - val newBoundRefKey = key.transformDown { - case expression: AttributeReference => - val columnInOutput = output.zipWithIndex.filter { - p: (Attribute, Int) => p._1.exprId == expression.exprId || p._1.name == expression.name - } - if (columnInOutput.isEmpty) { - throw new IllegalStateException( - s"Key $expression not found from build side relation output: $output") - } - if (columnInOutput.size != 1) { - throw new IllegalStateException( - s"More than one key $expression found from build side relation output: $output") - } - val boundReference = columnInOutput.head - BoundReference(boundReference._2, boundReference._1.dataType, boundReference._1.nullable) - case other => other - } - // project operatorId = context.nextOperatorId("ClickHouseBuildSideRelationProjection") val args = context.registeredFunction val columnarProjExpr = ExpressionConverter - .replaceWithExpressionTransformer(newBoundRefKey, attributeSeq = output) + .replaceWithExpressionTransformer(key, attributeSeq = output) val projExprNodeList = new java.util.ArrayList[ExpressionNode]() - projExprNodeList.add(columnarProjExpr.doTransform(args)) + columnarProjExpr.foreach(e => projExprNodeList.add(e.doTransform(args))) PlanBuilder.makePlan( context, Lists.newArrayList( RelBuilder.makeProjectRel(readRel, projExprNodeList, context, operatorId, output.size)), - Lists.newArrayList( - ConverterUtils.genColumnNameWithExprId(ConverterUtils.getAttrFromExpr(key))) + Lists.newArrayList(genColumnNameWithExprId(key, output)) ) } + + private def genColumnNameWithExprId( + key: Seq[Expression], + output: Seq[Attribute]): util.List[String] = { + key + .map { + k => + val reference = k.collectFirst { case BoundReference(ordinal, _, _) => output(ordinal) } + assert(reference.isDefined) + reference.get + } + .map(ConverterUtils.genColumnNameWithExprId) + .toList + .asJava + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala index 92887f16d70a..668525ba0a40 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala @@ -22,8 +22,8 @@ import org.apache.gluten.vectorized._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, IdentityBroadcastMode} import org.apache.spark.sql.execution.utils.CHExecUtil import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.CHShuffleReadStreamFactory @@ -72,7 +72,7 @@ case class ClickHouseBuildSideRelation( } /** - * Transform columnar broadcast value to Array[InternalRow] by key and distinct. + * Transform columnar broadcast value to Array[InternalRow] by key. * * @return */ @@ -80,10 +80,18 @@ case class ClickHouseBuildSideRelation( // native block reader val blockReader = new CHStreamReader(CHShuffleReadStreamFactory.create(batches, true)) val broadCastIter: Iterator[ColumnarBatch] = IteratorUtil.createBatchIterator(blockReader) + + val transformProjections = mode match { + case HashedRelationBroadcastMode(k, _) => k + case IdentityBroadcastMode => output + } + // Expression compute, return block iterator val expressionEval = new SimpleExpressionEval( new ColumnarNativeIterator(broadCastIter.asJava), - PlanNodesUtil.genProjectionsPlanNode(key, output)) + PlanNodesUtil.genProjectionsPlanNode(transformProjections, output)) + + val proj = UnsafeProjection.create(Seq(key)) try { // convert columnar to row @@ -95,6 +103,7 @@ case class ClickHouseBuildSideRelation( } else { CHExecUtil .getRowIterFromSparkRowInfo(block, batch.numColumns(), batch.numRows()) + .map(proj) .map(row => row.copy()) } }.toArray diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index 2df2e2718eaa..d837ac423407 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -632,7 +632,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { } numOutputRows += serialized.map(_.getNumRows).sum dataSize += rawSize - ColumnarBuildSideRelation(child.output, serialized.map(_.getSerialized)) + ColumnarBuildSideRelation(child.output, serialized.map(_.getSerialized), mode) } override def doCanonicalizeForBroadcastMode(mode: BroadcastMode): BroadcastMode = { diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala index 11a8cc980904..c5323d4f8d50 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala @@ -106,7 +106,8 @@ object BroadcastUtils { } ColumnarBuildSideRelation( SparkShimLoader.getSparkShims.attributesFromStruct(schema), - serialized) + serialized, + mode) } // Rebroadcast Velox relation. context.broadcast(toRelation).asInstanceOf[Broadcast[T]] @@ -124,7 +125,8 @@ object BroadcastUtils { } ColumnarBuildSideRelation( SparkShimLoader.getSparkShims.attributesFromStruct(schema), - serialized) + serialized, + mode) } // Rebroadcast Velox relation. context.broadcast(toRelation).asInstanceOf[Broadcast[T]] diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala index fa3d348967d5..977357990c43 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala @@ -26,8 +26,11 @@ import org.apache.gluten.utils.ArrowAbiUtil import org.apache.gluten.vectorized.{ColumnarBatchSerializerJniWrapper, NativeColumnarToRowJniWrapper} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.catalyst.plans.physical.IdentityBroadcastMode import org.apache.spark.sql.execution.joins.BuildSideRelation +import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.utils.SparkArrowUtil import org.apache.spark.sql.vectorized.ColumnarBatch @@ -37,9 +40,19 @@ import org.apache.arrow.c.ArrowSchema import scala.collection.JavaConverters.asScalaIteratorConverter -case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Array[Byte]]) +case class ColumnarBuildSideRelation( + output: Seq[Attribute], + batches: Array[Array[Byte]], + mode: BroadcastMode) extends BuildSideRelation { + private def transformProjection: UnsafeProjection = { + mode match { + case HashedRelationBroadcastMode(k, _) => UnsafeProjection.create(k) + case IdentityBroadcastMode => UnsafeProjection.create(output, output) + } + } + override def deserialized: Iterator[ColumnarBatch] = { val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#deserialized") @@ -84,8 +97,11 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra override def asReadOnlyCopy(): ColumnarBuildSideRelation = this /** - * Transform columnar broadcast value to Array[InternalRow] by key and distinct. NOTE: This method - * was called in Spark Driver, should manage resources carefully. + * Transform columnar broadcast value to Array[InternalRow] by key. + * + * NOTE: + * - This method was called in Spark Driver, should manage resources carefully. + * - The "key" must be already been bound reference. */ override def transform(key: Expression): Array[InternalRow] = TaskResources.runUnsafe { val runtime = @@ -106,17 +122,7 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra var closed = false - val exprIds = output.map(_.exprId) - val projExpr = key.transformDown { - case attr: AttributeReference if !exprIds.contains(attr.exprId) => - val i = output.count(_.name == attr.name) - if (i != 1) { - throw new IllegalArgumentException(s"Only one attr with the same name is supported: $key") - } else { - output.find(_.name == attr.name).get - } - } - val proj = UnsafeProjection.create(Seq(projExpr), output) + val proj = UnsafeProjection.create(Seq(key)) // Convert columnar to Row. val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime) @@ -178,7 +184,7 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra rowId += 1 row } - }.map(proj).map(_.copy()) + }.map(transformProjection).map(proj).map(_.copy()) } } } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala index 6275fbb3aa3c..12280cc42aed 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.joins.{BuildSideRelation, HashedRelation, HashJoin, LongHashedRelation} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.types.IntegralType import org.apache.spark.util.ThreadUtils import scala.concurrent.Future @@ -64,6 +65,14 @@ case class ColumnarSubqueryBroadcastExec( copy(name = "native-dpp", buildKeys = keys, child = child.canonicalized) } + // Copy from org.apache.spark.sql.execution.joins.HashJoin#canRewriteAsLongType + // we should keep consistent with it to identify the LongHashRelation. + private def canRewriteAsLongType(keys: Seq[Expression]): Boolean = { + // TODO: support BooleanType, DateType and TimestampType + keys.forall(_.dataType.isInstanceOf[IntegralType]) && + keys.map(_.dataType.defaultSize).sum <= 8 + } + @transient private lazy val relationFuture: Future[Array[InternalRow]] = { // relationFuture is used in "doExecute". Therefore we can get the execution id correctly here. @@ -78,7 +87,13 @@ case class ColumnarSubqueryBroadcastExec( relation match { case b: BuildSideRelation => // Transform columnar broadcast value to Array[InternalRow] by key. - b.transform(buildKeys(index)).distinct + if (canRewriteAsLongType(buildKeys)) { + b.transform(HashJoin.extractKeyExprAt(buildKeys, index)).distinct + } else { + b.transform( + BoundReference(index, buildKeys(index).dataType, buildKeys(index).nullable)) + .distinct + } case h: HashedRelation => val (iter, expr) = if (h.isInstanceOf[LongHashedRelation]) { (h.keys(), HashJoin.extractKeyExprAt(buildKeys, index)) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala index 60f3e2ffd966..e9dbeb560c68 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.vectorized.ColumnarBatch trait BuildSideRelation extends Serializable { @@ -26,11 +27,19 @@ trait BuildSideRelation extends Serializable { def deserialized: Iterator[ColumnarBatch] /** - * Transform columnar broadcasted value to Array[InternalRow] by key and distinct. + * Transform columnar broadcasted value to Array[InternalRow] by key. * @return */ def transform(key: Expression): Array[InternalRow] /** Returns a read-only copy of this, to be safely used in current thread. */ def asReadOnlyCopy(): BuildSideRelation + + /** + * The broadcast mode that is associated with this relation in Gluten allows for direct + * broadcasting of the original relation, so transforming a relation has a post-processing nature. + * + * Post-processed relation transforms can use this mode to obtain the desired format. + */ + val mode: BroadcastMode }

    nOmHR#KsMc>Q%L&rM^omP2{8{Z zFE7WOj-i&my1HqJrn}|r5uBIYJYG#hzs2se#An(Rd3wjy_T`y*n%{%h|D|5q(^Kk( zZA)xRUcS3jzEmOQT4g&GLZk>Jth!F{Lp|jz%kKpT3j~+0ZoY2&uF722(G*9BgJnyc z!U9lYVq&k!6@(w>R#~a2s7TmLAOMVorG?|y( z(HAjw%$9u#=VU@hgJcgR#y%4j$7DmJrWI(chou~MgDQT`?&4mC_4fKqJRBSeD@E~a z3JCRyrjE{HeTIgI$d?Z-^Pr5bAy339kpleVF3;jgfk^z=%?Lq{)3t|u;cwN|)rh3V zLnw31cTAEHE2Pz$EV{qHuWx8*sI|VjwpXv^WDBXZxRjKF!e@Kh7@;iM?2q;y=aOB% z`SC+fQ)hvL3#?PG6rBv{L z(1zOmm2mcUw9Y1Ia|2LqVlXrzQMb*)US{!1QeI&3)6YvcBm{LTa=)jB^h${?IB3!R z*vGtoM=jZu9Hv=*!Ps*u4a^7<2@vYi+DBJd}>!-cvU>x-mZ z%62~9KP|rcQ{3Y|7naWl5Z)0)F4UU8U9)Q3u1<>4mt*FD|2`jq&>UR4J2C5hYFe9M z$ZWlvmvJ-}pu|awpU8tTs!7uJICRj5hD%YEKGX-ykhG!R>iPUu5#~qEK z%VEl;x|;WMnFX;wy3BSLhM0>fy$`D{EJfPfw$;TMQ&PUGg7S%`qeaf9Z3{*G?$4W? z`1#3{7Q%Yb4i}ngtcMF!aN78|=4M-S2l==JzkjdyaXZ{RK3>_EQn2}+)_Tlcb1{s1 zE6qfKg{zjAZDr%bTBJ^kJ3CD`7FpA7B5o$3mJpf)7gOmfh|0? zwb9Db$D`!7|M-D0lafbmA>QfVstq>1LzpBL_+$#n@8Q?iD-;x*FYM?DR)|gx4#@u= zsb+h~2ZMqUB|O>Qffe~7W{L$D#IgW+GA#3iB%$>1iyepA^bjOBDCotD7nnA-T-01l z;%X-AVTr@{%`N3d$J|Qw@l1Glyst^vtqoKc-P#b+Xm*3SO25%!ZZ^xZgeLIB#&)<( z%Lm8ksHm670Yybj&( zFTe2j_@HBI9n#eoN#1Z|C4vY5r>EJ;i$7aiA36hGFsNh-xgS#;^Nx1F7(eFybm|Vd zLbRoOz8EtgRI+v63Y)C15I)=b5&en|k&H!!jrjpRn{P+T+r+3&e_fy!Gnd?byMul? z)u~EJ3s@nz*@}9TyeMqgV9#aGg|g_C#Q48jfUXe{bSgwQuj&xp%tdt50eT|1y0p}Kzf-nasopIS zBq0%0pi9%%F#E2g#IRB|qm2mXTTsMGIeeuY#~)D7NQgzkncGs*)>32mnK3w9su$5c zCGr^`ufw2=-ErpN^;c1kqp!GWT;N;x#{-uU0(_)y*Lf+LoZ+HG2szvSCCiE57Vr|9%&;K0%Js zTiK&5m8*8Lv&KrMy|z$XEbH%R*inY7$nQzTbf#qt!AU{N1S@64f~+{V<-#+kz3e>t%!r&JTfOQh6Q zI~XAuDEXd~lT$#zI!NrL8en4w#kKknuXE$U2T1>!OLM`Taih7pS=PT|*+z?*wP)pM z^)t`wkgkZ(v-Qv_wZePeR0|GGW8>8cx3Z3gsU$HFUdc*zdt<4qGp(Aj@#5_A^n}}T zAocNA>mQD7g*;{pQCI7oW|fBiy%SZ<^=5AkNGtC4VBx`FJag zS8Bi9pLm)2e_^m+*GJ$Fi1Zk)~i{reL}`r{NKy}$En-W>e%Te7sYxS+PS7HakT zy^=*HDR~A3rG|s%LgRng+Kc?&!^!>Z&>#;pPFmP~|zgY&X zw0*^%INze{;M;50FDNKbSI6CO;KNhEAfRULdb%ihB-8KmeeUT)G4R+)mgM@Rqa(m3 z#HJd;IhvxzuyfSlb(S1dtA*;p81r17w}*~-3=};8);Y-9#%8WUd#hsPGoVGWv#YFU z46C0CQ`K5QPt(2Zi+6q{CM9vKS_FazzNsO-r`Mwn4|n1@S31wJ(z4Z>Q$g84a?r^m zI1aCIIHh7f#qOP(?Sf+`pIgdzzOiAiwRVrwQC`k9#e`Z;`^`c6ro^WW8jWo!6sb={ zGzye3Unt)NW{82QZ%E#7z3ot&>DYVzG_w*S4GWR|n7{e({K>%ex>9#9+v}v#OF0X! zJ6#m}t6b7vv3^|MTO8X@`rIsg7q@ zayS_N{;U~mQ#RCz89*0YD}D|C;|tgH46u%ql9Ryw&k>jrdJ>!Q8+kyerv(*u{j26@ zKo&?qbaLA6+2=WQ{#E(*Mj52~#Z4c0+0V>wU+m6^NVL41J~RbSl>8hEDWBj>|lK#J{Y6>ytXS5I535H&Y>M zh=qKlf|@$6+nLg%>sH0o*5!9ijoZ&{rUdW|^&%LQ`P79!e-daZMn!W3%67`8fe5t0 z^5*7h)btU%GhAjf5$UbTjFtXDVf*Lqw0~si=;-DutTO4-3EsXMp!IWiAoy>q z&gOjsivz9DU;_;~@Ck{CbhQ}<(2PZyH`^8kN|6YXP0-nTBob&;3)0ZJUBI#z{ZY*B7eEPo%`CA zXtvVlc0^lMwt7xM@^=vI_8?_GP~=@amE5@$j3qKO44M0$b4=86G91+B_+J!I;S>=+&jMn>X^Ajk> zT~^%p1}1!%y)IEMG-j$w7+9yc-uQbVz%$RR_P7=v2zq(~!afk^D^>}&*62F4$lL2l@>aim2==`<1h}P^nDVe$ia3Igyu?{gd@bPQSk& zR2FHsF+~7TtN)Q~a6Oz&EMDb)lZ9{JzrQKbvwY8q75U(ulHD^IBzCdp;6QE$ zq-Y2ZVjEGwU+Z#j@tV6C?Q*@Hjj0+_kZ7IyHN_7diOxzdi>v&WfJ0aeBZo?>FpPno zwlKZM=Y1!y*;R-UuNY5QK+mc`4la%I4~u%IFbToKH%g=lnMhMEfnJEA0PjI z>U=ZM7TaGIZdiED$ms)WQYv8r5*daT91K(HhK~KILYK%(?;a^5GxKrRDl<+r`_3?` ztqhv$O}g1)rY#wWjf#BN6z<8_Phtulgi%6X!&}b5xZX8B0S5}pG|=MX=C)3!^+ygH zM8wj_HaOg2OoIssi<44TzTKXVs;mKzM~8vhy{~oL#@qqSy!DN)^g~;#*8}PqpA=ev zcDiPj9^u$vRRxnlm*a0kKLcebh)veiX#X@zSVD5FS{uTFH^V%m>-$BY97C(iijxy% z9PwA8Y9h&54*wcVHzXmX-QoQC<>_fC{jZX)8|tjJ;_grOuQOui{WPI_L|N#)_8latdGp?gC-^uwKgMndh>4mLn=SQWCvxxGltk9Q?AeZ69rzkjX1kIHtFynx$+LF zzwQa>+)9`8wI*NJd^7=;89k1Z&M{CxUHQ7!G2(qR&rZ=xsjlP9+SJ$xGXS6iI+bw2 zv$Gn*#*t}c?&|@!Yksd;UsttFmns+2Gcpnxw`YN^*u-O@5@>dQ*~q|-y0u`D)A5?! z2D5%vy;e7O8QIV{61=>;Oh@Vm2KnBMbOE@+=nU5izZ@=8@pNn>*HX;V!U9nG0><5j z8Au4gU1BPAfiG`+d_X`zNF;IG8A^_cK^sP6!HdjN`jM0tc}4CI3M&C2*R*+gAn~AX z0k3m=(q;1>O8)%Tc~Z(^UXjaZGBPsO@(Kapr=LdqU*)UYv>>}DEp=;}+A0CJW7At~ zs&$l|tL$Oasm-g+%gdC@f20Y5rwvTXrM;x1Cb>x@$C=uQQ-5{O_XU8I zmk?o8e{_49wUX-MVhmdBXk0$uR&^RyYnxZ~^&-@#`B;{VK@oEJ?)C`K3J=e2+3i^r z(U~}MHs{)UKLtv)h^hO3b&9ZXDbbs7*cc)Y2&T4(gLqQMQ@Vl%D=4wpXdA5!?+aC| zn;J}|=Um@W2(pS2pmKmPcXmws3OI|b-ZQ9T@SPe1*uMMj2;t1ocBEQY>lY5L#kDp2 z4wVl5(Y@QN+izN6^1_1Ayf9e3d)t}3NxVPqsCCXP9zC1(%n^C)(UO*@v;1>!Z#!29Q#)ofHgi?L75Y44A;QVWCo74Ii(6P&80LPy*{5pp zfwM>m+rA_qz5L1AMi5QxFFlL{C+8$S_ip7tM@QlPS%qHi{hsIi zvNF-u%B1b_rncMrzLbptdt1E_5J6%79!#W*ZD<@*S7A!<9WHX>K?J=lt@+;mxu!nC zrivfP(=_k`R1 z6X$P~G9EmuGKvUi&1BZ57rrX5)muOn^-hThw40k8)zt^9eeW}q5(XK&b=$0uXGwm2 zbg0YAGg5M$7>2$sn#MxZ!85bmfJn^Clwopye*UBC!_y^#tYk6O=Jx%Vu$PEAfjsmY zO$iViK6YGc=CeCqT@|*Sv<}BDUXQVyL7blD z_Ig#ssj#v|JUZ(=zLxnKkUx%uREfFyuKZn$kH>@*mi_2vLsjKKF##||iLF1TUf+s) zhh}VOLm#<07~~_~d{#Dx)RE)_SRcM^vRxB4Z7zRW*h9|`Qw}ZKJX+{{JaOI|t0#ma zxziSJf|HNhH)mXDOny^E$*ao+d?9EJu6UYkEGGi|9E5W<@$lt^hbXaPiutC#R!V%b z8|%%_zjo>W{B^}lzOsCHGYSuams`RgVj#ag5=|km;g3V@x)f8nk@nG6Ku<5OzS&IE z)Z5VAw3j-X5NTpMIxjD8u?+b)SMl#!05og2^^@tJ`4Rllx~O)IBc)GI$og+L>Oh2y zERSN|p@teCn&~|T1qXJ0&+CFlIu?e~yLVVKm&o#w5k1{KY*FJ^BWLrbW@gcBjsjeA z^lGDmLZm9VmAPzd=|aw6UqHC)Kf=k$SqMjymmp_eO5{ZQ=RQPgB8@FzU=Xc_t1zjl z)XF%)WwDHkHU%;;$(3IOV`+ulQ3R4mJ$KLi1?!Kco&(iXi6hERlg?eP0I{J^K!||y zC~)JtysOZn)Q=V55_-$K34Ctxu0=g28kM_ZL7 zR#IYvP(vs=d1mmCv8;SE-ja_vJct!<%0_dB-C6P5MO}hFlqfwjiQ`ficR@ZRvniM= zRy0mi3kyYA_bGl-Z)kGlzv2BxPoE4bZ<;#Qn@}TSVb6xn&KCgeR8TplG4cR&Tq=A+ zdxQ3do0s?de}?YsALCkJ8vq7Uix(IQe~-SD49OMrEjBbXHfm?2G?}=i2Ldi2c=-6R z-E*Bc+~zN1fGY`}vWKp21};*`@x0^SwJxuBeaZ7pbC;df(4_G;MSjh*Ps;keO)U#A z?`?R$Xii3k&F7KfVXw__I^a%+O5}}mOm?+jgSrJz+kxU|9;oyAW{L2a*ZxG?*@TIu zE*nTViTNDXvC%~w=01OI^ggNrYO|n$;o&JnliK;b@A?P(`(nM*e+c6F>qCx4qtl5f zmD)x}fbv92!R__o91y*QhXK%OMs_xQaBVH7gR_-4SA4@1zsvsk+1Wq-NZul3)DUuh zzaf6%;SrIpJz(JvBi*@Dd_(Z^yI&3n$oo{ZJWJKZGT3svq3rizpVO}YgX>KUndq#v zwCU~G=f{)590Ee2v*m!XF&k9vrY40U;M~K--ByB^F%WS%dn!Z=@}7-iE?lm+k;yIL zAVBTBnpF7CY5=)nXOxK{jr?;Zyu{ORv|#j?lY~TbiB%s)f>_X(0gw0Q1Sk-C6;n7c z@CgXx5Ir+HpE_0Qb0Tm$5g-Op_R7b(@l_UCJ# z#5YQI*pCe)KE+(OtKUSQ`7vWjg?%|}V~$rF>l@DWrT!L3bg+_mGGomV;`YN9COOR= zxlx!#$3zo*?p!~UEwz~q^YCn)%bv+=#zK%GU@x~51{Q{-kpc-H1hfv~+0pj=?yAp% z!^oF7ghV8NMO^liwU~Nxbuz6M(li$L7zJoA4Evi>61@@x58&snYAjn(ifJYRgra;gya(_As z20{^%1~Fbw3b>PTXSf-Eu>I+m4wd=y(MkE#6~j}>OzJp7y%qDYWtLl5INDGZ z8xg`FZ_{38Oz=n2p`xu^m6t#r6$0L&nGsRqMm6y^ZrIcL{sj?xG!h?LU0QCX(V7wKTVcaYDJw-H&#^ zqNKTX${w%Qxi6Y;pB?`JCqY-uS3!~cn8jwjZ{;~uxi~s)p3(74oXCy%N z{hJsobTL2$^6zCjU;h;Z?Q(ZuhqV2K%`3A8st86#6qGTsvCu5h#G0<}Vv`mh8JGxV zb14IGFd(#brqmSbxk>mGwUTil>Um`b-}A_vY{VF1&jU72x;LGu(=N)Zi3WG={NDGC z-}S@t4C16!{IlTHVaQ?L!_{!ugZz_q0>o>h>ap*OLXc5|Y0E*mbw(b$$g&=>l=*R3 zn7An*>R%TMjg(RBwGHqErqP(hopF;4_W58j#2#zSKp}@!6vdH&U|Q?Yl*HQ4;2Fud z_weR$1mYZlAm%|QMWMOYu@Q*n4z-+<7zc&Z;uLl!%=xWFG=!MUm6q?vvh?cApp>q&Q`n|-UI>^p zuwY$RWXU#W5L%h3v04$<4<^lRjQ`Z0{*!ioMYLV&`0?K8yu3rLcYi_u)7WK(zTn#; zkH&gV?eFR{-sn063My92;f9ji3u(NW49TV;Y02D?f4+8Jk`!B1fday;7CSPkyyNdM znqg4V{xZdv_2KAyXAK%pSOWJ}W8*adN4O0Dnq@1Ii$P`36oA4TXhjs+5}I)LWd0Z& zgoh9MrFxnT?}d&8kdHzBGNc~8D#s*z4j_^C)KC_VKrJkV*d*j($(INZjBnuz_;grZ zb@p1WlQHjL6yS3GkN+T&`>Q^P+we5yr^0f<`ap% z-ofkxrYDh#T!M!aMwr#vA-k=?Ld$XdDFw~o{dODn*fyjl1ujtcYy~?wjA)q+;YK1p zjd0J$-h4JlFd)302IaevIV1=MB%7aU-%Xnp0weZ5Vo&29j;ZsrcN_);JV?#)$rV=P z-+bdlaelFb=G`^N1Fm^{>iNwAzRJV9V&;TnoCZVhjuyvX)I|yKIyYYw#=RF(NHaAo zXy+$-Ny<8cZaVG+&Q;!f0^8({X)ISoo1~$#AT1!{+a8QbNrxJ=vGHNlzq#Y zb4Aep9)0_1_t8^(`(ESZv*|MuHQX!OPO(?E9?aN;24`T^N41Hbbh?W84?_r7zK;J{ z_CRWbo6eb(mMiPqubHs;RMwh-4qV18KD@ z`Zm}~_&PUu%^vr182)YXY_BD^cXj)aw)VZ!3b_VXe(h%218qvJI1$VBH*~RC=qRqY^*7XN4bj?6<3wegty<(&St* zSgB)f%;@&thrX#9%(sVZ?`!qer|9_do0@w(Jp2|g0FRx59PgN67W&IdQxz^sb784Jk<}q01rQcVF zZXTYxP&@nKWA40uNAHNDg4dJrepj*^bGB&deOh=t=*J98xQj(pQpQuyRe;GS7hfD_ zH)!EKtbL!16ugKhhODBztomB4^xDyJ+wQrtrws+wrD(ke+fSS?9?0- z=9HprA}>l`X$QsP6>a|c_C@Sg2MKP?>0tLHD%#k5d9b0ye`i0pffc@4aRMRGudPfN zbcS5dhS&D>dg#r7RD75uMi|y0zZ~so32@bkN%}AKP%gbq#?lC8TZ&`b=(k^>#n!Er z7k$`xni!%_7}{XdEdARP5=pdKz+oCDo?Ilu3l0{kMzAjH_d=jdXNc=1Id=bdrQVIc zmp;z!&`QIF6--=Z@Srit3l?^s=aTb?H5Hg>-H?z930Rz25H*Z;oyGU-S}aXTNAX`98uvNkQg zRg6OK7?qrC<-pXmMXyE^pP+@FiepNNyxfbTH#pgikk&>*5j5z7G91lyt8Q!4>YW-4g*4xw0taP7P4Rdi#d>eL9CjigbNB#Olj{+6HgwG zOH{{*b`J?F(Rp;)o*o!Tt;#SZ>|tkTqmch__0o>DFT^tC&E3`{c{SO$%)KVm-ayA< z!-xkZ+cRjMjM&*6^wz4nIB|6B2*tXWDBde@bs78obr%zaVcL@4goDq$vjb72>JOXS z+11a^j9`@4i?7#gIhha!>}mH&R|>pbVtJj_#4yQ;HB*Y_>f^-QEJEZHkTP(>*$Bhs z$dj7*p_wzWB;>*^!m^a-tV=RA?@#!dud&kGlMJd!)bBNJiO6aE`@SodJJ5v?zPQes zVDdG4FFF!h>QBFH#Auzi_6t6z%Dudfk{q}69k>bTB6M73-C*5?IyG%%ZwX)&-;_7= zac&1wnK72b))J}@5zctkblqabRo%f9(kej$+%y~wL6U|wokO`>@}ULTDBtx)*gwS! z$I?|{(;jsaQd5E`=X*yf*LQP%R#a*U`DOaS%+J;}zbq>X$?a`Ww$!pIXHp*U*(uc~lMIGJ2D9)vR=WNzZ6E_n z(cAdHa|KKs%YkgJEJoTZ+`!lvp1;CHNt@%cFnnezGKv}u;=;|o+z~NTb9K_U>A-sd zRbCD%7+p6~)WPKtl7^>WFZV5?dzExk^;om~sGuZEuU#*TJtMtTx-@7S zvqf`8%dK0EMgC~GV8|Hyp-^qx*8x~v3HBgwO3M15K3CKNC|G{ay2q?^c+NiOmr<8{ zfm=N{4#5k&Jx=Gu@yGz)bJ7$V=)MAPRIn93rx`KwA!?7*aY97BlSJB+k1OZ zid~KPlOek$Z9$m;^$}7y!qd9DKVMr>+~H*n6{%BzuvUSp0d=4^@tzu)c#cG9;XBR%Smi_{E$ zJnh)O$qXfu&4PraB=x5e{ErIi>LBuz%(HbpR1V8q>N>k?qh|=mgh^aV3%G&Tws*Dz ztrZ4@Rl808>`7^wLe&{wuee)dK(-xmFQJ`fqw#y<1Y&jua#nGqsk$5Bun`T(Ct~A>W$i) ze?HTD;o~+Ndl)o&2Te8q<*JZ+E!n0K9;PxzrH1xG$CgthDB&S#x+LG zU2wrY`mGq?Q@ z(8J<8WytJR!L19VM}Ihg`spmv6qcG7DAzOOT^xtGVFMJCnw~=mosaQ`&B0iBTMtbR zB59h|DQW!PoL0~;zYI4k2KorG_!I{vk z>u^g){?SBk>SW9qA|(HTI{A_D))1Sqk?0o9-3z(}fpSz9Qvy*|bzYc1+e9k|yP0xd ze;-Io&nDlysDxi6artKI{yvt`DT#Jr`eiw`G2tP&Uh^%>>_e)6vI8q-3+0L-9+QG{ zIZvkgo*ut}q6Ej)U1rXf9}-i8EE#6hG`4qeK?48+z8c{BTT!Z7p`2ekJTY`fzeWcy1WWAV zNrUbK)9|fBxa+hvgd0*7+;EzR)*Bz!4?OIKE_hY`B@)J`@+-d81ff<5`>k$tO- zymYJ0C)D?Y+)s+qvE&uLFAJrh|Bb3xzF<$&;<-w}y=hsf zob!4RNAAm@$i`e;AmXGvxMjD%wEB!7TcKT7gFx0d+0q>@l+J39_}?(Qr;h|O#Chc{tWofB zcxaJ{0+%ApzQI~tjwt|+U-4!fSsD$?oNgxTnhhTOk3iOkx$T+sjW}iM?8M&8&GCZk z+R7tm^~xtoS+HtiE5hd6ezjR#s~>`gt8clK9%pl2&NXwMCiB}(RLn3wQS_>F@3?aB zC^&hWkbe|@ygAmbuUj0ZU2M@fZCCn^f=>pM3qWVlCU5({2*s{knaqFGp8HW%cS3v` zb67-Tjt&yXQjmH(TX9WcPAerTJt@=~aWNUtxw?~Mfk9(f<{KV`)USfq6k&8Dff5-B z(96PA9yrXi?U$m$qRMv9HIu%hUAa2<@I(hd47Ihj)zy>1ioAzr0-}B?#V;Z8Rg3kj zDdEk3-{hn**{{@z{Xiiq5$hSA_jUA}Ppu|9Wt*z^a7sR_u=hfCsv9N{1DCC8J2m=e zayoyCch>XQV`F5@?br6 z4aC3bp%q~8a~|Vkmzz@OS3kd;9?v()C99bIrrU%RiAB7<08Hrp^vDRf#2OPos{*MW z?k?V*p2(RRZ>-KQV0P1mWIqZc`|Y5_y6Sb#uT1ULbU82Lj+&VCQWM701d_vdd+>6m z8S}H|zi4Y&-xqGWO>L!oePj$-paK)(s&yFnZmK?}m%aS$dfNL%NCm&DTR;bQNF;q7 zf$Im55L5QVXQC*={2nHhSNR+@5}KQ4J+fX8WJEnKd*K4qR5Bm$`$ZaCO2Z|eM#8C} zBCkE}#3?+Yrh(kfMOJ$pJ033EACj4nl%^Tvhq<=b9Yon9GSc|>M*A7!6|ZnoQcB7D zr4a}lU5T12wd-|Y0?hBuh^m!BqLak&Ffc1C${88o5fYL{^;5&fzXzxR1f+yV?VilS zQCT<%4T=(|Xy{nZ3BJ^3t`a8M`2F7yOm;V!vGDuI#`L<(CSXBv?l8SC#PTR?PGyO)pzLS=rEbcr+jX=;j7oy#7;+d+8eGa z*QaEY3!1`7QFPN_lwV{ z*F#&Co55icL_qJI2^M#1|0etLJ$Ehf4j*NUa) zqxt)|oX2Ggl#9!*TXa#xQAq+d)3lu9*WdO}FL!GDkFR&iVf8nuax>Xw>tmI=wwy4u zY4gL2udw6;$4gBO2JNQEOQM?c9_!r!&3@1DFuu0^yDM*!p+e7zgFmOWiQW<=ia{oI zf-zx%@+>xYcy3)@&2gM@{#>hZ+9mbV;t|&`zn=yU!Pvw_KjR&XGQ3D%@ z5ln+6A+*#xFOR1yt-c+2WNK!p^NGtNIup#{A>5y3sgY)xjIb2KL1Mk+0g5UhvE;z+ zhLgs8BOQ52tV{+0*>icK-)Dk z-{Wv)A@@^d^!9@vgdb@wYj%-A4&y=O;&9o?<4?Pa4i)@qVF0D@8v=ss3PiC?gws+w zA=G4$YN<_6esaD2OF$Syc{#vq8TmO@(rCcv_tI1N#B{pCOjH?HR9eXqGw1JGV%b2I-EpS7}77X}1TVfxXL z&#c>oNTKg@t)9Fc!%Q?=ZQymG==CR|8-hs0Qz5ZEr|*;e@o2uuai!I@82UN0>*ao* zPJx$KlXUkxnM~n3PCw(^D__Y`?bu{-1aVr`1+W<9V0ha4#?zqn-Z+)GQYI_)YnWv< z1S3rx?CJ9Y*pZ;@K?!S1XquzQKS}k5&bWFCElzo@@EfM3B`r85CtOmmQR{k(C18EX z5(r^p4ifYFoBj>TLxxjURRzdWxHbiH*K|x|sdsTik+=c~ANw{XhP=MM`+tvUi9cJh zqG%wf&?PFUWYCPbZsZ!C%Zg!AP`O7-5Rn)5`N{(NG{M{TLbrDJpsFm)jT>WbVexuS z_Cy+Vw*Y-f4#2Se-C^2GEC~r+ER*l*{~b=#DM>C3S~?w2vA-A2UMw4`%Jv1-kn&E8 zvWb<2FP61F*oE#N9T7*Us10)bDz{qfzdh}=S<8M3%m+l0L0u2`_qT$-Nnd?#SG-=U zuDSd8RS2M@-p3)$YhHKQWL;l%RjmNS^u-yU=cCfE&9*E09Zro_lFIGZe}8?~e^ZN( zd#&7c9$mVAC;ECto&caF=qC%cm3*SguYQty3ibf2eY|K#Ev--kxtH6CI9+hu3yGxL(dMZGkf3&wj2^A7p81%``A9?6`@;=~?g8^kqdy zUrmg@sF)TY#0rXFrpVgN*)uUPa2QYI2NJ8sX7f9n3lV^6r77XN)>KVrV&77rSb-7@ zQ>;SW6^WW`0n&UU_~FX*Y(UqZD70oEB^SQV`F736-dF#FgX#L!fhwmXOxOsynCZPR z(t|J-qMY68l8TN73jsIYuV0LIGP-H>bi95B!%cv}Z*UP+RFZ+#PiKqE_^*rJq^@%@ z(EtO=!vpRa7M8}W@K<=P!X67UMQWTj-(6x^*N+B8u$3(YK`^$FlqQ9?tPBb$03OmK zoZ$KfKEAfxE?T$@0nDWNRl7==)wJ{&*&k0mn=69NWIl&kvefwx&xcq$G6;{*EJ2Hf z+0nn`O(AIjW+Kh<*%sgyC9lVq8gngu_z1=b%*VJFUYr@TIoDE@IUypb06c`3az{Mx zi=?rG^2?HPj=S6syoE$y#pXQWHLLZNB$;~kR^D*12mZYTI;ym^wxj+KnVy?kT_xp2 z#Bo&~KG9-I*cl!g9ZW}x&Jpnuz~Y23rbD}oeuoa#shFDNo-X!9es_fx`;?*UCSzQi zDfYoF#n02}3J_h#mx!q^)qgc`lsm8$v7t?16|{J4vZH}(QQb0`&`Q;!gw1^(9M5m+ z^qq;oZLE}uplC z7KCa@5X8XbADPOiw>kWmCyIKq18gkR*bIM9QdVlUO!3x?n`oHiV~INgmSa$_a|FM#yjA|b)G|OIbAjh5_&1Y`th1o?fAa+6 z8o#ww+!=L{!)^af$MX&5%>5cAcsaL`Gb=*!<)xJD$$stWS@h`-+0$L|>qQ}}Pq_FC z(EE@qqL)z3;ci=pZX^nH@4-I{E%rV0v1v>>;{?o)*!;GL)(m$|*$<>2cPp0~7&vI}?=p^jC8> zH1K!V(*jg2@bT2j<6@8w;dS>kl&(j3WzO$ui!5Vm>rg{o_+?VbDl#W*PWnJQSE~^)v}^ zQ5dZcMP9mntO9M2q^iK=;VVp1HG7=bPh zYP3QUfc7(+5WVvX3#r4`)lOUPT+#s^lcs+6JX64TBn-;MGiyKJPy*ry3>U9p`oYXSC*9em0TNc{MjPBN6o_);d~fDb35C`@IY>#FSw1j;)YOxdK75d*WY0-KA&2G& zs;<>&AUZxx5BiKU&k1EsSWZLU_d@)7z_=FAfyqVOjg4@t!QZH7czHZw1Lh3;g2e1s zTU?bSp3`o|01XH~_h%h_cHCC@D0N)3yg@@Q9$0^yw{Lw!y3{+(=9(TJTAlF)4O8sY zA|w)a#Qa#P9C|>+ASN_HLbJXvCbSfifXNV`MSPz)m0qMaSGB+IE;@5P_Ed?AM5YDL z14v4h2H#`Fffc(^E&IO7E?C20U&6u*Lpq9hK7bZ<5%hQdJ1I&bS{`k zVZmlD9-eFwXBasvowg4%4c`rXwN-xt55rx4$f9=wPmYx9kpx7BkcB!gU4Syva&uca z2)Ml~oq3qq1{WB?}>Db<+^dNQ9%PyT#*4LeY~5itCnRhd!y6V33| zHn@BHP8J|LIyR;{eRx@=8l_uWJxH6a(mP%1Y%$tNe`tnYV=5R^(t^M#42RfM1^6sq z_j9im`(7@fFF(JSDLFh{?VpXG4V2JsQTcpcO4q2+Yr%@`jAc^iG4IJc3BV+&hqyb} zE1X)bp5HKbS=$JMLBFBjURGXff&J0w@v`hA`1V8l{bnR<7am<#J5Q(E;Rm^-OBB6M2!nWEsNNUZ*z;e)#rCUke?c|#>N2|_D>rTI8rh(vL2l9iSh}V$2B+PyN{OP zioU+A#h|{C8uZ)TEtcN2Rgr}Z)@RK1f`K{nF#%ya^x9OQ|XvApy$Un@M;*iftIGP1Hr(3D(Uv^?oq zVX0E5@{oi{&Hns~53d`tmip8U2`HhDQ17CEBj*=JNW|lf0wsiNrNUFDWBthTkkB%v zBKrON_ryFZfZ0cBBz>o%N2M44gyS^Js!i_C1y{g;NR3Se!x8w7;2~7n#zs@GV^MiP zy|C}QV3jNeRzL_6(+{J!@6*9>jA62J%QbT zN)k97R6iji`ABSWbC(MUBB&>+(@2qMt~MWWL33;@AR%e<#H6I7!^0Ai=0jtw%Uyn5 zn&Mzpf?D7h^Pw$UGSVk0`Yxx=O}iRfsxL7p2K$@JELq*__E;4&^qc1&85yIPyhMnb z^*78~H>pOWp0ja!>IZVcYKPaCXMJ3x5W+}ektb=sAxX8kt;5z;!bll0C+NC^hX>6% zx2YH&w#o0FX&6IG7l8SP{E;gEsr?AB0C@kF$#kvT5U|8BM%UVrkx|Tp?H;%@75A+o zan+O!r=HlXPYw?Zqi0=me%7%NBAp`)m7%MvtH-`j1k0(-Rx5F381cK5_-O`_H;| zFyck_V#-MO*VjA4a7${f)fjvKxP4l<1l71cro18I?ee-{l3l22eM>RxQHej^41|e! z@f&01a|CKg5oPZf85F9%y1rE*XBg4I!35~m)NheEolm=KX!Z~jl!D&*=z&1U&c?RW zo0q_gW%u!Zgd}q0EK|yo=}zbWE!7x9vxS+NnKuqTy1DUyf`4!9?waF* z2*kogVn5KNLjqQwqACo0(g zx^_`=Mr?G^f&sPrN5NMYhB7Pp?!$N^t{+&LXkI=64K#G@^h}7_;IxwH5Gk#a05QDU z`DL}+?tZe;stPqzrVQgLd=}jwrQHxFG0fbw^-*S~cLUd=FKIHOmk0eySBR37$lLgP z!0<@!tQtl&&8NkU4S!3uVw48+`RARbSfGlE%1@iaP#~m~z(~MOaioV!AnB~HZhq_s z3}-D!TOT*RkqTUl39@$j@(D7=Wnte%+WSOmjg`QFvVZ!P{t;Yhuw zgoVcjp`rooXf&Ada#o<@X#wlnSIuEj(XRxc%ER8P2ZgiKm;K?})tL(@wBz9N!FLa( z%jd#yIH-;?E*^KgK^7e{^^7W^x`6lhr-`NNE$1P>RN_V0HMw(9LEJ3Lw^dUjwF zA^C3;pQ_beL{lQe!y7})GkkN(7O$2(CFBsn{PF1&JF)z198Il`b{ZdI4B*hu@AfhL z=VqB{^|`PR2hf5T0wPc%Rj#SXbFX9SWWVQvF+e)j-^+&T{``@yE~7{#RZ1fuf=e3U z;}V)3|H&fA+h1HG6Hps(G;G&WmfuiXDmTRiIZ$=9^`5pq1^nPpSlbP)lm|3+<{jB5 zjk&2~2EzhMp@xzTc9$tyrn`-abZR|HL_lZoW%u2VOLEnF0TnDTRvN}9d5_IrBuHg@ z9i?`b5;`J8)d3A>v#HD;1LfCkLt;m*aD!mGR{* z$f2;_27oq5W6?1+r`2-!o&R8{)WWJdIV~q|88$5|K&8f%w>j zG}iu^>$&2ZZ|=4(Rm+Pz8}Q_yG`W!G@#|pJpP0}8O8i)MLoOw(oSmHkCoewYmp;I_ z=Gevn*54xAWB0t@PnQNhi$32vsAz%^Rhp}ss?+PO%6EE9dV3D6$6v9&pxF%^6@Rr?4M%n9; zj|q96(oBuTPEMOIN zdr|2Z6}qj!*i^{z`Iu|{h)k0hL`^RVpkvGRr}`&Tzxsnwt=$Fm(}^%j1YA9=JUbLK zyGZU;fD>W=*THX=Ze)J+@>wn%&FL4Q-?Hf6)Ivul&8V0D*DJCY02#UMDBz@pVUNc+ z?Ep+XTTl)UZ94CFpd*qJ{Z%o*5lZ$F+jY`0i#rJht1G2rc~cI(H~eI@y1h@nFy^BX z<_R$jfde7s{``U#6Jx%!qx+H|-;=Kprb2ONku(Sndt4j^W(3qKxijDeP z%jZ928zO2p81@5Jk`Z!m3e-}vO+-l2qBJ!sOy&g**QD(QuVNG8G(-e|%!A7o)59m&h4ExGQaLP9 zNtW|0M=}vVXsT(#4`dP$0e>Bc52AN0aGp&@jQXfvPx+~9mn|bP%zEesBWz&{9l~R8 z$>(>{5%7y$T5D-O?t6~OGg0_K!LLD}1znEuhp0Chtt<7(h9IL%lBkGw+jQHP9dN990SP+>MVfb z3YvDkEN;Qw-JRgFxVr~;mjny$4#6R~LvVKwA-KC+@BqOb?)iSZ6jjs~aQ5uXZck57 zzt5{}rEg24Cl7=2^W4s~C9iXSNzh^h5ybA3{Tk;x0K&$EAdMjASW$+v1(sO$_+F|5 z471IR&XdJ?L*EPGu{t#=3tYs0RT}ih1GKNaj=FRHZ{0ybh=Ac;#nc3w>=h4=_wrS` zH5>@Qj!b3C#oI`!VN1zn6k+P^BdV$S3EputDUyfj;FwE5Baov9rkt$e>`@g-bAR~p z$?1Y!->Dj^2}Fe%M)DH`qQ}ro6k#PM&P`n(Pg_$WB2@pfJf9F5EI!jFLYmE7TT%Br zhJcp_KIzMvC^sDFl#u8uNloJ*A>BkvU4%fr9>dv}1ihF6eTHD3O@q|s|5_flB#W!F z0dXB*hA|cJc3r-W4U$XQOYScVc#9>x`f4sYAl~He9-%c++%8y?;)5{0X{aDA*+jVk zDytPq9{gL4S718osgk$RYK;d4j+ShZ_wjUi-0Si=s4^YBaoKbIyNb?%jU+{xnAC99 zy6MXjJV$E>qfUqR^~KfpJDIXfS(q4#rh&oVd&b)Cr`_fPdW#Gk1t*(j8Rp%^0yc>Qd6~5RoR6IiL($UB}@H8wvLXDn$I_7 z$e!1cr``%h@$dF=CPPa@<`NTkqH0cs^IpcS^MFPAiEiL+{bx`C$AAS$U30yyQ3Z;`%?^2=;KG6xj12;`?&5e^Lxk+ zn?0^ynepfLfDSU=~Ds(Q#^@AVROxK%B{0g*m&x> z!@Bfv!bf=PbV)mYtL^N(Wz}gd32$VP73LF2E)arg#+Lu15Xo-q-rypR?;^e}!^^cC zHkM|+r&si`w@Y zk`hGghCr(sY4-Ei$b{0_xRB-5ueUe<*BA}e?OJ9w(QCPnAR4Zc4mcVvCkBd7^36?6 z%$b4%5!$ySR8?`8x;|y#qsdcIbWnbNpD};W-*GzYptvey)J3r)^bHu4L~}>OZ8QK( zvo`y=OiR9*r=?{leGzDeCKs3PaioUu`v*ed3&Y^{(c;rC_p~GTw1P4jN z&?BIV$;*LM|Da(zblFz3V8P${nb1&!sH@r>*OtaJ_5ALHC?~&jP?5%;rt5;X@A1}~ zK369#_T1mCrNKbfTAtyVy7y^iLGfS>-(*6+4QUoJqL$-RmSp4jS4IMZNxn2tS+8^ozbd9=%bUO=4<-b>&pq zLik*i9)Em#I~W8hNkNJz(kPv@aX!TYGck=))!Q_{g)yf(<_URy+3VQP$;n}1F&qXl z2kBYsTG8RKkyBD1GNJ!VO;6Y7%qbR$!IhEB(&EF+<8=4-(QidBY$y~Wj)JPN;TB-W zH>xw)xx^}qvgz;Q7dFh%ru+@*^t{kU0IPy(joP`j^vO*$HB*2&mWHB4+MfR){b>ry z1h{wL2WZ4(+z3xjp|2^91!QBz!}ro58}Z;KBZyJ{83C9Qg|!*|wDa39;{}&uHfaO| z;Gn*T%RQDca4AL$sny*x$?N+28`N6MkdeR1mVb`n5P&r$6#`&n{Svs@5ecEtuapdPLv;KgS&t-0w$5M)c<<#C%bYdmr_Xag1l*#Okar*}e8|;q2@zPWJO0N!B1dSqKgp z-p&07X&7o|DR;-$vGl4>&4ot4&7(s7I$AvL3ji!ayRDD-e71-szN)IG_jXZZePpxi z)(m{hI!^0EMMd%vava)jU_&>xBdL%9W~3Mekppy~By^wyHx_oJhFCU|LzBbokWV(d z?Pz-MgN3DDr1{Q?MqXFIHdO|YqrE;3+;7}Ai@sh5g9DtMFSw>&zPQw}n$iGJ#KX0V z&O@evCp@vdAI>%}Ll^J*d8rWbIqq8HR*%cz_OQ~Kjg_aTn$|lK0md0;!Z!d;!t!0T zbJ*I$v!$o!GzU6DgaYN}=KkfnkxkU?{h+t=R>W3nDp%lgSD}^`Tj2{P13DL2*XEN= zFqT|j(+u|Tut{>=unxc1@1>TFsm@&ZCDM`|<5xKIK&-RRj-iLVovB|B;4; zvD&bL{x_Ge0O89~kaqd}%z+DU3d|tb0Z)wtCy0rtlhd8hAtJzSgC9<%GRr+KrHYyK z`4#n)iHQj##l-zk*pqQ=^L0{iM^^k0lBh1vki597u6I&IMVHR3=XPj#9aIglT+L-f zH-c~4}h7igf93e~HV0YHhkW|<+xp=*>Ff0$1BjD!*Ap1Qx+|i&sTX^%T zste~eE%kib-=%0NOR&j{lA>cPxDIOp9eU@*fA-N_grwq_Ao=S(82V+vO_N_=gE!!E zv}W@a;CX$|w#Jr$7R!&8{HbMqn#Qc(?aPn);YNy5B3v)f1dAR{h76RlC=FK*Qpqjv zKJUQs+-SDV@8#v>ll%!uP_Gz}QD!mhc(_~Rp;%^_h$~acHPZ$W$LnQkP*cXNz?qSo zRku!dB*eyo#v_N32I`kHZ%^f8kbPG5n{$gyH|s7B8a~sAw=WGAeEaq-<@M7SqZ`gY zBUoRSV4*;biYL{{ewHxZu$sYVH{No`j&A=noAsRsEC>?@A`Zk)4@=Sx1#B!nowZ++ z`v9@%zkQuZQMVUWu{Y#@Rk?)%Im+64AoQGneObwqlr)VMH^F^5Ug{mq zIRx^!JWqmspa9W|#QSF5GS4gkitra(Vi)qEYmDa4ZB-HN4*R^)`#w zNYqQ*OZ>B^%weZ<&t-oEnT%vmoJgR92KE%!Qb+_wek*?%eLuPh-Y|WzVuEoH)M|NJ z6eN?FD+NXqDK->g6$GZPcWel&0-+$5N|lO}TZLf34PijcrmZ9}7&KG5a&@HQ<%2gmZ|} z-BPrmcp5MT`UsN1R0Mfq7|SWlo;Lm28oU_Qou*1M`aZxrCc*2$+6qDeg^`%8EemBs z^%)$75{XhZu#+g6Ae(880R(t3bXa1o|0`xB)C9!z^5j(8gEz0TT*SO!WZRs4!UhDQ zXb=yh1z}OE%Fz}iU;~?DCd|6?@Md7`X+E5UPVV%lOpxtYJwDN5>d5-f21NHrB`)5C zNDL!Hd}vO!71`m~qkjZL%o_$~E!fep5lk3PL$PR74+_Q!t{zyR85%MHZPfm$4KA&V z%DDAC2K*^}gAxfNs!nfTD3^<=DNjx}R8Saj=ZVLF=ciCC=%eHuDI)R;uapC^~hLB2~zUpRNU+vUs>bpt!KX_tUctRH?96k#}pF zr2ok}cJ2wqBMG>y{=A;-0_FwWND?LOFeGg@A~~f0p^&wR-_C2OK%sAT?9jrEDx5)q z>Q7MhStK$yI)y9*H_7Qi?5LSMUg~mkh@hMMd)}6&Pc3FT`FYqWSo=}vDaOB{QsYD% zlb}JtB{l4~=nh5YXom=+(s`2okl@t*&!B0wN5zs;#6VYdb(9{{Hy*d&a$qTw<79}P z%XEF)WeKU*2HI<;!8Fatu)GKy&?-EP-7dYr;y~gvaotlW7{)I&AHhuLiTIZnoMc{Msq4lFr{%Ho0 z&N#u51zsn7V0f$&10`PP;qu?&yC-bHqSlozm(ZEx4-nIaUlY6SG{mNVH8^7zkt`N2 zP)uf3xb_<}7M!fgB7=haw^>>pZXQtyS~H2*w6$`Y9QFB+y%{19T$Z7_aZX#3z(;*9 ziUPrumP^ibyGn6*Q`r^r3m5p`?aK@&Ii<%WAjMBH`NYP$I%(e`2_*o;hjth zxicqZ9-A^fM(}EqstDqp3AOsI9o^lpFOS~C*|jKMkailC$s!Mq3EoMqG6#i&*hYB~ zIfGS|;^h7T!+0tb8h9QZGV(IRQ3kTT4i_qS%e3&~rEmT);!7Wu1IKpsM-1_TOpaRg zoe2GeG{kTWixLeV*}ZeKxRw@%x?l|6zDOpnHmHXU)cNFb2zgBXPp0Z(`FZ7UE|K2f z+=A}Ut~=hFPiMwyR_MM@UNa3^h7w$mMvch)tsb;%l*+x85Gnt%4CJ>4fJgXXOMi}JucS=T6@m!}uBAB4M@7z6PJ==|L zS2bCbe%WJEhrlQfXnjj4zWeRr*CI5qk@Uo)wR^{}HS!AI=&@~qUEL)nWjLb6RfLLy z6~sNFYGX&0Gc)98_I_r}Z_$!u*gGe$-p3qHV<=};1|r9d0?u8j9QO|;S%@cp4^4D; z5PgZGv;IRleGAQKrcdMoW%bWnF9~g~#%avdg)gVr6vpBu+=i4ThndIumEfvPhFtXY z#NfisJydgC?^dwsV@NX54q$Eqk&Jlp@KNmCANkurTQJ>RDvfU~vo=Q|aAihqMUVue zch!qM#g{E&F_JgqD0=kA z8y?4qEMZRHJoZoDYG_;<@f3>nL zAYbf=tjb2IB!43sz;n3|&5+$0#C`8a*FlB}lov_K~kSH6!W z`0jUb2gIaae48vMqi%9N7f?r1=9~v1?*}`EI3zF+%4@grRNVFxFC+b z5tCn$J_;f8^R%#vaxT9MZ9e_~5fFj?!@E8fXaxCTfZiX3_$>3ro{=THQWi69;mzx6cr^20meu`g z6y*O1%R~4kUWUxmT)qDK) zmf9|eSOu2L;=1Ocx#;|?5xLV3JxXE8T^i&K%rsTg|cNQ+2*a45{qSZ_ko-< zd!>m)PkUNon9ghVgMHQIb9Se0B?Si;lz1i&AWXDFZ8QB?(X=tD&y~-2i;C> zL-I2HM4hGjPEz+oTYc582=|RjpwUD^~o5uij zS=fFm{Nyvze>V)Nj03QLM@=m{`2*DoQ7s$H2Np68GZm6e=NDyccal!8xBCaaC#;hF zZu9t_v%Y0sMa;VY)YTBz5E2(n5F*X^gu^-hojWK-0SZzVi+cCLGyN`QKH&cL7S(4Z zwQjS6vJLxPBtlNiEw;&Z_h|s8IjnQ{$i9-(&F{eT|_)Qj_&A`Q32g;S?`zK_!sqhD4uun=-d zqrKdBmTuN~+p&e!H&W;%ag2=lQ|^|0Si6|wd-vaNC7M$4shxgaUZ{c|hM<%-pbm-$ zQ78Q1>ibVM=D!v@11WOQ?1fT9=k67iKGmTxep&9$7kiHgE#;{6GC0VjCxmt}!VxzA zswMXf>W{z?(>Q{(@Xrw11L8@XB=Ikz1p-%GR^2pYJ*%;~rEC%h7-;5^-`C_TlH7L9 zc}sRWHVt!W^o`VRk?(dwXIc$5xda-CZ^ThY(%0ec&+?kWOe1sGn*#9|GN z#{8?yy7qva>q_+NVd2p-^J3Q-gT<_>rt$Po12lsz3e6wWw3C)%Es0s3?*cq~<99`zECK5Vqp-dLSfKD;evi-Tdxo! zri$Rl3&}efu`PMfhTIaN53`&$7vwYFY>YeQRV{0UcJWL17g$yZ_pMsNvGaQV` zgbY3Py$%(xr(Cu5!pQhJBbOz5hm>ZOK@@at@U}T=4JbTjQB+!icUWz*E9`vYD6Z&mo~xlLVVaNlDCGxl>G~suA-tdTpX5| z!XS!N277QEwga+Zb`Jtv%rmTNxkAmhG6(C<5d$vG`;!lrv35N@C`>!LA~fv5AdvjG z?=ZCCYBGu-d6QTE)vA4h`mx0t-;eLwRNH>uc{+c%bhti-2HTc;4U8X)*&R@d?KUQM2SCkttv8LO5^o(}#I=V!|Y+DSE?B;CD zzp(q))?AsQq>*fj%6y67F#TQp0!B~jB%u5qrnayqYW~EX*MRKkle6V{mTmvH$z5L^ z#pB@}L&P1sZ>I+~Ef&JOzK^6!zTbSy!6GqQQ!)ULl>{^l`|;Oo7O{g|sZ?8C%}ASvOv-&Ma|e7+G8&6yGs z$a`5Uvk0<$WJU&2hCdXymuy|AB-}gvPe}UO7X7ch!6zNwN*a?%r#eHx4-UG$`(x;i zeuIOTt?ltlyr}w!M5aiUc~Y)I8iU#FH@8@Nx$$g4w#5|o6w4ws4j*FtgfQpXPCtS_}uIEz{j`*eT^)jUK z?!0Hd<`I%T<9yMq;ZhtRBsfTd+#Cg61tIkQH*5KDS(P~==dPfmEWXc$rfBY-O?1nM zHF*^Qd{dVSR7JwRxF!&o#n0>u-yJz}GDv>&>6ACwvP0LYM}C*LPBfT*9NWk*K4Y|g zyoiU3k#zrhl&L#@!Gw<&AI@C499_6Z^!g>SdR8w(PP%)Er9aa1n1x=$#7qeX740xb zQhJ4*A9Af$)6OKo((aS4U}bJ%WgqgD9Zr?0>9!k62<*1q2<3o7Ili zqU%fEVw8;k_M_9j#L&Qn5!B=-&EUeiS}Q-#O0@P~o+u0{FM*%2lfd$k+qX<g^CAQ?msB;)@JcLHtcZ0KR-`uk!-a(8_!R&{_ngRI%Uz#ZhBt5(JMzy?jw3s} zee#qTFNx;S9HY(Nf|^nou|0ji@z_J|JA; zow-Fr~IR-fLIOH-5gN3sh5Jv|E zR-pq{!sH-H?577sN%?wJ9!^d~*i~M6jpRu&Wroh`YRjoC!2}u*`GBf1FB7N<3dMxP zR+&;k8VjHQm^M;b7!nw*R;UV8TZ7_IQ2yKy28lI@OX51vjUkE^CWVeek`N=4&Y4H+ z%Ot33{@4p{B8-Z`ARd`Zh87D_(-$bA^vg!nnbL3zkHJ8`Vc^=CE!Fqj;NS{BOb_%( zZsoRme30C=w50J@Z!wDsH)HQto+t$=KoEH4<8UkoNm=%bas-@@1KeRF8yf}0+PD}D zd?%D@phF?W+>+VgR#oq`E^(&5CMO3v{S6W22%BN>P)19)`mdvDyADrp zy~H_VMS?!05?*+gjB;W#O~e+4M}?gpBWJ>2%Iq3v$(5=UNSf+(FrX0_YGN{r77VCV zt2}mF^mN~t?JLkk(i{s~5T*lNTuVk2HdSHyUK_=7`zs}hFK%;lZDOmjA~g;!;x) zfzgZP^|%Eo>#rN&`#m;JZPe_VHZu1T5=`h)c}j>tpd`pUCWUJff!Op8oAjdG7xCP0 zyCcu~&&)n~n}?PGrT2EBJ-UTnb?@_GoAH39!|U4*bjap);Qy9KPl7L}CD4Sg$GLzb z9%-M}_ce2XqPo@!7~}ap*9N?v%2H#ZzyS{@$DMl~OM5^-C6oRD5sMCC&Lp9D3U$?n zgLqAEnsQQnXK|u!nCUU7$6>0nTP6lp7H;aYi9Se*J06bH}m@Z z*;Yqp?5%YRIQmSJ-g(M$`qXS#d^!QYT~X3#Gvb@W6i0vn*OM@$Zugt#MkrUtu95rUi`fy%gHccuR7Rv ziAXS>Sdg)w*R#L*`j7?fG4A}^fw0fr{6cGRgAx!y{FLyB_~hi+Bl%s^9DM%OEE%nR z?&#=ekCh?oLJl=pFMq`ma_Gu((U&$C3{* z8S?D)*?v_~PTpDSW@ThOJGb$AKjUTP;a;O`vbjuJTdb{MMwX@uc}61^aCvB=tp7Ul z0*nn48xbBFns3|tMo2`wUg42zo{yyaeAnY|`_JgiVEY>V4y#*tnv%Va)htop`?J-P zYYy(|f6i^k;ZiZUaZJqIavsgRXB*c}N`_2`<~zHvyqj$Wz6Q2JLjQK&?$Jm^v_EO9 z>*_viB2O)4S4SiFf}zFaZYm+i>3J`{rz>dp-SDUgo;|mV^{d}xMDO_* z)EZq6m1A%cbb(XlnB4HwgTQ1~jlgoON*jW3BH_WlTriOKZU4AT@Kq^4xBJAlpttn$%PNPh*vI>FR{Oi9x z4LXs(N4Qnxu8VpfrQ>n)6n|b17p+kx7AzTi8B&0-sGg7&NZa1uXpj$*V)OXCZ4STj z{zR%?ICtd>xaZy^iho0p1wrIV zjc3{bEq9e}SD%dbZ8;TS)~Ig0+%VJSdfnYfc4PqN^J#5&ut^4UVydJ@aCx7OF80fwsJ}Xmnds9Y*N9A2am^`H(DSYLWsFNi!uM3)*xP)F$$E00so0W9+o|iaw5<2t&sJ}Cp$3!^ z+eeG|P*f}O`M_bJ=N?TSyWlF?`E|(o^=Zq0b!PQW=KWZOm3!X3i3bj~ik>Bd+Z0e3 z|0{A4Dv*`!V7#LK3dGiZ|I+GsoD<$08^mZhBxcN9m74#Jsf?i*C~!a&1zK*5D4$5` zeHc!^%TiyR8oz(S@)h*>yWHi)1+pn+bGKaVj>Jd8J?;e{xUdEBr*itN(x80!7GVTX zSXkJZ92=4}W=}9|4Bp52T)Ub|F%-%teAPnqk;a-i%HWA`GxRD|JrDvjg2Nw}AhQE1 znQu7{HhJK4f`$y=%B4-0*U_-&yPmlj;X04oW;BPDmt<@I8T8WfIV*sRa<=K!$AesJ z?7ora!~1?zHuXVDQ%p?E)6-ez^?0FDCr?0wCA~`OuYY1|J7jQOFt!f5dEi5 zpN#!(n`CYumF9olMII}fMpy(LY68~6}#b%~)a#Erhl>FMbi;QJ3<*5|gu{$iuy zHiPWl!GvBhKY=8IVeueN(fu}VsB+S=%TYt(y7piym%shX3tPaW!dOH9Kb1TlOB)lD zi!A4iP=z$M_VFv=3AEN@r=d;DR>|M##0*Ab$)y`n<~Rz|A}Z=z%t?Z@40cS z_kPPW@B4%G6rAsV>xS1-`@qGSfb04NZ$x2>88eYiz3p5nmW+si=bhX1MN9-dff$tS zlC+t&c$HD>`C>EE2mgQm68b|)u-C0l8&7~ZDwlst8D*|ands=k%lhy`UjD~Qo%Ke` zQ5l&3f6m8$lpJ8}x^S@a@Ah;xQ?C(fwb(Adxh}q~7>O+DO z+QTi2O~4?LJ=^q}{f*L9?D=@gQ)Srxue;m%R79lB@|sKZ+4dO%W4uUAdI$smDu@vc7#YcTFTkFuIFwRKc)`39M)*BjJfR7&@30qv3~xW$P!G) z+WDQPEeRmcJ*I!1G?rmSg|efRXpU-aa4Y3a@mgLrxmdX51Li9^79yp#Ph4YAFbaUV zEW6j$A6ae+1iZ95xP)d(%hHre?U9v4hE+t+^TRlV5T>Xnv{=U%m^BZ@=E^XAjfr1$%ayxm{h0A&J3$7pT&-qg@~D);>{ zr+rEBZ8ya8L-5omk3XV&Mw&Rui1XJ(GhYzYN(OG>CEqQK>P zAK<|a*ZN;3X56p){Jq^s@;zqt*ocW`r`fd-nBO4B`m_p=Lvp`1YyX5!65*G3v=ETps%wZ>g@Xt?=S27c)fP8 z>3I=T+v`|VTVL1qb+?%8ZM$R`azWDSY;gH2ne1h**YNL=|NZ;Z^9mPTfaw~E%KLUF zbYdh+*cp?i*mcj@+8Ppa@B6#ol*(dy-dBrrzx(dH#b^14CFg*;Md!B-QD5F8EEV53 zR{&nFBt++Mz@H#q#OJqWjI?*R_lB?%xYF z;%j)S{=4I34AZ@ykO|AZ=$IHXy~fhQs=BxJV?6&?gL@NAvA`KTga>6gTPCl1c5L3_ zI5^PDUgrfzBO@ag+)QCugn$pBIE{bril+LyDn4_Uf1Kq&n5vDo+-Hr?Q8@77*A@f8QZc9m)%=$uFZ>!W1!j^KX~G+Zz<2Q3W=ptKBb`zvXKMTc z7Cltd!!*R&rxP5K0+B#y@ex?FtDIVeoSB{?$BPPB&W6gmH-pWes=WT=rV?jD^#kz0 z2=IHd?v|(*hW{qyKM~Aw?~wUP?`bFKzO%y{Bo2s*Hs)GSl|64=W_DzeqL06@l4(_9KioY|dn_@`@B%XXgRD-Tt@_K#0M2)}g%KdW-Me|z zi!%u#1z1_3D3YXVF$+y^tl&lj^M#M^&RQ5!3S@;J)-Xmu-U9bJ0sP~bx$ zCYC7^lEvTL-=U@9;@Rn(yPa5rtfGDefw!guNurjKM#z`}=gJyES5Cq>g(iW{j^C-+ ztxY^cutB@KKRdX}BAz0!DTYeK!EWMXfToM*<41UKkgbMwr0vyzcn;rDA%HVSoYDlW zzh}F;{Kt>Y%mz(1J3nFl^Jcad1o)QG$kn5NA&ff=nYUQp-l2X7qtd3tjR|K_xJ3(% z35F%l(PRfzC;W=?c5NC^WrMER2$JteZ= z=~8VLU(KJ}EunaaBTrFDvxO6_nNYtI2X~_|D3@Icc?e>U;Dz7zH>LXr7#g#kbmJ#2 z9b;p1{C@?)^BX5c_U3`Y7-NNj;|4hi0ejRg-FIV^mF8^s+dG~irgyUze?u{9{^KL? zUm4qjp@ggFAIU<`%^V^iWmA?)e0Td3DM>jce7}Cyy4B-ze{NvM7Gr90;9{Zr-tD-? zGtZBcEm6fZjL|neogm2UtzCwlAii~5sbkP<^Td|xBSN7uWieU)J%faFdO-jRojq8c zoSKkkkfuQOsAkQ%^WrX!0|8tCl17e}5Ap`#G_ZV!vSG(;cGyRQfvEObkLphK_4Ung zeT>nr;rA)9U$_bbN>4};1D9<$^zhea{!1*w4e|(MT@-Z?VlG*vj@#0$0-dV zI0J$q3n^J%5=@Cy06ZoTkRX8dOK;>p~HL%*PvGR-E4Cv8qY}@FI z=6P8+_@ zN{9#?72sI782LUyMWc=BQDVS?V|1)yQnQm2n%=?<%tIMIu=BKfJx8aK&13mR#>dB8 zp^{w{q26Ky)50tg&hH^)i8Vq5XXmi-9io#+#Wo>F3 z%hGpQ`KgyotMsGVo?`C%$J5NoZ!M*zX~E$4;FY=B_JhpTj?&l<`J4%n6#X`GxZo(z zcjgf&N|Q;fBW(xC9KM$nl}RBZM-ToTmAlZ#2o( zafaI2*_mPHZ}&Z4;KGa#k@{2>a?h`8zF&>8E%EXow^UoeVVY?}0%-+j7#$@7fL@|s z>XaZxm@x8NkLqcVDZL3*Gd+gZmm&|A1?FT{aNz6nCKCd_oEIzYzM6KDoTQ0~88?0y zShwD7ha}4h`4J4`VcOTGs&mRcEFhb07Kn_hwDlNqLFhpBtR>Kh1dSEa4~euv@^=|K zc;?xVs`7>70rsoKPn>;QH(^?%MQphA`#>hi++S2J;YM6Lo=cW?*6_#b?PvkV0$}*hlZ9wDDn1D zTs{GC0R+%2dLAqwi%aZwp;f?7$n{}btYwdrFijD~5(%BW05h>H;6VcR;HLeWhR_Z> zrVE>{5CMTNor%D_{eMsIflq#bt_ysFLQZba>zOl?ri$<_HV8l95-$J_1S)=iHw?_! zaO#Z&jQ)&#&YE{vJ>C@}a~#iR8^UTKrLLn%*T)IIqsqF5XCy}z)Rod=Gh}J8Ajd-J#onSn?~!(0Og^oi90D>9`3=8A zB_ve}M8}O}mQZm?3-ZNd8w?Bt2WS=owAR*ECWatgv~QB+!L}N-E78B>@^tm|R_50~ zFl&&;j9iMBP(2);akDQk=5BNno=y(dt5RT4hyPUjRte0=&T^COX|3_?176st=nyWrRpNrnHz-$w()~P|Ao-qXmsryM zoWrZZ6=M=X6%rkmj(u?gLmNc_0v3A~gwUGKbAds)U~aw+v%zSUZ#hweEMdhAS?A~H zxw*NsNkHyaI*1i@?((x?Z=p;XFwNy5M3Sqk{tjsn|M&c=EmXnexfuZhbAs8~1CuZd z>qBjy4BO>1lJ~>)`hKF|h9W^940hmZa$-g`dAO-mWt0C1D;ZZ@NZQO%1Aywqjwz}m zz!Pn5ZUWB*+gSEVyQ`!co_v6O)bg08gF=pzs}@{Gfd2QPlC9focbEZ0UI!duBmTgg z3F0;K+L=i2BV%t}_dLwWy6}5CCS%IDlb?GIQ|A$A568Bi6L^AE)TY&!PDrY+W)pHg zKB>?+1j9|{2{+c(#wTdxw%1Hzumi>}Ggk?s~IbiPR4Y3P`ORoR5WE7^te&!bnCH{nLZ|!o2vIAt$^3O_XpFzkqHluhb9FDz9K0O zhW;^UM-K{BCM*8(oo>)X_Fpa)%w`R;$}mqMFx7mACTt=H!_vlyP;=WcDD$AQZ4_3$chTdOc=B-5#_A^eZTk z9NkSG9j*>4H8lYV@w+adsI;5(vA?}B~>6dk@-Hw4irAoO4>fIGUp zh7%(-dsXHmwK~dec-R9CEYS$Cub_N%^gxK*L3pWjnunfTVv;sBCIdq)47VHDOi3~! zW=tY&*2c#S&_L6DiX8RPed4XeJVu8U(&h z7pt6{ZMD;?+2bvVDX=($6(?yq66S4BH_GF3RLapCwn$5!z#zqf!a|@_+xv1Y`rc>~ zBzk)%CDzfE-uR|NfsT}-*l7jFP`+`FiP%|NYkmD&tC{Y@;=%%;m-N3JaU9dBecTS&8>XJg7IEE) z*6N1t_H#V|&V-xImY6Q+`cxt1ftKUrtiEGJgm{yZV0hW8bMufHxF2Nzyt zxE^@6e0*l-C^z$^G}Vs|rrB>i90#=2CfFLk(}%DS6|8ip1}S%dn(}&w)2r&box|BD z?tsmY$&AL{$229__@#RrX&u=>aQo5ZyYn}UC8s;qOkPS)D6tMZHjCYF|D?GZ&ptdH zZX7amlBy7-1>kw)^>j2m?6 zB6`)dnEsBnWuh|Sz6@C~Ls4LW^z`LQ4D?DgZ=NQF^;XfvvvIoc3g|Jap~%A-ziFNHhy1x$KbyFGm@D2kG^Zsq9RWvlmv3kSM5Vk)*7o1#l#)2m*;`L^~^)y8R zU8zhO@H8|fs~isk(<({OKzCm~WuXcE>)U#ncpu^XW#(dz~F+ zED0evQX62p6&AjzyzR1~wt%#J^bg;k&hH}%uTph9o&5dM#A_^f{+??W)@Qhs*!BlmOhY^mnVl6D{dVnGa(bEfO}o3b;fq&dey+{eEf1AR}UM zw$_LF4-a+3q;K{z$3uD|4hiQ7#=V=R$5up;jS-Jks7(Eq1D-PUQI&p`!OqpmfP&mSx*K_(Pey+ z4YX+P{CpSWS-*=FdMiE?bMOZid#E8PU_sIp{YJ>`+b)kXmnZy>%7t3#ZEoizfEu~K zsp(Al2;rXbfpVptpG3p6)HKbJx3~e9fuS!!>PHQt^Bb${s!tVyCYB&*ZXgdaV*z*KMhYm zBf$$iyj}uL!GblsbYzSij-lR>fI5-h00M=Ekm0=+tqd_e=?X)gQ~NK)8o}9a_tn}J+A-QSVd>; zKfL^FbFsw!rTwy0quXWi_@`)Z@9L>V?7G|SO5Y9?RyiYH=*if_;~lSKtA5oobm((d ztK0dd6G+&{y6z0nJAHwf*qB{v3V5!0`-KpOW@-2LHcdy34p>5jsaVs)L%Q}UZ>b@H zl{kR2-S}?=<-`g}>3G>^R$qqre5O5y2z?GzD$hT%`8*I!WJdqqK5x6MPgG#I(>veH z3}5?uf=>3FVaVkO4w<@fl)GQ}l365*lMP;Zy-e&{JSVWl{=>$B5r#>FmNi*}m5&e; zOW&0m=i5VLzCH zRF)kByTUm*)EMSBc?NvAX?qx`$QyTb6*)RJ`j}Id1~g}ZC}b}Hz{!jZiUw#y3!=s} z!-Y(m9vCL_`P$s#>T-1t%)53S)^6czDsaB9%ik4K9tFqMcP%oQTwaMSz0{ zV&!4fW+evD%&c0=kjfx~&rPfJwtxrRRnk*WWnk&mr6x%Q0nCEIgHfXc5u(UZQiav! zbBJ3YXw8g~+l^{o%}o-h8`czPgizV^0%fO*HK{JyNKrjWh6VN{By9 z-m`xw+s$jsyB578J?;2^k|77waRalFBY&>iEL~$tW@98Vh!Rdvn?$h9JSEMNwNto&~ZJEy!b!fK>NCtoxsd z8jFj^6Vk?bCoe~63L2UgK<99A;MXimYPzGcRZ58n43-2E0asmJ6%w#nI0al2q-laM zNKvWYjdCvd0lY3bUqRAipn%R6mWMNFc}Zd8=QWhD0&EeZ?%zt}C`Be|is3ZnoaN*R zVU$64KfwT?709-n;0^a1GU+vuz4>l%*qNPj=5D#1WG9C>fiohe1F#_I(&FmEfPp~R zNMRg$kQ87a3Y1RV-NXf1MP?HV0+N}O#sZuZA;kXUBY3F=Ex>=PO--Uh4m|<9mMOYd znV8)fyvo$*pP|nJ@e)tDbkIzS>w6hEoiGYiNG2FI6kw1N`K`0Q#v>#UML9TGHTD6; z76dbB&N3DxPzTm5eyYNbr&tD{NXy{B&cGg(i_*%%($LOCli_3Jb$LelFutLm717P| zas@*w&7F{g%zO#r+Rw=WcFsIe!nVBjBb}W^q>XBfM&gh6Gg>L1Z)PcAH{j}s!trNv zg8vfVuKoSY{r8O<8^oxf*RyKJPN-QM&-gz_x?zZ;foGcK>gxG3uRH<(0Cb%}#vhi? zV~xA{^e9y5COU{&`)&IBL#g;@c<`A$x9iSmc^>!D#;Has;rrpUe+bd zC}Bh!6p{qyXT^!OC)GkKQeOAlp}h` zWNXcW)15c{cJEoKW~*zb?-L@r3Cy-Br^N{G5+e8yNRYg^CAiJ62~n9Ft+Ue1y|y~@ zKlhGD%s$?`zxe^AbmSfisS7dAAtB(>0d@dIXJIf5QrR1$kc=Z5=_$V2 z1OU8+6*hOwJ)OSW!onyYmF!@Jt3u5Kd5HZpq!4!dZ=qvvkG= z7QomKQ+r!-_Z1TaSb`AXWo9m$%dr&(cm1?~!{!#d8<9mu1d0NcbaiBGbg{u+)5?mI z)h$+t9dNT)KGuOTAJDLp4@@PWJi_&tn=GBTinB8mr{3sR~vIBT1(g??|NBLN2tY_1T)NpP zG~)8QSv%v2Vl7t_>npIF$VH1XGEu3_Xa@08_m1y`K+<1!WH}t}DIp>}`M-V{0gWn6 zz(4>B$8_qa9xy+Gr{ey}>c{V!1<3V9U_b0?U;h|-sEsO#{@p6+s;hQ4_50^ERzZ6E z{JSy;puiX~;aVB;T?Fqpg}$+)@Vw-#wfg!IKhXx6xf(E;6rt3oR4z3}FpQ4qJl42g zE8c7j0DaG|OV0zrZoWpEXP!r+1#9OaaJ09?`D`72`1Jb!0%8Oh8#kvAQ|+|IM=rqa z!P!E^wBWm~lub4wEJ=_5=eybv{ZRlD9xuv80^2Q7*(Sxg($Mh&;FNBl>Bu*LQCarO z&4d7|kLJDQ^zl8P+vss2kscz+@kU4~MjcmhRRrXB2{Q#17(Wd%DtF=lb2iiXvgTYa zU{pSdZk(*az2!+eV9vChzf}J)5-@|NfhO0V6LZ0E?UQcB>eBVi|G_}gVH5x1j}H(o z8<;h)5OtMIVA~|vSv=z1kL*OxA9L$BA0X+)Kx8sc=l@y{&0)73#V_qvNVlNj&a)_D zfdY~LbO}P1P$V@6)cR&=BjY^4Q71DbGjPGKq*!^YIM*Hxn0v6YSyJx zC!OG5!yw<()Z0ehnOkvM4O3$r-5)8hFK$vbnu-dz`Lu#19_dQSbga8M3Yew)_wl2Z z<<>?#2~DC-GRoT8+J>s$`$?srZ>i@a;UE~F_WBJgb@b!x3IPhQHmBZT-+x)Fsf^QP zZ};QBSsqI?zHVd)1?z8AoGz*C3~o=;;m_>)g>(B4#1BJS$? zU&|KQuT8%70goHrw_tv8BKu4fTavDCVjVfzl*>Tue13V;dk831J7=RcD%sgq6{A$^ z)5bMV}BO6ckCw_S)ou1_&p-#fRn=pRk z2{J;Im|^Paq4Qu{9|kgccFE>iMjMBWT!Q;zU`Pw@f9c~Saogufe%rFq-7azDa=#{5 z85OdrdQbD%g$U9quy&6;u;XDR#v^ezdl2<%ACzf0Sgo^tKNBcy%k#uK7>txDyir0m zbWVKDR-BUcnc3phV~LEw&^PQ=%#%H2W!VeBoVB^MJevYlXi1vC%)C-#O+TmEWF%Vu z#EW|Kq&{x&bpw>rJv)O8k@-Rtm(j<04H-qy?v~Ew_VBe7@#sOK`V~g!-WNiXe>HZy zZC^LZszElT?`Pw+Zr!dAL-}`!I%2+M#2}H!>z3($bV|e0jxWBKc$AkKc?&$hGiv*I zuaXI#{+rwxlZ;4zZy|KjU@N;f;)FxkbZ1L>n6~^Ih)X9t<1e@2XX6Pz z&Zzx3W_~@2c;~%uz-;DNB+Gt*-KvcdPX(nQdNZm|*-F$agekDNzln0t?0@D= zrk44T)Mj%+_njXtu3%4>Dz@>4=CFAl1B74F>J_eL^-HBkGU;V{!PtEL0Zd zTwYeq+U(rEbC@AmTB{Us>{El2d=S87o+V3Ugizi@c1Kvj+}EvNQh07TqYOgCKTF@o3lJ!8n&okM zNP?8_$n8h)4YTPx-Y*6D`p+$2V`SPdkpDX$@kXl7Au%F=oZEm&(x3BlUrcbP^Eshh zL+K_e`8!`YH`TXWt;Od!<2BPV?=}znw03tMt0Vm~e6ZvsTc6PBFkbTX@km`YA>mra zo80)n1v(xc^*!$o)OY%~B>T4Sh2MukI4#>Am~anG^@klqAR1J8X}W_Yx7#tTS8bK$ z^$&0Qn1HYyj!*%KRr2Ms#EH_7c@Jo|sDr#>#QF0J8hZ+kI5NM0mG1irQ-p?!Ut8Cv z<>{xNq+>*6`@^exQKgi^d=$T0xT0A+GUwy*FMKAXc^}-kxbYq+o#%a0#Z=8(JSBWxK@dgMK?D!Nsq`{cG!X zjews!$3wQCkNXKj zBR_)QS3MzLAs@P@2`<+6Mh4F2ErW!rS};3->*9N9raz6R@p$Xj*EfNC_rmA10t!O4 zu$sC?_^D1MzW}C9(BLrt>S9I(rE=rhmKZ<+O2$n4!3Tb2?t0+o4k>_?Pcq`=ARo^k zA2|{^+M#$lPc=-h$X>`aEeUz>a0g$YXbt-0v_@&B^_2Ug^qaW@_s7GXEb@c9X<&kz zm&vi1k+5=u=w0hx0z>)8UAjC?&&28Z_d`D^k9lsQQk3PF=tYD zMnr|-F@|f0WgI?Bu{(8VdUD67{p8{){2gp$na{DcdGwnxGeUi+eDIc01GO$8p^`Wh zzKCl8xHfBq?N^WWQbIB?<2>85O?oxUxB-sDafcT9{7BAr=+uo7obIj-YLtV z66;qKrVr2LyV)(@LP2QcAUk~Wa2Q}j2zJ#{!J%i^faiRaktL;LQR&pEG|)jjC2sc; z?OSZgPiHr|UtoX#k)n9PEx%+6;+R^gp+rGwHJLP-KtAX8@tHyznd@M04^ zUx(DaV}@b%c5 z({Pfx!(~j#$f!wUUl&d%2B90RP-$jZNwtcYxF<#2w(XKPo@`!nJ^~fYoJTq&a`f@t zoSP&%?`9d5iDj(KTyDEr+rF}^H9-c6QiV{2B>DS?Wc2}q$G`7M@U|r~8MeM=_a=@* zXT@6fu>7Re-&qsUGPY?IfCp+z@6q*9XVznR4>Dwzc-t2Dce8#!QW6PdHd^z6HfG+v zs-LvxdIK&w-)FH$=bahznkrj`X~3|p60V|t*MMKuq~ysn^Zwn?ELtUA)=!Evo}cA) z@cu{Zk0J{jHhjrrlW`b)4`fJTE8f4P3HlX=9Uo{w5BSB40(N93iY++aDoX=JI~+qD zitXlUjsOa3R1h={nx&S2U3s8+_`Xn$(}e8b7oQeNFlw+2fSjxmA9s0FPiA=a#!{;(&baxBR9EE{dLI`SwuVUDpT=rHZU<4BY~ueWH2t zPXe3A`yqu!EfjfY0zCv+7OUkyAdNJXpuYC?=!+I6VIiIsmi~z~V(kHRB1$e-WC@*N zYmL7Yjd$>ake0&+Kb-PYj)CRgbXm^M@^SpZCSnpR6$z>=G8OXCxyv>9dq({vhU^71 zwA{9(^j+jl_s5?;P_d#yZ9t%LHg{YbCb^D{oTVui{`X%KaCG30V0!j+q0~)!JS0sj z+;{mftEu17Ai_l?D#zcm>w2`~3^E5skEPuv`QwDWlquF%#`z8q(m{ni#_I5Cm0pj> z(a8(4lvCn<3tTdJA$c6O=Wl8^w-34C>WrqSvfz3|q!Qxdg%z>|(CE&RSW$UWkagm9 zj}~}fG>*WUjBO8j=hCoorQzL=xDfrwk(fA0aWs9wk}gjyMK4hnHZU1x9R0TRsA9!{ z+*HnMX;SM~nX$RM`R-rIexi>t*lZnNZwr@|Ta`ZfUyEPQU8fvI0yrBfG}R?O`Hj6a zBsM-ud`GGmV#cTPo#Z~+EaR>|D@Fm79bD(WW|CpyvMoZ5O1P>q zahjJJukol&M%D}1!~P?k#WTw+m(~JFX47!Sy(7Z9hd-DO@}#&7x~UwJZ?#T+?yW!X zfS&LL4=+cKXyvI|zqw%WKYypgTQ#twzM(ra%WG&XuCNWk%od~Zuwe^ey^@1Dd8+aL z{l8d%KA&1->{N0+jG-Y#LQMLD#8QoSLGCJ1a_)rc@PpW~g^-DZF*TS3hFDBGVp4b3 zL24#;^6S%aF+xSC9CV<_D7%%ixrT;@2(#62(QrIIuhU-WV40{LgO>|KNFz-nNGQy_ zCa}`bBQ3tz|C5#u3K9&2Z_DprePR7Ab{&YBCQ@ZZRTVs>JRhdO*22uf!a#IpD03_d znWQ=*enHjCZCO81H@K$^d{|(aYJ5B&9RrZmlF@X}fC66FA!GQ3E-4n(vj8`3gwNcSYwG71{?}Lho);TImuK zp)_9Z=r}C&y6o5ONuJ}sY_#Y*AJeFz(m$OR5#Ww0lc9ip(bB&vxb!ri(*Yd@gMHoa zhYrOka-ctm9nW)(^0=KqX&@`#ceZjXwQBg?Ne3i%o@%!_em8ss<$rnnEw)*!q6?45 zeLv}sudmE@?J>R+Ob)om2Whb1wl<|+KXSQlOtXRB*x<-eQwIS9uu>yqV#ML_Tuf0kUVl#3>b95W@yXH%gNVe)o9ZoL@sRXQzWi{3x#tcs!L#=U7-?FL+1;paeBeX4=pR zuHF_B>nOkX!{dOHZh0*;@OfOV{3W65E_FC7VNkwDiA{+;w~e_!yG?cP;OF~tl1J~|MW5Gpe{DqVhecnmeO&?VeAhbmIKiXo#*Ufv z9jE@@3U(rU)Et^i%Hv9foX{C{>;nzu0!;6-0j`G{Ef1p399nv^pY&`^C(~zhICIM+ z@Gua~{oz2d(+{8P*T>Nodk9H$g-HXCji(WTX96Lo2D6?XuV0w?Jd)?llQ#KASkcs{ z;lfWp50xf*PPtUdZVKaf2?QTbs<7%YDPNG=l|J??g?QF*E!dMRrKU;TR+mIh1@4X# z!BHZLSc=0pz4|epKaQ~Me#ix98rqdDy=B3SS1M{7mrQh+R-+InPS&}0&EF?xMke*~ zp#v?QElpr1zgxGx&aZO$xNJAF8p$&4gkoX9V8k}O$FcM~CA)&o%>_Rf0EVO?I)-kO zi(-%(f=?^^1uKh5pu(m6N|jYwA8AV+_-jV5JZ$6kAOPfIe;L?^*`T*sm|GZfpLmy5re6TOTHp=Q*F=-@^jT zkA3%823&d)WZ(CVMs8t!oFZ#)dqK+7DPDesD3DpL2jQpl<=M6<0|*#K^(KRF3VMyw ztG0*RNzz=tyGL-0|B!J&X*G%Y^8DULm$%<~=GU^6M^WPwtoll1#3JCm_8()%S{e}z z-K8I|C^Q@M9gWpt?cnT4NwX7Rpec0zFuM;G}BMy_)6P0RhUfE|S z0kQ&`*E7&&x!_|4h|vQ9GF9lPZN0>(2!pl¨OpK-QI|v2~a0Ljxb+G3PW8ky})l zl$Me4KS((_YV{v{W)n}l621-0=(AKHl323y@7%o>2>?#gvJ4CmC^{HP4T1talY0su z$*YKv*;ZX$U0OQWvX=@rCbgvQ(d^;ssyDH*wdH?zAhRBggsaWc;BBSlqbJomC`}f= zjERMD_2Q_&$y@s6RR*j?SAPYc-p%M0cpgL8jIR3v@ls>keciZL};O?kr`Ty z0q*x@3=|MYeO_8jx(x?GP+!<;#28r?j(s@$TRuLO-_pTQC+jSh#1>|vy|BVCEf_y# zDV>G>YiIyM#fc%oB~B!FSx(hZPwh?(Y{H1ON$5cF0OXJ}v%dd{FV}^gTn6K||05zw zU1@edQ!xf)!~o^-k)=3$xJmwtzavDjnWuz5&840O`}6 zg-Ye|K_l?sDPs<>G7hDlzIi)(ioK$&}n5Ku1eL~L|y zSmi_j00edgbplqzNw_eMN!g!qj!w+(>oMs&+2G?!#995z37r~d2FNc2tnU2P#D|p; z3Dm~1B`ZP@B?YRmD*$zfKOe|ROHU^YQrf@2-V@=?XyNv_?Dc#Pb`0GQ8g(>v*=?4p zgsKfN5Z!`EDM>=x%uYqkUhnfi2gI9QPdCEkFu!+ak29>VHqH}yB$%Y9QapnYBqiNL zL+_WInot0>`d=47H>hQJT+j7MIH0)+U9H_M8q~)xZAqCR5sl`wYFSuNUBO`#-SxF) zhZX13!$uF=ZA;{-f3|CKOkLQB9we|kEEBZtu5YqPPoVDO2a@d zB59(uo12^5yNK~`0>k;G|2~wV+6`thmfQ~^L>mUG7=_AJ*wTd{&;zjAxW;nip!NSJrtW7z27xh!QmD`)3r8`-D{(5# zwW%`~A!zU=r5V_6pf+v?GY0^#6M2W@7LW^{CyNRf1xrBAz@I&sP#YWx<<(U?=Lpj| z2(N@{?h-bWs(n3(;k$0C3Td`k>b&{L|MQ0ReV!`^ZEHOE*1}A1o+Tkf3!c9L762Jtv55iOahc@_0a@HKt0+%@puUSs=3jw|kx< z5NvpQ#rozypZkS^PLlvm`eS&@K+|f=cm7`{n`Mpn4>W)@O**g(V|gsnEszss0Zt#u zy57k;d2)A?`CoQQ;5ZOX8LS1 zi~TL6*NR#*@3UYSPNoh~*i1HtjCa2jN?`D7Vx~6d9HsOESWdrOZzBR3Qo{E7bXyMR z?bp3>bBzF|;f`_G!F0iHzE zSw|e)x|#Z)8C*Uu?35vpm6V`21F4QU33!d33BX7oDwsI?;YTB5j<<+}wGQ{f>s`!z1QFF5DZd-qt!h@6*daU$DAdTLij@s0|`Igkfo=XMXGG5 zA|ei4_uV28wdX8YBbL;W@YJnw(PQ1OSL*5PWlnP@We&=3<{|ml>h_6AcKYAlKTg9i zLc%q5@EzHCSLP7AZ{KW&apQ*>n4!ya0=DApu3N+UlY%)R^{HC>C%-ZU0o zuv9Z}C1T38T;!5ZAgre+HT#0AqKvM^O4%U?SMhHKh*Be&RTd{3j&t}eWJnkpk0(%7 z|HUlKQjdy^g)e0t(f2gey8XU)+^vmGue`~UVzONWW|{A@Wc2-}Id?e`6)ET7Bm-OV z*SF^*3~q`*2vgnl*=XCa*}_e$wW;^-oy&4OE(h)0BF{eyNB)g^-`7UB2t0~RFi3>9 z-64wN$_RWR)#at7zKHZP3N(i_`u=P<69GfK{=7LY&$m5qvSkLPFuy;V8QrmBPcf<8`(gjMjKZ$`>ZPIpnh+4?2cO9)e=7d0!+< zi;e{XEd3XLtIXEi++_96I=jw3A+h*1)JRIZ2*oU$?gwK`jPzkc`Q#&_~?&F%D% z1jQ7c(du#^j{piS4JkN~?&a{f{P`0h@G*i$CK=y(ZnI5~+#ucjy}T)8Q4;bfb=7tp zB%O1_)V4b}UIvGl%e{89ZuCV-m<*}c0zQt(f@kMA_htrvVFK>)+)LC1cZK8PGH~ML z;rE#z5>B6Xx!wIf9DOpgOZL2R-;E+SY1H&EH(n06=(lIp-`V|Q$6oVsDJyVR&b zt>X3!H6u}-ac{88a6B2#>T7Ryv*oy>x$aBy>`=Y>tmmF!H=8p>~7zu zgh>X#0&imp2Q`JPnP~Ix_xGn3%&?*MM~4m9eJt1z4D{=oYI?orzBR|+xrtkz z?;$`l1Z>Gb20EJoaNgIclsgFOATli(o!H^5tuJG(%M6%*y9|Ced@2(z_arg{iKfHW zul(-n3iFA)AY}+)4*xpOC*aDI+FpuCf4C6{wFj*GK$n5eSvH|M3CqLpjJj-I=@DJw zbGq}}dKx+M3Zm^kK10y;I)PxycF@^6_`cT~eSE}9^or6E{!-foo^2TgEkBBpJ{c`^ zP$f{OqEP}9ANU*g>%|@w$cFPicb~{8XG0VF&ycR7wB73pe!pt>16%D)hCUweQ!J)K z%rFzaeKFWQ*;4(L5#s&Tba{G)9)DFDQ0N)5xg551Mty%XC(uJ7TJsRkQq%q~TpFoj zC@PMxvs$ixgV))zStKJ+qu@mKD&ym;a@^o~Y`(m216_*5BS4GY7e=zX9p^*3J6oGe zH}?aQW|6j^o7Wv?lJGE91RvXu5}Uk&p|&^)0>npI%ZUNg4Q-P%!{X zJylVC75yYqRr_dpGgav@LBHzL`9%gh_ULidd=J;;rA7k-g3mXvxARQ2>a^g%KI3na z=I^^|Ih%0xrR(*lhk2zI_kl~>H>?u#_68X<={@Q~t~FAYFn?Xrb@zVmEYa`Gdt zSxG-Ib!Hg1dCxb@o^UzdAaHFkx44hY_Cu}fX3MMwMz7M>;`{c;$_*`Q67l@&u6_ik z>D(0|g6QUuWgdG&(rtIO9&>>xA=Xw^y=kuXA8lzTdfV)uwc_7QO*_f3dDhfKn003Y zP?OKso3@5`w6_!Rxx*K!x|0_ROpv3$6hAfjLT8Y(u65e#$T}!cY)9pceoc&IZjoiy zt+Sj!LY_h!HU2Alz%%LwTGI(RUWX)J{Y6}#CI@F75MPF`_CH9JLRXWKd?TCG`{(-h zU`^?s&%Nfj%@lwawF;mtdOhs0M+oDCdctEbY51=1?*KZWR-`e?1+y4CJ|b|@;~21= zyd?J9*-rv*j{MQx0H&~ii6 z0$k3Y+7i_j2{y(zyPc0_GcPVuV%9k-u$2Y&$FoEv%hO_P<}e-e3vhd zQFl0w9keYQ-Om)qU)EL%oo~>9y{B>#@byCXOU77P+ee4HvX%|?$D!9!5>?$@T~{>KX{Fi*NN3$_G}+jGfr3r0t*sSH#W#?O|3C&C zpioV%GuQu9smtd7^et}nfgG4eufAZl|-C|nSGgS z03R$5+|Lhed2C&187pB6c=v}Z?C_EF35!NHQYxO=I{X?m0T=_g7M2nC0zLvoGKBC{X&o&JrHdnyu{iR(lAQS8v0Km31;3@xGav>%3>XcyG>gP0Cm6^^E;7e4fdaVMXQaLLXEz;;{W!>cm^rVg+6Wrpm^NhC*>>+{8z`^&Joa;^<6Y{ zyk@8UpV*qnjI?>b&nj#Ef4=@|clmsK^w|l5l27p1Y%m*&!J%93ge9-(dddqwAg^&SMfNE3uI=3w5R!NJEwfkC@lxZ_s73 zDFV@xAFX@^JT9hqcQFfK_T%>vq}?uA)gA`5elFh#R+g zJga&k*tX=6aMT%nbb= z7kdGpCsH18DOveiHGT+y&AL_rCp~`OuMoZ}Tyx zzWABIn#XY`ug95rizyRW`&65D&GbUSZ@K4i?q750k0}>d!QEJ>5c)2sU+}yKdu$5^N5cl zP^RBdK+vlVbd^ttQ>?DRWp_I}z(CqWYYvnF^web)sxfkNc{hB0`yY1z3q6%fU7igF z<(l9$xuN2+6`mU{7-UH@u*8DadoO=OxnVIOi-gnU4-`rChQrSZ;^;RZ29k2*Zm1c7 zZC3f1382D0XaF(59NXmJM8#)&b*{E2kJ(~kC>9Ty95}7-28Ftv&tvo1E}p1}e1&y6 zqXR($1iCV0#cK^LV&otTO{uwGc>Y6TZMPg~8x7m34nYWX>#dAYdMMrj4^3mj@qocB;WV9*%O6yMxaekLMoLH%|_4W0r zn97_}h3I-F&e;G-&ygWh*@1K;2_uJFrpGOPLQ zHN8Qh23_FgFgtxB%jpYPt2wFFTMhvn=$ni-gcX@Y-^Pq0nk3An0@0 zuH_asI2fiI;P-ko*l02Bf7B*KMvRJt%Y0K9K}c%8`85dqc~ZCQY&sG#Pw3;C2#Ume zJstlOuH?~S6qr-cy1#%{GGv9z7E>dI%O9@NC(oJb)!MCj%K9#s8fgTdU(ek}`0QOT zVR-9&e8B1M^vL0|ce?`A?|P079-1`zS6|#IH%tWb5W9`)O_hi3@yltgH2wS-`TC#l zTiBbIeEFU0#hkr6gs67}e}~6=%d?$ucjqwu{_ILSMdEt z#qgMNz?$256D2lC3UN`4J5xhy(V~EF!)iXxjis@lYVryLFNb({0=Z9|W; z|AizaZWLmw&IX5UoH42~Z2S3`b$Bd~?{b-=6E`gXKBA0)jt*o`f`J|owDNjmCLT>> z5?o!qZ9QKxAVy|8bX9bA3fOcz*GRS4Z;{T{)muew=fw{%RU0q}v}x&89vXU&1ek0H z$(J^Qg89DDx%nlo)&N54EUXp-zK#PlpXhj+JdW=y^X*Tg4~Sc69F3dx{I{PDr}-<# z(JKG^IeT3;+V=De_#P9PthVFT zDjr{^(+gf0<&hQ?8Y0Z!wA4(DG(n=-iD4R!uEV2Ol9V{|UAT~-5`)J;b?=C4TmSpV z^kV!J^&Vq%3=MtHt-qJo|GM1nDdB>B;@r_6iF|icMuC%CMRVj&<>NqN$h(;G>=I%7 z{n0c6u5M>#1p{hovD6Yfk&r4f`%jw)0`G4(xdu_y-~Yfu^37C_Nst1#l&#y7bLs~){ zxC1OyT-@B8>>>-I*|`!dwM=vBOy)&~*^DG^0vVBEG6tW@Wj*v16SLVz2{jRyFdihW zH%AgY7@_t9fP)00<8VSUbsZFue2y;*JmUxLSCSUdB$tctJw0u{=j3P>Ae*)m&hTYg zQ8Po#UC2@4v3|S_d6ZHEjYQk{p0E5an_J;oYqOk0KDVBi=#3O2K*pl zn7h=x!KJqjZKR6yk32r}vHI}vw6bF8u)qOaC0NCd|MRzk&r-cDt_`5(oKU&@RJ=ln zbRhV!Vcgov{B;eDWct|5m2atu1eb9`6si(By9Hl|Ov?nP&^_i@xoXO)_-CM$BqYqo z?GkLq^t7Dy=L21qdETV<606YAkppA|K8Q>4LN>Degi1s`15S|43PJthDfXKU6^+F> zfm@p`HhrPM7P$yaT^twT)zweSqe~ye7@_;%K%FWxGe?PDJ%>)5DDfhcmBw#JbMsPJ zaewojRG=^@C{SpvcNSGb?40*Q%g<+QWXKP zl+bKLiaVY!D{*vVBeU$OaL>sME+_uEOb10-=3+skk4TG)_E7fIGh;HoY(zLXRIE3{ zF&Ch&?{1wGh7>+c9+)kZ|A$}xe zq|$X|owSYvNOPP+gQObE(L{z=(_tFRbgWxrK?^4mq{7avLwM}|*~jdRHocWraS|~< z3_L2U(K0lQkAw{goiHBkFhvr0N!WKKi!Z1Cq=y3)-tgaVUbgS>O58ai>+YD=gn$i! z#08F(!~e!AEOUFa#j@{3CI*kazZ`twRi3&wmVf@A-jbr!!mRXq ztEkdCQ{1d9&?3YuTde`D1PJQ@pL1N`Xjq6^7`;&Ueehi9EB$U@ovIN}eeUF#t|l#_;9|_|aGWI~^rPKanp?on612Hm z!>EK;btzM_Ug==KkIe`Igosd=$!Fq6t*?7rJBeZhLTVs=4{>$t;IhLMY5qMg7kc$_ z@xd4N{7CS2#bVo>3n_9wxpDBn$14><<1z7w@PC^pYHf-@G4F0DNSii*a9x!+Mo&oz z?Gx?^*7)oNGcd4o_c)*3r=oxz?2%4gY$p^!ZtWuw`W$8A5F|u=`>-uKl%K|t7VXWM zX2|n8>@z{@P{j87`E0P*Itgw>eJ}qS+WXVx{avrT<@a z=(l+sK}Z&?#0~%aL&s&eh8!EVH|mT1t8`2cf`T%$Ib^)m){AW6>ZBOh_M{WtwO@5bnKzlh6* znnD$OJ-^75osBZM9$BSe>1S zxwc>D_|ZqMBt*EGLJveEtv2#4Z1qiCU#$o@eSkxRMlV+xPSO)} zwpcp2Jvu^!(}>$?tS&L&t?4QOdXnjuAFqWHg0N>q=PCQ)Au*@AzrgYhLt=dVM_q3o zI%o!V{KND(yMc0+q9iMBJ3BpNy%S~DGYMZ3Z(dqh*K0;xR>nX9PWJ~}rq@_(lX_1T z6%})H^SQrhKmC7K;X5|ekNYT&z}IV&H(c~av6PYYtQ~X&3KE zr0cVn1gy3n!o1HkM#OWl=h*TF>*q{X22%${Da&Uj)%iTmY6?)OIitN_-sA~`oTa$) z3I*o7xrNqZoc`?i?t;oqZqqqQn6$s$@^O8dH$n~~NR=JL&^4we&Na)x248=Fx@

    s{MojXuZhF7Y82-v4kD#UNk1zy&h3{-X&prN9+h&otq45n-72+xS{ z;Xl*X&J^|Onj}6I&nb&@l3Z**K)Vm7#b^OW#jqV5lN+jLON9xdWw4iYte>57+Bkmo zuC5IY^HsDvUB7JbGeP{GjU%8F=-BJ!;cfvN;0dWvpa$g9>P8lTEOcH$qX@bDOsZaL zjh!lrJG?Cbi$s_bLu2aeTT_O(dixd}0_2|6GaEfG5s}f9Yq;-0f0=Q|OiD-l=Cm@zr~ zwfRtFd*UTgwV`$NbI`IlgqGnD?IoxX3lxB%KZl^}CbeXK**BLR5)wkD&;q1v#gkO? zV#Q0j4nD^PY&h1xmg_Wf<&_)FZy@I*%)FjASu#8?dF@9uw zCJ2X#eE=F23$2Tj)Vpx>04zwxmyfgrx+GyA5FtMh>@{TC>A|({6#O3(nFOTOb)!&V zhbiWAMVTWB)*%vv$;Wcx2LN#Z1Yk64L?hxMb>M%@#dsn)OVZo0)J6RSBslYc3DfDf z==fn#R8$SHtEZ=7Y2{x^Y;$sQ#;{JmNh1dU}1Z;os*OU#s`kF4yRHYP0U< zosuK>6em$e=CB;T z#l@oM3TUinq)RTWYm^ydxAn3_2epZo@h@j0_QEBz*rV375bBV`MM#7i10uSO0jqI5OIYChJQqe{@4v{k}c3tPJ4frJq42YK|$&5)Z>}UV@|`Q`D?6|7pj@$b0xiZ%WHmazNmkLbp$s)^kt^i z*YWI3eY`@w?Sv*^R&RDY@zRf>h%Yl1@T2v;U1_yU^EuC2<*2&j?R!=ezRZi!&^J6n zmYm8=o}#|LBs{s%gv>5G_&qxnhO`tZb>{7o==d9N( zB`wolTljBwDwF7+%qnfR$1bj0XbKU(JxTO;GLUS5x|k-O0oY^2JoR^RfY-a83Jf=y z7xo=f_6w#O7-o0ID%w_Mm>kCx2(4CO?RB1u9?DD86>-LfzG$v)BUt|~e ztC3p1{#mZ9v9VYtuIKeAAYdzCr~mcKx159CwX&foUk8}J2)H0Lx!st97MEWGgh1Q2 z%oK+4m9!lS@kjY0&4r9Pfev=lAc+RON7E35K42;FMC_al4U>l0B5FAmf*yXOd6}$K zQEZh}SG@3`i0_0XjewH(8$eX_w%j#za*O%xi}j=b#pzo2EWo({Q-V5zlwL#oYBY1t zp!Mq#K!Q@hJuQZXuRS5?VUffkFo!~eSz_ws5fuoyL8p**MeM4C-1dxap#+NnAY;-~tz+r4HHhF)ue1~#Q0rCgg6 zMt(Lw(I2Q!4S%&!7#+VvsSeV;jP^diox-xg^>H1_=$_!X9rL>_uVAD7wi0dUy;r|+ z3#1SD#d#VUiyZV;5-(9)kZzi(sYHK{PZ2V(r0Kcx+ZeXiwJ2vF%;AX(A9ZeZ3))Q| z-|UZT>#p)_X)(46pWKFC{B$Ic%yrlX2$ePfD+bqTWunZv75*)BgA>Q{gN5yMjZGgR zj*E-TjSn@2=+CLCx3Nu6Sc-qLYqW4%{&s&=Uz#!Q%+%UN<;&mi(%PrieeU4rsf z&HIu@J+&Ga*Lt1jvm)~J(jSt-fg?s@rg975t6A@p&jS+4^F9)|MhkuoHigZlP7~|e z5wPv<%hA0`*V9T?B%g_q`g9RDv3i^D04H$&VL#u##dIw^*y#2zS5U!`PR6e3M#?Z& zH?(`Mx%O&A5757U&d}^VFikC5@8$mK?)I= zy?Sz=-*wsmlXnu6y5IemR&&EHUbD_Qk_2Pg)mX)ZH$Db?1T^^YI(`yDC2d_mL<#U- z)g(l&HOv*dGZ(}r$3`2Q>zQdbw|A+A)^$XiIATAsXxF=r9hKDp#VgBUHUpWy*GpmL zEt^|gkAZ5?;?FOtra$fZ`V_WS?Meguz_u~`VPgv*!a0@pvs*CF^z$exa63K93;&W` z>^O7QrPVEQw|NspNtwh^JMO|i8es7^?A-HW7O%{8I`Czm->sVpT!~a z82GSKPpuk#WjAQZEu(eWnwWBCC$r*TSo{PzV4-8?|Gx^fI*K zsm4S~xO&%s14>q7rtrjV`}_bFb)nJSkLi*dB$=R~qC7l#jg!4Aw1$^Mp@qm`h~P)u zxe_p%QOELn4Ne5rd76)x*O1S}($#*H7cAI369s~+o&E#;C*;FS8N=mpR1(Og~ZfT-dQ%FsV;2)_NPLf*P=yy8aOonv{Ni zGRhU(|Ks`ZvO{%0lTBq@{usqYD$B*N)bnsEJ4djcMd1_rgRPkFGBPqNVVB!mD*Qxk zKkS9m@N_@xkGmd2{xldJ6%XD7_FJ*g9-x~6 z=i9@=;HUD3c4(heQ2*K1br!-7L=5R@@rx56qc?uL%XxcPVyYnGRStT7w21V5YlqRu z@cL--ka};HJ7cgbSVOn`Ip9i{%h|?2l6o>FoX%1V-q%-@m~+<1Ddu-8sQJ2>s5zM? z4;gMYu}Nj;7aw(O9U~GZhe^gB28V`*&RHw;l@T&Njr^3|P}w`WjLx%?1601w&ow!T zrtj{kmA^PvH#1YFm^xgFGfJNvosA#b;B-Ax{!mz?oGyHN%GT&%am&n!m>s2w^Xq5? z(vNUIl?F6qs|hhrG|mG5j)n5HI>~JPz5WSvz7+413ROVV+{gO+WomIR>rK+tB?sIU z{<_^l&*_QNXw7SXH#~|u`-<-$?2$1ksT*5dj&n8uJy)N6{@7e0&xXb>r==CWhveks zg5KuMK-JTD1Wh2LfLNkhw0@I2O>8a=DYaTOcC>hlhgM%1rJA*G6Yas^*;!OshI(Me z-kxl+<#Elf9(fvTtfuaRueF9djiSHi=6EyuBwEb$Oh?ZYs12gM%Sx;Hoe`8J`1~#a zK|HYgbfUhEQM&DZOg9i&|0?@rL-W+Ml zdA7Gzf4F+if*84Rw;RcwY7CJd!dY+28Wi>oFYIG<`4#Y=ugLG|*HxSY%@ zb~>9kPnzH9oVMyFlKS;2P!h)H48zE_f2-`zm-ROA-9vXwqFm&_*pR2eS-I-j*aYp^ zI5`Mn+&L$A*QWsFalR5E_>wms`jP>MvPmnTlCi**Vb4wR83Mhhs;oLne*E6b_mkL1 zPZ+KXD=K`yw3dJVENfBh$MTtni%VV8kh^pH{Y4G-DLf$|LG{LN=8u3wvcfl+d0bvi zH5y@Nd8W%G*EhkmVCV;=Pav5`4MFLJ5l8Gq{Vag8;Uh*UEj|H1Q-^v$CCDF1QoYdK z%-;#___aD9;Q<>MOhn=M#d@l8LS4F=bbcJ z`}VJ<$O1{UEW-~$Zi6f0qk1Z2wD&q)(KoCAG<>)j_Or^48D|uEUX*pu0WN<1;Vy|z zH`__{dfI=VS=Tt6)P)oE?pdY~6UzUm=|p-=dGLitiiyQc_-xl(I|>o5>*_pW9b+9G zdDXkZR9B+D)SUR>XwGz@=ScdyQiitwfXFX^tx#>P{kZ&id>XKXkdSo zb&cGsvhMSp$clP;LKFeV2@?7QkUiuA&sk1lP;#0#YDvTE^_)*Oy>ETuPNy;PDF;MV^?J5tXi2(;4?VITuAFr{HQ*MHRTmTpW4CUZQ;kohgE=(Go zFW~1W^hA{Tdf+s$i6ki^N{J~V-KzK8Xdn)!MkPiGn4ZOx#Xm4OSsV0WJqV&hH1==z zT-c`8c!yq#6Yu+&d2~wbyB8D|7Jeau78AdDB}p(e6i!v zdOOECQKmO|@LT>I!>-QXJnr{BHo<3zVao7_ll0ZhO~#BKz0Z>2C8=fLJR}O5{AXUV#h- zeH4A1ih!VCgz$p%)ok!FK(!xvGE-Aq>-^R2?1R%7|5%z;2}`eXqwq8y)%nr%7&%q0 zy>ao>m$EWf88f*H;s|83r#9dMM6K77pVWKXE)XkH>SA=zQ^?GU>5mT8#>rzzn6W zSU6tg7VfoAxSCEW!O9Y~!D_p?NoTAW@6Tk01TGLKYP~xT#!diQvCh&|=h@yo;j|TY zyZrT=I5Sb}+I$g5IV9H1#6hMeHP$sy-dcBfSx*Vpr{8tAUk&`x!g-IHdAu(D2aokFl%8+%@Odx04KKfD z&m5_xcc1(V<-^!}i4p$yn9=2PqQ~|nLk;QUdZHz6-2u`uz+tjnGFyp`3=6kc7a$6p zZ~2v;o5^qW)WbpJ^m2{lDYb;>xxI|(C}SqIMblhMMLtW{tL(bMhK9@blsE;Tqf%AH z;dY;{*VUzHg#w7A>FfwAf~5cEz|N11LXkyZL+S07BBChwEUa*=jH~V zA8Lgb$GzdCqm``ULUJi4Xh8!u=S*gX3k3i3>)1>oxoiwzr4FjIU|^X6^GxY^-2S^W|lU9l9b$RNdFF zM5~G@^!0TrQcBM-QVQ)S|H-&Tdd#YttAc665ivU-g#n?GWQEhl;L_5scO)e&j%lwz zOlE41DH<__Y)|;$CzvS8D6gQy1(Wh)!cZp6fIz00PvBU47+=t0>rEY>4T0vHV(u5d;#EgrufUeL!&(mt(-b+?5r z%E8x{m!Y9rzQbBr$l~NJ>ifUYQej)L%M~x`g23L|1>X`jd{jl86L{{(R``pm#>Pgk zh6dobNP{P;CRx^bA>_$7^epSQ{}Z=!>*q5M1fd6hDlfN#@6Gir5yg;eA=4WP%hDhd z*SRZA0qym^m1R5b!T4E;2TCSyrpn6paHAK`Q=aH$E{F;d&9}^*^%rYD$?#sTf=l4> zCsI%d``uob@Vm@kZL98b=)_Yf*3{0u8TgW2Yqx1f?R(ki2NXtbkI!8_%}2c&@^*RE zm;X}ft&B`i1WMR)EY?Oj2E;gp3Hx>E|789a#NgD(^}6A*ehZ5W;@4izdFyT|vORRY zKz}|+cocTtbvPsO4Q!%ySf#kU{BC`qOTy=@_U!Db8aGN5g-8m1amm!V<_ibitKA`S z_dD48H);W+2a{Phu{$M2#bwi)DO69X=KTtDN*`NFeE8D5dzOL3Xt;3Jn{YYv9!E7a zIzutPy#Z$%27|S$9Kpu6bvX>MP(Z8|1g6f_;|g?$1Ovy&O8?WCy+HffSKfT1eo01- z{3;lXh0jdE^HhvkGTYc6c_-`vZf>r9D(){N0!`^}(%6zx5Hlqd5EcSh2Z0$efDlz` zB^LFAlr+ZEW|<$U8YW}!82zO|Z8`N9oq>>OH0t~$wii503h&>a2Hp6WnLXke?kd5k z=xuA0RPyLN0dy_xyCx^ z9!$@(I%r6t()7dcL8L4YM$8`ma%3h_4cu_qMgvk75Pn#tI`-o13qH{|)`X~5Esu!* z2?Uaw=8p#91EfJBdlezOZ^@BKTg%HRtf}jS zfXhJJSxda}zw=$@crge9=VDn1NI4FE7N&@W$h5NUJS*nR{la0i@P z8E3~x(?4^bydM?%WpJYiuIV1Hvb4>;6j?;Xk$%Y^El>9BpBd^BeU3*%n9DU`nmcJu zAgK6oZl86*LCyvBGTZjlwr%%g;P_NFEq*w-P{CAbD6N}w* zXdtEV3u_ZhIKi;Nh94&-TzI(?T1Vm22bn+IGDO?6=0Y{=um|BrrVi% zJY$HEYp&XCz@$Vw#xH45rTse)oYWQ^!LL1a#|YP)BU>5tY-IOc`@UoOtFje3F&x%orEG(#iLD3zFao+m!UDlx<&tfDmVsOU)a}44<-W@^Yr3p-j zUdIGlbPE2MQ}JdvyXSjbj;5o8%QwjAKiR`-F)P3=Qx;^sdNtN%MRT8uwQzr~7Hm`H3=G)_BuXKM=LGHjd?Y4KQ8HU)MQ!YX2~9x<_Wp-3`6qGD1SxrA zuOTRd^^sC+FHm5CadRV?)-vCU;T^a zGj16&qB&fGE=4j8IS8^&*!VT*`3rO^S)kj^=iF4g_s}B;;_+nY{?-<3e_!h~ToQ`# z%Iw3J*P!u0{w5ZLXZkvqoUJ|+c+S@s61TZo?pz=gM|dDaLt$7Ea{&!xhN|FJSWNIN zY%vF4UR6#j-r%KqCabXv?a1ipsKif3f4$ppgE~n^YTRnuSb2_XND~U06mTfnCB>V4n zh??YcqZ5>7KEnRqdlc^G+p^M}?x(~Mm%RCtkd^=(lSqt`PRV}dx{FdzR<&OCviVxD z_39L0I}UogyayFf(Nl4&eWnK~MA(6HrPV_|RD#VUG10#j3Aux^u{%UjFww)bUb*{= zCI7&e4q-%rP(6K+>3u{7kyL>`Ko+9Ov>Nd0_&;2|bySpX*FHRygv0;?h}6&}jdX_~ zC|wEy($d}CC5?1%brPh34a!mM^ zGgi(mI=u}Ji`%HTrD+!%G%6I3ODOYNFd3}KOx<_8``QssAdD6V9|sd;M3MnM^N^0d z87*k~XJHNNTP1Ud2AeoLI|K7re#gT?t>cK@Q2N6!vM1KKK-wu4tHOn!TajKj75f+mv!z;UpS0g~r=^h~ z!;aguS5OFaYK=zf5_C%G-si@M6O4M*dObg>&_V03dI-4Q=zRX9P1Rlh*;1#~-R1Ut z>mprim!qhrAc925eag*nTjyat0T6TpH{9hCLR36sRbH0-NtwEDOWVRvYh$tmEG{+& zy}TSllj4uaSY*Ps=frm4@30*CYai$7&@GnMN|%}NK^5EG-7LR8`E*clJksR!^u+Hx zv1;8jco++by{7g5ke6YO-p8Gr`6PdBse(y zBonH99-KZXt$Ikn_Z4Yz-3lehSmB&J2~o!N?D@Op4pXi_`9b5BM%a!KL@lwp71*zq z?;CmgN7DegN0U&3RA%vohFVkpi$y=N6bqF&aD<(E2v9%5( z5S*Kv;EO`b(D>Ib6bkny|kL6#lY3Xr(9r+>%u-Rkr*t=zV zT(T$_OO>~~^$n26fYqJc?s~_KdhF#T)U`J3^+!|QW2I@dIG@R;nDNhlX4XCc(X|gD z$kf&zXgk{sCFS&`SS`O{e*e?y3P+JEMm#m^Pjz0{(1!h1i86*A!SkW9aEh?82bxFjpCldAR{rHVqE}~ z%WS)p%wom$cz7U^nY%YERCyrFc%|_5@HIEQUeNa6#h7CWEXH$y6Cm_-a{`#ST4{S+O-Bmd6`oS6N|ygb#;$_T z1yV-6UGJ3;#ETj^jJ1Sny-avotp{QP>ln{8+&fc06|iWXd;8wI@u%!pTsJQ-w>^1z zT#ww!3r%>|wiXl=R1}a5GNn2F0E|T^T5gT4-vPw$_WGK}qO36t8X1P?ZwH|jV`-LT z;q{A_Umh1*kmQ+KtZUP}O|TiQB_$;t-V!mV3VGhFiee+hf^>t-*U;6Z%x20g8f*+R z+pXw+&nzvpdtQYK)+ci)LPUP?w=Q8|VEATw=&pLT+z$cg!8YYoP|#dcQ=q=qH2+!c zP1?m~?1lCAJ5F+R>iC(3dSvB2N!bLhI-QWd^_M%NLUwCf0;B~UOHB?XwY6{kaYGS$ zM4rsQK=AN*cPxRw-oq@YjZ8YJ4t9Bwu@O+#LS;gWXUcfJ?slsvgTyh)7mte*d28xi z#q}SrvSZc(wN9$A$LY#uD5@`%ckje&*n#jL4zH_5?u zeS4UYV<;4-{&ht(oivC#02dvDN;!#DTZ>d_ET$mCqtWtck#^ev99dJh{LN^3w;xYc zQPunY9$CLig94vbqVYfzz{3Cx|qXt$IwJ`A7f z=CGeMGSP@VVKm*fY`gIn^7=1&RuMGcs8F0METcdUI%Wpd>4FUHYMY^#7u&;z9|@YL zrc1;Fe)lBH(LB~}*GZ0*{u<~7%pl_kdNhglo156xE0%87#+DnOu4;7B1ZHeYmMF3F zh*ogZN-!XDSi304;0%>Qx$SkeVu4jp9~s~rSwEn*QkGqN&u$3!J>!wGv9 zXPgnhYtT#gnmw2-<96Rhm4@W|_lntjMlTT~x;_No{it&&nT7P=2n|Qc&73fK2P`7o z=(}oZVEzT(1ola#!MP0$&V3Pp)$C#;N74s8Pe_2EiXdyXuG;r?veIG>1{@AqeH|wU zqs*Q1MfZa~BIB{NS|Ffht+Q=8f_WT$(<}#`A{)`LJXP@EDC6N~W{U&4?M5J^8;D-Z zZsc$c=_eoLA!lji#?z((tiB3^ks^iPCFEL10i7E|*{3sm!%yq=?e_%1`MjMdxKE?M z#{?Ga+6WcHp|jxxN{BL@z(msX4wj-0S>VQ7= zHi1J5%k!+q6E5I&JR9{$z>nU{MdEGaZ-+=5%M*uOG&~p{foS)FZinN*{2!5d-^uss zZ$Xr3weW~6N>lhE=3XK`I5aG5<@Rxl=r3QS zoJ)|wWxB`n@ke-cdZ7eB5!FN7mb_OoRD)*!=U?a4IwW&;$BTFsJ`~VNb?vzp zbBC)r*?P$?8-4w(Oug-JiXD04<%Kyx^sTn6ukxanR3EG(RJ;T8VY@B_qs*2KHiBt# z{0vHGBix$H&2tS`x8J9UOe2V!6F*?aMEILacWndA!5?)5+!pTNmIjskupF=qOJcX3 zNbv;P5%=%!UGD=OmSYUn%WL^=ItKnhN#yR=0KMSaRCdqHg9@E9NYn-pT90*UpUXi{ z9do$Gv%X3_1swP8M??bGY)gw{&wxJi#iM5sc62aAh$h<1q=8U!TbfV=;!SBNi}lZj zktj82Z{#b%_zQ{x*wi|nEDO@W$0RJ3ipp#vGHWYUJD=%QmK3HoyN-lPUNFGiVc#^> z>kDF3-Z7^(pU!Vjvylk7IJXyg0~-BkqsS#Zq>}843V8`Gi+S7Q#YQ+t0)*i9>kj>V zAB9-1t_U1b6%Ino0#z4CMYY45^=pi8kbYJcH2K4Vfm9!U5zg*tUlrPk0VQA2KNp!3?;nLTwYq1JKPyfN{}fK zA22Aue$%~$hd1`-%i_!&c2jnv-43Slq!2ODP5+Gh2vX#$uk3{B0&d5*Whu&jleWzk zbA!_WG^97H12c*X3X1uUn{V5TGd;IASg?l=7dYd=>sYAr9eNA@-Wc^2|#%X;59loGYF{o}op!v@@=IBZ{>S#D03 z9=o9oUYp!6HAk$XpQeF>9er8yTCp<##J4Awe~Ob-80PG1sz@QOh*FVhE(hOdfg(-l z)>&O_Yd+wDW%kVh*X^a(`R^A-YM{N*nS~6m^En06#3Vr^G&Kbqn{TQ| zy^S@Ds$+^j=nC)xRG4Ai1JRka9S7=+E zL+oCs3%CYzh&$A3*8hlibol9Y(2221p; zvEw<*=r`Cw6O`-C^$9o63hg4x(eYx@6Z@xxy|~Ft&~bB1{>-4UF88f*NbD4Zn;Q6r z+%H%<6Mlz2xY)m2|7MA0x$WU9K%bV`xm%z|M2MMQ$r(c5Nlxu8B^H887#M{n4va+U zyF;)OAl{sKaKQ(xjEHRHAaal-F^C*PDF9dEi{ENmAY1?i1>_o{jtLbi|AlxKOwWVn zEAdu)Bj&3!If`@$l1~(6d_z7cm$y7qq(!{9^W_vJO)3~kdLt&#HV~(Z=lPgGn896} z8kKWnszyvS9Ue)995msNgEEMc<~o?JBM#3T{1(nT3S^C@hEhd4DwxBELBcKpG0oJ? zGJ%taVo;4JOq!>Vr`m#~Gm{Q&x;l0}5gyBtMUS!G>o^&I`V=y*l988P@K}OQu zhVa|%z`Ogw!NL5edW-+t+J|#=@9KU*-dG|Hqj|rtkzEPFRKQbv7gv>83ULO3uw8h_ zxy?pX8^ouRcS7Nv_7+_>;c*qPdB6}f5inhcLLqXdj4r-pTu~>YFB&muP@4=lMyp5R z1qct6>@k9V2Bpw5lms*QNrZgrxktxF)q==_Rzcn)8zo?`rNOITHt|4DDVudw|>9y`qckZ<9kRh_xNO zNUxwu@zrpUG$N%!A5G78w6)D*VX`3I+{*H1)vuj4h$9oxR@Oc$49yobm&b3YBNcR_ zB(&Ydp#S3;Hv#jxl@yhR0zhpFd+|5%FKuuTr0ngm>0*UR8)D>>Gq0FGF%(8Xxx3632h z>cLSv$aa*;i|P{td9q%}{iEg3*k2qjQEKkeB4~g?t7IjIVe`w62*uF1C18JYiOu-l z7x1>#ilCA5p||x%>*=z;n{}+84{ptM;TuaaNS&O@`=S{s8{=%h28Y?Qb=>ZxnWs9D zJueC44Pu^n5}e+COWN%%94xI57WrWj#J2LKj^w|$Odx_(g{EIct;^}tA7eW@712Yx zkXIX4V->x$S_6?@#>0l!=+TQ?IYp%xeg2BQkf}H3nNSwL#|Dit1COnh3|4K9c}x{WyIMF==D*uh_E?<)NF@l#_NNy_*(L;4@yZKgMyA9v!f{rztEdeUzW6Uce{ zs#O+s&-3A$C?)rW=~r0a%^gX=#1A*>&x>U^g9cZky%FG7!u=uJhANo-kA&Pi8y7;y z%cpVahFbiM2TTtJ|I|&}3zrRLvhCj@1~y z*$^RNqnQfG&_1vWu~wLdQ?&MFtEN_9{E5fbuf<&D@!3b3xDV9G2A9OPj-!248-LME zHmtNV*}v%VtBcmtady#niyP<(1GXYE%H zSE9ATqlXikQ|NXKA5`@8{|J+H z{`0@b7n!h!z@b}n79+3g_oKGZz#Wd_gUeqb0NzO zE77RJP2oz{nF-h2*Qu|qIhwdM)C|N{$J#Vm<}Y)Ht#O5x3Xg3o0;>03;R*#%8s3Jbj}wo6Bu_SJ6*R?WCf2NVU20Ng)Zo}~Bd#gVsC>KjC;93?oyjZkE!Ch- z2MD{-0T421otf9^?n|)wo|Tn-srQ_=8>=!>dE7NN0Cmgk^InHT2R_dJi6(G#8Y>UL zO{s5DP;O0syBT8il+F~MbcDKNTAfd0?oI`UeU!iW+x+S0k8mQ3_A-Y^sr!PXeTR4k z^%V5M{dyo4P=%GYom<{<-CfXqV!l(^EBDvzkQMbT(biogN6w>Kyyrg83_ z42my7*$&Q69@MJvP&q;6p9Fycppjm#)Ifjm90rZProJILz5&aq-9E2t>LOPx;LomZ zp+QDD@xfJBH?yj0?XpP!=qL@{nR>lL{FCGwAzEbt$on@KEGZ&En!NG{r-$Yhydnrn zaCAlPd@l4l)H@66Jc{(OI%kCBcS^>)5uvNX0%c#ni;1Ado$nN)8$0ClER={M8bDUH zA&sI`#ESbt8PRM-U7bUW5hXqDySRV*q9EPZeo%C8>0hj+rKO8i`h9Ky8|d`ZdJQiC z4x1c|P4#ZF#qp&}w}76=Pk4eCErr&Q#QrF;emM9_Fe?0r+KaQ?VhN!H zefSP>1U%$1XoZjUhfZ%4AE?}~o8YS{i*2h}&5n zfm*p1Z%%1q784Uw0*j7zmELr${OCo}B|x&qBojJpjjS7f^SR|l=egp!*&gCX%Bx!PhT~fpU7LBCPlJmOy8@ zVeWRmq@q-dYe(Me=1~_dy%GRn0fPD?5Jvl3;kn~Sa`6?>%xGcZp?Uk`*76c}vZXO} z;*h?o0CZ?tGCMPE2@r2~4^0G{yq@FDW+m6Ve_q`(m8eyusw&EyTGW5WB;!-@%27m3 zm(d11r!j^S)P5w;Da&4zl_anT*X#AP&=UA3t=SbskP6vPy#(>>m~zjA5C2+bA+o9i zH;%BhRoZTy6B_2h<(AZVtSqxsaR21Tle+7g8DTM>+lMp@-QHa8{qCRk{U-?Sl|CtZ zz5SfswUZRr*KF(9mk69nT)v#>zW^M)Osmm$s~AtcLVLjf8wIqvj9Ru+!rAgRRO5~3 z+9GctP5@#6k5r4H*+*CB)8F!rI?aw7dpqeaj<+>B82|+Fr)oK_VyPvsy|%HSnehk* z$PY=5kBaS!BoVxL+jt|(Vt#nMZ*sDWJh44ObYMC|Ht)51bB4QRw>@pXX#Y4F&l>Ok z)HCw7_CTIY&~g$+n`sXRTlB%pMw!J~F_S5y3(?es#{MI*rl2qtq2}UijbIsjgyevYNX0y@9KPzfp zIYN7brrZ`T9z%CEv=PJtt~AndEe~D2Fc=X2L~gWv?*5S`cJX5G7%->(-}O8-&?OA1IB%@DLdEHPle>e{{nknE!`erU@rjrrw9 zk6C`mktF=uLdM!3zbswJCcI5sY;n*xbvC&^9&2xN_vmeLIhe6(usK7U6hD})XmEWz zUlD$|*)00a#8fZFa@Wd79mp9^9NMSP7tqx-$Dg@p&-X)GVBCiA0x=k%KOW4e+khbi32GIZ z9uKFTVLR=n-@eT?JF)y>ay5Pb&F1Xx?N~am%MCDO;3fhbgoWqkej|vWoW)E6))J|L zHkaL5e&4@y17A!`yY=C^*-BF9O&`pX@wPm7HC6z)F8&m7CJ3Xi1|pQ`ST#Lkw4Mr1@$tmgLP_-LB=7OP#v$Mt=T$@ z3u_r&%+Mi58Ae7Kz4DqyY;Rgx2{e3oMg~Ti(Df=!rm_i73{^Z|S{Q_-BvDmK>tWk$ zq0VZ7`F3WIL6(6*W<%6X*)Q062O`&b8;oV!!fiY61>B-tmcYIuk%z2G{ zjfLK`aV9~r9SEU7@cUJj&7PI{)Amq+XMJ)Khlj?SJQrSJMS&(>^u3KQXF#-VZTR~L z!GC}%U~`Vx6DD~vDA{x7w0zsh;CQ^4L@WcBDqNsegQt)q9SSCoCABm&Q>LaCK`Q-iT>RALZ7B}O?Cig1cqzBN{zAl!-Y z(C+X_30bF#f{w7mg&Nw|#*m2jSm@53wkH4FeY8WzZJ15g4V8DFjrp1M#+V~_r zpGaL0ucTPKiG|RWKK;}!aaCn?!fI3gjT)1HZeq)IqXE7mUWJ0~L?dfVO3)%38>av( zu^!c#B_C2utR~5Lx1Kl9oIMqK}(QV zG8%>f!_a?828JWQnLt~TBiTbyGP%!05g zqHugb^y;;o;MG*u-GPpq1GBX}ODgxDg{0A3|9;BTZZz^jJI^&=+T+E&j{)So@FK;z zuhL4{&hZdW#XSVY_w2WZ=a!a??fOvpP2Q|(KpsWEZU#uaSI(jd9g5~pP)GIcSpS#Z z*MSNNBT*$wt*>phLE$i|axB^!>Hd~S+y2x4IPBHCaus-iuo zc~2>Ay6fFucuJ$uV-wQFcFXhAJwsAfmQT$Nv*nx$QU0ZoJ|#Z^3w@*Y+1fTaLV$ zzJ87!v!a&c+IoEF=II&7j*k1+8pzKX*-g6x_do`XRUB&SFankr&7p&hW`M+owOwzl zpt-4uAL73K&{LMeb^CMK++?!@k03#-<+N5kH}F?Qd)xVZ{?(f9*w(>SR@2(%>TShn zYh4xJaBlgMIVC|HK0#WmJ)Y193@oP4KTSqkT)6pS0a$act;ooPXKsAH9!UBrz!$ik zZGUcE54I3QQX<<-6!sP`;?Z$moghIX@wbWcD?Wl1s|6olcByF3ULX6p~kocFa z_M!6Wt!poNl?GXX7YKVs3;9mTP+<5Zt6E3>Fll{Ye`?I_^k`GQS zn&GuZ6b?L!xR?VeTv+?`q)P4jUdHn}w|3(ZtnB*v{HupJM-*r;>lbZIQDwR7WA8_Y zuQV=K`&Wz!{BMe$n&$T%QjDtoTbIg6 zNWLKY3<3uU4EmK^>`$AedTNxmOQCo_3jA4jQZ?b&-1N;94R{D;*zDN zyq40M)}vooLQS5ngmj_%G5)>)I$mR>0$bU8e8_3nZqXv6kV6YZ|7A$0>yMcw1y=eX zjRIab=Zjn)U&9?(tE@9!>TTltNDEkpd+tq6E2sn`m3D(tT?0|bF`QOUi@1%3-R^0H ze_Wnc{R&D>PR@Tk8TkwOBvH8D*47u)G#1_%s_2TjY^I_9hZLk#@bzbOH+D1%lw2xa zzDCweDI^3WZTEp3^JR7Fv%_^>sRWgo;6TF`O#^^+ zKk=P~E$xH3p69z6}8ipMA)&JxQmrYD-9w+A6q@ZJpU=!{85Bh(f#?mzgaA zWOLHc#d*L&Iuum#^uSW$=f#MsdWvc<;*f|%P|`X1egz1=+Jp?ho-S4O2{U@-t+Xl? znplmK`C&gkQB(pELx3k)0javCnE0cmSpC zpFS$=s5b;L724pt*Np&+w@hyadto51jXa@zl4c z2Ax4KhKT~SNv$-cz7MC~#nnb*egg&dDA@0hZ<2kDSq|9<}R$KrTr?z6AX+5mthHd!3{_deswPA#>TX-WSMVThl$9i>6Aa5)Y<} zHc^5G2>#^)NJ}6Y3i-bbOz?8VV{mr=dmY=FryxJ5GMi_?FqNAzjk-08Y()5@pc{kL z&-cc~B-r$s$*nPVR+EWteX4HX%ZcW|oTlh}XDBD?ZMDrpNVq!Wa4en;Nz9Rg;S1jr*A{y1D72^BBhu$io zpN0?8!pwO^9Q7VA+&~2r1r;=?`YkP&pfk}tSaHnZS|B<^m?TY;4VJU}h+3XB*&BO5 z(oN{~_{Z%eenfFxtaJE(oQOoLgm|mOKjPYz1TRqhX*~Zs7Xt9DM5}c;L*EZEPSzRJ z91bzK?`U|!43uIwML%%G_!5Jo`lThqtB%6Q`uhz{7d_0`^WVPaeRS$_kBUB^9n5|8 zP3j%VAbwIMc-LufGZ6zw$Y_DY2Rz;t(D6#*>vmE z@LwcP;00-lgx|oA%zzVl($T~zh?SxvDEKX({B^n$DjPPoLFlZ!Erz^@ztLw2FS&qc zCij)UR4e_XEF{rax-(Vus503Td^#@G4m6C!!JsyE9^DTwpHI8qHn9nRCRd9B$D-}5 zzrLOLo+7SDgX97hmljuL!TnApLqAZ@%A8E8*wynT-Io?MDm=rOLvn>R-{_yS+g5~G z82J_!gWaNvas1z36yf3qa;P{k*nEF@D-nmm7UeA#t~n!DH%Ri_W^I1d1oWY95pNNs zr;xBw4X@KBC?h>pRC(%LhGyoIF)){-BV0n=YliiAbb`ezk$L?w$)_T{%9~sC+d>|g z3wHUh6hMP>7z!)<{|2~!@C(96#1=Ux_bW6rq?Lfu1S^jvJN8Qy;ZJ;yLkkUL$ixkx z4W@fGf|c*8ilH`DwW`zdBN-$K;X_RE zg3`ed$oCa;T!sEq&{WS4p&Iw&;m}t~aTc^GbBZ@8jSRFg&#IyjmdC9MultR9?MQtd zG7&;MO6ly+42ryXgS4Je29aiL*DM5ZU#ao+DslBfwz*wDMvlPP{w|*l!Kto7eeeXJ z1TEnzjTDD|1z5q~&1NG=1lY#7bpLO}u>q~M6w;Cq*jegckocwxTM*4kt${y~uH z_loeihK|=|zt8AX7P>}UCV5z)A2<~a8-fXrQgHl*0g4!nqn02rc~@If5OjPQgYrsj zIJq4O6bCZQDl`a=LW4ltR&awcdZ!!kx)y|Q&-@eEcKxt~oP)(V{#hj)Bcz|f{&1BM zRP-ot#d(sVfDW(iiyK7rw)A#vR#mv!dAa@hKEqWxSq+CS@e>C+bpQ_PAKaJNK0Z;? zHKD$O@YGR(P*q>3f>AJ^fppiG40nd^@kDP3h|O+yO3K2BP9Z9a4=)HEN=n=XhY7}l zyY&_eLKn5g>VT&Z`=&TPv7Acsd`P=*I`a%$y7hX9d^RV9i9so-oo!`?lgTBe*~lYs zv=P~$C~y>NBRnW2tP(BvmmBQ~15mnc?C%(W!Whw~I_Sb0&Dq~aCX*0Eu7?)pw^)zr zBuyWzP#F?}B2o9hp{>)38V!ng$Sd@W_cSTV$ztD)lL4U-|MOLvdWLq(X?OerB+Q~* zTfNrP0-zQ;xg9)txa|`q#}jrF%~X_^lqK=o{TM^Xx&(-@t`8+)-$lXqD_++n2i)by zOHR^X6h8BAJl`x?U3;BR)K-kOUkp8$9gKF+W*j2?&i_a>Tc*iTko}=`<++Cd;Nsek zPMepOeXhv|RDmTqS7FcXt^5{N2TkV^&pHRh-XJMODr(b8rh%L{X6fbw;_KZuO7#3i z<;ODWeDNx-4H=o1D{jIc)9q)it@4_c42_l6yJN~aYnU|aNwN8=ddblZ<=AGG}e zq2R1$M#r>xMv}fp#DdN3fLK}xX0#7hJ7onu$TbjLjaHsWG`Xmiix|aoESN_09hk+l z-P`Kxx!+Oj6OjUGa$XOsTS2oGt>-;a!XhE`(RT~&w+oB~!a@%-npL)DW(Quk1J}x)L1YrY1d%a`!I;;A*AI1lv$eIH zxM$f8w}^~NaH!lyCONi^-bs4FwbjBYEK6Md9U8yW zk@HqF%QHX;?<+AJ%e)Y69^zGIxBUXZaI1d)V!Yq zNFZFUQk~c|;=3N775KHAKeA-BaLGC)nI}H3%@mcb@f4ZEdgH(iWU7 zo=$i9TM=gYW`q_S%~I6n8x*#(A^Ywhq=GjzRNkVC$6YtwbYs9FGx!7}mg*|O<@SfP zHpAK;4u3>0yIp@zUjtlUPTcips}0M_g(-E#J_HBgmdpx85t0ZvuU^qrEv^J*JU>3a z;e2!7E6b90dwS{YRuP6M9vyn{#F~)X?#A7R7J)i@PLMdKn?%m&Fr16@` z@FSR<&iz1BFaLOSW+#0~Gkj}(%f~=}2rwd9@!07g$nZ$bGmN#8vwS!!IUv>Gcm3wR z5OI{@@i5xSy-YXGF5^GC2=xvfuro#QG&g_~*I; z1B6eZ01hNps?5&WXnb@_{59k;YXtCqe1^X315UJ5A@&nGsC1)cKupf&{8i-Bf9^0%cc^n^`?Zee?fj}nxx@5)ul>cW z&ch6M`;%Vy&*FSv_ol|Y+Ud9XuU(n7X|Brjr90d31HB9G8{jY_|vg6KuWBrqY$ z**2eymGw7ofxc3pm5UA_U&A2(F*91w(|3hYzj+U$)uP zQdGFV+dmHQ!J6Bius=E$GR&H%6`C6O8I~2<^VyuOOPcJyOdBL7u?7X0i1ZuTDzaHh zb7OotZMb9DnuzoQZ&uH}k6QSrHT6r?sG_bb*^QhFh zX{J2tJoXFy&SgmNQYnw)Wo13x_CdheD#)j)NusBYC=``#W*Agw@b@mEz$q>s&eM;R z*_b_-zyC;x7#v(t&!G5_qSFeSn~xtmoQKVNUF2Xbdz{DzEmH^opb-7hY+a-6_2|%E z9~?Z@lXpCXuf+b}2b}0O7lSC;9Yhq2ZK4=9*J*~p=IizNI6?NXbqy1&xgPJVr{&lP z&?~q5G-OQaC&|f+2JgUz%$YQX7g{oFufyZ?S+)o>Vj4TuZbE|D)Ix{QJJSXQ83p?z zZMq6M#S1Z;SpA$vy{fWSeft}Zj%yuMNQh#WGik|;;N4bZn@Cz=6k4=A6`ag8H`tVI z^y8OC+dK$a7Gg)J2#U~WxDM3nIdeBR{zQ!csHa$Lta1RIcCG8yZB2n}vjZrZ-yw&u zxGOkK!AgGR`MB|?)$=jQLS5T^XUGSmL2Q}5mo;1tiM&zl*r>!HwVN`QiWa2;*>cIh zRt?+Q_4Y~7+)^0@oOa!UWk&T#Vz-m6)`ih@ey3t#yY=5=+KaV<5opG4chrl8qIb3J)t3MHxG?=Ua*HBt@j!Hv)mJtlf;4K zh>b;uMX@?UHFj291Xo%B=>06h5KkqUDF!BQUn#iMkab{`=#-gSoNu!Hl|!kH9P&kd zXHbtVVxz&<@ujy`La|J}Zcl}_+md>0(bK2flE3|g+iV(!ZCfH#&Vxd{V)w2{!|%9GJKCf*CrzVPPgVqg08B%0?Cj2rPvE~ zoRB(cxmc~wSg5l**+Q_&xY_(w_ink#?D=>nImg||xzNg%fJ93aISjiw)JC~ zxVl?b1UX!`$OcLlCC&F&Pc|s+(a7X<(Wd5eRd%z3Fhhxnagh}=tOp@=I+Uu>Lu~yK zJ^WnG=0C_K!DDDecYlpVwuMAk+&9KfR_7Q*!Ot~QEoyI}h%&hh-PoDAs7U-2QGwQ* z*`Q)wDO4$RJyi?{hR@TkaO*sfanYOQMDiK}2NHvr414N?um*R}UWwpdY?ADp4Lj*Z|45l?tTS3yBviy!A}lVGBmN@%)hmkU%ay0F{GtU zPDvqnp8M~JWZTl(m_zjEdHYaa#me-KA^3s&SmLwk9y~})x3Cb0?u`XZ*AN>{`aKv; z9NYW7Dq;4Mk61|F6j`7mvl-eIwOpesP3ihGZ8{#*af`;<~85lbhl)bj=-AFP{dSsB^x}FFB5_Zkv6H2K-+w$1A&L4Vnw3WL57POJ$tD^%M4C(ib7Q4Ma==o^^EynQeC3k zYz)D>GRtWG1+4|~hgdu6k6@(uc0Qgsvw%gDN9?5Q+5K-!eAyDdG7lkeg}P1T$KeQ% z9T&^gcKj;)vub~gsQ)>=#@Qvn+%g`?_-gRNXePDTR94OIm#MfpEacxGxh0ZO#H3LZ zg{w3LnK=xK^@Oxy%-P$r`%=G8dHZbpLT!ThX%&jG99_B~f7{=SgFboBpE3KtbTZWa zTLM@Nu!|byc@E#zR<9YdWkvM_$y#1>-~OPtUa74AtAaij2GU0MZ$L_ZmhP3mQMNADVlK4r0(U5XcR0@mVNN3_eZHr~z z5HIe|-a8z4uyR5tRhJmkAQjp~`0y-kQp0*tpxNI}y;=pf&m_8h%bhX`6pC=sB8qtY zCFMMSUy$wPYD6a6AvkQB>;&z*Ucjdpm+FnBp5FSyT+KmPXC*15s-MbGrtZaX_H6$3I)ftUO%_UfSYo>Gm zSyQ0$DW@l)6v<*vDu(!QMb}653ia8-t2x1nmUFa3X#HDE3PkT*oQS`$ zuH8!}QE+axLx3s@T5v@Jxkwc_OghTMx*HyI4IUcG@Fe`Q}=fEY)62rFb@z7)+0v=?N!`0@B($cpS0vj2*%QQWKu;Y8Lw51uoxj z60-?bTYePcfQ2EnQ=|3v(UNq1X|5A|?yAXXxql)`U@n;d;dRkt^~Orm6Da|9IM?RU zH&c&ALi+mke4EqBP=)(5;4E}~TXZIZ)+cvt_=7e}1ZU;F0d3C5j%e|Zzs&g(#* zBtu~Pi8X$9$`b*THH}jLEzL1nP3U&p`Cv&rt3>8q$*gwJXN@lR=}-UY?;2-;>qi7Z zMOH=7&TV`oI!H){(ihqbUdfJaF|NkpG-xr!2cvZ54we?<=J*K`j2Swco9msoKlUXw zH>;z|!S2&uPRo9xj)K=+@iQ|2&*uX+y3UcC-j^MKaJqNn0-t%dyW5{9n*ez)N)c~5 z-?&kPo8kT+2q6MTRQ8w`87?mtgTKz)btkP}`1w#7E)rNGR$dRWL8Mx4+nYR7TLtWZ zw<9J7+gtnmWM1@gtwzAM4hpSnt9Pp3?;v`ve!uHMGVRew@d8;;SB31US8rfdEpSmj3^)?hK#E7HOotTy=MMYIH z)SFHerC4RT4cT!6sK7ealrA zan_7XqsDj^M)p+De!ID(AB0JQgE;!Sd(oqq5*qJ~La*(1R68gYT*_4b&&e(l{3vBD zE)M*V1^HrQdb97HbGm{_{Mgz4yv%VAc(K|e%Kf)5i#{15M_1_+Wrz-wgqZ5C8JOLl ztE>tMoNupcXwxSq4)W_|@QH;E!h%G;7}*LCeeFpG65WVUBv1x%s(R-_CbX)?n_BcY ze{CqJ`9i&Ex>$!_;_MlzQu~UD=*G7VwLZJMtTuWyXRNfGx*HL*uc#pnh~_Ou;xTlV&qeUgh>)GzCpCDPp&1|uASAs zm^LW(`m5q-#9Ewy24f+psKtGtf*Rt$mjtd00gkv#k!5rDhc9kMIUEXwFIf>0e&~nhI3k7P%<# z$BL4`S>&x6R=dK9V-Ab2Fkmz~&*(yHR22o9y$KtF|q!Um#-QKH0$CI;eQuNw|6 zc>RYLx2OILkhB7{{=D)-Wgh48M#xOIhVY?*P^Yl{JrqB(N{iX#wF*o4@^YvRdxRP2 z85A})+MIot{6rl)z*+ZmdB%m%?!q={AU~hREZOY;YVTtvCr-hyy#)JOmW5(;eJsR` zB3eR9|6fyI9oAI%_CHEe9Ss69Cfy;8bW5{=bcY}zsWhmBbc~W3C5#eq%1}Bbr9`@i zgdh!4@AU(JFPCfVhA(} zkI=i<1E#wQl1eso>OHe_xv(ZzA%z@ZLr zD83l96K4!B3M7>nNGnba{>15k3GBAcUiP(p*}qdclUPMnf!Uh)1pj|kE_Ua$ec(gx zYdmeDaYDkNDno}fl)e1bi{h=3B(%3s5BlZd~{s zYaLI0Pu{wjeflI?!q#WG@d%)P|EO-LmuHBoOnO*(% z!jB&j+|=K9=SF@5=1hAFjl<7b;e-->`oy?Al*y=rVQVQkKBxaTUx=0GYX})jOIC>` zV^PY3#h$O#JUMWdu30|E5HEMHvaPv0|4PYY%;O4QBPf;l%Q$1EZFxAycwV&7WcD9w+dl_a00T5Skl;sl4O`W2H8_!->YZL6`{K=a7Y0YJ(A? zt~B13{~n|)V2S*%%{JzP@V@zR$C-DH979VtMquMbOQ zZE3}p-}fTbT6Z1Cw)5_+*4Nurp9Js`-rV$nakxXoK+MEj!<)TI=p=C_kJx}gB{SJf zKNU~dAFHU{+=0tc&AgB$v>!K3Pez1r%{cAAosAGHAziZ_@KtH9=x?Wfgl*xV>AI=q z?7m9F=+H(b1zZXr8C@WJ2t8kMAt5=h0wD-!tFX=@Ga6^Q?(CbQQ0QiVJ-$&`G$T=x z{G}hlEMG#z^;3c(0r^7~A$1T<|Bruv(fv({i8X1h9)t?q3LYJmizF4bjVFTErwS@h zoT({`Twz0xJ z^k`j_y;sf0=R^&&iidI&mYlu-Uqw;>ChjED2UEn$oh~I>o(;untm07{fk6|Sl*I6z z;Y$Cc^#ENr*TnzcS7uEETR*-&$<@9<^XHA^qqv4~GNTx{McVn)@U_!*uY}%HO0$sL zc$y$aqT+B|uo?@5Og^{9zE5GlWcT8%b>psD^)Yv5D@paL&*>~BXHa#NZs(s4;WRL% zZJH7t3590RHNx1qc=r0_`l<=yDWXKCNJf-uAlX?^;>Jq`fpF&Jw?L_IE5eGsRNjJY z_RT_%0q=mj02D_JjfjOss;LPHsV>*cKfy@?7eC|*L%iXhdI(XM{M{w@4aEnrAC86ewQFlXX@mKuD;NBu2JIxAj>erLUFBAmCIQ`aX#MX%mMg znIh+NkQz8T?L8lekpMRms}uuu0VzcDxK8h9k}Ij3#NdPP!0VvDVpRcRL*$_lT~d@($Vp=)2PMY`It zf_}&}uqPDF&(AB$3J|@1)fAXNU$<6%{7KboZ-#I|Hss!d~6ce7tx*M0V6Tg1YR?~OTju1IbE?9U4#8?W=@-+@?VK2IZ@b3L>%(z;Ta^~#$p@`XMx2HPnQ8mZtPhwypydhc*Hg*YBGWi$< zT#85#865v>@cPgk(5$&?#G8K!SnaJ9c|VMxeDojZ4&B}PPn8%UImzQM3oXw#6|X7Z ziG{4Vo|`ckc9~;4V<@BMr8aX&!EZc=w--)kn{z=Sm*;X#*Bcuxf7`6IPX+8%#i zZr?oVEU3F335Owqhg2}Bu}KR{&~2DTD{(j&EDxu)Rw&rSrHClw)?ZA8T(qZkB3ja& zKst{XMnyi3GIe`cu;{b-b>cjQsK`^55r3i?7(jB1}XG zQ48ec+-|j)gZ$U$r|(+^5&qvfz-Pl>f8VU5Olg5;Q1RtgvOvAm^= zDf`7W z7A+4~pkZ_7dVwtPTA!j1PifqHGtJ@j$Z-8D{GcZI>;jp!2}{cgg|y?6C-qs;=G5P8 z%m?s#dp@t=i>{1sWpIv*CH+fJrKgnIxdj@&rct;B23H!t^D-C`2?v?S%PzNWcRz4m zGKTl%jt}*|)#D}Z|F5C$$=gl7v(InSw;s)pde5n2&&{0*GvQ9kVj4^pSjW1b?oGlSGds&6=OR5G)ZBz2~8Jr~krk=w+10Gj4xD!p?fg4U(Gq zrTunwk;fkIz~w;1zjN3zC{=F7?6PZBdSGwTK$UniQ7;#-E^}lvK`wAwb1kDkE+=^= z`MKwpp09Hz6#g=#EAG~!|96`erxguJ;^ITX|30!Fs^$?-7J!Jpkw8_%4_K)z-}(}- zHGFJ&-e*M|mfn~>pFM&N-F1H~2LWa3hdNv>)tsEZb5gi@pUyT7lEM&A+7zwXkfUO$ z45udgO&=W{aDEb+IR6csCP%?l-|dqT$sUh;DQ46ng_8Fd zP%9quwyz+{#io}Vl}3~?YJDtyiG!u(!c zPV^!?0cVkR29CRQ@T3PFLd1x{&5%4MYg>Ls*MSG;$TF&sBE zG~^@2gs<@13QtSmb#PoH0*5FDbqlClo1wmHb8 zv7C(zBSj>9jY-phn;=nrue%)5c(DN$4~WG*e-B||$_IY^D%%V)GeoQ+c9ybi&mt`?vpqN$M77^?^Wy=j`P^S*xS8RH~z z#yIv9ECSNt8=i={pUO#YG3nYN_{_x}TIhN?iPhDH6doTRW3k;8uO|1;zM_nlAqR0* zOs8eaLNp8oCY^he((HW-S)eq{5-B4e96EaXckAsnfzn#si0$od`#VLPZ=uGV%FyC9zvH#SsA_}F|>XFL~O71A+od_?L7evoKepz?}g2O-*x}H&_W2WYL{lFwco8a z=nqfmq&`J>v?JN3aor{!&KR3;%!`Dhl?aYy+hQV$*eo&;HGO1N!@34eSLS zH6c)R?S<=x0Xf6#JTuwYd-k|NcLdOZ(YXmW|3!qe!=Aw|eNa9w0c-A8*03b8i_)%aV^BlC-2H8Ayiw zAVRN44A6OFxiZUiniA=~ji|YH>-IPdBT|eXVSk&<@0+_G7)+Uh6^PoU{1J(p#)$Q9 zDJRk;%~_kg`pNr=)1(}1A6ks-uVDwD^xg3L7wn(24&BuO2~go=CBKCEj&AmNztN{@ zhRJCf8ygE}-HDJl_3UyV*Q>R{4>4hbN!!;NyzH3`uWsABRRas(AXC&0pqhObn2c?0e z9#JfXK2meDvoeyBCX(@Bz`_4yXBy)&!40~+yxa}>VD8pvB&LZKoYh}$0 zq=X7EbWzcRcY{Vzg1}>FxxO?W`*!PkEAlpjk=&m}fzY#lbAIF8w3PleDJiLdz(AY8 zoA(C0^FU;BK>Xr2IuwbDV5RRhO!WA9_$zYA!Av(nJEX&`lx>8}!7sO8=~_&Dh~>Il+e>g=c`48tt0}nWuSE1# zJ}gq%_5iD`D|7Y4%#1XL?6&3QHXyOjmF7r0?2-m?MN!akA}FfPWw>6aQIiq2Mj7Lm zP%yA4z=4i{23HB!mo!}|%fiFKGM0-;c*Q7<8CRVhh|3C4j5XxY4XM5}k(}toXx}*Y zcn$|oZ!MnRcsdLmB-?viIV}i1xj8nWD*qu9qHbqr2i9Yx#i=$Qp0|)5`xe=6{1=eM zlPpd-A5<&}`E~YTo2RRy4cgnYOj*?3{MzD6`2!+!x zqu#4|S-3^552MP0YbpX4#_wzWSz-YFa<)n*%e==hJv|+%CM?hOSdF+C-;0c^DgM*OEPnzXT9Cc1w9{&*5z ztCf5)iZH&-|t z=wW8QK0Pdnj!?Tl^&xaJ9nE6~gbt6_B<4i2(+L?;K>`X~r^K6^1#(-n#@6R$k1h4@ zz4@jl)u+Pg{%vB87e{(|(Cm|s|EKFE{~D{tAeAI4e89Yn7>slA(=PFMuw$sVoiM-s zP~OFMlmEi7G?*apWc%gn$Za=mZ5&K8vdyjsNFcK$PLpiyelO;>P*+kV3D0DdJ3U^i ze}qRwIV0{7;Ih)4j_uxCLfWYe)o?yL`w+Z1_%`iJ9;}g_Ktq8Qe z&)TxyQCn?&-$xkfIW+*YvRXJjZ2&2^ZK7SjWPC0);DQh8XakvNaf!M{eIpbEBmvHI zjR`--*YiW~cY!{j0=qURD&(#Y>p9;(L28M&^w@_(h$@OxZ@-aHzmm+hPZR8kVhP$`n$Q@`mkr+gsb+;Ot8HyVQ)W=a zR|&|V0kxp08?$9LClww}Ij`0HX>LZI+q2fJ`gNdaD2jX7ol~hSU+pB<>AOZ~p5Nzx zp`hf_QIvRilEeH9L!?=rB7j8pDF*$XZq2_sR7HXxH8^%P=Q8y4s3nGmBxf1>3JhzJ zF-smgf?4yXKOMcvunf7Ry>@l++W*F92K2bT%c+Uy=hejq-&~8bpyI$h&3m$&a3OU4 z*F=nSfBJjNkd1fOTUH^GPtv4w4D`w$M{G#*l*kHqCcH42sZ%acdUa{jc$BQh?4c#T zDr_jk21l2mCHYr%EhL5+{VIP290Jx~c~`bR&)Qi4uL%m;`u75P1pkpF=DoNWF?_ER zAGa3tQ9&7WpP6}g$5}&^weTnTNZk)?kQimpb1ta18!F3m8bLvQ`YVWY`Kr6Z@F0 zqtpkgNAUWIv~&jgA)8vwZGcHT7*ssu0Z`F@DN_5(mps*xnoHApA4OQBNB|e{q2hE7 z3i|qaeHraKDSLT179Jjsu;A$n_O$o1t^EA?WVYn72h+=pb|2%(J|88ot<=Pjm&+@6 z7eWCy-rempQoAfI;ko^v-SItgSBv8r`FF|47$5z;qhwsQY1!^tBW5AWz6g|9dwY8g zdO4Xned`B2($bllkAiWXo0FKBnVIh~N%|hQD~o*&SR>(gTUxT^mzI%CcsSKj+;5gldZOLlnt_ckxdx||M) z#7OP8Zxg@c737nNN@w<>DVcHy{Cs@y6(F@d3q0}5%e|d2?{ktysrm2UlZYfm#3E4v zwP@6V5slptq}PYqJo)7^L|3lqeM!mM_Ef#vyrdEQmR|taF`Fau2EKNDYN3W6d70b? zW6y|xNX`DE+QfI~!Xd?0-MVvYQid^tH2K}Y8dEX_RnT7h+P9IB-_l}oXtzN9v8aV> z8Nlew)ZIP6wbYlV%ta_4sDGF&EI8f9_u|#nV`gSdTaK0|`K=oN%#S?KJ3(K5ar)h8 z4eHau(`(g&ut3vRxNv%VOq5sujC5Q+qP5cm5sCIuK?|nxb>O=&x?(`Qd?px&##)SkmauohbRuhw{f2zcZ5y>EA z_tK|ge@$oo{9?W*UzhEj|FfAoyMDW@n320WN2C(CLA?R`SYS zwpV~}#LRuGNsJ48p;a)3Td9@A#nttxz2x3f;43>5u*1HJ8f!s;OO;Mu1bAIIOKZfp zD&)^a3C_fmHnObjY;Ky^{+Vpu#JjXfxCBXsE3Y4dLSk>tT^-_vesT5Uhf#M%2$Fw) z>oBNv`Dn5_J{sjEuYC8>2Nx6LlZmBr11Lczf|`iPjZa%&c0DH|_zyHJWmjhA$2i?O zV6@4G3d!JL_dZh`pO~6Y(Md>*$2^H{-{cy68^$Y3*!j*wI<6&55eXJ!&Gq&zqPhNa z_j-}}5969jyC5$WiLctt$iv%z(ahipOnx_XU&Y_4r-;YQe`oH?(Z>4gXt+sHIC4Ar zu8wk=Pv=`bFcC*=w$%Q}H~aqmxh>=Exi9sh7rCUy9CS;AHFO>}L-^#iB(dkKD|f{! zgE!0;F6Y-dbdIf>O4FXqk4Pe7t8X8^?M<(mX_g3P@ypgMDeOhMtnmQKNU6v$d+z-6 z<#7z(j9>H$PjCihOy>)+Mwy4tv8#Y)@|hohF#A~fpVh1bs>f*pJ~ieO&0n|r zoR5E4{5YD!31tI3kILb~TxPjgtR)$9V0!`zT;&YSp{U7Pr zSf_xTltc~D&*=XU77XPB0|Q+jj{!ERLAnTAUTr?NjqWzBPjo$iT%%X1LYgvq=iISniD9{p-^P;qV~-Xd~t*ydvrgTUcw;=XDxBv5(cybk`9pRu&W!dVbM4C3pYQ z`6Eb8L-2mci{SIRwGO2fCjoPQ7p2`+G$C_9SuHyFY`@x@mS4GgmZ5Z%3oG?m4*U_# zB@}|@kS8m7*%!&{VU<3-?u*eZ@ONT&=DovJy3A1Xdqf8Gc_$lr!a|;i-=P~w;viUhSCL$vPCDG^Wc0b!6 zic}Vqm0~9@Jzg80l9g2Ff%X+3^vTqXr5cWsNMNpHeXxk z_j}W}maYel5#Jxnze+=d9@po?OA53+ax_>eVwAtn%zVidd-(6T`hffXhg-w#09OJ0d{kfW-V32M zeJAE3e3O|8pR@W7Blho=LHGmc;MFwxN;#NT4yaVLB`fA2`F_*qhan#Li?#Y zhxK7A^FLlT4pW|mCrl%Tg?}-Villm=(<6@rmUS4z!90vTpyfA*>T|=@ zZ!;wqt(L~?s=I$&zsRBsr%*S+S(ESY(#cdtP$~3&+=w}PP^1K;)pEl&k!Au!1_xg& zty-?y-#qqS$Wa-`#}Sl~`RcXSS02L!&wYt|H8mWvqej+>T8_;-Z6Wwx3RAj9D}+@M z-N2}`Jbk&g*k9x;{cf;~G6+s6OO91qc3+!3W7{krT1}CqV z3*(U{>+7HVE5h~2tUmH&I3vJkfcz>mFK}&bVkrgHgMdF&nDQSD$K7tZ20Q~(|9&8% zV)TyQw@4br%_6?%OjZWP*L#bUe?xZ<7rfke3ZmLhl_fI$3wNltj?bSzDjfh5fQFCW zPAL!?7$CzS+h&Zq*90VNIA_JckOit-88@0vUpk5=Qb&RX&*jShl2a8EjmAH{Rkdw; z|JFkp30A0P3-S{DG;^XsUyVq`x4|#_T*$(bM=g8VacXxgSwVa9_inwiWd(JU?sZ9<@-U z5sRFN3>+POb#dS)Eua^F{eE((xLDS+L+YHjrBPe^_@t%WPuuvno=20%ujWXvuw9Y%fI?a%Zzfsby}VJ0Y~I%4pSk$=%e*e-HO09`QVy&8@dDvDukB0SC>}5R`OYy>2cDgt?5=7VxsHc zkw_$fi39L^q!0I7C~X%M_r2EZuns{jR*E6amFuezTCH59N?((2CG19r&dWw_^P@aw zDd7{MBNd}4liQ_d><}1dklPjLHrkOBa|iL2!Xd4N*Z7SUhwypL-wHf7?;*kxPttH= zFfJSA?0oi6!qItUl8NhvNn3(B|0ICm2_a%u8vHImV++gYGSQp1IrYq`(RcBqaVslP zQVV;ekM6|uA;f8CIK;?0O1ptYXIA6vEQ*u#&A!pdr+3j)HowN)aZR}%TZFFHZtPY2-i#>P#Wy)vWE(ps00Xx-%f&+HSK z?!cwa3jB7@#hlpAH80FFv3LDzo3|uTUi#R3|1eLcH82$ydO7?kzKV&8ofZWp8TcGk zG?a|eG(AUOm{%xPOFGsb$x>@of~k!MbjVIla&(Kpt=hYQzrR3R*WBFP?f4A{k>iyD z=!4F%(>wv@R`Hc6iYg35ld{p;c2Q&Ti;kBHzuY`K+P;1Yu=1lI$P@>RT3Mb0rAi~_ z+j{$e)Z;lxXUWz!5~x>m#bY?Q_~I-73%&yxe!`UMUG#q^AI@%FDfA~U)inQnoOV1a zm*oo?wFOKwjE?q?0^S?KyWtkSRQr`urM}V8-uw+a-g6U~QX{Bdr}r8s`6U_v!3*UY_Vehkx7u8(9f;2eCZ?-*8dlFriO0du1yBk!Ui{ zb39fy=T06O(v0b!@@&b5?%rq)=#a9fuaFh7&Cw9&&5W=+ z^BhDz@gurK)RkEozV~yQ@aBk*6FN!P4F9=Fcn##=$ea9B$!G%smxhXta?&p2a`R?!6 znVPDZ8tLj@)~l~|b)=Glc+(uF`W?c%8yRU3u~ir?$qAFHSQL(E0 zU^1`qWm|sQXSvaLDsEZPp6$YS#d!lb1yt=l_v4@c*8Dk_BB@9!f#k3o0wO5nL8uTx zC^M$*g|q=dT?hh-Hqs>e>Lmg8=Xgv^46w4pacASb^!(iJ^-Ab7K+e!rkRB^~$fRED ze=cvMsD`ev|9(bq2r;F(9=JLn0k8e9H$J3)v`GKYzvgZu=kJop(Djv?^$qa10@Zv8DxW=Yy>4go|8=guSjy_Q@)t@YVj1?ojnBz0&6)#i4b?i-(^d{eON zzlGAC-hR*}GJKbuI(=lBS+X}4|D3Zy^KT(}Cw#7($$gUVv0`gP``ia$E0}-pS#Rk{ zpksL}h|H4lj6RQ@ZTy!MGtm^D1s|lifQIjPhRIZbEH*%2|LY|1{W1RiM^r4+4UReS zO`Rtg@$^S!v$xspzx1S47L0bUh6k- za&mH?UX0EA{`-id-Vz}kG7>VfVz=+a5Ea?_FI5-|JXRWC!Iw!n666g*gK$=1& zniMIDsE;E$Iy@}wJQSko-@}EPrlassYZwt5Ijq~LA@zH^dC$4~B0n7H8qL7soJEBS zNWs8NOReGedu@XX?Ec&=P0<=DuxV zTiDnzQ2IFZQg}czNdIz#c1uSVMDN)6+)TbKD)73p`Qh!-1=oH=GHlNs$X~vhTE3Pf5x=HD2ug{{B=huO_Xz^T@x{_Nu&vvrGt*6Wh1n zS$<-Ce+2ztPw_kTs8f=uKSW~sR^BB7%yF7tbF|Q?BwBuK9j3vOCo>uZl0c$F53Q`O zPDb<8*t^d#|J(RF2M8?^aP7y}9ldel(GWR!gpZDA-M7$KDy2UU|2&YDbE0hBTa$i@ z{8wMJ&K(JMoS#m532KD*hhEmooOeF-4k@NX=nnQ|8J%0_AJZaDE_9==i6!OlKRgh8 zf5Oo()NK!(Lq9iCnGWjPios>=YK8yy?VF(xPp5IT8Nzy-aaH@C~q4Cp%K(=W3tR62FS<=C@Kp2 z8zk~eGx+;kkjgK(@PXjq#QS@N-W|G3W_}ioo<_vhe;HVNb_vVqUFX2A$AZ2V}2KEjd<}u=v!DumM$bUzU*j?E3i;xmUNwn_sNLR6!-}aEu`L6bE z1Se!{Ov)M#J5Z>km7K73G+%D*MxH1iD?AJTPw}a|B+iSBJyDD~c%amILctB9|n56efO)C|U1RvH~ zT~F^b2?j!D@wIahB_*XqtyHp=om~&M#Psz@kP%^|m=hvG?{VCkd9~(${bOQ%;4p9l z@}Fq_Kpgi3!cXSAPvvC?a@H%qX9n=*$;A=cZ|j%lXhgsx%YN#AKc_{THidPxcw1`6t^NXi{%=mB^|iLD=l|4l?l&e7nX-cg@(yFcz4V3Oe_=ew?+ZQ6uK zLWVXB(`XZa{Ej*7Uwvvh@=rXOSRZ`$`e=ocE`OSCa96q^*3%PVDxFu*(uu`Ww{cwA zFiBcY_N2WmhN;Lgj@TrZCtX3hC_}T1YpEICXu@~uQLC^{uM7C!#(bxL{!knYP{_yr zkHUR}oKN(rj3IsBye~UbGFkUaRZqB4nYn!LWyFRDk{5|nh{oBrK7$nO@B-iED-nT?sDOM8C~OqDnPP7Q7zp}pG+U~|5%zwRwnYI|ibV`z zSvLruJkb9L0(Us+Gxz0$&Nn3Pu?J(7y z8v@%W(;ZoF3mgf~%tQ@)N%Zr?=?Yc?qOKEa`QGbO9|2mcKA=lgRML^W{6h7RQ#uL% zvOPt^3-btHs{s{X!-f5ZxiAnY4Qu5U1peWRVD~`IT9%(lHXlR^-PTaVmnfCP=ZhK; zfG9M8Dnf7goB(OqXGw;?VrD)?!yN zP@k68BG~odM=*mDR2Y#0SFO{o_W2HE*!j1gvj+tpu@6J5#l>-ku*+*NXaZtJjU8^T zw+tu9hnunM(dlSySw(W17~Is-LKT;D%j)M#ulA3>pDRNdMXh<WnW?v^+5V-B zkgYR(a=g04r{dro^WWU^nPx?}wi27?gN6y!Cq>awQTvnG2V4qsI52a0oEB*$ORl;r zt0ILnyOyZc^B0JXNeLAkBk^O8Bnp6*;&GM87bQ1}^mss$OP0$AM zcKKPw=FIuu=*0%Bf$DJ_&?XkJcqPsC_sP7rVPT(yNJzC2K_rs_0WnkT_^=qmj z(^8c`F+C~|3r_+x%{%4Ejm9} zfJYNJ*uyqF#>aWw-T5TeWHJ=>_O?Nnegb9b5ZYHF$}CdZ47^QYUxM1Tn{EiElf zu%wJ*{wz^!!KEPfxe($P30gc~)6d36bU>odUWsH$-u{roQZ zAPO{O!LFah#h*}u#E#aqbxvC17%U(YZ@KHwS)A1| zoWNqylx2wg>4#%vf1Kls&1UEL{gPginjn-0^(>HB!rPp7BLVxu;__6M*4l7HCTMu1 zpj78E>3bJ5Rn=4RN8h`qtEedmt(Dz4wMP zDa9||pEtu`VR!n=MKzKyH^pJ;f2l{g^Q$&e!IhHS|G&ygUf_{X-HND`)Rbx#^7U#5WaBc%- z>{&_^+pv6lcP<g4oP?K428 z;;m=Kh^vTJ^oX6mAuO|R5Llv}!BiUaWYW<+0hF=5eRx1dLi+m4Gg5^pdABXbRU)GR zjrt;cT42+&y`=*TgWjH?O=#STxqQ~lG}WTmE>0Censp3hKDh-s=4@-_kY>!utpHKu zBz9EBW3;%bioY{Cp985|1n(WJt$$037H&kR*AZ}e;E8Se?q}cKE%EU3_OUFRfkr-E zes`J#LPg5yaz>W}*EV-4R&9;`j(I=pqDxwvN1}}*(bX*Tg(FxKjofO#9>Po7$A%dh zZnXBk_#C9J1m5E-lYQ}*8O?xF|E|^gee7?%+u3JiB$^?a=$10y_3fvC*- z?;GY+?kLq@KYu==bd`OmV2c$r;1x15vM6ouCb9hBWS?q|ijRPP;c8$e_v!;iKuO#1 zVP3NLce|q*Tw;Bg8~wD`U*=5-SG)2v80f4x=v;_ruaiy(5oWt*XJv@+Wbj6yoaJeO zms9VHS4py3=q(Yv{Ll~?_`QUKK`7zi5=6;QXyZ^pWZ%ftg5YYOnrx(~ZYFA=ZNtc< z&`+qf0@*y~^v+j>%JC6us~-HiBm*OzGfSw9=l)8u(%=qMKqHQw>SsCDLe%~~d;`R^ z1HOFe%VWh8qgckA!7JJNJsK`7+Bq0i5ZN4H!Sxy9FojpTVFHCAhQ6DrK`bDx8lrI2 zCpRmNFwBTl$~Igd053&WwA&4{<*=sLPWB`oQ}_>_kS0jw(gKgj#G_AakL&fq$_!6c z@3Wkg6Cmghx`ie3+A`6P2C$Knmr~Y1%Nt1vWmB@rAO0N`o>+X(pqX*@8$O9747 zeb7y0;mIeK!(j^)8QMYY@bf9<$#D5FP#99SaKF?j;7)&~Rs2SC>nh<-Hh_YmqdEQa z1OW(3{;bBVKtPEzd|e{fm`RG0vj3M!BeR@Sj3RZ8*Xa@C{yIIk zSENsP64k2n&(9k=JRbzHfeSrGGpu>&PQ%EBi!NW1ILaYU1f zyJm_~@D;JEh%yZ(RtajKh2TgJCbO3uz$<$b>3vaJOh<&gZZuRRnq_IVk!mbHVO`VW z&!Q|c!otFij*j$L&IFPO7k>%_nJlMr#K^WEUPM3E>?~}H<-4_>p}^)?gbFHj%C2}e z#DvU+Qi61oF8hegu_htY;{M?_oOy6)1%ajvhsiayDtv-iGIt@AC<^Vg>N!8#e(tP5 z%!EuX*OHo?G4X`&NQPF6u2BvS02Q(3#rx&j61Z~vsG9G0U2jpE)EkW_lGx~T_gtR; zMnCxRgi?G?RUU0qdG>FO=uX0$%ZqQg_S7BzJFJQepD z{VJta%ONI!vW0^t&iNhYTBT8=_kl0WEX}JsZzX5fN$x3igkW<6z z$oV`S_Mp>tI*@(47x=+Wq*NEvG<{512WEU?Ni(%B&mduGaoz25t1qkrM%-joRZ-x2 zl)M+bgO3xLtoY*8`eR+M$3UJMDU><(Pi5zhaW!H`}7& zc5G}maH+R>IJU7q?f4U#={35lN?L2Hi*8n%Q)R>Wj^hxrLR>06Zz~U8-rf(V8^}ki zw|_vGbQ*FK6V9V&XEy5(M}c9(Nb)8_m3z9a-@nq*#(fdDsX|`J;VRccAN%NyV2e4r zXxIDf#sD>AonH6(5p@gJE>uzd_RWC;;oGf{TKx&4-9|_0_vIpM=ysh)uMVEL)78b9 zKc^~yx!%I>>gGewfBRnxFSn>W-SunijSSzOucM-xC(1FQIy+Y$&Q|?zEI32-dYqQi zzGauEV~m8s*j6DE@H(!AzDLV2XHs}F-Pw0^A(5~>MMalQj^3R4ya&r_;<2Mc1rkm| z1n6aUxIgc#^V%rpSY$T#bQM+lxJnxvxNDB|1;ZZKfoRLxk~V@`6NA zE8qw}XQe^{`fjObX=$m^`9WY3%J4ia>JHOZLxB$N z-o^cNv-3ehmF`b|!dpW_!#7OZa_s$>kQc$V#&;1}IFsAV<$BAD+2jMj7z$qh)r$tB zwq}c4u9?2~;iY9blFvmmvCBia+Zj(lOczuTqx(SE7yJ)*rX4>!w*k9s_TbCcWRWIm zTZ=>l1wn=(Jc0o@QWR*w(}x%6qkv5amybY6!;<2b-8fQF+J#!1Vo=T_*<(@w-rNS%$-KFC)&kft$f z%FNY`)Gg1Fy;GVolMYUMiHiLDjaodIwBY6C#l|+i`?}w`TmP(l$^dQb4=&)d%@5o6 ztzcx6*}_o71N%)4%%dYdzx|-F49@qyz@A!uUL<9sl^wrlXE_NYgw_4+wI^?CyH7wQK$_-p zf6||6VG}$%C3=Z%?|%(%DypHm#P_%rG{T|b=Himmy8;hK7NBQBV*xFRE)9y)z*V6K zmMi4)xSmqfM|=)rhjRQ1BhKJsjb$flY7XZi3m&Y&Zx2ND=ij!|Ic%4Sww6U(lp97R$<$1wmAM=ZLB(A z)WX0$Td#G(h?$C?ztZ*gXZAQ-UY?uVGnc=cWOFzce`0*R zQV?ubP~P28^!jvQ_`0{+n;OShgZ%bK2`5WX@log*Jrww}K61fxxQgWqGBWjFo3SMG zyVxJ~8FSUZ&qM%QnGIgCeo{E?=6a|3{mQ2DlRd(V*E+hT>%BFLcAi=A_8Fs$FYmH|x+nO+MADfIqg`7A-J(PHvL=ocSCU$tGz zo`1eXV!iu}Wb5>m)6U0dzN2_CzuCp5POD}2=^)?#VcngP(*?4igoi!3&St*s`tAhj z{Do#=Rav|KZTTSoeQ~w-96#cFrB<)`MB03Fc(y&qE^KaXtpQ2G8}Jr6wuE8i_G)hE zh8?zwk{m0N%DZ;H8x9XCxvsL(3Ea^z0DFJ$+N^1(_kMaa{r-Gri1ax`@472Q=R}G^ zMNQ9$%EZ^#cVJ)un3JB)*$Cq(QhO5!D#ccxcSwKyY6R0#P`n~CQ~soTLT*6}ce^|}oE|YFANi4gJmOr9zHckhs=*^hJKL7+LCgyiPQAcB?=yCbq=f550 zvvU{o*r?t{#SJX<&qxFd+Lo|2iFNG_H{Vz)cAhRY-|b^g>9sp~iVm=+qU$xgO@_{o zC@lJ51?O)ARAk^4Zr$BE^m_LjPH~k5J@$r{8!eGSpE1h)M}$P%PN&Bvbz4sZqp6XJ z1l`t3g?taTM3MQS(8O3W(tB+dD^*7XI(^<^(Ar@Kf};(6?KXnuTAqo7TG!hAu5K$( z$2(^iRMga(ToyYf?(|DCaf}k*zLcSShPmdbHt@OAJ4?(u=6Zb#Oq|qVf@iLm9j!$C zr`B}s8hZN!M`1&c-?W2NJ4)SFgPlVUA0gIdq{I6qe5qEarw&{;TB$uSXYQ|HH#OWG z&6xLiD>Z2x`a8AdZ7pdT|0{@nqYbpYi0#_$vT>SMd+*t3iQ+L_of)`BP{?F&V0{zY z2wysX>+RCk+qowpDkX=Ek>}|)5k+^P;38=ztGgOSiD=2_ap|LvJiO;%ri>JJ@5sl) z_Gf*4QpZM$*0mN*&^ zzZM6|CajqR%I&scFx`wF&U^$r$&?@lNHGO_MR)&rnU%%pgVZTLSE}t;X+zEVJKKQ_ z(5B#RBQZZ~m5ZI`gp#)Q^72l&t>7v?d2e(8nZLotZNnm6+(&x$x4jX*=9yg5?>V^? z@n9PM?xFPAh(JpYB8*AfTGd~N<=lIpST!7~KtCK6FT(%P@s2ZZ;|=@$<(LjzA)i-! zb$vcRAfCjRAYL4%v$HeR4d-ob?KO&BmVxNKdf%#9WXs0Zf8&-LbMFKnAHR5JW}J<0 z@_)GiHosbL_U}5lpiNUw=rS6;CZAUZi1qohJ8bLdwV3K|OiUXPJL+1?>slonR+n6z z#|E4&rn9Fl?O=k1B^@MnRV7d40RaoU!Lv`4@Iqvu&CS&r2l+ud3LkkY`-Dx^ zgCU>0M&Qc>@zYVZJ%iR|K$QM^gWFXodOjhKR!t_{3MKKut{ETet}M4M)WEEr3B}xp zDbR-S+Mson<6}k&F%z-sv>R;@;PupCCwz-r%-i76)5r zMSBxGhI0VUg;}vG_WV3KqL^pE9@bR`6A`e@11|R9gh8Dy~+6gl$-ess?-L# zIm3wG3dSWX?Br);hq+v8PGzdJ0Hnxz~;z6WM3BbXSprD|fwKZzu)%(~Q zb>_rRhXFY7``9RV=t2@>5e1n#yxwU79fKDSbexSzaGt2T(9qDH!pk7N>FSISGHO~8 zvwvt1djg?z6d`u75JvmA);Igp;KNHI=r%{!qrai$<3yGGDXc6-a6QanrOFszzkVG@ z=J_#sH>d-TIzJT)n))GIb^!HCI52QVxQ@)%cKG+HF`78T`|9>;FZw_(o12b9j&kV9 zvX-;sl%j@R_%r#h{@sxN0RdV_fj*#&qm|Vl@n;*EsRk{FYqF7NDydIN1yG%XN+e+e zDJq4Sko4M!G=R^{e~aoXd~b^9oxm(fO@|o?hH4q<=?^vKc|Q9UKOA=hT@u2sucL+0 z_=Z_2BeaBgXCQfmr+)v!ccx`LWLs)wTUg?M@0WGetj*(cEUIZv1tQQ{A41iG#G)t= zpDqWG?N-~2LDkF3k4~WHkT*rVp$hg!_fk zY2b9`7B>3=GcBxj6-*at^_QyGUj3bi#DFTTw*?AQHFN-#29?b=69E-C|L@<#Xq2Wuc2`xCQc$k2mqD^hgwFqUvCMyn8I6AP?7Z$w}PZ| zH>eg1LeVHC5=xZLBzb$cL}0hf9+Ey1Een&IP2m^U6L+pg#=*<|e7x^T1vTaqX4ez; zr%}Ca1;P;^1z-6%;pQ`=yG4HhmDg3N=G;&Y{9J^$XO?kL;nTXie|`>rdBqJ=BZ`~| zI;w4uN$Jarx1@;mtjU;u?5Zs7;?fc)7ml(KTw>CZMe>p`6HmjX#J73D5&B`;gKT68 z!;9ZIgxB9$pT-1QsR-i|f#@w2rfC$dFAx?M*DPcZztt#Q_1m4_6FE0LOr7^&h)tk| zLjK}5=6Sw_-dC0Tw_fbPUH9HhmebIF|4#M@2xyvv4}!NhuG0y20KhSIUV&c(t*uTM z=-J_gB*_a=p-9X+#BK=(c0L!WjmWuyGdwiEd<&V;`BcC*!hni8+E_<(!yB@U1A%@g zoXm94-^OyIQ=y3LGY48gLFbkW+CqwJ!{f$H|IHL=7(hk#1bD#^FHI zoT;fdyR!fPl-hkS#bnCcTE}8Z+MSMyp*`E}Uw1757?6*Jx+aL9W%m9xR!Y!~iyL^yT4`!*xw9t@AEdN>% z`z5m>C?okko*F8ro+>Mh_8p-VmxsMxmM&L}XHRq7OMgZFbhGuqVrLO&&l(N#YV>mE zXQk^(#iV-q8%T$>@slQH*oL?N!D*-_=l8YOm3V)*?Zdb37eV}M3_*$F-e^mp$Ndxb zYHf~}<;95c`1=%RJ^+dfH+ySrHmhha(^}ga#rEgnoA1jpKCh6#5WtZQjmYt69oa3N zp-s;~`|dw2_E>BNf?tXPWZz<#A6z2JU%Si}n*J)k_(RTp>XZsdO%ib?bKINTqtmRd zURHXFJzKRmusyQ+K{3UHn!%+QD~V{rEL<_T;p?q~cTzN}P9ppjEz&BDz9f398h40M z@MUt?R@1-r9liAG%--%3iFSKrqF`66?OQ)IW@@(K`@Eo_`$xU;W${t^ zTTmrMnRQ@w8-39bQ(+zj;WzjojV{bixER9GNRzRlY_z^_-E4v-@}vL&RmU-x{Aoi~ z)%m4$^gG3PF*45`KZId*0sx6nRsmUXVPcyu*AKUcdPJ%i&;oe?99$bD!=I-jn}p0X zd2MD@G?G&CQIyZpVnMD-$DyT-&o8{i@JnC9P3s?Xo)Gcy@tY1j+@OcU6{znf=KLCy zmnvalVZ(}T&ijWn@p(%uzAU9o15l7YzSp{HN~@lh6D&DC@KEfd_FvGU`^MZ>>xy>Er61Ppd#f) z#XTg9s_;Z4Zg=WdZNNu455BQ@xO<2b+X-r`-|qOM*JA;ODKzz6KXYAL1=Sh~3a%}A zb=z4{l#N|sQO=`imp2C()ubRGcZyRw!>cr;og)0)CF?c2mR1>bzqGZZ2@KpjjY}rF-m3^)~q&J`njwN?BTZI`R$Yi(^9zqZCT{i$cSnDg$;3 z$WJ;u5~??MX`6xu>Q9eZl+EmtVJRvMFV5GQAppdmMM+0-J5~HD^Yil`n#3m};y4Kr zqX49OYG$IqhojGG9reQ=pPT@sKm~t3l6AIytTf;$zauj(^-vT66}&uWR#shW*Nunz z-hK0g0!U3#@?6Q+Ddwlr@zAP`m^B#$WcD&Qs~rcYZB04mUUTcuwlG1h-zaDdDA6I$ zVJ4At(9s|3?2Pi0z^BF+7_V>>hMf9fVt8ac{FmFS9vX2R1(qU1CiLVCjvpPH7Yhpn z=ZAzzDsi8`l;*LOen2yz;r&g)9?yZDs`Pw59YR8wB$hKgk$NSd*a|foqKsvj$fwyh zyv=YW8jR?neSu$xI->laRHDR7OM`va>;FXHelRUSiI+mxez{8|CNxeMkio`cdU=gT zSEbKLbuhT2=4Vt*f}H8wa5S0pZ5(5d$reJ5*jg0O)L{d@vl;&}bNdVNFJvG&60De6 zYi!hmg9GQe2k!2Yl7a$2!1RIaQ_J6K0WWIqtS<>{fcrq@y`6{K zl)X>yZ`^pQ%G-vGHjB}U8n4gk@2|V=J@fO}EA_T@R7pK+m6h_TCmJ8s{j{mKqfqYP z6}72}nW~EuyZ6(bTzPqU_{YTzZ3aZzSR~KQTK*Sdg~gT{I*n?YscH0(mi@8`O@>;n z7N_$r?ntfM-HIJaWWqSw8gsiv&^Pl5EP2N9C3@Z!-FU+1k~c5(f%I{hCav>;U3ap~ z=`IE6RI5FuqV40M=xAb{Uatzrq!5^Xnc^a+o2N$0uw}!PogErEe%CU!!5pb)8&#`5 z9WoBK5=DbH1MPP9!~}WHV;{7>wIF9~ybsdhNnP*E>}-qkh`}K*(2awFpQ{!$r9{xU zFTmBf`&Y;iTiBb&KLiT;x@j)ttRT-7SPF33`1ZDnYM9jm^A7sSl^!nh=Q=ii^k$pom;Fok4#^PlIU5 zr?_JIVzjH^;y1t)W!nwM*rwUw3)f$#O-DgDHa&ad7FPG%bLJh^sgn3IjgAG3Y zHF|$CUz_A1mw#oMxnLB7v?y6KyXR_fhqZj~;qLB>xa(0+xU6N~@zD`()bSG{u@2LX znWpCS?0k9^had;R-hE42>4XOT*KZqlQzJ5nw12mirYEP)+3{N!hy`)*;}$o+Z#MW2 z-zMd_zk>my%&#l+?LnF3lb#=0w%2RjY#FY>*;%3*N zk*9K%KY)Vu6+)dRv&p&aA3UgvgC;dKlPS06R&eQ%h}hq>d;LfjmO4GgisOO#JqO@V z%xORj;>fA|kAiWv_LuJ9gX#RumO_>?mz`KDNfqJRE{AOqxu?V^#W5TaCgDsHC=)mp ziks+iMt=`&U3bCc98PB)dLnoBMj01Z*BZTs*Xy}WKECb{=2o; zeo3rvr>EyEz7fMF@HNA`&08AEA{DT-zPxzNY-5R;aA9O~yZQW<8hg)+lk%|x+fqz8 z^t!z~zsb+^&zQ^h&kMz6LG<>=YFDV7Jad26SoV*+tn9ABFV15dr+539BhBh*31dU1 zbGct0hsQJKVmY%3ND-ZVuo?6stffpW+#L!8D}8M_e{?$BqjlB?5%`m#s(uV!hDf{V zS5;N8^i(!8KUVz6y0SQ)e1&D1yOGh>><4ku?Co~_!Z&NC=T?soOHAx)yD*trk-xs; zE1GOmlWWZfg4s8R>+kX4Tf>4OHWM?CAw2!1Xbk4cleWZn??Wcll}{zq!dBnD;(YJ9KGFLU?|y85TTi`VSHDI|i&^7RY6D>r zXRE16%-8f7=!qa8uAu0CFhYFh*HGy**|Er+{&yg8;tm`|Fg6Zh;(#IhD@jS`F$KO_ z)@B7B#DMlu#|e`(!Jp93_(AW!BdU3f$Dr4hc{+rBM`D{Wv%>Fs3`|}nHtck}>-*%5 z8x+CDY`YGc?QuDpOE>Te<(i#eZaEovu-o`P+YeENK4L?IrQK<<7d9R{yPaq%Y184o zBL5wg1gdE(#H=P%En#f4WWk;Sy%VzMgQ_=Z5;@*XHx81S9GU9vxo>@WM{@Y8Ry(RfYSB$>VvlcSVQo9*GrJMeq`swm zA+0wVy6(x__&8JKDyi7kzAsNTnJv!^EW*YU@W16+_NW^e^w=)N6-(Oo-sE1Sq^F;) zG*|5(MusG*@9nJ;Z_drHx9@ZdE*SO}N+bMoGX3x<53B92Xw4J$-5H~{=H}F!wX%Jz zFjGjI2rV+w0zED~r1~_asFEP$h(5Ilg?sRBCab5XXX@VGN72`l2J7PD93d2TM~ApV z{?1bIZhIxDD08~!vM3ITxQ=^nWd$e}kSv2WuVeq`m$8JSvcLa>q_>-Xms9MvDP|SF zPUB^^x;bVxo;xv*o-YkE->*U+{h{G+c0O(YnFG^nJqmeCw)YA=f?C8Rrb6XTIkcE_ zgbez^#-oyOol`*Tgx`4YPjRED*xK2{kte_U{N-irDl#Dtw~aUC^}0%qcN5fB?{Paj z&M5nHxmEfs{6$hoc8=Kz12+(Q62&OorD%WD81svITHU(VoyT_ypWUs%mZ`X7a`pk{ zC9S2e%sx*x;7nOgCue86uV2fiX^i)4~=VY+5i0ryaWji|6*ODTjxx3gvFQ zn4Ot$e6Gi+5uI+DWSUypyC2RkR(N^QF4c)EXle4^)(`UM>qp1P!|gdXnG~1yzu>nV z*dLuAKVAku9OK(D`9Ll4u`=+Ys0hiX{^WO(iodm|G1P2#lqDnT_PhH#G?5NW;v}7& zUvGXX?p|$CORb>|kCF_3u4U(d7@eG~`AL({+%3kM&C8CqlyfvQqob*&CYP^)fp@sv z=zXwPb%y{EL|35yfy7LTDFrRoXWyB2v12?v`~I>8CS0o9G}{e``O1RW#^BKQ zY~@1HV-2pXj5W9ODXFb}`&()o8N%bUYsHrKr^#uzUD zMPIjHR4~0r7`;!!$S}vY5B|^Eqk<2MOMV5N3>VAYbjpvvvjv|TJ68B5d)?20nP$A9_B~evZ_lRre$UoL4zu}PPmMR> z4>{b*Cn^fc!hP@e`?B5-em>VC#j<`B{Fz23kEJD?2Hu5rb(ROUpD2cG>v|TJHXFT{ zsI#nHpPtWFTfon~@4bbW5!sw)lY(I7hrQw#0e-H-Z1BL6fx5n?;09P21DXZJ=PaD@ zBPeaYzaWtgo(OssHa410OTH{u%IgCDX9w5DhOq%#A&fOBFJ2-!9gmh9>33;B7)stu z1At_ivIvR*fJ6-x2yo+qA#o-hO}V~tVw#NnEh1Vrg*c^=WRI})a#$SC(cD5}>L`cv zgK@;?hes|7a2Fg>Qj7fKm#PDNG`0oAy`Or}DihcGC%asMqffv?w|hzOW6= zBbzXb?S2vNyy43m9jv>*N6nQ5P;h5Y(*TY=hfl+qhRSKgEME5?s(w2!eGX@DZS6Ut zOl&Wu*K&h-Zf-6)Ir+|}tog&i?k?~np}F3hs4!rbN+hu&rjBTq{>t1*Clo|M2Z*9z z<(&b@;f0tskxS#*icx$@8yjSTvxd@w#(z<2+cnh?DE@j@T5FYzL|#Csz|<%ULFUE< zap;1dB<18(I&n+wsh{Yha)S8&#Tcl z!iFcjvv-ZchrgITUs}09jtMHx%pd>=s1}7txF=NOiX=|o<|oDFl|YXU4}WUHRX3HD zC8Q>UCI$!_Y;K7^SP5nHK#(n4Qfjec2-}R_Rx$Ykexp>pcw4GU^!;blgoK2mv#8d3 zgv%vvPM`YPoevDUJQBsT7T{%tg0-kZ31isppi)w>1N+BbX$z&8L{Y@D7DAQ1P&dFb z{PirE9qEosa4McQno>yXhn-oH7tM)Z2ni{rSS|g}n$w@3Kfu&CE!$#jQ;lHppb|hy z9J-^ET4}i^E6={sC={e{@9Po%baT_%pYGNbb9gBWN@&$%V~hEq%t9-87&xTs_u9iO-OO(fPj%E zZWaRj?>0JJ9#>a8%^wV;>1oH~T6P*Zbd|w^pr9hXO)K845;Db&#luDa7kov&m)jl3 zUDlHVobd_9{+c!GL6jLt8KF)JSTfPM1H{MIAW1z_{?{6nI($Og{aG#& znT}HS$P%wCX?Gd$OS{xb7uD|xDn*S7YDiS-=9kzSQVG^lswA<%%h!HA}CZSI#zlw7dk-w2jyRg z1O*=X*~dxV^P_%hdv<&Sdjsa?_A|bo4pnIRjv4xo_dI#cgFqbiQ+oN+mY-o*?-ap5 zLY&bf#dLLbpOvAoRw&8px|H#d01jE#5%r+1ZVVs`KDA-K;(Zo72k;&256<7I%gWTA*#xg|X7WcuXcUoc%pi3m|khru>(%``VSYgxNzT3vn?+qxzMPiuv4}%E%QS z!F!jq5?P{^wKb|pNd>vE5cn^4uM|j4TJrU5^Q{JDBg5Mnijgb~qGF77DfTjU%i}l- zNLTR$O$`nhK2}5$4#QBW^qo1R$!t(A?q{^91$^5=;Aq!L@%afCz;;6@6{LBtrz9bR zt@RZSZK_(k5j$lnN5FTbeq~^0A5~?zeUKL7@2=!LRZ_u;J4n$?2A*W*pdQg-?` ziK@~7ALQ@}OvEfpRPm3-*{XN03e5|+LGl6P$+-y@`3r2e`l-r4wHWw{xkse7-^5qe z+5LY#4Bb8ejTugr6xm9CTEPXq9NN2wR-X=ySPJjWqq@JSYXzdJ(y zYr8t+B83h#7*lTsS(pXzB#CxTQl!IF2(hX)M8yKhEkzvM$*WF~5{jP&*;?4|eV~fJ zy-sev3!>-Kc#7}rYf`z!jN-HgjbaGkPkN7YneGQlQzLCXr8nR~dB z84oF~(GfB((GKg!SG8-&SjD3wGAGfjIuPzXcJ7_AsAO<~6^-E{g@Fzfria5rF%*WR zAle!ltNzVb@TU1iIY|twM&$0#bD69X@QnjWP(+Ikgp?UWB-&?6rdqD+HvAc&ya=KFPQ@rWPtY7YZxnDRbNV zx5`CLx#;lbIl}v7FR%mqj!2;}-CK;~PQAmIdsXtRh`$EDZVls-F|03~K-7SFwym&? zOu8PXIs?eJJ_0KlGy8j>tfpK`E6WCxr%q_p>UKQ#6-z3%)ynoW z|Gk02LaFSD?$hF{(6p~a?1TP2OqUY*6ep!oGnMgJx*xSe z9fDV~ZIxU}KJ>|XUkt{}v>ug|0D7eD1%TzA5^UOzERCHo@u-6HkR7VE;`qN@fb_d{ z=^z;Dcxthh{6)qn-3~z&U)fEQzXIsyI3lKg15k}2l>)cRv!X9hBT}7WK_POyW^5f? zC%?W= zasO~q{ek98x85Zu?r*kk>@;UNWctj~UY}Je3u#n?;ddwKKcFP%)WvK28e~4Mlv7e? zT0O`du4sZB@9CSgE|DvfNp%uZ2&Q#Cqg{&1%3+3ae%2nwS=@;qf5vLyZf{^2v(uI& zIU$!9%Sj91m>pm(V-JIoqJ6#A>?P%x2A+wo#7cf_Xn*UZIGbkxgt0D@C!*TymQt*Z zc{Vl+q(VqK(Xa?00w6wZ_nGAtr#re&w46)8WSSAG)pukm>O0MOi+zUY{B58>0$??a zLh6)MSmbZ{kzM7xRA-3&w)VGxfW!nX&x<>mB27xnl-#m*&8OOiH&iF9R^I|^rPJyg z3O?@a$4@_2?>+QjYYlr9-q!jFmXQG_6lenO_x<+>#hROcr5G+#rdc5mwtP9XES>9^ z8*$#h)=LvDpdb=^xtu+Z*D9UPN@6n_a2CBfs_FAt{nIyxp4Q))n;k2?XOhqX zfEv<}OIEpi&nU<9b;q?GElRSQsR@LvRc3E+GVWcMtCF?!kfw zcMWdA0>RxO1b4T)bKY~lx>fT7nX2jOUVHWG^+>CR(p{RR;*gGg4Dp9%5N1bG5f&Ho zR_>zyAIZ@UML`n$3vtai<&b0hXCX6lIL~VP7=a?k&CGUwFd%qEYvyxwP1i^XSe^_R&?))fc||Zz|B`8De;9;X}< zD!X2Lohj2f@muF=wQZBFe_k{E{Px}z$YQo@HCD#E@amJoer{53aeNfX)c#Gv$K%z9G=5H_UOq?k4liRZWWnDp1Zl(GHO)Af31 z_0?To(r?LSr2;pr_Cg)LV)s&fZYv?@%^D|gY@!TZyi>go;y1`QZinrq>J4a^|Cn3s zmQ$n%T&cSCvk_ZUXxgRm`It@d=iD07Po!xFp$dOVk3+Kf;*dGk{rI}*HT{FOit+gJ z{jH=)v1>ndw*l&cKU6)1Ydf=Q)s33!YE!vq=}DQdSLe7N+vR3RU(RMY{Vt*kvT=6kZF zQty6~XqNr}8!Te{2nP-3VdL!(xnkWv!`L4NyXZe#vEzqbes;-gP;Yl<;h4H_7gi{m z>^KY|wFwu+^TF;87JP#8K=QkoSig&~7PuXbz1{lMd)ABjwR^ikB~=5KQViZ~9o>y< zNul+fT|NAw8$_?g!H`FlbV*SeM?x9LwAWA+NI<`JxTTH|F-$WkgFtvQE_$6Ec8EH-NDMr!cP?gTl2I9s z>yH{DX|V5tTq=?1s7V7g8Qyc|?^uddn0mT)jqf1Sp~6P*fu z1pZv#x;p@lO1*u1#<=_%Ug+T6T5H&P{5I{7lm}D_S|7IOsr(ZWOZN^~)nv6)&l$H?~s< zCh4jw4{0N+|BMlcufPKl;K%fhx+pAJT+vjdx%5T zK~XP)6)U*$PyL`cMf*4Bg5cg@PE4s_XiWZlWzkLQsX1x#0*OC6I2PF-wT1}xTYgLog8J0OTy~U*1;zyPyWiiyP$^3D&(|~Vg;8Wk1 zpsME{2-7$-^-u8sFIJLus1?(PYVS6CE!92>7aKyFCV!|^`q-R9lLYufLctKqm>q6% zN}O?H7;0BZP~L}d;4KmibDx;C8%QI00ush8g1U!(IS%b>YZOJH27{;_B^RV!@m0?L z;LB*^L494Rp$3@*M>o2TrlK9rSB^Bex()wK96i>ClIHmE{|)4X_CH6VNPRxE`TWa?6a?;OVwn*-9NJjP5C7#*9AG^hezRZ;7c<=nT1*Btpgvr(%m^ zOkZA^5|`W|Y~cEx8j4(o*@I~Qsis#p^}l5Y7UP!8O&u4@KSIyvm`3!yk@%b(?d7!a z+RLRjJR^DX=_3V;=pYoOyu(&Yc{#<}bkgCQ7ztI74q5gA*VoSdAV-UHv*Ga8yU9GJlh$D_%zwXAwq@!#fF%q6X%U{n9@L0oLT zBwhhXm`C`kIQ@s#ipOgm-wU`m#v+Cy%n8z968^tYuCt%`ntHN|h8hwje4s^wsxgt? zKpkSC#by^1kC*rHU=c@@?oOYNDgK748boGZzuz@H_${2zqQB~}C(;ZFlYaArg zhfVTbqOnazXR(zSKI1z&9{n#-v zF#yz?qosVcGuJpbmHTpW&K=?M@&8g7&hBIXs(J_rlXx_ra<(AV{B z&A$nat=nm{Sgxf0dNThyQP12tGq>2f-KQ%WN$z^}4 zC`nchb@O4i4F!*hQM3K?U_}{e29N90P|rk{G+P$kQoHTpfz+NtDXnR(0(X^PtNnb@ zs5-157~caBR{9CskjDw!EI2FKzGIFZz=qM!<+&Ni5kN}w+-fCxr%^*%-z71c`p#oF z1MSh(@2bCE>2>!xZd}izrt_A&TF@Tp^?A-KLi`jE(pc{i8CY%5m&kisoMeTQI7bv> z_9e%}PtoRiQp{QRT-f05;uIcZgl zXF&dm3_g9V3@sDhmXVa4Rg|;_m(J;2B@F%SS0cPgH0yGcomRcs>?HGz-uKB*oL;`F z*H$3#_7I{b417z|$$@eCoI)idz-Wh7;W9sIk4^n8ZeJ||0XjK0my6RS#ba65{VEfjI3RVUN~L@CN3TEQv#Jh2fxil2QUS9Z52pMg}CaZK@j< z9No?Qj!CQDWUW-}I|dPddh$@wV3icHN3i9P$oqy;I~zL^q8+$Pvt};+D9JxPt@TVK z0vmxWZ-#lq~#GZ ziGC*gUO?(en*+>#p8AU;1vyw-2C%WuT63{Xrshc|Clx)-$Rgfbxbp_p-a< z%NGI|sFsqB`;h^5B>L)(kFUQEt1EQ6N}GJ8(k-}-l6RuNrjusB1>ozSq@8P1wFW(b z2xh8pg?i@g8!-BVj$X!n9Mo3NJ4BGbck$d0ijC?&Y&zGQelsrrwqZ zpgIYRiH!S`+*bw%nGc2jmkEQOE2BF(Y3I46(4*1n?}&Z)g+|&88#^;A%2$ZR6wTe* zObn~SzWkRI*BQcxOpAYTapw3Fn8ZSn$@akF$l#OYk<~%W`s1Bwe z=elrgHdGV>dg9zkYm2W!%JaW!fN>W{q5rHZV~lvfW159$@kjD;9g!W4^0Bh5z;-l> z?&i&B`BdEA-d4uQ8@RL<&H2n!N z&sk`^WJW_n3)G-RimAk*k94Zue0-@W!w1sk_yBS#{eQ*Ax)MABLgtq*IvaMKe_+3f zwi-h)5$ZB9Ablf!*hvmt1U>2Upaf|T?%I|3k zV@0MEK`afuJ843rYUUUpzq_5bd)|u%%BJ%NnXC&l2lWaUR(MEp?eyXhM8hf4QXZAS z&ZTcU3XAoKP`fD-XF$M!#f{5>e#g~F42piojF<5P0;L)*t)qu0z2Zqv_mXjm5MC*7F4Q&Vz_(8R21GFgFVWI%WM-a)CWlGX|DcHIxuWbSz~>55`pC4M5TN zraSm6AnRb>BzCtTOK9vRuEoW_QOtx@3p`XOod)}j+qG~+9 zKjNVup$}n;+O-5th{*TXzzB=O!UH;uo`C9cp-z`f=a=2lG)y`s<~m(*QECQb{2Hkq z1MiLg`v=?1+`O zzv{26+cmE^s$JT-r1*?_lAQO)8rpIy(3s>~S#6@Lx*f5#uG6X2N^eioG&B=V;rm7| zOWlpmc0xiO{0^hN+k{Pt&h0AHXszVE*0&3+y2p4$O1`V!Hr?Y_F^UfRai zd&&41yll|(%ZnM|G%(^k);q~-aBor94tI1-#P*_`Lv#tgL?m4Ht;i6to{`{Faj>z4 zOFrP2;?0T!{m$_4Kz}IqB@f1ao-I$O@$CiQkH%72s~4{SMQE$F3EP*!0r795q37Jy zomsOjIFaumSCkuGSy0 zn&E6dU#;y^^!uZdz0e0)K&8y(0QKkBCNp+Kf3XW7>({`m**~zbw6k{G>46LDT=)@` zJzK7Bx6tqom!zWd{^97}IYgcdB8~D|Ov8|3YXe~i1s^`WwawFZ zct;Ze8%g}m41cy>3XiqU^z`Y!{6Uzg)NH=le!HB_(mKQ}q`G#p04U+=M6rpmAt z3s6TU^bed&py0o=?~R8&E-x=Daaax*kIEUnOp1kx7Z#L1ZcJ9ssoV2ApU5h$cSl<$ zmB?4>oR#|_iDfJly*&L8n$v6Jhceg60COB2cKV02=*+9L{;&N295MM|Y<-_fzqbW^ zKpx2zp&X*4r3Qs0#$&RCh@eqd44MXKL}POGic*u37$|)Nq3}y_=}eJJKqHWVrWBW{ zo3rIV!D$zL;T2s)OBIM z{$%;f`y~luEk*^YkHVn^S!IE>7EyZ2*(nx|H5v+$ss5A{cC#&(u`m4!6Y&;WjVK&n zPtwe|Te$Zuw&vFIzT2P7BU9xBF4{p~U*A#l{tN?oV7u@Vn|=OI%g*{$QG6^yLRNQ! z1FzfoPr~mZ8o0aT2El)cD*zqgY0-Pyod}baB5NGE5d-HcmB0}$R)@{&d}T`YzrA3@ zmVJuSVu5x-6gh?z+4*+07)to!M-d&I|6zSYZu>*%%sSvg~K= z>47mtU|#e_AT!CO3M5u}7>ar;fK~T}` z#Gvp|UNlF=6YT|CC?an#VxoXw-oBEBZ@-DCjK#vsQac4{^0GU%^Si8Z{M?La=2~T? zjZjHXu3h!6CsT=t+_&zS>F?cNvzsTf-eb4DMjzah!*&;P z(vANSM1-ZRuBN&=5c9&5C%2?})$CNuMeGA{YQ8(ymzDxvM_!G?A8(lmA+@+0E&~xU8-)$!;_5Q{i>r9Qf41Dqp8YFDBu@`vF|zU>#pJ~-rBWSL>Hcdsx0qR_ zYn;|DOwuvcF;$QPEk6uI$HbcRZ|cR{Oa;aMa-l8?3t6`6nW zL{_cMJu(P--mO;2i7Dj>pN8DMR)MhGvgY`1k^%_=<wVl4|kv%plR42Qy+Z$dXYC?cF@k+#IJ(Kj9G z{N}ec#o!GHf`gAwi2H>pq&%!v9be*|7p&yK1LmFBMp1AEN6 z{Oc<#j7FE*(;D^m_~i8O7)}q4>6qv%zpyT2f8Gt7z@ z*RQGRVy(gY&~OzqudmcJdkpZTd>4MCR?9Z&wSRt)CW8NT<`+ENgq{8e(69u}q2MXK zb=i3@O)F9+{_Wpg+%2vGg7cE$1FeJryAp(NvJQt#`I|L-C{j9y@8&0>3M-4}oc3?Z zm~WV|6-rWuc4e#IYMg~gLtO!{oTsn$8>v+~g7zn3xxQxsa_Iip0E;)a!e6|f07=&v zme?F{Thu`8ml-6m0DBGV9d=+(byPX+MDd`k+yx$r)x)7BVmS-`*GE0aZXS({F;2+~+ zA}rr4Zm0)|Z_O;4l`$k|>y71+Cl|9Y!)O8rwl79Q~t3#fvcr5%b`M{=ijmp1g?Eccs(cxOl zr_S#eKJ1P_oApHnT z4sdAd+|Re#+IW$mK^%C*g9k_u1PzTU>k zu84_tXHX`jPek@Uht!B?=9IR1CJPLk)7`bU(C+gX>+W~k=UkSvTvuG*ar`S{6qe6^ zZKd-iN0-kH;`*$6jB3CAz?s=B_%gfs3>n)#0IIV9_9~#O# ziab{%)AW7sh5VlNePCcD9iHyavBTm5O1s>;A>f2Yu4T@jgkPkc>`C#gt?-z&;aN!q zA9gpb(bS>Fd>zJ@4z5wf9DsjCgpJ39@8r-Kj}h9x*ybN!#9{~`?~P^i-R8dT`Y~kl z7XYRg8=V3rO1Y*GO!5hU-C(ud#+kA^aGq0jOmh0tmWSO<5e1OfuevhxBLTC~387_8 zb@jBs60G|F+5#eTfVH&AaH0b*_sGuG{xra=LvzX3Rv_u?|0ZhKXdG-60!iZRCqoZo zmCn2?ET;=2F)ONeFqI+PK#emWu4GQFPL27DLqJ}j6mw)fo>(w|7SmDx-ow!2IBrTS z?@cHkA|6YOW)M-x4K3e0a*p3}kMdVZ=$j{|p$Fc(ofXV+(dstG3O_izPSeMQIWMV$r4{5^j z$^qn^SRmEeKKV0_QR}G>&kx)(Grv~_F)o|0UTiYmP6$G%n$@RcV(Oe+Vlo1KZf+)i z@BTGyWWGxGaU{<7YuannB=}53>ksE7 z$gET_+aiB!TzJvmYnR3!ac|##m1(?Ir%{Sz)bDs0yFaYeaJ(Kd)qnfG#&3O(GL#HT zVxXFIGqx%9%a=>ZL4gEE5yTR!FpR4mHV^#kpS7;D8;ra0vFg(_*HwzK-9~>M6 ze)a2{zXU)Hrb*S&w7jmO-@W%B!9I6s(JT=TB?uNY zDik~&Db7%|+#$08WZ}=qkJK*#b8?(}TU+19+7>IyH?7c-f`d_!k}$C_-{jj|7eVtX zbLG6mw22#AK?MY(*4EDe{0NLt&yRa=vv#wa2)e^tX*P%W*vg4Htg1IG7n0 z^4)K3lf!%QnrAY5>A74c(q!65u^S?waL5XTjJNI}Mw8Y))_0^aalgNSh5sskN}5R? zfE1MS3Hl4m;EP?cv3QJbgTr}psR|SjtvLebcDzA0<5`g-!>H^o2QvWV-{HQQRYRzi z5mpr`x>#Y;9h_|3HZ{|a;$_yd@|@mj^r=Lj83J$3m$?*(P63$4@&(DAda4BkEiP4kZW`sbjx zd>NO$CJPK&t!BDX$F*(FG{XxkjZ>*(6swuSp!V?B`uVUWB1x`n`vyqvQ=eh@7{%ot zXER5@LJh7;cjRv7T|FXb;lpp_IA;xQWYCcB^r1wnIVQBcscF&aBLc>Wz1?B+9N^?y z(Cso5{I$Bojiq~R5yi?%8#MyA2@zGt-hRTMS5>~u%3i4N>y>&+LFXyWLR9}qFvyU? zOaraX-pI3hg?N#+H;lx0AwC$ccTTO^Zzn+lM9;v0Z`4hyb#~Sc#79y00l$r+o}Cu8 zvWJD@V5WYr(xrU{yuoi_keK&jLizvgQ^}v`ulGxG?UtKt*E@Ygz$Sa^H{nMoWunkL z8;vBd9M!Upf2fi#lboD>i1!bD5TGEVFi8&&Mp7hlb8Vt&^FoQF7B%<{C|7cF1Q8P2 zd!msTXv35-#TK#2(Z7na@`)!a5))2RV}{rfv|cFid_}g^k;*}Zf)#^1sqeuj7rl2` zM6MP*!5$MAO$a0Y^0@Qc1!F905WO*N{Pg*=InYA}`6K5zRtmbFs@xr+O5DeTX@DI% zajR1EwSS&Qsb20Ajs!23BbM|yrxKW<^d+D(X|K9c=XIu=l#`QFpvJ(^Be((;_3D=R zz3z9{?Ku+z!m63OHv4ODt+b2CRP&tkb)%3)iC18cHAx z=yXa{#7>?TgCCEltgzJNpgsbYU?h1k4B&&VAsvpWSsH?-As%d_m*Ig4cHC&PUq`PY zg(f32Sx<;U?3P5uCX`vmgBt8%GuUK9#?xjo;ISTH3p`6l3?HuR?#1L2BWF=WjUJYj zH9W#VsXdRsX;%E9r)fwL8c7i+^x`hJok9_Yo~o4d(4VTHtl0j#_B|l|1MZZT@;agP z6kR+>d+VlY4nyeW^Kz;JpdC*oO@RDq*O&*x$Fou%|Ay0mjrEvVV=ukPftl0 z1Ox<$qzLSc482;9_1u0-lasXo;>WU<`lI=sZrkUNmJwtedbT`%n@0svsbk&-lJW9H ze7-vyd-|04Hc{NRt91{P9xC**nl{rhhhaCzWtEk3JqYBjKh1SwrWy+&gB}1JC-`tx z+{jLii-CUSTl@O!rNDe~p=k7t?Q*-vLGnn?S{3u{Dc|QS3nJD-DXt3LPS086-QA=m ztuLK-J3*nLp?AW2B)*sSz2iObO|3>F(Yy8Kk=}giBPa+%#{=0yyCbV%u#Y!K&qfn^ z6sTa$ybyirY8>;4i@i9&I3>&guxFe=+CabLiYy^DYi(kEVd?Z#-T(r7*(p$S9W z8gkyjcZ``oP#5BTr4f4c+AAzA-3PAzX#Lzu93J5Q8bEOlYp>DaG+nGP+yiJ0&^elB zwGxug-%5x+>l(ljf0YUOFC~~m#BPV5|J>ws(}?04`40o=lI1T+=sJTyGH9<2jd&iH zKOl^Emarxfa^zi1hZ z{2t4mpIhQzSQayU>jq&#Tp58Y! zc6e>K%~p3@ZQ1+XX8t~*#2;b)W+L8;UWbWQ2Y4$T99cYCJU-T1>o#=Y3%=kfbmVthJ&Y!s!=sDHDa+x}4U>U(IU>t!*@OCb6! zo$YyX51>ccoQ~Ji6exGuvl!%!(5d8lowTa_=vwi8@phb%ySj~x*QoZ~PQ5OFbMkmJ zof^9qbEww!FoogwxV+AFO9{gTjm5@NTv*Za{W~`<4$Hk-=V77PD(#rLxE;P|vHX(q zoVu>CNrK{Aw!c6U2f^{-e(WH5`CO&KAoe^#=37nZBrxr2pj?TXti~gTxTqQkv+j(; z`iMAyRu{Rd^&EZ&7PE210y)MX`5DG=Ys1QC_4TQ19Hocs;c@+$mpllJRd{ot%(i|radqoQ8~dem!YR!dM^ZLDYlMZENTEXW%p02`H6WA-@P*<3}y zAW|g#=v-iJxPT63pwsSj0uGWSxdM*-Aq!PGCm~&T!&&b!IBsXivt?MtPZFqVQa-u4 zX^TCZzKKju5p10%tPkvn1YHM+2_pat~#r} z-%Ah2)I!ZhJK?-PYzUOm{^))EmCNkf5xX^)lJc~b} z8PEIx7AJN@O@P;Jt%8p4NZDnPm zO;X>EyccG_m+G9C;1A-GQEx}NBgtg<3QWuI=+=2*ff_)!^HUHDjN5{CNP}hrxId91 zpgY)g3$cg<8c7>kB;BB&G&2uT3>*Xk6%`3%_y@p9ZydX^s~XY6uB<$IQ%pQ+jVz8IElX5j##+OUA>)>pB}t67uxaPLvJi zkqn|L!VPX;GNEwohtxo68^NFtx!zuTB}mW2LGYvbmRE%!F_JQYHPvd+eRL2?6_Mk{ z!J;UV7|8+=Hr57g@*(wW>0fi7J8q^1)b*Y%kMDs&2+aLju<38)SZx9HuA|lMYV~(p z2^3Ox9C{SefMQcdX4rW`@Og_Jz;iLT$O#yk%E1@-q2a0TbT9hMn~q*~3LZia(FN;> z*S;(|uJ;igE!LZ#u1)YehN=(~VH0_iseFy^xX2D1RDO*j{raIthB;RTCiyel;V&W( z3pB+>gRT%L3EP`?(WTGyv|I4s_FE(ynS6ZpFUY2RvUVRsGio}kW}b=oy|F*+_-t0W z=5>%lN>a%?Pj^N0(Z;4K;=J@7oM}w|idoW{5sdqm5ffFo zw)HeU64{mxCPHU7a3*=2TlQx~Z2SKG5zFpoC1j6j?Ti%FT%yjb+%iCBf&4Gz`M$<* zGIBNwZ|wPEv_J$Le@;5+k5mQVv_eb(@XGJC508`tOAKzBUS7tg+)gT%fA3q!Zn%Xi znvO5>V$0)BDAIJNc=7~F-Gs71`Dw+O3@snC~zb!SSjg(i<`#lB<|7K$PQRW!`sCdI2A3rju;ig zfW{2;PluEAh4ggaV(6@DU%(G94Vveu(^F#1<+XMW>2M^_|{`tM6oG!cV z;**=Eo8P}J*~>ugM4vxjqZU$9RO0>$;qd(PeY9mfduISDMmH=roKze#P}agiK=pYq zkBD>SQXUMuDd51d%d3xNMB$-na6d1+d6^QU$kx0~QFU(gZ!y9)8SQO@!JaXQf5C=X z7LV!soRT&bL})}I*^jj7Kw#i%SWCzX+g-ac${a_PR9b1BX0Or3G(0msxwJBO0z8TE zOr&u!_M{Fyf3a5e>UO8Rrjlk1T7+JDId}t1w?Ai-?&OUgtC81)b45fhZcSXxF1#1g zD*?~BCGfm+mAa}OhH~-Y`kn3%W3Balu@Foo$k%8OD~))<>G~@vF|3uC-dVMwg)+J${$> z=2)pSUWJtEbn!&#mEXNmm(vcEyvRkI-;=Sz>MUzA%xn3^=JJyJbjh1O*zzQ`y!7#>cz~hI251a!+(R+fJYZxS%vu*JCt%Yf%6g8^lA7 z<(?%7F~W@RHXRM4&=#X#HK#x|3P*BF+Fw ztPS`2i5g6?GC!}bqxOQf1hJ1EWLeKJd~ej8C|IgYh8=nOSYKBB!-)0gzRJH(t*Oob z2sc;(quKAjhz;q+;mUS-z=IsVg%t`o_Gxjw2fjAqMm%u=0YD&v7xe#t@0fhtq~Qo! zxXZ9%Mmh|S2p(@;EQlxW&orLG>sN(xwzrwQf7K41ii%PX>l$aq~qgQYDtie;(aa4kZGY)LJl#j_uK0j-r-*~nrUcc0qQzh59RvLDgcIxi_UEl z97DP8bGwRd&SVld`-hl`x;~_v)HFin4AR+M$na~gg6iwm1zp$EE;ECycc$YbSxq=V zNN-lk6?{vIb>gJ4Zeu|h*f7Jv3k`1?iwg@UZz8#q7{AgA7$lQtxDR?YNU1za>XC6L z1z$a(?`0mA88>%t)5Za)*TpgB8}-ISAYrdF{oRB|xJ{t*xvA-cj#AYbbjTjjNfNwO z5$EKyc^5VCi;=IduC4}Qr1W@m$h&-10@OqT&ZT)qf?&Qm74i7H3d+=prpYv+$~k0%&Wv^Vr_ThvDt{h+hgLjPBn~AIJ6RmOm6ouKHGJjz zp40=mFI)44rK`t3%@0sCbAC}gy;In0?wXG(ZOIQ!OxlKBB)`ZSR>}SkdBS`M%<#iU^ug z{o_GvM-|;;*d1R468-uCYWR9_xQxt+&zG0`{&RE@+a$MHGGpzsM+^QU1DQx{uekV_ z;YvKdai!=pkxM41vvqXBmR{vS?L({5ru;7qQlpTlAOl74lfzB4gv4aJvmUwK7EE8K z(jsJV<$X)Y>D%($G=L+I7&}%lSCJujEULM0u3kM0y_x`tiDvnS(QS(3B=rDCU$bsW zEsSf_9lkM}+YS#f)q^EqL*so|SSZP1eQzt9rK7P7OdtJEKwixR+Pf-$78>z zGlWFq`;Voc-6WG%5){+Ay+N|yR>qT$#Sm;XSYz9wgMpDZj1k@baQB;ZJBs4xo}wf( zfJ1l#hsJHv-YUwJXo&*0$AGy*?yVfWctf@_1Uew3ZfPjiq1IJao(W8h(w!~~>s;EW zz?fmk;W%nk7qM&M&GMUa9<0WTpVnuH2>jt95Y5-lurY(#)}~^I&>P=yrLXaZ(}IcT zeDBPw{;=1<>`NE^@}qne!yLBrNu~w8A&ZBD7{JtM7>`~2NM%7VSJaPhP`&vkMcU2; z2OADYAyCKYfSitOEy?}9>|v*5r36xVss;U(*1>utGEIwKfinHq-Xlzo(1t|fAUc(w zDLxei1-Bm$VJNIgQ^Eh@VM1P8L^-}fo-%)20f?Y?OqyECn5U&_`M@xKhvSO_u=@c$GbSL;4T2nlJx9ew(rQK%QSXkAF5++^Lsk5mamaF z0ssDJIQA_gp_Z$4r;6ue9i4$ezr4r5WcU!)a7Bhnmb1b;TYR!_pw?qN{$JdyX`2kl zP=jEzpINrg=8C}$vC}2!bw0$Ugf0}${ zerY8+5U=iySgytfs-VK!Z8v*P==IxjGgXNK zWfW}(vA@d)vVlus(y$nVnR|m!nrQM=o1uDC*!a!8`?A-~a=m@2;%`3nZvHwJm8>I9 zNgT(`$w>vB*35#LkiqL><>*+-Iw)Fq&anL{B+D6akG8e7&3Yy9Y#@7DRSkg!{n;gX zHTi?aEnEYgb|ZreV}#`kT##C)`H4KM@`CeelFYo_1WG#R)%B!caN%KeC_|nlH2xAZ zD(2z06tke;xsd`;yoRGC3DWvi8t%C*1Q1KKA;T0`05`mHj_PCP>=iYyt*jggOQMF3 zw~+;c##R;=7iXtwG9D!M@2Hb*XK?jcTV)u~w}uk+FShhMoVHi;nC)ix*jkHzIk^s- z85h{8BFT1%eIkL{?KE6iT{O?p)X^e3t?v!;om{j2iAPauGG;t7Z%fu4PXb8a_Qo<) z5G*Y%WvGI_cVPqb*9?b*1lW`Erg@qU7ZBIO;j(gP@y%;|Tsr(Qi2c3tpQI}!IZB$T zqpYW@I_uEkn4Zk)^U$!hwe|86K-$t6_IjpYxtEcch-ECVqB80}*Y;Q|%JPr;1k7e> z1P=?=l>9ab5THe(5!p#iL~JDL&c)R<42U#?&(*=Aq~!3vTtrCZl02DyJtn>P!DHU6 z{=;U9${O&*`>C43qK2dXj+=X8o^QMi5^P8=n&AA03y*_A-Jymb--vEGdFB)twic2> z-!EQfO&9-wMtc6%N5oni%yPCA?1^6saYA%|yt#!)79twq>$P--qo#B!+Gh(Okvg1( z7$jiD2RYV=n{>mEBgXxqA>0ceN=U=6D~{8C*p_e5cBh0w92)}jNVHZD#d;%#6P%~V zkit2y%2;bgbVr}1n&Cvg{ptIKwz@^fkBT(N!X{av?4}HmuF1r1i75d74eQ%2wY|H( zlv`8PXT5Nr2>&L*6K=`TOW@YoFI=_+lHz8-ebd*sw+K+%Z(i*~<=~-iZ*PIV$KE&z zg`HWTRs8dQwi~oD^aV8O)gM*^taIHF(U;$|Y0h|W>W(DlD|A|c^8#IpX+*NMz3p1p z<8~W^`ardYO2^;k`tYvAyPq++$4f3nor?s1eN8_|blXgvEKBWmyhg8Azq`%PaSEeK zMb1}gt9Llbm}*uOwL0FOOb{GoqIy_!*=cneEzZd74pQCvG#zdv7)Oz4KhLmFq~S@a zzzFOq5wCqW*Vo_BXzv#1w)HW(Of9K6qam?*y|_acNYh2{I6aiy+K_iUR&&z~0uF6E;P^vl`=7^n9;Mhdo~{1?XB zO}2V!IOuCAvT8FZ-EEqFw8)CGkFR zrgP$kNXYl9dfmOCvEVQW`wlqVKkWH=xE~Zoo{Yyn>wEj&98S~Sg)2V%@Vi?c zbOt*5t9+zXyD3R8Ki0c0HpjDR4%-Zm9)K&Omg|&<+HWUI9^To@r`$>a-TwPrqm!kC2uT(C{UuG`bJwlwUQEgUHK^VkxJAkPVKX+IbM{c}`K&0-_xAL4 zagn{ocl!aL1BZ>OSL;7sj55Ez%=`U0IP`OWS?TOt(GW{R)d<7uedfB$=UZt> z%Nc2&W%4M zS@r@J#h$D6&bQ>4UvJi5e+fYlzE^iX9NeY-0HF10l=S<{z3c01z`9!OP{-cwwr0ze z+$j3~;#c0uc$Gxdo9DBEG`bWXmzR&igHUSF6i~uoHbm5qfZ|Wz$K_&Ya{_95EtjYx zR{=h~Ya{Qh1wZwqYHehAI@j;zzPgjQY5VDZ>s#8vDK2Kb@;pd_F5Dz~%L)JWHYEHi zs)%+(VD!VE@5A5cV_J^)W90wt)a%19ycra`dit$x-Kq_P-8J`{p+t!_T^^dNFT2ciM72qy3K*hgun^o zx{rR>QGV_gH)HkY6L(nl#|KsA!ZcWW@T1ZQo)r?v!1qT6(cV)y|-ZIpW zo=e)=Z@ps4si<&)uHd87Yt=N35|8wcDrR4xH;~S5uITsb;f@(V1HE3Lz<`(Y;xy68 z+zFbadXn^}{W?%m&5e+@hMg`l!(tFD$>; z%iNz|G*$Jz1j$Rq^_8rx%WFb34n5us$KEtOjJl8m=KE2yzfA~XJ7LjW@H%;2`N5~6 zKH9^D`{n9YEr126tUJeqGnV3wi%UZTB`p^BK*H42iZ7Q=haMN*suE0u1N~7+%^o=W za>K@WEavd~-TWaGOGP|0%Mm524`TDMM~W=YZs*sH4!-tB^L4D~JOo^)ICk=V`P$D< zx+f)H_=!_&y#YQ_Pb_8eRa4bUyp^XK{urWWhu-*L`9le9$q^4Dq zHc(-a*`|uNXJe0`#Vt&yE$K`(tTn|=l8tIFSGVd`lpjmL>t?e7E{>LP+zBzyb$?kj z$Il5m#g-izi5Tj}=Ye_Sz6vueB2{alY^v#~8FBkB_XnN0?;$vG$_WG?OUWWE9J-0U z7;?F1x73hlwe)%G(~KN&A1ZPWB4R9{xMhQqkYk{ilgoeaQe(9|(kgC@2 zB5tyZPNl>A=O zPCJKZezBJ1N7pMJi?fuSWp5Av!+jmeHf`w6XVsJbe{8*FSewzdwVi|j!HRov3sT%Q zIK_)Qlv12R@uGp^?pnOKLxEDARG)}HnYf61xOpkZ<@Ea!X=8@5Roa-j|j-)!;l zL)56rc+^@Hk|QdGjFMpmdQ5Wl$;;A2(}!}aLw4iai>lSJW~%m@XqSF1p=f+GBcGcp z(UU+7Q6a+am?Mt%08C*+| zq^QeDw}ivrJA)eP=E0jUR;Q!um(VKSNf%n~4&QQLUG2Ap#tIPnJY>$_e|}>$9-XA?z84>}L+zE{WImfu|b!yZ$X z=mR8+R`1S5^I2HAxf`RRJU71~48r!8LaiP`<$kgDPU&oi$)=Bg3)-LQH`6dX}J zP8MBucPbOtpC}8cDfr{}XH)%5=t9UmqHaFz%8l=4P!CuF33OIV#NGzP6VM6=+F^nY ze=YfRxI5*ueB$=5Kugj~Jea4OCf$fJP7WMR&RjtAEVb=+%G6Pm*bfkUc2!1N%%!Um z^PwnwM$IHVq?qAMtwMV@x$yPo`HHfnuarP4?m5Oj;rE?j6|4WIwZ01iX#ojGL#F+d zTe+s*Vh{zHP(zw<8MeGToSQfs)9(4{ba28sSrfwLdjD7aY2o(r_h(SUA)t4}?v=4P zF27cWyo@C>V2P)#cJA;YNuMU4^12s}2|+MA9I4-5$K}~Rm3|1C;YBqTNAxohlg7su zpp9A;S@D@ogASR;mm6_l89QNb2G*)r_5-K*2&mI+n@%J*;(ZIr4-Cl`oVo~_mOX}H z>cSrPr(q=yJFI+raorzz^&WsA8O8wgt`xJ6F8fllvKh-dIy!%cF9vUYe7?|_vfwM~ zwK3uYKw9jXEGhO`;?1o|dfA5I96J~cQfO=kORCfv85yFK;}54fS?O5@1||c$7lA-H zK}iIjxIY+xlhn0sNI>Dq40(NYGDe@GR45g_sE>b@oz6@&$TeDTk_$M z1O&ox6p*8o#Rm-^S~kNiO7oL65RrIGQ)As=Ga!0E5ROcv#?U3IJYAeM9K+o zxmOM6z1uzywFyLDjWSUysmbfA=~|T!qT6WF5zD;oUhzis`b?X^0?&JoR1_pGAeb~!wd#!qrsIUjm8wr1RPxf`=lzPZaaAtYdL4iPdh11tJFu!8=5QruNB{y-5+=E}`bDI6Og_pM# zIx}k`0EC%N{MoCGA6}}?NOfe(wJ3A4n>~-m@a1#rlG9$3^D}@!d93rNP$^4B>HwWa z7?lyZh~MQfe|kNBfeAKaecxH%(RO5NyYr(=yM@Z=o0H{wJ3HM&M72_sF-u!|UHUmb zG?%TgD7F4;3*{=G7~!i7 zTY$C2)`>v2kn`=}?be8*Chu0V22MeKVPRog3+1A?m}H7EwC8xgx1X3p%=@oQ)lrg5 zZp&42swo3)N{D#VOhu5CS_N~XTMbkD&t9navca%UpGJ8Mmw?Ew$60voJzLg0zlWBW z&!_FHMRmwfFsx=gN9gH#`BcvgN?=#r1@Iij7@DKcQsrLMVLHs$>wB108O4n*f&blRyFNH2_;4nL80ANHHK-x)ld4qka z5*y1WI&-vlfTn2oc}|M|M2TeG?%E7Y0Q%X~qDhk79K5YpF*wA8%#GIUee+3MIW{_4 zA5wWtp{4jeWpHVrS=q(P*jUUjPlw3T<8)OUsv9ET4>a@rYk*O4ms0Q7;Ywys1Kyve ze(&yjg&Z8gp?td3Qm$7P6|Ks(nc?N2yi(CrXJo}NV~t(-60Z!80$`4W^}4~OjP(zO7r?v>aeWL7E2`RfE&Y= z_VF|)v37VXOM|h)cem|OgYvq@+<>v`IeY#rh3^2cncMw)oLm4J2>?t3NdiM)34|at z_f}KVfOprImv<8}aAO{0GWQ;Q-%#B~>?wP(0`R$e-%B64yx@Rk1~~zi*CKizD{W;@?FwVCcV*sM?R@lP(O1#sg4KpncxEdbk&Ei40cG)5`H){WAMM zmF1`Bz^#!n760_H@l&7UO4zD=*XD&)a2$P!g!o-$#@DnEa48Ip2;4AL2eYUP>oqK1 z*XE}Zau?mY8D;Aaa4?HZUl;Kf*WD?IJw|MjIODSS;6k zPggGg9%GMn#JWEZY|yFs2{qR7-b9){I=foD?i?S90+FD>s1MIs1Nb9v#)Q-D(qdxP0z9(M~)~R&VSV3UWRQJl!a`j7* zVwnbegA}HAzckwYPx5dyNd=VBQgZrj8AT=_)b^Py5G@|S@D&I{F$Cmxf&l@IfcFJv zaZ5OAa6IKD{KPn95&6S`2+1XCNL(j1&XV~nJsQQn2|-d^kR5tbgHX6soL4G?-sxPg zuXJMz0D!X6=6w?*=3Q=z(&cp#1@%0>k(!p{m!wS(QbxaejFc)a&k%wT4S0pjGzn(f zh%L|qBT>8~ey=hjp$D|BW(6DZHz!Oq1K~D93iNfNe7}Crd>q{fzdoFt%dqq6IPne1 zB@r`G<9(fxo%P-l1cWUmm1?nkE(DMHZ2!E2z|5vHUzX|t6TGAUdArb9#L%{Lp+?oD z$<4py^1CD0te9dkDuK4X1g{F|E7Z+td>M826qXcYyAru#mbm;OWPN#9)63n1W83w`hKPq!ceVN)J^khHARMy!B6O^GtJ6!+u39C7hQW%e{VSEF zhp3ug%u~zsC|?1RxuBF00DU@1&>GtOFrdc$bZ7wT8X_!!B1uh3&SyF}nQ8VG4lU~? zfqWOl@5ND{;lnl6V*!T*A-bp1cJ~!)Dk9I;_=jxQ^nK5%e+Q2` zcL{<0iE5z+Rr}5FE4wBo9D<_VPzle^Uu@00WM?t27nTXmI5#@0ChSf7{meiQVIbO?thR&OWBIBI8LyH66|@Zl1g@bF&m$_T^LZL z?J=XyH@CL2TNM^`+C(f$b4exnx5b}jx_u+`n1Hy74}_H&@nn{kdFo4h4vscUi|fwE zOS9jOSL>}}hd9(?Vw1b%F&VW0K@1?op*6YB9Q35AOiW3i}WO(=o_A+WP2I)irvg8?RqfuL! z6J&TNn*jmZ7--rWsJQW3l_wLO1f0}pE;as2@*@I5T<=QE)YQ~87k++uGN|fpuz&=X z?3~vxC>bv!8a;=7>ij>#% zf7^YRx&QN)R;fX#$dUnycMJa6wicstSyE(h9SAHZ7baug64IEV@rI2}S!Eo*KpgVf z>td)OLVKyf%BtCjl03`IDp&YJLMM)yQW<5AK#3xwqg6x|rp8xw zN)$;a418s%!LBh6WLRB5oX-!9G}ZHl1K-6=E+#g5bV^0VT5{FRlSOF>mV6`hslK5h zu6SLT_DFCL+JLq@5*qziz)Z1i`41Y3oC;fPA))z*K{g60CFl%O&!I-uzL&+I!TJxb zru!~NT5gf%eOk#`9t>Q1D4<54I^XG$JNLad6Wq>u<`@0d6hDClA;;Iw2`6?XI6S0% z%>L^puj{xo2vD~P{65^Q^HxR&J5aKM;+Wk}*IuxLTF0#*9tRU7BrNW`{d<+ee>=8l z_xb&9V^?-eW5>%wM-vNXx69GOWIvVnC4!Isee(jbOYafZkLSx`|I~r4Sc)%?y_!to zelH^QTKu6DgFgaeeSdyo8htCO!|c6z9z1>8uK5Ga?&Jfsdy-MI zENwKfs7en4JRYqzVtiM%F!k?vwTUNPSygo$4VKR4vnwj8h zv6`26u^d`udOSRuXP&RqAjkzcAOl15VEa0w+IM6v9@vD#b9)P}{{H@UGyYEpe31aa zl0haaYLjuBds|yWWkKcQCLdguJvFj_^Ua$#0Zw7bgr-Jm+37UyW6&TR#NY43uV1x5 zy>%ki!je4nZ;zh!lbs!6q<%w+vRIg0*f&^83R+xWpU1l}?eMCU>I76ej3TDT^VMsJ z%<-ouUz466+6?VKMEUq+Wuj52rAXoxEEvQSm8-Oflgm5-Xp$ z{|iGZFOC*LeYSD;LP7Lg6es7WS0@qlA4~DfE3Z6vc0x=PZ!fQtwf6M!qPNrq03!GR z)JJ@MVy;wCrM=ACzG~(0z<@7REj*bR2#Pth5fMKr}~$MqMv*_g*bBkpw)olryDs| zs{&+JL`xx3P5oo1E&GIG9Q*2Cr*uv>T~uG^JPJu8bd%ktdcm z9Ijq9;TLT?JGX5fQmIiqF3fh0E*!c4$$7YJr9xJLYWz7na6#SF0x*}CW&7#XA;#ju ztE)KH@)_9N>Wp5&)po^N<8{%jng>>}pVf*Lg&MUzuXod@7sr>I7luU?ojadr|EF0s z&Aan`o>Z@U58uN>)8qZHEdSFiPf?o>y~LOlqBRd+U5cwr+D1mgo5r3~a~{7}SFe+` zx%~Yz5gofGaS%@+Z>b*WX^gG$_#~Un+6>l#sB|-=w)uCE!2wxUT z+)~WozqE}wM@z2+3RB{$ePJG96mGv9M??Rcsj2P-MFX;skEv+L0bz0lLoBai*R`RF z9}5A%&JX5-1bS&{3`7RXRjlBN9xL*9f{bl!7jszisaB;~Incf=K2=X>=VQ8&e%0 zm8Xqwuc|U)$5-1TLC)1Q&ElEE)OwqT6KwBp@7ddUT;$}b?;? zG+y`j-pe0cc%~Pn7zof&S~Am{uW=c1;O1sd6G?xsD-9csZ-{10e@KZ7aC4h4yR(t) zpMK&XioA$?OB)e{w5{>p-c?W0WjJLGQ9DP*M@L4-Pzq|R5E*CeEqH1PmzP%#`!=y7 zqpFq{?oa|uluWmsu!Y50M)0TQ_MTm?$im0187-D%AdbdRKtRCd(9qDihN1V2kfym7 zR1~YNG^;NNC(?W;y zHxEspR+l_K+uI)@0{?EXPTozs_@sFsgp#HEAwsXkD5PZ&*OU|Q({Cwe`v8+5m^BW% z`;PSV|7ih?j>FtzZ`op=HxE`DVxsEb3)tuW_!U!nH}kXKg?za6`LE?cLNuDS^zpdc z;#`@FBiiM|t7$TRt!kgRl38|f){(KX_Oo(R^g0{`X6+HH^ihj;=YyuG%=q@nU`d;nh+yx8O+zH(EYS7@a@LHnohi{8_jzVT4mfJ^ux|$H@i*1 zY6q>a$`>_I=kw)X(qsG~YFzp{DfOV5x&-A)Hm*OH&bHQu2&P0A4u*|ba5o#NdJlsja+1^)8eEC0h(;YXx(2f>nwY>?yHE*P?OE)sqgyMzw;tK)Hl76)}yP}juc!fcOs6)*E z196D2O&V_K3*1mpXDd9vtEdygiz%#U)floCB#G@$uhoMlV|vuRB z%3E1+3;pf{l^C%UYcTiG# zgaDGAxiS?~x^&r+u}@ZsUZJ|Yjf6og05~DV4GnIe-`l?bGz2nbNwjfzwP!*kW$pI; z`T5O;y4mfV+i)E~nVA6~c>Y$A5FHsxKvEMo-XQacxXh+%U{b?}w$0hI-oZm`t{wNI zIo_nQ=y2=W+I|WT31np*?&i!eG$bZwB(k)+oX*UAiW-QL3bPi}W3wRIMbw8+j*e7I zwHI^Inaw`64Gn{KWQi^=+V7z<)G!OMn5uKStAlL3N-;4J5y2ltmcuJA5#c2ros`P^ z;>3pQ9Br1JsZu=R$nB{D@5YesE+2-_us4&Ofr?hnQGtgse#%hvU@4*Tpt!+Q^IB_q zk}#f+7+(st*zj0(>O*i4_|G=pauTgPJ!ndNZQL+_KPXQfkctP$g|H2ud#OA#8~GL?+1EBTKeh6KwLM*oK*!C=mQ1ZY$C+x>g_v(e={>;#og{$tdx@5ji5`8KjeUSS zf?d7;jxEzrM8sgZX!q|3dO9q54WYTT@KHT2Cza*bwF$o~j{?oQ?-^aafZ)I18F@@i zia6oa&doXcK8Wymu(v(z9O`5G9tT?i!Gb{*uU3!@jw~B|?6JV^+m#+`BowCfDM28( zVzVo5tNQAtA5rvuC`1KCNaBEqIOr7v7%dN38$9_+ZP6+D) zJDFMUEu7&~I2$Ct^>XmV{rfOpUX2-2FS zq9*DKJIY(+17*?K{d`vK$TY$Vw?A@zl?t1in{ySOHCFkse0{uuHr?qX=Yfwa&64V9 zXUBut)97GPvJYGSKDFrMm^e;{wSA^Ay)Y#q$jO;&oQ6doyN?6~!K?VF^-?}2aw~Iu zXikY;w<80cr@M|aM@a${S6^WwrheM_zr1&KYSkH&&_CxoDmhxUz;qOi+kIwtvE8q( zgODT{4bhkDDy|BOEVKcII3QB{-->-H8npTIR_JJp_SuVP6P8jaT~JJBvA|T#&&MS> zImVd$4Sak}EDa0ui~M|ipB6vTfNKDjVDj`J-ptHQmu90vdte%TFGxqmJO!LKNND~B zIL)i{qdf}YAMCDz%EF7@Lg^z?cEYaZ>E83ooJgZ!%4ec@3rlt7%34E4lIsE8WhEBE z%PT9bKRJof?^{E-(ns)y6*Mb~i{)b(nC*vI=6Aq_Nz748E&rHxsQ+i2iP)g)$TFAL z%Z?rYx-k$}8CF(S$2MQvt1o*+6Q(X*w}8Z(-P^qz^7v7XqI=VFA!3?KEz@9wVp`AOo@Xo)3tG-=5F zd$YrKH=p~=M=XrCbha(Ug+&3^51uB6nRs6dkEHT&?2ZAv0s8PVQcjjk75`3@(w;ck zOv4}>Kc|x?Pf?R|Ucw7x2>`yCwva+z7a)YD>B_xWH$|~9FsTTHq^{T)kuKA0A)g_f zBGEbq{c+A70PJS%Br-!HH~mKX2O}7DaWlTo#Cv}?ftfn0B7DioEP5w(thl-1ICEWC z$m8aGU$dHc;vrhXOBLguyA%P(rMg`@+ex|3l71TOYc90lUz+nh?{kcfS`&ZT$?TjH zc;2%)=A2Gr!hQR$0S>!NlY7qM8*ys8ndNyq;jb+oZD39~3IZt*^)dmy$f zW-PV@NIql;;m!jgLF1cfp`b*3x(fZ!gKuoW%31Alz7S1gLx!oW9K%Jk!6F3fSnv4IALC=bKQ|LJ`deppvm#C?UcuGj_~IUj+BxtGNA{3=eK@cJ~eIQ+?L ze(CbiA#%=30Zr0^j<#aCg1?T3FQhu92gz9gJK~puK=rswfq{CDDL?z5*OIkwx&TMV z!q)C+)z1Zz%mnIkd5!>e>x_1%_0C*srBJLgts^a#vuGeZ3=LbD@bWAZ~c8Nk_LI(m|>2G2<2!Dsj zo7_&}hvs-$np-wAlbtIjDcftOq@-LQ`&d}ob~hf|BkO3q)o|L|3%AqdARHh*Cw0%p zExdjT=WV*dvS*@mMF6_JmOoTKJ`$SM1-N`XIic8dc(VTTW&3Y*i+r}{P~Fx{ud!wz z2Tr?}@67JS(;rkOVxk?tt6_nFxez7~;7!h31^DflKsyk?(ecQQR{OB0snUitulB@U za*ndNGVy)nwOvdG<=IB_4R1^8SSlzAG%!4Ov}KX+>C+!T*V5=9^s>>H(4AdGR9(vc zO{^^q3Zdb?tCK>)B|pYtO;#K!cL)S?^Y(XlCXHO< z7o|+qbRE49A0N&8KG73{0eBHkno;^6xou!_aWgeHYu|0ac{ozNQqPJsj?4T2ep%@O z_Xe$3hCyK&pA{_XRvTy6=2q}9*hk0XL2-6Z@maKVe2SGcFeOcJK-M??nQjmcB!|O{ zc63~sNHHjkP{{_(pHX2uf>6orN1s7Ppwfkl49>qVLGb3qee+s{Dbrf}L|gz%pH!*V z6xWhyKp3xBpE+(gP`25qN{hv>dfs4ckGI#_OS&;%fj&W*5omiH*Wi$eONj2=%~%77 z2?8Z6mgd`Qv$GCgKZp+ZwMt2Ip;3%$c3YM-ye&0f2IpC`r7KJIpu(n|@s7+(MY&sk z+U<_z<>euM3u)gM8kHp}0dirDyt0M4cZ-8QlXy~EhG-N@1xqjAu1&67Wh#;!G;qR$ zross)Fz9C5?<&|@^mOu^65$MkL!1+*d@jR%uC!myrarAAL&4u19KG&CW3ZcfQ1VsYbh6~ABGvJ0xcpTB-1>+!Zyz1mm#S-pG+8Vc zU05X#Tc8sOGr$k3+ zyT~0|YoG6#YWUuREkN&4%$ZCdz21MmM3}`{CLaGrau5;8sdik%P+0q&^${y6eK>7@ zvJ6*|+0^93#!e=*)%(o_FAB1ik^}+UZ&3|QISpy7tveq!G1h;vQ{u$fY3u6FWBcDq z0~=$$AhP$JH}AJ=*1b0#ztAdvpE@RTlpo_nlO|t^x#*kC3L@lV3NNw++JC2 zp__Na)YZ!pyOAulr|^D^XNsjNCEzX||CVXg>V3DVGzs^4oS4KP52`e0r_6b4h2DAH z%yZf(-+q3TxUQM}xOhPvI==3?+0^kobS51u{;NYdUvFTI@+~Vb{l(EpZ{F&Y|6{R4 zMNO@s#KXDL3pbbe*k1kHwwJEZ^~WV$Y%0G;)0ZpLbrBbFQ!RO_Wsk=_nwEqPUysZC z=(m896DngLMe4+g-*n5hALcu+DR#%&b-$5eFA@SPbnqL}UGWCQeJ;Zha^~FyatQcm zSQOp@dvfIbr&yo;TeRd%DysG#`=WXwS*NW%d@5549YthNm+KP98nQaTk~!Mw)*r;H zhbCih7qQKaHmrde7#BbjKXU1~e|UU2Y*{~bm&jM9=|;swMsF}xl# zsJY%sURl8#w%{JBgNwo8uto>@d@e_7J@$AR`Tzh#0kCz3+C8ZUyIo#dlIrS2G*rXm z-~DKmx!4b~d`(bqMj<>-^E&CTjL*m1;!$?ugux%LP^%8*V@$C3p_|cf-a>J5D}^-5 z9XF|vVTBqr`Bf)teT872%p!gER3HG%Qi#wcqO^#7V+w?(4)?kvVqmAH5|Be|GQN-c zsxQC7090tP1Z$KQdtfj_ne81MuDqW1?N!T#c_1=B*z7)kZtiF)&xa&fyPBz5DD3qM zc`;n3Tf>Mw1JM1FeS8!iD`|| z>;47QA^MDxR=VR;&UK&JXuj@qDaVd%J4d7$YpniP4RkB^p6B3?$LAB1kF~XLMa$LV zU!={0FQ=y8?E*Ija%Y~A0rJC*EJlAH9gRQo*W#5K^oz!2$!_}E`I#2NX7m!g*N`*6 z8&$Sm{Q22|j6d^)$6ph)G@!4%(u8@KQ>TBQ5xM!xWm@j1fm9AE1V+u7_;jH3a(SS1 zpG@;EdX^z=?Aw>rcL1`ABSb=6j)+H0IxfRvaICoJ>C~MEa^l5QEsSQ7#YFVfXx=h3 zEP-WiziC|@Az@Rm-+{%M?Ia9KJS>ib7nYWl;9m|83YT>y@Mdo{6C7|D7y5V{U#_2; zk^obdkOZ_&Bxu0Q(e2*Lg~UrSm|VS1yG+bj4FayKQ1rh`eObPnv+>?Lp4bzL2 z*YT%1q#1PJ1&OOUZ|T_H$|)z0^1XxTLpry*&ksAkY^LLf-cX2sw}%qe&4*4CkpMdF zS6jba>}`Cd^N_1wY_(V)np&}|?s&LN?Duo}2VU4#d z76TB3A@k2$EDvVp?jG`X551s{c*n9EsV&P`Y%yF2KrLqu{%+4tZX15G&=!k^ard|? zLRYG6l7|Zc(*j>^7udei@{$=1fCIs>JVD&p8#jIi&Gh__tVtl4pmN;B>c@#q&GJ<= zP#z4$C)=r0f}g*;0-(G~IzSD_(aMV>Pycp!e7w54Mwez^o~EkH7xlKXH?`4~7x5_x zl>}sc=AUhS@;i!)E87xDS(7zHwcoiN;sF3Pm6IDA0nclBSuNjrBl{lr7GzkmJSuVk zX=sv&?lg}6s0i~4E%RBz#y z7B6L49cKny)3=I@Knor`^vd9#n1*;A02oGO!7(--Smx^H=H~toexxX5%BGXNgA0Ih z?_S)6r{7gR8BV2ZaHkVs2C;Rr6J5~ZPq($Tc`vt+CTX)XB;JL%ME@2F&@X?9uZ%7R zfH_CU#}oeN)7&1?wVRVzx0GgUo7jSm_D>FukKb(7_jouG4-?82v!)Z|0&u_~Af7)L zd!MCb*yuT_%^3|5U4X>~JM4s^-3vO8PgYZ3#2S&Q17JzBx*9m><~XpZz;`H*Yi0uy zBHVfXE)FX;-fJ3pf?Qma>&p#IZN|Xm?IFb_T7u5rioa)`R#x7^$+c`4zzD?uZ%Y|L z3GkMus;^@~wqm0Zke4c{Q07u+@+MK1J%(;-C^BJL}cl=>bNowZAyiR%yeqhEO$)uGQD$@7hv4orX2(+iCq`5HA_J6bko6NB zkVp2k*#M*zeAWxi4(gE4cAl-}1-?r1h+1vehhEOY+WX3?3qTJCz|n9&Mz4VB9e~9X zU}`iX?l{iyyyD`P`hw%RhH{-|Dd>*?G;ZqUuo89ddNh7IC?^0XAYEU0YBx|)&l*w8 zu04VrO*(W+r~@p(%>O#qjMJ5EeavG34HL-#WCgDrHOx*6xH%Y^5ul(0={3UvCD76k zW{4T9A~-%vY&eB9f8u?Yc@}4GZ@)=Nh9Sf}eCy}{OcZU3A_i8N23xJ|L2ze23#9UH zkqj9riuzkBasY$%9oZi~EBYvEbs&cOIVuq#a1sDt^$r=@{=2XMK4?9(*KI3?JA_AJ zV&)(8e*(fmiRte`eerjmeZ3?0`%1$HRxe|@(j=#` zJCW50(`(ahA_2nLlPQ@Rt}QGBs5{+t$f#|1jHRp^s@m_p%WOW8w}}}aM{mmb+m;&C zx`{#K-byumoUm~yW`+S_x*m?YRu7{HKijZ|H7B*wX6Kq%-{s$#TN&BguwcgJGDpdV zE$W6YgNB%^Uwa7wC%q=n!XNW(E)Z# zm5kB3JL#`T!h&*7Q@%f3=f0ZPE-Wncex-jhFHu2)G6KyT_I8;%ZP_0i;3Evq{>^1VH6pO&1LB>m$U7-_|C= zqHbT#pVU4q871Zo4!op1d;c^CAh{Gx1`Cjpp&+ehCK~#PN2sKIECL7s2E5}2fanms zoW1dII#Fd+1w~0XR8~_{lQFS|4WknY^wb5TC%Jrx3ch0laG`^>KMG`#SQ-2X(U0q~ z5^~s9-;%Ozl4shDz&axA{)dXR5x0JN!8WoBK$8r!UA;P<^|Pp$U#*ppLYY`00l?Ke zr);VY4B;!DMBhJrt3=m*NB(BSAgy2D(Cq3pnmz!Hx=UD@g!{r4No*CK(C{oM?6ynB z!w?sQi-|%P02^@Z-Hp)Frj!zsE8~-O6;zu2LlvPaiwscjAugbVy|SZ2LPp}-6Qcvv z(M zgI~!q;qdtvYd4X6$5~h+82gBqQb$Mz@20|(`P!AM_Io#CHpBVqc+N3^$II?c_pl_o zbs!uP4mZQ$L|dGiJK&3+Y~D9uESC%XKP!YwkELPRZ-)C~iAO*9>20yN(aVZH&8Q(x z_b|tklG_za*<8u7G5$cdsnOk*yB;i_V%!AFCPWEammE*7R8`pIth4u~459U6}ho zfFK;hrcYSNt&tb?tgLUP0>b{t9N|Te>kW|g^o<0-IY`dcMhlY~Gx2KN4(H(O z^*==0>Jk9_yov9;!{>H%-n=>3<$sJvB|}H-5xGj)(@5*i~O-}Tl3(K&U2(pe0g@(-_FSN%H;U_2ZL|{ zVOLXfw@=SzoIg1QhJ&KQx=F}Z$o}(HJ6Y~X24ZD>2PyjAweP}=fw*+&h^vW@Ppcjq zQU>{t{~osyE*P5p#>Tr5m$fLsy(@X=cq*YeKj|iO{(=2k&=#J!uS-K1^%R*1vs@er{VQS$%yUEx$?9Z;-;I8QkwaW;nDLrnuKWINl=SNZZxkc?$wX32mgL@uqX@tZ9bZOz(IZ+?~7is zR=HZtiS6g>P={6Zv|NK{;J6)xxd+Xzczd5)e6v(Moc^9aoss_tgPyzAT(HiZ9)OcJ z)SUBN-7#ar1b|rYak<2||;G83w?(s1F2odOC%dMEAep=H7N$yr41x zz`dhTtH=k0CZ}~@(j)GUbm{z0Qmi7IKaqs^SL7LAUFO#cyH4 zLP&AGPX~nk2DQ#MmKZk{+s*8@e(xEL4+#V`2uVOm>QXYJ!8W zkzxH)#c|_VFaH9~`gdZI-JX`T^;SRK3_6YWX?LJy*_bo1rvXN_o5f^U^T_<0Ht1gB zUI+c}r0(HwJ8S5Ha7PT5`(=5|&cSZHoihktHhONWg=}@wMdntY?3SfN$~gFsv_O+_ zZV)wdfYG!fY&U6Ij({InQxY!HpNfnAe_8;+dtxIMG)x%YH!I0EiZS>bB)o0fn*ZER z$|{52+VEi^Gx9*pVCFLBGj0NUK)6(wY8#*v51QPlfSLg5Y#h_K6M*v^DR6V~oz1vd%Tg>HS9`=jVsH@j;|6enh&sxAGTJ9nq zOKnE{PC$#7BnAM0(C`m$z28Hp`76p2_s6*$HifwR)})xq`}Omw$5Ks46%}?kZ;bd;q1eMU zKk`?UC03<+1kd4mV1d3SR5U<36kDJuC)NLe1&?7RKdL$xFI11&+yzt{c;i^bVKuhLCkt~exq{^QDcp8Y6@usSM51qMse zbE-$Qd3%Wa{*_U3srWYT z82jw$xplqoF}ARW#^3NXPxZ7nQtc=1vpe^^IesMKl*>mX*-|k)$bGr68LIZ$>xa!ix?FurnjS0U>EHSA?COh_1;($FG? zX(`cRxeVy!z8gTZtXZd?99(1t=5Pp^DAviD@$;iT4LyjOAXfZyD^!eMks2H@KrMNv zy?=1$;${pW0t2A2VxRZ+nbeHL{oS3Nr>GphY}{F8(yAm1`aUk;$BO&y9T_^(r|!mx zf4*H_f4&|^$Wp&M-@R!w-q-qjW!kpGw-w)m+Hv-`>G|5xFGXP&!OiXM>ZcNUm?JN} zeZ-f4H9Ljva6G)AzWe|!KM2O=4K zzbA4aZn#Kt05Q=Jd%oScS@UX3*&)-{3wbv_GC;I}ae@i#zDPK;S+V7@w zo?V%Z?^Z=u zq9KYz=)U-f)0_3!P(VQKf#qwI%-{!-zjR7A;ICa$dH>*jo+OP@QTen{`Im&s{-lLw z)HrzWG)Ms~-6d!Toakmef39A8yY5y-{zrEt@-d}d;Y?|W`tt56K>Fe4o5;g8!UI9* zbWmCEf3_)cdb-cH^2^4VIbBgjISMzjPx(&X|Jv#}t4-(QIYI+6mLpa;&9pB3KrHcm z5wJF9C{UegWT05|yI($@l$5{=FMMW5QKPP{PO#R_&`Ms7!$&k6#;6<7ymfw_Z>||2 zfD<9^WwwrLwbtsl%}qnUE_`F^cfQ%){%mxDl0k{`m6TSzhu{aNNHeEM6@zA>y^5``?pZyM)sAICFVVDO689bycRR4-5e8!fq{YOQs%*6VQNQDU&9njrPy13{Deq}hh6^bF$;#k zaR~>|F_eTQx!8yp>#*5=dl?Y|y3AzgYHP;R(qE%!Q2Wp;V_<<63Bp5=va7{zE-}V^ zWyxnqI`f~7eSZGip3i%nK_EvHkq0sEo3Y@h$9=N}xVw8GXmEFzMuNM$yK8Wl;NDp9;2s=;ySuyF>zR9J=C1dP#cJrUs=lgINA}q} zk92)~T8^`@Xnth8DK>^6Cwg#ao&@C>UOfRw4x0-@GEbSv!isZiny>hlUj~-i zUN>C_l0L=B@qI3xJX5eFtNqQQ&R&7Ju^8RI<*-Vin!PWIiOQKOj6fX|B_{4;)WZ(n z=T4e54AML@T1}tda`;Lt0Y@ueeEO%mdVH_)CzPR1MNJz(jTJLyUnA$OTJqT8E8(X> z_c*0^MV2M1Zvc?oB{&pTYcKMyO(r<{v`xBQ^?3i8kaBC5_@J*wXbHn6x_U*fj!J3> zW@`7C7*xh#JU*XUbEIf^EEl^gEnkIA#BT`+Hq5)5d1G>N@_th6#r`ePL3>X3DEvS} z0~e3}(q9fNj@Y=PS1mXD?a$NBFuM0B26%)f2&HV3A2?4 zWHJb_`$C3B%BMb@mk4=6pFF&e>hmQ%-r*LERN|C6tq1OnF!&3f)=z|y&Ry;E@Y$Mvx zdzkZ{ls?cRcRrNS%cI$fOp@*d`Tzi?s0VU1K`BB?2T=)YB*$1gD`G`*=|wDKI^a*8 z6(aN|m^l`+C5VyycflDza%%LXXqpivmG%~U!Ohx{%ONH5D#X;PS782kHMq_|MX5L> zWE#~bm-C?Y$!aJnQviGf{v;|An9?IylrU&k9xN9Ayeq^8q$9thaXd8CugxZN2x25a zM5Sx4-izr{MlS5M`=*97sm51ERPhnvazokD#a2zn!u(jF-whV}k8lA8DylwWS=$4!q-(ZA;*t4Bi#m=drX8&4P3@KsyQ9k>dg3wUE zusi{04b~JEG{N`~7Cn*$(C37)7u@de6*)^;=?SD@bsSc|n`s)A0py}$_j*1w#32p_ zYTKLu_vZ-3oPGHpm^m^!RZ~B1_v%4wa`SXo5`(1}7n@($X8m;RN~L_w&cVa!bMDmF zbjHgj(8xY7iG+X{#jI;Nlv@ql`^n>N=Hl}Bio?~celcbuC#$;FaE5m|OM-u>xMblu zm&1@&U6xBv7k!8P3DwjmziFkny^|CG#}TL<5PRZz``T5X zvP{^F4kVWH8Vra7<%I(aWe+Chi)T75SUJ2v7P(`kLNY01KK7huUa-k&xE&Jq6Ri)) zD?s#R#@qd@?7U-cS%=+Scy;kgh#rA{l|k9cW$feD$@lKQ)R(~8wo@7P%UfS$Np%l% z=dZf90@kCIl6}re3rFGIkrW0xH1eIG2BO#(Nk1PYzhqQv+)tTi7gisapi*@qxwxO1 zH)E6KI6|&w=?e{M1u}m?ATP*J_>x6n(ciKq$c14Q{%%{InIVZQMh>3ci{&Q#6yE1( zZdS?2*%RY(eeXelFq?Pu-kAMVKK2F(g)@Nb`Iuu61W;#BsQ`q6SaJrNd&A=vzKIXq zA8+>t&7TvOq0|Ds3hu|GWGXlX^>kb26lspDxpdy^rhuU90s=K;i$IesBGb6hJZwg# z(NolQPF;{c(;9`kEP=F(C}V|x?ZW@@syg$SDXU3}Is^ZtYw8c2keHQX&Sxej->Z~d zORw7*)xoT9+6wbIM6&yLS{nX)$Hm)4mxaz1jVTVSc(QQ5($bU}r+-$d`2Iyc&l4GCM?R=g~L)E~X-}-ei zc3brufo$YwivUj@eMY1>;(igjc!r!bk2q@1{dfEThOHfkukT~ENQlfw8B-Rw%Zr*j zhZY5Sv_pQb{;41n9^Q9_D#Gh_(Xnx$6i{a-;%*_nb4DkX!%8pnjnls8!tZ5yi|b)- z<5$iuxnUSWg-J~ESL@$sgrg9mRuq@ukC@;>f27X zJa{)w(jfV_w;6Pg20ls34p}%rLP7%JyKLu4;MenP8#lZ2V0nyekUbR<>=4@x2hsQw z$E6jc*R$d&_J5tR08J+?0+n^lSg(?qxGhd%uT@nr{f3?d=_-k!R))t(uit&0658%7GJ%86^RtwZbd z*bEI0gAIj<;z=*}N1#;Zi+2mGx|~+l!fPrSkL$;-09ur;mSp_uN8gyI?3R=o$E03@ z?R^WLFeC*Ts zEWD}T{ESs)9j^-O5pZxY$D$;hf$=Y8JcO}blp!oX?%U9ZTUVL|Nu+NFjNvh`AQ;qA z7Jb+LK1x@AuE5mthvz&gqh~U}sc`EZuBNZAuM-Q(9*4EZSv9`Z3FKczb(S6tyOqTs@y$<*kzQc@7VihA@pGHeN zgo_cvgb0TT6N`a|=IHh7?{}a@-6kWr8LNzgfM~%`Z#y{w2PAph;K@h#--TmM$|v3%i}zc9A21P>Y~@9Juz{8 zN=BU))4e3c^;)r1>pE_PWMz!T@hc$;Lh&D__w|r(8MqdQpsnT)^D`{2{5nTr&o>L& zKho6rN2%+o`(Fdw@gDRjKs~SxS#eiK>D8!bh_&wvPResSAuot@l1}wu3LZR5(mGVA z9#&1$X8V0{xu)O-v~Ug2U(Zh(9yjw4<7{k}hWIo4Ma~;EEJbE*RXD%8-Q=0wzRQ3;Ohvf-FRL(6y)~bvg0O64TA& z(nWG6SAt*$+xD(Sxut^ZLk}+|brGr~Ck#T8*ZiMmZ>Iwos8EKNEeD^AUkevs{+`Q! z!~Sb#A3iG^vS5Eeo~%=1|D@lQb1j%EOSaE2)oK{X{rZRo#;BuM8-T2#Wp&=J{JL>1N^Y>HP1k9DD>l z9UyuZG~^Z>SSnkTzjRjy+V$r|o&w8(pd(so2yO_MjO3k_vwS%l=9*lZd$a%ry*D?I zKHQjfdjj3kZfss%-lOlIJqSzMQzIj1K#sgWWD>-!P`UFP0bL9;Y)iDt3wi}aN!IDA zn#_t&Cj*t6V3zXJc|?A7j@#*fmiF(l?0ApA#bH5!q4#zQ5`!h7OL@xlM}a*)P3mC< zp(P$Q;6h_ufp}6|TU#}I^llewI_5VvJ|=n;)m(Re(|CJ(f7b8FvHm!~WG;eY!ANDy zsQ{j3uK|mI2^PC|`ftFlu8+;O@QB;;QqJ4EwN9VA3IiwZ$no#L^1y-caF ze^RLg1xbVB8mR!NO%rtLaL|PF1G)qHL&AiSy36d7zm`c%QqB|ed7zF&rT1!51;X>d z*A>Gl{R~JJGWoO%I2f9-ZR8?pyQc z6Gql6Vs7p8MB4?{uvEMZ_K##kyi@=qv<4t#v6rQvRFocpc`{_^pVRo)+7Ie8KInsX zOuOTzP}q=Dzsl!S$!H*r5#>#VhQ6>KVKi!FWJ;R-a*ONE>e4`Y^x4^YIls@Zst!CW zNCpz)$22VlCDP@~ef9XPl#zg$@4?-=ww( z8O+F!B5Ss^O3(dB)uhc^ zkKjkM|FP_&&~48u7U)9xtPC*g3^xk>_c-)9;;i}uBE-PRKGy&h(#sUXl2by!3Y)r1 zy1qJFt=?+lf%VwbthQE|4Tjb?R>_Y;7dz+rn_c{<)5@;$o9FO^=#h!cnDqB@`&sh= zdqVs&6{lYycRZrunpY4t0_<2B1$_o=DK5^-%R6zasS{S3*2@4=Au)&3n*Xz-jhmNPY@lWb{wLVKi{ui*Y8 zd8)}Hj7HE>X?I%P-JkKkEU#4SS;tG6JW*5(hCRpnx+uwS{VwHfzrN7`;TuuzVzR-O zmwDGc4-BkqcBl4kax0dmH@shN2eN=b;9nWV%Omq5Ts(X%mY6gR#I(?){l|euFIvQO zfTs8W3IPTp?nuB-1Sz#UP~&3SM2^~aXTt3&)_am-%4C-A>eSLWTD`|o9jI76wum}a z`QiiWAdU5=PW!@Rx$@r;=tM>#SC@*MIi>0dQpS+2A4DMg**k;OZF%p)XBz2y7lmN7 z!o`0-H;_aVOC;a-J&n@6LikWQMtYQt3OAl6BlY)5uCLof_yb_{Osg1?(|%dg{fvPw z!gi@{@7lc4JkA&7#VINyVnn8jX(CHuO~Gl{fAhXutp&viLK&n2;_cfldwYBOKLlO# z?=2D$?ANTA85lf)z~^r}SrWdANsy4u_8e=s!c$t}iXaX5yX)eft^VBNV+CkMTcba7vY}47 zIJuO8x=IG%qQ9VxQ&)bWEbx0L{62zhvwpjKccM?lk<^CoUUn@(U%vI;nKMB-psMzD z`AfXiv$Z<^^o%P5{uU@L%%0oDX!WBu9u$h?Cm|28Z1J+(T6yP*{&ydtV5zDjClO{{ zyy`r?FOaHY2}QCRinkbu$QZYLB4xcs*NHTT}%dY9mlAYVeP-Pqo0HW=i2LzWQ~A;_a%0~LlE zq9!%atLh;GT)HE+w2Ot?UtC_l88SA#O6RiZ18h@DTTHi2poS$zK{I-n*Qy!IjuysM z3w#_MJvp0=M@iym+CN>+RJNO`P%&L@PX`c>;*&e$6`9TA?H(&kiwUA zb%NFlpi-yW7e^@7AQdSoLwFP10H8tZW}49L$Qezx;glFu4)u8`rStntJ@P^|>Q@T1 z`G&eJ_(6FBCJNOD#Gd9Y1KK}bvs@VMf>5Xjpv;zyE` z7m34ZZAX=iM_6P1n}Xh)cM7SjQts7Q(FQsCA#{!erJx#vIF?9X3|Z;}cj z`F;BY27b`*I`Hv17<8dj>b}tGoTvYLYyN^1ZhWvAe`sxeUS$wT!U`rtDS*L)%&}NC z!z2tu?*Bw3Ruv=t8-oK14EFsO3Z`Mo*49=?ocO4^UfYs5eaj*t85|#OTt1Sl5NR6} zF^^1606aQZtJgqc&icI=t8sX^YWZ!Vr26c_0yv8m@~M>mg8@k(-;~sFWzbXuD*rNAp0z`k{fpw>c~= z`G{P`)|aQ-?^$yI#TRtT|IIQL%CAv?tjTIyhZkjv<=LU1(VH;?_N4Dk_X;@#GppYK zxPcMNtVbTQ=u^U+HJ9zuJb(JPHn+3Pz3&@dFQiLj-G2;Z3bUFnTAhw$?^Fk%l?9VX zM;=9No5L6QnjC$d+f1f2)IdVG9O)Evd8f`rCilH53>uP)d@V=A2C)j7&jk{g1d2?M4foOc(cYs(h(bagcUEU!V{0XU5J#xm+BKXyTg z?<*{$z}`LT@EA;?=gY;rX%_7A8>)5k&ii zT_*+)3xcs0nUbk7l}~m`h!{N~Dk`dgTlV?oMbP7+<3wMV0(0&B$nT?1(QQu6e*G8V zP0gyZ>)lNC&#(L4Bap%~j{Z{y<={lNz});ik6~=pnWBy6$Fg-zg}kflI?vCKn3CVv z;$vl#KLz+?a~5wiW=h|Wp9Zcd?Eb+59ADc4xCD7*j}iG~Vde#Oy^Vy2*K<2x5wm1| z2)rDP&nuL5yqfBo4y2A5_?t`eqBkByO;&1a4koAo8hks%xC#TC z`X1K66P0udiGVs(*aHM+Ja!K(1H?j1LM&1V=zwA&u~8V)z~BVf8Z+A7csL&tYOIW# ztc7=;nYX>ci;j#U@<5#3AgyZ=8dBPTQ@s33D9i~8LTQiZRl=0`4w>F4Ayo_`M99Pl zGcZjtFpo^-AaNvAMK&;>C=kX|c#`y5I5j#F>utZW%c{d?|FzUyRcEvB)pFf+u%nxv zf`TG+fXdwhV-<|h^~BTIg*B`l@H{}w$uPaEB{8W@`g=csA89# z_+|lnklX3$aJ*Sci{=~C&#i0AY_MA4pBSu|2Z@2i{fMeR^(lx>Cyu(7Z&kt`sOwbKA;BEM4Tm%`4gt42;;A6STo zQ3K#lH_VF2n4#Tz(H^7r2#k?w+sJ+XlAWOe_#+E8nvgWxkJ33)?;#SEb&miaT{(*p zUM@o^LOpO1553UPv=ct18aPgS-XA5XqVQtQ2a{}Q7v-fnQ3M?>dkiEJDd%}SUx{I^ z1#j@z14N3pvP))GK>|mO_*V`3RPK*52&?1f9XBaJ9UzmX6L*0S9~2t`qRj6$~l?(PLn{xL0Cx+;vNPi@e|u5Y_jc zTU}jWwjeU}qz}k~!l8!pB3gRCYjxS175i613&v0zGj*YNXk_d_!phYW8G3i;Wq8>| z$2;@wn~$1{jEFp=S(-8`McM+4Q2JFmt4+I|uO-~fET#>rFfb7SGh@DsrGp&(Mkh}V z=e{1uk1vT*^x`^qwc$yp-t=JXN=zxq5FLt0$%gO(beP17m|+oe=e*=Eo*7;mlZWDQ zN9WT9ie)C@X#5DIjKw&|;)Fjw(|8#4O_nPn} z+x^Y=o&3ls{9)Ul$*yet9dYUy~Nc+!zNr}M+7r`7T;nFF*9ufgkOL={~DukIY zv!BH=3gwbMZy~|T?0OrK2jCgmUhDgRY#2+Sw}*?_4x{`TGGYC=`FP<9iy;Sfe-s!B ztPusz6K*hx?ZBLjt=BWR|19=VO~ycm^YxmsWja(1(D~3>J5e7&zn!2jMulv(h zn4}Pq7f92^2!9C6X~j!stXM+6Jg*W10)-V&L~t9^l43NWDU}rAcHYelZ;^w1ey<+Y z;d7B)L_-uN6G{Eby{VTJMdLrL@!%VB;=Zom$DLC3vEG z4Fl1RqV9(>G0&uBUOWQ8E5Et~`R@Lcz9qoPZ41`}%Xgoc_#0l6G9jrRD2Ph@_+eeH z=FGSAjBWR%7>J!!No{zj3;^oUncd57VO)NR$$G)r3g^e!2IEtj|I+U2dc7)hls>1n870y!RlGhFq1eG%TK5;$J*CWXaUG7P#a}%Ukn)e za+C9^l|0q(Od{b$l1r2j53V9nLqO2XI3d&^6qea~lORS}MW&=gc?NMtNygeh7Whb1 zO6?L|DF7V=t}JoC*Uus&Zo+zHa4~pnIaG;EtW>_>2xg#Ms+<7|$~X(!Y`w6qh{Zh* zT)AR|mV*S{A7Ap!VN^y0DB>jCah7Q?QDX=ecv2-4)n3T|bB1AdVmC8x<*Xz+b67g3 z6)DlFOg;GzEaCLEU2M=%^kK9D!YY84p*cT1u4pvAl+5^^Ygv*b)XR3p$heZg`(!7B zWvW{{^EE&<@#YB)Ca4)zRuyv;Qm=Wn)z{znd!L1av`Vc$3S7PKCdLVfv-Q=XHUyn+ z`-dkxNuz8wT3d`(_Lryz_}9)F2jf=DDki~zP$lD8TmbtOR&O^;YGd!VHU;9RA`s7Z zd#h95@0gnrnaxrKVt%Ln)hC+V)n5-w2QrA{NBw`&(gb+DPCZy5cQQaE*L<~xZ>I%X zFkUC~Rn`mDRUlj|hy(CwY%z3lBAdrEleH7|z^)0IfRpcLj7?Zjqw5ibR4|T2V)IT} zso)f7G}P0Ae;tdY2%r4%5uL$hyROpVdUMKc+Ig{Ye$3~&>)`!Tweh;PF*~RU znT@O@*?jz9R1*BC(*93*82$p@ZMnNC^2r^muLp9p(!;9R~`8ysq1i1sri_i&rl8k|%f8 zT!}jE9iRxhWNt9dZ+bneylnNkUc>KM@cl`Dd2nodZxjP^O>({BuF$#3cIAFVR`A{b zWALJF8lpB0lmU7?8fM2@CBBp0pZM59@z->Jf0WeMy+?=&!{~&%#%H?`rGo-hs{bdE z9D5OKscYJsjhU>IGi&htmRtgGDs1(R6gf52bgjL(h#3~u-*@*WCkX}JD1)Vn z1fOn+jiAd>PYo{4O7^qpp?=0MtgL{RyJbr0xJ zacxdPS1PWi>D`%B&7Mo{6V@6iTEwqPVQ_8^d0j0n%E^TT^*QUh&U@Fz`CI)l=lP8% z^UV6Kham3YtAbwStdYt2PWQX%P~|aVVmzG>_Zmk1&W1Cu8hYfryN1>FFa0+!FcKho za!eL->AzrMZ4 zFSzy)2$PYVAA^60hEO7|e7gz@`^+dKo^kPUUsLwV@jL0-puS9BEZ-Q9ir z-NcJ+b_aR}EEuX#NFK1bcmO)Ic0QF;g-X+!4Vh?-Ajg9qo%u5Z9es?m_Kqu{(Ij27 zo4y|`e5uML51t0|ip!mMBT9GGMup>}b&gD-{ zS+Hj!Z&OgAn5ftI6_vDlZM{NMMv5c>Nr*(tE)k?60inMAW+~%LG2Ulf&c-)3dgDFP z52o{4-!#D^P?2M!@tLt>`%Ci^#2B$5izl}8VqykK-c0m@*}1#SEpWX)&gpaNc%SyRsGiwogK7UvR5q2nLIa_7{y9qC zhJ5!lnkm|#avfpqd}7r2V^2jT*lBa6gX;z$zp_#Y#0&wTWpIptN@N}gS&8R*IGYr7 zuu_=e8g79Cm1 z;hC|7#hA4h@dGc(lUncZ%ci_l`ZCNGjd^x<7QXJU$aXZ909r6Zz=#qfP67_k0P1LK+OmtIvmNb+q4m#9A@0lCthH~U%o zjM_S?El17%FKwWNI5Ce80z~mqvdurGuD+|c61qx46tK9;ZybgwzVi8*plJKO2OI)~ ztQAL0YiMkI%9srSEH>|huw$w&KYS#Muk<6~ScTtz6@szrN$_{UfM!T0CjrUe*zW*v z6{pVN@jrLYURt$GR5FSjz_eMt)LbR`<+i7Sa3#G3#wIHwB1Y~hdwz7wS5c(bqpDc_ z{RL8A_nq(1zeJ#~c26Al%S}-#&~3UU1mcF`ns2?akUU8`o5%O_pGfLqIR&ie>I|=3 zlZt_Ij-kbVDRb?%kp16iWSZMooLg0it&*iDD!=89-G{8}6t)Q0Lb zIL+-0XZ^g6{}lOY3TtJgW9B?l^obS<>@(|&bMfLqjI%J-J|3e`u@*aHvksNvV}<=h z5bf;^1&(0ww_Hnbu<#|z6z@-d?wFm)C?O{^EAovB+Xz)PdaF!q#E-*C3?dUt#rkPs zYxgcHYdk&?RkfF;nA^h2$_nv_B*f3xpm6hV8D=Li>E>-%UnrEf{s3{Tj|X6(W!`Hd~Vzw)31+O5g}6lvyBI{d*SCI$8em(13_;9oCLM@?ii(_ zoQlwSRY)ppxAj8h`ROqj{Op)Zb-6EsWX_`I`GZbPn-6hB0JTmZFD)IA$GNd(^(cC8 z2!mkMcA@b^{|80XK8mH&`ATx__^i{2mXslw*KUHT?^|=Et5rjCQV6k8x*Fcd(k8xX zQio!?fiQVL1WdZxbY$&!oX@X6fkuoOcBi4q%DIL<4)wC!-8j=*E8D z?iYz1Cy3i)cqE4qX2~Y8H&EAW{W8FG4Up_N9qpGS3`{l-m zB~TM>)4CONdIzbM8XPupx6+82msyu5Jd(H@<#0kmMVf$rYXRR5&2d)iqF59{VOsS( z8l6-!pR!WCSveBR>)6@(L`A>UNQ{_W@w zAcJPSJf8H=f3k{DP?M07G6j(mVtg9I;3!exDD^>2h>~k8>_n`alE7e7>eYnoDZ(WL zn3|#KFVRZ%2-`r&A4nwMC}&#uvC8)k7U4nXtmMxzhXg~rt>4l>^UHf!j^LJXrzHZ> zMa6^?DqU3<;Um#~bssZ#0$Dr4q!5f$ifulnDGP~3M5skSBUN06kRzB$%~DThS94_4 zM%nVZVW!=>fPntLy1@ukEP%~zH`#Hq z7r>==ucuFs<+F87_5O&uRL8o=gP-eP*+8gK8<5 z2#_2$s-;$OlMH_MPa6Jo<^98Kxv?fZ%>Z?#AG$J9?(VD4j~(vlpm8z@Oa~@adUO`x!h8&pw@WE?%(gi!_aWqR%OJXdmFFu zGi>Vu4Vx=Tjq;L~ea6&K5fIvq5M)+}rRkJ9-!nlfDybELm7JBJd_|WSHTFJU^D%PK zihX;LWUiKMfQI%n&=iY|njGC4O+o_qF5!+F;h?~PmTd7to)A)P@%v?s|C@}x{zlJ3 zTv@LwVg&6Tbk0{|s7jFWQqk0LYKDa}BGy(ZOB|LJ2gFmwsTE`z+%ZW)6DKF5p2k^( zkwPi|4D$WyfD|yr-ZPQWw&~vG`L@&iH%f3{>hmV;0wkCsn}ly_G%Nfz#5O4pe5C=I z(HvYQeK8#jX-Q$_wyGNKVR-$7PAb(-4}vf%j>zQ{+$8P<6_2G&Zu^^r?pqAh`1KOo zF~K;%&;(4z5VM$;jxF4sV;WPn@e3t9d4x7Ax=%$n{#{HI%7_D@kQxTmBQ02WGDOg1 zHV*qeJG?^}0%I*rr||DG=6(Sq8J3}4t9bbLWFTrqrEXhO@!5Ry*?-^#pHho-OUE=n zp?0G^ppqde((S`1B&5|j!+20tK)0fzD+}T)N*U`L0qm#8F#*q1+5=h>wZd(0@~+n> zhL%(@6Yz9R&!-Zi70!ul-5(>1>p9iSeI%Tmjn8^V^Ho@L+PRskDzh_dp0AHnVXZJE zsF?9vBjZ{0&9&Gs;Fe$Q4QwEYZ#gCH@Wp?vn1OJVs z9TMFRFAizPivEP45m4#C*2_r2QCQJ^B|-5MSECC-tlIffi{INbH`M^zW5=2;NHoCG z!Y0t^_jmjC_LdT(nIGF0Mjn@&;gX*@P{3XfTaiylUchn2&X11-Tn{@4Vwn7jip)T% zgA?@>Y#dr~yR}wOpJ7^DmAB7S;vQ#GyVJXet?gNQ4DHyCqOIqChPe!aaYk+aNp3E` z+v#EimP13G*9)kQKgPfioXg|)I5hA*Hvx#M9o27KyWsy$5%%`rYac`P_Cs(~t&n`s z9fb~TnpBCFagw+bV*EHb2Q@t9J%X-+0O1RRSY$*<`mpxhAqt45)ae&9vGCr!lOVT8 zMPXlQL^lTMYeOP>cOrc7YK6J8ZFXC+ik@0sP%4Ha)Ej}m2+NC`t)?ebiBKB2BbSho+NjV%^e_-AfXPV-;Y1N`bRHkb z9w-_Z^7Y#J{QpdmYL_jmp0 zXIM<-EZxUFrl;<)13~@`j(DBsdsxAzVMU+Xw9Z!3AvN2LPS4R2&>2BlJqLoK+rPA! zM+HAtx}}Vs>IOvn-ogbun5}*pX+J6I`5Zlbygj`28Lu!joi1uQU)>ICGu_p1PvztY zkQM+TJA4l&en?sL%vWXUKOZT+uLyovd-pQxHaX>W9rU>?D%srUbSPfjF}Avu)E(zD3Gudk?N$5ROI1R%!`+v}9{PzqC&J_^A$ScbM%B4PEA=Y;etg zLiq$t>=oiRYA50U&@p6uCh>o4c437aJFMIFIQ+xn%le>TYP=mZl+CwsC<_d!aBi@e7yoc~-X*`z6V5bpqZ~1t z<`;vznF#qRsBlg4aYb_T?aQLt*>b(*Ov&ov{n-i^d#Vybf3{AF*C$0J=%7#eKw-vw z)SyGLkSQN1K)jQ~Ktxa;F#x#`G5AoPR?Qh(d?{~WwVxxs1T{3(1k)r3on;Ao{6D-3 z1oczjYPNFS_-hRmrkIX){I&$((`r;Uxog;@F@giE{wa%kj zYa9h)UO#qZX1&B!TmJ8w%EE?oC5;-BM7+NHMKx0=wr`=%*y=V6_BO5gp;y<}F=-Ro zkEgR+lvte2>-5?D)~crCjV118;>IhZBrnJH7=v*{lce}eL@P|(+B!nRYugPXICtdA*T!+4D+A@$(XoKKIaG~i8+FB(pRV3gUxFG zes9?YLA^|hRlM5Xha<(4Obz~&Pq}{0ZhLhD8Z~;6Rt2S9T~B7gDChVxZ}(g^%sTD8 ziXfdt|J&my3L1qRH7b=y%&8%pu=~4)jbCoA$p?2RRPQ$@f_e=mjw|EfLSe98-!cip zEbSUR?qzwobY8Z7CL=%!7~V!}QR1T-I-OJ1Rxe)m4|nUF#~bH1I?ZNN1;wMcHxfY} ziAN_FcAZOQGD@O)jKU3gE_|pE{6A#B_#4706I5=`&5>RMUH5mocMd0a12AGj(Isx{$!B&j)v=d`lh&j=n?2&$Ry&n|~V2rEo8E=CC|+ z2v2=d7GbMKV}CgGdX6uE#bhXA5y)Wori!PU236o1-FpS?0+}R*zG0+?<*gB__7GQVelB(8gre9E*(YE8qT?fx@r|pPufabR6co zU9>R}M?45kh%=!NCO;r=W>F~&FV=sFhk&k(aw#N{OziFBi}X(NaTyLmLj^dFj$Poe z>-i`+0nSJ=vq%r1?@?pVz;9ys?*1(v3_fI3J{MqWvM8<`fQQf^7fVRh2t zh6<#&_Asr~q>W%-5$OI#aGOW3)JSecju$na#rtr;EZ|=M{+l3Npov`2%P+2*^w6%L zAJ@al-NgkH3;V)z{|S=+gO!z?a)gh=3UpEVl>X9D_%{dP2x~S*3Fl)e%vPMkA2A>Z znX~+B9J4q{+>iXRW>FFIxjg+jAx|xJd))U$dLW*N7*FO=^Op@(P=V>CmTJ-a=H{k_ zN*bd-rVZaB?h8XI(pza%Q1x8PeEjsAaAEk}!uCA=p69HQeNH?Y6C2yl@hOaX1i$E5 z9#vAU+~fXrkHMdGy)q^%M~BD$zSH+b3Z26Eel;Zw2+YoiwzLF(_fkX6bv=>^R`{w; z$7r04Pk{FhLjSA)bML99mwgIJjh@3)kvsg990`T5i4Nox>IA6SaFbEHt-@hoRPmW= z)_wP;(~e7{5DO9&pi_oQ8*su_t@!#{n$^NorPcP}pk)@7Xk z0AXMSdOdURBvU`XoWpGYTWBW!CKCK(^@5w>q|LG5*9Q<4zT#t4Lk~ilRGmyo;yR6Y zJ_rgZ7x3I!OfjgSsb2vEen&+7*5;ijz)rEfki(sZSpzDQf!v7G^J9;LGp-YYOh)lM zr+Yc3Ckha7#yutZ&aIKe=fuER6p}1Q`Q!5H8-or2psR`7HRnuVMK=G<`+2s7r0daW zAa(qr<6(Vxx!zd1e7QZp)v7*MVRmj#w`CcuLing;mBXY=xa3TIX5O{J@9AWtB&CgK z`Tk;KV|5WON_#P>(Hwo)Wo{pN4xs(!QgHqpGpf(|4J#lY%9Q-h08bq*^{~IZp2E^* zZ3jn?T-@A<`Te%Tz1=q|yOtIg`L>XYY>wza z4Oc|U8X8=x!(jg!g{wESI(p2!yf#PbVxqQ(_3$ryaTI)Lc^FkjH?A-vX2Q_SN}ENt zYRL^Y>p>fD_lMEl$)l$h{;gXC*nl>V#Bd{z4F|JqUpJJ^zH{`kgWjiiJV ze{k#J$k%1gi{Y63DU1`q|2`lQ1`>i9+K;k5GC5ZFPNNddzZHdmh|z#Wyd?$Ye*lf7 z@m!6~QjB%QM%vUFR%eHBUOP6>4drvO>@^2A21#HnKf$0F4cjpg4}Nrn(jqaUZ(m_$ zL%_*b04mAI$qDp)#xaa=!DeCeE`#LQz#y2v+Mm-F1}3I>zo>=A&xOH}k}FfOx`|+2 zZvJ{bEHzN$mbME*;GB^MwFv&}5rpqdMqEs5fO1E+xOBm<2kSqu#ICVM1}2tT_UC)r%&UB1&hyv!d+z4Yc&G3fb^|@%iQqWK@l~k z+x0>;j-?hGq%F%cIp6qr+epI_kKTu&1>R>vzSQWnY*WzpCw-mc*AJW~2+m zqA0<7wSyd>3`e{#Dym28AAG)gUW_}jP=YI0>4@o=?Tzz&9v`oFI%_lxSu88ug`t$a zq$|>45ZdlfW75Vh9wSt@@bY0ykn+IPE#gP~DdK+N+N1)dCaeF|sZuF|8LJnr4)r^E zG$|q)xmW}$nMep68hL+ke{dY>%^fc%HN(Nejxz?Ll^&dd$2~K&H>UXf(Y~JtvY^rT zyicZep}fR|U?V*ZMo0k`>fM;aOO%q&ZTn*JOXsxYGT)D)>qd*dH@XGtnKO)kTrHUe zyRS0r28)C|Vu%jGhJv2-&m*^mw*%`%QmuwV1xz7J1D3$`=h5MoNXzp^&56m(NYQ=XW=p#h@sQ&qif4=LgG&X#TA$dn#Dp-QA0H0lCzTMR& zmFapovt4S)-1uDn%Wj3oZLEz-_3wd9B23gL>(l#p*1j|@rc3n?y+LzmKR#HD86U!Go-1yx6H#@yihyVza`GRj*+}s0ZG+GaPQV+Xp zJ10gFQqjolH`!s7X=11ASUfUdxS7_@)q9yc>HCdV`2zcMJh|hAUiqw$6Y}nO8A?=` z|KCGjBMq*&-}{V0)j&Ni7a~j4`zip300l!8Kv4VLGlBE{bHT2NCxInFh1oEbickch zF723&-)Z{WOt+(y^h~Ts&qD^Dt)YcqU`RT+BpcLI!ALF&cwjQ1|5^m?e#uH(%tFRW~ze(>61iNR!urPlbU05+m8%Sj3xqwfXf>iz{R&>*Z zmc@&Nd0XAIVW8LPsp%&v9jRO5`*Ols-XD6Cre@i>JamqGn{U?LjYZRRb0xXGzIdnj z*__=k4&}VbQ#eKGr}Wa50r+>-pVkNWzqqcC=47Q~0zLdzG`TfCVTd^`SF|W`h?z7h zJY6Y6EE;-UN5V|X<_fs#0Ybkjh9q!oeM-hS6jzXLAhqRkFHS^HLXaXPPkHT^mU8m_ zlHQmGwvFHM($_wE|L!<=&%8e2o$b1j<#pZ@ z1OOod+`7CYJD5g#7mf}B{SUMZSb{R}4#?tzudDwbTW1y5R@Zg=;9j)2yK8Z`;x55m zTHGCi6m4-U-Xg_=ySux)LveRIdCvDe?|UxJO}OC~HhX7ft~KWv|IsL!fmiWL(Ixs- zj(vnM7RTx=y|bxpvsk2%P0GS%8C_QA38yh&)fl z0<;&t9t8#TMgOs*Kfxf5*d0&OV%yr4!CPTCNG?!1Yd6=FZ{@1*evibCB2$09OasH~ zXvD7)F>3$NIy9YC=!X)EUKWpCt`FoE;Dp5ztou;!&G9-L9EzY{ZDPLqWj7pj%?p`{ zcf7Cp_A^gzKDKUID+=77<9L!jwrmH~Jos-3b(>WJpYS@-gkfK*5_>(}J&PE-pnkDS z?bg8S)=O)>k|E@9kJo@}@Y8&CKbWoaK=%{xQ!2ziSS!bHG;R2t*O!$HH#>FU_TVkR zQ>u%)@OApZHH?DOT$%S%izTEiOd=xA1?oKwvllidbd(#3S;I%RT>ky>5}5jWP`kOh zEiy7Bg0taF^s~-(VSD|;M3;i-*X@<@O96cMFT|L4<>cOggyG|gj_mK#>fGdaS`sw} zQA@>n0I|+r0>)k`$(1F!f-N5`g@a#6_8ds3C3w#9F^RP)^ef3o;Z~juoJDFl1yPk4 zHm*!EFfVY_7l*RhzkM-fC10dMLd?ZYH@rR(b^V6YQ;4LgWl8G&$+JQiG-{s=Iw>=s zmPoY5Bp@Nl`uVdmH`&hpM`%pU`0-8-lT{WAih6Dr_xG%WCBgCTytd6tO3Dm8r>Zj0 zy{{YY2+>b-we#N*o$F0ZtiHu8@M*~n>=8{1eP)ALqfJ=5zx!`39N8*6$h@m(kyIfK ziOv}5@zc=-ek(1nOnovib^YbecNVgI;z|%oCJahBh1EL(jT5t#{nyaEVJYUHr8^md^`6H6|7Xz0 zHsSLJt}?F2(aG0LaH@ub?7ChJ`peyQ>*x~gJzsXuY6pzhg&xH}o<$1v;RGEM48&eQ zP+G8EOjJSw43-RcA)SjRj@@9;nlueYVP`+`_g@}9LyUJM#oZr`@ukgyZ`cSR!jPa|unn$8hS*^e zCxywe3qWK)$;3T?oR%?Or)OFZhX6FpG=M3gBFclelyJ?g>)>=EBS>Jt@T`3(WFRx| zFr!x71@{5hyJ^ol2o#N=Q!L1Xq=?bEd@&PzI+~njXk@}366Av%Mo4KEVF41K<1MH$ zo1W;K{CanP^`X#B)hz9qrmbhrV!{cov|A1Ck)`AM&eg_`kF)K63qGWVqNCm42J%|J z-J0lqLO~z0f7)30n=!u|yagb90K~qfxsO+e3&aM68^y_@F3O&Pd?G8hY5mrO;6dd-{TwPtg^_I;gv(e%I z?15Z{Q&m7xw9)1!r#sBm1M^xKLwtB*SJ(-gyw?5n^c1!xIIMsIkI;+`x)9fwY3Qb* z;LD`Rc=;L>Sjyw@O7_0zd0%-+40d|k9b_M5 z`+{rPQ_|8Jy$?^O@?;8y;;`)(H!5xKcSc6!84~#ec6%VM&$to{qNv{7srN0mj(+wI zzte_Di{iQVPW{;9a)u;I!-F=;|C!$zHXtv*;rWYW^Is}(N({1CPG3qD(G1~hKNZ2 zzFb_yr~K1nX8QIJd%SgJ2@Rt&$$Rj@Gn|yJ(cMEU1MxBL%(eoeEt zyi6{Y_9YqU((4S;0O;>R-}%yPLLdIMyA~U&M|(6mKmrI`eyJ2j3y7T^mZFN zXX}oM_*FSOzx_~w zE~`~uBHJ5C90Y>}goA)G>?B}D{AcV0>;eKt1#2xlO84((ZPeu6HTmZM^BEVFSsdsP ze5f7wwNCr~VY|%tYHfpReWB$ge47at(fM}$6^YJfaHy}9-v*h+_3>utg{!){2&?It zknn>@)A@8-B1GbIIUQ92x?PH@j#9P}t!xN5$9`rb7F}OhdH!o23(0ER@PEF|*|2d? z$SX44XtyS`Rn8JxpAl?)t&%Z*{8GCz{P=yr#(?{!J z1qC1O9&H8~AVW3ieSJAym`?P?#I4fNu_rs#$mehl^mv|!jZG>&CIKTa1!*7>6ARuB zQpNhc96ATsyj-|hPuYyr>7Aa6@I z7pwR6W*CHfkl&FeP)5U9NPv$ITI!b&C+F4F;SLf@QR&L{siXgOPQX>gjGVR7T-fDQ zm*~qGpP8jGWW-h>^xAs}8FXawwVC!T7v6E-+`Ah$%^_`PueWc!+?|fz% z-`1C6>-MWr4V6q0=c~oYnGBvX{}Sg4W5LFj!u!((P--UWMzzP@waRQv^pEVfMh3hS zDspur{W>2rzugee06cfE^Yx>DlJnkQjRQD&L`>-gKt+8*gUP4RaFG>evJPpZaBiX0S=@CZ%0LBl4qy+6ZI#|y0X0lR8>E~ih?6I8iXKl6l| zxxTFqYmGp1QZFHU)D|00I$_t#JcVXHcW>*tWVDixECx792` zvYE80{gnnGGgmVI>V+6Ad?=;A{>di%;AEy$?Q$d6(<|)_9JQu*Y2gX>|!nZ#`_ zm%-r?%x=t5X^5-@p+NyDt^pzvl*0cWkhVu+pK}V?F@t08QIgs3p?t+rHv9hOhhM6z zLF~k|Yr$U|oQ7G_qi__ZP-rQoXB@Ur2c}P~48!rAl2NuVyZyQK9Sww>PyWXLsgKb= z6_$&{??pD3j-Jy88}oTQjSj8Y8+N>HYAT$|N*b0c7`3t+X=u3abr*l96ScGb>VMEC zY2OAj=bNKSr;QPNUa@p{H~{e|3(u~ud)Jf;nwB-x0w{Y)`;d_pY&`5mJ+CI`6v!Gp z?^}Hza55jl>Po3-zfx5)qkbZCBhO-KU^V8OnVx-l7WH##DTN5!1NcHiLuq_HM6j4; zG9Q}jlEJhFqlz&%bq)1?J}tKm;p1LYSINo83TE*z-1;K?pqq!tJQjOlpkpmsG#w6X z!q(f_3a{g8>)PGhR-!VsUXxWFmgd;*UlXAIHC@AsN>+zrT5mKH-m;#aN~VC|ClSLY z2be`D3k$EJx`L*{hU3-HYr{5SCSZlyB7MzhuakCN> zz}Clpee6UoWL}w1qbbxF&re@H%&o9#agxt2R4ol z&1(xYu5T}o;OsN4QYKSN?Q|MiS>iFpA9oG)JRrzjIKCgZ$StsC1U9ErdIbi4%G}t6 z0A@lR3G&D#!GQ$XR171}uoSRx#XE$71uaV(OWxI>F6juapz`G3SeVW%eLL+R` zKkQkZwShdb=8wzck=96XP)k`(-|mLz2Z#fBUiu_9P_VfpF)-PDuI`qoWWIXn#LMkE zeU6EQa>Y`co-NJVf)oGFxjd}2r1qrzq#I2a*hzlMZ{ZHuA?p?#Xl7l zhbUJ+YI0Gb(^Ai$+7f{;#;*;zpx-v!zH;h0huD-cIA@DmHCB5r!0CLf#Z;&dcV=f! zOVqJWI3|dt14o#>LnGr}ccK9DX)zStEp|coI_~2I?I@q1?biS#dJ6jTq{^~?S6Yl$s@ytW3UZFG3BF=hQNd}7Q^~lI3J&TTlRmK6=iQvZzi4fV#CR% zZ<_I#z!*JxTHdeefr#yAj16FW}hIVs4 z^_fTgnmV_Hv!nxIOeX^*5HYevn1lsm2h+0m^e-u2WqE5Qc~o_EaBy&Tj7JJ;l~?ZR zn*R#f#>el;Ny#6Rde_G5Gbpp2X5)^7F!8z6SPj*U^l%=RuMv#B-gL!hJ0mL@7#YHP zCRX3Vp9^X{H>XZxWz0h@X$o9NC~!WeVy$!{=h=K)ts-Finh6^u2fJoM+*YBXx zUAvW94~P5p9`n3wbbs^w!Bz}vc6H`ldp2R%c zZA^{gW01KmTZF);0s^W;4J2Npq;mh)x7X-@-o^qsb9DE{9zZ_yXWU%fbmNB{&Y2Ep z;wNQsW*hr_=m0$mtFD+IYMdu08!h*x&z`TlId3bSUJ_f2UzmYh9z?o1 zb-;k>l{J=tlRPY|((vjx4RXtmG-`2b)ttzed1gPbyo2MX)D(YM9@I)zrv18VNY}NL zC?8XDXxj*VX}Iv8ou#_fbhon`{ShmUN{?GhoNnmSj)F2PE04^(-Ttus9gXLsq0i$?Tg^4HoOZ3x|Ef zYyV7(7sqYw->IlnJ*C~n0vC9Tf+BB%jDoUIrt7-bi_otzj!nPYnbBrB|6?>ZV7S6M z1JwB@j{2*Tz1?aoy>ws$6O&y(<9DWac3w72B9-o@>#wElYG)8<>A6mm-(e8U^MZDH zjC5x4<<=kl2Ib85xXWI~K*)HZ9&Nr;)<5;@;_)E4x{gj|y~|$O%)p$NgoBAiF9l~i z3Rb)is#;FzFh=}?Xk#`;k9&^q77@%CSQ+bkAqu3~&K*OLu`wrq;&b9)`Zn{g-W+oWlEnZhn?pzs5Cs-< z@x+Asnn(&+k~|zL0>8yqIW1>RHm+{AU0;j@i6aq3tZyVuoPC0j1YR7q^=2#YoVV_V z;A()Hn>#!IA$a?pzs=mL{tSoFCLtW1>}o^}LCnQQK*?Fq)dT?pKUvqIYpHnZq#dzL ztqBtbZp=riti%QtlY^_Os00&AMktC!zxielFUQu_=BYvQoG8AcK~hR9^5 z==Sao7Z+FL^#sN7!`ty}?>7n_T~i6Cn0V{n2mro$XH3qy!E8hk_9cCLnc1PIv0r<^ z;eN#j63%|V9a|Lwmn6IIc-poRu+_RDk(o{oTx760%v6*`g6!-#&R$E?(?OSCp=@ja zu53Jgwrbpah(}M*l-&jEH8Ce=6OfVNRZ=zWv4^&6Q9;rIvA3W)E88xHaUeTc8V(eE60z+n9CVr2obuqZ}S{tf~#uGKEqRg zm@doSYgeLuymJH@(qpUgoa{4)+m#AQq47i0Y7cn9#sc7KBsTFq)iwbAFa&0T zX`Zv#W+6XS!HVdC2n%s}IBHs9*2kQxs!~HweSBX%~CTktDN-o|y(;S;n;a|`AEn$nJAga2R;dJyhy>X4mMQRz#}oKXN3x zM6>)zE&S)M5Eb-+u^VhO;bljYl@=%B7aO&mP;yeCC%WOokJv6C&2k|Zc%G?4C%lZ=Z9J~lQr=9)B^a22(_PJM-G>e{M( zU9e}&0#A$cUCD?vOgjP!s1)RNRIpmMn~j6X;hg9!O#;oZ?z6+)X_;JHT>LFuON+>% z_9ih3CAw%Ji693*OF3=0WQ^AVoxK&`_qbF$<_8*GLfXxPlt8>e4TJ%xk%(pR=hjwR z=?7{-qDCUK9D3=zUGvoHME)e|cLXdbSk^)3Ph4F53GzM#draZsqL>)P^iUGyyaz*A zlLnDk-xievslYYAOI++4MH7`3m24Rc3id|q`2oLBqxryjx`a&Z%vLToezYv53-(X7 zO|jC5*)UhtAJh@!ASEV!%fuz~qyZT*vzlfPf{&an#)j$P8WrN=h8souGe}z)o@eFt zXRD4%S06?%x)SGDG2;1Cf1 z6%2T2OPS}1%^QwyK3a=LbN{%fph(LhFnPNbPPXoF95fatbMl!A;f{NL?j))Yr5T}r zcqHmG01&hDa*O^ppHe~mb@g8Abi8~d6{qVu#?R}eR|dEkd0J6gyrj{VpYP`M>)n1Mx~J{Nakyd0l@hJSGL5rd9-8 z3vQed;d?!w-b-H%Y*UcFG_BU=6k&acci3nJceMDMxkMQP7nfT9QChDco5gSaZpWrm zu5EC~j@*&*)H(;3uD9hkI(@0UKS|So#G3(#c$+~R*e&&7CKd+OZE$w>{0fVHgS~#e zOXyHO*jNN~ck-A7ARlnPo7FyFZ#S2d>l$==zswsD5NIJ_c8a*E8Y6mYS)rpVBp%SJ zMkcu*qPr7q|GLK(a9I{fQ*zq)%7arSo@d8{!Jg4P1vsjU*MPSY{=;nefDhheS|NdU7K@Z?y zABu%yU{=@;_ISP5$mFc;K z%hVK|LpSOq;DO|7w`X^v)E*5|5Xlikbrl1A_Unm>Dj{u?Q>1G=fFSKlN+LqR@e0j^i8yB znR1zL4<0*-6SN7dD>Ju$%>{Y1SRYS`&Mm1Eer7j$T~2d76VfFXpJo3(jkRA>ZQ35= z-JniOOXo;+a657Ev+{v&gL#E;T$o}oOV#FvYQf;;N~^xmpts*B3l3-Yew>NU0?Q84 zhK%&|jefUbn4<6$yfCBDE?*y`>WFy1>$gJiU?m?Fj6kPK3B9{Xo^N`t<~uWAu}{CW z5m@5jCKL3;*bvO)afOoK zR^v%HOf9f95+!$L6c+4$T{jT-Q4!6f-nwhyOb^!)g*)8KWPaJJrpd}W%~q3KZLHWD zf{C_PCO$ts+-Nml;5B+G^)!B7&>ypBxCm3ySKkS7Y`-=oJlo8tLk>ifCg#2CMo~%a zKLm=tyHg&f$woq4CV7;*WnjG=d?LQ9=R~90+~2>q1la;!b3j+}6K?sY=Wi7xi{4wK z#h!v;Bg69Tw=e5gG=1%)5+p8}Au2rc3-g(R&bc+7Cc1O;meF^C0AQs1A}7L1zK zR`;uFOlRh2dsXYfY1>GSK=;?`tAzG-s78mqba$$dwzNaJRO_#(r&V5?5)`7#jPKT} z{Q2auW|n=Ib;%zKW5_fQUuATN~Z?bAc)=r@_!S^O%mJDdX|pf;%i+n?ObO@BN;Jta|+ znpqY|+cGghl(&rX-Dg2% z$6hBRGwtE_wn+jvW<`~4P-wp5b!tahkkou{gbV+{~NHw z%l!*yqCyp3tMz!ycuQib=f8>wsfchG!@vGU4to+a#v3c*+;}%NgR@kfC`E*$pdf$r zm^O4HCaLXCzw-je5ZHNs2?FX5#4tU>6419|&|0+WLlGBS=&=MUM0clbtP%I9KjM5! zKL0A5$W)1H=>VTGsfcp$P2R6;w9d`0TG2~P)>tbi@Xo1jT&;h{IYr=lQ9jeNMII;J)MiH8RoKI-$D4Ac)=802;WP<%XY=(-4lM_7ODK~NE?>S%6`;VhmV{d+q90T776Uyr7X8wRN4 zglTJQlZheq-C8QlaNvfxvB!kN7|Z(Al(10*^S2Wa?M_L{%8+C7)+aYR1+Wy;7!O*rS}-RBe6GnZA~ZSW!z34q#2AwTUTkXFA065-Zo9ffW#f1I@>12|`}MS$ zThbnTuFe;1z~}t57*E|N-1}?FD+x(}i<_U1?+djJy`GMaI_^TtmJ@fP?ePvAo*WPl9x13m!)U-q{80?|e)^7RT4WPqR0@px27`Bq(f3 zsqtLTU4#g@{X1x$UfTsn6At0i@<1llPy#|ii0l8ZGcWSS_TUrEOY+ZfKYFldDGwIAiX^dA|PW$U(!xJvE+usIwD$2_AeA_Jf zjC5LU4`RL_C>wX&MN&ZI|1UD44;dwM$=Ev$-->M3d~j!0rl6$&88Yff0(^@(!$0?u zOIKu?fn^2-QV`y~G#uPeZG*>cRQ4G}_dpvzitlo4TRJk5+WLCDK=V9iQTU%m@*1nbu9 z>n+G~W-V2MZ7Mr>Nh8aQYD$iemZpAhp??59+;C02twAzW9o8BH9-oqfy3}mHO(YNy z5gne`@E*};dbJk$+{f6aj_2_ELQC-QOl7ALm1rk;kz@%=s5SihV(or=%X9#7&hRMh z!G(sZh&C~O!6^?=qz(C|RZXfdq3c+d);l{nsTzf%Zh+6E*9@_upjeAGJ`U9punI$f zMR^bGF*^VPr374vP-t^-fZ1(2tQ@h zn+{d)f!-TLGa(tmm0o07`~jDvRbqMth@GhD@DP`bu;(3Wbqv3lX4i*eUan*orBr79 z%X`Q^Tayya38;Vker^2rJr7#yQ{v0t(aZ1FA{MedBisiTh?*B)=eN~$HM~1 zgBzw-+=!FoG4}o89XSZ|@sa7>Sd+7a&d1Auw;B*tHJEV!DS2d} z!hp8Om=?-OntA~~PD0rHVz^Wk2H^hF=@0_@ATy>^>-=b@mC->!?k*dj_Ow*`PCSK5 ze?&g#^ZW#Qng9aSq@-JOUT=W)#FVeQ!*ePXMp{~8#P73iT1sG#L`Ni!*$mQ=n9o7o zeLYpSkJFUTxu7XDS-C#!T8~@%6*Udj>U{d#5&6u!{o|gQ;)C`lGt;f5k0U9)5%9jx z->3>K738-ajAar_2wkQnNDSLlM5={IJx!UjtThBV7OOsV#oBGC$wIb|Z%2P8$%{I9 zyKV{EPTYWK0GYU->B2KVYnk6I^HIN^) zS{#-R9Oq~Ixe7VO-MZh8Dk{NP=65rqS|WDt%oNwkRC$>~TpC}K;G6XTUSj=xS!-AbQ~Fw3?vKmm_CyOT#X1~Z-1V*00pQz|@|J2B zZ~va`89MZPw|fR@)DU9GY0%cu@%-kd_IE*Ffu*z>>B(1wQVJrzeYzD!1_oAo=71}v z36a}b4FU+k!a4WX7dcM;>8ZII)1GDpyoppcHkO!{2L1HVqC+bUSA_$J=Go(H^>RV9 zsQu+qrZ6@qd<^V{I$-_6N7RMb|K>pi)9-f|Z6cq7#R-pK=!T>UxkC)pvIf zk5WDpS{Fa}h0GN^BW>V^LN+?DWHRjE!Z@L`oTh(5qEKT)CFlG0{bWQ31ag+qpPQ zMD0y*{b0TrlkJ^L)Vg1%=rHqeagat!-+h`b)QALU2S|RHjpKeYwNB}ZK95TGC;p?0 z+`ZU15#BTsQ#>7k)Mu^hU3F#b>Vj5 zZ1v}y03+gR-G*p3MoKG70KEJlQ6Tmvc}Q8QEf1MS``8D?GktAKNTJ<|3+j$T$$`Y< zaMY6sYd!1~OQFtn^6kDxTuG)F?~sMsK|Gv0tC!HKSzDT*N+|AvVs+)z*ha%)A0qz)z5Nw8c;(7w@=9llm~Ot zo8HJuwkwK?fSSQ*74U-J@z7ON0aHl|UEx5O%mAu%lFNs8W5T$k6r3`>rjzciL|P~3 z_$Wu!!ps6)7{RW~cQR&%mQap<2NmRx=2D&DHbH!I$R2TdYP~?-gfe+{G(UJG>N~L9 zAYki)0S1>oal%eu5Z^g{+4p{So|Pd62rhcKxG2c&o-VmPrj;=>V&oQLr;lj547Sb@ zncP@%xjoX7j1gWfpn-!_6H(A^k{TTzwX{^N>O&pJe$Nh>M;*5}4$P*F6=i&IB+rJ; z^6Tn)wIxKxT|!85L`srT*}yz8vf4Ozb9k!HJtvqGU}g+-1*&)ocB!&^7CIK3=Wszs zbXRH^skd4^R$~u*K+%lYYy?PMRn^wVOC`}=MQF}t-+l9=eB?KBY#(VEjL(y4Uda%S zvUpCwOnS(utd^MbV%*(LPDWqbWBl(_?v(;R#{4mBGhI-0C^|ZtMYr0b%Fz`)x5qp$ zr^y?=$DR<1D5lkz)77M%(8*03`toQ_3zA1k6N-76V0dcZ5pcQ|d6<-tQP39rY~_u> zoL{3#>7flSE&+pCnSSicXJpEuNLx$a>kfQ)mPMy8s&qSy1-hOmfSPOBiKj!V~4tF)JmcLlW)7exTn5I9>f60bS?ey>0teyl-14C>KyC z?xdgo9&M2F7jXA;VNAT#uF#!Zb%Rl@XGnXA<8}Td%pt-PKRq110_Jt+XIC{9p)p+z@xYBR1(YDVm(k+_q{psQAQsI9=(BSl=U z*XH8pM;(8KaqF3;&w-?CM%Zohhps>2a5Ukn0LW^rfjPFXis!;>U!^w6i4j7g z4wad5xBk>0+}1)GIXrqX`9%R}JT@-F$uuG$D3~ZW!B^n!dz6CvdEM(-*j(x*->TVL zvcM*{t!6;=n-=ZB04=t9Q-H08ZEPN)-zf(_f3h4MWbAA}Mxdx}u*ErzY~|A2A|Qix zt6zVwhRknWzZYfii4ix6h=PspTbG7T5POM2)-6 zAk_*94GbWlVGOEJ>TOdisFZ98r76}&HR9>mlug`-%*O@{<&P|ucDv=jdR zVHxVDlxX#bw*O#Ogh_OPgb&Yf^vlaxTO&;M%+;gYL{>lJ!U@0N_~_`%O`&LX7fQ7R z2A8miM=XnO*);z~Sn#h27AT&oZ94nBrw#SHD&1fHy%z~HlE)$ zh=m?{5yE4e*~u}@DBvTz5vziOHPu{fo@P`;*A^!EvSjPf^LgwI^fA8IOsGe}B+4r0 zw0IpJ(&gavjIoelF|o2n#8~Z7bX~n#vO|y`nc0m#d($R1{)UP(@ z?vxG?f!SeUPV>d|P;c~~J4ixc){&#F19YR=V~KD}!~LzBk|K@yg`>2N@o(+AWQAqy+dxtz8Cz-@$(rC`Q z{wRQ=4oy6ky03;4SQSi8#Gg=(cOk5;nf!|x5}N=f&g(}WS>!0L&fxtP=Lcp@&%tM& zED>Wn`^|QbWckReeb!(>p>f#x5R4|E$mAcE{fOzpRCRaVnFbh ze+?&3oK@mBLz>(1X@+{ko`=SY)HHS!C#PiFX-~+jky>k;CtnXHjn>Z@YgYWO_2sa_ z97a;&zupH66>D~fgHD;`ALPOeD2SpdW32GudyyLo9WVZd(;4~IUwXK%jDelCwHlFA zP4VGzIB-I_Cn$l*iwao}7nR4SD{bdY&nP3|Pl9rA+cOgRq)8K*uk>l02uHTZgy{I@ zw4dR95o9=b*?+)T*gcliv;K|VkHhCB}PCai`XYm=%u+4!gDhtN^%skI*gJR&#Z$6?>>47&oUaQO(N zAS&U~l;jT3Wouf-!nAh+L#+6w}+n@zweL?<6Sq=1v0qR1C{4uo@g14BO_k^wbUN%3eZZCn9|M z!&ag^FSqLsvR~zI&$7sf1-fZx+;(P2O7D5O>ni_P(}q3!wkRkdFF%KKI`II@8R#Q` z3+F2uK*vxC$ZyXtP3hH@oj~4;{>Q`my31Cc!cY>>xTd8UynfwU99b3gn?rImLB#Ve zUrt%`e7z~}EYNFc9|G~%Xg1{xL&{litSYpUvDH|)aYK$MwP7}@ec#lOh6}af^oN8b zzU<$wcTo&PG7`fv#20;ho@~0tn;|G(YpZE*Usq7f=JR)$Cep3T-M*2LiRSC>Edk4A zi@5(#dW~s@sS8#34Bu=jRAib6>58}}Pn@jjYAm?_y`clLrr=(H)BW8MqPQZ9x3<=Y zBEKe#Hq{g|%4Y}M>2GikrMcpX937nrLM}TMzoHl%&#rLKoo(vW0ms#q8@8<4P44 zTB{f1R4r3l+ZqN2%73ZyA`!Tl<+@+12#;oaIOG|CU8^W{7F&V~orqo3_1RouD@mhx zGw5X0(9j5_DQM#UUL~PjOtov>N}F6ni%1!POdOaDu{k)G@!~FG=uXhe2>+u95yRl# zg_UQQz*I5ZiZFyBSCSTUXaa}@q){_66fq))b|ztA%|S~l>P{pjD|`pNG_bv%d3Ln; zw2VmhE|(*|{G$HHwJ<^GTfp!+ir<}|F)TL>Uw#Qyb({R3YW?v+ic1k04+Hpm8@BqX zFKKZOwa5Gx2Q0$@#dSqx@`>_i)gt^(YO})T2|giTKGG zC}tQ~qc{eMXs$-ZftaP{r`9Yd@L=eAs@i->MH_8jCyf0qnWahlPET{TpeUWB<+2cG zrD&wZ`pKZGs5(_)2J(u2BNq3@YtDVj9mYw<5f_SrM)lIp%I!6SNrMO?`7UrAg5`;9 zO3}~sp09x*&J;?8My5rG5~C;|kk1cK&RD!UrFjM5L(YqdD}@E3^}v{H5=L>KEP0#U z_d~ik$yGQ6yCblWh%2Px-({&CGT^ucNyxaJ93UqpuG!tdVvR2oI%g&AhPG;;@bgm$ z{-73jAlA37D#AcmX9{*9ilAz!4w&ZZ-%#ih!I`sV#Q;2TZa&r5nU;(>^ame0E0 zy53u{?f235>%QlS_2QvBHpqe9+`EWfSWYoMuz6DO&M*kns5|L6fW3$y+vjZ^OQs*4 zRjkXHN>JNs22coJgXvex0tZvc!Xf_6|1Ar>InevhL%V5tkhJBD&9xlt$Hv| z{j|EIVoH!H;moWroZ76!jbpFj-4IrjfW!TXP1;rtUYW%n+P6^vaHx6 zLQ_yel{SavL6#V6l$nhSC1E7@Jg_hUGQD!Jc(l{c14SpI6}y*DM4?S*YNT3QKUJzz zZ83*rzYce>bT53|U*Lje1i+$7*)ODhiz9=wgrA*qt7lyp=4aAocBL|CarLdrX!kjt zLCuw)m&$?~7a*%4ISPj+`fCH-Km`>6R6w;R+MQxZkucl(-;#_^oTn9@j7>C(%?Fo{QB zh0y7tJ*Ps_i41|DGOMJc%M2J)+5JBrTt9w_@60eYx^VEf_jeQuH?Shttr!5Uy+GSI zM{Xf~^`wbkb#cyPl4hY-$n>fK!^h7j4>`4Q3!^Y&V3+U&ycfpNYpm_qh|a@-_Y3s* z@DRAw^!-yae^Z>MPlic3ObMb&6N2%WNr+2JR%a_OoR-+mmLpY%`#PH#Q%7^jw=HB$ z49v1bhYpV3r_shLN(Yb79W+NW2J$gCLn)5fDDcN6U5NEwI~U zuH%!#Qg_)qj-S677*IacjZU$LJ z z)+_tos<8YhP}ozg7z?^T!lb495(*SjvxPjVfmSK%nRR=spCteo{3XmyO5}=)uu}%4 z{U#KdzCpqp)zYX?Udr?QHu}QeN`K^k^z=I0V z{n@M{Ar6IE9uH-jA2J8gcRNHW*eSc3q54VBy@v7N_PKcBC)y25wcYV8_Gg{{u7C2g zOH9_HpF{ge!~XJTGZV?o0)kl!tKMd;IyPI)r0TF~30Nm1PhwYJgIvzIj+=L_9d%cw zyf5YLial4uBBWldFX7}d7JslKj^;+l&S9vni_uF(uzk3u8pK_O2PB=->|e6RQ$2dW zwME_HvPqRApj313=UQxu{f3_>T>2bT?Cvm>sGDOd(Xz-*Y9QKy=X$5tnDN+Nl()T+ zO6t&G)z`@wgveP`G6(%g$Ryf2j)2*Ph`FXPg4fdZG@yC4G0ty+TyNK2N=h!Dc`a%c zTJAqCETpp#OcxjpXRd`nT1;P6kh5PmZX@Jf<;v~k$5OSu(N->-R5P9L8&1!qdz)#I z(OaKqBfNZCQC8HGThbIbAV%O>d@1sn#V!CbD*x|y<+oUVm4rK$FikBj|0c@8q>E|K zcj#gMqfKn~Z(SbQ!L39~tit%$xT}W-xyGbAXmOw6>XyaM%E3V<4nL@@uz*Xk}q> zlNz2aTQF;@n@!y12m@sF(_MLdKV;vTc%Fr3f*7tM!gp%su@X}+sHq|AfXk99(P{w1 z5+Q`bRWU667~$LUoyE)i;|7DBQQA`O!8y1jhRnc?Y;>6n4pmpomcZa=Teh`KIK^?C zJ(@%111DzjIprr$qT^a6336z-k=kLc-FoXfv)m?oD!vk*1s(vvPJlfu+HJMPj(UmNlNH5Fg=7}bMG(JR57ehLIXbAG)jdu^7*p`l=35LI-1 z{P79N?^I?1{*A8CI*j)MJ=ta`AZgVM7xrDW#N7-1L#;{W%I|T zU1{uvL_>qi_57jz*mkIzT7L>FL>3wriTwaIc(6F0>nXEszei-vUjPaXXDrPE$}WW31WM$KT3K#7$Wg70g1d69E1tZvaMsN)UR` z`DvT#lT?A|jWy>7al{dnH1ZbsXt_NTWEKh`MP@JRygL5c=`UJrs7$2Mc6vW$C#2|+ z?bSx_Di}UxdC5Mk9H|ArepbjpqZ*!+w(BsIkD!X-+y0-IopVz9pvW#M4DoHZQJ>#) zbI2TlQZJ8e>z33Vj40ylKLd1eBgL=qjQ#pYBf3-F(_cSeGJYecxA~k!)?FgG;!T^N zA{Cd!ES@2A0M4r0llCs`c)gzNoo$svoiE@MZxcgwcHi7g*zuems9Hu84GmK@{42-( z_l@&7sMC7oeu`0j;dlu>4_~vwksFRNp=`Pyu=slQB0vk`SJRRp{{2cp!h32v)sG$A z76e@w0?2E; z4h%P%S~6Va{LSAOw$FvlKgoO6N6$7a0dd$CzhA~;X`2Wb}!zJo~n>>=7JbB{k9qMnb#ef#~i z>`H7wlebq#%UQ{SfXe|_yx5$U+lh|X!faobXzTHxaX%p~Ut`eoopCHl%d!A8U{GWr zT$=ipZ(GOoo3HvJm6C-R8)4$TJhKqxaxvf2SKYpfk}Laz=|z8=l;%IVA}VSk2%sHw zkmQQL)=_FGiRRedAho)yQ=QJ{9)Ep;rdu{ zXR89_#7OVzY;WTTQ*0ey#jhGe>W*O|KWHO+IU6fF+K$%|^*(RSzCE zw${Is^eD%yZjg?3Ki#2Yb;n$0_`LYIsP}E(zbGlRo0&@*8P&8qF2xTI9}_p+;tN~y zbMbFl*x4b(_#vWVbSC0|KDG9>W<8(@MZP<0_r7|Z?~&Dh6YZS6Tt1G#B;iklqlt<* z82H=Srs*_d5a5K(wp3beE>kktb}SC}#88|{Gr>yGi`_p6g3()3`(V+DB~`}a(kN9A z!adVhozpkwSKhDlcRvebSG;qS?!clcqsQ}ByeFp{x$D}R6KO{!dq5oWNLCkX)>Pmj zXRtpbfv6^jo?-t#Z=i%S{$u6Eux%cf5 z8%gfkkQR;a4dQ9OCH^-b6*+l?vwCq544}oA!%6&S4bN%`kv(Se7pKbqS;ps#d@)=j zY^1^_pO34VO9zFA)0F-|X^D>SET<_QHyr1)Nsd8gsi25kdv(GEq_m9?z=+w`56e-Gw zWcIaQ=Mj>|tXY|mBw11o$^}pGmafPTeq7OrQ%zCi7JcJ`E)tA7y_ffnL+_ZN0D#iz z?eAOxZ0WGcZ1d9UKk0V0jEO2qAM=&DVg;B8dX8Ja9rDk+PY?Eml(Mgdr6bdF$`iFI z$0vBw;oxb>2@8rU4eITF6SKKe$oRzn|HTUO$US%ivwDg&Lv?MspTn{M!W4On;%h*`ma;6z`wZ%s! z*>S)&m_;bKB^B2?&^5Rg!aHT=wMha@|NSSM&^e6T;s0mLk8skToU=za)>v;hkGc2watZn z%UC(`k>c>PS{V7f9yz~&=RNu$;&+ncF$VH%K}gS2DPD^O8oAl3CVYCbbT^HV8(!9Z zXpg@OEpOO){0_wj6EDND?a#6& zX!Sg?LP9!i=lHv}?y4{JYrmBvy_o$k;Rx zFKGR*FOw2JKL$gCST(4YUPO*~>j=Bopa2lLTf4M&l0rAa3E!AXEbbM*_+@UyZ4M9= z3a4aT8CP}5*QGKsP}MLvHQgY~bat-StzIBDHKvFYIhzot-B@Xl^K(bsV&2{PC#O(v zByJiN#GH>@rX3>de=E9mi+zarjTc#)(wfi0+K7>~YWjnfPS~(mjamF-Yf#4Ym512F zr1CB`e0q?2NRHRrNqtlz1w0I__d-Vza6RyVgH3@|0RTl&N-;?>`5j-my*_piJbrqw zs;*wm!o?+USyB46lu+ENmmE$ebT|aw9}CC=k^~M&;-Gep`^inqP6JcGJN!_o_n`o4 zsnL}NhaYnAswc~>XAr2Xi=};qx3w1`dUo!E*Xo8q9N*DWJKxUPqR?;C%`dWuSAWpo zlt_t-NoWq*X$2!q$baQNs(yW5u7Gcsrru6bD`olBsk0xAh^HaRAhSPIof^F}bMrsT zp+NTMAvgD9`%fG)uQQk`2OZG(@bR z*Xd?ms_jwoC_2kPE-_y+Bz827OmsuOBjK(IC=FHokr`wi<3|3BF`JVN$lPoKV*&P$ z5i=O^u$7WgpxAyKp&izbO%l>gRY$gRqJ~(*fORop0O1&%pFc0a{1`C`JsAiG!AWGU z_=k-y&iw7GXhiEUIyF4)2J_2{isgaG;P)<`XHBaYKq!7Z%q#&eM zLpb$WyY(H~G)bc?+L&9-O#QT0`k8COuY5i7;#ME>k8PI^6b^`befdNH0q zc-e7qi#KTKl&JChKKpLmmiH7%pvuPkj;VaCS$@+vBKVjO4@S{x^G(#U_`bDkv_gjM;{WnuIU#-${OVBS4 zd=umJ_Ub!Yt03lc#>=beGej&jwQL|@C*p!@vRLfACblpuS)FjAMx^3hcVx)0*V#t) zHxeIluBj!`-0!M-CiMfw!3m{b40a`J#ct_BUBGA-6@;4=!N=+J{+C8qOcfIGuQ{)` zN|l*!lIxB+y;s*hUnW|921vhZ^Bd7eW;RE3t$*6KUK6`Ys*+H7^1(=pJ!jk7ti~Ii zBOA9ze^3ulA7ssL{;F-*Pex#72$}BJ*pH|7^_<^(>xH~Z^EEYlM3}`Py_sVXvSJ0@V zsEoyPqnvCJ8fl^L$xYz#8G9Y^VAQQqYgC1y$mE()w2fY|nBhMcGwSMC< z;L$qyV8j5M)s@#9TbR7`J;L7)R?S)r;*Waf3Kuh^Uk`PUFYCuIG#^KC7Q4Qwt1zq4 z+uZb$dH4Fby^9|y!F<)i+Y*_)?G$91c<#)u;r{2!y`LZ!%#J6tFIh~UETMeQ6+IMw zy7T$>J~0Bo*lo#Bv$XT*N9X5}N2;Rc@#?FORTlQ+W#yMfg^?n+_3;9}gX#6RXiEU`8$AOsv4!-v#{90p6MW@pnpAsi?kD8BvtDV!1o2^%b#axZObx~;u zfI)NYsm;Jt6dnSwK=fi2X?O#J0Po^Mpm2#ZIH48NArvCpqK1DX;A2B@I3Tm`@_vP^ z_Z1x6`&+nOA5N<-f2ZDrW+MntqJ;4QSfS#EYi(%+NkkqVPw=SGB?_#DXx_EjVxr$o zHX!THN3#*~;RwH`;Q@Y7f}jtlFqxz65*cEWV*{-4E}`&ODmGSDY3bkm>2UFs2w64` zScnKX$}eTf*BD2hPyB#?)+nmKslzy*#gZg{+0V?KEyO1Aj=%gAzURzG%aX zsnN^H$!p{OeS1Cof!BRHLoDDor8*7<4hwTPuvP?oZ{0~R(QGNfHG1)>1TnEVK*SIf zeyU(~LJ0IDAUNoyz&7pKlaK{acZlleR&y!=0!DR3nW_KUYUqsq0xQ_@#ZY!@m6@uL zCGAU&4O25Mv-8heNvk>6+_PO`H1|H|Z$^S_*VT&!;9^zLqId-Ye0 zw3*<611VyiK~#w+$C8e#Lx0wIYRGv!Hm$b#0K)4e`iHF{0jMY?|I%mb^l>D6k&hLM zdeQ&kZ6N`oHrzoJ?{L!&k~DM09*ojo7Z(p9q;#R{L5%zh)I4r~~W zhotqIF0|5MrDGk$hK0?Bix}FABL)2r{On!jxXcECOuC|?k%LH?*_fca%5tk+&@sc! z(Cd*;frJ`QGc727kUCCLS~>WNcYt%>kA2M|cdfaSR)4?K0f*6ll3wFZR1n@$6h(~I zi*pOGWfvGPb(|YZNlA@9ZeuHy8ykXm9tbB9e2eju$XOB-!C0Q5vf=)m)3dX?WeT^~ zF2JkvAgs1XEENAKHnzI>4|Jn_kk?vG^)?$hX{>wzA3C8uUxO*c6>`SH3&x%5*pyo!HKny z)1%YuvR9@+czgoD!chdU;#5F-E~`ol>jZ|z)CPZFT6+mPScNjgBx}8))eIZXhP)D# zl$Q^U4(=bG_Tf|aP)=#>^j_itO&LUvrbp&()0D^_`Xf4kqLE3l@KCjyuz?o^vTslC zkz6dFj+ZFdg^-O*$u?pRU#xg93y{kL^m)zs5MITe*<2gBbaW3)T^K)|Ceu-y|{%l1~q&Q zMuj5Oc_5*W!PS88GICEbCWyHmKUfyIYaeLg`%yRuDIGCPL}N2NKsxEUm#su zAIZSzc*fZ;A1;VYO(A%Z!x221Nk@qIHUIoc;RCz?Cm8yZPJ4DfFH7~hzF+5m*Ca6c zgLzz3Rjo%?B#*_K-(`eK(oxU1*m^H!6}^}=v-iP^#zLUirAscxW+s+cr`!;9UZ}3N zP8$uEj5Liuz9dXAFaR|F0iAba_qnCkY6?qf?gn$!_iaW)FM@wvRyJ}twWXFOw9AzE z>R|$<^H){WeLE?$qz9@oci1<5`REr0bi!8ZKqRId4P)4q-og?J?l&F z!Tw<8f6BDgn?IowV?^Gi;DP=6Yu4(u6Ooh4fL#61eP3}LNo<$1Xk%5u@FFh=4NHe- zTsfU~q{sFbS;LCOZa5~%PQ3EErVtVM!e52zQj?0TGz;BxQ;4@7YM3w!(2UPWFoL&U zQc6#WqnS5mCcPx!C>j&f&+V68I!v|8f=pT!4RP(5%3!5%!q`$NH81Ro^9{_h#@Rm+ z6J~9$SzS7?E}&o9V<+n>HIcK?b-#+B+o*3j6d)`KqNype48vtje8s}Y z4GN9dh^$}-1qg!>;H-=%n6(w&am>MELC8{yD0%=OO)~TMZ-gqi(WvVeB0|ieRR5_3>tEFi)3j-`(2NdTn?4y~&)S zh>mQXCKS|qZh#lSRhWCieZ0q`{51w5uBJh1`CexL;Z;|HK&omxBKddAvUU6+jw9XU z@ZS9xR8%yHM?CP(a78qb7?z>`PS<5jdeC$QlUS&}VLlBfokxIl4pjRg`J(Y!gdSt-&w(PlQ>u$XyIx>I*nNO z>sG|{CiS``B3arA!=bP7oqpk)W@jxbP2MSel>Yv*DB@QOzrHABuqttpv((x*L6sxJ z(MKW_B=u3y!g}>n>`J7^Dw^=Yz|$plFHn})%3Vc77S;sNP~f7re)Y83^VC{z37tHf zje8F2frvekp&6JI^k?rr(zxRv=MW;UWW0m1v9XVXh!$py$7DHn9bR2;X3iRc8oZxY zYz2sNKutAO;V_0g5Y1&1~zX^X5f z9$Z@5LE1XJQtcSpT;lzsc8>>XPi?w@(2gEuX&z2aPC-GERa#TicZs_tm*f1J7B4#NG43)|#s)B^*QE$ry6~Flps%NW2yT_K2kTvHfLr zUq+HJ{=Tfd)1K|Hwcs2mw=>hx&4Ns9sFQE#Lkm&TU0uz&w5H?pMQ<*D*0i^6aylA;E5qat7m&JD1LQU+m zbAl2;6zjpt%F4!zAL3BKD)mFmSaq)B>O$*q+j>4ez9-WyOy$9-Sd&@c<13P?Rwj$N z$hcKGiKXW%TVCAk=R&+ z9t$h?^^aE1N!;^OV39~Fj_<^eyumW$cquq81H#gU=3{zV`{G#O)L(VD0DNvzQc~E^ z;#+?vfxdS+vEAwef;6M)nInV0e!1KUt(Af4AAzW#&>$UF)L1|dgtFo?2eRajz%#8x z{@q7$54X;AiYX-v*#GPL2KjE8DEQM*fa;xL-*PC3#pLw0^JUpup z+ZQ`w4Bgqd=*fU67GI2W}2?;?Fm7#^BbrKp( z?0;ls-8t@}$;)l*hhKKW6!JSw0_NLl*LM>1%x%pZd|r41#bsp$eZ-8FyQ0aN?8f&# zoN=JG?lb-yds)9}^?kg*TS>P-ZQ6BEi%*ENWzcP>bYG3~);ZB&N4{%=#ru9g`Y?;Q^H(ULADEL-*P6%q zHUN0}BPEkO$`^BRXq0!(Yc+OKG}kcdC-q0M9{R3_0cVy%ri#@#hfat9MQ1Aw-IbS2 z*nqM{KXu1Dc&e{YjQZtVs@?$eOg7lJ7DlF558^Z!p=~N3pRTb>=864ns?Sq+$*8B` zFr4!8?YI^7;u2zLoc6Zw&6ir2_L9fO2+WrR5XgyH^6}ZIzztQ7RkoZY2$@`~Zr#yQ z_X%Tfn{nouj!Uyy_3vjm(jH)a=uhGD2x(HGsO=ByrHumM~*+;yU`rngGq z9rGrH^jPp^ia{Da00B5aD^#ZT$G(nkne3O7D+myTaW;Sd@Ay8kCUPJwal}iB6$lqf zZaJG`&hrI3uH8q{r8lG(QX?=V#2SJ!4E>i>F}nFX!<(N1TgfP+Yf<+TTc83-#mBIa z*}*C)KV#@n+WvfOoO@fNMP%>ebqRu}<&^`g^9T=%;Pd_KNi{52!IF|g zTpI88Eb;YOkd~NUcCva1)4xhq9}qu@PehcGhM&+j5!tsE@-PrX1%TbwLTB{+T1M!w zo@x)ZNPWwFUQhWZfgshYSRIdBF^2(I6$+8r&n~jmQDZpRdkHj%xElvox{bDu2*7nR z9iImPa1koqJ6rTPlor@gu#@7Pwp@9_q)rHpWt89jBG(CR_7xN}TG#YPd`sOMaE|dNa%=762S7cJ$x+`n&1MGs$FtABajK zSTjJBf{VFQl~w~6@&BaKMGbcljRx)79!43cu;#%dK++B{Gd&sbZ&#x#f?~3?8>`aQ zXD$M`5-orq$Ctfo+;kvlr_Hz(0T_M@h0D<&$`*x3Wk#$30;33-5eOxrq4+=(FF(U*;R_>fXBV^>Hnk09VH8srCyTTT~V@WV%r(5!#AD7N>hEahr6MqAWrg_Pb zfCJRBqT*4RQlObnhM|&q%Jff(i8Vc1&LU7CP9hlRF-}t5FJ*h30B(9Dmj{Y0Y{PT7 zPv(!0_-NQK>46;`-IAG7V6wN%1H5ypkr^i*?crMEg{GzdIJ>>sb<`0F8NNo>nL9yD z$#XxRX8{aALcmD?d|_x+i9-|-5&{dnR19P|+Y}xPAHD^J7a=KAMzj;DvXFlD@Mmv0 zFx;9MY8(LvK!;~0zzzyU=^&IbRDmiIzh4kF20$}e{8r3F?;HF-4Tv)EQH_x{)EB2k zql=5YPrT5K_y~xp!dWu8_7lJp&e|1XpvG1)GLGNy{zMPgVJt_6^2QPmKQc3U}$q_Bnu$w^|KX;{0eGZ~)VP^{nXge>PbpqV&&E zPD9)ijglPfs;8m)io3iz!ytU0>opbCAL%uRkU>;!d$t$Vb~;cqb=w% zx|WoZzsekx%X+?s=grnCpVy2mYw3jK`kL#-4H^QSoSjwEz}wF8ea>gfYqe}FbPS@W zr}a1(??l^6v0ZN1__FOM1)UI;qa6c%Ig86=i$IQ5@_1ZMRu>+&|GLVv`h{wV7&_S4l*%V=I-vU*PZQMm zzn;RTz1IO-yW!AKmJs0DK>V>MRurFUcL4J^DYml(YhpM&b{x9?4^A-C zK2li30uP~qLO(93vrPaI5S2^DW{QNabA|&@Gy+8Isaa5xV@XqLh)fdMwG@v0;Y{V_ zVG(FH9U-Jj%p0IQ^dqbwjf$2Nhh~zJe%r%df*Q+}@})JjX3K;UcYuXs{KSG%^muF9 zAC?PYZRhaZ07yVxu6gm)fxL@FkUP&GlKQ0C0d?0#T!Fw=x%|kKuC&->bse3D(dHDv zx2^mwBU~h2fWC_SufT@Z;w8}tVaW_C9V}lE1_jzVSs+-46^$J%SU2DZgo*5MaD3L+cFM_<X`bWpEh1?D#)>3TN(OEO8&Q3CfI^bj4Iw}kP)Zvyn-6EPhfP4@ z*7xj3!lbxVipNBPi!>Yf+5Y)k08fxUJTLyhCL`xc*`hRh$8oCM@1+KusmQj6y3^0L z+|Gs$mbN3ws-MxjnITlT^P3{(KZwMm_EqEAOx|7EFdAo7yoUYG0!!s`jRQnOw!1S& zQD!N7A;Tq5XUTCneFM5L*m89_pA%9){DXdF(At2U6I^?Xl8F6iWar=F`$xZ`7fU4x zkBwCYM;)!%J1=obuZU8e#0G*oq;v5U8A`%e#w+US(U~GK>9&|3O5NPtEMBk2$&5H2 zv)Y^@3DS|Vmjc24E*l_GjIE=Cp`8K5fLm!d+FN1WrAl9sSK#lJ`12z*T^)hIeY38=qtca zYjX7kx$b1Xjgv;|tC4x;Q%~V9&y#cwU0&5n_mQ*L8@;|(sIWED(#7Ql`P$#`oX?v9UA5`E z{?|Yz>m}VPP8v-FN7yjag(`oRJd$cwo-kn#ucIW@1i`c~mYaRetOjn6Ba(E_mg9y# zrHpZo5KPM0{^fI-i`URl1LxU?B#IPVfrbxa;Y*D`vVA zi;QUHUeH(M+Y^&ayEdU!uZKte!(Woz&y)Ux3>I&n7E&wlm^rN9$c6*Gi}u!S2;X%eR^`+ynd*jBk{fNe7Z`1>Ti2-!Rx)fOxD2s zEGcC_JS@N5@}u-%`hcMO2oWhW^||ZqQs`Mh>ZnRWT%3`eq}};ym40736&|2(2%b%K zc6R2Yb1jkj;rZC?`+T9Q{p1IvWuRNBw>xX9wQy94`IIcrq1Ti}QTBG8ttaoi_V${> zWQGs;jD?gdUA^|CZ^nVHs;Q}|p%M+owd6efki^*SmF5!=7q= zX3jQ`ib7vdQBVNGg&7(S(#@*QP4C)CKD-VI@gMK242JOCk-%ktbu3tB#8HfL+xuh} z5ssdAUn$Hgy-`HXR9d`PNjbp7#^Q3Ux>Z_+Izx*JUOvrey+Nx_eb^j2+wC&Ke`(W@ zggRr577b1&AO6nC$=UEI~bL*StIWBWN!8s&mx+ zqzyx5HJaTu#nwC=gvv0OG3m7$WZ|-Vo0(W>E<|o`Pa7!cT_xeBrKKSq0(i;sK7PEp zzLt$tkd#Fn(NwA{Ez2XLp)s&Dyc{0*qP+yey`D0yHaXiUvr1RP2=X_&gL}oa*;u*o zuN=8-b(^P``RxKwYiuMAI=!1?aEVo_RqoD~+s57036UP*dBIWkRIkUS^*X!P8hSng zrYJZbHfoxh2e;ePu^axcyKGcVe0+Pt8hhW``J_xwclBPbd@~w2jhvjWEgiR3n(vaF zr$4v9p11bflt!2_B|<_AiF{r-VE$IK zG0xh0Oe{VyFW2tJ=iPek*}UBF2oJ`(n&o!KF#s%`l8w1`=IeIirLx7|6r()(`Vcs$ zkjCqLBus@SVRA5=d^qTAafE(0e6Mc?1N~Xe0kTI!5PQX11q8CUaM%Am?`LB^ebMD} zt($r%hl!A<8qq0_iC#mC z-Lwzhsrj4lQ7(aI3#;xxr#TvM5z>>M0&Deo(mES|+pLvf;=&D7M?ZX!#IcH|-=PCg@q)Ht5CWt9e5;lu!w z{~LNN{bYYX9sngaF)?vgvYdJi zZWoITp}bMWRYfgjS?ml0g{C>RH!OaEI8&>hdnZY&vtJ^=$jHh_OWJ1$!RY3Z>-Pp| zl)lCz@ydb$7NfG5PPGbuyO%h|1-NfIpX}@wQaw)&Biymx2NzpPyq3~MSk0Bv(U!ZP z=OD{|4G2s^bZce9g^~5?;_fyHq)$}py&d;1FCY8&cV_}ga=+p4ey%?d)XPvYC&@*@)ZB~8n;nse1<&vc%jLRn^YLde?9wG9jm^y@9p*cHt-3d-h{bkw3#lUb+N93Q?LSy|BqQYsz0 z8`#0BZ(7PYl{MThKYoad|C-}-z3QtC+e>Z*;H2h+E*72acwZ+$ z)_W2X4+Ai{Pb5|Ek+}XLUViJUTjt1k-B_bS2RMDOTk%*QAEJ`yq-b1;$Zz*}UcV)7 z)UMR2V4q-N!a1_EEUL5B&Db@aGUuLKcDP}T$)*k0ki-gU$hM!-cRiY)X3zS{!{j!h zsTljVJLJj4_2zbQ$PwcVf}%9VxWwAnH9C23HNgshCkl(&t;!eLqDXJwJttOp{>Mf> zSr0b4zx?vI1y*K*%G;kN=C{F7ryoZn+*5p?mY!=ZXtKM%CNS15w|XK&*lK_3dEXnn zWz)$;U{oJ(J>)B_cna^!sgjGi#kbV5XiF?)iNG$3(p*%7h>#W_= z$=?2azOov!ST1;ZvKUe}?cDx&9vF|iCg6jmW?NWFOUJYnofyl?Kfg3-4?DHjYk;Sx zXVt8iQqvD%@7S8x^WgCFl2jDIU)!;^*5)Gro_Z=m#^n>mEtkWd0Ro5VGxxt)+)NY$ z6v9Ub2NvGfnH3G@-r{ws8O^>gUNSO6;W_R&1w61gLRSw9Bz(chPeh~X-jf>xgQMY|;z9gSVt-+@nuDmdS#*tYIYS8WA7XU~(7 zviv)!-~Ep3;frqip(2uXwCsvub7wae{d9q=Osox0%47ZAJTv!VJLY!O!X8FYcd?1) z3H=_Kv{2~jYRkpNF-$O-(*vVqjs3Wr18$}`=uY>K{}rRa%PVIuJsGzh6JHnKT5a@I zCqG4tvKb(!Lu~+LMc#6O`ctFe5m0kg6)9Le!#FzJ&o^4OJKVvItKj23N z(Xkn|+RDRdz*zUuzHiV-jWN-uAEi^V!__h%_Oc0F_2jJtssi8xMU4miz(931wHhJ! z=k-_IN#~g^yEZ<&^y;p`pU_5KE>ol@kJgfgu{C z=&Qc=S2$T;2MY+WD6}RqNUevp`iBL)5e9p;=5GE7KHIzXT)l8%Yu%`x&m}erS4}Jl z?a!esy!=XB*s%Ay-jK-dPA_kNfdec!SbcXl!HCJODl%=jncQmTdak~$FT>FSU^nC4 zD+d`=0j3=t9QyiyDLmh32WK{=soAyMRIjeRRBl~#e;U;NUd?G!;+w2wy&8K&aGa5t z3><i8D49e$rmN?-kZvLwCg@pLjK^!Dsy)eaaQ z9U%PjS>R$tPmQ6x{o%I#?dYsOFYn5sN<28&_x?~jf9PT}4Egp8|NP|qv{N0a>s9=t zfM>8r^VzB|uwZ89Y)P%QcH=r!c+AaPXJ%~8<3;av$yatH1cpWmt^6C~`!eYSJ}WA9 zKZ=-p+gbBjnsR@`Ws4f2G@=-wg_o`%zhom~%m<1JjKRn>`ko+qg?yf%~0;Rj>E8nb=j#rcf* zpr7QGImTA5ODC(55-8HFln_Ha(~2*phunUY{+2~UN3L@#4Q_w)kA6UhHZI(-bkM=k zl5=|Q7y1ETz3%qfN5{sX=|EUc!QwpzdzvJUx!_2AXjW6+X;4#D*r(B(Tle2#?;=SA zVB}kC0_H?u69vQp7cl+4*0XG_<;Qd93ozfv))eY5JAG*BIurjf&B~IP zn4r9r(RZs>L-1j%A7M)ul$3m2_c)Xc8=&cSy=-tOAsJS{!o|V)^-DaY=6JYdVq;^f zKeE}W6n5nDe!A7+Uz?otV)Rt;u{k-NXj?`8v$P8bYsK=t-U#g>W#B+5ZxhW;Mc>0pRIR~o|XSDI?h1xxpNH)_Jam-D2^ zxi{PlxV@ULCDL`f=#`iotruvBTX&Ef9W99Kf#n&chewvjr`@!w&a>L8>*}(yjzzW( zD*yV!?i=J1kzUUOpL9OkxSs8tF1S_{--UHOx0GNU3BEm#&wY8O$CLHPvtj#m)@-|^Sd>KB^3q8}r@z2XiNeAUV{HWIQ5fQi+3rR`1Mn)J^ z^1MJbwS-oW`~5FMuf|?pP;3--eVUTAs93$>(d6GN%`Un&f@^DY)wvD(wR3CFuR}~) zOMCzCrHO~wLb&m(EE`fooRd=Vre z$hg4C1yk*qmf7)p{z_6)B)exPFz12)#&h<3t^V!X{p3*a1fTDnFkMV@Qrhv*|t6;VVz8H{Cv;;PSaY94kFZL&=N= zk-?{@rN`=KXI2y(NgcIx0+hwx-4EVsLk(8h!P!{w*aWqy%gZ0~8(fc)Jzw(3%fGBt z*q-RW`Ki7SYuklG(N9jfVy(`z$b6<(q@$(fIiG~(mI=#rVd#lL65BtL?YDTR~2 zBIk?3la1K*xHQ11R{2O)_K&Q^WPt!AW;sFMo!R(+tgaF5zF=+W%Hw`TesghYjpbh2 z)1RHxq}Y90h83Ti!MNX>a;T_fQPK+uU*N6N;nkf%(YZiz5W>i`v8$>29-` zVq(H7fl*efwWw(G-$uggdA~wFRY9KV9va^elP>q%iqLoEnuq;~Y(3wXn{28;(b%{+ z4qje|#~#$+?n<+GkivMI7iN zNt&OSYHzKgq+l`dG36$8mQ!{(>>IuTDhK>qOk`s{J}~#Zep-854Qrw#E+<(|C>p!F zyCDN)+LXhQO0Bheo{e5YWnixJ8pF&a&ulhU5(v?Of8e(9`}FcerROSOvDC^zXCrr* zV}=yXxnqifd5h_Zgl^PUQBkoqkj+yrRdC;^@33vln3zQJ@;F^jd4Ap#9;zkhe1g_Y zTc#tpHnXth(kwuN$()q)pY<7+SXuH@5G4IXeJ`%0s(O9-*Be&CbvqeEj7ft!A&K-n zFAs|*G&t_9jwknId~r8e`E|2`u$8k(PVX73S}!`B@@ug@Hdsbl+ETagF|Utp7(Mer z<6?i`+`wQ%rkup|YiQk$w|Bi)(G&=hqNJsKx}^XL{ZOr!Qvpk`=aWPg z>lB-%wAX|#NGc}}ylYYd+Z28JTGfo+Z(+!1%MFW<<4`Qw_)Na4Do3jR;IY^H)@dA< zSX0aQ@*`um3oC#0 z77$31dp(T%By?ZQ=)JNEG3lV9O8<<>qtNeR#>w zxEVQge?W9p$<}+jXJCVM2R!170Orc|+PxoAfyC*T((*_UQi>|kNtn!Qzc`Y@XFE8v z!}Qjt+mn@<$w`OP=bh)4}R~5FZ}D-9!);H z9E9mJVHkZy;F#Gd{|xgl{l3I$iNLD-Kk@K@I2QsME-tQtAk5QchteS)k8AVCM`r|Y z1YP^Q(iQy-m9yd4o$7MKDhQdfg54XY&@(d^UtC(+i-`A1PX62OND8G@M{4TssQPl> zPuSgqGrTJxtBk7jN;j_U*RxLmIXU?pqfUkKu7=)2_ZOkGq|EfyS6F+SkF5)>tI_SM zfU`>aYH|H3oW%iAHC10Qk&kt>M-{a44YmnAsS076(EqWhIDyH<unYu z*m!CD>E(-hA2udP74;9DI(Hj}TE4iDGF9nw8#Rk9s7!a;j;Mb+{1~ zV1Fv;3&GD}dZh+Or!bgB=vNHOO8+I#Rb>z@6nP97On?+nk$AuCdEaRfdM>HT93y=; zWX1lLoZU_CxV1$ogPJUga?hh}2n`zFmj>#~P3@1sxERjYePB#Fg|aNl?=dkIA{COc zrZ{BL{^Xxz4-)*fNWSem3jQM4G`F-iW{#0bvCUerukK59d`4DGr=~t^6cSystwYRf z<$HvpsPhlAQ&$70Lo~WNcis0%=+(6oDIhg9_0{+7%r_%BY1UwrZPUWaDo*xg%kk|x zMQ_Kf3~S;SbAi z@Q`~B-PZZ@y97mkkk2OR*p>#!FV7QXEprkg7(jwZUgE)*O!fIqUD zE?`YW;ol~J_0uE9>bwLZB8n{#vB5YhAe67`8lSLe~3iy)0q3%Wb`zEh#1H1oL`9U@|7 z_&t0!Ica4{ z*7Im>{Lx5+W9mfGc3y#KboaPzNZ8p958T$gH&4#HN(I_z zRrQ&=Y9FN?X_$`R;8GbQd**_lDgMU zR}vqW9qY!otsvjW{qB|s0!)(C8Mm^df$o&Ka>A08vs-X(UP8RB@AdwKto<)*-%HR+ znxeYqkG;NLG&wUpp{-BPY4;f-9Aqx0!Hx`DHj%6mK zI_{fF7%TN1I9ZBZ-?(@@dSFb-io!TO*_DPyhj6WLoob})8*o*Mq{GU;n>YL~ny$e+ zuIKGsYSh?HV>M`O+qP}nwyj2uZQDs>C+~iL=e+k1*fTpbckVMEJ)UnRkoeYJ z|KxjL509cPZMTTbta?5pHxib0xYC;WFCC=s{>{J&HfU%$&SMOGi-n9P`hgE>F!Bc&bi=km4!GwG~~?)rYt3ERH)0 z9Hx>BVjvt;d^jw6w!dai7VlJ@;Kn5t>mAmtfJD|cp`#Z^5@XSrB9&q7^>a3~eHApd z8_RkwR#shw`{73a4dGw+)BL>S6;a(W8_!UHu6vn1aO|4>$Y9 z?2TGO-PYva6fsY0_1T}g+T=*|y&n%S?gSiBu>whHY1yseE9=W!At}&5!W^omNuVa) zPA`R~<9M7!%ugSemO@L`RX{6teil4_JUtO85&%uIU>1uAYJ$9#T7@*_CgAqejfA&a z(pudCd^$P|iNs;>=)WoR?aIsR>&P6XG~vmXBNg?PKi2cD4bSlJz`@x35^%qMAyiE` zfa?(NdQudG2!e)-gM=!}gA3?T)2O^Ut1xIz%T8}r@62q^Y<#o4vU2Hg)tI=qQc?Y- z`pXg)90*H~@!emBBp#sH>&5uzTEvJ_aF);7)5Wlr{Tc#M!bnWa&8^L|?YVtCeibb< zs(z^XB49411p4~Ih_2&Z{_ENIoCMa{(fM*Xn!q-T|8@SW%hIjLcgNh9q;A5WR1!cr z3)n37FDxr7>h4|@nr(N_jpnxR`4Zf`STN!x)Re5Ls`-g>vF$j0-P|vfC%psC79M4( z@H@_NPa;u50|9|Q9G`SuvE|{RA)%Cy=z|Xh4so=}$sUZHjDC@sN<{j1yY=zct?QR3 zx;8sLxU(F#JROAJ`lX9WyJ-r9Xqe1T01%*y6td4$nV08B+MZ39&guz6<+99nu~qMA zXSdUL1y8vzGJw&{wys@M)ud*k!p8I23Tn?GQvd-#pKaFtn73`}N|^R>SQ_#VL>ff3 zimu8Tk+0eI_zPB98NtFWp|4sC{ho6Z4+;ws-_!j#d7w8wq-A!<`<&THyrRl}l67+L z-gL5LjOGls@`OIRA zr)XkmxT}Zz+Wd_g`JJc*5ZKVrRY-HZS-1Udh@G!=iZ64HKai}cs=EAi;$u6)htYV9 zgKg>hd(}+s#IBLjGFqrk|`&5doR)8}sK zyfpUnUf>#^=o|jYH3DRcaR(;@p;kujO=+kZPWe;t_-FzHJnD+X=Cm4KY^9rWi{s2cFq)Q^e|+F;5hBP%=(vkOAzI;J zV$!#1(El8KU_&AtCRA7Vd|c$a?EY%I^Far>*ln$?gM+hBVcP_&Ei9!fe*7Rr4(*@m z{n2@$2>^iGu#`###8g3n$Rc|?cavkE_tQ?8Obyen)Zn1z#pK0B6)~5;0==f1=I+K= zh|?@_6YR@NfsKug)M$PzlXYoeT0D*g#+L4%t=sRUnBnAS;cEA)pnL^%TsTwc5avb( zC_@uYLi|MO$WwLFwG(9*Y1`$=*-z68?Cn8rw4c9Ik4-5PPnzNdT=~S~emi+PyRRO^ zKyr@m{F~@CcJemXU&1i>DYmq8ZtXb&^A^B#bob(NS+%C9Vl7+UzM=jA^Z4kp^_$S` z(q?(O=Y6#0HXuH}Uat@PFLLV=4)%f8`(viwgo6>*H*P1h9m(UwsZL2{7md&U6qo4@ z3bCTZc6^QO-0TKP%?O~+?f!IxG_AE~$qk6R#g#eZxDs z*3{8CidqGb46Q{l8nXC46oPC)(y4!XCdro<7L=1xA5oCOq#*IM5!_5m4B}clus>H# zZEa|1QQQV#2+qV`J36nn8c3FtFS*^T{1sK5E!lQ9SwGJyii>-~Y@ILLZ=|82p#i#k zhoQ%ezUD4~s`B!!_g2%yLD`~qKvcxI(WF%{ za|c*b2CX&#i9ezQN9_=zTtr3CMiYBd04e$p0)>(>aVQv4A&`>6NyizU7E-~JiH#Nt zt7#WbSh9jL5&%SJN{~qU?jSd<5KxaNK@m(4K()>IhXzG3Lm}^bts*X72P6R1FCm^16^c2|4Am}yndrqpP!Qaz&1shqfJ%LUQ*+eY zPLnJO^%tBFlAI>q_%>hQ`PPIizE| z7_Nf}DO`q{HT0ekb8}(haNLDZQ2{|pA2ngYPr09cE|MtXtB_Zx_UUX0K}G{s5jJAT zz^`0e1PEqDpTcTA;c{za7pa4sbn}KmCbZ+!l4L2Vyc(s3!n0)JA)UGW(PrUPzzO@T0UUGQ%qNK2 zgX>=|v=1HdH!?jw?AW|&)$x4TT$am7`$?^h>uP)qfdF0tzyzqV;sa$SlQCk&@heKIn?87D8oqEcWM-l z=D#68UgfDxx-0SX)old%yDZasNmKR7gPOWM<}O z7WaOvje+4 z3y1bUtIGqwZYj{=w9rUEEEUOv&bqGLY~d9k3Ll2TeOC|qoODiQqz0y|Yp3MXyPNt2 z0g%heE5L+T+C6g#_HaJ`G43VBNy(Pa^~Y%aO{x5=S7an?QgY({m_sg? z_j|LFctYGy>TKSli7A{Dm?;puPw!qzKHDb9$HTb2Yxj(Eq_8%~8qOUbB1jEI1qMhk z~W ze(CeDFuE+)q|O^E!Kt>9oalM+d=-*106nx{@*KMDRf!6p4*myt|C;)ERl|ZPvO<9Q z*Gd7JWPf#2ACi}fYP#4^Paj(W-Uu=SJVR)3epPN;rTBX_zizG5F4#&hQLQ1Nh#7c}TTv$Zp)8Ce+53Y-t8kDscsONnJLMGP*!llF%l_p0(l zJl=c2d=Zi}iK`1j0Cd*nTKD|1c9?hpz&&%-69WZ6#*~l+i3TSuedhRC{HU$zlzkq7 zkKjmctrr3S?!i`EmgoW4>l7B`U~M?F+_}_kh*GglgdU)wQ?B?MZ@M9y8V+*Pv%HQs zQA&qh`K~;f((A>ax!KL1Hi1+!nWRb*w1{S2R%RX!RW9O_ z`GGJ!yz4XD>(1vsrU7UGWfj}E&fYg3j4bvYkC~@|WZIr$EPpW}As&WvU!QAmJ1}Rqdl4pgJ z=iGd)^;ptZ9+(oQIk)Rn947>Zje~Qo)tNv-1W?GZpJ!G5?fbFr>!P6O9*n!fon9}h zh|WCqcwO&V?mt=B_u)%IwI;J8AB87<8mbw@_U{!dE;rVHq=2GwbdWe71FU{ED zwi<$velio`I=hWI=PCazuGJd4C)!f^9-rH?aCN}kB>L2amOr#zORr0(uMHW#H0%r< z3%@IN_^`J$?Rv{%6e7}sGB(PSK+U>NhN`l`klHVushv&p;54P*zub$8>(4>R*w9a5 z?E>Ve$IGf;39^3iiJ7Lj!*hGQiRhfD%)-i--42eow^}hQrLo!k?ouk@7KpbE*@@xL zP0PCNhA8*4DcCrNoDVPIXvhXuR)dZGhM}*Xoi+j|%uG=Ef0BZX(sDIV9NN9^Sg0xX z&A`f(MfI$fYqTTpw%k~uzcWTr<41=S6<3=hWx$zTnFgI_dh$EJOskP@<+ z%XcxM-yRZG-jRk9A-NQ2$5UzmMd|&tj}hu5iodPM7B{gnBbf5>==c zJXmPKKDs@GNNMZlpA6YUIr?*^HoX_dJN2LtrcCD)Deujb>{K0XI((tOx;l@Zyg9BQ zfu(^}S8c8B?6I+V@Si>tMR17m7SpaCkaPW1%!N6pth$_cW9hfDLXc4>8xJoRlwbLa zbY^*j>>Mncpf!l1MULC{ehYnABmscKi8AVS5B~I5XeW^`uKXfUz@{CFVu*)9N*B`2v&rort`lw-S6)Pttmlv`x$U* znkfNR500y|uC4aUZ(z3}!Cpq>OhWIRDIQgegRp&7y`ABsS!rY{i?`Sc4rw+$xto{i zC3E`&8TVLfAzw7%HyxztPpEdRWMhtwS`4Q32C$5i9u-SF1w!aEYBeFsqhL3GBp6Y^q(_TbU}eEi(3Y+PNIBQYW{C_e5j z`Ms_CQv)^#z&vGrXJZyB>vAr{aK@Mm!#Mw|20Te4Ui?;e>1t5@LA5R{mIt;>jr8f4 zIfA3;h#RD!q~ut2n0`aLxNtq~tLQgxo<{(11M@lXJ$Z5cgY~*B@ZK+sDT5P+hk{Kt z_xDKH>vL4RNJa%9hHrRV32NXBx~)09K<~fKCQ?0Kj_(vrX!p8%&h3+zT=FQSik-tc zZ*|_~z10K2Oa6(k!L7J%>vnFlcO&FJx zAX%WC`0u{fI*L?JNiwkG*HuPeLNz3v4yQ=mZC}0VQA50qxOJ2cVuZLpx3_Xt`ELTy zQMsKC`=5#7lWCO8wyqoAye}Z5Y5F)hs12-M0OT;Og~iva?Po+sy~zJ-0hX)&L@X{Y zia*&t3J|gO>vKkZ-QP_QQ;FGZp2hVwB1C-M9{0An-fS!!z%u#V51wL-Oq2AZq%sKj zY}~D`GO7mjN2b;y2hrEb*wetoLOvL|Dg-4t{>TwH_RjsP{l!e|Wl^?pNw3crZo{k>Q=NolMD2dd? zW54k!_JSNIQ}nf1huk7xXsYKqosxO zw%yxWYRYx4s*__0j9vJU8@HvSsrfb)fe1FAIL1d5)br~v5!}XY_q&O4(g9^9 zmUc<4A{*HRy1U_Q%fxTJNLml1QbEG@Lb|z~S~o^*hkxs*eQ%aUo@%!`-1+2Wz{R0O zgWzd#lyX*&RWnMyh$}HvkBX}J4O3c@=#OWsjw9OB1B12Ne)=Gtlc@PA?WWeQrvQ>r?_S7Hw}+mv1PzE5d3V*BfeS zIX~2~!9--l3?&h=U$|CPD1x7Cw5d?Rkq#{tbdm+Ii<_X(NGdFU_$4VC%8(slVRb)D zt$)^kBDC6uq`x(@l+GC$NLYfUw%U6xzGQhz>7WPaYiehFuTmWgg^322t-a0P?90}gM&@Qo+u)C7C z@2$h#_jw+J$_X4AJkM|(#q5%a`+uu3^cKWiuqH#jy-xOd4PDLcyW#>bRjtRfADt)^ zu&yS{^w*A$xl(}zwC>;rad?9_COH= zq9G_KQ1+~ciNo!mO;5!!Rb8%vu;Mj)Fm#C7ujko`L0D7`Me&AW>+N1|&-pb{ylBmh zas#gPYl_9wY$hNL4Mb6oe|CC4ARq@ zv8CfcY3@f`>(l6Er?*DQ=&V>U@3=(oJ#7Ffz`tqe@5?U zZNbx3G;x6$OdUo^7ZpSXPM^;VcrQ%stV$UBbOZG*ql zT`wMU=+gdqqQii+riq7tLyo!}xn6gJJh}fWal}KQ6Lu72sdn56i2-pfc-k8G-&Za5 zQ|8lezYp)zj83x%+V+K&m641bvi}Yw1(#iRMvHVnhk$^9Ipenx_~=j?80|0hyJ|DW z!WGNFEBdidE8W9d6PJ{?3Je?viq-JJk#{+1Jd^ZO5{!pU-Rl59} zqdpP&iVyI!67u^CP|7!;ahI|RKwY|=^Z>gs=a(pQFJXTNc#R)GOWH9@eA1-w+*`l}&a* z(?9bpqxA9V8bf>wn*KL-C^;SQZRrY;^a@K_T6zzsLJ@QbWpAzt3Af9Q)>Qdy&d0S( zTSskzZ-gd5>#3IOtz!pM-(#HYv{ch9!MkQOuN-7o)GRt0{|W6nE{9lvbNtyTSR=a7v53&+Xz) zp{DD#v)bI)&TFY$L{g9kdQ`xlji+r7<6#!(t4AW5!^+yGab=VOq~gTaJYd}1wXRFM z)+Lpkw)zVhoT7{d&v$2UlYeD>7iv;3jD2rVl@F!Yx=39w{vBx@I%|(&ghMGM8B} zlNB1YcQY<|mO}VikWv|RJ+)L;SLPmX;9^ zHizqH-E4zWIWL*NA}ZP~CU%fA@;YL5&c2bv>Or5n&My$=EFkbH&EYg4H%{H9e26L; zM6}38z(ioccmUsB6(#oY-7|b?#qE=?OBh{{%#zoEbt;>k44qurp4}mQa6Sem^QcEZ zk+cGu&B{{cCudWIo#NUW{+8!gq3-9tE!%+G`6P#jXQx;wOk)-@3w{Xetfwb2-)V<< z-|qrA(ap{}x6k(%(RaKcG+=!DIGA7set29Q4V^0>MvKR1&&bnoa{$J*3xQSz3+qt) z5kgQSrreE^mYU9EKbj-w(Iq-Ms*NZF56SdTW_NeDclHx-M`|#I&T?dm@8;i_V;Uu4 zSoy66*>ID%HqmD@jOXKp`*C(F-S`>7BeP<2H_fxRe+FcU|3Xo80;o;7XQxM;8W1z3 z?}w75pvGJLlHtf2NJn}@@A0&>=>2cE6sPBN^WQ-fQddI-WkZX@{;jLxw7I1 z`%>wv6;bw`1CAOYqNVF`n4plawHEP-D_aa`QrwN$(AwJCSs98kD6B8qV`wT1#P4uE zB$GnZ(Um8L4(~&4KC@Pt{rfse%y1e-TqbZoS}O24aP7;(&xcZGyV*kC@vZCO!+r1S zRRo6rb9UtG^{sZKBo+2$_VMwkB*c@Wwg&3mIy7_ZKj+xj*|KH6+S|?JQA$XSXDc?& z`;ew18@ z0S}AKM!dh;o!<_gA%863{#H#14xkMe-J2e<^+@M~iGOe&XLm$ENbL-yj70x_!2yFi z75xs|u783WUpH!JjA51))<-!53C1iqc*?bX-;Fw|d=L?2VM@Yhp`ii89k%N$#ex8jInl& zoq7LQP$Iw&PUS?%4>qbOq8BYdP^Zxw2HsQOC4&i?gr0>7L>wDk=g=Q|v-Idy^kW13 zqWdVA4YI(Pl%G7qNep%+CX$Skd4@|qqJSkH!=z}{FsWBgo1S0$u{z%80|>`c@icUb zXR~7Heii9q4F$M>AmMy9y5GOsC--kq1BTU|jn&=Q^e79%aKkm*gDC(}XiVv2(_}_K zdmkJ9CC}ZJ-C@Hwmxp_&P~rR{^P?ji#H#8{RC#-}lK`GdOT}}Q1NhK^aM6OQ3Q!Wa z20A!6OlC39rtsS7V?hScZzDkXz{Azm^#^X0dHapoM0-Drfj33-Ba5w%-OUEny38;c z={)Nl0{q*XVxs{NPB86p%_B}4^)EerC@#idNHTy~mTe`quCBIR2GapAdaEOXG%OeS zo}S~Hgq18F1#A$vNOT|;929v8^>&Nl!u~$cnEM6;u(TXNdyWR6NgpP92^VeX@^~vU z#a9-9V+ExwQb2}**_D`sbCpjpCde^ZR*VH$^3NwIrh;x4t#xV? zo#hv*VyKZp#m1>gs_rEw06+44Ojr^;UmNkFrbXC<;*9n><#_hBmTIoXILmac3Y>40 zAgo#{|!S@-)<(aj*?kcpE;PpFx(=~HsdhXToPwG8p30r3z*!yb~G zhOi5=cHY)Qh(E2J!0lxX!JQzdsPVK>#AGXr97g^o#U?tP7+AWWo_YMRS##mE$}mw^ zX&%`J+ZijFn4Vt5y*@jGt)c`lVMhwV%czmcFv+>NS-X2%2m4`gYcYuPnjeEk$xkF{ z4sIaGz05B9LFr9TKC7@1l5m^BG~K|BiYPkl{V!@_JLpjYhnu%ZAJF} zHvy{C9S6&wH)|R{GEmX8AZ3)ipZt_wb1cZ8JaB|ArpHH%gK+_BCSIehs8FGz+KY2i zcDQohx|ffovD~3BFyXLaOvqI7P1jYG#JJh1M&v6g>9EhKR+8Iac9)5<0%n$$y?1$m z53w{_bLx zK&36?2a`a3&3&Jy3Hxa}rciy{mgYZ2SQ;pDQxmKgJU>?4$?>D5>7`+Jow!*p@s+fM zNak%$Z};}5`QIjt?UE0pfm~r%Tc6Ud9+j=O2>`|ku0v}`JRKAY^zeo;YY@`mS7zVR z+y!iCNs!^s#fD+j>W%}z73@9;h%+j{-?9h#lbDq9xjpZEzKT&+H~aZoOu6nTh`6)1 zaq{4CNd72nFX!hNcfb0hO$zyA2SC|ymT^ga5xlDIoDUnkj2uvhRh63t6qN235eK*q*TtFQl3LcJJ>l#1k4~i07Ilxks zju90B9S_QEO?eDnbH1D6!V!eK|}tFO%i|fK)d$bCNWWf4!x& zzW!$9v=a^`lP5G`|2i1o(d`*B0%C8C+SbH~{J_kC>6dOfoeB;K zm9+h|u{mDvyP^M?KZ+fE0Z>9VqZKyY6J0xv_XOgQPk)f(m)w!^l<;-)s~34e-?eL z`*_qzrZGRlB>sCHv)&UHsR~8YOl-T`n%?iG`JHhy4CJ7Q=GWg2`U7F!T&#W9=DB!( zlqo*qrvC3G;8_r0mllTaYj@W*k@xiQC?MN3vl?JOS8h?KE3sms#I3AZ?JrWxN)J)? zj*1|Y5Yqy9bnBoxySK0IAe4=nk&Ob{ECizHO7l?Lx8@g#5TW-^R*uNS(NZ}jQ`ytObv5BGgHlu z=lG)3qLNB8)91GrY{ZKO`X|9gDV}kSHsSn&RwM=rQ*x>Io7yXmjgRlgvmw#tx(@gL zK=Bi$ZbQHoPY5?NGhR&Vrf)-6Lt61;j{c#z_IAMJsa}IhY8oQFjEGE$#ysF` z_VqcE-RgO@DJM9!QROt^6)E3wN)<0$S!O$UNBN-L`;@3%ytA0kYuN$Iem0xF`|=TYf(va&Lr zJe1Sq%wOOitI`^lTlZ3&?nZDnFl0D&!Y~nLG`MOyMufdA)?u#1(Zy!0*b$K+!7R%d zBnk4Qtg7VX!_iTDf~NfR!^T`+j|02$iHWET5wjry7N#V>EXnDMR9!DM--E%Ei(;$P z&f=b$yUz#z9x8CN2eCcIuX$Z%l_dp-y?;H{-Ps)6-*{R3xjO24JmoiY5qKU1?(P() zYamg}ENil=*%oCYk|t9n7+|njgqDxbWYOUqjm(~-4MX?x`~EIO>B+s6#`HGr)o*Q4 znXbi($&7cRuBf(JimQ)uY4({KT5+_&0hl#!B+~-k)UopRXvu;34ESgUB`uJr zcaAJ_(bYqVEVN6~=2M*8^_4FAHVofn0o=F=42j*^n>-*p;KQlvhj*+uLL_8WQQO}r zvDZ*&L(1f&WlS98npy^OG`|BqvAqVJz8z~*GHgm;y&1GCzn1tc+Sn$;!yP=-M zKKG;bEdw0aTb^KN=jj`{qI7ElKimoq?KT>4y!$shTU1kVbldFwi_G)+&u(%#eeNYR z?dX#~W@hS3ur+7yFS4f8Tcpp4@msmB zR+=sU8-Fxu2LOcRpdccCv$bgiPEC0t$;STN>-R&^#g~>$hAgQt7RiT2V{K|eMtU46 zioe_u)*k-#(9|NF-)vh8x-A68#uAx#J0HsdJNQA?^Q!VyOlCG(7~*e4qy`C!-A!om zKocCyRMl(X)s}BwO#|ok`^z0<3WdIt$IbK2zO{A#&ugvUHt~rVAgrzM#_A#K4T_DT1OV8JPC(TNVS(a|SleF_0ymZ0| zx2$imZW@|vbdl4OGWhr%Sy%n>=0encf4@^#ZfI1qfb?kED!(3!J-sRI8!pSs%a>!> zo0@XdG}R`@?RjuT04#(=L`0Sr7Lgf`dvhL>Ee@^uED2HdYin!4?W=CA{wDxRp)@6> z#M*DxR{RdUY~ZH&`ArweeHNQj-3WJ%b8;bxKU$1z2sKG%=Bp~osMi!DS{f@%p3WNN z2aF_t6kff zPo;Hz=_tzlB2X>)YiVpu>bg%Z>Db%%Duv=fbb|1Vm~(5xL^>l~ zcY#fu6FauLRxOXC=^McbF`2n37pRCu4T|G6P&6Nk+51D)FGvEFCTzlMBk(QV+}ga+ zDl0RGmG#Q04O-lUeHvTo)G;9tFY@OrRhRQ5m4W>|J)?^9^#pUAL`lLNi^ZY(Je-h2 z-QLZ~KAOB?gw%dW2(qfn*FRQMukD!Xp7weLG9j5nNNQtN8tLyX$(YzR)vxwSPz3;S zIiHB_Mw?Qb9^Qs5yU_=6!0FT)knsh1L`m ztI;JJlt;Rk-xT}b-8MM*eP0J}{=9D}{OIe4P7($8bH_>H>GaSs=lj9VhxI@)FF~k? z;W82!$`E>YVtE}Z+W=|iYjLB=PV9{w>%CuHe#xy z?`BTM*qkOuZEcm9(?CCBsBtozZ;-@|Vp(rTPIH$Ps0!2^Dv17clg<_Tz z3{QR3h`Da>>*Lxddv*ifnmY^iZ}XaVCo}ra(7N3BesB`JNM!y?fu%KkNlKiLN>3N- zZ5<7-(>V%Ks~H`e8s3382@@v~#)*jubNq383;EDw9R)?|kP~4*oKc2}io5ZzUqZZQ zKbHdUJ{JQgM<{bPqzA31t9mcp^!0o0-d(p>iw<#;o0f=i39j}t;(LoBM|EN;tizj| zn-T7$mN^0a5hKvAIXN~wR?_TjEViJx_yC`REnQ;J==z_NrL^XpoHTl>rJkD%>RJgO z8@oq?do7lB_P+^@lQG(I?uD6?rzuf#(#`aFJ3B@{ty>r!EgxoVB0@smEiX}$sR@RY zj&G5Jsi_}tEvnDQBLg>(`QS+kG|>rxs6kIW!~|1SET&Uc!NPONsj1c51o*IY^y_7- zWmL0loa_%*_1^9;FO}!%3tZ`s!OY-EdM8n5E;bL7azXpyh)c^VHy>8r#1osx*al6?9zQ!e@}da6$>Znybw^0=zN$H`^NBZjN3z)qI; zd!c+m{mKx6s&#&}oToe?PtU4Ny`7|r-#A{!*Rwp*>KuWK<#-(i zI!KQ^qkb?_gT&I#$iUk8G}rej(-d9LKXk94$jee6yO(RGz6 z4`0Z9e}f=NL=n=WLx%2=ig4lN>R|0Leng2LWC{yfDP>nQDM;@YwcWRezl?_0qX@hj zSsC5W7K$V2)GBaho4=!lYmrb5yIz%Xt!`;(DP{mRM}u|sgQ@e*adPHsyXG7Y%EU~_ z%zZvKod;$>f(RLSJ;A6%i=R>HnlYjY()?e8gM*>O-n{J0H6{VTWUjRIa3n3#bK7Co z(dR$Rk_L!YG2pOPG)LH}m`fp;C1g0oEZFmSQ)BiudwQ^C(s30^%wW^wobDdJxAFgisJ*)D-3beDLB5y*OHGvl{$Gj12a>35J&IDOWBx zPCghpbku~Fv@G4Sbx#G?Hb>C2v9@lT<#G3b$saka_FK-!MW<%GulR+AjBFTaL8fhg zM|>_cHpmLPNA$j)nPyozD+Wk^em$P+g6VbiNimx=tW*iwlqIOAh2#R;>=F!>zzjrg{E3r=62{Y#ux!yvz%N9Q)4A z&_ht#^8afAP>8n_m8rv1yzh={wZbVOPYH@;e07k3o-GGyaC)x9E^N!h7S`tIChv0M z$6k6diFLS%f9I1qwZNyxC%@Cuc&s=|qmvuK_R-))uJnE_3pBN`UTimB+euf()!`!| zK0cuTl9#`(G}W;+<8(S+VoQD!1j6R;|B|0W4<&ePZC+XrU&#*LoTr^9q$3a{(Y&wS z@?E*PsvizFpFRO@Vho!8fnGeX?aQCwzAHrarm~Cy6@obLi!DbCKBR){x-2!T099J(8u71%Mnr z^W`*iIc8&F#y#r;)Ys_u^W{UwMx?>|m#VnN#}}*X`JgN^kbk}mN+|X~=P0kX_XX9Z zvZxve&#FtnTiJJEL}Od|R*9gDOnm&zm^c~HrFVZpR3Y@RshDr`O&Z8od>eCmRQnkF ziGhVmigN+et*+Nwpz4Qd>>y#=bN9W~u~A1jI-bzu6}UZ$kFT*7bEFn-rf1B68ABEx zigM+MW@%<_?Rs4Oj3a~|NB!3^IR(r4eoJPN8P%LRH0SY<=OrIRzZG(|;Az55)P$15 z7F6yF*-zbYY`M#)-NfYdzH{M>f z|3J&F^{7fCYVP zYlR~pld|-MhB=FNRgH~292{TSu3Pnq^&^|vjax4aV-E#pSE&;6xI< zXIDi=MeVzSce{_GWHRv2}E;EMoLj+}vzXKp2y711_vEYLF|j6P2W`mHDI#SGPYF zNu~siQ%gJ;WSma9k)C5xkRmGog%_r*@OXcB^(0JiGH&)gFVWrMkx6PYrdIEokSI78 zcyZg)1Z^lj%4WAaQds`?`$UKS*0%qx@e5i4);~rq&acaj3Eun=#d{}yQI*!I(S@}nSI(>so8NI%5 z+J=T8En72vwXCG(Yj@-J$8P57Dp&Ywf_=^Dcc#!j9VEya?atx;lrmCD{fv!x=?y&+ zw2Qfhidg=)1VM60vEL-9&zl0iL|Qr?S4*UE$5_d!NjB~)o;=loig!&yiDmJaehEp! zl&g1T7*%G5N=lwCTW1Z|W689jh`b>6vJ1XHQG>(Edq6wPtRRU8ACGVyTtX#}{1c&R zSvDBKaI%+!*~02t)@2TM`VBWhGA?Fj#l~hYD+3Mm$dJOG;GcGpLqk8oYR0NnY*Im= zo0W}?2uQ-JN)8*8X&7ng2yPD>V<3TKXKSe@@K1GgG}2_KZd_COU0OmD28Ku>h8YAE z7eB98RSyX0wxCZzJs~mrPf8q)!L@p2#N2+M`Fyp9^)(020gngk-vN;8!$pfbxjX}H zh!wlM_CpD}c(3QYaT&_-NQ4@lb8vwFnu$odKYcg?>D5-ZuKN9&Wo`G7E<$L4e{wRG zzszDQa~ul`%jn|=C|7anDatVkT`i=NCfR+Y(Xo*>La9`fj64o#Z*N~;QzMtXm$al% z%2~~Z%+~zPzSzizlh9Zc_K%n>3Az3g@0r(E^3v<@(o1GG<5|^VUj~wgzK+k4Csu($ z%J}BWN?S3tfFY@LaUL6qvBIOm%M%CC_*QB;^G4n49v?w$;GkXTMQox*_J!oco@5grXs57rEtk?mm zY@Ju1KAyHgc;X)`NXZIfXr`FIb|QhirBIZVlwx93R_pMPIre7o2ML{z3(d7qs>{Cv z05lwlqW>n`Vu=c~n>$VUFruR$f`>G%Xab}oJSaKB(BKZ(|0(EESZ7`~N=S>68|Ksr z(lZKF7B&6TujM|aA+3^As#!N)GjO;6T))Hdk_IcmVM4{WKyQbHqlUu+M}bbU-C8kv z7!o$jarVb2-viQ>NGB6Tp$oz2kqmvjJ*5#<+gkITO`$wv`xOOrD|rIrAtb@$@xJgX znX&Qjf*>tqq?%(h*te z*VcWw|71&A7c!b?M-S&$9E!O+?sP*@k^W%lOFxwr&h(i;w%G z<2;Yu!x)sfzU*%uFcBKy;NP3Y^DNBpq}%Pp5CN5NP9;yY-r9QA`!P@}xgzipNh~WXyV3U2m(yx(vEcGK=a25^Z7Tn-XHffzh+_Y1M(qf}@cGM6M&FN{K+o3U64i!Vc-A<@HH0l= zA%%8lOHa-5t+k4pm@5JfZdTd1Ax3_pOx$k!WB(-XfJA%u$Ga!h<$zR1Z45;S<~Fg+ z$C&bMM%^BZfOsQ!Q@JLh&FrUf-A@NeY>?6i$HD#ISuw(|35*F&pAKu?&3xU@#ZI~XM-{EJQ}wPGoX4>>O*AhR*&6Pr+@tm&;(9=OZWZ0vAMp+lcP7bfsi2D zw^8Hvs}+SDFI;qktDW)dsS$8buvndkIy4_oNQPBd=)@lYOzOjvh^7P7{el6=fh%o^ zFeok_Wq#5oh9!}DLYIhA6j!cPWm^PQ)dg`DpVzZB0E%8+l6Qe<>hO8l>Sg3`L? zGCOzuK=2qG0Vn{rSx{D>gl+`$$pwwkJz`#`AhchWMDYhLStUo&lcIxEHKqbSjgz2k z51qQAH}rhFaHs3?l*h5|A6^ob<_rLl;2}YvAh`~`>pcz-@kO2jeZW9~#J*hFAoSDp^t)7tv_<65m^t-QL?mD%?Lyj+f*67G0 zV@%Z3|0s{9MD|eC^iML4(-&rbj`@QG9g=m8&aZB8D`69V2mq%VU;^4y$Y)B|`*8_H zK+D0Zyv)|(<2AKc252Cx{#8Bzia?8JNk9ySoT#GMoX58|`BA`{9X(l@5|$NK{3T=X zxLfj?_@L7J@*of*Q?ez~2Z)nk210+g2|#C+iYDi9EF^0Kii-Bku7OienorMBH4jN4 z8o=Xf`rrNlPf75J6J1-0R8|gWuABi=%mSRM733-Z3%8xhE;3ctd|iNuOIf1{&XCMI zyQ26MxafXLB^*g+c-|yXUqs%D2Y_*-#*pm7qJS4svm{W57t#HcWGQqveY`;F^zF-O zyc@W>_=M_|FI?q&+mf)|@?;TrA`^zzV%d!^|`@ENt&;XO#F{K&A=^R_fQQ==(Axr}&&1||GQ0nu#A6R?QYga=d zPLsQ$P1eyqRZPrixN@gWAO4ejAe!gf%KJ{p;n!nrZ*R&kg25mpPg~Rs@+|!%VN=be zpecib&yotl*Xqb&+Zk>tl6~OK0?bb0u>~W0fEUG7JPb*(x?*V#-u-T#Lr{{W{vO+r z-fsHpo$t%RG(Z3I*{Qt1m>sCQNw925h1iE3KjKmJ%84PtQe9?QV2&Va+CTT%6H1kv zzhZnqHO+xpDTP9xf54-m;cCuZ{O>mWzitECRWXwqb~Th(38?Ec3pw#|AR!;kZJb$v zXTR-o&w}F65#UXJC`yYeFl_mi|N2v6t=VD&e`J))ryI>IzQ3l(J%O`Zg&fqQI*Nw| zQ&|*iH|)Y8KULjKflnd*BfxVKHW~JI z9P}ZUO3CDUi|uOD?&Mq9fm?i49bks`(2-K9YB;O#{Rws>a@bBN%%Y$KM;O; zIe%}Bv^O@;-Xe^C=Dyl7;UQ00gR@bWozG_JscWG9Ve@_S{q(Xj#=1ECODzqGxE?~D z!LVeW=$F}Y1m)neS}2$;YlLd9eu{+MP`k9t&BkMJz{_Glv#>OC(qjjO+&`S0Sr1lg zBL?gjyX35)RFPtH=!CE&I1VgIDL z@#A%bmP>1=sxf~Z6ICF*_8)qK^z;|fx6)Kfibj+`I2*~+RHB(q|C3gk=#Ja#BK%*c zJwnp9S%KLvQZ0#eagI?8OJAEZmDMMo^#7TK>rrCQ@#LdL%^;r{moTSNuWcd_s{XoG z04B7!N{Mb0Q3d^!X~a+%+^14;=fG~Z76eiAbas==M0~;OhAR!!vPfh_Q133otRPW* zpV4MpNeT_S65gOR@x$P*~Vc(o!4C!hY?y)j?dg;rwbR#XxP#AH1?z=hkbeo-$aOK zhIZYzu$m53tBO6OQnJn16I78PA97I=`DZB&4X5}nUic4Ki;JbL1uyAXU4QQTG8+TY zBXHPF-Ia+!(`CRb36+*7?Io&r!Do#04T;RhhY0%c44fIH&q^Nt9czYd+E`p&6@jWr zQ~+6DFKJmkWiaUKOmt`HbDd~m!5JAk9-9n?ELf6l)ShpnpKb`b`)Wl{UHkDaroo2^ z#SVEOLrq@kGbHM_t|W{{2U$DzP3Tv}8>P303S{djE4HZzyL1#NXNlQ!VxQIjTnv?d zGiSJXdex(mub8-X=(PL(ao@2Wi4l|6&+bUsjdpbZzR~dx-T0~5<*|PudZFP@%-M2{ z-EHFT;0D>KnM+Se=rWuHl>7uMJ;6sa$cYqAs|)U$w$)^LC+M_2rOXY58m7pVKJ-3X z*MX#EA#D^wEt|Gln8@&uH{7(OfRda;;sVE_nY%8Nl#19j_h5Dr&38SitEU*c9nMZe zJwenue<$DAiFJScN4NnlTMUPvC&mXVnSAH*llx2TW2L<--@i-pWk(ro@60W62AV?@ zC^+=lZL0|6hc_aBf^_v>0R5D?xbcXSmmX2~C&n=fZw_Aku5(`B2!E6Y1pnev!XdSW zNI3JT`9OhSm1|8jDDJ5p#K0rZgSc863}6f<-P?hNTO2R4cB-preUSq zT0@8LL4wcvta9`&L>m-0%`T7K6KuHVSh812ZC3$4acF6wPAv%VTNlpz=Ba3 ziKX+{aPu+c7vkj|a;t5$Z3&>-^>_gL`3+t~%UjCS@KIN8p?#jP5dXV{L*;}V9@bBT z@5QlapDSky05J4{=*_lLS}&Vh%g^=P84*E$l9-E?9OsT{Wof92hM@j-bthuVa%pLE zqK#juBDpp%*Tc(%|Nhk51uPAO-8ChRNY8*GUv}ck4woU?WiM}X&*Gq$>+UN_**$y0Ze^KSepA3SQ%>N<$vm@u|){0O0 z5~-uyx1yCNh8lXAy7gQPRn|8!mD;-F!p=cgXEZUU*<_luGoKS0CXUVh#VO)d6K77f zY3CP-!mo695xGV9Z}c|!hZf9#>3PXOrg>i$kx|Bp)W7qYD^xXnvaH!2!y~}9a-4(? zq(;jp&|OE8^SX_bH5y!mJIB^4k*xokj*XR@>y0XpRm#C$BA~J1Qho1h;i^yqZI6Oc z_>*~pZ6#DnWWT62yBte5r$_LYV9Q~q55LZ&xpZwdKYdf{a1kMs?sgcef4DUDoF%YQ z^6M;7gRGd1M)**zq}dpx46@3SF&=4=aB6}}ToD&Yt#s2Ji`B8K?b(MwuOO}7Ln2kB zi%QbnhuD!Oo`ISbwhhFF2*V}a7u>4~BvYv5t>g6gp?=RE!aCDkTf3yt#o-oH5fT^- zkfi#eOPH~`xE1mmy;4qDp$TU!F)ZQf5Y)4+C)gR%YEBIc2yfTxDs}FXz%5MKL_}$G z+_@*|mkt6brcj#Hx$lh<6Mav)TEi;(mCF;gvbB?>ZMWY z4ZN|xkPkPk9KpU~S0~6}6{eAHSBw8OTYJFN=CR*0D)c62alG@iY`cIDNIN~dSgCTC z@Hv&*d_b%{YF}W%)AhJ`+RMZEP0dx*U7Lr(Dlx!L*$BzKjuNRQ{8C9sbgNDJp^g9jm<=cgM5*veQ;8oXeD%DAfsyS;kd5b;`xNb&`tjz_T_XI zP{;YnNsGgJXaiWDJB@hE+2zoYlQI!+giXz9lX^61%;)0RQql$3GCAqydmj56e0F?p zQ z?z-`3@|~^}KGizJ1W8@AEW6O_&t=EUMX7$0+7P3D8*LJyTH3YLiVuQj#d6)ke)6!Z zYn_yQ7_kHVaNPCsn*U%&wU|cgPY-GnL)ev zeuS=p=Qe!Ybhok3YaS#%f-U@J-AnpS?X0Yxk7jF4^T0a}OkxrY|tSvZo0edvja{|?g*%LgDKasM5p?nA<_Z933t zQ=}D?bhgDVUT_B#fMALUY8|(M)PD~A9B-0Ep&O>G>KzAJ(cOPwKTiJWJ7blZrQ8|qe1OBM%c zb`4zqgEt{`x6!Xh@b;_6=a{}8t7i3S)Z|>non8mgEgdUquBuhr%+3p0gf%Q3 z^z?jE;`bukkuBLn@>ix;45s7TrxEocryyW_c+y(S8DveQ&Bix0Pmhw=v0MiDIl3w= zGjKM>R6fF+{k;119iCiYFaR6}@3QH|V--7N|M|M@`nTd!fit?yEx?H<`rt26U}lMz zDMJpm@~evTH#Eaw*T%-T(@TGuICI6HP56l9)NdSu-xDKaSv}1|oN>{{;;>R#Drc)~ zl7%icJD=y%2VC>#5THy9g|18>-O4Rp5PfcQv8>{sK-X|34csjlHWl@aTx~X;U_6%e zm-p~I4Qh2jDI;L(g)dQY*!-l4b_EOB^%@v*?Uo~p04NaGESqkej?g}84})uS?C$AD zJsg~+qKJw*n!=E+e$2V{>1f=mob3V7%?&Xe=~K5;bi^IV*LrrJ>EccJtVRhqSJ0Te zbfK7jl~aSI!F=)%`$DO6PFrcUqI#h-O#fc5@H#X*Fpr5qlF-BS7l$AI#GRiwYDwE3YcVU^dNa)XW z#bvh+e)`JJyA_V5NAA2UTU_DQTO3<+ZV-?9W6RS|ftljJzP5EmwiAIa17Luw{f{sb zC`x$Mabezn@W%{mjNX6KL?bhX>6iVJ%~1vDi-cR^mrX$PAj0wB5pB-LP}0vbH8QWd zZnpo3x^LFIkQ*$(HxTsno3j~iuWWspnj%xa6I7?IwTIQ%f0Xzuk3R{k4AYCm$S0I@ zW!RK0Q4Lr3I^DF>=db1(0w?0L!-oE5)LPym*@}lY2?bn^FE0upJ_aa}}#x9tsHt|AMoig1!xcQS0eEhTWz%L{+8jU-v-D0@b|Drul z=JFAIf!J`-b~O^aI~ZL7`R~W7yFUMw#gGc>A0hv*f>sGMSw7-BmZW3+dB_sR(#b`V)!B^2`%o_+y`N0oB!WA_tYW zC(LB9Yn|a*3%DXn_B4Wa6^`^_u>N@mH}79) z5`N!0>y4g4QXlaF=?wwzvfUf%Lpvf&sf(lP2j9}xM*FvwUJS6SN5^B&t>vV?`WzO; zAA49!8W_A}Wj<7>bdGpkyfN;$^J80qtC3PlQ&ZtKiQh33DFPqYg8^1Dzc&TE^YeC9 zvpsXqm`R~&M4_SE{9eFPw+9BNmtOn(Ny2vaSaZ!@Dv}Kjn;UgT+7K8>DWZY!trtg~ z;*x;tNRql(WW+uyH?*OI6d4sHRL!d`aSHEf9 z72HE99QCh%`5@`a1gYsrjbn8}c6{>bCZIP{7=)JYxS$6W`NxjQ@RUcJ@~3Oz z5^w>7%dZexa%Fb*ZFTdtmcbz%hT-8Ro}G!wv@I8e(L2+jsh>>0r{cz3a)t^JE|JPN znV|hNZ3@YYefC(p`O|)QtiGmgz9K-;l6;vk&0rdE)m-tgnHDaw?XR&hR2mfd0E-mb zh7Q5m2w&Gu@xDx4t7H^a0Bc^_oc2qkj6l9d%tGM_3sfkV&g%ORY}^Pghkt9lPfo7zHz;5+ z($FI=a~A!0mJ~)Ql8}A)O&`TH!~uxx{pvT4wq-OV>JDfh3I0dK{v}&x<#LE|*EwnT(LbFd0#6nDWHs24jL>EHxwz2t2 zFT--bFxYL69Fm^-e&X95oU*H)Ri`&X-aSs*z?4h~MUgk_9zAgdLcv{6pyXxZQVRsx zdz-$YeEl1OLZ-ZZpD68N7UU-wqDVbweIb>|7~Li+XIBgy65U`4FM#==4C-a#ZD(=H zMEGIkl__pCNbF54Hj&aP4W^u^lyg;&@=dj>s*rtD>>8y@H_#&wuVtjYOW7%th9$PJ z?A6}8h2Qp~tdl-C8lPH3piKOO8A&fN=p*{ry!s6e;sqYQ*JY@%=7TyU#e#CTz-d;i zKjL+{CH70i();rP0YRd`xgpBxAAs6Ga#5yt+lS6&P?CWPdIPg$d`Ob~_}jdi$aQO~ zp}^HE0T*&xl|5h01vhBBU(4y@D+B(rt&8>~qaj}WGcJnK;*(E4@iy1-fu{+*k=;OR zF$io2=o`i6k!7YeWqhb<5j0mzLr|!iD=_LmnJ?89(4k2zpS=u;Y3gxg{DG6`WK%%; zV8ZuYKaeL@&69p}*{8X-G{$!UctY>(8zJ!BwU?u<5hyOHox3@ZY{~4SN>Miu^J&gE zalr>Q$}(|jlzTK^R~X?-HQ5*#GJc!o`S$l2ZouK`IWnqg3aNA}BRs|sn{xV_v3Rk$ za0)U~LpH5mC4=#ZyjZegb2HlwF1vxK0Wj z7V2EQkoxb33#fQ4U~Od^ysbv)k$~y&XDiw9_E6UYulTQnlLPW#*yO)hR*f0ux7 zRs7#4w^xs~Q%Q@6Ki!%*gc0+wtL2LQxP}+Y5x9d33;`$Bz$GfZfnl9;p9Z-nMbe4N zsl%eley{VbB&SNj#iOUANX=<J~V3LFLu zh7PfbaPfJVplRPx6f5)ReEuA~Wx;PsPmkXgeCrz}9%V~(n==U7BrrxQGkiuC%i*5! zjb5}=FlXKi z@ybBL=wp(E&r9L`Opm#1qvN+cE|VH)=D*U+=gz&*!CbQHv!d@@CK#NRO(w(T&Q*me(3^ebgf;RExao= z5@tPXONs&*p^^o_%~W4FFLMP|>IdwN{+ZDo)ZvXD;Zo}`{P9DHYHEglq*^_6M#WIT z2ERzzIl`=DkFU6ka*VfRJ|g>*iZPzEh@yMH>invMuE8DaMQ<@q1^#v22HzO@FaF5S zueQKvqJLurdi+G|wv;Rq1$RZ6my&A5`)?3y^!vAyc8@RW3e9zXf3oJ{FZXOHFEY$> z3o6>%N7}SiPmWI-ujg)%qg{BjR7VwIZOC@H9-`!SJt&R`=T`?4%)Hr#@k+cH7vKHO zzuViQuc^6FE$>niCcV0@rBP0-*NZ5EN)Yi#9(v?2e!8fp2XGPrfir zyg^)boNqUmaGkj=b=QaW3%zyaE3@|i^vB%AZWA1xSWU)D^hR>je5aIzCv)Eh^S1oK zLzZtv4Z5_w>icM$WfLz7#0;D{rgkZ+3Msg%gFPxZTz_{(v~S28KAFN{<3z1G@-S;_#QIz+*~o zw}29@9OA%{ybf=4-(|kMv>ufCKu}rw7@S6N3zQ)g11EYiJA*n$VIF+Z7)sQ?7j)AQ z3j&MOF)q?Nhqwx5bNhm}_@~x)@%99E%f%ay8>!*`j&Yg1l&*Ha<6|CXOQx_}u>xPU zqw$&FrQ2C6SY3jl?VR8Dz zV*JD^2R2&#&p)8o!qQ6TSH*H3v;x{ApX#+y$r{v+vIRU`pBvX^T_|M+GMs<2A%B_X znIiHKkn^to;`&#vpd`!|`A%s@vxkjpXN;z4C%W$D>3q=LH3F{diCFiW#gWyx44DA@ z$p#zXQRwcupEWoM7a|Np%FP<4rsW5K!q6RsHWG$I16H@SI;kqoJe_-8RG*u>iuHu0 zE*<+D(-It618dI(au?xGdfi4C%{en|g~uILs`TA-)GWREYN>w(3#g-Fo_h>w7E}$n z<1>GkKC}~uus8Q<)ajTdml^h?dQUXL^jXHp61N;R*d7?a{=0NF=BLkDdDZr-O9V$e zwrMga>8P~paahO1S2m`6Rbm4r$N4SOEPv0D?zNAD3AtkV#7VdRhb1vP(v6q-#8*=6 zh%qPJCp^=d^y9OycDG^TvJ!xcJDuy*<25#0eOHfXdk$2nCvU^G&pRPtsC+ABp7jaw zt4_30`LC0`{yob9o#*f4e%9$USLDfC!{s({1X4N&IZZ=Kf&myz#4+np1@G#UjAB{{L z`@1rBF@?HD^=oF=f36|od-ay6$AQ?w><7rQ44$wlwBjU*FvcMA3)nVMkCMB27?KE- zOPu^Ki$9Xd)3j(^{`w?0NO#KNLOn@av8rJ*IV9 z1KNQf5e0naNn2bA)qP~zDGxujunIX1Y=)^n7#7)klWYdnK~c^e;n^z@hPmBDr$n43 z5{avb8V;2aorxulH!bQ3CMI}AU#PX#Eqi$R%jY@oO7jYx^l#ckS<0v$N*)!;I4c)w zikQZXOismv1aSvFuVX1Y5fbLl(y|X{S<~XYI{FyQ?@=m>srBm>M~-Lb*(W1K{%9Dk z%c&6;-8;a2Ye7^-S%hqfeGXaA2w;wAgA-(3(Ai99u)GaXYu~gvPHLcabvuxHh z^)Uo9Z7gS5KFVzBF&PU*54mG4*;paq95dF4*013Hz#^Ir94W5K5a1=d&1CdJH1?6Fv?Bu@IAoCvBJ?GSz-(~y*U z!Vu13vIiZu5WT%DHg#^zSssX;5MW-c-0VDmmeh#H-B=d#TIVE2O}C4>(`>;S;dArL z;iEABJJ!QrfGG@DuHa9DY_ZQt`QCIRuQ~>r)sgQLlD{%JL7SqX5=~Yu5;@%6@!!ES zUKT)Pg~w1(q619`vX#&p>vvF*1i z$W79DIQa`;O2MvR&Zu;-nr0|6?~+~|(*}$$qN`DGLoPBfy=KR=eE@z`Do3RGvnlg8 zs;3`}3a0t$s*e=RtZ{97y&y=+lozLQx0h>JWM+LcWcvKKcA@_Gz^JAa>j-gwI zh}VRYeoSNJl;POy@e5NIBK%a(^aE=LLb>_=h?*7wG2|4}qm*F6BPV|?6Nbh^2o#IP zLBXR32miy5WejZc*Hk}GFf&RJHEdKygBL9eR6!M8f{iz2SC?yuUpBTtMXy(dhl+I~ zla|&+vi)o^(Uw6j1}veRrj-2)d4NO4ruxqVVZ17$VRGX!3RYeO28ZO{s}2^SDn&h- zG49}0pBc-QpOhqXeU~x1Y{r4X*;kHM99cf9{7CPpRYqb%_j5NXD&9v zh*{SbUonYLIX_7vWpN`~=pIb$ec96(ocs|g^(o|Zf68mkwNKRDO|%4t`TcKV$Z`pF z#;9+oHLb|a&p^QuqT zrf3A*uU!XG_$-+puXiEwNlBQhzG!$9J8e|L$+#o%J(#iI=ihdQv(O(5ypOxG*v}xp z$nqth-@)!@$>W-ksDvOWZUuP%mwEU@WQR{HeV&4AAoOHL9Vr2G{a7N3oTgoPayZzaog z(+1E+64U-l@%Hux3mYtjjg6(A6RAF%w;AwyV=04S&|#FqSg`xlgaklF7p8@*jLchOmh1)34mCHAtHn_+N!kk-J|TciKRFT zH}83-WK-BgFRBrL`;1RsQMgC2#Lfni6NK&;q6S2Ja;eOjJO~gf%sHr}unABum;OJ) zM(0`UIt@KV!HUF2UBV9f9x~+g84goy4tB6N1@xDt83INOhD!}?3G|m^&TprT>jKg? zQEI{4Bn&Dzq#dau&<|kT({t6^M>uYxNZz8Y^Oo zc&sXgMrjpHC=r>NQDo2eorxZ_5S>Q|6#^a#MM+H2ALzVSp_AR0Lq`FN zL?D4NRFMJ%;Pj~N-~dVVH{puhMd}VW?tkKsrJi?#@Adq5?|CK-j^NVVP8ToMU$%PS zZ`pfcFG+NNsG&Mx&#AZ?;P$9NOVG zA5?6O0FubsqI3yzUN7!KRpz$`&n<Xv4ne9AF2eG| zR)rKQel)tii2qS}9*wXv($d-5!&L{O2i-MlWt-DuaIFADCenRV=+7Zf#Q5t|dERef z-jBI|BKJOuop(2-XjoS_0+&xOAJ+jkI#p*7H18#;FGScQjidN>=D71^l`KnW_2xkL zx8K#-Q{(IU=KFn4l0lPa1ECpYP@bC`l@fPKhz9;2Z~sKz`qkR~{8-vwTi}CE8|?jz z9v0A14><23Mb9^I+M({m0q1^^#pQad$8F2%;iNr|qW<}X;LURoDn7HBvdC<72eLgo z1xTt?Y8Lhz_6LJvn#PII#DfO4YEArG3F${Mx%phhDHW8mbm%b|PJx>&B!!X$*1oTB z{ro#6@qdov|8*`G2OcEW*0;HHXnlqL33IqYJ^+f&!5b?<7U-TMUw=}Xeb>!w=CT;sE(-(Ji{d4cjV5O$HJeRG?R9v_kWdxrWFoZxn zyOJlfaon`l>bqQ1Ag~CFq~r)Ay2Yebb)_3{9~GmRv9$TBt>&liex$Qkb~smpAuRa* zX-SV{44G%6+WG!Ud2_W=xCkT!t4$`|m2Waf1J98B)Yn?nQbD3!YccN8`J%s>gduWi zWG;%!r;2104nMY2`wa3vnJZDd%%s|nJq?CZQ+893Jzr}+hfvx#gzgj%6QUt4Kq2sB zjG^a2XBOWksIRfS$?r(#l9=T~MYYz$)^7`O)9236{~^lPqI-RC^v z(|U#<^QibbzMh&0909kZhD`e2r$C__yPheP%ukH{P3|OOIP(9nG0`rZQ{){yEa4Di z?C!I=&-GJ7%OvXf6KzgM$A)S9j8jf+UIMQ8nOX*bBH?VRzT@J8P0_M=x{&a1&_+dr zi{)T3bMKGTRMb8g#Mrt_k=yrhQ`2O>3zOBgr&3=~#Sd@M0B>Cy4A z3LE`dfhr2DC!QNRI(jR2`O{wCDufg@^3|^;uhXfUNua^5!*zQt8HGI9Z@v6bSDLl6 zGvM(u;Qpw5G|?TSPBCDWrYSs3Fih>pvOS)i){kIOSEZhr2%?5mrNGplTJfy&-%EQ353T8|}*;==2p!eW%G()FY$6 zxRw3de=emX{|o}zVsqvE*Mz-`D}y630|+wcYO|{U7w=192+c z43`})n-z9{lGftp)!VD}gGu0=7Hpg@hUeFN4PRCkGFpYk&Dc|l3_XQ1z%47xvG>O0 zeR5{uTx7UEWD8nLZ_k!%MLc(YGmS$evpsg4WqzZQPePSoN*Xcp$%vS&eKwIdIHD1V zWUDlK%;&=_lRMq7@VRVngwPvo5uoHwLf2oepplT`v5T?eBYVbDI4m_XAU%~*8zvWx zHu-aXpG79CVkP~D;5IF$uHar%K0{SEx7mdadm>#0rTZg z^83RXMfP!qIrKDRQ%5B$5Yf;!r0j)Y2bbQ-kLv>64Uu-c!_1=hgG&+qn?s%reyd*H zhb{A~HmN@yAHjnH6Kx;hpOv4k&o1BRDRsu>!_vJU9uF-iJTVyvq}s9n8M|*}Nf>_ndnfCcY)0zQUg7vEnOy0PvD^A4!!Q$ZAHLU`1qHt4(pT;>)y z0=!r#uV(U*#Xyi0fz3FcTdF8qII1dkwK98|notbQqxgRuh)`rBmgctZ%zHl2-orX3 z2M^Tt*;9oIYE`q(bID8l9Hn!1d{bJJDCIdROhP~dS-f9S+py+gr~&4?cR%T?1g{r& zl}uWsf-b433w&+aDdNI^H)mlwsSW&8&8e8hvCc+bwEX|C(vSyx3ZRcU6d(^nv(}B4 zDsYFU{x6`5>NR)6QzPsbw31!>INd-bThKR{uXc99k96pQJ;EPjwOn|jvN)LC{R<7Z z=Sm_qKC2y3ryBzPwpz~qiZf9^5(V)?Z7+Z!2KwvQlrohD?fi+EB7%ZzNVGeC%1s#b59vNKKr!RZl}27$As3@KoXb zU=9>vNH9uUbln`jixK9F74GKjc}7HEsv(#| zYW6alE#f_>O<+yp$w`{}a52yq0qfUSIG?xY`)#&X+kT<(!`8@W>yh>9*Tnbtjp;}- z2Nwj^>t07-N^mp8yVHHuZh(NIT94&W`g{0Rr*599GPLuu%$Ff*CKL>5E;QKm+;duw;svxZfB z-5Rs+tMgdJfyqN=p}!R~1)s(dJlZFBE(|)s?*Kb|>O%~Rr&|Wfd~?@A6Miv4%kH>+k8*5>g@GRrEQaitDVCX7 z<67k(XSOeth5?=rnH$mIh0rHdBEhdq=}Xa>IRcBe9S^EB&dX<7g>TNpB_N&Pv7s=b z=k8-6A5zls>|43P)cS`F2(l*V&|p+nt~<6ct}nf(>4QPIk0yCptHX79pFwkAphjvk zPlGDEL%7{!*)Ho78g7SZ;tY9}86h4O+*jN{XsjK%7Mf|-Z${TAJhsaV9^kpB1EtrN zfh>_Hb?`}urlGenj$1S=0D$b>T1+1+#yg12r^=;Ng^d;eD{Ohq{geW5-sTM8niHg` zA1O||pPTfLZ}&ci+||BJ*Y)t87s`UdJkjATp(5HYYgmVLwAiVujE`yx2u<9Q|U|c;o@CH68|eD z2C@*TMG}=cf+QfOrR@-O;=H@LK}1?NO7+OUm=E%;ud-U&^YnxdJme-!Vo*g&)JPYH zLZWMI>`3@OAeWo4uF{{W?RJA;_&@fzv85R#EbTWljsuz_BtpK9!|5^|FOM-?9Nv#h zK9-O^=SnF{fZxNyk2G$xUmU%`S@n9gd*SynWCATatF1hC>rUVVN=Pt%I&OThU2Z5n z?@B9E27;)ES_fm6tAijkAG^(J8C#304`)p!7MOaNP;WUaTidm2yVD*iRQ3AS+P8j9 z1HuWjgp+7ef6ME$-<=T}YIVL6^!|3BCPu2n+*A4Fd0 zWuKdxf<3y5-}$b|!R=w8{MJ&$v%cuGA3syX2Mkmu`8t+_MqZX{RLU6Esvvi@A9(0x zYHI4VIf+z+a5$?UR9($B$v}DxqoL^cqPw|NZ*4cCwK;R&9dv7{8%-j#5^$cXGKnoQ zOl^@3BBC9_WDpJd%wPd8AFVDqwYV7*^ivrX2!uh@B~txshO@^Txs@Xxj$8(pRG8;4 zH2E^|Kq};M;A&~f6EDt1+W==cwHQDGyC~`F2-~F$T@u7Et_Br|YhM#j>#=9o1leWe znE#YP6Fzs-GLz}n7bk+)icOBSAm#M~JAuV={o54=mQVVp^%_!=;y3&XSs_OAT)?Qi z2AG2(>u;gsJ7$v=7vaNN{}nMfh@|4DK!DP);{CA{oeuqaZ>}v_ zYIw)yJX46pUIgs&>*nX29IHu*7=^9-aLTO;&X#14cR@~GvVo6Jn~r2~0slgE5UAwj zGC|%{WhOaenIQhrPt;oZP|>zRIJU!3V^mZ&xP7cAtvn%rXTEH80E_r=RzhjfVkrax zH4!f7Wmmso+=xTO#Nw&hP8h1?|7FKg@yW44r~{ZGmf`TBUzrWFRAIQQ$`ldBA`u$s zr_-)Ox#&@}*e77Z{zQC5{l8cM0PO(0q`w7}Xn&tDjtMGkT>OFTcBzcyFUDAuw(rU0 z-;(*Y%&n}7oxtheeSJ3l9x??yD$*Ov$Uy{9z%`diA+ms1>sQMxRX&c>!5AU$LBW@U zQWnc5>xmxfRgZJ3G{5K0*U_mfS%mvU!BB^}IEygZzYS(zEW3x0u?j5Tp7E8ySt9oX z#149g@N#QLmBc$L<@k!t4npHFRq^0qBmyeBYPc*Ycm_fFfjAj@1j0~JH3R;)#ZG@n zDdJARfy7KD&p%n5ev~e+K?OmU+SEdq3ytjr`wkkgZny?v^Z887<(_{)namAa+1Y%jjE^7zP^y{5=b}*@ z<|g-F^}6n{exXtgXx`NsAcbKuo20Wqf&pjE8WyTAR)YDWhC%D~ax3^RM#Q@`rgbVZ$ls`4T)|5X=aXLmt&x0I@n4`Q z3>*z7kSIi8lbT5zqu$P{L@jh)803eT7GUF_K)W zmPW)UT@QS6T8_Q8(}L{^f0us(<~eN`1<@bJ=+Xs97wYF0*Y!RpT0uW`9@IOX&< z5g-&GHp5pQe%I#nSIh&m75>)e%C{R!krtQ!bI7@qqG{=FyV-0$`*wTC33hesxPJbL z&*Da4{*SxW;gHuU*9bW~{ZaMJor4c;j7X<4mvL{I4BW|XCO?tHEOFzDq{Td)$j z$sr_Ogd@A|}=$SIA{>7MsnB^d(l!j2y& z{kKG0FAg2|;O);@NL~Vsb{i1bSle=~cMA&{-{ZpZ0SC-v@*L#Lmm7B4`>Xx?YJF~H zXK!x+0ddU~$a5tLdF`x2+;2+iA-VU<)1*KDl-EVqJHDOKpe8yS3h)1u3w*NL$dlsw zJ^SsqJNZn#Pxj9uVb`K;D9h) z0+OCvQ$&@DBPvyKXri8tCr zs8FeXA8)%fF#%9lQ(f8A806HZKje%epQZieEsk$iL!Sjce7C}~G8^qy8wNW*GCnRn zIy>1<*4+n5C>MP;{P$xzcszPT(M4XjcLN@K$rKHb;IpgXh$EHRC$w9Ow6Q;W|6R7f zY#>7Qe7UuH-^=sB=*TQ+>0ldCJ(*~|9R8%|cd^go5`RKC6s__W-0I}wr~G=U$Ks1 z*uJ*nd%)P&;`sZCk&sKr^_!0Pe^sHz%p{3~u!|j=e$xtGCL$)$&h*XZi!o4*; z><+xdkoh0(#t35;bS!Q@LY%c)aYrv(JaZtOfcc_mGJfaB?3k9?q{?#rc2(%Wgvf|n z3Mv?cM{h&T(-00*E290v7Cw1y+cn?2s~{y2-+-`#PoMM+A0RMZEhpW}{-7Tiwog~> z5;B`!rtw6KC)vl({H$iKR#insGOz{wcqY_g2-m%`Qr+YnQ=9AF?{{uGa{+#j0gwHS ziOlT;wn3`2)$ex7iw;9{YTPd_)i-Mg*cHsggdpI)n;%e2Az`2odg^+4&(11|Cn>)(J z*YQJLd2WDEX8bT)Vc6aXpfHlF(ZP5e2`cql2c2wlohw(;11hu%x%RSlU|2-m-p+tI zjKkCV&yS>!`v5YITFLyzcJmLi_xjcV$LksMA3th4@5d_xPKh^h{>B-Di7Y1VQUx3| zQhin?V5|BPwMHfdu-`Ehcbz8E`OtsVl07Xzuz}+a2gT9k_niDc7t)(urqvBM7Z#*< z(j_@WVJP-%B!&w#)ZJksN2aDk2O<`iiV2{)Q1C{F_MKfD&Tra(E-n@k_gP@_`-awp z(gRddC8_k*W|&D+M%RC6gXBn7!x{9K}=Y7CHoV*uqd+Zea`!>}c28Gju)D}V~5K$(LlF@=Zy@`pUJk`3L@CuBzM z(IF}_AC-Avfj)Lt3WysPW~q=IpGHr}S)QpA8V-)AkFcH?{g5nhu8s6+`ph7uO&wD+8W* z4OK>-KKjXVG-b410q z(9$ZYP<4)8p$rm;cnnM1-AJ2x}j=9mv`#a?Ve|1p;&7evFOm!?Mu>9WSrtpisRO4x)AOW6)e7LM)c*h|+I-w@Iu+ z#q7VnmgR!9#q`@pv97-)WK#hwZ94q@9H}@OO!`r(=p)%mT9cRv&rZLLM?Rh9P9zaJ zNy1TJK^qN!rjT8jWz@V!PWYeIDF$T0UnBfhbL)&T{N!1#0E@}W&z=i{1b_Ocf-O&W zZb3nAe(X0(FPj4PQpepll>X@1*q6`v$D?nh^kZeIF?1TZ`xEjU5QsT4(vP6&M|ml_ z(}#ISB7Ian#|nxS3e^#{`J*I?Rw^L%qoRCB&UsR0o6Fu@u#bCvdh^`I#{0HR*4bLU zey#KIju&@fj`e~JSr#cFv5@;o-tUJC+|k2{Mw|6AZS-Sd6Jmnpa7p|9#Xs#-o43{*L)>K^oC4ng5C)JNw->B&3p-GB=M^q4|E3wRY3kOIsm1Y zKRcJK-)y-#)>c+7{?sPdD?_m0?tMv^e`+b!M76i{V74b(HS9NTx)M%nY|bG^k>UFe z>g1nijiIm$ zO=%Mf=h_lX@tC|dYqVYCV7MC zlNl9EXVbIZUh!cyC&)al1DM!`{<}`adaL3Lhx!SgD^sHq3})7;D{i6DhCTU1hw=l4 zr+1Z8{9z?dEwJSEf}&`*!4S9SqHcHhe&2s0Lk*oT^HYgCkVV zVnW-m>B|1dfqbhJP*oivO5o5k1 zE0z5=)-5I}>k0&iyv33y;=MV{O1TouGy} zU_M@1t{ZLhXsBA^a%gC4fl0UVBJ?#?A@}60EHgpy1 zv2oGj{Ir$hVL7gU7fj=S6ocO3apw5*Cqpc%R0QJrYKq;WQBm+}Oi z-DdO81)#3a%d>(1)BL1}NRe#G%ERVIRf#ot!A{^oXpS|G@U(Ts$&6wtOBcbm)M<9O>|E8+9#XWO2u1+5*M`X!mCgHAi-#;J0O({|V$*_{bKScb_)l zS0KW3>fh95e*u1n{)SLK29zq;OGSN1z0I5{7SN!zaFkvjWt25cR8;^Ta42x5ssNxM zq;$Mp5O9rLRD*GF%!djv3mNHDfUyXJ07ZyUH94K;=mW-Z0bBrJo2?CmwsMnD2&Zb9 zMe%DWM2?PulTU#b9{yJt0R8pGjo>eD!M$3?fSm4>5Eg|-d`T27zAuDOUy6o~&NY$A z5$>>9sqfNU>SD`y+9Gr300(n&Iy-9Ld_Q|IdyRo>_9LRKd{#xHTJJCzo>W4WA0(Pc z6fc-3{Pj@fQ!dJ6oTV|8#UmUSJzN%U&$nRCz`AIdAxGFi{$bV3Q8hK!u!LP#$P2|y7c5uK>P2u$F4BI9=`~oMJ()2$CHZd2 zmTU33eLI2tu|LtlQ}%$IHqB0M^1f(1TY)vD8|OdTj~5N!U$JONi=z9M@C z)0yY`Uw}rX!TOprH<-~l6Fh2@2CGUmLIoZmkN&; zU{UXjiV-Vhgr87@uP*)`*8eapkOVM3L|Vw8LZ*$xNcbrF|Cl=G_&B?+ZO7KcXsn6Z z*j8gTY@Ehu>@>E~*lKLsYV4%3ZB4$ppZ9(5=leUq?3wGDjkVA7SZi_FSJO`ghn8FA zrehu_1Z8(_mv?bheu9krzU=U^>iMyHNa`0?6hhJ8r0SdZ6rO-d)b~!bDE_7-WT#og zW3(**NfZ|bOdMK9jVlGlJ6@_8C2>RQR&{@XyiT3pCv$w0AhiX_F|@-3iUr%7P1y@B zv!6sX`aThut;8X+(=7O*BKC;r1OU{UJ$B#mPXyN9Jm8K4i`7+)V~@LDmy+g{f)T9c zBnmz7`pA0vv=!}J{CxBubVO|>l5tNFV}|#|Yzqit9vD(nnRE|4NFX$USm8s0NM+@h zuji>O))???<$~0sY-Cq`e{{yKWC1rcP?HHz16{=;-&|t67{C48MnDVW3MQ(*5RQ(( zz_!5=Lvaz;Osi95gnUhcBO5^3_AfOJKICWOk~y;c6~KugF;QN_D-ztHjG55W7#}AV zR(Z?y#rRt^eHhO7V1F_)Ro3T#Gka=`Wv8sf$-!i|eN~8WfpvTNw2;ZN za`ZIwAd@elpQ4>cdMN(jX=v#};0s|?+uo0w;H5k6_^yO2U``1Ae|KVn7)N5!!$~3k zm@edg43W8!kTd~MqdMh(P`hfK(6}%RQ#LrYrFYyJL`uTzB z3(%f5+v&FIx1m{GJaMpAEog>}EA2R6kN>V|X_+WpH1a2pl`*r;t)J#t$mgXX(O*(_ zui!clMf}lT+pqHRgqD=MAZ`oMxH&uog|N5KUcYXjzF&*K`Y9>GiYlx+DK%xG5L{Le zxWQy0#4Ed}SReDjF45t%udlz6g|e&~E+;QYyeA{D5?IXXLd0zgmi=$u4oLyYKP1hl zH{3s7y1H_Gtwu{t9crB3eID&^ypXJ*se_r8Cgm_mr-k1phws7QhGDnCNf^CBTbN)} z_g{RZpbn{#7rGGu+qMnPQ9MDA0>R)9Rf;Z3Phs^ZRZVOLKUhz_scxRbG6rC#pV^q- zjw*u(N1XbNK(Q1rUztp=RkRK%1_dlK0`@NqDY`_U$e37d9sGg@nQMpMRAg&Ks9e!c zN}B#hlM&}?(Z~T+wDIwj8@ewDh}22{4Rd<6Fx9uWPJn`Br70GP|A0?1$+7Q8!w}XW zOABn=F*3)GgfWh3f!&~{8qHR)gZ=H>2x8@q-f_q;VyoP2quZaCAdT^rgQ_ZKqL3%L zZ=``8au}I5WqfoZ5=ESl4C*ingsS=usc*2rONJ7?`r7&eC^{K+p6m?V7eriEk1t!` zlsx^(VRVC5%WyCRHG3dDF-{XTqRq%Cs(_|Z6gS}%43>p?7=`D3n3Fp`BxI8b6{I-j zM0#+%$>pTN!QH+Yaf=zZMzLX75`n84fUrI?HylL9CBcEXXsZIu)6lTjutr~H-mrRf z4tUt?L-5PQ#tJ=_vn8p78b~2eOvthjbib4z=C2DR1Erj0#P)!mi zQR#FxBvW4R(+ z6p7=*Tu$ZS0QWmpEq2_Gvxw?)7>qKN(>@v z{BFu=MHix@8KA=|s!f+q%;0(tR(x#^3|poUL{S+z5ilN-t0U>(%>O@mAJH7DtLNSG zTafzwyhEaK-1~Oxp|^RC0I9vrbhbOAcl_5C_cBA!8!hR}N{u$t+ujuS!deaMb z)J6aJ*T?E>Azf@BrGTt|AgxUv1U8B&t!jutWpUy%yN?NtF@-tBfvxd@hrx@{g}Bl> zzmNU6vet%Gc<#@F07?PKz66JTW!!yRO^sjmOTR*$*>6oXH1i;!Fl5h_kk}b+GtZ}Z z3D*6p?pY)>f9t$XV6Dj^7f@d3v9M$cdLAhY4APZY4o2W8X;$WZH4tDIaKBp~#VTPT z0*dEJr&NAzGyS46aC~9O>wfjkR-fZ%Fa5NJn&;dD*fg_)n-6^z5_EbPTkab8(uxuC z>-7(1_e;)4YJOS0=j*GYU(?69d@6yWGMOpGKDMU1x{Y3Yeh1a+ilg%f2%4LaKltt@ zdA!MPgWdKdY2Z&!85m3xz75XsstWZ_5p26CE6Jv30?I_+>32GuTy6J@(*bE!@0U}+ z)%sFXu|xm9+?Lx10b{Uj%gR?9JUuuG?M}`z3^ZBe5Jw3(^l8k1cOwZ4^0?K^5j=uf zmGEevN~^7ZNLFe$g2GF(8JVIAL*T0ncpRUu;vw;rw{aL!ZMJrz1zb;#0*Swx98~KpUhl#*~q`LimxW zB+JcOj#j1B1&U1_QI<6ErX6}VNouO&uhCAaX~RsWNWO>ZD< zto;`Dsg#eEU z?~k$&j?^9)7dliTQ5idqh(_R@tr=G9^8!qmocJCH!hUVm+E!|_C{6Y^j(XHNLf?eXRYNW<&mdF$f<{r#tSMzdYj za?QgnWlm#L(`4GmpNZ&GdykL1ySq%Gc>SD@>+0@v0(>i5+xNfy`ek{`eXtx|-9A1_ zN=w(bsA|Pi?g!}>%-)VCFAEitGs*&7f76*EkIs(K^vX!z58S#*36X3!J6v{G2XlyD z8tT0qZbMvHgg*+lI^KsZO;;1A1$5sBEnb;(Vx{_Px7~%>^X-waB;cZ7j;1K&j&w2Y zb_LT!FH0_6F`~Iw1|#v-j#&R^dVyzILaMA?*hMHkRbj$dI6tg7UxNAyN&guIjuPGQ zQaHrZE*cC^kEQJWo$JEVQeEjoz<{oO7n|4O#88{Hku0?_^hQ=z6;GT@gC+Re$NKu( ze5B${5h+iDOu9h}EO--M%fzj1X6KYz)1HocV6(#dlIFZB0|}XM<)AsJFW;btkk*%GbbR*5;<8uyS#NSS zpSI6d(APiNoE~4CTslG89Up>DA8-V8B1D7k5Y%Z0NSw|Cl&=6^+>$e=wuMf}aPvb46LtYt0H zh)(4Ts?QN*^%7EbNgl_`6m0z%xK=|9s@^(#-P|eq)S>2FK$}#vW2ywc$2V3mtN-Uo z46+2h7DcueHZLBkvh4=TE*^;FaVI^$T|n_}VMj}SzMf18X-bi*hAha3f@&!#Xna}S zC;znweut;reosB?9{Nn;6-!mTE;d{H$cL(`HGa8%*DSv5_MprmQE7(4YW3wfxKGs2 z9bR1;g_L?7d%cbn%21~{={KbwUXFshRZ_XZ-;$J-n@8}lxi(8_Pe-sCzFTY!pTr1c zK<*K3x3H--e<6c9!mZW8n#>BTGVB&g=VAPJ{@t8!;}SqOOWXOJUvsfP(rL=%swvCv zQ>g<@Y{Vt#xCdL&L`5SFMd-qytc_t6Aust7+ z&!+|6Is4iE#Ha>StH~87;oAI6UU!X4O;#pdstx7{I9Mm|jlk*WT3#tG&OO;xUq@4rALm-hk$}R&jEoZH^`_YA) zMlTu8(jEdbs0_AkY1pXXU?^K0>wQ*UYN7v+1=z3KkCz$vLJff*CX$+>n68Qs>Q&>< z_Mxvg1~M^q`kq((`t=LXz`ONj^&Z^exVfnzE(#4?NV+76$$-aiJcWs)WE!F*Uy+`R zb4OzX4Zwg8h1}=UZx^V+sj9A{s*ax!bHE4amw-Cg2q%18BzOIW4Ku*F-Ib@luH2IA zpTzi;r`l=U1bre6Q5Ku$qDPrHjDK~_%E~I+?|A>DVrk0T_r=`0;t)v^3O1Dt<1;H~ zrqfEel$@M+aB%Rn(EVO)_G-2SVpsvEs-EhiioU*<{(D~}%M!D`_sc|U@nXg%QyC20 z*1%OF;>rH@X>xHJp3ZWoL09hSvm3u2u%ElXed?bh?>{G>H$$sZmzYq1I~J^r!>}yM zjg`4sfm6`i$j=F`SYG`LB)GpeqqdNU17ApZu4KRHXOP1e>KAY4&EMS^!y%%;?$H9s zS=R>weui3~tiqrrss?T^JE-Egn;mo#>KzvhYtD>KNMfg@O9mw~XRz(I;$?Sx-1ki> z315`#kP6+x)1=Vad@JCzuj1AtEl<0eRL6@OXqIMJ6B_GoL-$}l_Uk82k-XEK!f)RK8FJeweqcC4+Hk1B0u)&SoS8S>d^A=VohvhNOKO^M8#63S(l?4va^wEVzm*90gPECyxj#0ppjq-hBg2WzhpVp6R{i?yw%z#4wPt3=ii&t=`EQTIY*LEwpN;oq1V|Jf z5$*4j>`lXWk>2rm2G>7m{C_;pAM;`Jdb~+!X)VbVdV?7oj4RbW7^i~TqFQjiv+kn3 zcJsyUE*{vEazZMLPXB(T1Tcw0#K9&~MG~7ZQyX^yI^8_6RabB4^+HiloDV7ysAfxH zxgKplJHNE8CWZrw#Np6ntYuNrAE;Ez>TU>*aUdYX(UK???op!~zf|&uBy^nU)8VGFG}E zZXOY;|2G)L9e>~VEfv$@vBF%6Id@AometCAJZ)=N0Sx+1x?`dh(|cHr17YAyRWq$l zK6zV?o=`_u$_%PqOZceWrpqs6lsD1pZ@BMz%24WPdtZ;Q#^-@6(_Ne_?940#S@vK7#u#qgH0$NuhO9TXgFgEKkF)=vu zTHe*ia3$-#RBf*>sBPNt*nbW!mVn4fUTFMbo#}~edA!5IqO4iT|!oL`3nsKqq1CX2hJ@etu`eayMV*n`U2A$sT;4l zuC9E1{w&l6a-sFmA*~lW0$mLz$=Rurj3s!0?vAUXC42g0*22i~Vxc;RY~h>ZIm}Vh zF$8szQQ7#Y0b@KC-)HUpp`h&-qDdXc??z93T3SeQ#e#1mI~Psnp$YXC3QE+wA!?bI zRZqoi1{%Nh932&?#`o@a-=c`{5~C*(=|!6>;5R6^>DQ{;20!IVH9p+mmA%DFA9I;a z@CT?wlRWG9t*Lk=F(|IBB+2v5xUc!J-m{;geaF`Zp7 ztcW_?YE{{7tW~{a&@!v6uCD&|E3eP0I_|NJ^=6~}mO~#aCopY(2xe|!rS`EvsoB0# zr`@WvC49ia-lfyxb^>%y1(iFf!K%P2F|3poPOYL*YqdKRO_u`NoO-A-pdtQ&hY5fR z#}@&77xnZRpl*R^c5yj>KKi!MWNWQiP0O8q+?-qbhp08|N5kjK)n{)WH0DT?> za(5~9!r;%2Cr`E0>+zcr_tx~>wk}xLzrgUAeygUka*dJr;P0E^WGeo(Uk_*Rroi|l z$NY50E!7L%j#)!;)-gG;e|L7X**_`6*)1i+rcVA+8Z9)VZ!FG1hsC?^QW@DKKJ|oeH#0=4n1b@eDnF4wPkNUGf<(Xy zWZ@Sw88HzF9SH=fURY!qqMKK-)mXYUyd=fgUR_cMsL(lbSRE9IU0z_3mPlIAr@XsH z2#Cc#&GWzg7-W&#`Bwq*;}P} z0#r{SOtwZNh8}rMP*5OkJuWU#oE&0P&;$c27YOUF#l)}X_oiI1IM5ymL49J7458Im ztNpvC@Xml!2-1VXkp}R(>*(g(Yq!MW!QQ!*Kj~=xt9qZm0xHgV(Q^1jJGNZua|Dq&qyGSn*hNP*^of&4{Z&1&K!7%jSUSAvj#>klcn!Rd3p{0}x3xfpk zB3KeA(CfbvkiBg7NzPbua{VEV30l;!lc7$StUM3lERux0bPuqgN0o(dMIeLn52Qzg zVEU5;)uSvn#?a3+riBHOXoqz?1BF3NMlofB{_f{RG6W?;2@6bRw}LH!F3yF;=>8g# zn4P($Wggzqg^&+zf>Fdv^hmXN*AXjKohDXA1ua5wbf=pz*+ous{g#?!v6*=sj4cYI zOkv>RAZz%U8e+ zDUui%#tJp2e$bY1Q_wqm897UF+>QXMyxO4kD-0hCtKUKmG~|8ksP>w(F*puAuF|Y) z`1cswa=B1Lty73Ui{ip&Bs>2w&z>IRw*I|4c!uCX^!i(!vvwg8gHF(T^QZOVZ+Zrl zdLnzfmg~)&51(&9a%>U}tc?nJSpV5JXe^v$h1dIPKYasyO;!k)5SgG2{mahB{i13; zB#t8P+1(`)Ou!8e-t;5w@VTDsTueY4uZ(aPpC6|-xbEjlO&GAr1AfoAlvy0^1|zGQ`~d^;!)|b zVb=7!mszU)GaZ8X?0G6S9uJvuKMYe9d)ecr*Ja;6Vcd;H#LVi>@?%QlW{84?({*Jc{^GI$5gZ2!!w}|@$i5%tZ?(F*}vNWb*fjksH;4C z`<*O~`&hFyVKrx(TSpuoM_6gOIpGHnORDFRtJmx zL!;c;;l(OD7QL)=vX>6Dmb}D_KUD}!A+YfiOd2^-5IclvSV0J9`t<~+={jZ#Rti5v zE{@$gJv|qn=}dZ|$#y9?%>gv^_UI&u_bEs;)wO{Z5>xu!DL>ABETAR(h8+SAnI15ykfE9GFJ9N=8(nJNP8&03uDejq;9#ZnRkCcbm z0uKf(Hb}&-b}pTCr)ENa)0_Ug(Kxj#5tugr&A!lp!pZpQJ0uaOD~Vdz+$=h4B%y2w zJ{H{A#ldwlKxE5GY5wD6jtQDT%B43k;_T^#Kx0?(&t7D){XqsncU1 z@dc*Ygdxk<_nkpzkJq<^v`PdUJ^W^tR~COD8`;w{GcyK>N>aPg6Bfnr?WS(TvJ-CC zkaBG1D-CAy+wsr88atG@6qlnUw|D#B$)75z$;8omQNt^O^n;v(@_M{wB`_$ytu3O4 zOYBS>-%@aD;wf&`CN#HSA0+s0cXD`^9`kQMRQ>*sbo|dcd?woNDi=FOMfP~6-oYum z+0smEFKRvfnXdOs%3&+nL(mavdC75=Qjmeh>zqSGFqsHsRtp4og3)pPF7|`F;wq?w8Jw+RNcM0w$^U( zNDE;;{fnOB+%`)JTLw_$x&IIEiPzoX4PU|LD&Ubh7%uQwwSumIe(i8LxtIi$4OQF! zMB_1^uHrne_!OI=jj5FU zJk`+^!^f5GOxs00WdChqVf669XZ*pLuthS~k63*K+-Fww`)(QAEr7Z|yv6B_HN}RD zT78Vp^Wjy=R=<-+!A@?zr>7@+wSrbL^XQlen`eNEKH~>hx#5F)_#Ke_vc$O<>ao5@ zsX?y$;{5#ll87^3ocw(~Hm9YPc@$TYb_6H~2PGz_nerS2Y2DjRGq& z^1en_5VjgnM|*u?#`9=bpDscnozMBUIN}*y=x(_A13z+5peS4xF)R;f835fAXAB8Y zSq12&g^l=!g^`n4Li%HSSg=9h!Kk8fw3<&>p?__V2|x%pv_SC(8o?R^L*PW9VDTVC zk;M_9jGZ7BS&^6#VIuN4zFUK1x-O+wO0wzdEV6-+*>U5)GJMVoy+H6W)N}hq>9qX& z@S|RV8xvIEjH%3Dv63|5_0Ez%DBwzBl;ChwJ>nBA&&0R-*M9V%BpJv!oHi7gE@$sm z*c@-@SA|8Y+LDJlN|agVq(d1ADi~;%LWZj^#Z=w!ZyRp8&Ho+)TR2?=c60s!O&M#c z=Tigd0H8lgbLSGGCU+>5$7_)QfX$y9vl>ae7-N~~Bxe6d@~xIo2@yGY5!vZdB_Rn( zms6#IL`N3aTbKU~&Ptsx5QUno;1i}R6(lRwM4>n8Demq>Q##Q#V7~jNIqozliQ%%y za7$6Jy1K$N`cNrpE!4gy!A+vTm@SU~MxpO7w}VL*w|rPw8d<=s%WhszS$tl~P#XYy zk<+!|0j(SKDD2|^62D*2+I$ z*$6pR8FlymkcjdK4K?}5Ha>e2A@9-$&uG>nG6~P|2U<6q3d~adS0Uc#LD(=jIv}`{ zc_o7;BiLEY|GQ~pE8x0!6usI8?g**4T?|e+{OxTj*xP!P`K2pG-*>SUX(CqhWnB#d7sxSBx9NIy3SWvx_g(7j=EGxHq zN_u=3wU9pTeb3pnv0;O zH582o?6eu21?NjBX7Ew7Ve`A(^&DC#wj&(EH~MO z524q~ieP|ZOnF&^0kZb>?>g_h&qmJ02Jxbm_~0*6!N2AnE=~*{5~g9xHv@kcOmn0bIvaV zviQwT{j1>d-u`F2T=G!S-(u*@dIIdrx2Zmt5i(0XHRND-bW+L2!0T;!JHu9RJcZeJ zI~dRPV%?*%=}ThOx?Zc<_yLD>?7EGuUYqyIElWqoX_l~lr8Va_0I%1>#4}h4DeGF6 zY0FNdm_^2&h(vuLeRI!QY_M7=Pc?jQcl+3l{mlI<82q_90@$a#Z!_%_vutTUM4{qX z5v@)LN%iX;Jv~Mzo0x%}fAfOD#PzjyH|A8(27nAEgc7^Fd{w*F?mhlJ=g!5==C|JM zIx4d0$x~=~gKNea6FkNe+-Dt9+)x>~?73|Mz zWgQ|PzIV03Yfk^v+tnNvU>Q(Ad^jxp=KJM-;o-qGvHAc22;c6?&JGAj3jkX=to0O3 z6!iM-Rt?-tk$BYrvhFCu>q4Tm-6pF9 zoKBjTXIldF`vl`LevZPI{Enw$+GoUU{_u|0T*`zoE7V6d@$KKM?&BsmSZ@wz@mmUl zK$)ShG`D>fyur2Rdk|3B{3vD^#z@w*0@f$Dj4 z4GkIooI{8EJv~>{TL@#Eo^tI$ls9(LUKoSczh<3RI+Lc!vCdgcg<<0Y!8!UC+>3tc zPLgk#%$71T36!;b4f z@v|e2qOxeI`wO@T4@)+$@56E)56-5gRVMp=(PyG&gU)LsBhI$(laz*8yzXxUj#_xzwO@MGh9LQh@2suRlgp=d*(#lSA;E$rbTZ?LtshXLj znz;BUg+A~nqPi^YVt*+WL;NDk!r0h!j~3e!E6|WzQTh2(J}mNqMvh=MJLd}sV2eQq zWCV*(&t(NUz8=?^koP3`U_00E`tMtC@ykg{W2_#p+uc>3I33S+-4oYt3pO1iqfWEu zak$BAcZb8KzbE>qj|H$8`M0Chr*nhc$-arRp^*~`1DGennmGShBz4f+5b$5gh(g&s6!(DK6EVURe1Spe1F z7e=W7Sv^?Ao?{}#n3i%@LK^((85%BZd;8Puk3ZtzijsbNy(@-U{GD&{U-vo8X+plA zBqiM5M@#*9ar_1##}Aa>*@z*;fYoG0l8HYdmK#V%qA+5Di{PMK>VrvD`3B@Hve7B5 z$!)9^ttE%uy@@_ZNoH7-$d}wN8l+ZPSXGDSk}8AY=hj(mW=Yq?a5RR1fF%zA+R9+P z@$f@2v>?IxLsIK&Ed8jrnn|+Me&m1vD$=ovKn*{VV?xefIppW%eV@sfEK$ly6SlTe z$#d>(TMUCPwILQ)0{moVU|?ZF5`4XTu*mjJ0C0hgTG&A|YPGRQXhJFw`@5;;J%itabA7_$q5t^O)G_RZ+dvB9{My>j&%@4=jTg<3Dop zVp8A0)!%p%+Sr^fGz>M<+r{DXCZS>i(#sXEgR^H;-6nxH&xHv(vjjO3UiTv~b0h3? zeqXf-+VFa0z$sttvVFeUSBs1!c4}YEKqux?H(UZ{6OFVj=+ZAv0a=J;TQ{=KY0+2B*X5oN^m{yk=)tZFmW@k(C~S zGboyD^>htB5fGeh+|Tr~WWeYBSZJapS!?_1|0vA&W6yg7d`(L~YkQc<-hi6qYy19Q zK8A{;R;+MrWzFmHlxV%sh*&_UUbA1z#E``yGfD} zHl-tu!7goxQ8~Au$D!axGVh>s*7Sdf#Rg#d4@QQJn=TinQomD_@Z`m5aZf%Oex>J7 z91T~v?P%U^TdLC2zx`RcKUd7+VQOG#A+KeaQ(EIL7}EuWv%BfpR+b zNyXzBagdY9rQQhe8U95ip#{yhxkQ>9L&HswlpyWGCfWyL$7wN}Ndm2m}u1~I# zfW(;1;I>_^GrK(o14kP+U)yZ{i9qVS4!aG@Y%!1oJ1j@9zFitVfFu$0p&iAihQ4Iv z{MK=MnU4<{Itc8Ov9KuF(*E;3nAPlLqNMxdtdRxH&-+)y&EL2k$$yATQ;|q`lWms9jTo@6~qIyt2*zkXu{-kXzna7)*M1%8Xp@ zmX`*AlVj4041+euh5miD5|jq?#M#>ep+6fQ&Za!B)Ar=!Pf>MPV--*eZ^ap(E-dZO zhp$50A3N^drtMZ=o=V_ji4;-MXvd2MMI_{+q*TKhC^oU=_98h=#s=zf)eN-?nyPd= zeShvqD1nVCmrCo!>KyLnIaNPf9-;gsTtgjplw=1wOeR1eW&jn=NUR-lgK)NdOv%*U z<0Gx~jx(+R*ac+pcb)&~d^q6AhjpE2AUBA~ZULgyvf}&1CfABSnX|CAMyy1L7|BMX zu52k|W*ygy%&Klc%&ByWST_I0%klF1x+B}KST%qf=b02%tgfaezBN)@R6(D@sN-;R z%c43IT|RNyAeIQjK)Be4*2UXhx^>qsTU4k9-BS1@FRypaAK7biz=At4u0!B)F`vg0L!zkcXySakGkB{i=a_4g zCR&(FR_Ag4*v04bR8J`lX0Vxq3%~t344s{u!7Z`W!;u5Fe`f~L>wzyR)YB%g!Khy* z&GrYaximGFy3t!=s60JA2~Vu_{}-p@a$EeZeYwz8^$)?tWasu8T@*W!E$Eh=SxsBh zUk0v6^z!0qZX8j#J|H4?Aka}*wJW!bIGx3NKO5|f(qGtJOWoFW3xkuhHjSeSA(I;{ zo@8Td$LDY}`%qrFu<-fIrS<;$c&FDmi=>-p>YrDi)X8}}f)CN38J zCgbfhI&d0(rm=0*@zIfX>)y=45K+}B+Hm6iB=OmPju;UVle_z9o6Clek6 z_t#C~Bn+lPfngeQ)&KxXpK-K=0GrdpYV>e9Nxz86?)~^T%yZV~@9q@rX-TcOq}d#| zkzk~Ng;^g=bbHYkfh(LWp&=nhu=sxE7vOOKS>2wD4Tpw+(-OHZ6r* zZ3ho*H?Lz7`gZE-@hT$0i{fib?tfhUY&NEJg<`71l#%ANvn6iqK$nt#)g4gNf5==h z)7->-)z$hRl5*-{R1rxSj*IP5;XGzVY7xj_YE32~E9Bt8R=xyBBGVTdpyGg&CK@t7 zU-umhP*p~w{?gB5O{ADLL3X55w(uH{2Gd@Ko<-n(bl{7|=O$NrTv*LM`uBova(F7a zv>?%m9SAhhzR75XcnvhQj;bF#b%W)5zj}N|LK&Et6 zWo3u@JW~UHijuofX%P`KxXd}y~?XL2x+{#RXs z7T)A@c6QNUdW0|n!oPXTGGElz0*e@~CC?km)Z`2D4nhb4{d>AR8*DA1#|D@caxWsNYd>tRK%-y!MVpw|VS!nw`cEP{U2n zUK)Q@Gi#Ngpb-mM;b(AZ;NVDQ@U@dM_u{Zg$7X#}82Oi~s93U|HtbSSR#yH$>Y!ug zWxGqTA5jZP*dh3DG#zOsaM4Pi#FElmttqU-Motru_d{pQ7ETC3COp+I*@9BSm`CWT zx1E1_GDo+uc4yMC!{ub)@Ksn)@~@3Ywt|{Tlj`pN->|qaFopmt$q3i#LA@KJDN!~v zqreMa*dL<)@^puf?a{T-TDfuI)M|r5rwM+|lkWS$ms(5n;Igc=A=f`Gn!MmP#a6yr zPjgDnKU;HWdtJ1CpDPjKv|p65`IcMp6Qirrm&pxWt0;KJ!JCS=_)juRBR$xtFyk&0 zxV&vxNr4Jo&qLpQPNbwufTEG?GpBlZ^?S#YMiEbw5?brNJPPoUOY?dBfx+AT(eK9! zsf=`ncK?3s2cnf$RR#q7Y+(Iyzteh(*X=+Bixf|^^;{y%lK|C#k+vb+@qW4a2b*Vc z?YgAduVl{d6sqIBX{!saUEJs65xRSr1tP4EqA;HD>=<0&1Qhq+2rvGe$1$LzT6w?V_2`>0 z`a*@o7Ph|a_xkRK-U}-UYxG6z_!c9t)ez)I1eKm_L?HMB@A#G_6(SE!f} zaP%^aY5ve5nQCoky!~K5%v@AYHp=%Q*m_aW&|;{*FTP1LFExAz zU{aE4DXgLoUcV_Tcv|D#X@jL+zfvqaKsbLu^`+sdG3tknhtxv&K96k+;uJZEiQZP~w z!t}UoEH0W~K1HSC0vZg><$rR@KwNK;@5|zECMyU_kFj8+dA!sOTXacvwBRpNLFjG$ zqCLb1SFnD$Zmoo^s~d^Frur(E9?wM z^nl4vJZ8P+T1QFEzE*cLow2J_#ld#8pw z3w6`@msFwf)0w=tXNyd-Dvd7D+Zf4`-~+43EnH4p-qRrotUUA53AjUlk z%?ZDBpfB*JtV`o$a|%xUz||jcdP&aj@(Ah#rb=*UZx&z6sXI5fuIUK}vGo3qh#REe z11Jyo)Vy`Bb1^I|dbr#ws(D9?t@1Ox7hi5_wES)LsqazZBoj z3I0@X2OtNk{v(A~XjL;apvcp2uFqeA3_+Z%Nf=1*IiI^Hf3O5F-%QL4CI3rnfOq{$ zdI^6htaUBJw_0Gr^%z-wD#o_0qaLoy&A|6vxj(3To*Ss5L4WD-xz9QBd-@Iru1#o! zY8>BJ$C%czD1XwTlUklyau;l<*2k?+^Y;%(DIB_29$#DYo4K%5I zvp7PVDLx=NnTVd-4xos3*YLyjM+v$*1ep}Va13~NQPseyXwuZ6357=^7F_w` z_&!ie$swO}^`;z?{gU5y9??ug7CMm0=Ydkz8=aiIMv^l-=jgoj+J16k2Qd?sgDlFA z1fvy+?4YZ)lpXtMztLo}af0NC6&`G0OPjqT$ce=Np zRlZLY-siq3mT00HA*PA{A{7S+8JhcfBTiwAvU|SufsVnE^tfm`-35Jmz!o`lM#A^w z=xY0C4E`V;e26T8Wjbr%%)gth_65r7%`(dA)iaHOW(pgB>r%@AcXcu4~ixkZfC*G|I`ayCLb z2srYP**;};F54II?!OxF_@U?A$(KhFfK4PaYXlQ8LKYTqb1Tn!qTGyfVsix>!PDgd-7`eUFtQ z9^w6E!J=BP%l+m~k}|JWg(STcW1LFme{r{J!8jDG!KFPTg@Bj#34L4PLr`X^S@s(e zj6~_$u}@}~fPCrSX2Wc4`hPuW<5r^eP!#s9^8tBx2oYrp_FfVtfP_?gfV(ujk3tv% z1&p{vdd_WlLnKB|D>L73n7G0vvP9KQDN2I|wD2f)A4T*y83_b3M@W1giIaHA zt=kQezHJvGhJ{JRoVAA95n+PE7;u2^=1Smn z@PT8#FKD%4g~27CNKc{+!!hl5Po^Uw0#8pxA!f&i3q_DtFzv>a1j%7lIf|$Mw zr}D+z?x&`HZP@tjD?J*5XQmg?Rk>c4|D$~+E)g$eC=P3M->34KG%8td(2b`~*682W zxL@T!s-Bno3I|b>7T(P1p~fYpr7cIg+q!3Z;(%OQV|r5kajj)0nHQxG2peo^-}HK{ zsP2-J4WrAz{9K;ah8Y%C8l@`@vO1-}T(?d52WOk=_-wtHgd4mP&CR{;e#t`G&i5kx ze)(bjRYkoQf{dCXfj~BaSY_qcMLMtN=L5l6ns(5uiFYo6|47TMg1OnUGYMkUS6EF> zi^g>TD&2vAom;mWlCb>aJ|makNgpDzs*x1 z)4y}zM*q;)G|p%i0ACOki#SlAl4B0SjypQKJDw9y8Y>kml!ziaJ&hg)Gh?Wh9Vzyd z>=d2H_X5VjJq}m$5g~g)}OA;i+lKyWPCI2L>;708JDvAvK8^NF9dj z`r^TS?6a>1l>as&8&-@nd;MtJIfO4GnoRF~D9k>C5Y5WM%7)-8WlD`Bb!&~=={wVzhWl3yw@HUQcsRNy>3}1EOjqh zY0%p|`gM)QK6B=9Bvm?+jBf6_1koY*A_*7L6GIg6vOA@kxx4DHnfn@0blKdnE%WQI z*!Eb!#%b#NL%T=qu37PLug!JHcJuXr|HL-!#VP9R9w=Wsv`0*9FB{S%#*y8Y-)%&A zd3*KpWIMxXf6;^VssF#18vMx#o&+$-SE8h6klx(C5(gGG0>cy*mQ@_`Ok-&L2@{Vmu~29pd_gZ2QeFl!mug!oeQ8)8^f!)B@qdU8qd|V%UVAN*KG?q`a_K z7)n9Ju>Jho^3LPdD(^VbcfjPdgQxGo#c#aiBmn#2IpnNA7x~8O@bLwu-3q>VniFs6 zn4r(UUj&AMj>`?k# z&D5y7N2o8AInyS^d0tWk1TBks-}b{6o_fWS2(&V6$nYF^oa2+x9bC;^sZ|Yxl)E)r zsdZWVt@rRMuEQVh!~Sge(_nRtyr&V+^#A4Vt8}D5v}b2h#>)@8E(-MxpasRhjr{K} zaKG@YMuR}Z+hNZR~pC zOn3zO>^j&t#Edv>J5hd7{4Y2?XCH0>#h_Scqehle*~gsA<5eKwF@Z`o!EDKX(Yo~3+_ zipTXN?NU-#JZElddNmulW`i&PRK`4T!4Hp?n57R(+{IM3ey>^@_GUG4%k~n=W%O`@ z9d?x-?}IAtbC;c4ez!v%{5M#hzq?#=j7lKTg|kbDq0wSMWMM@{kgHB+_CTVGrm$Vq z8gxP6gin zZ;AYK)1QSZmcEE)E|`K8mo4A@%g(N@$8$kaK0u4#`pw})ISwxW!FP{lLV<4;Wxdef zv?aNn0*Y+c)pf6~^ER!f^NgBWPG&W#EYlQ&vcG!&y;^Ou`nXtJ?(}~3xZG5JUaRb# zrIuqc>aaguVwx{giX3?n7XE%PnAs$&@wRsAp2_nvD6n`qhai$M%Ar-vS)`nnXlb z5XIBjHae3QJc-XgUi&k>1Bxd%p*LQJC+?DLth#F+96X0#X%Zq1wq89ng&BO({z-NU zY%JW3aIBN;*uu%?<0wgB~)s^Zc%#3l7Z8d}9i(!iOi)Wa9X~$#?bvty(2{42 zl`A}gR-6B{Ii3mrY*mR3BC}SuWCNw#R;Uz7^w<>D0~`2B2Uc7TLH^1CODecEO~ zfTi2P_LgtRz&AHfZORkv0<3!ufd$Y7K^+yclcO#u*6I9ZXa+PS|77HIQ8f}r)a&`7 zugI=C8HXsdLMuWKqrtA2-6qiF@SZ5{Q>)*T2dq55IL;O_djJcR`+(PqKAKsk#0;Ap z)t-92F=Te>4!2orbUum$_qZ>7j%@8Zo&UHRJ&)fFzc7T>taI0G*V|lvxO(xE6)er?pRx17d;<3b35ohz9Dt_-`^S`e@0=Tx_Fs#@kP&+yUQ%#j~E5W!Juy* zY&!SQ(T!hRMyL-A8@+H#RM_U)c|9_h&O4oMj3Gt?(|RrG<1u6+v3?_Xt1lJ~kK@K> z%BBM{5H;Xo5>%^ztXt=k9@Rg-#;Agg8Udd}XmtG9oDl7E{-x>|=m;z1nMyRRLalpm4Lv01ANBAo ziz>eQ0)+rY#i?OwE#j5ei(||YCbylQo`A>sK7&r{4=~foZweLpbK1yrGD$$d?psH3 zh#wRxv*svU-Z4rn63+Ck0@NXGOw`x7zwUV9L_|d5;^Le_sEHF^Db6`#DMd5q5|4vp zd4t6B4-ThDVH>-}GytlHH~eMJC6yx84ez9qQe+iV3^nxnt?x^y=&1l`W|B%kQL!9A zZc;^3TQ{!Z~sGt1r1`y{cR|(Fmsh*qSk%f& zT1YE}u63zerc@2Q9SHF8fdaJ!fxh8*Eu@K~*K+%a*BY+a6G8ME(qglkMwt=mOkI6d zUCHQUcxL68ssQl>gxjMj%@&@{7ji!Y7Y3d-_GLEhzxW1N7?~3skr7n-B;K64!Bg`A=Y_m-O?# zmp`X<*z!<1UcqKsEHz`nUbhkh1O|e@60s|<^Q<4QuRhB8FXlJaRka$6mTkJZ6+j9w zbc;(ARcU43FKESdK~~C6X7i7L0eYwkE_nNd=XZv>DI;H#fwXhsx3TS9p{waGk_$R=iIz zr?Ij!hgltxUXbBu2x@9CHVz2%=t$cB{@2T%^^LW{E12-0Gi&OC(0VOH20IOjj4ub2 zWYX9xkB!gd73RG{m3--qmPlR?82h2bRAekybK9%;HL+EfW++&2f#tt?{XT7d7^D8w z;FzW+Ohm*bHWiWBjZMcIhW$?8L^G9`z<*pCl#RC@L>tP5c%>AxEn z)UgT(&<$|P1y{v)rTQ^E&Q(?#fqT5Yk0$iv$ zM2E7g!!$UvX^k{vA;?o@hN80>{-)hz&d2=!8}^mZ=<|gn3Ok&@Yzhk2*??5Qs-{@~ z8hFgJf(+tm)5$Py{)?%n{)Wn7ErzK^^wQI@_aN7hn7Ru~Y`~9v&V(DB&u08QNFp#b zTD4q8psD0_9>M3oCj&oFZ#N{!+v(Er~uO~wl znrE7y;#fW<=Re%UX{7)coM|*nk!CPF+?xZQ9;zfg5g$152=QC4#HmpGsX|;eBiVrw z3l*@&V<8yH5ZvsMxE5$Jp_Xsh6BaPBy1{`f^aW;n=p$TzS1}m&PJ9sMB~-#2F;cWm z$gdZEvNKj`M^!zgGE+%O{z{7pL}8?$9BQYyjtM1iW78mK#|NvCX_Er|;qOD+i||SJ z>S@BzLcU@I;>p27;-J}t0nlM#g2WjDJnvg{i6A9-9k_@?*b^m=qy}y^3U;)AlR{OZ z2Z=<$Vo55-NxG0JNl|EHOe9ij)0^V)XQ7J@SPQ`6`#I}8L12+3N=f1}H9!(*6c;tz zw?GVwK!ze(MD^;9LfW9l;ZOr zW1|Ni!Hev!HvBEQ!3?&pobZl~v{eiiXPsS9A5ob@Gp?cq0YoJF0lnq>%TpSjZvW!C zR|jsdm=u)-67EAENs<;b3`x%wr$S|;b^=-ufc~)#s9PBoA(O)@!i}}Q#c}iVriG!_ z&AJc`CmPHMChB{&AhOD5OKVYmX~+u2yq~wM)ole9TS1&L<#08rFyag}R1iWn7K6*J zIsV?a`;wCDi*|KJ`^y(qoKS9K|ZgQ2f00G6E>AXX${90EYGnT(vwdhE8; z7Fq;6Xd}uz4;I4GOqP=k)_y89dVJcCchYYuX%KQ)?el+sDOH}X(eLoS2!6HC5Og)a z?DczPUwXu-%zr-+21hDrSNb)&p!2vX;32th^L6c#*@#d6@IXg#U3ADDhwevM^z%rcv;KT*HN^?59ZxHU8L)znV-Gj6WXzzcwEPQr) z>hm6pnt{2VxAJi{fT~Ch>u3w~NG@aG{n}Fj6`lyD)aJ_xVcm1PeYGNS* zgPim?LWWQTM$J0qdFUE-LUk|!1A>~%{0lcFO3LFIzc#(HOXK{lK8uS1@U28^Q3j+IJC!r}gYI7=#eW2Y7m3oVb)~qamj}r-J zsJ6yt)8&q}Xj`?9+j?5837hM+>Fu#=jV7$Ztnxz%YrT}@Sld;Lu3G%QywW1Rf`#60 zrQMvK%G+wZ5ZPg{zFhqQlWw=raCW6x#ec{V>k zzmmaU$P6@CkBDFe8 z@;2B^JWQQxoj>nvUi9wPfZzSHbPKwwGK&AxO@+!h^Z60ny(v90F_q~({V6rlB0MO(j#{EC8^FO$}rxI8>p1^!^(4Q*U1JK7SNR_*sl^!ua_^t2b zghT)(5T?>PLD2W==dSpx{C(?}pWC2!kVyBbW-%0{Thaim%ohwxX<^UP*i9!y83L=B zm(}ezYtDQcb^xQVukm)|yALE;1V%v~8vP#z6U!^mi!dm=9P(>TvK=q)$5~~JP&-w= z?tS*oyn+Ow^rE8s?voDoRnez3&+-BLJgHf$L!(Xs%NU-XU!HQ|KWw)*o&Y5aKT_4#vvfOP&!3Ni|K*-4VT{qd=bX-HY95=#p1rL*%Uh4EXB zk+8#g=lzd|9`}cXhJ?Ax9-j6+tL-kQ>;6;R7d-ZjW%rs=UfW843Q`Ja!R2+`a8Jvk zivuKnVxfQ&Yp3C^t=a3VsG(aN7Qw|0qAAi|oI&f~+sm#kHKnwF@tNNaq=mJ=I&F>a zJcD5XJcmRvx9sjU)aZ4K%ltDEXQ`n*sbZ3R{7ZoxRg13FOIMv?Z_3bJP|)P|`_pp= z(ve4QAASwCxF1Z|?60kh3ukw1AMb3I9MoS1nLFUAvD1#*OQeA%#rWEdw#VFLn)s@d zo!2nX%{&QM>CUOXZyq1+&77~t@qH~>`oX^&G$3GGtKYT7>ag6&w4SYJRJ_8DSh6or z^5w)PVFvl}?ydq4`vH@fjPT^LXTz5+)T6iNs19_(ub0Jb=tEXRj?aM5osbln^#X>^T)s}wN-wHw zRYq5YkcXorlhV zNRfUAi`*c^R}Zp(t4z50U6g_U1Cfn z7`}nZ(cZecrsjQ3EARPFlQLGblN%jYNLr^7%-DF1Pwz>O&e{r4OrS4 ztMZNcV5y3yRgjY@_s#_kj=YHs=y?E0B`Xo57o&bDf-AVfCm=YT+lsB40%|6L?VsLk z<}FL+Q}>&-*7!-h%-!iVl4q=95C^1ONhZU?ROS&tC$XnfOt@g5yf&8HzPszh;@ssr zgl4cyXs(ineosFJYq=PM!?KgaEehhZnpmlgb;gp)X#jX&*XZ>`EA>rf)vBhZ4ihSc zp=n5a_w4;&R5!x72m_;)Cnul^U46FMo_c&Shlrks!z__sEVa(*`+eiO^?lWL$4wDw z5LjF9$^%?HdVXCQ5{!Gd04oXM8i}Q z%1vQH9V3teQc0t-D7+Ymp?(Z2LT_AUn?W;(h@RM^v4wp+2;Vvm<$xL0QaLO&uL;ob z!=f5}`|zlnHt;UAxqTq{I1tWX1#P{O_)6$X4v?`Y19)dypLUADVm18q%QZib$BnQY zb?c`7r&tX}lc%Cx*ov;@<&wtB8S`=B6-2oJ{`_&a$A`+Z@}jc2qY$*J7jd1Bp*kaz zVyS&W@@OvGPd}tT&3x}`I>P&f4Qu3O0Mr8{HVLp3=FRh4@Q{3Vc6RCx8WL5WSn|ZAf1&T=0=R55 zEXIK2-#MIwyGfCN04erQF^FzFmzr_#Gpzoqp2bPEG2i2y6>g3)B`)bitGQW#SBOD7VTF=(|L z6f;Z8!EY&g;(UDus1^1%F}d|7N0 zp+mc@rx(rr$WswzUKa8}AM9Y$X8yQSwn5;ytq7DNQff$Es7?W6U~r@(4oKYsIe(e2 zeK}xlne(E9UH2{lMODR90`elkIf$&SkX1?Kr(vVxY?6f}B^GP_sz}`(Y$-B+r8R=_ z_W;ERQ!xJifMlu5d_So3_=k%l~d+h)ItG6$+fttn#R-tGk}umHtG8hKWD{ z`;?RDJYR_~Gs~mJ;T-ng{{n063ePro7?-x3cbaf6Q7Yy1sKn5>$5#rJrYOJ`Y3JD{ zDK0Kz#)Q4GhRTMA!yzr{_1#x;(9BqCS18l;lYT;h-Le)vV(~Wq$T{}Q-Grs2h8LI9 zNIEOBV~2;Y8M?oiwU}6DcCsrcg+~l~c(CK-PM7IN3o)hUce!($7_Q-ul8mFrE3>tm z&gsO8YV5~|TAge#qsiOqbZ=I%zx->^b5dA}Wcdyr=Er5yRs`>ikA-<<`|T}o8P?YO zB?g}Yh9JV>@{2*I>G=r*5e8O>*siWmSHGSwDC++774FlgbQYtJ-$j{Dy*DPLQVuH{ zF4qLg_G?d1^IEl!icZz?_ya}o4$8_iC#iPC-D|+YDeSXDJD!3(hO_i5{_Si`=Q5$E*w`vW$5&{4b|Z8!!Jjcxgq;q?wME>DPx zBjDlAv;i-Q3G0?5!1pcF;o4WL#%^7spSA4%77kjbNBaY<`9K=?um15r;aX+R|GU)0+A{kAMtS?Joc z4BW9hC&nXUMg&XjzQsEA*4qq(^K3uu6FBuW`Hf7EEIBUJt4yM_>N`S5TDSXL28F-a zbbark44b9+^w46^)#(J<)d$xA0!s8#FDF-Vl4MfR&7wa&USz62F7SBxE#tU+`b^y6 zR&)7tt0#$Qt$nv)O7A<|(Ago9m9n49FH%mRBsbX?RdVc*K$GYLog4VV+23F*sR%53 zF@Z8dRUM8faN*Y&f(;)-Ok;g@{q<%2(qa30@TG--a}u`^dMbTAg0uPIXW_v1+lv;D zJ}gvays?Uj|Mz3A<@6Z-FN`mwnw z5D9oFUo>STZ613=C}1~RnGLIMakFP~ZGtq=@QN@9`P$?AXUh~7fUy5 zN-aq(G=znmi^|F@Cx&B{1v|_*jsBh<`B7SZ`hb4!##{J0f`ULOK=`@FLnhct{;e zZgtMjX085Xk;ibm3nW5KV!v5$2`=k!j~cdrI|kp_h@N89-2Y(fz_nyB_j~#yXu+%< zc9!K$!$49f|AS%Ac^uwP=!>2@AQj(L)Wvs<)`n9|q6EfB+ODPP`M}5Zgq^t2V{LPQ zLCfvC`+@6FB{`9HqFgE1q~La}1Buo`MJg2nmY5VymF&5pTS;DsG!IS!ixe_KRD>QM z*M?rOM~*B)JC=TAEea1CR*xM59^a0Eme)Zvh13`p30Yi@3kKG_fAo)nI5aE*x@7OD zn2O2rDa9@H?p)3mMM{icAk(${`wi4J;(A{e@Zx(x1D6k| zcVMe)y~HX^^u!7@kYX>8(je0xS^M#U2mUzSOD z^NVR6x|ib*9QpU@zjmz&b>0sKHj0VV11L^T%hv+H%EFA=^*?U~6z*n%f%s^FtjPR) zwaUa{g`|zqX*WOOOb=m8@qK4cUD)U#%z_+Ch_RxQnjr)C-+faaC1Ac05@7RVYb{_( z5EiI@UW^nBi6oakc{(L?tgEml8NyV$ieY)==nXGG8yw{LlCApYxFI~k5d8H0#8a}M z$aYua9PdQ>2m_KFA|;kKeHxLvx`awuBND3~vCjnxMgyg|F&v8ixlU2PLCuNJWXa4< z5~d~fNXo%9{d!KHz6^7lD=NByLaiNsFkv(}HbO#{>w_<^`36FOylBonOdA^p_+(rm zYAyqiTm81iDCndTtU(nhqWk4dqgTQ!E_qo|!^X6`?7#70 zQ7DJf$j>z3e)|B!I#^6@vt=EGuK7mKP>Oh0eWF9jl%$!ApMEGtH(ub)m9H05t{b7M z!Ecz`OhP*)LrqvI&{ItnyJ!~WF|O`qlm3{Aan4VpPsR~FGhzMQY!i~U z>vI**e{HlmXg4uv7m3F~4TqcoD|W6RvZ91s<8L~ZfldGE4IL&k>+`;sUg~U7+P+ei zP`K>|UtIA5=l~ejzS_~|$4?Vu#Pva5d1Jt3__d^xXlUGJ-64+5v|qQP(Bh1)R@CusY0NU zBCy(_2gY*fAVM(Grxw$Zg&l4gvt#|B+utmupW<-&U>` z2$zUGE!63Ik&McsMC%?;ZRGVTqB*gIN9OwH7ZuW6Qu~ItU^Kw|K$VS!9D_ac6V8;= z!9nRCJ6^d@C!4my6o)+;Lzf4p3cJT6%Z%AaK1E=$RuaCgtlda0H{YP$t}c{borQq| z&zy0JHlqw1m+rBteLrfF+Oxtnozz+K27U}yi^+)vXwVrOsVbbF zsEEEI!|71RLtnsd&J&2t8&_x!S*F`SKWms_D}Y(tJUm3DaqM%0?X+L3m^eA6@-=tt z-@{l&a90%~dWz~D?UBjdN}-3&ZTJ}sE^!d9SFqz&cPd&EeX;?2o2CRj8K&_4lP0;J z&*hh;y^!c5j@#9=tw1$eY_TY_0Dz4zaoX(FuOW_%X9ZW~Q5&K@CKc_y-}|JKcbA(z zPNX@dStUuSk+vd+AB7gT?qA!NFzRIi*+!!P?M_ci@eJ#(6@3w?H@anYjd(&qx3SaB zUK7703B!`znhz_Md_Z9$X2pH=^e2U zr2-T|`SdxTkY6zjT_U^#BG$YyF`ETe^XhbMED6^<0$e`uexaG zPKKKnkAsnGi7L*o@XJvtSXYWrQDNBB)sc8cE$U0=-#4{Ye_(r(T$#TIqH^YXgx8mu zO@jkl_i9J)V2ojYzD>f|)8ftUqewfhMVSjyDTGCodz$U*N@;dU0;Y9U`4;X)zk)K~ zv|&xPGVjk-$c2q<{H<{w|E8lKdmZZyVoL=-Y2n2a>(N+6o4avIEyL=J6XklO5L9hU zR8#k*AK98rsgv2Epk+@~0Keo-7?W`FphAC32N`RKUP)r+g~LILtKr2}D_~Pgcuxwz zL?Bz~r~i9|W1i2RXo@3xYz4^aGK9*NkC1JBJo&WLbWL{5o_3>6=qtRnV#4B-5Z$<>C`Dkg@w3 z{ZKgiTH6%qus$D6%lID=*qxqO9W@nO33q%i^P()T3{2RW9Ei3f(R)`h*lf0si_fv{ zb1D6ys?S<}{i%za2GhMGaW-DL#n(kJyPS_e%Mib9-GrX9`-5u&x7P2QjbGNM@Y-_q zH9;&iNa`!vF2~iFMO0?6Jry4Qbgsbnj(aI1Loq4mY2@2;r)pWHZ2qLg+kAcA^*OF6 z8J0f7{jbGb|5~I8vU+UZyjm@kWAFY{xqdp{N;@qs zEQh+vH(QC==TQ9;%)0oX3!jYKeu2{#0WPKP(57?u={U8)M?|h|*kBE}Kk&r+cEp|l z^21~)tPW+(GDaPSb~c7>HQqk^@J>^wV@?wJzj#i#futcrn|Ra-JD~|TEQO%yjFHypR9Sr3T}lr-1Yp|Z zGli|K(6Nl^@ZG)nxsw@oTjrz-%2=r?&t6YG3@UYPbJAY&-aVUrC^od!1R2{e=1f_} z=XapN>;gYsthZXYKk6NIg^TaPWVlsRH_!&rY2WSyPsMKqT%QeH)W!`TZ&9GuL5HFx zS!xQlf1A$=!&L%%Zjfj{{2VWDt)A{(U(G`;CNt>t?eIBx%ke_WUaC_`rAX(f4k{mz zB_e}CrWz+O7Z{`Fyxn=9Z{x0U6x?=sc}g`eYWE*_$aIgi!(W*r1Qq#bdibJ$!3-x1 zVBS!jUbeCdrgS49Py825cAqBp4J*VD87R{nti>j9sL~NBU?@|9=E6S(CZ*MQ?J;t| z(rJ?!EsYGLW*5J zywUpI;tYLFeu>{6DUcAJH5`Otrr;uC!htWq_Ii3U+vM#S(%)KtOiMr@))K2S>7=_9?96v zYTc`jcl(gRtDB#HXQ6`V;aS|K2^dp#PnlT_KHlJ z2w}p|ZFAr&F~cy|gV~wPy9Os0sLj+Pvhl$QNiax=c4`%h*c-Fq;@ooTp6%@k zC?lJqSWqMo>Ud+Kh?q&JKj)!L5YMG#BMa>*=)yAqGUCMfc;!?CEQ9eatPpreBrqXR zikUK6d>)93(i#jCpOrlAstF6DZJqzcX2Q7jo!Tlf%y~39pmJ$q0t8~~6;l#a9C-*U zj$r7hUnl+SbOOM-8`!^-WWbypoFaj)g&HlNq(MLvIj8-eRk7w5s^Pk)0A_J`EG>ka z$H)my!pNj5Ye4}ZHC7r22PUg`8n8OP2FpkQss_Ja9i-#erF7C3_G#0n_5is?A+CDZ zv7>4o$B^syKIr;Frv!{%BFnn?dJ2KKQ&RThN3cm z7OMebwCk(Nix%}_4A9t#qe=$l=Y0J9GnK7$=2KGyimVmEUyag;eypt6&#4AR0(<>~7F+glNj25PIQgF>q2$gHrE< z?m6h5d^vNw_%X~m9A<9KU-2_$yT=Z|GKctr#EK`z7z23+vEhfCo&ad#ac9l71C(+X zZd~L)1FJ}Xo;kCYewA4IVaXO=HF}vN=?a^zjsE>P`VDk-^$le0+$@3vhb9%omh<}; zmIcmat}r`3PLyfn4>z;8vyA(X{_!WZRI>iv~LV+3EukC=~NTSBPh5oQjIc4kb1ocS8j@;EFy>}}4E zK})K+hV{Fv(dRSO2EoP`EwzS)0BV^#4_xw|O5YB-h+z^Hmipc6Xe|;a-3|`qwK^lR zKbf-YpR_4CL~qW99%z=C*Sc-47attSlB+Y_3(vg`Me9*ocITe)zd7$Fz2Z*wToyxH zK~JRJ&BgZ8v~pu#PaTz3aq5ts3|j^N5)GQoiX;F24d=Nn_o?K@d?eT1y;uon&#Cd( zG{yi0He;;Xe) z>z{v_tmh*j%H(u^+w`_v44GK2(_v#>kurq^4r*+w5ys8r%J?-qpDEKBC8=mF*Y97K zfA+fqVpp#}*iXfDgje*n@pJI`ZU={*-7cmfP0_?HEqN*nx7RT(Rv!W^gnS;p zM$3yh5wX#DQ1X|3prB90^d0jPpDue(=NSL(DSrO3|IpC|OdvJ!TDuURnk6Y`M}#a` zpp}vMR8z+PdY!GMEaq2mz38WfMN6(IAw7)};Oj^_4F!!gtX9*nE>rNwT>istLJ}6j zED?sQyB7qk&{eOJ3;VV<5g?$@Z#$Q!%@s3JV636WAOpvKp4_@)ddeFNRm2LEEyP!4 z*1iPDU%O3ENpN?bAT8?3ksjQ@m zE-vqk%Rjqeoy#Eh{;*?-!KtQ#LZ~c|!&ntWp>*ExIUu$|qF0TeI(pv1pzFr*HZq1m93s%XgD7zb;iLMwzPwfNU$krtrQ!G@(O4s*|xMPQ9v?8%$81WGDO zz9z*L50l4QLzrFC7q(kp^mpnC^RS=X#{3}3v8jCUo*9y0mmM<==0$B7l$!?pm4^!0 z!tT5bd@`X{BXafCx(^a5(WbRT95A#RiOrmkUfuXcrpM2Yv9Q2}^CAsyAi|5$dVe2W z=?-H#K#ezdIe2_D4`)x3TgG+S40o~T$iu3^ukw#*UR={@vK2>RHsEtW97;o4U43M# z&UQBf8ROCRzZnZ<_iQYE{9T(;$P4GryqP~<)8CFr{S6Jxjn9D&2~gK62SQn(V_>C)S|lCRs_n1}6~cl2hy7yr}QI2XlVPF2^$kboKia6MdLV8Q+!3P_^A zKTjcpVZ)b6Q1nCyUaW`LGq%jV5~W5@kc_1+tJ|ozczOFI&N2VP4NXI!SM3G4F)UwZ z5#||vChiFz*b>%#KaV0Hpnw)`vBFCGWbcQ6*1hClJD1(&=jT~*v~eA>`JLx zX15i5kcJ2pLe@;~@z~a5kZW;1WJ{ew;){@i&D;9Oq~I_>UcMIOgR~l}23q}A^Q;Iup4lQ{u zZ{v&@pd%<)+{PaodYY}zxD@5!zu>?~4VMS2GXL-!JhDyozqoVnRzZY<7sE#jEg(gT zc`^5et!8C}KplY7At`y`t?sh}9W4p@8$_|Pk5aK6kjdAp56*95OGqrjFNPiug%wHT z(^H9TQ94JB_n$sBpwrpcLX_Cz3b1eZ478;ohh&VI!I4oOfL#eiV(0$|CY|!Juxsw+=iIZ<4TWI(YCaEjKRca(Jl8Rw@c9fLeJDlo>aY$SL^WEl zmBQTZt9R#gF7FgXfkRFD$G$& z2UymP2+B)E6E_O6SAPg5heF^jrC8481elI!s{Dk%UESz?AHm?WrFBB{)|HWitk1JV zvGrpDTKW=kjr!=uY)u_{*KvU@hF5e4s%D6U*ZdFPnt5Ff-%d0o$tO|ou)_P_MqA{M zcm!LiD<5-(ynw|#+5IwN9L3LPkrbaaYu6t}frrH*=jCzT6CM8BRkCt%oUA3)vso*= zt~iVV_DC3&iHU_xBaewPDfYPGyOEeQ5+zt1UlV=k8vf6PF``|#8*IB63p2f!kW-@2 zXBsH39+DvF13C!t=kI?{yqho2NX<3Z7L_31<8Cmr2MR61kgc5kHTH8r3FK#^eNoWH z6J1M!H#@tOV~Q&$-l*RpH}3+@mkxJz&e1h*i8;1=91Sa5ew z(81l^T?Th|cXxMvbM8&fef${s;G_5M-L5NkSC{?<v7V}~{#l{TE#P)W@gfF}D+D|zV>LAgjFTqFjLTf1u@hdpRN{-p6i-f$Y zg{voDVyaH~Nt0HnL}})S`ecQUxdi|X1mz@l)a8xZlY14E>4X^XNB<2TxU2@WR0FbJ zGK*TTSIz|0;d5d*e-pSo_MdrPv69kp6CbeVO$+zVxwauAi_t#`Dx76Mtl1K%yz)584(#G3`I!(Oy?gXa9PMg97chcAY>fO~RXr++$>^H`H;qzjdxLOB`&epe}&UufR#%vS~$Dq;TlW7={vwhLtIW(J}&6^X6Os;+Xq1F)nq-yyM)C5lqVL4VIMyU+!vprqM*E9%s7Rd z?`KDV+2LvYPm+c{tlU+`KfDT%vMq>R8oV;jzyhL}_-IiWhPp*?{&jH4-TA(nA=>Zi=$TNd+nDuB_0KHh)V-1B33vqiBW9_OtqLiU zD(WOiGtY`j*rIf-%*>I*cyS8+_<%qFMvnvpQlu}uu$-T+p)<|3#pNuGl+6FL6|g;@ zaytpANT`Rv{Kzh0`E7{*&PNalErFKNQ;DDQu^erBQB~&&3WzNkfL_rKAd$zXx9^VA zCqLTf1-Hgh%Dyd<%AmnyICLMrhSm4S&C*p~>%_d3(w_2d(8i}nA|Vo{`RGoGivWYK z_ViZmWHKS2s@}T(=9Z3`sV+SKE_ndA0>Z!;o@LSOhly)iYGRrFjt$R|iUUY_?(j6I z)naAF!a=~H^mOmkjebhK(Q2l&26?omgS;-WV;$bWm=ukKeN0g_#rn*Xi-kavl=KHh z`)b>@LZGm1>y;QlX)6$y8QXP)er>Kbjri61e;rV$9c;TY7cWF`phyctwl&M}^ zgI_HFd^B278NlFq&(&EWCqP|7wIjMo#6%WR7bXmjrMYq5$xYfTM_>TKwUaInx!@kP z^=l8Ja1edQD=v3yuO}@U`AYTZst;uZF^;+-TUX0wbzJxB^7&z8nBuc zJwZC+px4!rj@D;S-Mi7$m+uPN+&k7@H^X`ET#1fKmk9 zw!KkI2(nPK01_G&NVyCt$iN(2#^ie%3DEtcO!91Li__X_DsntH#7(g6Tcqqi<}WJ} z)y~U|-VyzG7L6!WUDTQdw%we-8g_IaxxkASqCCR$K0?-)7s3Bzx2zfa5? z4OhiB_nmyUvxL>91*!vT#*LrV%t?CKItt*?P>EmJy}_f^8V=U{cQn@Ahw5Z$Rq{Xu zb_2SPv|3H}2RN_-5AJ7WLO`vS@d_Jokqgo#+5*h-UxQ=Wbxr&=%Lz%-g2hE5u+f8| z{jsMKx=%s-BZ+JXg;pVX@G<&(LhoPzDRu;K1J`DodZu}NTtQx51?`TGjxID5;ltrr zwg1w54EcNgKtrN2LXQxkJ#6f5K8J2#{KbXuZfx!5_ zt&5>}#tJTgaEWMmNmS&2tpA@6D=f@3#r=N1I$-8-)ar25La-m5*diyYhjbG-)zj+F zG?F=Hizf9LKXhJghe60MujeG}WzIyo(9Mkl|o&{`l)(vz&A4J)__;JS>}N4lW_ zRpUYdg4{y6m4)T@8z1b9YwDbkkn+GixG_Sue{nl~zJ*{0QuatVIiav6nDS%)wpPN< z0tg=PjY1xqDy`18AZY_{dstBOh7=8!r|0v zg9NZ>fK9ALWBq%t!@u=ApAe!t6>055Fx{M>w-G0YN+db}7IFpnjMb2WbfU*VS4?(; z(Uv9a=Gy8LR->UnI24$j*!r>5W|H0-`<}~%`nczZm@&*lO4p}0vALd%zg&Rl%OTjg zQrW^Z9xqy=h&u;|PoF+T)3NfUv1M;xk_;^4mC0cW3xda*+8fnc$BZl0wB4(zs@GnN z!;q!mvfCZT6!@+Ps=RsbsT&NPgd8)FA^s4aT@)g!KYF|{Wu2_DG<(;xF-Ek|z(}cm zvVeGjtBUoIb|Pf_Unj@XoA9Bx+bNmqV%_ucc#6d5=WQ|USVv`5q##tJU~GbtsMcZo zt8BXaai6ksF~7U@2SET24HOO&=STIb`<-~5K#C7cI`d6CH+Bv;p{<^?LO=SJh=s3;Mh1Tnj(+7+?Gxw|{d#tz`~RAb=j8MRv)3Yt|k zV3&2BMv@LWI6ib!x_-t_`o6@jnnIcQ_IG-?24zr-=k2@jd_sAwL4e<$x&3cd5&Z-r zdKRR_7(^t{-d#^VHZqAe#-_>`yT>m8wJ8Swsu@9q_s&QcGZ0zlF`AL*p+ug#D;`rX z0x$IfNTz1dTY56+Lh9%tKjzVmdd0c;Nzn@ANJ(1rfRS0a-U)dUqy(5SCioPUQcZ2O zkCNeyY7ga}7y5kzFR+eY%Y8-=J>@UpmXky!lS@k)nkt?kYW+XAyBlP*sTv6ZQ0jUe z(JRUVUDp4nd`fpY!*F%Oy`a*QW9c+#mU6^nf*!^Z*IHu;~~M zukjacj3g}p8x36q0?tPXjPExXobq|Sp}twXG*-VomcQ|l@={n!o&9`AT5ved9pN0l z;WDK@Ss^n5P)3t4Io}g_dnmzkszA}^9}rGkBr^Q>1{#T|++h~5jk+@Q3de){b|k@~ zNLfW#2bfr^lG{f0l{is9%DTtx?M&s=inh!KMFgWM$b^JyMB$>8Xs||#E~8lL926sM zq?JA3d2@J?{YipRd9oieE3yB*L|NETn=frL^lxlJTD*XZX^2sq8^6o!8dE#YYWF<+Qm zEG)20P7t`KI~YU!yMg*SEkhJ8hAgB7OO*}||IS6c|C*W%qT`a}&mn`QS5qJcOS*!r z*$9^%lwe`FjJe8I(mbx55@^pmfdueHqriMQuYfTn!;lFG93frFrMY4_8o&Btg^0|= zAtm6X5J_hfI}r1)&mQJrldKhvBHW~{hVf8~z+5QythrI2+k6Im`Jy`!n$vO@@r+)^ zS@5rO`_)J)GmF4VNeL$u@UsvB9$2l@rJr?;PoH~EVt9-GfzE4?t%|%h8bD%%#4wzB zK0my5Ri=nI7xg<1!sv`p8=1YXfBB=vzvB%%uuB$3Vx)O&xX_p^oryvbP@!Pz*W-rzPB29c-I8Pg0N}C|u%hl8Pa~XxN$l`$XIYmPznn!~7Z2)i>PIoZfnC zIcDhbwyV9sKUvI`01QoefhN0BMIvCI{8Nnv+MDbxAe$E28-vYdC~l?xxK)e@h~3C= z(t$y*?e-Qit6WOcP9ij!IWzuEQsgtVQ4^JswYZl#$p7!s;oA8noPW=>QBUx$Z@Dn$ zl!u_WD4p?>kNMlHF98#EIXg|JL!pZGM-zvM>@-_og`JPLeNvDr6OS&&pnpE~WzvSSRKLy_YC;hmR9U(ZG`O>#ZuPliJA<2bBvuub&0mym^3wR?=6L8{b>i!|BlQVoG|44@S z@AH+83pu0D`A7moA4UGIlW!dlYZk6U(T2)=;8^M%pzAil=^Xo<117jLemQ~<0nt+l z@Sdiah6Mo0iwr2pdxD^4Da_ZkIsKRh=Lh85U~Y;YuCx(3ioq%&q4Gak;iVy!^A+(y zQwzo`0(1v`_Nh1nx?`xI>kIP6+N2a8$y@&Kv6sp`Be42nfGUkF+C8Jy6T~{Dh%b~p zklmxw`R;}zp;(t9jWn*|S}1T{w=;v{MoOQ%1m3*(An03q?pB_*#EDU&EPY&TysVj})%;b*ZPVjdD9?qwr!yUpjg^s@q@bsT-PS7V8n@G_ zCgjDy(#y!d20)cQ&`$+KV`kE5c5DxW-7XCL?bfZTx*aW!a{OVnR6K~OR6(%AQJsMZCq$Lp=tdmreB!qVIujHQekHILQnt+tY4!yLVy_3Sk6nmFjw-5x{M zUM_T=9zp8WXNL#oD;J~bS}i4c<~QKVj@QHD^15pZ>J6h$|9W)1F8_St95kEg7l`r- z0Tmz#lZSx(a4z6mRe6`|#rS#J1oQ>RjC*4y58iM}{l4^2r86(T_3eocAvb=oqOh#V zP{5VL$}%2PvAg+Jw1Ezx`NoPVvR;%^Kr>cT^(_QHfkL~%TaB^YT<~<81*dcA&)eUSk!hg0EeX__s8#%YmJG=~B%WT`O@#11Gl+`*o&9QFh-#Lg3_r zKcgGpPJ1}o=$wRP6G;%y-so@N7{a_q@({?NsYOyxL_X7XV;f4>W z3GEhQHFX!h5W{YZ4;e~qfLpsG-&w#4(g9LjLGVc|Cab@e_$|OXUeR`EF62j28%1H|T>=gy|NAjUI&Uh|&TWsWSc zZXhr%`Ehv7?KBd`!k|_8JZfg-wea8~%oj`r`_5#f(QeUl%7VxB8@P$PVs)VtvrA#2 zTgNgtB@9g3psw{;RCzeRoMMcPf>Z0NNf8a>^X7htXoG}IP`#wo5T!8d7nm97p|U-; zf0qSy$S_Cvh|I+Y4GBs?(43!4-!L(d!v9_9G$dZ5o!Q(3vz%Uyaw=Ltt1uBgaSWpJ!t_o{1C-&JUe&pI+_3N@Hj-BrfOeuiOIaZB?$8C(Uk- zC(qq>FBY>_A8E>dSQZ7*gjM9A9IsG95h@p}mT1+}#uAk9*q&Yb(Te`T9??f=6-LaX>BCYjjTc;f$F4#gwdR#5ybm*I8nUi z7?m&91#Ql&1ss5G3c?RnsMrzgy!mqTa37>8$Vec$q@yO|K{ldT1+dHs?}w9hG}pm} z_clTci$$@11Jk)fzX2H#U12FehOxgm-ZX<|w;15whA zh|aR%ZaHi!6+(xlGE+jCTgQY(T_9buo{21RGK%R>2lhsMYGpD7@BU_k-IAH1GkOME zAc+gOv5$b|!oZR4gNTN~=)%GvbvWr%SEyLW&rKGBUksVEhTv z3un@Sjcdo;xgMJn9NOuj*hF?9h3Fx``Mhe&P$WenBc@d}$G$6jc)th$J~p8#1*?1H z(ECV8W@jTOYHW`ltFw-Y1xr*>r^-3@SY(xPQnvIKZaN(Ph}}~++|?yU`>z)sMxET& zN-)ACCPT1B9rOOO)bSz(72ZO9CMUXT8X#ei{@DpARVNdThFIynd|zI2RYuOX+bjOa zHz3nWJl6SVQOcr@##Q06+2IFcE=8>(N-#oaSNt#Ha-KvYaOvuK{l&v$tm&e@ic>1* z^nJji(|Qr3LWlp!A*Jfpz%tjKaJFP$tCdp~h+S0lOt@MmCE(AZUfr zCEKIkOL~1(Z0OQm>~B5)`|nU37d%rz8WnzMuSVL5v>+W&;7ZaIUvhvDZfnd74Vf~M zs@9X;jI8c`WwrxXD89O76cOd5l;n)jyXD z4>5E~T}~qRqy2Np9=vCX{RU)b^~WYAtTBrc(?-g|f36|oNVazD@~`^0Wl9k~z7@O03>U2$;t#cHj+zaG$gmp1U0 zaJ-a2&A(EMH1x+pdYLb3g#m)X&RaL75>CZC(xpOJzHKk)jCLZTFb2`Ow#z=Nk8e=-co(@->10R~2@{FAB;qi0A-WY&0g2`Q>7Zah zrEdzW2yc!==DNiV)?aMoNl#xfvw~)~aZhJ@zelYuUT&9Oq`u|#oruItlMuadT>gpuTM1Q+yBqt^0s$?638>|< z34sBAR*dR)EJ67Y%;vwn^N7g-`TYj%bI{)2`T)2=I%pcSI9j7qbV@ib%x!;=k3R{5 zBhbwd*Nyv9VuRd!*O(F7_VRYg8prHYAs3^PGju#*5L%4jn&;EW`2!t|d)`45o4?uy z{CY=aV7Xuu0=fIFhp9&SQuLYJOPl zv^}4;+2@OHtTfvXkPAEvoO!)j(}VTpT+Ujms>n9x1HR(O5)0IsZh*jQdvV+A6J5=> zVBubpi{pf7nV$)qZ}%H8SGzAqUN203@HjRTRZs8v-sEnI+n&miL2ByC)tRmj(^dC^ zvU7EJ{9LYOE(4&i#NT!JN~`QM+APLeU!G72k>|_R>m3&+DeIM0l=GPyWk`5nbE(1f_r?!3OPe!Z!E zx;?Ql{ULu}@)5W&!Ni7wS)^L7<~-JNx3;i4wQ}HV7^%?~a;#1PDkj|Fb6fT3>vTc( zc;D&Da0f#!Plf@>^+NBD&C70~qF@+JIu!KB>etv@7;(i}@CqXV!Yw~f9wEw^Nyv;S z(Yh5Ge)@sK-!RQGWLe6p5vxqkl?A)xxY44N&$-Qo>HW<|qYL2pcLntFC;oL>vcgi~ z2a_yEP=749nVgmJJ?)L8db~9y<@J6Q|E7h@Pptr(QrKLh|4^hsKg9T{&bMX3Y1G=% z{PgTZUFT{Y3WOMVgwK^>3$~8buu6N>RPIa6!CF&2uY-7)-I;#ccJRrRVzr@5VEvL~RmS)EX zN2>b1dow6ZeTz92>-mb)@l&ne@l4>ZZ}){J z+Rhe3ZG`1~#O=x47FO0KOX5pwKxZ?VnZx{a_stoB=hj9Zj>R1-aD%k`r(0oPu%33B z`(vWO^XtrLnN{vR$HN|H4EY$fd$()v`PSBYY@Din-6v}gYmHDu`ux@Jiq+5g0xt)| zo^L_nr=H?zD5X|Vykuz{ZoQ3*i(r+;4w&vDUGY{lUc%h(zX^&XLw#Y=Fvu+U?QSge zShQ|E-x18qH*xdy8xi4QdT$7lt=mWkIv*y`0BC7GnE(cb4=(w>y}6>tzamw7Q{sF< z1c;aDYL1*q{oVZ3O^p@JjAz|HNSnxfq08x(yoG(q;x8wWv<-ywiS>|yl&C7tjxX@9 zrVcJI1g|7ti0*ESSgp(&RyV@#Nb5^6xp-pS)GvpC;C)mGi8`FEiV%W>t^$ZC2j5EKIe1zWi-BiY_FnjEIPsSOX?-55*r6 z_7Lg|ir>3Xe@fvBEWj_^fl-wn2!K?y@L;p_e1Ka9`HM zxqG1a6}h=AGzx&?BeVv+ zT}0WXLTXp+xAq{O?Pd+;#qE0Vtb*6{Yr$DRssPr)H@)naIQIXZ{SuPDHT6@~S1P}T zrAp5G4Z~-M(%J}KVD0LoT(Jr|8{1A2p-rwv^-%1W* zd&DlbOh#j65>>wATC~`FnSt-=684MB8#0jvHw6s))9+`K286$Q<8C+u*GqarRuQ2CU!wOgpWOTmFIYi@t;QwopH7FwK>y zzx?ArFJ`=sP7e$ktiy>G+AJdSqxX-L&leq(Kg?QlHJGR>cIEbj;Rw78+gZ0=oKcU? zBKGZ$>Tr!uweGBx7A-FF?e{NLcq)zziZ$~;ROFUgXx$X;rF(8hyj=$6of_Ix_1j0a zP1`(_*4HE7I;`;R-HLOT9-3MRocFlUVEU7YA~;L2d!&)uPYbE>O9 zk2+zhd6Jmw9PMS>YCBoPO?IQ2M9(y=tzLfK4m((=v|T3a5H5SIZxO;u*va~k9wfpJ zL^yid?EWxN3jyd>)EB33>7+GX-^Uc^uPQV;ceI4(?VjGEWG%7w(NR8jUChlXy=>04 zJzuwXMDTwG=egOl?l{StF9X3 zOYCtw1Kmx2WddDKe$``EQc?ncXDZwT-K`=LXB2H{h$zRla67GBwLRy*rR0$5F-hFE zR$IEMT4xp1{J06F1oukedYp+zS-hBNZF?9g-y3b(y05g>kCs5TudiV*bS;AEp;Vk@ z)Um%lAN6~>qu}tY%FK2?ygodPt=%(+mBgD|nYDjCPNe9;R634d@mvdyg}rNaIvyHR zq*~i4LQsX_lkg=li0l~ zJ~yl=tu8JrQvoJ7VYWRV=}5P2jgq(+cMZa<5iB;kjqziXNeqH1B+fahGRP+1QucQS z!%uzIS}*Va@!$ONsGG;o0MYoKKe*uVo$RVgXN;@W!tq}&K&9_5($)sycm*3IMrk78 zdKi5qkm(Q|uIlFIQ*f}dlKMJ*j|5a8^?B^P2ua2x;%`0>6qvc8$>lYx3hPJ-MMAV< zNQjF;cTh2yZHkKiju#pHx3vYGYSV?HfUC1*uYERnj1@i)5BT8((7LlFIkz}kWd1hy z-sp@@QJam`RiN?=y(;12oAO$_t|N^Pn(yjqS%*JR^_#WD(-*;?CJp8P zs7SmA){nk^tt<6YQj=5gJOhP!KHn>jg4QNh6|^zjuA_)wwxh(MTggMwV}Wsg{p2@F zcyQ}5So93dxi!q5jjz9oVx(&q^LiA+5Div*5T9Cy1=g8>;>A1*ee)ODPKr3!Tmt z7K;wT`jy%F<=Icn&YSZW;{-!uFV~EX{-S`-p8Ar|57t6@Tgq-g>!!TTH_~!ruJs>JhtROmpf3*4MS90~6Yb z@c_Riye1da@B91dW3+*WZV@ZL9-p3c+)mbXTn}>Bo-fNa7MHZ$PHtLXUx(d%l(B)Y zI3~egZ)@w>uQR!Xv`p|9>8;Y6e;h=6G_Pi7yk!$MTa0Nrb`BEJ-`w5i(xl;EmThXz zBfvY|lEFwe%+0~&VC3*GUc5du#lX{H>VK)KZ!_utVk`y8l;Nw9zk2zs7%`p7{g6+{ zBkJ2n1QM9S!!ME|;OyO8MltKEn~Bk!qEzr-N)+3Pn<(HP2qvy&n5D8-6{(<*Nj2B@7^@1;rVB+DD)n3JTUMHScq;1D2-xg<{n&JSWDUK)yu z4pW_0xhvkL`qG_i%pGntme%xa!KlAShWTruqRHnbSc1)8?TRv0)W-qX&{CCx9waLr zqHHo@Rq^48i8LKT=OY&e7R9R5sh^joR=(0uIk?>s5=iIFmTEWIdMH=5-No=dVIZf1 zWh14Nzwb&`Z%-5^|I;`~Jbn;n#taY8(QdseU?Wc{#LReZ5-ry*ElznY=$J{My&8#>q@%)E{Z% zlKv3G<2=DwQNqwLG|Szek6--|vd)#>&~-ubk?G-|YMX{4({n;>muL}-d zQb;cy2n6ae+L<$MxUMR;H^q*BVaA1hW5IqbonK@%`Bq`RI?KAk4y98X!$Z+r1_XT7Xh{d}J9b~vhaohTJ{pKRTV z2Xy{ier>??e4HCvHR)7m70|Eqb{My21vzWHfOL?^{2SOe=1*XO|} z`QB;@5Q*Y5SsIl|5{bc@{ANf_3eckk_(UD}W4*H5@qB!L?tM*ksQ2JaF7TunUuf%X zZcHfSnlg+~wR{lVFQhyZC5vU{mIixcb6ciX+`0+5+mdfYGnfsas3VMV!Iw^YMEGhm*=_;FA!go2M z9zZ7~Bt?X!N2rITL3hjwZ@=I>NPlGU zw~&6t?etvVx0KY~uQJ;TUnAFyGKY>xbmFhh$nY z78E~h460IVU zcfE2IOrI~jiM{H2R0;!!8y?I)9N2y1;DCaJ>ik+-|4mPpFZKuBwq#XtezOxv zh0=Z?Yh@wYdHUPb8)t9j+tb^HSKwD{md7Wr?S!K6;H)z|GK{E+xYBU zP^y@c*)?T_!Sxui-wo;?a#w;L6)pB)L(?^>Cb=*T&5s$AnL>oK8F@S*k%-J~DR^YV=4Lt6_P=?KA^LL1h<$3#pp6X2@YArTfEITg@v@7Y)t zRsL(6=i?ay;nfQ^_>lEf3zhhoWXjLF?IHa6%JeNrA!%~dp8_y?|+eoV3^Ak7nTTIu_ z%%5f1w)JH>h$OCl+4}6bEza~Thtg~}Z#UMq|AY50ogg7O^vlm!+oW?P8av4^W3A`B zz}QJzX{sm)R)#3PSO$%{U|-tiF7z*77z}EX2}}1RYLrhJsCl{SaR;wo2DC@QRcN!LUVa4v zCH(J1i3T!Svc(?Xv(Q6BPqvs0{c>^O;O|YaIMSfl{F|WUyJ9E{p=wp3sZKK*`e1Dw z9czP(2PHh{?SIz>&X~!m+y-Btx;SxgeDZ}pGzuakBkW@%FgVLO&nNq-yn}(%F!AX- zYp^bA-Ediu8umgCBK?X7N5@ZdLpQyeCt>fxzgCGA9&IGcWt%Et*K+ zE$G24Z`{SA0#;EV(1)Z#X7vk(^{FD4Z6_ii*Nvp%MzEtDAwt|~VM=HDL>xUCeR;$O z|M$X}r=dm=F_p0Yme3H&*H!f@?pdsdJwHE~KzKu=k*WyavItXXZ-7zkPs$8!xg|M` zxwd1YvftAA2}!8V2d=<52Zidqrk{gKlE{u6=}-Tq-_3;OpOu#Yw*^)0UcytG!_%#) z+jKo{lKw#%Rh8;EwHCoBqun0UJ5TWG;Hcc^~Z-&tKztQ!_Z} z^P0x@bW6;2ZIos#A#x`))O?i4@`=F6yIdcN=kGi|<=a$F;~Fu>L?OVFbkm#9qB>uA z^Mb`6N%(V^B?!1Htku?|wXbq1&f!-dYb`j-big?AXxtET*n1|=#|50X%igxMqEc15 zvO@K@^T603emW840KxH(-ua8r0}@g~_^g>ms_Ac(!?T-bUqx@_7wl-Sh&_^vC0R`T%2kB;R%et|-?2i|u*5m7xJ^B`Yv zj8{8!#_j67mk)ajjV~}WuKpL6UhLe=QZ80;zv{_<5_q-o&I1Cmsh5kq7nm4FSJxs! zCoiLOB-#m>Xw~cVcRE04gin`OSO`w`V1OqX$7C$g z!Bu)=)6kdqucA`H{**!hsz}q)(z;x4z@|Ijzfwp6uzu>uA7JI80DgIsLMmK-Phw2x z^LR`&NiWy3AL|(7Z*wk0&myv26*KI0M>VvI`ygN$Vnyz383dvl<2yD%0D^0BjK7O31cg&vc>XNor~7 zh-K+WDOlbCD&(5I+G{e1Ra6uVH9g+V>8yd=?k0`GzJ`TTKL>!FVQ$rC*5&0-D)20l zS&a|^>!k6?gB$B|zbq8E&f_t_|9P;>>r~k){Zknd(bRVySdr;3n4ZpEa~W{wY|TT( zpV1&M+`wfxvs%^GPc4m&5A8U8bJoro)`Iv9p0t(X6Fi5fA+P%b5w-xoL`&l+tFj zD`%+?ulu8|j`1OGm{{pX%heVahTriB{UZ%Gv%1V1DT$>ub1<*v(W$4-RL8{lhf2}F&;@eP#ofwtYg=nuX}lU?V#vS z_MaQXWvgR02j{)%85ylCEiJ9A(l{-X&YnOZKG7VK&huZ!S=5;C1j&SRHqO`qYG$^zwpf@C4 z5@+r6+3VwlrFEObwtcnSa#zLE+p~jZjL0um2gwp)G#H*s063l&MaPN62wEvqhr!Zb zw{6l1VVT1(zP`PEA0MmJXZO3ru~E+=004e*N@AC3%04ElsG%$1bBwyHHM4zRTJJ9nfF4ig9K z)7AT7`tYa5rgQw~UpBT-dxA$Y-vJO8}cqLW@2WEXlfG^FP6TUWo3*iwDk1k68B*U zf(cwu`=_U;hX*h3yZ$hehA9%xl_a`Sb25{xhV3}|!d-mw{g+IJOTIhmxR^yD`d;lF zX}i7njY8}-tl#2Qt1z{dm*g^X= z!*p`i4*6?G{s(c~BS}cr=Hy9vRQwprZqa(Y~--OF3C$C+iqML99;OrGW6Djt)!67N7T02o+ zwn3)fq@7%aEg$QX^*&=99t@%2zB8=L82)eIWG9mNt5vv23RC_UNt4@N8Hh*Va+(Da zfI#D(-vpM`;%nx=oJNO4C?D_dyWeB>S?G(Lq3m52EjuLV{hss0u*hq2pXL!i-1SQ_YLf0JVVz992%$WweV$uvgdhaWlwu8`_bz7ven9ZC2N*Bz z7t8Dj?*-CS))y019F&@}XeE0J2!ZQAIyg8uJXBELv1Lyne#HCdRb`@w8lPENS(y6p zdW-l$S4Rw#_9wK`B%f<5mvcP%a=moUi7qsSqf|6F+>oT;YZ`~nFEUzF_r*B!noU&! zPD_maitsX|6w@%N|@b;xo`xMFXuKJj{DMB@kNdMjZ3v436Do?aGBVJQcXPFGKb2v zACJ8Lijc0~o^9YtkrEJH)&gK1yFkD>Xk^pQ{z+}ecFzd1zI?~NWj4a5pF!#ejEx1? zwUKnaH$mpv8mAK`S(anIOc?#o*RWIPpfLIB#+Exe>(%gRDX7@3-;n*zWz_EdA1}~Y zM)isj!0xt@Ln1b>AB+V}1`|RT+{v|taFVNh#c$lQBPJyR86QHnbwI>gmK8EkFoiev zqZDrtwa3Z&mBW$Znjs0n==Xd0-zK95X`eKwdtG-r8?9xPPwf8SPK^*UU+T?rnCc~d zk^&7~%UNBp1mUq>mA!sxb1%eIZXJwwNQrJ|sp@A3d~AH0l9BBU&Ab*h8#Z8r4gcxZ zU6{2-L#ny_POfIr^u9508b=f-6UhVSOCg377mm7swX^uy=AoU84H<*@W<-Y1#3c^p&F*)# zNbh{KX|-?|4mDB-8iDd#f-IUMkha4GKhAOZp6L)#%CMMy#M*icHlrQPC7nTv zsTV(9zuld!exH4kCAflp5uBjYJi2~XpZ6`5{JS8_VCEtQ`)$POF~UnFqZZZlCBrZt zdK|BHIoK-RIFbWhCuQ!~U~$ zA*S6&9ekG3_@y_mA=MEH0l!y$#qk)V?cpWzJvCr~>XF3s(_lu{Pu{fwo|55xq`z9> zHN>rl!q>hMu}`J>V4@QG`tc57w>mFv5aqMno3NMLBgw4CPSk7DTz!Q`X20o}&5v&X zaYpEHyd}KFh&{?2p&TKay`nQP>E&fI+v_QJCa0)vrtt!ozmd%hEvc5Rew-2hr+{)X zXmL{KuD_Z-g*w`Vg4Eon7dC3V+Wc%0gXJ*orn2~QfWU5hFZ`fc!Bj7^u%*9#ykd~B zdz-`uaVbh+b>BoIFJCq@-oiH*AN!S9QGK+7eJ#iIS9V=sgc&itJr={uurk=@>9KF4 zgdkT2jl>64HDLC>jJ7S)QQ?w-b6%Q3B1GU}KP+6sqX%^s(`MkS4I$4ZE}du3<4S3C z*3&7(f?UKD=Edo5b4bEJE0t5)X$js?M-4V4n9nJgfDv!IJ*Jwy1r2Q`=8KrVTiPV= z8_FL5_w)D^q?e8vm@37coGjkvCC124c1vz<&cT7kGIWUsL9Y zzk9~?Q^6S<01*L#g@MGWTK~l$7O!6H-GqgA=b+QQAYG;N%?UZYQya8kAT5U!VixY) z^TdNnguyfh`<6I-@y{ih5ECZ>-ziKX)y=PZbC{yg!9f(zwxg-VJVXb*UKZnAn!lfe zCbLT0!kC?%ZJ4bn)k2dc*!t^d zF>~jq?RynF-K|&gFcvOnRh928o;RU4)}tc~{7acAKF_}5TPHtLzAUzX-cpa7%+%cM z-FShGBYwS{PGZm!tZ(qt1RmHsuAe6qf@>1Yg!xhuAWEZ%6q_~8HsAhD%N?qo+UmwR zil_D$$YG>Zt{JO@EQMQ zKUHi$k5-LA*B3=eNuac@`pV$hnnkd22HuQj}1 zXvbDfXNxViQGshuSA&CHycAeQmuDSl0aufa%^g)to{f0O!E&_g%k??spXy3)1_Gd- zSa`nE!xTlA#U&p0rD*JmNvKI9Fr_%x z)5^NZPUk787gpHzXnbnAE_^!+M&MrtC+|y_D9c(L#?|mqJ!yU4gV&2VloUh}- zRS=1{3w1&`DiU(s>X!i!WITpf-!98XfuA!iZ$V8|nw{mIy;s6sr*BVbekUZ(T)K_N z3+<=-3B8KrCk$Ew7d?n9$181jM@LPBffr{y^m`w`)U*st7tk-RWYt)^#X8%W)}1in zmHD}ared=d`9EjmclY#2Fh7yg)$BD{-(YHGNSZvqjTE=l`@4$yLykU}$wpfpj|=6z@(fL2?7(hMteB?J zWjsDE?)`2Ov3n}QU$D-N6B~@AlzzZPiG&TkHF>cvY%Ixbo(Z$S^`Lr|#=BM3|gx1EzuY zq~#I)c^d?NJFNF6m)Ah2(`x(>Q(CCq{_!gMz2EeF+c8fK^GDvZ+xeWL>G_34`xmQB z3_ALZTKi5D&Xq3~oh$ovr^65dP#)ur^LR7YrCn{s7mvkcWBZ5r_g$|WrEz09GrnFH zE3Gc(!or@1Ljj+XIhWgkA5KEJvaDFvepk>}GD43xk*me`lo!E1wl3a5!$RV1E(f$W&RAx6L?S-zB}lfytvRY>vZ{^gs>Jtp(Zj+SOkO9l=>OIur8laOu$ z?`3^;!=Pp0ez9XcnwE3FIl9!81^fy^3deBR$YrF4=}m$QhYeNBV2w_pq^IBN@9$TP zL1x5r%|u_bZ~Ht8!Uo%OSXj`mDYboD!W5SY!IGO$XRRuz`~LlUa8lrJunSFtg~0X-+2;LnD4`-(+UquuJBchl>)6lMy;4Yxw~ubfUh~?U4n3}gZEb(qv0VN;=j z1v(;{AG7HH2C~w1K`k0=4|{iXhYZQUbJZjWrfgOTNQ}^-X5JtyJv6KBHEa&dgrVOr z6vqYcCw)yr`SJk_lw-$l+l8S7X=)nuebL|CXm=XEv#uM37rk5cIf`M~o2*%Bzc_n4 z63!OzeO`YR|0)vBraJQl!;$nI=_XrapxIbX+#4d3MYW=Y*aF3>`R&=0lgZ1@r(}0t z>5PDGQ@Ft~#J%xP7c=I9>`rqOLE|f`2|ei-Rv}kY`Ox^fe4MYTv^<xR^+1`jLd8F4m};jpCMu8$WuCvz z5qKpWVQkU?K9ptz!!; zMdi}e!wCG?Z^`>}ElG)rp+wov`po9+c>mbO?ra}lt|Zxg$^@}OtH%4KL3p)!wr(qI zlFC?AL2ys=9E%r23hw4X^Cw6yl2#xxmr0(_v(X7h0L68xUZ3|Yp*nZC$wQUi{--@{HD>J8hru~!-vYCKbplOrfE+f=pKNaV{fO=H6tUtP zg1ASbz9qt?J%Zyld@gTivFqT#%@HAcvNf%uk$=WAXjYlo%UJdcPwGF(X}XsMy;VrI zTu&)8(F3Jv>azWP8Duz@LRsDY{+LB&n-))eLCY1+WR?r4s$=IqqsH3W4GHCI%fVb^ ziZWA%`pxNT+;n*9yU@MEMfTm$m_6@@67q6;oD>j!jK(8C zzf6FUameqry9Kt6-*rmhHsBuHXsAH?cWj6 z%8d;Tf;W92-I$wCxYX(yM?AWMl%RvJdX{r#=1X&~&!aVsUzWHyE5{X^Qfu5DhqFO$ zcc=1}jDoDr7n}1_^&Km7_0Cg)b?ylScOPB|>Y%Ow6yyP@`h4nNVq(h5XIz5(mf#+6@2tAp&SCI5kQaj&YDpESbzy5BbY_e2YY z4EcCbCr1Lkwf!-YBFiQTg>M&>{Vsbm4;#Rq)2~w|G}FYob9TQwtaZI~N88TYU2b$b zNU6!gP`nSHMaENYBN9hqQ+3fkqhB5b<(=-?Z>4WBW^75L$kw%a_AV;wHkzvi%`@xs zUd>{MarTD63Dgl~%FD@d+kqWkJCfPo~3kC0gheDj>7Bo|!qnO=gzh7r#ot0Z%%hK~6tDVw4} zGg9zRia+4}W^ptjcy|p+VNxS?#!NQ;#^NV+8ob;Mw&}^uM#~0)_$;nHfZ>A10OL$_ z6`B4xg_`>=zZV`q&$APb`9(ovhhT%2^ZOR-abn42-2fn5fsrzs2;?}L1e*J%jSq4% zu0DlCIBnu7Juh6uJ7wOuP2mlm7cRfN`w3ky{gX4}dnn%58}SVCWaw2VSP|*ZFEzP#78dv0Kh4?u^(X&ADK$#U z=F-~Q3q*WPt5=iys;>F;CSI)KaKW^1!?)!`FIxVv0pa;dUrCKqY1>KF(9i{6?-Qrf zkBIj73NFWIj}i~Y#&j<`E|$ibibyX3(k{%z5?BCIcR)cb?T`U)nlijY z@`?mDkLs5c>tJ#Kq&tlFEhi8t>hE2%SLWgLC8z9;3qW0(xx$Il&jzguFd~b6%^n@1 z^pt&7D(ItmjAa+_M`Py8li|caEMoR70ExBu*e>Wr0Fa@vs?hX9!i(~IHLh*anDzDE zqk29*=@pQBhBjp`a<$D2@*xQJfjiXf88?~sij|Ybn@=iyH=63H-k4Yo&E*Xc5DgDM zW$p6i<@&YwD_d1=&4z?<|D{HFNvH!ku4Aopk&p`V@*!N}gk0j(( z=V{XL#RF8o(HnP1(R8N*hv!gDYoJ^IX(e;E|6=p@c4_A`3T06$|6d$jkVo>(&f9kr zD5Pk#n^4WkX~nPhX*4$hJ0EwZKlEU#peF!dGYJKa5&2kXUOHS`*d@d z@~~CY@v6G!uZ04Rb8&X@B{)0PPltr)RAl?3*IsX%Qi|EqMzu9M?Z9qtLPeyR z^PHbF)l@!VRb8zy$VUuHlKy)88m5c67b>_8={ zqGRx|wD#Imcex3W!wAt9OY}U-X`0OLDj=nIzkxmpNZ$qP_B$TSBkML5(pwi)-WIL% zu8~9K93Z7vy07$2#(E`XgL`2{-2}%8@ zgMGY{*M~hmqf*UUru0{6G7=CH9v`1#(%+m(cpTfC$e$Hn?b=isSiXS%naO5W3J1h7JwJ==4W5EOjhN;kN%!EhgrO4iIofxr zE5j8sS>-~a6AHPn4|V%IY!yggi3G8I7%5Xd-8$u1(h7n5vA(`e@_HVI`?&zD8YNW` z{=IY+bJlteCa$8qZlJyqS0#144<>~YTuBlvW0Nqo$>pl1U)vm^cwp;@fMT z$EOu)^B#hz`g7T122#l z>$Edi@&1J9LCItbJ-i|jQF8KH)p{Pg+uf6Kb5#2M7wJfcOM<~?d-J2J z3;rQEl}PCjF_Tspvb41A8g{ds;}a>}aZpPra?&fwHY4q?%3sMIFa8`c{=M zu-eXHFlTOK2o#g4q_{UWFqvJ#WFIN7f0B`_p?C`%BkC$~(?o7{QmB_uWhhxbw>$Wb zKP#w)lDNcaj(5E5oG55fzpqPjfU%AQOiw#oufdWa=(QJJ4r2?uKS~QLCGe}SC3VPM zpXo>^j=)L1A*H3IVPp4aW$6T1d>4#LY#KNDi}6#P`q4u_hs=~9)~EDL$YK&s`f|RE zQbbQ#OTFcM*P79er#LcJqUG{rjPyI97z^nLCv+iuz(kdSYESpTh%h- z*X+16!f_)FOv#Y#E`-d7AB=>T{Z(vb>R~pp-Qf5a3hR`}>bxPRr1Sq$MIZxk{-NL|~^Ddv-zSDF^Eu72S&gRdu@lhl4 z5%9crbE|6ez1$U*9H~5Br#-YQCcLVBJrbNgggO&Aus=9ozPVZ+`?y#<8}sX4;hti6 zOtQB5Zq-YelUKlY5y|WwR!f9C<}v`?zq9jo!_@P&;K2w6UgoY51X5UtWp9W ztx)tyrM%HEh1(Lc2kjn+$<6J?k4X$^ig^Y=w`UHL5<9)N%OtRV&bmq8`jm-<1zC@9 zWFf;{4j|5+@#|-y)9!fo!%)oWb_PqAyP13g9P@lr>&Hq;Rd2r;tAEnV2jM}iVlh?p zNR{iBcu`2Edm_LOgpnPwLa3}^A#x>e?*eRU_J_uXo_XJ1E4S4r?rOKm{Z*+DF03t` zre%*88=8_f*{K0ne*A{lG77i6;f1i_AqP)cW6=ZdkdTBMw_&@U+YD>(zna!H6d#N?t#)XxsRZ{eaFo)^2hI{$@ua4f zgu(JH>OMZfW`oq>7d(vbufcd3m>J#3MPT*CSHtaa8(vu8n?vs-kiae8^s!Z!Z;5Yk z?PTS(?XO*t)BK0m^CVwZEn zuz>(KcIG~@eVW^)ttPQw%30>tZ}Us9u)hJxu1-);P|A$Cf@@%6FtY;oXJUQ+CQ~Uk zG0xly>3Zs(&1D2iks$~=!U9WAF*I6++At{o8tCvKH+(*1K^RMVNVIuM2=Js=Rp2Np z2=^iv1xQqcD2bE3hZfTVdl&frHIUg|0=uBnGBC7>cdf$~E^)2{dDHYvv(f;nTUo=p zD+=BdD?VS^my#)UY7*?q%+8P+E&`&+j_`T*?B4|iJy>qy4X#^E6PK#O_W z3IyIqwu#Ahbs$U_mUQ)?$3_?LWA5d@5U9tU0CwcN8pKa$1a>=o4-(?b(!O!7b?uLr z3}Qy(Zusm2-jn;x174EdXr;^JPBDynMUoEy*0)PAu#D<5M8%uz^D?x~wz*s`2TdpA zTgV%Mphbbf@a~_SB%RLLPMr)Uc)vK!a8@;GE2n4g2?3p<4}9me@+B4~#1Ou+xpq?@_D``IE%2Pe)|w*Lwj_K4Pw^@+JT1VzX-R%XbY>Ne8Q?+; za1|4KQBSQz7d{F33RTJd^{eSz=!58|Jbgi7{|rI*8$?EGO8mFR<`sfsZkSPiwt$xQ;`tG!Oo{!m00m$!R?`?s^k@ma8=l~NJf#2Qa z8~blzz_*W-ro`{2ggc!1e7fxV{wcSOb=(*giBbLws1J+6DHPeC_xs)-UlhXdtMEQH z*uS>tu2!SIT^kh8cA8IC>$kZQ@p~6lpZ*N&m$z#Aa%9!&adGKVJe|q3#9d?Pa6S{a z#P8K)qxW`sHThDr_Oi5&HmUpVTT|OhuP{+bfhE|HVztA1skW8E#bpp2Lv&A!XX)r< zP8fU%bkgMp+O6q4?bNpF#%PUVTr}P)ZQMV@M1?(q!S?H1$BRs^3WL{&(YK35T^xee zBfI6_tyW9X88c71?{W~qt=Lv(=duY2oA*$Erk_o&q@n*co$)AJ8COauF9Z8ybHd7| z;YMPY4%LeNh#ite5Xpoh5>y=(%_;I~Tf#rAq2o0$X!IJ4MckXXzNM*Yx@OXU{q_9d z#G>D#r>5Bo#(}q7NWj```k+1)AtHbAzt1tS8&_y*BCz>w}2D#`gdAIep%Dz6g zSGN#u73BxRH18)2?mxOWRD5r?8C|6L1CH;Xf1<^tEJ}}~0vsIK*fx8z#Y9>o$N1ef zu~e_MKUnoXoiy2@poBT$2s>2W7z>0?W_{RtUQmZ*!BU@S6ldT4bX~=r^Y{85D%ilN zJV0)HV|oM+Ckn+Z*`x^G)cJXx>?S&XgUAve-1Kt&Vm9c!C@>ess(%;^%Fv2DHHnd> zfc1ShHCD0GuVNe-=L6$^gbvAAEx*=(Gz^qTj^_8`3ET#bGl|)xlOuJvb-7(n#?y3@{hDPBiaBzhOY%T)%qqS>&@G5_8$*al`GFdF} zh@*ym^junkuT9`6fv#FWs(2YgzCd4HlU3>(v8(1GF15f!!e zn}Jn+q&zdpFVEg{>mq*iIhEN$9WSO@nTaDxR3|MmCNSkH%miZ0CXt<__h4M}Yx0=> z`MuV>&c_066J}^|4GKB{=)>{y zER(Kl7)INIeKGGY*kW*=Pn?!Tug>NqG{&~0{PSoklRoPcSj2{ooML>CtjFuNd%#q& zndw=7I)em*FsJ8(wzOw;6`y`?_%9IIrI`doU0Koy2ZW;abopGAjuf#Ehbg1Sixm1( zW$I2s$JUXTius#|CyL9E_pp%-#{Ug3m;1;qzP5eU?R}8t*BB$MfTRYTWFYg7hh(jsHk{w-LkT>Lelxxy;IYvNePWK zD~~gAw%Gi=t?NirzD-&|8iz-F-r#S^a6o|ZKUmEpzUpsGF3i?mZH;@w%a(b^Ypc=% z5LhY(ISx1vrwfQz{uE3Gc^e);S%L?|mfN1dk<^0|fdd!;$z#hs1M3hV$okc4N4nl- zD>_f6rTsje(&bW&p&{1k&cop&BO~`o{BDxKlrTbJk2z}vi)kjifn_R9JX~B_u)Kz$ z?cb4mvMvMj$10Ce7W%P{Nuv;gd94nh{C|KDGK)&6y#%}lAGF{l3G@X3%2VLNV|;`l zu76Ni4+Y@U^~1u9MoKssQ=s#TLi&M;DS{R+feaF4`+)%IK3A#D{NAw1s>SIPY_U)< z(aXL1&l(@iw)=?-rW}o8iGeQ*HR6%~nvISSncK4$uYz7(w4g{rLg!~7ALvMRJeDvE zEnzeUk$|5oS_2nP69_F<=o+TVcrk{Wr{4P_bSL{b& zk5&pf6#?pZGuOjUOrEmL)Ee#?pzEJ}v5S)7GPI{ez;P6zcM$F?3UqHAisY4u)-aY) zYh>eQa94m&MRsKO!&~~-lYQ5~9nCc!!e50GRj7@h5kZAIo1Hp|lD|c-oD?e42z9af zuDl48)ae7n*Urd5lYs+ZVTy#;Q=CW=h4{lkPNY!B_?7&-+JLX0j9s!Et?p|?0jVG! zX5sSZs_~UlW?Q+$UB&M%j83;aOlY^rVJme{@Vh8m|T6z7GI<~GAtoZ zXJA;8e(l@a5pz=EeiR%3e8HBB9$jOd+Bc3OTuk;^v9h2ke8Q@Xf+9(!Dq*T|JCu-j-U2tllqUK$UHmH|Ep z0-co<2{QgjkaKQ32Xyx9lV@P89OiXd>}~8t`c_L_{%~M@MBEY4TFAeL8H3u)EfsrU z(qid&jCA+6dxZ||rY_+9N5np@#|HNLIb)V)eqf(_7O zgNT;6?G~F+5u_P$=WX-vy9Xr>7dy8M$iE@?bmf^ zA1y~34a-67h=Zw&`l&W@^(g*Me7iN95+ckH#wTk@V#2}xM=jw;ZObkH9nZVtRpk;bO=Acz zaw^(%EPskvN4(Do-i~4t;buwQ5MnBG7Od?O@*d=5&Bv?E8+x0Y!=JVM-S9*B1N~fC(v8$9J+Yr`nJ{; z(Q^E9IsjAueLd}2qZ8PZSzAplym`nW!)2%s)c)t*82%}7NC3zTnp(IIHpxDyX<|TT zKN{x=c|BW*hMB+nfn~GU?r=261a2Z?Wozka{BVh5;C-_)|8{cO;0ayKmBoFwP-PW?R;uy^bAOo$%aYF1RN8hmdDf9d8+o?c z;WR%XICzLMKTD+Z&0W^^B(}O_35P}Be7QU=9u&BIQnK4~)5gaam)FUCsIT-Zn0C#` zt8QTSFYpn~b~mc=3xUodGiW2<{U>(u2crxe+<$^B0>UsM9t03wX|^LODbAE?M#0F z0|v7*^Kf6qK{ZWRnlCre1!a}S^AjgH6FEmGB<-uyP0kO8V8=Ft&S$QJ^eH34;tws% zVny*2vA5tKKxbPqTia`fb;t)5p*vL|FowOLrz{+y^T zoBBq8PKidpll3eT0Sp3xKy{n>2fM)VyD>Zx&ebm~t=>`&}_T!IVk%8J%Al000MiMlu<=YICEXFc%U6b2YfP)o!FIGJUczltZ^r z_?GloK4f5v^(jnw;-yxXQ3?+8giv^;J7oQU%V2DCCt(G>UX+CLoWOl+1&Z6C#-hX&eFH3#+J^pD-Ie9`$W){7JnT-@1fT z&z*yl=lpY(ckm2Q&a!G`ci4bl%37No)6P?j*#TJRcJ$HNd97Vku4*Z={ZA|gr_8U&W( zH^Sr=--VJ8u<&J)`5=@80m{senP6Y#&9kbgpKgZI@Bs0v&nj;9#Y{XQRYrIK%+AJh zlq$FBrapSGww-DR+wb%I-<`wB5BcUUB_#!3ak2Gb)h%n{RZH5JMv~bS&&B(fd>z5J z;AYmW`~(g>m`^IXXy#9RGeedw!gK&-B!9FWUQt&?sV$)>4t19p|363Z|C>4Hada{5lpFzC-Ju~ zqv0OG?;ICv6;($1Hlhe*a35}}Dx}jhpGI6O7n7!&H}x`2u=D-0mG@kv)kObSvj+}Z zohH@4UbMv%i^!4w{J-J{zpMRW;gT8loU@%@p3cMn1W&EYR~i=H#2V?CKH@K<`VWmB z3d+ADY(h+l+Kp0I&L)~EPDvWC4uqp#<8l=yO}NEY3`5ip$8jH!o&SC)+GAfql3oH!@j7suK~5$=`%S-F<*XoB zCIvxfwakk1TwV%TXaA}U$_;m%*#`o15 zcBIc#)YuBYb8#=hRG~AW>Ikt6+T9^oPKlxdiM4{#+Ax!F@lY&;+;7Mn=rJt#tyvbb zqL8t3VYIZ|LL;iPb~qWd9>$@(H_#VYv_s{C`^3`^OX@(e>wGD}iVM5FQ^Z1z2Kxrp zNy7&FoHHu_|C+Pko#vwIk>DU~M;SMtA4&vY_yhvTjfs{f%&td}>^FeP2NRN<4@vOk z5*YF}tF;xbTdm|sn880(M(iv`OO=wk3gNdWwHz@R%2yu7VwngqlgWcy)G592Ej9Ew z%tGK-8!(bTe(l4+sxs(un7u4!3r0|LjAs4m{AeIZ1epWb9N_Yo?KfoBG7!e_d+Bw-8WoZ()D> z99Jgp+UVihEhs_bPl|wJwuZIS6!HOk`}BjK9Z1L@lAw7@N!OBjy89V~l$h!5pZZ4jwZchW9BlGBGN(VhD-1#U#=-kaC$ zEf+me44xRG?kraD0}!4QS^{0+chz4PJ#Z*Gg{4^HcQVc+z8epjmxuGt0ptJ=m0;L@ zY5b}H#5RVH1m^@cOW!;_B^|tUm%fU_R_k|SkW4l;1O@;GdG3wfzI=zg|qF&oY`JlNXLjvp7aCdK*vVncA zgmX${Bq+AAq&04>#;zS(Eur;(;5dm9@0jHAJje$*hDR`viQms_CAqmK4uApH9KLf?60vx05oWeG5U4$95=q z4O)BAsd{10RsrIrCz3azWVv4Kd7Fhs%au!U*xLrT+b;{vuf=4OD5ml8QE)i?8ltu8wYApOxmt!bY8ZuO>k~xflfh-pJyA(- zXJx5A=(Ia|W@D#B0}Ds>KR6L;X1PpAGVi$PG}XHxiOhN}Q%MCepj?}X$~S4G+Ud=L zacg35%eGRmI~g+9>rGW|;z7MNh)1u>mrFJAW0&ts%VeA{xamveW}rV{@=vN~ZeBX` zt*zK$XWAofA}5TchRuVfw?mL926n=35>;G-Fl< zNJSUy%6f1yBFcR7`i7>a;r354BGALB#^8~DfNDiQM=+;~wB-+z@nG7+$vltmg-R5f zk-7Hjz&urm??w3aYVGk6mR*mKp-QL2btQu0|F2wtwYWMg%l@(7mo<&QN@&X(_m1?J z#`!ru|Bk@gOv`%o{brL9h|v1Nnx0Hpe(lo8t=E-ZA`vNPyc1V#P`}ZafI6A?%3c`Z z0s=92!5AyTbt?Qaps{5&lf#{cQUH>RmOW>??QX~<#(IDuSO5L-yTcjcQ)tu7#*?w zcO$YdDFfRfh&fuKs1c6xq@#-^xQT(w>I)G@oAe9}qnM&s;}vLU88i6S&9kjO57CL6 zyz~s1#r@;R$c$hSU=k)PI+Jc)S)=RKv38sYdq49GKd(X`sS7>QnT)u~*$+}!QB}r? zOhRx{%Ze8_C>ugHR%o^qcT$M-39c8afbbdxsr4fxdej|7#-EP>lfya00P<(lq5i~V$Wz$=x%Bq%bQ=4914C3rTUA>}=W)y2^$)k6Q%*E7-&EFv zYA6N|k>zk?6wG4(+xzH2W~o%0iZSzFSJmTF^EWh)Zn0;gqBc7W5NfFG z%fD)fzl?+VVb9xp;3boVLB8(`=Q6?umfIo9m%nAu^e5xUM}*I9Tr)D!WQA`CfZD{s za85abfoS?sWJCh4T%`YG=HOD?KbiTP@7as8?|1R_0`koLZHrXz{S?}HFKwU%AxE8h zhqGx9-+=NfK9`@MPMD)&O`!|4-UsX5$y~pS+Z`6vlc%!0k=I@6Nz>8@2!KvjCPS3V ze6G0;-)lPlITBe}aK=lX_b>Y`wNN=hHZnro!?ZxLud0*r0OA`387CSw*B=Ye;xjTG zlgCfnlQHw1N=_MQ96hUFHM|F&1Dgh6)MwyE9v;gRO7-0*WI_^6C&|#zJ^UUvLtKPc zxNQf<7Ou>YQE$Jy+=gltxn;Nf8_k19fnKPZ1FcN?Ucu6{nNvZkYoymyHl2Q&wZw>) zZq?I-g*5nEO_v+#&K|Qn;c<4TBq4LAa9c$we}>%M-Ce2LK?@=dJnYm+tn=e{koy=B zzw7pG)j(roYIk?{erpIvyHu!0xe~8+w4?2}!MiRz;RV68WGqezat+0-jm-;8={T;n z&V{Ktc1}(ed&!ei3!jrXzvPdbEjWIvgmE9nk*;rwriVukX88Hq_#G3)%i&LvKY^P$ zh>T1cerLqT-CbX4=3(Dpt_pa4)u7krvh_6AQuZ>}l{VM)lGb9+z&}1xE}nLJ&xC;j zZ4|DqoO#hxyQxM+?H;D!XSZiW&tbl?B{5-%TRLWPW#;B~N3H(pHH^lSx9E`5#>VFO zRGG!k@uiL`j?3VQK1J4Rv;2YWzrlACJ9dMZQt`*>wAfQrI@x@e?Jq$TclUCMWZ86cJtv#Brv94?``0WB^1S|5{1{-+lF>)^Q z*21yp_GK~o(y{O>n-REo6ra;ZMR&k!z4_#6mtSm-~fB{5DTWIlY5u@P?pF{s?pkP6}8j_^4uC zQQ>yBIJY`?dCG66-=TSy0^!$QtrY5M#a+Fm*JjQ8;iKFB`Ce(~{d!?slkG~U>xn0_ z`hn?W!%@tFj|nukjI$&hU{d9q_z)-;63k%n*}Sg}5eDhEImq`u<*}1$d~~A#?A+Br zz~v|F(varqOScqO`GA8>_tr+wlf_xOjbBD@{S|_8Y@yI_>i(tqEmc=PHVRpeb=+y!O!dn5-+yh!-KvbTiN^iq*-BnP3Su901)rJ3^W8*o|B=LS1uqqdb#cv-97*A~nf zFaO$I7>2U^#k!o|v9jH20FkAgvOFMZ5M_SNM!@y?iq(0ETSE(Ib7W)fb9U0iY}i4j zApwUcBteR$#VP}DkUXj1-d=7dUeeT7T&~Rpj_pj5OaC^YN{2oIU z8ej%+?*s6fI+n()o?@-wcmO0Q<)&6*4QYzf>Dk~bB%*kcA>5V99{016AjwAzwdg!q zDBMt}u`xE6bM_oLw|SZ$_B)_@&}hvI*6F-ZC0m}*B}w}bVEr`;-b+-H24{9$?Vrdx z6COqcv3g;T3k^SK(1$2~3*kj5a-5Hp$3moYD7YqhJ;iC7$m8|cyl15nfN3$eNch?J z@w1Upeo^~o&9Tm3h~c#6>uYCSU0toHbbuGwB~aekdHXKm%g95O8s9DziqI3gYvD|x z4dQm0-Ay`A2`KT*IH`kmZvs}jBabex>W4bO9~gorhjzltFjk7dPW|3^{QX6A<(9K4 zQJ|F9crY(03VtIT4j*Hkj+A_k)(5`ME|%6%kfKioHa=0xE-73FS_EsE7T1Vun;74V zmG-Gp00yLU-&vp+v3`ZB2O}my7!_G|xd%Xlq)qkfnM%U>9Jv}nj}BkMaAkw8iX9(+=yjQ}YxEnAR&16`KGnUJX_Fx8q<1QNRcKA}{ZjDdrkot&Lv zYq#T#AQEz*A?$nFPma*o-5Co`{1~aiDew!jKnPqNFWrR-Jnb*)8TjvN0_$afcy_jqQBW2K2qbdir^!@PsR^KHhTA2 zI|N!9Nn>o~&$d zRaaIfv(uO$@BqiYlt>QnPXPRw&Ysw@7Qf|bVMK>64=}NOPqk!8+vGBA1?33phQuR9 zfTrjTTA&zWY#=gzYFjSJXjuw6&crMxqsUU&9XdB6Z~D*hW*4M|Jgv4gi0P5s`&loA~@ zM`?6IZ|Olsqm>VnZ0)SjlBV=Oclggs5!)#Jgig`q=dX7fM0~$rNgxtL0pw5^K`)an z3$?k_+Dqd1zfWaQYlc3UHOky_bm=L7pRxRWuclk%0sUQ8=08*AKga8A4YxTu`wo0y zOdwb8)q-P`SLsO`oExcqB~!_VgbAO^_}#MC{g|zE+HdX3lHUA49d^N>o-_DJxR%RN zG0#Xfwy~64@5N}F&*&*T!S2+!u^}bWjj~e`eM@87o`a!enO~LnqLCcO83!?aeO+{u zkOAWd!x;D9p)4Fm*hMpIoIf55mucyM$V~-?KcvhXLcNdnM;b$dB(tm(F_X&gdb#<% zHAhNJlYdw%V}?9aWUNina5l`L^hwmgY)&|4*BWs{Ym?sHv&-S{WUP5#uNy-ASoyR) zTLM8!frP+zAa1N(1vp+dy_GVDcUZuWwq`En6=Iv%gi9KdNHqhA7)v* zD5-Jq_v3`7-AVCAcZE=_2?(PnYOK^902=2ixxU&#+(QnUl&Rg|W?a!^6 znVz2RzOVa|={WWNG`G09f4?AE^Shrqm_9}xB;De{$bjM3$Cg&6A|+v z;7{j1%aTsT;?Hc{Gv`;7!epnzbkDf``%sblKTZ8FP0%?OVcK68;MkhB@Sclssd zYltr*+5Q>nig(+XJO{@OVw(MQ$Ru*D={0^Bi{kTJ-3O+NC+uTGL$yHkMiRYTSH$n+ zIAMl{@0gk{B>%?+;0~KLf!{0PM%BW|kmxFV5@Jfyy1*Xwka{+qApLh*8km1R%q(ZH z#y0|pd+#qS5oh~~CW^tJiPpbT&qF1IOTi_w)ClReA$5NP1)kmDC8*JeP%Cm+w7hVsBe~Z|W4%zP7sd|OhXpXtKT15Nf0Fb1KAmmzG@X&lW2$O~p}dC1p0=)UMDdc9 z_rC@4fOv283AGN02QN;=Mc+RNPpOG;_rRt$+p6}$=GeGdv4LH59382w72bhhC9YaZ_} zwt2l7hd;eJF%@4U{}So>`1r8igYDKEO=TgD_-I?*j@4ylOfqb$t?%!^8|}8&fjc-b z2mUo3P`gD9uM#6l_>g|z_m(;y@yh-;XRA%n)U)B|xy0}?sj#pNk6#4vBvvMIMBi^eYS2R*% zPHNckSM@9E7R4`rAW7a2!WRnsn#8DH4&E z5~Z(Qquid~{p!}Sr4;4D9Be?7C1-|0y0c8TYZJn`q3p6kxTW}M*}Kmw&C{ETFaSl7 z^<<-Z?hr$y`|k$atpyXO-wv8!_buCJ5aW2&?Gu#|X$ z*7u=5b@CfGLgHyFydyU(?7n`U){-3)U0hHDuj#y5?zTZo7KKc{9_O1TT)&symsoPd zLhqHa-qyFh2&aBQaN9r_hHy(wTU}XQoyTqYKGKO55|R%e1v>RkWD5R#Qc`Z#V=wE_ z;#gG;6yja;dz_w%y$E*@pqL`UQl=csXfKC*BB?g8fts*QF*7~g@nNQ=B!4~HY-y=> zbW@uL&F{YUt!9*`K_T9lt!G~&Pcx69x!%3ykhnAbT!)YB|9lZspT*gd%uk44PpWE=y zEN4Q!x6SUVdwcyWCAr+`7Sr9xqH&E4D>4F6#Gzlmq(7}`8?%6&F<&2$Y8KQrZe5#( zH0g6_i2X%5-2Zl6JZ8TK`z*TOO)$`VuG94ejfDpYiu3?A_J17|$Jyz<$-~6@tv`(C zzIm$+?R2<-TN(MeIA=Fz?E;B1qMvUVM0A^-XKx0P10J-^B4;gFxSpX>uveq4}fXW?ih7$E4 zL^~%R1Rg(@A#lsBIZg!Ye*Ubbow$M?Xtn{7Sq2(0Vt^*JMhAH{O^+I!5cGX-~1El{nR~jvP~t-INFK)3b?O z$4r@q9p|a0c{0TGIK%qiLmI97*KToiF2sIG8fv|Wm~e}7BAQ=5Z4ub%F;JKE0`Ih% zmKZH2ssa<|`*-JCJR;M^@OXea0Iok?RPXm0x{xK-1gc_w4qUEIk6u_}_;gr2O3qfA z;4+zpeQVrKP@A?5kc~X?_!gd7UxXE zHW3A+%Eyxvr=0>0qXSFMm>-AG2UIhbEu{+BfhI5j$4%LKQe;3PKCQItolcAk``m}w ze<$$kl*f*GK8rOH0e~`ou0Axt2_kWfgRqj=jaGCPeC0nl5s4f^OUs;HO%o-=mG6YM z%yhmRBG)$|nY;>w0RW{G{k)qCjXAJM2y#%RX2*e)U#&%kOq#GKB(;v-@XiYRhKZJ{G>g2&UMPoInhQ(hhrKs)}sFPbH>#Dv#N9Y6mew3Vf+ z!9HE!ht3FU)ZVw1uQH4HJt@nt{B(3*@a2*Q44Wob6#IY~W?TO3SjYfmNy)GG`(j3? zCJ@8L;JruUQLa;F9(WE%tN{OwU~0}mttZ-x;XM}IbS-|&(MzR>vkdBV)VQTTfa7L4dFO-aggs0b}2x#l*GASS~j76Qcs_I@q@{TmX{>&}?^>Z=049xg6spyAZitvm4@ z#^RFwNS6+4G_M+<9Q}2iATD~tPznJ1(O+*azEZfG*H4<3;7bJN{{B8xpSA6fK7*n7 zBeYcWwAwqarL)u9yM9P3IeQkfZw%jJg&v-ZCPUn=Y+ZJMm6ea3roBoWYqrjLd?zkx zeRB4D*yNJ-7%Eks_o%p$6L18RF&#l>z1eu}A^dHs>WOqqOXfInXISKMdwF@p3+yWj z@6VK)A#kUu9;CQ)DIXPa9p9*$E}>W6*}2ZgPxLONiq*1CT-`;#26`hxSjeL5C$^a) zmA^O=BHHF3hi2J zY8ey}?zHB55F^#+}{2d%dLS7U5p?IeED+xJsDx* zyQn<7u(G$b#HZDw9NJ=-{B&RMys-{u{codG0x{U(izANisUR|HMzR&y z8z`T|v;i=Z;#3U~&++aqq##4`%Uj#kPM68Kx!vK<;q_73N-;7Zrtg(`9HN$}OOXqi zZfY?~u}YO|t%g2B&q?;V-+&G$P7GYcK*W-g>Xw_cQeiyw{+XGX`ld|iQAE^&p5Nov z%**8^pp*-maJdfq_d3#U-EIjQnbE@?KGM#CTR}6x!IFd(LG>U7`c!(6P={&w+ES2YQj)y>X(&V4p|DYOzWT7Roq@dY4?6+x#f&N+N+$20yI z)5n3n+pz^vw_bb6&!84GkdcRFnaAOziiwF)P?Xm??MQ}QsW6av|V!o~MDw_((V0~|KHWX398$k+fp zRQuU}zt@#Gftl-)tsYKOSbxDyIVJ`f%4N07ez!P`jUH#;JUOZi7JmzGZyqjz$+FLw z{p3x1+o}7#iVj0eyGB~*NxOErfxDV!u!JNC?7oNleW_80KG~=Vz1b@xTI#Dm)&kD2 z1}9*V{nPOUN&OP1zpu43c&*#tNpe*V?@&-#b*J2!jco7yf5n6W>+tLTmC=FAL z*WOt#aIm{e^qOXT1R#a}hbGX+VdP*gRDFR*mD_`q#6o%*7)C0`$D89{a=F+)`Pe7t z((d}$aC&1oojs2dRG+OvXw$=_QU_J&IPn1KCw;$wPI$F-FRtgKaF|}9iH_mxvVUU62N;$Ndfm->7 z1yQ$7IS0%#iwnHma9_h~0{LD14pE1Xr2&+k2y``_cRm`9?y??^HFoOWFv_Klef<$yc9)MWVNHzx6ADI{i zJ^-E|JQ}Fi9GK#H|9~Pz=?b%a3#)mnJrpU)Q`ST(2{5B0s}Kpn0U&DusMmZM(E~-f z#QXi}VFHY!MfY}EkzlDPnNZZ9f7_HZT875SpC%{&g&gLxv$DR-8*HLanf!nX<(()2 zgNu{zuZ5)g3Lwl&m7Ujl(wg`36(CVq7+ESTnhZvk_W^(q8`SMsrcbXIUJxtPWfMyd ze-mmvdYJ%|IBb$sD#QSU;q>^BVF9*4m>GBi=*+P+sB08u{%}kgM02YJ+XoqRB z0{*aUYaH`W3Tx?>I+6}CR$5Pr%v@iUxzQXU65vqp?!q7jQ5A(@BNK(wp&05_ExHR- zD{!R)WCFslX*1MoV3C1z=|p(cI67fEqkrfEBqd?KDo2OZR?|f5?89>ef|@q>MnBaM zs!-ZcFq*iEC4*2lq9Y!p4MDNU@R`m#gDC6@R9yKQ0m-Go=RDA`kvp$SkeP5aG9qQ7 z{K%a`D8OVLjSG9m>g57W1rQhvBbsQmIPw92!U>ang!C_6Xy{P{Z>C`FZbZuyGKt;}~t6-@h%r?GgYu+ao4K zUbopNkk7;^N9(#TQdPmed+tWAV!Fn2n?wec$dvdQkcWXJa}i1@Hoe*-XwvAcGb1V` zKSBGPH`_|$!`yz%)=P;CV?CES{p;4-VGl7WTtZ2ect#BmlOic-%KgWjg18BLFhYES*w~FY67&@X;?VI{QfVd!NXD4-IPAAFv|w7*9YtJ z3TO_YRG2s6bWZ!$Dn34fGzQsbn`__u9!T6(__m{$A0$cxyxF$0>>i_Ov)l|3WbeET z^LtxZUWSLk%3~6`MZ(}C<);o;>UKL>y`9EuU9vA;0mVwDbi5`jF(dks^fLIx;$ z4k&Wn!Cl5=WH zt>uMH!$8_gn~L?TDv75k)Xx0NiK71LZ&dC9Cq^l@;A7S~6 zF2w(%(3jVpZEGFt0~?tk-qtjBY}II{y^-j^*gShK0Nl@;gK>*`^^e9K7}WIZ&%=J zbsK?)7{bWE!e$Qj`%>2{4TmKK^sVct>$gZbYaMU3CXau*+}QQHLaPJIPgXO?bjBmE zp9fwjazAe@%ufRcyA<&XO9dSsdb>TPe(b!=D=b|MnU7o~Cf3hVZ4EV0G&eU(^(ru? zI6$K>&(~B1YFc|mgxoLIL3c=1@YIMVUdbGJqC?`ukT49g5b7ULUJJuuf_8^KuEfZx<$= zXK!#*{uBg@zT|9lxVj4?&`5M6e5ubxAx6dyZYeLfvQJqc-)eI@T%~CIFbKEx@aysR z^zB;YRhTFe#eP)e?U})+h2;JF_XC=(KgJK1y1VY*UPa#c{Do3TveURIR9@-0CPkbg zzKa)ISd1;Lal+}t^1*TvecQ$s|EVAJzvrE?4g?%S4b$8tZeu;ZT#S7G^H3IdaUc73 zR`Ryj;dlJJ(jdn6fsGmJ+T*qVqRL?3{LXGe#iP}CddCBy4>bARBQ@>wd4#5a^&rtB6HT zQwDIDzP4bHt=E~o{xhPp;fAeUZ~KQen{7@Fb;psT&=F1jXGCMRUathVgh+`!;)zx^ zGBwf&4iX$Q0ASfO?c!1ZND1GV)KPr!Ai8$E=2=F5&mj$via^D} zBk~8>P#1{90pNpxl%hZrKmZG9CvHMKfVDujw^zLtpu>q32onIgnO5ggkjmpwhu2>y)c%PKHM*FIV9=`=V;H&RTT0^TX+5`LiBokkU5!r>mxYD@KRP}Zs4XmLZFLx&HZ3X|8v2629|d1a0E3u zUwTq=!$>=$pbn;tYse$OQsjDX(d{*prPOray|lbOhe1(D(W86Froay&ye#cD$NhXb z7w*CVjp{8UDr>8I-$KKseI<5RXUtRrLT@l6V=fUkB+2XkqGqFOdwrM!A`N%D&=J>t zlMPkO8S`Uh`zU9%uSOu#Ad2Rt;E(g!EMsM%@P5>1OwF4tF9=i%T-hfS-|7Wl-&&n3ocYrWEKw zZpuy4bFy&y^;8Wytd$l*loaEhkgTm;d!GM}+dm();0DMd^5tzjK>xhn`dh|-mzxu( zy1o5z;I@N}C;Rr@LyO&t^Ucw$@o?7{JE%PsHI-J*e&5Z~G9R-?zYoV0=9>j8!K zCohCr{8~gx0DQrB7Ee!44E1X0B2Ey)`=$E~uY-9CxrkKeqIsA?m$D!uSndU-#bD?Xhn(T1hNw(t+{s6Lo;P zM)UCeHu&sk=g@04&CIa*JRTGqdy&(f-Eo&pKp-hjgFt6GG(@*AM}VpvUGKf}H-fV# zN(~)!Kq;C;(3HldQx877U4Z)%W1m9lETl`*JBClRc=|qIa{kfiL%01-)zUgRe z%qYKUIT;%$>C`O=pHX}}Lcl}hP(h#hTNOc-PG@?JDe_jeSaGPh*>!Q-UJ_)Ohlk%i z*lC1CGmlM>lkxbdb+XaY^6EGbficM4Nb-A`e~HauR6PNq($6l;cl9`GK{L-~f}`ZV zMny%Xrm-hR16m8bN0*l!fL8PgVT$>>>UCCD-g|}jB%^BIb(#)~z+DfgThD&odC+KX z(aGw`w2VZWag&(J%9;w-*t2^mx%?s1&tqnGc44E%Nkh$2OpKpD(LhCAS68LZq)u{( zahy3-DLS@Y>q|f*!hiU7!>B5k2`#N$;)+VJ1B>>Y8cG(Dqc|Uos__FhZq#h@*#{IC zs#NuKl=@n(8A1ZWmuq%E_3!Bn#4K^n@QHXnrw<(nrgBgtYrW({Sp4JD3(u#?q$F-^ zDYs&<(gk1LC%>wL3We46oFR>o2&aIr$Rk3+2XB{qql7(>b&uf$R!2hWj1nYFr%T zaiCaRxo)S=0^DD;A6;wFwcy&R0M@7BK~d zt-Zb4QTy8Y+tu5pww{8Tfrb~Jk#XVc)i+##_v;ljoSu!@0J=)?x&=VpGPB)2Ki%{X zQ-szw*0xgGTzm|1AjGrT(DJCe`n~a9!#KB(IM(TJNo`n$D?%#*s8LKO8SPg7h|?*8K2u- zUbqp2UVl8K^*jvo;_vefhsmM>Fg0IU=97$F&mq; z`j&-}9f^m4O;`61Yq+FQ#n_>nON-QGy12ZBCCtp;vV)TJICP3owh z)|zc|^*%W@UD*(!dOkjWw9_TWzaTn*Vr&c0_Qc`em4m1l`VO7P;p;=z>axv+m4#0{ zLIg2CzCd;o#0-WCJdlO#uf*NN0(k0-R_0!--v^$BnsrRDTvbV@ewkheiL&T)z4|@o zzNIm?1`G`z{poo*cxULIC~RX86awuIZGb=N|G8Z}*L8zl>pCVz-U9N%q?EH?o21^# zFwG2qbMNx*=AO}+IT>B5^Q|h^I1;{O=la8zo@TPT{cLCd*s9NcM=YU7r;FgyMz`ag z_`C>FfmHsG?^cCC7iM(6j^DHK>TMc<%z7lI)C zbGf#^qag?M2v>LC!6zp+WF+KhnY!IJb$;%^l^q1+PT;J#l5dW3t-&rQTlvJ9?g+3wQN0W)k4Tv0*N3!vA?CFY!bJPEQ{tw zhgV0{?dGG4b-C-sbDH1l(rMaz5ur7_CVV@2ctir#KII8ojhvR&!IAOt>sx1(NR&}d zm5rAm=#*d{aS91c@{uUSuUHxv^r=v zWg}}H5-KTW0)rZl*9I%bZDiCv%-V-1A090~AQ1$OsABK>YjygP3O$LCUqa$hlvh24 zL}8Iv@Bo~@N}1X{kxrLV4!oj4L^XOHccMPG{a>OA&;EYpHeNA5>`hSmm~9XJK*a+2wYz ze)SH0sLQ>cQ4^QaKb^0}S!-xcRe=FNN^_#q>FMBF?KJcKv{INPG!-m==?H9zKq*cb z1my6?j*n;U`dBtYHp6rBe_Q~nh<>812--5m8(-@(U_m5DOC<>zErP@tm{$DfG@hBlqwJ=G4Rk;tR8S zO}EeKuSYdGdxM@%m-EEAkSHT=M_<|5$vZYg6c+LUZ4SA>+Dgac&s%?_!9BKGAZ_wWY)RXbyFa4#V#HY|IY^0TgR>$g|LSIbj5L~-N>GKL1VnlV#7;~JX z5h`YGzE58s)$^Y2Z*;3wXn4E6TogA|9IVk~`rUb>ajWX?k9|_EkI2K?+jC|t7|BjE z(A;zF*a-6JN<_gzMxend5KnU{dhlF}mqo#rZ5%!&&p^SzSf=TKO63J}^)~D6WLNsX zBv4MVv$oF84pI6epspEz+UBqNm-cnda=DKrEYOe&_}PQ4uqCkTB;RMSJ z)ETTtf-9bkA{B;6C83%mMu|FV(nb$DLS#0Xz*dG^R1t8)NMM1O*8jO-c---c!cnPd zxCjP*xfH0k7@+C)My!gsybrIQaOx+s6%+}^My zBjbBg-HTuOjyh*s#`MjGhFzhqLyUL78xRqga{ub zE;xj>Mq;WZDyd?F5JH8kITehh%6YrE#_%Dvrn1cT;cEKzR{c86{PQd0AmEQIeAg!( z8A)VO038~k&%Kv4(yCpS;7cDKLH9$|fck>?^2eR%@kq+z&#w5M{lxd9Ev1-K(wgEK0){|nMUWEh2Lt^?040g-(COi?HBYTjBC{7kc{tI zIGB`D_|S#0h_cg4I_s!( zt<@@1QK?fTE+LnlpsCF~7o zm*0Kh3ViW1;?7sL(8M&kx)JSv6lwS)(9SYB)-bniSpcU=J z)z;TrpKce?Lzj%=;+PkQ^oh~Lk zECd26w)Azp=)C^mnxjW8*a4e1zFW0k`}p;G#qX)b;N>2Q8P>d>_q;vTym~>?T~tj| z#{c};bLSA3ORbSews;y!mmiBxU}iVSrxKM7i$>#;`S|47J}hD7w14sLRQ+?S4{!PR zt4;*AwA20&DFeTnEfG907WJP}oa#<$Kp!4AKTZ}YnsmXCDVnsj^zh5TuJ2Z>A3sqf zHqi$=WMz5=E5e`9GI3V4XZ0P zg`m9~)L|UG?ITGC6IH35Zu+*3GjI9Oh-aqK&CC;2SxV?(XB}F-hd!j* zVOi(}sbeA)mzWiN@Af7`w$#%~R8W}(a%gq}u~7&qQRsmP0GxP`V7uqr^(5bPu82mP zt(>oKx9L><>Mp6)=)rfO5D6+Yp65%y=|V^QdO06IIkJ3MoAPnQ+h#GAz znCz{n*|%HglUGqip563Ei4f^$T+CH3^%{c?OKV4yhPy<7%=&bt*SDcD-s0M5{@@8@ z76Afec(eXan;}mu6)w89B!T(?xGJ*lg;PH8dSi9ZZUArnplg3UdQM6S?0=L&ot(s) z`1wMH?RUu=OE1`D73K!5Q&YN~dgSvo^z*w+kSUQXWd73z%90urM{KYTGo@WK*RI#6 zXfa?HwgS6|?_z@ox9MHPpVoPTWv(6SoG_j1lt0&N!o>bqglyHN)zv3pPY!P3^DBbi zXVsGa`C@*HZG(A6`7_Xx3v+2nA%y0oJApNGspf;h<)V%~b|$LcZoo{QeQmzga7={; z_4aY~X0^mtDXZ@HhbaoS^fP)uIywAYAHQ&YZ`gxEhKTgE${XMl{Co_v*LF;O@ zkuqh}*g=(n9;cAd1Y;U&Iz@z3A^nOvtE9N$KkeD~Zea54^z`Lg!-2<02*cKAT+=0Y zpf>jR1>W^`4_2AY0--ci7>-c$;(xq9^&0FyncQJ2Rq8Yq%$z4qY#$$I^4XJ@m&Gxj zoSksf67ILzZFQ4Tjrm6GUuN*gW?Necc|Tk(RA?|TFeK4C98S_3UW*e;+?Si;qebl< zzb6a}g^eahnDAe^xcA;2$sa0$(9qCm*flSl37aHVv~=)J(+wAkXTOeIUS2lGtUePuiJ6re}y-qG{wNgwBKm- z@v$cynmA@`yorU@g6AWfp~o_n!+edl*9r z*3y2wsKcZ_Gy0`T`~BOuI*ZwYi-bIvAS1lOfINGkgZ(X=wUo=9^Jll8W>@_MZfqYj z&x#+8%(J&sKP&R{`#@DN_Pl2?^%mfxA04%WryrM-@nYfCu5VLALxcJ_Fdv}R;A*{@ zr;|dKa1H;$fog|N2nNQE7LGL@S=l}n>*i@ z>_;fQ)`73@GMJG1>mLh;gNBMFjPt78Sji_JnoPbb9kuR1RmS#(k@C}*MeonO?#UyW z3TsMOp>Lo>JbR!I3hEPLs%xD4Rd?7oR{or!DFRNgXSP^P7S~bb2Fw753KE8TYdV9`3MVz?F4o0m#ht*hBu{95eE)J-m^a4K~HtNc{TRe^j;>$v>;9r zR1Fp-ij@da3t(@zMzDnHVpoC1x2rWj3$B&sag*{Afmg9QWvJuMbnID8R2_75TskCW za||86JM!aDAC5qS(x4Os5AZK(*AE6| z4~`>SK*%iz;{V6Z&tOj~F~R>TG+W;k3p11ezg;$G0y7-J5({p2K_`EbG?@ zj3I2|DDprA`#>~C*JC5xjCtM#-Y3kISPY0_r%ypcRZ`zOK&6qJ{l;o{Wp_~#f|ddT z{o~Wqc;w%X$1r(kn6CgS`Y&e2#;UTiqqBw5lNOhML!w?@HUO~am(+NuB21*GGbm#RIv#dSk3br6-d=Jk@s_OXz-vaEW= zt<>3h`F=`}VRdaSKsUM^vSX@xwOL+1Fn#`>oZS2VyhqpqOzi;kh5e(!!)wdkMP)VE z7o6mdgu{HjS*4SrW%k=&oC*a04Lw9$qCRor8_y0CeZ(vftMqqD6rli*mX4~P>YC^W zF5Zjlrx*Xhc$?|`!#hYAQe4k!N-S@9yeKN`4w1wk7c;+8J?-UHMcRzoU7FQr-Z@G`9m-^^&-qHf+iC>CF$Uck* zhB~jfUiOCj{mE|Gva$!O9Kmw`wRF0Uv@eNcNf!X)qrMCH>K|E8%MqU<$<^nu8Rh>j zZ@DCmt@KjTR>4J04=z37I0B}}53koGb&^m-A^I$Q>uqYy&o(Zul8a_>WYf};CZ0P< z_2uP76L@!f3)g|n*|D{g_cG{3JI8?>X5C49P^ibDghBS#KEj$rM>g;GcX z=qmn$GVp<7;-O>dYU}|f4ssD9kF(>(yENb)w@6~Lr9$OJal&L^{VLF(S}KGMb{kQZ zGd;bMnF?1Jc}$Ct1`D}4*}k{?lZyAnj-dT!{SOBqKM~3y2~?or z*(I0C(&h%%-#IZ89b!y)7YlfpF)=$X%@%bOPY%b9r^~#P$yXzR zNpgPnu8lX*f@6vd9~=?a_DNiE9FE4}9QNj({X(frDnW?=)k6%8_oWyg5W%Z!s*VXo zPfPXrFCx%UA+pGGV9ovscXB3m>g|RTM9c;NPm3p^RKh-)$ghMyUxiG&K34fksk3pg z3L@v%{v9i_oTHl>IiZk>sN^z31`53c0Yy2g!iNS&U=w+90D%ZNLHl9*q*@_~2&e@U zIp5PiIqfitiqde_M>NzPjAltM4!DZYu9AKkT&J;~gB9a#2A3AHHajEuKH%Lk`x=W-&{V8YPg@ z8OGlKw zXI!RgNg4teT<%NSk>qNIBe$jpUlMn>Q+)UdCozqni^69Ckb-k zU?|~>`L*e#aK&g(5Jrp`rGcZ1REU(wWKFS9{9xsswt%cV+XGVH-^>ZNxQgvLc8hAx)WuRJzNhJR zAocWz_Eqg2s2CHAR+TtX5{GtMWgtHpQZ*LZFIH5f4bofsfp1^70XG+eT+`a8KP{a}qR3gbl7Uu{S>o8A6->q4et} z6)0r0d@_)H>&msf$dtI(8T9RWeC2FuLWppB@;4KY#=EA6^35w|34g|&RD`c**%olr z)IT!bG_>dEq^LF>rKOr5*s%*pbF)ZtZ33i+Jdm2B#HqiUaE^OMm^L9YtQ~X~WA&i| zB=K1ML9=z{zg|Q*DduDvO;WvN%)Z4r8;A`CFU(o#6wFO*8JSaLNYs3(kex4_j4|8?o$q3%5Uk}Gw_OJ^mixs|%L85^2L^~>RMQaCd4q+*uw&gb~s z-jV`P>3{HPx{y`{797?Nb4~XY^ScyG zz~BpB*0Q}R#T96xO(e*W=$D(mcX!kI>z_Y)^*%^8l6Q#+G_aZsr90-FI;?H=K1F9q zA9N=tKv()xZs}KHFY8fGjYe+p02E5E{iJQ3y$RLScC0wJZ|f-oo_W)dK&@*vO3vYW zgdUR>*3*_UIH4WZb{V)lhP$4nfjDb%GC}k^I}e?08>?Yyd+pwz9A=uuknbSln=^CE_JT zd#EgUsAd5p7(>e6El%TbXn#vUrI&T^0lle?s5-0F=;HZNJKFi`3lW$THvX`3x#3@S z0(1d1>5Fg;i>z}(yR;XdWqr1!e}}l7Bp#u>xtJ+_tYK$&>L(m)C`!5(U}0G;l>VPi zcycKLIvSzS{Iul65K6z`qn~D^rSQd7boqj8Rw^jsRu#CPe*5vU(9Y| zt*&ZzRaGku4x8R8`MsrJqwA6-u&Z3lA(R>!w%Gh+KBXTHOOVX^nwI-&yzAxH?FMFR zL%sX`d+B0TTl)ywjD9$}0Zozjz!RbGl=WK&XHfa!v5MQzByP4c1%>N-YnXwq0-c z(f<|ze{l+K^Lu+OR>*{EQzk9;#uk*=bL=B=;wS8vdupuaPdti3IO8SYUec!g{-H2J zQ&p%$0P4ng$Xr4?Ej9x6Zqo(z{mKsISK`-0<4eZjlwBS7XLhBVoHZk48K_zOx1%u^ zD1bV<`R2*@OHrNmRBx9daf;1uu9=$tp)J+rj`!7MUxtbbC4yo~l%P$xOsU!K7bG`( zEiLWzu>Os;wKd1Li(d*lsmbgbb(iPNhZz|efn!5Qt14}$|ErpFG#&oF#>kmZ`*InR z7`zO{9iSO5TE-js{iyypANO^3QQ?P)=XXKH)ag-J3W2=a>&olwVKa7f#WoD?wJZsz z$zuZuZ9U^BP-g#c-CG~7QL_+`3yX>nwK>gvgE8CSWRyvS(@@o$WtC?Zw+q)(sp*qz zHTV3n@I6P&-kIstBpYJ<6U)wf@>8oZP8G{GG%^=~=bVQhR$1eEM@&UI|J55FcsZe~ zJT&Rz-xn7`ey&Lp$8k^xspUj&cDnanRa#cGSL=2gSy}x}Qe}9DC+K?S4!H%towNJd zfR7M^eYbkAc83S@kJZLCgxvD%JWY|aP(CZ%tsozt0sqpH2$V%sU*z%+|ENBc z5zhu?`2z>?HyjkMG>v2=`d^>j?^}m=1-u{Hg2FKLgqD__yMI7Vc)i}=#bG{NG&{$~ zL%r2NAQHjrJ`}FM%x*Sb=S@qmkB2=%M}JY8U6iuxTCfav}Ej#cl^gC1&X9R|=i+@9C-_uSmokB>gQ z9UrqroK{6hN^3S>PG-TcCox;o@Lz@^Q<98dF_p5A>{s$7!eU9+mwx|tetiysVvMAf z6=Wo5J&-&Kz@rZwm{bHi6Qti@w0-?l@oPAgP0#+Dl3}xDx8k!NdtD>SEf`)}-(zzA z&$Y`#!~Xrx|29B8_Z@XA6H2UahQUAij>e+$8e3@EzoUyvh9uC((-5J@M#sc7wzOmb z*U!lyqnP>?Rh#o`D+D3S%SR&*;V6KY0pov|YTeHN@My5GLG;pLo@*46v+w)7-L{{F zz3$e#7rifT%CHXQ(kcw?n_y`Yh(EzJ7~utbY(5v9km<1V_&7iPaxo*%w0c}Ju+jZi zwLzl>&bF&B@XDrzh(<{>ae&y#$5( z7ybN_wV-L>Zf;EqQ3#v;_27#&&$k2ox5hIHqUH7L`Q_ngz!fy`{O zti4h-35Ci}J;io?m9N$m;;vP63{K;I$pP0aL{yomYwOw%KWTCflpSNU6jn0cJk~%R zVTYTTS1jL&b;uLWc&9#F#qCHW)2?ZmmoiNT2q=`gFZM`(@NmUB99mh8pK*uvw{fdl zxe$nRP;z%oH2cqeL@2r>8Kj@j^DR^?y+6gBz_?F8K3b+s$qt#;AD$7Wm$JY3=!nch zQzHY+X=c>wdcOM7H8WC-bq^cc_p-_KVb+%xK7s=Xuj0d*j8OvQ9N?f*7zD-N^Vbjh%&s#iDBkK8-$u z;Sv~a;zf1nDRtoMXv}G9dMy`a+~dAKv}E9svMuoNIlr|L9gB-iS50(rbou|W^_Ed_ zZA;g1Bf%58aRR|LSkriLcL+{!cY+3YZGua12_!&phXBFd-Q9z`+uP^fo1Etxj`Sqd1ZpzP6Yi5TDYG^wc##2JMz*u$7 z`l~$=RLTc0BO>r{aQxUP;LCA2?bo@#xNvFBXL;Wh1Ox<#QIOBo3Mi3M2#*{N=XHm? z;dSNUui4&AaVj}nqEJ;*7bxp>uI3DQqk;jB;%kj*B_l)W#ORV_gS!_G&NZ!ol@8D7 zx$h<|)V-hsty9S045eYR3Bk(_ix9XNxJ}mO_100@DtF8fhdrr+R( z>X6qu5Ybs?dDs!1R3Giz#%w&~?rYLKbi7t|oPU|GDSAL!?Rj@5e-~Y)EJL3=sn~Te zgQirQFLHER9U2l}PIGvL@ArZVBY{QlqKIqRqQ%AV+Fvoma+yWn^Fyz~lQeT(#@iM? zTB~#nq+)1HpDtZ$w8X)xGV&|0Ww$C(pcB0D;O`Tcu}6bS_7ZAFek0 z*zPei^9r7fH0kadM=Tm*HV!)Vx5=DVTWL$UN^^e%^#KFg#NNilJtXn&Kv6=12q>aH z6lY}z1f$~}cx+4&!D!tM&-n)@cl!%&rDu|tk9Q&ZMU@@%v=0?S%~sv7bNWUGAFfX( z_ykLddYxxpYnxvs_wk$3=sz+SrgtkUru}Yo=KWkqYv!fq^D)6Xj{mgbRu(1tL0t{w z`hC*H%O2k}t8aZ%<}2EIY+by3AA==w%)4E4QrsW0k5Ba)p5?Be2A#Vak5O?IW>U{u z$aoU&jpH5z&tKN?#V>bw{#Lp?eCv=G+jO+#6&-oM7x4%h*6*m(T4;KbR)RgAgob0Y zd)H4`AOyE{w1Dc?@3uf65izu6<8)REB>2_GrQ=}1AE+3xazX^eD|xf7j&NjT5A0n7 z!!9GD@Ck@rF9i-l;y*s+n2Jq9 zv{4zy)fc6_=u#X_(ebV`o0Bs*t%u2pP)jP($` zehnZn#3JVoB(sNE4VC6qa9* z#otpIIn4!?1LSDHPyId9ALl1|!QgVUp;X!=mc{$0wgemujHvSWYKMuk_#<7p<<-5e zs3<8qL$V5v&YR=8vf9?QG~?qwjpnzM1BqXfYGgxVV8iJbXKNpG@oOiqtLthK2K7}< z5EQXa-5w^py7wA(*a#OK_Z#e#bogMb*^t)ytD~De4x$;(6$vwTsi@lOYSSE&@%rHx z>7S$1d7O@>N@^@w<+-$sn^W`;?umgAg~2a*SBET3<)*TBp4zfXT>&CQ>Ov`v54=tx zA8v0_Uw%nD#-PdcI_u+LNc1{sBS{njhCjuz;Wx(6)9Y7zT<&F-MiB&r;y<`d0ds+lpJw$(3+ZYtB72NffjB79}T27LjUQRA2Y!h4RpL!bV!17N6{v_|@W< z?u+kO_Z9rkb2E3V+5T-2I5x#gF1zixsLIq`6$W6?n{TJ1(S2Os5#sY71{WLB$4hx)`;*kfeO6ir#T!`)EHC%7WZ73XFiNaW;}>14A7xEN`CPw}oOXtj5)vk-Ra-9B zr(;bD%kGC;PoQjwm58F{5;=VqA|a^i@C@s{h~4N#k16$ed4(oz9kA2`srSVgY`*;m zG`DfZn^(%PkBUl_vv9$dv!cerauX?S{5P`a-xb(>s(Gbnb31ja=Ib5EQRx~4rl7V7Vy_OEE&J!ZfxUOkqp%X5%qjF<%~8rZpC}dI z!Qn4qA7oo@hlZPg8@1u{XHI*E5;auu#+`$PfM?Yhf2CoC=bfX#ymyDfUjn{MQn@!o^f!itLSYM9udEym{dqe z-wR){@_r7JO1Onax}*q+b&8zLoHThGKSBxCVq;_1$1CrffPeNwkg00L)?+b~5>5z_ zFbz5583Yg`ZX+n~`^aDnsn6p<#FwufN!;T;a~$E-6st$TmP_UeHaCoEeFvcsTm4OC zyfK1do_0n+0eF?*?Sw*wh%AICr`{W#Y8N7}_(2Fu{SKvd?#?UkVaaga^fCNSp|Y94V%pIpIVV*ENnx82VGn!@9S z*-<7ki|6s6VSC`#U;5T-#3>qNOC9bJwHYAvd3mjZgYM5A?ZG*a(ulI2V>E^KN!7lh zd!S7{$XwxpU_`h%e%m@-5{1$gPr=Rlk%jb9QfzXl_cn2+(@VsEL+>BZi2!~5QM|7I ze5pSW-c#yuBYUs?e~3jrA9(CNNJr7)-xQfUk&xN{Cg9a-HP9@-v!##+B#zAqonhB&!V$ z$7A}eL^DH0)MWOocKySKI@tVA86zH-v+iWt?kr zSr6D%$J)}5&q+B>o*(?>v$_n+c_Z$=g8bRI>#oHqv@NwQAu)`xO=5p&Wd;-jp?*nC zX-aW=bC&v<54#E#3kxX>+IAzsH>SWGG=NeD41SM2Etjz5gv81`H#5cQ6OAi56x~iH z(JR(ik(aqn{3h*sSgzt=u?eb)3P9n5>(n$viUq|Ec774__Htw(YAY z-Xyfc)u3UJ?)jD;o1z5k?=F!DIVGWO83puVinxR12q@%0k6r`z{;(QE17%i_`0XSP zL&}}WExp`D1R+CEY#2|{>H;#woWrsEK7OD4+1XhYgdETfjE=58GDFuq(#=>~C0usz zN5474E=5&*`{(}m{2fW}(1cC;4-7@B($W{7C_(=3YcE1*dnDKfGH6}1brRU7>D?b> z&XnUSWO5v=GuJuW4>iX&=qbtjnS2ghXs}SVU(C^$L2e2AT8=uqL~{zv*K%moz+2lb z@=I7Aj=^I;T`7?_MucnoOk|VA4*vrj+Px%c|tPd#G;32>+EM7Yb$eqDzqlMV_01{Iot6dhj%UL*$%da!9fufdRyID{O6O{VsVWW6YOIcs8TtuWM1pc zA}N+5S1%<~9kvwM)cF22?rxkQw z-DP3HFr^9O+FB?c1X;Jvvnp-PEQ`SSAmrsp4f{%iOG#VXaGzTK#bNlDX5HY}KjPkH1To}y=3ZP5gTff9#h%H|(KHWd%K?ehthCjh zfaHG;s{gs)zDOwq9?67&O6f!xha-(-p9hgMz(&#vT>O4EHWiKl z&i_SsET-Qo20Rvu+;S|yAP5zsH5Qp10of=7HJaT(k+@aZnS{0dYxzV~v<-w}J;LoP zz$m$!nlVm3Jck3|=O3Ys(5g-#BqV?x5%lOm;P>T(xt-_2@YtM_ETC$ROR{q5=eHU- zU^Km#9-U`2JOaV=jriZ)XG{Iocj3EOMJu`)aZIOS-J#wY-}>%)Uzazy6^vnQ5W!kA zGV^G&#D+dFAPuSG)s7|+@G9Ch1ML4WSaRC;eHu@!bPo;_dl(NshgGl3m~n65PQoS@ zmU$XEh*$hvZEa(T%6v@iTd00YN!=^#LdUC=eR)-z!Tq##%P_>Le!_D6;4Qi_g^)#R z452>9Z@C}ll@z$^n9G zcrxLiIh;DA1|u0g)IDGO_Qa!>^Lkz#?6Qn#u8n246&#Ht(%_Y@MUx(bVwcx&;`O!l zTiiR?LNR?@@?C$Fnr#g0Kb~DbUmc$sTD5rJcVG(2(MYJ5>fFNGo1ZX`PcPg()HXya zk_ujlYkQ7--oj)ma+eU(ERNCF*U+f96#ZE{!~p}#V5_e@AN$ANU>A|TAtV$iw=3I{ z^C~BG;-r@UtEhq|PO)gHDb=TOdP<9U-|5X-<-kUK(g;#SirvTE5O_u9rluu%`Im6N z7>1UfFZS!KtPAHqa^%MD?jIf=A0mz}FV(@~7D*C4kE;hB1YqBLeRJ4YS4$jPrOhPf z!s6mz&Q~`(Sx2%to#9P9ov)=hQKj)(Fy!mea9Q%CB zVoj?1GBWE>|GiE-iLimD9^G{Sa`F|BKUdFjHj@-tJ|5KG+WR&=DKjL-dIV*r2H z?EOzzP1exyvR?iATGT)Z0zZGp5?t${YkQ-n%h6(e&ieYN{pk!|_uqrDBk^>Wws!iB zF5(7g5Y_jpIt>o60{ZB~=e?ry!9_vG1AX7?m48I|!T=ND;E3p_dsUd26xk*^b_6gM zs(`HWSNkTn4Ukfdg`f~HTo6jVpH(&ZSnE73$kRe+ zCJE7Ox~s7LG!>r2DBaFdy~r_1SWArsVi-nY0}*L!HG|bvM?I>-xDgPLP%#v(#NG_6 z;e-5{7}E5$h&>;t@b#Z>M=~^#_X%-^7|8pl@x--De_6VS5(1F?;n1Ox`g-bm%F1VV z3=7XL_q8}*P#i$ipJfKRs>^&H{#jNQk43(XrMlGV2lndt__!vcOJm7Yc_6EZgh7hb zYSNb>7KT&b)BfH`nmZ?V*df~0E6*_0-S-n9?L zk0AMxcSqpf2SrDgqyB-cR#Dw4;s~U{xbGkkHS5fSGnx>2IY06k?j)X*x#sP(8D-UW z6O^`5iyW^p(U$pBiJd9*WACdzGT8O4%YtZuAc{(te)5Y5GVj%MMIyUJ7o5NtP&~mK zX%$A4MjdQaSs%3Tv42nDTr)5}Z!gBIvBqx*{6jXVLtPi<8dh@+&`|mavF`@HKPF-e)QT5)usMlu-3z7*P^l zsNJwB8V>nD0;|-y4U59M^8;cC8zL7Uggez`3pd#aBWE3N!5(W6NDrxzB|=zc(ZLc^f|jF&IF z;=@8U_+f4JzX6G0*bfRh8-y=G?u99A2~QTb(2jXNe>?uP1+0A5e{cC{S?9Zm`p;3* z(l@-o7}_A6w}`DfU*C6@BM6C60sZE^8{u{b8gO`;w0&vS%ZonNxj+Kg(s(qo|Kf`M+RG1p2&Zh(_6mmfCuRlrcGq z+@?VTc)niRLUQVq5 zr>QyC0sjF-Fi;hU9|N!h)xHoF?$;vq zbq?)oF#C9y94pM2-(P(!E7NK@t*`#p`MY9hub2>!qXS z30VV>>qT#v359N>rLMQFT1=FF%d*>P|4Uj5#NJsmK2UZ=(7%U=l~wgtaFY2Yyde(z z6Dmbm|A+2OVO3$OlVZu1+aBFVgeKC*Pgy7_74{w6l>ih&4NHV_TqHRm0f3@10G*Ep z;Y_TuaS9lId)5dWOXBryw`wFBd7X_hjIC3m9cjG09pd7{+HR~{(T75J&F3GSvvYpr zPtXkm1~#Y3$^r2~ro8E{xTFBooqqRXL;$fDIl|AACf4>Kz#5dj(h!*%i_?Ct#$B%W zlUn~2v*njj%y1A1At9_DvT+6dKcFtmqrIGf+%LhJXruWDAsWg9{ey(Ymh-__5UegD zDw|e&VYiTnli<~EK}%xs52yf1_c}^{)P2(&St5Va4TO+ZJQP{WWMCfCn0w_VhaAo8 zE%DnKj=5rFF)YqdB!eJ0&7Y{&P(9q{7l?pYddqWk+RR_PzXU0P&ZdExLqJ5VTtSSq z7EZY-B<;T{Xl3S95r|>z{yww9oI90L!f+0VfJpGU(73yaPA1kdSfV~BS$c|fqwx?N z_T|I|DSjkYdnK@VzPvS%gwW}xoJ9$sGP8M% zpC+CbJ;mV=a4$>EgSTjKGP8!Vv;Fn$CvnWYkv~}III<3RZi=*D1g68W4NhiDEMHaG zEi+M5nMlpPNAioot-l9bU_orq@R+nVRcHipu^dj@{KY6hqmu>is}q0UgjE)dHZ{F{ z_G@&vn+lY6jrntb4J_L~)amNz(OPfPjg-yE<+`B+nP*jFyNIr_UjPQ6oF9Rh9l)_B z2@&iv3OVMvg(-;`9nq|cVM#XFuQs(__g%OHX>|DQ)Yv!R06C@0?@ILkTw7Ts<(um7 zqphJUG|_a2Tp<{^DCb^lzoT)ncRU@;^KN0*ri>2oc^Z35-f4c92cPNh7iE^>c@%;% zDw|SzW+o$w$gx0_KoBdyK$rr{jPw?@HvyMDSye1PJjHP+JD+o?wAx`~f&kDaCLaz( zx#gMQS9t1Q8o($}@Igy_=DxSl8hXCy3wg5yfC7Ln5Q#Bz}oRPC;(Z z>D5=}9(%y*p1)Ipz9eqG)z>&)ne{W=#5`5niY*J)vMmQJo@z#I_-100&4eK^0xHLm zZF4|yNrrB{0-(uxcw1jSyno-%|K)#Ay}%Q)U-15lOK<*mni&e}AGV;C;8aEJz+!k< z_B>u&_LY9sk&tp78mT|nIinzOppAnk&pi?}YJQe-9*vome$>suaRUN^gd9D|t481r zfT43g{wNejk{LfP+Rk9K@5j&jeqktaMcD?Z*9r3TS-9$`k01LtX=0V%;sK2?FtUrtP`QH%jDdU z-kb{10OOzFw+b;scd1_9Kqb>9#CplPNs(FPuKThGIlXt&K=sce4bNs=xLB=ljmvku zeOcUiS6Kb`Qgcnle*X5VlvHugMHd(~ob?|JAU*>4AaW~3_KNOMaSTvp{E~Q(0SbHl zu+*x1=5?L#xt8cgO=&4JGTI1dS2G(@icJpt6Cax@>X5U_972wHo>!B{Dk%1ZYt!TP z!=J@dJ*!#*G1Pm(5tISsh7_V$Gy-AxFffoFj-$6;Z4()}7z@ix3tzbfAYbbv?6cIXXM`d%l9^5j#19?pe zcmd$PpBCSeCF2~U;-Hi>AO}aZf{V$Ks60q$j+xtus6^XgFsnjmJE0-XKen6uYAD?7G(PG6qA1dFr{_#92nbl7Qr0IpThiUVf5?xjG zk3eUaYGQt`P4$ULK%!FlQT?T$@9k}6oF)aR#d+-^Rx~t@aMpwzd2^zkV61f$8PL2>7DP=yRRn8s&8n zTGrCs{`63oIs~h9WTIpG<&^LA{3JRwv?ReNskc;{w3p@Ehvr*(&62N}^{V}QuOy=2 zA1;6dR0!UXpjLbY*1hEFu8CvE&)sjHfXmy#|X}^+nb%>Xpnf;y8 z&dK($NZWcii}%w>)njvLi+iq5-?%uebDLi)B2op^K}r3E;*W-p=Xd>u#>fJP`-k&7 zDmAcXN4~Iul~Gg@;)^M4`NQe&tb-*(k!e=G0sHZp=bq;GXB2&#kN44ogDYaksZFxN{gQ%ll)p?mxozh&-NCuN|+uDWLkaLtQ{(Rhmr1 za^v)8CJ)R>7{0i?hL!zt-)N*ZKHrkAEbHh_c@)t{%IVuF`YHm}YfGP|in6>m*K*BA zvJiyYMVoZ%QeQ*z=)}98egzX1y=BSdf4a@TkE5$-ZZVb34eUWADwXNnOqJE z2DE$$D6n^rx>6!t+<=&kzU5peGlhq;_li(}=V4`egh+%UBfU{k^LE3p7^m3BqKH=}1@*nO!3Mq2EB)i#6&#PjVrq$- z8QvA>_-i9diB8nH6V}!PVt+VtG)hVo{}gL0D?IfC%nR4!h5MNS4I<+IM2!n|uAbgg zFsMCu_WOE=J{Jb}8OChui}ENVMiXf8Qf!C*d$5*nW9|=oJy(r1C#4M%e;BCeW$7UO z35FrD*ho_aE*~w3ii*N*h8xpLD(n0wWcxA@q1QNV4~rSi1HMTrBqXUF%%0NZODMb% zTpS-yY;0Tv7r-b@Fz#$Fx(rjkXM+}O-gcT?6#P@{x1RU{Io++1+UXg@Z=$g5Gq=Z) z$FlBwBj-)44L`~e1fp#WLP-AV^$M0xKK_2$T*ogNTVS%Zm|qLObIzTH&Mn zP#9YgQAY^RcU_+khZ40Tvq;`;1D%Y7Y^Vpr3=k3V2qf_U!Vy7-x#Zwmb4r&GDlZbS zk_2|iQcw?pArQeYs4Vv+y)&wnY`+&iWEFr%0AGwKVt)eX_TRklX9U&`6YG#k%s3GE zyikoH@4?1tUd|TVqQL$b?Ic2LEkwfA zzWs4f`x~nE=CN*MgH`}>g!cfF1G zX=f27{wQmDCDJ8ch^jiZCrhp?8T#uxjtuaki=mz_`oFyT+Wd~POU)ZxgU%#sVI$1{ z)#oLhL$b!ujW9$qb9-0b{AN~MEG!6>4V|di$Kt^ASzOi%J^JISjRih*DVwz%@o=yH z+0%meUgf^*DfV>s*KueCS@283dH=iM9vu6Ajfh$5lA_YFiQJ$wos1<0!tH4!kWzbz z?N#6GoW=V28Uhv)YAA0LO_$unmhhN)*jXTXKr6Lp=RipidQqEUYZp}!PLc~pgV=a9llPfDa9B2vIr1eFAJ}v#pot_414njF@uVEfQj*M+lo}bSzptJkqf-nTtAW_b_Cg~W4KXqe z1vsc8M39OC(Z~1NGg&o+lFfx6^+Rbchz)ztm1inpA5}4PgE7M*Z2kfb*zu|a>1!dHq-^)F{syt9<{B^m_OygD_Kwn9)iMuOD zV8uC=jYuG>c{K{z{#sN-1dhLU-x`q|PD_(Rr&sp+A4lcO0qTJU!}G(^xbR29#%vNKJp* z%u1nsBdFVwU2sSES5O)LQL`3x6p4RAE3D78VQ@6*;%0=1#&o<=dJ(`KcszJS8$a-8 z5UU6tpp6ib3P^y@A1~`Pi-tf0kk-R7Mf?iDBC$hZuH$92;xqa6>%1Hkls&a+^Y7!? zlQf7ji+TKSMDzY2@656doDsO7sqs0#0w4bz=DtfdSUY?*SV zM98vrfHhaUY@o%{z;TM1d%j(x%r9%b;rKE#rE2$D#G#TFmorZ0ilmrYzrIl41?W*mBF;4Gl@w zTqL}A?hA)UpGNrpi6;0x1%58h8k12tULDL$=x0_0#AX{(XxG(S4J4N2NBJzzja^Mn zD7I8=47D6QmF>&d8Q7}X9x_~h$yXq0o?6sW)ft7+)sXF?{+i24Z&v(*m>6jymR;N~ zTjBW#UDj?mNMFQhPCDJdf8v4_ODijsRb9)Yr+odEn)-&yntC2oJcmGUU04~<$XBp< zSA{z|*T==1sa@mzXp+HX_mYystu0<(^{OqaiO^;!?e1jndpp-UYzF1`I(#iI9jUVU zm?XR&++PEV-aK}pC#q{`EdQFL5*JV7w1tt@XTwDARd9^m$dHUCj>$GGD*n}MK_MrQ z6_~7?FuD3vxqE~_(D)(Ylx=z>W~+SVJqZ>8B9c)UA`K`xjVU7>PZg)z=H>$foIl}a zZ$csqBF*A!0BCptPBWj_zc|(R0N`2nE@ZDA?usTz6A2XRcTMdwZB6^-{qY zt6>bAUuUUh0>z}kT38Xo1612D!r+J94xosw_b@Ov8b7&QArub&o~CMjtt${gv`-bh zHb_N4e4t39f?1~1r6(s=%n{B1%jQQJZ^~;!sSx?%pK?Jc!|{_R2Kszo+)cLP!P7GD z4Fc2#lCG{XY2s3DM(rUfUKY1+In{&6hRMJvj^FzKn!8*dx$T^?7+hxO0&B@@>(qFl!>fmIBc&xPt<6GCg_^QoR!a70|g7ogA>${ul#Eh|@<5Xm+pURy6mHfX4 z&TRYR<1m-d#^;ZW#er-_1oc6ZPz>4X7vcZM*L?iVl2pbCM#JEdO%M5mgH3>d=b}%- zYsU5s);vUeVEcM`_vRegxBk-x()(4hC=wneJ@pFXp0xDW?>;^JE}2f;pj9wWFDWbZ z*zGFeU()q`GEh{RV3K4-$>Q@+#S)!TBii4qqsH#&Y2lun`$QC9TllY0eMI>mx6nis zxBbQ|nCiM^5d=>z{KpksnY$m$r!eW(dL}@YUH3x#Cmldp4p6z;6jR!YZFLg!1382k z1PKqKL?a8I)rEusiLgt^t+YB3@!&fxMl3Ji5W=yQY1J6@k8sv462E!Fl*#2ArSZpd z?a_XmwhnF|lnk=;dNzZ@Mlqzrp%ES1NfAP_sT5Z9p)+0+vUeY#`RS04>$pj7Bk&Rs zg+kG8f@oYn5;PGY1f+=jj>&7_;@3M1-bLJC}geZ>my4faJks0ZiU}nI{eok1Ek%kB53b;5u*_o+^mi-IA2Bg=>Wgsyi(h}vNq!j8 zHheF(es$nucQAJx<_F6fp%Q{v%jKMwRK)Ck8doq z)bv3y7!d$PJwEgiY>e#&q(8GR>Rw)z))91%x+)rXv-knD%N^2IB^s{IiScH=e zHv}7f_*d#(E(6f-<*i9BCgO(Pb(DD|^I=ilBg0YT@aO+=HHUKNBBc-wa=2WZn1XcN zU$vp>l$HSr;To@(J)_CRaIUg5_e?elgYxciQ=!}ss#lZ-L2_gwalFBOzC5Ur9FJkO z)u*%OVPYmC(bIc$Q=%VB}hh&Xd;U9MS1n zHVFf2zZpsjH`2AiKrCQ(5aKT~d#jPf32SMKMRdAny20&H`DZhX&ir z{Y-BULw6qOZ5Q)_u@1?P97`qySm43E{OQLP>nwJ0T-_ztsKb>Rw<>C5S*yLhf5>3Y zKCF$CPok2qvKlJ?3);g*{8*;9vcUGtbnmYifSmuwDT>NHSc>Zn`)) zIHvZqO6iymnkN3KHBQ{V0+=AqPiXzMu@fnWs!-sS7EPm}lXrVJe5FAEIlQ)mIP>FF zm@!{1?_dyb7m4q~X>9D;$d$KAup|=F3q9M%ZTYmqDB=g_uld$aLwRp_T^^oWp7oak zPlvCfFrOX_X#~!0`n$?LmK+>6IjujuH2#HOJFTHsD)?#BUb5$~i-BaRvL4HDDE3ml z*o#kK(e-zHOS#6w;@u~^rps;8=bN7Wx_5wTS7)D_HGSo8qKd_ucYD+Q{>@L{@IMag zm*0Gu@p+n>V<9T_*2OtjJ_X^WwAim*&GT0Tu*v;@614%Mb|~qDwyJ;C!Go2zc>9;UE52 z`vKiR0H5^b<@I92r@)LCxOhX_KX>at>60w^?zX!l?=EEBv^;IqF_3=XRS)S5$wxuav;P zWc4&u=4$(6X?*rq&*zKFE7*@F>YZI3-{VbXZZzZ0Ia6?Out4A8)%I|0qP;&HuG960K>Z_W%lm^@IabiFWDNoIrX_5c1)R=UiANK34tvNZkxM-=2rI8xmy)M&x!;1ST z8s_?ZnG2Uri;^X=QRmWIO`jXK`b6uuQH={fiqZ^z62 zA??h6DhIh$fapk)X#}u}qw3jwURy3P;^I%`Msl+@S_bWz0!T0=Ip8 z2U2rYf%{2&ucHcgZgA8BJ9crmVQF^u=lM`wskis#U<%uzt*yGwaYhu;%a^<^M^+wI zLw#&J^Ye|sE9$t`R)0-{ctWRdx!<0`UUaeYQo(~!iNOCYf$Tv;YuB!A5Asvm?EPF( zMC$9Tt$VpnEjzJ}uQkInE^NP4w_wN4{;T5ru-jTO4)TY(M%dcZeQ%sButt#>_ ziE9&1La5_ZZZ}Sicg5{d!nlL<+INqj#m$z8YCaH&!k7=RYKhz$&fV$+3`r7Rdk8*v zZ91Kg?6&rug%vsbp3ZY3F)A}>xIYKBq-#|aHnVw-Vr zA=L%{60-9Cfk_O>BaU?jsl#7-4FeFN(9Tsf?yZU{cz77EoWB)Z#Q}^xV6>w_r0hWG zQ*HQVuMbZ*BpW@}yOZ7cG8@s&*tzPKE$%16FVpmO zo+n|+Ei6>CZP=N`#Kgp6yQEp;u#wc$EJKC0)(1ym9|2(?(c=S;X7wYUysWI(^m!*S zSK=f>vnZjVUeVh95Z@DZeQ`l$jC<<#D>(uR4A!u*wXDB9n|}F)h(R&^#>&|O{G~5} zNplvKr;x%C^k-ZZR{0ltkY|wzhBczaMgSY-d8KDT7Pn3j3H*C>It#itLg6coq>_Pe z20L|7DzJc{3IGN~ggr40#Z!uUeRl#sZO^LC3{rb(_N_uy6Y~6!nmf{>Wk^D@|WA)Vo;!$wHw=Yh{Qq^)z(<^!a@Mvr-^yrR1t*3-p$ zPov1bPYm^wtE&cDzxwA(#^yd7BLvb*y6D{o(2{Dq=QoD;d0O3BK6 zT)GevU>c5}KVi8;9PdD!(_~qJl)7 zVam7xul6OirHnUNh%w~|pw-doX-){=?{rF#Q2HpoJ~3>#f==DtSbu+itEtw4UA*kd z?!iq5=~MN`CAOMR+NXVGu&D`7?C=86JZy1FSKr>Zq8AEr*J1@9c6>RPZ1CPTdp91oJe}mjJmtL7QU&g%g^3mW^F*g8D2Y~s z$zgmn@2G+PvG1EZHw$((9h*d^4|f-PJrRfqkI$h^yuOYby=REe-bWx5Kp0=|;HJ0! zx;HdO^_`6J($Yfy<#0sF;r!F+MFuRmEd>Tpz2?nS|ATJQQ@X$Xc^-FEZ&vW+i1`kFfszL7PHdy*-@0Y z3V9iyK4C6rW@SCDpAY!Oeou`PG9dnd;t54D_znOXk#oBtYNDbVKrj@A5XoIkf?`$s zqF&fiD6t?Hd?3M9}YHVmI!IyakKb9x7CQ;Q94*)9qucm(E(iTl3`Vk;kkO9<* z0Wl+&4nP7Fo5LE~8qvi`v?+D1seF1=RE}=*Q)qT=UnGK|&5zw7j+#+oTA3Wd!(lCy0PvKtV9on*zo23$?kmHT#FlH5eKwF;sdzw; z6+RUfsA|+$LYbckz#5~f3=si3AnwOnMh!4Vj5l`sy7(sthIRjcs;wV zyC(kqwn{q9R?GT;dBsuJKL$Kd=5s5Il}An?l?EPN@1~LtvaGJXI>@X^pLSGXDawP0 zicO+S{as=2gO{ulxuq6IZRLFF*oSW@r5$;I06_6p%=BxB|bl*`dv zb5rf{c{HwWp2VhG{xW)g*>j*R*HYfveLt5fFW2$vxZ_{`DvO_1>HCtWVXm8>O&*Ew zht6_?!%)U?&Me%mcl$I=8A@?06|~+G*X=7iVf+|cV#3b&&>-nq8K`+LF%2O{Y+Q72 zLAK&zJla{_+foL=fJf6JTKE{pfl3S~DG_ z?)yk_VY@X`V`t^GJaH`i>d!yZUoN_LQ!ylCyl42R<7*4}D3EDlOO--&I6O1MaI@jO>DcOr_iN@#?v`2 zls_28bNrQ^pQCJu)oK|$!2Tw$>$orpsc8_dDG?mQO!xOd;q2_M7ikbp@mRAc{p-lo zkmGHkO}XY5=A%d(Khu7=X%st4C?zCj3E$@L)swiB|36>%*h6DFXJ@9k`}#MDPJik6 zA%Q`4gtQ>|b1&m(#2Vdt(eFf|EhdQrSRQju(7)SK!_2UWRsq$%JRkh!-=wqxUo!CR z?;qJ6gXg!xEDCm4kSWG}Fw3ZU`z;gyig=kZ{?YudHYf-Kfc5bewZ(cLdK46~6AZw( z$z9I*mqSYQ0}od#fmV5h!t#3xfj{L25Kyj4UEEwL*F*-ZDRqLSla*^C>$S}!|AJ34 z-jZ(8SnuWFxDq2k7_(AS<02Q5y{={xPl5LgPSG?ylElh@2vuQ5EI|1WPFvHKHexS{ z)n0fJC=+pOR58d6%Ueb1Ejjr{FT$*h9DFM`L=Ml7b}E0kQ991```8&*Vhbf16q&{Q z;(};mB7P#;E^r}IZ6L!+;BQgh%Fs{AK1Z65ZxzIg$gru@A+>e+uKDJ=1o+a4+DhGifCMU1&=f>1E@vEuH~;!@mQgA=|y_j}uS z+%fKt`ztce$=Um4Cwr|q*P0WMZ!uShDqRAcAm_VWQc>xI7HTPHzeg^+IJoBpUlQ>w z3-j`CDk?z;(^?&}G^Lq8WvakQ1miC&Re^bZgU}UasLmM{t{+rhddyp`K)J-I%Z-0% zS_H#qZft?Q=Y2;**C(Or9wx)a`KGsO--m1|hUu8sYu6fHj`i+cxJ5FA!y=yEKfLdtQ*^BdSnlQx(EeDPc z!=Lu2YOFIuWIt+Pjm9=3y;gA?3SqeE4NM%Y1(&n9?@0K0R|VPO#qvh{cL>)QI>Z*X zKzg@mi4kQZUq@zUqr#v=>gY+` z&c6KdI!m+_0vZ1;p;IIQPdG+6+HW%kL%+3UbGzuxydn}aHPi@i(1^=87x){$DyR} z7_|O9H1F}( zz2p7?7wHV7L<)m5AEi~guZiryXvF}FJX_e^jY(Ge#Hle7xFEZl{uxyJ;e)i66a)67 zDt+c{7q+QF`fn07UE|0C{9R#GVdSG%2JL!hq7)QwNR+SKkX1-OciwGlAr8y`X;z5j zPL`#=EGGRb*OM;~NB%))U877fFmU(Pw*nA%P?y(RD{mwS5aZ(G>ra@QRGJBlA5~Vb z_w2!6thiESvtOPzt0QR#nQNTw{W!n!(MiWMrb%{1m+?5n8FzQ2Omm*eH+0lz4GyK4 zt99Q@E-f0}7GGGTMy6Eou)pVK3r`PhgOk=d!6^iahz6CV@rVOI| z@?N7TbX}eWchbtZo2cO5#eMi{8`f`#60oRIT2i8xjl4i}N$Y~bH|u-6 zT_JdT>dM2*X=OKp2j6!Wa1MPuQ^7FkfIR!;!ADgY0yM_llSAQ z{!&w9iPeCE;k~`P4MrdbpZnoZ*5Ko`5CpOV^N6JI+KUh+p&%!1yAVVsyI*;X%U<8Uzd|ao91NP2ut&{R^1#n0^|I3ryCZ`yYj@iVD&X__ z{QUgEdGHZ7ANZC-zie>7oRF3aL8l3R& z1V08TEde*8qH1ev6We`n+&n$CdM>+8s%1|7#X0IvL_8iK&>X=KJES-(5c1i|oq>tX zpXL&;*J4r;@#HJuYYnoZp2Mg0=ZafLqb3|wZ>5{D`}s8?U+$w4nC55c?+uf^TXF#p z)`Q9G>*OpWf5Th~YP@3sFPE3ZZNYbeO0-v$Fc;_BcGDOJ*o+2&NPwg+jOyazq3|s`ec-;cyIt z!@Wx!KH+~}op?q@b_me@5|I7+1GBUR3~@ftf>o~hUG45{`nj(Kg@D%X3spoPq=h)_ z?0&kAPFyT5l6h9&b#y-7>48@7l9@X1I$uo`Je^MzXZq$@@lg_>D~Zd!(U`_jBQ(Fs z4sg5;3PETR$;|z7#mo*JQWWaJwIN(emy5E}zcoo6Fhm$$b!;U^6ixnq6>tnL+x{sUYMz z$M<%n{odKR#Nt8Oklo?w8m}N1`}vN)*&}S#`W%i^20T@~#9d|+ z0N97%yt$ecc8J$S6J0W2JNDIt0!xX z?Z5N39G~usji7B6f}VHRnORMa__7q9$W_^S*f=&-BtP6dxu^k1!D%a}|6(G@lUJ-3 z%$irY<8cj96(C)_IIIHQF;dMut!(u)YWJ}?gt|){4lCK5RT-k7lSFh;JDDe$4kAsY zT8_9DhMSN`!C&a89w?nn_#;=XwTXD|RbWg`hE&Y#ST)%#`ySs^8VPm0q&jE9dgIVV zp@jw!6CxR3xJ}737oU!2?##NBTUZ=B?qy_eckf$J3x>j6cNRfCbdjl4N3;aME!v#% zZI2UCzpI3bT?Z}uaMz2d5q1vXO}aiQ1@JBT5##sQMb%hz5EO z9_=po6-@QLKx{7elZp7%mw?+{=)r-rtYsCSDAxrs{&|U($JwID(|J{N!A8sR&6`#q z9>{xTWf}aMh5MGMvRY)?mW$s}^WaP5R?Hl4_IK7vVll+w1b1rn#Tn#^>>s@r+DijU zX*DIZav3H?OsXW3pybx4lUS?ZPFvUNGt^{~ye~|XyH`nA-|X@xwY2`_V3U2YqIId+ zGSF?u&xdO3T;vwMq@m2JxcA{#VVQ6$Bk+y5dd^Zax)n%NHFP|Y@^&nUaCmB#YE~-7 z754|oxhcY^aM%f_rzU2X$&w_wAoX%gHLk0ypN(XfaM-3yiPEo<$_ddmVNEO82ys#< z6Da83ke-17`I5qVkOiY_@-0g`6-in~mrP(kQz;_4EJ|&}kitGr=<|>prQIVXS_eW|`;nmKT`1T(FfF7#a zw;HCteiCY`)Lq?(xALnTJrr~e1=EXv2-)_9t{z(7R2#9AN6^d3^P~6yzAh+y`bg+( zV$E73IE62ZE90D6L6>zYP6$XOE<=2)huSM=S%ThDRVg2IWPZe+=q8n!+rD=L<11GZ zW9ibmLcHMteN=D@*vV8fKMT#|!OdUL3w|eVESBB#u0caUAz0z32AEc7uh@~O0^iCj z=zgP7r*ws@xwie6bCm53AplHq<9bw0EV*}ea8Xp(F#(8r=!GaXN*BcCCqb{mt!xDo z(DCFj%uG&j@s!jMhA-w=Dk9;KYQyk6`s5>n=ui)^$=D;Kh)&$4=lG(e3D3@QrD(`e z;XxFkNuP%#02$gCeLXnnioaHxu4XD6SG~F2b}oqqHph;t(J>;UgaXOshC(ttD)RXF z-^TTS0R92eAks%B{z_hOddm)9T^kpT7--^!fdF%OqS*`ZyjKw7I+iC0jNshK>(WLD z%nmb)3Ik%9M(Jd4plrr)m}1yKa#`@00l1b9G3<#1UYOk_9nj?JcJJGbU#V=Oyo50O zZA1SB>TTz`L=re93B^Y>?kmaTKUSUh7L$!3ux;P#OcwmGzVt%g=KG2x3Pl`oB9hq5 z>`u&Vu!)wg(%MzvZ&0Hk>}Yr;8X9SW0S6!7<7dK?q@;D$9X0&X1Xr6rq?k-Wemfi3 z$t1O&tX-43fP$`2yy+KXr*#I68m5jnwZzG|{PW%9^l(jBnAcuEynSNzy}=b1kU233 z1fk)BG^@1YCx^MwHL9}`?wTGB*)4Gi@*`?nwGm51=0uNBTl{Pk7)bz_7xBkRCm|A& zsg=~%W2sU!yABgt#y||*mayS^Xq=CAIzdTYh0SHg$}TGhC@|x!XS`&Um|E7U!;OPh(mRQ- zBqk6gF@I=F8{r>CH4{yxBaRf!mxIOwIZ+y>2gqa;PHi7wpZ6kz;&RA2!-k`*C$3-R zav8%C6esrP@u9F-0PfEHyHGEllq`ZB!vkJ~W+ZVlRAZ5`4KNB1FlW9MZ z>@Sp*{%%M6wbRV%85PHZPBntSCRTroW*Y!yEr_}+bU|`9nK1~}4reY-F)0q8OiGiH zwpii!`*;z4_otacFOQB7Cky@=-{DN1Q!{H5pC98FGVxqpuMAbR3tWNbWEF%^e52FRl+RsD-CPiXsdVNicdy!F=6DUfZrdi<}TXy;Jao6=gB7jl)bRpkAIYg<3*jT9V$aOiz08#U{a?mag%$PHV4Ab+gQc0w*j@%pq~q?82qS*yyJ-~>=JzpO zUa~5N3-wm>#aJ)445G+IhVAD*Dk=}jYflvw+m%RRe|P^fqCi>W5WfD8FLT1ajnKp} zgmroEfa8h&H9tNl2r#AemYmOH-^=RwTQ()jj+L-Uv?}6d zVXginicA$8R5_t448zWh;=A*UdPBx<1M1lnkxZ|coOKo8(lN5>kTbKgn!$S&RnM^M z20^!My6@^&*_0PWh&QKT$icytqtV@JEAIGOi;r{TIp4bJ2!US;wb$M3V2J?u?!N)% zWs$rbZi?cU%J^$D-E)7|G#v#<6KWWZFjQ%-cg3xLXIkxiI4HYv?6}$WMX@?FX#Zq2 zaApL3_u1-}Txibu(vEy0RtV~2=k$vPty#a*vxq-$w=H!c;fg)4r){u-mYyCsL@pH2 z2%Of)&1!}&gBjw5&o}r|f{xNv@`ORYt&S@{-S;Y(9^LL6VMs9hKSJ+GC_P1OIEM> zs#XkS*5M}})giQ@MW)+`=bv)1B~{_cNF`-do#~=l`4QzuYdI*w;3jp6&@3cJLQ^av zWi^Z#gDTLdxq}jl40MLn8IY;QuK+Y79P4V-5{-wQmvdMFJeS)`Im_wHy~#Fdd%|V7 z>)7f=WZ|kA2-pRnqQGHn>D-j$d-ygK1L{|?P-4u?Iu6oj24CC~ZQ@2#0aLSBK1x_# zzfGRtJ}{%1(4sw5QGhI_Yx#PU_)Q#hemT?766gO1@Y;{CAlZU2)|N10f=w2ML2L|6 zj4xo1iVWh4j1Bw%FkXy8!S-4IE`<$?Fi^^~hJt zt$*A)ZT9dCq`HQUjTQ+J1rCqBOHKw!BHWbnmCENHBqteD$zb#Ev<#bDF*Vdk3Ahp4 ztFOeTI86ZzDbjs%$0&f9Q0vT-b$OK`c3)god!AnI@mXB}-gr@{HbxW|Vv>1>1m#uk zF9|Re^YH%cgQPK&u>^9uG)5Pq7MWyk|Oo(&C) z7I9AzHSoOW)WChBM+Xj(SptIlixB6-ngDtdezUk4=<*=~6<#V=YMie8HGmHb6Q0J# zP8BJM7F%u2oq}*SEKVV%4JI0cogIdaA`xyrnwASShgg^^B7C@3|wia*!D70n@YG?3u|K5pHhOd!t_D$%{G>Hpw=R{0=@wtl>cnNU+~ z^X!6NxYE(INm}^D4q4MSv_f7_Gn*}p`_dX8o>hluytSAlc*kby9nM9OzTh9ByV&0j zhw)*OieMB%c3klqFIk@SZLxPExz}no&+%C@lK#@540zeb)zO76ohqI%8Qo=_d&4hu zT1uM?=y`^Io|8oeD!QBR+!ap?R!I_wfFZO@$POVsjW%FS(Ms>Ha|=VB2~|{o@q38A zS4CbOQY`3Y!0aXb)qBnEK4iAa-hsfM-#LMnKHBn#&@1 ziE_tkCOfPw(@S5%zZQ$Ku;eFdyfA%Z+~}I;v~ZWc9&H46|vLsQ->IvHo8oqj<(MQ6?>`~8YL?o&=7CYmJ-Ut2ik*yPfA2*xj{8SNRl9kJB*8T#1 z8o~O+G$=nOT3>MsCa}$IAaDRT=;!yV)cl=QhnnHk3$T8NLz~xEr#jv3>zv>j%OKdO z-cU2+eSA9V!?_EkJp4ymsg#)cy2|-UeOZ;i=yW@AQiRBEY;_ca*i=E{L;apWF6Ghi z>l0;V%#Y9F4Htiwf+P9(wNL}*;Je4};Z=uAgEdb6(mNY}x`Kg7f4rCJe=FqG?;!+i z9J8_T(vQr#9zVqI5yvOLF}Zndu;2p5LG+Fs-fId=k@mhb_IHJiCQQRQRTT7hC42sb z%vmFKY2HTAP>-!%dm6r@xRvkJJ~!N*!5;p$Kbpe7jw^hmrm-Qh!m%dTZg(I5s~=jH zK2-qHripAG5Y6J!nS%;=dsvwM@Xa;S6GUfW_Ee_vZfpWb5$4sG=u^nzCxguI%6L90 zh^Uc?n*tK{T7DvpCS73OfG)URa% zjV8?O3FnMG3IOpZxs4_%2owx?g9)UzvzbyA@bK^ycXO5oxOp5`vqyKn+YfK(ro`7b ze*5)c$g6jDKkBTE7S?lp&8=C*G_#vQ8`Kb%U8_XDzrT+X+;t^nTj@H%uT`f0x!<%* z-R3Fa@9+0HVE^oF($clxlp}6ydq~Y3Nl4K@n+Kwk8?@)$GEv16J}@vqx+o-tP0SsJ zn-RI;xai$=wj`3u7GZ)4tL?dCc;V55c-6zEv)O6hcl$hUp!eu8vj0mMNImOnvNWL# z27`IB{13ey=Pe(XR@>xbMGP4+tGWGJp!jMc5 zs5BkpZ>(9Cu1%k89g3uTYYqRw=vCvNBck`@gRshtta0u)de=n;t+E^w8L*W8&aVuY zgbHxjbnONvGAP{fYTx&`x}fDr360I&`5D_8%$IWD*U(v^#oom(v$<0!ls$|%I{xUw zW4Ai5W9!W_Z_j)7o`rcSU&*CKzcm3nS4+;;eKbBJGG%lpDk_Ra?;*)8f5%IPilzpg zplC=8v2SXviGMkhDSfm!$@U@A1c__VS-UK>*lQcOYZe zE^)W%)9Y%y*zXY%mY@C$_S0D`GnkN?%QTjAX+i5+@fy? zSzUe_OPXJ9v>&#tzxNaQT;Or5e)I`n8umPE+yVCf$F(2Ze|G=|0Az4%AY{)4ryxz% zwq#cH62o8jBHz96NRsXFWpf~1Bp$@`Q2VevwDGAWy?>am67*gAiYijhf3BIrC5ecd zH|+3Tl@_1L>{bKgq6AFp8<@>pAk`|q+;tXiV)GiCw0o{2fK6_aNReiS*c6>JNj=vJ zE~F17rYq>O*pxB#@aV{vTQ#dP6>RzUf8Ee&^4vUKh>K?_|6zJ(M2BD)9ceatmGxn0 z%4(rO&y5M)o6vTF^NEDJe1TRHV2Z6)_fzr9m8DIqQN!!6zj!;id(cct7aAL>%cI3kS=eNSe&O5+-eQiWwtBn+Yl)TG2D zEN4H(;b6x!xz3-yS$=$Nn&n5Qlto(@d*HMDQzBD|K7DkY;4&5{pILBb;N?FU_Xg9! z+9IL9P2tIE*!$@dbP0!}8?c&b zAwS`OrZ#LDUF_STHAPByK7wmEe!&fM$1ClwE-rs0vWYp@#3&X87uC{;L3*2h_mwYNtm!=AQkYb6pbUD-0@47gTj|xpx)T76O-P&XBOB+n*U0p@z$xOm#JLk?3*2hBT^ql31$vO9vk} zn7TP-5EOST=HZ)lehJghw1hq#J1^7 zbkpNraQv~G?fqSOF-}9v`}#3kVidL!HeI9kLo3zJv_N9|oJ(a&QPQyD<)$+}5aISO zQW9M`dtIKoUK;yVg@H;7l3kgLn!DE2am&hvV%0M5?`-u%Qe!8ZT1sLDc7f^=7Eq#VVX_4PMEb>V&@hEeo!7_=SF)l$ zx4%vUR3w<-zgBlPbyj=6K%ZZpiSowO3s7>!$-m;tElPS1#`%l--um)WHdU=QyB#Eb z@CYG{zo@4bYrw_*kbnt2#|rpa2%cWin-_hwn;|{^Q@3~}q^v1{iigt`n`(+F5=iU% zc^GgkE;ZDR7DQFKm;6c$8^}x}m3BnUvb4@!XZ19Zl{@Y~a9d~CSla69TMDwEfl@A#qjAjc%HV38i#^h;_sT|zW% ztNye+K*Xe=`I3*T-X~HojjK&Y!Z)MQd4dOlv~X-q^`f`QlpU)(TQGLA6S~rnWd5cY zW#N}#m6oUOi%7T0*|7g@56p0w94oy9!yH=*Jn<(d%DsZr=--K2$WsDUk#g3 z`tW+xXzk~H|5<>ZyRm|S*A@g1*~5Kz%C!Ap_(G9Ll$4azV}{yXB}d3-_d4oyB2JYW ze9?<_DoPrhC*r<)d;pASb?qLzz60ewd3bq^bcHk*y(lKxe8CbL=k+)#Vn`o{c@q`C z3=er(EiHF6I0kqy;uC7umKg}b{MG-YV)_NavJPNx{O;~<=&OOX7GEvLvt2vJz-Chp zQ6V8dUfu}l;*uJxwT{!%)#dqWX^z2xwVw^(m$>oyD#J3FwvR#ib3Y}-v47YJ$Vh=- zZozqftX*hxl+7~t?eL|CCAj`4oKs~3O>3K9D}^RSSmMktaWn2|l`cdb$Qt`cw0GK3osyLGL17b~>L*%Osw@8|cdJjl){XBIUZo1>CoP zb>Pc-c0eEM*&Nas@JRgvo*hbhA6E~yB5v$oHqq7Ml0TvII37Zi^E$E=wtnXzQ~KS> z>$v2j8vYQ@PLkljU?cFVCjTSId1vgk&k^WjJc<$-HZTAJLn@5IW~bub2zJ~qwOk%= z&ysl>+u2)+wrxMf)t4$mw`E1|CSM?X#WXqmzNaTwpw6Ez)?VB4^vRjoxybi>6FGv; zn|(x`&$Oak-n-Wqc~9FQ(U-{=?YC?*oQ9VDi%p>6wS&`<``k?Oytl>AT9{|3oSEN< zgV%{$s!rcBi;_0m+WlHv17Dx#!<_5K$5OK8tgKkkdaIEsh|L}zAF(=Ju1`RtPH zZ}xUC{hybo8ARho0Rn=8iv2=^INi(N0<~^p`CHM{wmuwIGipNJD#jj zi>KiW6G9WO@iL}xa}P@!CnqO2yUUS=Wnf{l-s7VXdtzX-kf<={^(SNFyTa)7p<5ou z0~L$uL6z)g|C7tvNl|}JZoj?Lb@Rk_yU7b+M3n*bWM#Q2d~iVY>AH>!8h7%{ASDmC zwWnDl^L9~%rA5nZYy$coMvC)&pCI^LY%|kpp{HuFf}QBppvpKguB@W6)_!YTrN07m zp-rrnB6QYf7#kmdKQ^}NcmAmn|8$|YH-VYi9FGy7X~Nl41w`GL>$c5#@Aq-;PT2n5 zn(a!_ah!mrplHHP@xH1^;)AU0Amz&j<+e+Uo)Or8XC$?FoTDB_D!jHa_?H2(DpEMz z9xpzgy*$6X3Vfxu)%)@^`647N7%`KU_U>)N%4k1;-53`sQKYx5DO@r4?E~(d$!?<>bb!p!T+r2Mt^M(!G>?^p7!F{&`Jik^`Y*>NTH<|C zvhR{H_j9`|#0oGIetA6WBofH|k}V&MkGV-A2^sJBk|!yj0RUn<1B`8!E$OU;d;`C= zXezt*nIoWa&U@NKh<9jLs+%bfSH01z68;iLhdIoJ^ymR89n$Ej(Jq&O2cBn8Vq&}| z*cn@d3Hf%wR|PoTS0%*uE2w=sSoCfB^&T)9zdhyoii!qaiWoP2v|nP@{&ZQ{4ZTN3 z2^Y~sVy|Re^x=2B%-Pz*)21U@<5v4wCmlqoKql>VQ1U*p4p4oR4JUC|5jP^w@8P1b zTRt2_g&Ugsx$p}C-nT5}(s+hWxNm`YilHe80-rcbMDVKUb` z{yYEDte>FRJK!u&19Fu0nB%n6)%Ept8p3ig)&7VD-}kf;4OyAPQ7)~i%Ok_x!Brt8 z45~OYv#}}Wd?{6rhX=^faCdPc&`gOO9Ig0IEVMo2P$fT{7PCT4O$`VHc0$(+-?u+k zcX&{HS7DB(PUITaG6c$-PFwDIeWl3%6tMeJ(n#&Q^iK1H%uJo>cRR9VEfI`$I&N?U z@e~(>#`oeW{Z8&a_MZO!wO|((25SmnuYbC^9G|ROaqe1lEuA1=A#z$QxQToo2N^P> zMK-)y(57vQpR;Zc2YQ_;C**-sBobF#9s@-inTmB%KPHAeD*rkBHah&`0*AR#f0&OB2an&H?r_16u zU5^P=j1T}&DNtKZi`Ae~GbcIBnhH))L{tmY!04s%y|966eR8cL>7SZBF`(8rj}?Vt zT*S=XA?2%7=;;~M!MSS)P7aAxKKV+hTDQ_cZt`D#3YK>mK_-2Z`LH=P z>uD$qUEe4Ss+}qfia6_Vez?Bp4Id~GaLr7i*6WPp4#Oh#b-$9xnk1#Mo(Z!SX#7Z> zubcZC&e-)}^l16<=Z*c^^}vzU>QAR~o+NS=r3tX#fy^SJ!$$X?n)=S?O_?g7|K<5< zlab&%UXB3Ig3TDMiW69!T}^*1OCL?l47}5iKlik>W9;c{YkwMCUnZgZ@s2CaJUl!| z9(s>F1S1HPM=o&|1Hf0`#cf-<(u;dhYts68mr_T|OyL zi}G-|pBx3%TNy>0IeIyDUD3TUcLYBtEHZ^wftPbTawjqm$}%vF@3K&F8#(1ae7_J* zteQ|txIsFLO-Hk!@2!a&+wL3N5Z0SIP%nn3o?e`-w>Y|yc(?karByJ!1q>FG%7{tG zI~uWDA=P9|&0WpSow)957bk>=ZF#~uIR|b6jQ!3rwZkn2f}Z!^;xMs-GB$n>NTcd$ zC8R18>$7T=F_T-fvR`X~AzA?FL-mW#M?=bwX8oCGk6c__WK;he^n{jTp>jmQ*&Kb8 z6C?a@_4s^!=?OzufeuDAIjrt0Wto*u3Sd3;8~a-VPbN8t@C#Oa=g_wk@6#{Ik`8rMR|&|xulio z-HMjKzut|TdmH3NFy!UR^xdCt9p5#8>_3=s5m*VLU+&^x+@NE%zzD27L0>~Y*n&4` zclHhNIoy=(^xpad6aR>lw(C*7I4$%JdPgHei@P;6v-Lshyz6vy)X3=_7#4|6R~tq$ z?XZbHy1AJ?MJpOj~tPt1DtrVkjw(qhPoS@o-MngCB3~1wUg&=&1Ge%djD(gk+jXLVZ7<3DzsJ$A3Juk%75|0-cXUT-2GV~$NsG?Dv- z+kIKGdPPe^0jrZn3nDyZs3-VOZY~g%7|C7?Am+n%y%PDPlG;5pJq=m!4k0-0>DIJ& z+U`3FNv4Z1Zs%=LZx@KKhvoWW-x@bjVFGlD4r%ME?BV27d+Yc z-BOvsTns^lVIP;mo3pk5EEjv)dH2v}P2H1|lVLd1=w7tqRU-7p@uktZrg(|~2bKW; z{_XtwFPSoPmSZ)RQ+B5Dni_r{x>!fUC{JD1A4TpxT+RTH(Clj)d(_vHB|H!EF*u<` z{6Gm8RQW~e>fWxc{BBJ~bH<1H5CfI&of8U#zyyh18~>nU+?QFC3aQSCG!zgnjr$5G zv1DWoLT3kASrj8QymF@5TrzYIb_DrttJBbPsUJf` zkqV8nDNd)Ns=-c7A2v7s*4a5(&&|PqGPHb7DK?76iSShwHM-wLR<`8B+aG<&U%){u zodXkgvJ6Jw)EU{Xur&S$@KI-?Nh)RR)mH&)^G(rjNhf^`{*k@-w75;;`a5~oMn}TF z7m*1WGp6ff<+_`h)i*1t_n!K`bVz~N2w!)%qJ11~1oc_m?hYg=h6fC#D%d`A)qR@0 zDk*)O%UPO>P9uAgzG}96JQ2*Z^VYNln`qTRCVZ!7wOYF0H6>qcWg<^|E)@CU10)(CFyt>0xb!lRGZ`rry?( zrNxC(Fub%!!P~_J*E~&;{x6{g;q)QNJ6;M$nsWiJ%>?X5|Ex2Sj-GRSr_d;C#k~07 zZ!u}Hs>Npgqo6eRwv^91wFD5^>U~v05b;1^9_yKT|1iERv!t%!h@Gg{xKA(`l+?1#(2iw|<+g}1K zIMU1Tkvq7-%*MZ`v{;^vmP*HGcM2Gtr{rxAy(HDAJKDAll!$e=gN$-;m8vq>$<=+B zelu^#5*k_ri3!8H2$zZLyY=78hU>-FmB9Kc^EDC`^C;6ZX;xA2UF^9oH+_5A3S=Bo z5Y5JW2Jy^#ImlKlomTP8r?au`M(Hv4a*k7}gU1|4PLIPym3v#(_o}`s0sF8iO}R9y zDZ_Ep*3Z})VC~{kr;YI+MW9?-`1Z4zGBFGx&)vK4aG!{cjZGchA+o6~{0nASePm1( z5P*FL&a9?gc$9xZzdrP_&oiiXDhip&q+&q@UZIEz)SQ}LoIo2UM4m*k$T+q^5U zDvdZuHEOu@0cGv^&&G2Bfav`h z{%dFZ`)^axPT0glOT6UrKfrj0yo`*u>7%x1duU0aZ{PsaLc8`f?4yYDNrcUYE!HPH z$&oN5>-HAb$~roSS)H@E5MoFD(MF!-_W#9eu*S9MR;Ho%yR`h_X;GA7Myb$%*ZLoP z7qV%Bv)Ex_%aira^4YdLlDg!^vjs2EV$Gn%3K3vylmXM<4Sl0fJT-JjJZt;FdZ_&D ztPKdHxzqEY9PxmV+U6BRKx9Z_86{L8i=||F(&Mxoul5wK)V4xynWbfFOuLJA_S#qM zsvKE0??i-Bd9WDp&An$n1Kfq6%HsI zU}kJ1Xu}4JTx=C5<#YGoW^!HTXG+9@K1o6w)d3wGn@M)mW?}W9dzZo?BbA*16LziS zI~%V)$_lP5YS9|)x3H3u2* zi7*$9?#{`hD}P+Z*%|RxHKm~D^wp77{K^Q3rrhy1ti$F6`(tKAU8>SvhXiaq zfa8dWw*MY)+uF-V#u%FQS+`DF1#*-{bKqeg z;8(mC_5`bkj!WFS_oI+3mk`RT4m!rcnO$YhtpdXi@WNu!$j4FA$OkVQ*B*&U1@HWK OS$Sz?sVWKM;Qs>+Y&T5+ literal 0 HcmV?d00001 diff --git a/docs/image/ClickHouse/gluten-debug-cmake-reload.png b/docs/image/ClickHouse/gluten-debug-cmake-reload.png new file mode 100644 index 0000000000000000000000000000000000000000..40d2c777587432040a1cff80a04a2434de3c0072 GIT binary patch literal 412031 zcmY&<1yEc~)9zxygL|;WAq1D;vbeiD!C`TCcXxLSuE9faC%C%=x3D-|zWd$(zwbL$ zr>1Ia&pAEaJ#)IBr#nJXUIG<~5D5SPph`)ClmP&E3jhEHgaG~i%VXc4lJ^P9Nm)V! zP&4`I2ml}hNP&b^-Lp@-?$UIJ5-uKM#&zbqe_xW2p&$~X3&E@l^(A0;Nk#&{&PYt!xo#>v^{qNVV%TRUB|X7!!RvRqW_#x1`h`$`P!TQ|rQGwo?n=ie!e7Y2YMYMh z^FlCtk`G^b_+pN|*eEC{8r3$5xDwmj+ZFMlVbIGFYcl950c6QfL?8U4yH)?(tOh{~ z{Tc>u$LLv(kmPiYd7A9#BYBWDhCFNFLLf2BUDf*(Tp#SXwpnnwBp`%vLgne2U03BT z3@rPvB8Tju(uBc@G)qE(xo<_cU!R9yh!r z{id-s_c%Nbaa=8=@{`q@MV~%~^=;d8|KI7}srRWE)n0b=Y;J;>YrM=UQy@qE>p%Zt z&n<8^-A^!-;mx|7AL8^iYD97J#k5dqO?F|`^bYs9RkGG`6m8i}j*7u`ZL)4+d()RN z{j>Sypd$@Ux3!gR=(Sauy#`BLd|O5_m-I?yV1Y}~?_9q*t~{2^qr9xCl|UtGq0 z&RCGuiF;=6i|XNzxs~QN=WQac=MVQZAqNiajv2)NlG#L>U?@)(DqsY*zv5adMVNN3 zmR^2YDZ}?ig3_~uhtHFW-Qh$!*HM>c!bxdPzFna#=;};?W|sdVDeh%KVueP_`5Vs< z7bW(2PN&+b`#qIz%m1q5zgoE>#kiO|YMfopoj^|2VMq(63e-s#cO@E3lrNe*+aL6# z5$BQMeRenwTTOGpn?&!iB-MVRTDxY9t=EAnainZCCpQPN&u5IDU>QG+ID3Ju5*@@Z zxTgG;mU~!Am%UUBueCnfJYkPZ4!%O@>W2Qu81HY76-2c-gh7*&5tS7pkvhd?{pN5s zg(iXRGMj`&7pi}LTEvG)N+xn_W$r*;`I!cTv3>CX_*Qc z@>n*(MFJpH*KWo2*z^`npOO6USl?T{Lne zc7=x12raI|;n_8X=n%wn7W2!ge>gh+XF6Y8VCfB%=2pOT(U0T_v2}US5k`b0w;ihK zj`OZDq^|-aIObg(hR*lV0{|7b3g(EgdT%Gl)>K>+UNCHzTTK_xPqqCLAfzor9{ zp@|EpV+(0(_M!)_Acz$TNye-n9Madt`z#src@`Z%) zFlYRM;a&K@km9GY!euXlLq%cZ5yC;i)vdEi=HRCKgx|x-p{b5=DhoWuREdqT4d>0j zfzagf3mdV-ItNVYvokBW^yKoguzC$feUD$+q@TQaG9)9byU>?JDGA9iFqIUO24rbF zSv!puhl6G4oM@mlgU^l;Ii&q_Ooa>Ph5IA7QfxQ7eU8=$a;h0ckdkzAP2$3p18RfG z$eyMH0R9oSWONvRqBC~Lgc@RY$t2l=`Ufj8ix*S}UCGHX8T@2C(KPDN?j(l=gbBdR zCJu;n)tq)e{LJLk+!Vl`GE!&cy+3AFH6UjX>CS?(deqyurr>F4X!D^)^!%CMs|yPx zr@#XJDk-kUf1jS_lJQh{d$EX6kr+52IMZ1)#8aunpV*9Vql?`SYqN1;YRbvl&7KpT zS{#`sk%Mfs5n@3zBXuFg640A<7=jvW8UX@#G4U7@7eU{Vi`Gf*n|c zp9F2elym{OLZMrUt(#J+nB>y?4=eR$XGXM)-6BASWzLt{&m)P!u=zkX^hp3)&(iwP!@)Z+*6eLDO zq9fqqP$u730i~%6T0J;?Z7MXT=~uEULjH!SclEvh@kqHsU!bEiq7 zN0v01>Pm97a4KBS31gHw2BE$`4}s1oi{^X0Vu2g9qs4NKYs z4pkx&%W$~a(jj~mUBwnw5&OGJYkiKK^1RlHt~knP3lIRZltx3*G!8J-%~^EYWHCIp zF2Ia-kFJ+v5f-19oP z>*lpSs>{a*{}i4mz_?*03>}#PQz)1psA(igHbD7?L|2{b_sVqMb!UfK6f7+J3;yoR z<1XjU2A5cPhMp!u;n(=uDwlt+FJ=)%iGt7h@Z zA^8KrwHb4{Q-XKD)|NSXURv{FKcZ4&7=fIdcKn_?Y7GSH-8SL9Fgb0}1C#%JP^yqd zppys`ii9=vv(E?HTl_ru@nMHjwO#dlc=$}>XoqaX=g16asl9iv_IddA0D*KJu~NH& zV8!}Z4lG4*s0fe|={^)D3?H<|oGh*jc9@0!Y-;5o=5@dCyTL=j##sJjZT+E)H9y3b zrz%kG0aP`O0)ifc1Jyxful{_!-AhuiTl-M}5`Ny>+S*DP!$Z4x%kSygRGgc3-znu7 ziRaw2ty=ZI%k0?jKHJ}Zt)WC_TiufbNC&p=iXC0GAt(!0^APzw9+ffaKbBXU3{k~y6oHO#^O{*`|fOMTrG9#6w_z0`*W)@m)S0uc@V`D3ASGymds*WB!8EtS_* z8mPLe+VmIViuUninW_yz+MZ-tqjYtP{ zG7ygodjeM&w8DneD?-syK%xl>{^eK<>BQ^4FX6CWpbf<}ppZ8tr^JjPeH&BY;q}Z2 zd@+5Fz@{TI2T}gYa43iE2U1oU?#pDs5v}Fso+DC!(-nhb@z7^)=(3xAbh(?T$)L;r zb7IHq@i_k5Vi^#pY5mzMh@hzF35#+yY}X}6tzpAv-J_`M7a)VSA$MM0Ig;S{@l;H@D;kb>io z1|i5oV^MBVbmD!h?zUV)i_mx9erUK(^m}`#O+RyPrK#9dr zC0j660~$N4*us#c+U{%R3_;q_8!4iCV^a1+^s55y_&4+~*I6V*Q`|RKDP9*kJys7p zH;3{>$|5D*k!v1{gh}|8ZJzoM8{ca?Rqa9%w5xn!;fyCjw}M70aiL4#(DC;Z>|q@M zp+FS(a4dYx990I4z@H}92VXQV*Vq(Zx4C*gN!Lv*m|8G)12h3hM7dBuFJUwu^WXvzQ)!Ht)^DfZD?Xf zsGo(JmWj6XH8R;TlFv&;BmC)0m%tj_C@hvi#ID`;olbfgK2SQW%X)6B32QB%Rjo^Jo#nizvH z$~byZ01-p-4+F+U6LO1#ij$Yv&y?(9-wmFx4QSeHyBeN20O)W!pg_JzQ0x4n-AbL2 zxQmOM`N7rba3$l6?0-NmJ)`)=W6 zF_X|u;ITi)ZR7Uh?V72`#DPiqaiZrX%eoBhbhVX9e1rdP?Pg%g?(Hh>)m&ogPk-tX z-`6K6u!7%2wg)eX=iWr?#_d3BCKVz(ioeUr0{`RmWaVhX{nBuT3e8>jo$s&1oH)UH ztNBX5m+iN3s>Baecbk5jPH{Fb4+_5bOS5?=$7fzHcdO`dVnkeaOWFop)@CsmPyEEy zUYkBQ$IT~hwd?nnMGC5E5{X4Yhw`LeZ(Uv4!E#+|ZBB^K1dw-;+RCJD8 zZ7TJ+8Y>&VH{Ry7ZrXHiNT>7NCPS$wOz04%GG^b*)cQGGNhcMQ2MD;GXXAcgx9n=` zt|_qFtvar(nOxxag6OupD``g4-xemox7uxJb`T=A8TXJc-CcJ;;0z}@|!VIb#F=S&w|wXUe5w!S_OIoIi^ zWpAeXnILKU68h9y)1xAEaUuA)@&vzF)A4o{Mq2fSi8yMRz=JFIFKUXA2Wr*=mrX^M zk{Syud#Bf;PE1S;==$em4%(S}JS_Os^I^I^NxDMrHjC)k%3Mvu)uqL^&U|Vh3NHr1 zE1?<{_1zNVxk~cBJL`9wtA2a^rdOW~^9kq$IM)_d_##<*>Rk*?w`7<)nXMPH`!!E% zd#-kd(wMsMHUeM^tPW1WuM?Y{Jtt^sOxZ7EwcaHi;LCZv3pf_bq-r>QDbt~rvnqdLO4ViJvDDE( z3!T~K*B{cAj<{5#*5#W&-Yr&I+0gObWb^4lx6_@4oUmB(`(t|y=f@6Sox9Uzr>h+? zM&HLzf}xniZQ!#c1JA$Q5Y;n&(XGq-?{Gb0r71!|E0{Ky{pMiYs?%9`Ya*N9 zZC=lMr>|@1<0nFVw#&Kfhno+Np1bud)noe4qZ%gb8m<=`a{5O>yri0bQ;`{J`}vX@ z=1=e!oeoDU9HdnGO{PyR6D~x7^BQav#*Gbd(sa!gz9ypUgIa9#?K~@InN*26aeNMk zIc~2{JWR2*Q#CzbpLu;?Xsg6o<5|^)~BDd|w_j z*yuZWv^RX8&Q2D2ooRXX>A+8rGwX{UH@jPua7(cDN{8E23d5qLW~4jH?*D@)|2yqC z#CN9p#G*5x@f3oV7h4HN!-haYA{4n_I2iJSzpS^|4o%)N+;!fmTPrx;q_*Sb`aGHo zz9=0E1&T#_)UU@wg#Dqt1JTs|%aZT5CT+8hty$=B{>WK3>QvN^RPs6bcPb#G<6VMh{`(ebHTNbM^4}wm-}a)9K!On!%A0w!xhT zs!&K!ZDDHsvfSwe%$ot=!ALk$Y*mq>>uCW5}h25S6P8iu&dKP8!WH>=HQuQYN-rD7v{35 z_A)$dNrL~V-?5>r%yt(cA;{(>{;jdUL%XKudWLjpaEBrFi~DJ>Tvu0@M63ZE$l-eK zibCQeX_&jHXp!viGaVw%`T28?r}f<2+?CL31;M)#gV%dM6v%B1C zZlnDIp9?)}0_`T}kszIOI6x%&9Hcsn%Wmbem-j-;E7$Wis^{@H8vN5E7JX(-a#m{m zb;ckd2)`d4Rp|`px{Ag z1I{eWJdchs#l#NaG8(*0!@MhR0cy7a8fBh7XS#-`BD-jOKsJ@ zv!Q)ncbv^Gllq>Av5Z+F5Bo<*!{lg5jGh1?d=%QmtH?n>RbH25@(7m2fE1O0jS_;9 zkoOF7+GmoZ3wN)(i#%cR=f+S}h7zqY4E z!7lp;ZDljElmlIU^Nk_}KHsC^+(kj2-4r#H7^-xF-Ff^EUnkw%&Tfhtk55hn z-`b{LZ{mDs1aQ}{ev0yPRq-@|6--XH3J0PNZM!sVh>iX*>YR`))!+Vv_gc~BRbOx2 zZk&4CfA_+p{V|PN4vyN*9~GsQpPNZ1HQVi9QBT8wzotK6{Kmk!!hS+8LBL>Lp@b5f z=54XnGSGN>cgfqof9E-Ee2?3hUauttNeIq0gqhJ1)z|7Roa^ouYkLZs-KHK2Q*fk?FTMlYZxqqhmyf91AoYOlHWIb+T^T z!r#|dia5T%8Kto`rM~Kz*x5McHM!ld=XRoVOY=Qe`wdjchdF%btuD^7(E}RQnhWb2 z+Jy6CsZ4E-&b3!TSW74#B_;b=Ts6FI&zHG(n{v|ZJWi)Z)H3AjfVsKwEDA-Y%%AA?eF4un-fQ2wic37jVBAKT7&M_XJseJK20ayQY$vgg`Ll zlA-8*3A0vZM$MPn1l*4MnJ4f+9rrw)OfecX{MZ*k^9R{9 zH8!$^HU!gu1GRT}y@SJW7RR4-yR04(Lv9p4$)u{L*e+?hJ^sms@!E(`l2VpNr20mc z`Ll>ZR8Nn%JHlUjbnH9l!iL}TFg!fu;&y;{bA|1F35;2sdL52x2q?!F=N6rwV#=_1*NOwmZ z>EqVVS0AYeeT%amuiMc}M8}ZrD^WZtL_TVq`Hk#l4og8I`fq`Oj5|iwgls=$k&5A; zc(Q8r3G=nOV1nL`254}?nqYdHt;7bZ0s`l zZ;oQJxD0-@cXU=Y+4Z&0O572z^-M|Z?WS%VYc+B%+O~V%HbBEm;31`?rqY+_X#Zzu zT8d^4Gkn=f{`tMCs`5*e_nc=A#wQj;AxRiOQUeossm#HUijS3jX?~unDu`eN59Wx4 z8q3#9=g{ZoKpd#e&~80(oKz-XrO9nFk;#Gg#h|(-d_h$$2W)#+K#+?8H1fA)3G3ac zIZ0zeIr8Xw>HF`3GPtG>YV|rG{RM6xFV@+_*)lVAF7z6QaUW^vYRgyaApHq1M;to* zCT61RrGpH3t32(FkiKQEI?_`3X72&w&BvZsWXkd~FgEuQTW3S$DC)_@YR~35p#3ms z8|=GFG?vEraBqVKm(p++B-+!{`gFY?CD7> zl%Y~M`IeafJ4rcXCgM*RwNGfNW+f`_EtfHcna5^yD-$mg56w3RQn>qCiE3 zz*aR(8H|n#%4I-7C^u&3eJe_wZdWFTCR^0SJD=3Q2)G@%uPIPhuxC^2y$^Mz+l--H z_tW~JL{*V3Wy0^}RRknQmf(~2-B_GM`8@4L@7}hgunWEBAD$b5^%$A_jgNc0G2Nh> z{YdSbDv{z}(w_pFrLws_EtV5K&;)MRz`i|WSb1b3nQG1hl^kf`b#}M&Q+yzm_R5D) z16G0msRbZ}LWvu!H2~{2TMhiCc*0{$*U~!n@jhx#D92gIaK4SstSq-{^UM`Pp(8Z~ z1ceJb?^p7++W6UTWVcBh@AC+7vP(k;Nb#y|u}IUKk5|z$`0ZRt!Y$Z zYHvJ*j7#BSH`s(tn<@Kmo$zT*k13XdE-FSYl#2vh56sp*jv)RcY) z8cQP5bv&Z;Vban2AO0M!M{zy4F7c~zpEg2Q8bV(wUL=A1;#f{c4 z%IYkQ?T+FVR^U!b<_{%^MTA*At{eOvRg^sOJdk)YJzLwfFM6KGAw8YXSD8seA33?2 zsSA>^tG06+nC+7>X`#65qye-mKicfpeNO!Dq70~ZD7MXpsY6Afr04E`C`Vt+92szM zae;LjyBjKj%KH2r4LdOmF%iAT=A@g->RRgj=Ar6qD;wVwjfY7j2=pte!rA*gdA0S` zwMI%3EUcZ7S%m3Y=1U7uk1CG|-jysd`;7>vyEao>$E_yg{nT0!#O!!@LGL9cr%sPe zEp0XMf&oH*LvTuC^tkU0^NCa)PuEgXy6P{A<9is!Qdfas)6>&SdO$iAG02srxoJH= z;wbD$o7}X_J4wp`DI%V4_WzkeQQ)BkbS75F)kzHw@vLYYmrEAH8s-5dBOKM8(eW`` z1c^*GybmW181NEl$$?b$2?+^``+Iy#zean>CHo_mG$ka$m}L~(3ZVh{$;ydk8*mgQ zs3B@NsF{*PGQ#ML0-B_PPOnF0dAaXl;&TR$8b-VQdGr#y6oY(8rC!}Vxck9uY(5Gx z5^#x5`bnzJ!1dvJKI~*cb+TNGwbpEr2&}4+`bEpZ-0}^5C~Sqd-6wLm*gO-8KBb`I zCt;|M$8I%0lkb`Jd%O-?1Pd(BV7tpO&$I5bzg};3iz_^>7$R-COJpJS`Kxqh?lzgp ze^@gw^Tea$_4ueIrt(5xf*qbQ1m_*Vx;K%_dv`M>;B+;Ua<}P=MSbh62QhDOKbw}=T7&vwF+(pYL&{Scv`ZVH z6|LH166*M2WA~eB#^#eTVi*}2!Clu@b46vjzI+LiuCA`qw24_?@)Rm*0SbvQMTmI} zw8>NBifrZ+9tX?jI!vGZE^@fUl^}Dwam6&)0NU(62B3Ig-xX4HI&=@Ra58hr7%z5~ zHD9-Qc+!=-g83qIO#l4Wd}>Xb6K;)x*Hm+qIq2OHD0#lOI>rJc^5s7h5k1jU;Zp3% zw$qskwKg?%)}PEPEBHVvhA<;c-wk<|geqB<$|1$RF|AuHq)m)VkCUM~Hy$UYo5 zD7BaVi4-aP+oGwkL6*S+Lx>%r-yc`e3Mz03eK0R<87KL*mP223MJH>w!Qs4+s_}s7 z+!FkW@DDgLCN{!t=$J#T@!;ca%iYJ1ES6CFPNne(Y((Kg%k(#qd;n+|&`hf-O_qvh zkN-V&7wMN=gT|*@qGdFd4RxgK@R;VNCVZ1{XJrgERRVIapA{?zXKtHauKPSy&4tFI zCIT3e@B}-SaY~odku(hPXaC+7Awv2i>djTOwDoP2%hH5w;48#})6(gxYvRulr$EQ%tPK$lQl4MK+WGtst9)dj* zwzTAwm4I>w)bMJ;lpH%7l{d-g=>cn7ABro~=DUR&gIVWWD?hN`Ih3xbsAxp}(mNC0 z#ec@XzOKjLMZ69}QD2PP?YNJbHlf+kChXAJ(|9{GzmW3%{Ck(zx7Cyzih33U59pBR z!n9l07TuCyC=z^p@26_F#aaUmJv~HY@`?-&3QRQ#h|l!K+|2J~HttL^Al$A)VRe0* z8;V=as^eWH4&D@r9N{Ja9Viq*KtlNGhQ)6LNvt@e88(E;%F2qK?`=51ZACk4mAB(V z=x%5ejwWU(@EZ<*{W-kfB*>>8)aki!y77(*oKM!|p4a4V6lne;gH}?%(;y&--VFi< zl8~s7bO3+<(1-T0-4VDkb*u#3p8lk7GPsai)aU`M7nr_!cjJfa zeUS0KnOI>&bNcUsfI5DDy>-2H0?3lT9Afh>Tdw*Kg!e*%T~R&a>cuQ=zs9E^f*TE+ z%@y>L{>v@J&$wACD(j1@Mu+WP=7zLpGgFu|CRG_K#1WN!Pj}56txayuXt9F?(n2sn zq9zgdGbJ)ZeOieC8ck_w27tSv!cHH|It5~Gk24Mc6-%wb`>3$HJ3#X@c?SPCDVJ?` z`D$vw^aIK7_@A@w{ja}A_!)l)lkBZ%hd#+*@_4x$Bh$3p$*>DJ3rOL+sc1-!&lH89m!cPxyy8E*s(tAyBJRl{!Ob$gxcT+ZW_>ASgI7b3$1C04hMUv% z+;NEiq-CSo#zEtz1jL%rRIv!L4H*Dilj94=9jKi)svq zOB+fZfGb#Ndh{y<1_q|yy}_{QpTx)ZFnNvx3hamj87#akfikD{((wSSsIeHk*my*J zJ(W!AWDeJx{~a&SDrc3}s`HEXT6sdSUswG+9<~Yw z2TLlq)Ajf++MKxDsoPuoko~2AfG~CfIrDn}bN`-f#?38Kztqbtu1I>N?0++#^sR~^ zIK8m7vaVfkIQy00W>UYDVm-ktcyoXHb-(9zpDB2xT!dcggL<9HwqK%d`{mWc4a3Po z&6OCUg}pucDcEkw@$v66dGdR%dV`NI-PdnoVlL4AcdO;>RqtF*JT)6Wp0Umz=Xa)b9md)2#o z{oa=^wR2<6{c&zNv^tBbcd=GaNjps~IgkIT;3`CFIT4n;0mQEE`#c@sW1Z^v`q1;Z zQeQ-qsl$^#_xx~UB1%D{KrHC^dM$%8G7k}Qzsh-Q?N=I#T9m~RZiMX7VZWC+o zaj>PWfyc$m7~W)~37V;0AU|eIM&m3WGJImFQlN#$K;btxN@tIU=;gbbb!I=;2_i3j znTG71x%;j6%7o4C#`=m98^|*znk@6>OvE10>hl4UmiN%FwlZGc4m4u^ho3NL2q!XW zwK7eWh3qTH->OF$BXW(T+S8+%ORPeRpLOr9O2&e0i!{FIxgP|qF!GXi&hRMe=@HE5 zVwR}*9{deN#}U8<@AwX(`jgiQA@c#{eqAMOCk@34w!7R|e0D8y+1l{B&}*^yec|_} zBkbsm5+voiLGcNP@`0NJ%X@^64Jc;4=Cd2WleB{(K;Fgm^~-y? ze>09VVIva)cZ`vc?JpRl?}CBb>$k)$fG9A%!xZlvtfSe&uYiptp176S*>4 z0XA8rjRRT~H1_!L@bJhisU4zgv9J&S(lUhIR@A>{90zz>sw%6EFv z>oHV){XHI-fdjv6=M*bCXifdANXWm*UiLE}9l?-MipqG{Br?2MB~dF%0X=l+vR{Ej zpw4eZh1dF|Rd4flBcRJGvFOF+Hp{AY!5-*)wChBg$Is?4kD!Ap5)eWbK5o$Ym%-pA zQ?81_9&ElU_;ybd_K}!G?tb-{%XXz<0-tSZbA4kytvDQ0iCO9z#bcE6hd*a=g$Ik?cxr8SC+R+vkG%S; zul)rH32}MtYNw^O7%NGK_v>z)!1FYI9;$%A-S+W2Ho)gMfd_WBuWN1_Q7x!%Q~Rau zE!<3*f{85ElkpJ~O=|Npm)}dBQP0LXqQJsU;_0Ne=QYDtXJKK13e~ts)ST(#JvN3A zrN{YN8>0)+7lR+H%6!}WPYkse>+LrI zc-F1I6b2>WYxLSs@Xb~l-|W9lRK26ld^Cd}iPtulocDuXo+Nfg7@M1QS~%Ygj*K_y z=X`DglXPvKi&660E4@Pzbd~q~oTk%i+jj99qtl`hDFtzL%MtK}{@EKOff*gzK9T29 zUfEb=dq?W$^Lp)4T7J-9FA9%NR8hOB+Tnuh;GN%E{rx*KGxSQw#kA(mGwFbeNNbDB4};cVfu=5BTicu~3?kyk%Nc*PYPsj>*RD^bm9?w7 zJzeiKUxVM&6Y<#u=_#)!htqlERFC0Kk;KPSuvMqy znRL|>SV*|Z9*bqbXd<=K31L}m5SD+l)JIf^M(vL$8NZtz{K2IqO)u-!3*Xzu?;15k zktDnhElmxjHQn0*@MGe;^&6850R=_|mL}HI@%Y3WF@)W2D)c31x<)DqwF)RZ=2>r_a52!-Be<+yEr*xu&* z1gSiJ=d#oEJ&)c=6CqIk$4`96M3&{N&(lFEdO6h{ub1DWGzd{sBBe;a+8Z7vtuD@k z1>LWW*Hdk8%+=_moLe}0x=&IP+@tAEmfGXhjHcpgFD?8wnS5VoZFNpAbXwbdC8!Ae zEb8u7LXaH->vifDpt&nHGvN+?7%G zG9z6Ka?Z>5iA#smZmGc~ie*0G#sc*I{^&K$}NIONQzkFe~(}khY`UxE{8B}P69su7F5GB4v|BNmtpQzKLY`)jF zQ7$%@)Vbl4f<~xn@NGO)gewbXaU2T@*xKSU-fDGd%D})Il)BNPRd~m3E}u^iIGN2By^O*(uz~$ zzo1vWf+cX25=Y-lzzPhbeHEFYb7_e?KPmoE1l3;~SX3hlBlZHS`c2~HHe)4~do_NA zcEYAWg^waYg-R0?CQurUG$!p~#f~ieDK&h#E)T_j|2CjE)GDGsmormlRGRM)%!spN z>`#X1-6};OlM8?IpGmBPvMh1{erM&U=)F%NM_+=$rfI(SXT>xEmVgZLrSs1+Zz*fG z7!cJx=_R;bLu&H*Y16MSFdz7o2G?Z^d8XJR!uJ5xn9fktFkk#h!5)=xM z^R?*Ii_pOPN~Y_@r^B_Ylz`HKiW$2(?C4+JM6{S$!?<5a=->Y4yancyd}&myd@gw% zy6_1db5s%7-g9d0l;_|y6|0ukksr-ZC&_Wz{A#CSypl= zAmU5TAK#vCE2=wz$c&gpZ&GVpu_Aoy^<3n-8uBBYKf^=)aooMefTk-UKZL!{56(!b@$Zcd#&` zhJ=rlx37*%tmcn-WuyyDg808Ft>_wLMWekFKTpy@pn53beD6ri<)AW96u8X4z@HX? zsoqZspc7|5D+w$#3V(Yd?Vc!BrqWs_36_Kw^0|0=QxgW{*<{wa1C3=u{9%DSICKjl zPdZ}$&CGqd5r6Z?P1g5#-!60PrBD%p*%5z8d){CW7_Kqq)~HHpmD#dl;e~{{8_Yz~ zvFn%8x#It7lg*3+_uv?pE~(E%*=R56gQbL7&9MB*ncm|D9LA6^y0^b)Jv49_SbQ;L zf%q6uM}CjJeyGNp>4@~rdX3VpU)Rt%4RCZEL ziX(`a((yFERCV_SY&TH3!VwqxHcs#!Y$>=Zs}idGK&nJr>>54CVgbEH+H<2mhozty_#B2ZrBuk$M~Sb6S{?88NYm2swM}?Tw}4V04V03RdS-HHGI8T^Ux9 zrqA7VzikQuck1H{ugY<$3Vixj;(4|(M$0i*J3ase7d3jr$&panDqheLAbnN*YyRxv zKR2IV{U$mnN?y@vMRObWtEr@*F)%^4wmxNqy#HWqS*QY|O`=|Eaxv{~6OgbTAp*I- zD};fGuO#}WXTP_06#Wn;Z$~|`_^Y=+_z=K8&N=&BEs`Z%5sn)@iNUVOXz1h`T8PDJ zpH}$5)$S$pTMm`~J?zC%sTw0dXcj5%s^%qqugTMM@rbxLnOJ1%x!dp}CSI>d-i}sj zp^PV_!3<=B=Zvub+#6xl`&C2O*yd7{miGq&)MKdaFpz4xjKmTBSE3KS119Y!ORb(9 zu8sC9bKm6O zI0EDz!{xBRv*6&}X&R$b!&wF6$X3qby%eFhi7vAA$3oz=nf@Uu$Q}-k{zkQruE`|q z5t$_7+a1pMB6bkxA7mk@o{-&^X^w4I>UA{adnAazdXmZz-y9DjU-kV!t#3hXhxJ`e zYtZt)>K9pDSc7+uLWee2J|Z$e=G_wu2d3lMlj!uZDHx%&JTfx|WqCb3v(+OLx&~`8 zkg~&6e}Vlo8!U91gi_sRXvp%0kj8ClP`N006s~EF*Fv*0X6Z(zx#oIf+zm(7Lqg25 zSoE6|N{ymt=gnek8^+S}>u8+#;$=o@k}hQOK2T@%Mo6%1gNGIcR@Wd$t7tSU1U9i8 zF=4^*=E-*|toVPjlSlg8f}B|=a*~IFo!WuD3yHnMwf@T$5&Y_P zqKv86U?Tj>c$3C`4 zuktnMfa^`lF~;dz;51D}*WMgH>LGPfbzR1+#*dfmZaab2rgE<+T#(~g0d!s%U3vn4 z0hosT0?MLdN`-~Z$N*A`f*kN%G@RSKRde}SVM&HXmQ!h$6E?78%_eU5tYoE(?`vH$ zp^?LKCYPy#9jAj8&Z^2SwJrx>Mj#{KLdezNq3CLma@Y8Lv{Bqq97BN8ekWW`FsT09 z!cC$UW-$WNzJLM~!7d@wAnP(OyhrT)*z4IIKdy%Q4@D-iTWgsx2H9t>;eYfpejJ(` zq4=aA+SG^yB(@= zRmL5Sj5m_QicN3H8Z&VKaa6S&pDq zD_2$tXI_(^{}de_;Lgv_&*g0LT6^WE&AOil-8u|D!d_7j;eJK?Od>}wEKl01eefDQ zu%@79&??7m7T9e27`&TxSBVw-&z{Ho*?yDu4Ep$oiwO@F!57<}`|fMf9n~Mk2mN6J z>#r0Afh4m$lQHvhxnD|Gyt<*0jYg|=cvW{5aBSw{DD zqa*Ul_@W$rXRsUXddhRbhB)4(i*T!?y*G4{{p{A=|p0-=6l`D72Hc!wAxX}qj z+BtNtw{z2#4(SS5j{Lay81GWv7B91|ESlu!VN&o2FO+*f{p$>~{_8!ZDbSk{)c40+ zF-u^N{tqFWpR?`U7pmTi9KSa$RLp2D(?|u&n6bq85jC6Y=lRsqVkUH+1^cQ4l9{Q! zOl;@{<60x8x&Bn0L`WinOpnhLt{ZLF2BTlh-DS0QyB>TSy+8bXeOst#VjJl^eEB-U z%FfO{y-EQb5VL`Jg>9##eE$l&OLrir8?1gFJSxr-9Hc7*EOZ14-h^I}+$^aKn`Qn< zGDg_Mf#{Pf;F^rcVTPOS_JVhr{cM&PR5+uD#9Q`KrsTi-m4K&iRu%R-W`gU}Hjqr$ zI6nv#-@N%?K$C4a`8^M47+4ytybtqqg+QVCYzg+d+34Bi z#b%b`v%pVfblo{4Hj{6S!iG+%)cECM73@qvASWpK`m18qaPHgTXRkBOvZcuwgrVOJ z*AF*`){d14}J;Vv^Ie7kH(W)sO&8y!(jJhMGu?fp~L<7d|83_?FGT zZHK!c+V@4uPhW_6qWg4Kh{rHovJ^kIEz@X$3oiKie`*1yw)Lr73f!$}FzutBM9p;4 z|L!=W1mmd|gcwL3XaD@LS6&Oi_NEf=YvnEYX<|VgS8)2#=3<{nn_>fYii#Sah&sDq zOCkL72l6#v+~{SqGX1GfASU95<7AE=obU$U5A0n(&ztBkf1?Zk=dSibPC0DCNlU6~!AI6m?(Ufyz+p(u~j zVJ~}xALY$b)CmT!jyyGn+gr?KES7g}?$Fq>4IP}$a@bfyczy2)11D|6*d&UB$&ao= zy}zIfZ`aFC9N4$4tFDaXc0~Ba4jAX;OR2L{4C_oZo}cDe=}ObdmZv2*lh1QEJqC{7>GIlHOd)Kh zB)HMRN`a}da=xgT_H4T(yX8rl<8J=EVWaNg6p2W0na%$OFceFqDwF397U`1~95BM* zDEP>kqH9AWcEmky$9xhKM#diKt9H*C?5wslpFkorN%93Ex3+bdkBEJR&?wp4ntx9) zx;pXXAs_e^#%6!z(kWz3_xkVuq3SK8+KRfa(FB4+ahC>)6)EoS?(S0D-6c2_FHqdQ zxH}YgFJ7QH6nBT4=lkw`-{-FU%*YuhXPmwETx-oa_fBxs1hO{-U=}Y5tOSvLFXksm z6p8l(z*8Zq52!aOQK9}K<7Q7eIjPOYHYbVpM-;`j=eHhQ?H+;5%ojBlk#>8%WXQWV zka25^5rTt)MVup6jvar=F|`IY$zJ8$O)su z1xwB~lvjWcInMh%+kmJSJ~n^Fw`VrX#+hw#E9D}%q{ z&}fhdgH$kC#v*9#8)Dd>_NQ@k8%pH=JO#xlzbGCO4iG3)~;)>n@^J|Q%B7JCeIPb#!YyE*uN;#~_;+b# ziq9%@(iz?jmzOQ|PA;tTAGLx$z;obM_{r9he$w@jZg4MFE4A%(7 z%xaD5RwU=tLA_4p(oVmmrPa=M(8cISe#VqjTOq|v-A8UQF_?1zX!%VB0k9Mquu)Z~ZFl`1#OfY!n8@>JVf#xf) zK3K@-j0OdNf{pU}GJh$alqU-wRNMLO#}^~4*xy-voR%NY0PqL4g|f5BJk;{UUK=HCw@0R<^;Pt% ztB&SAc7WP_f@nO>mRy?@R0?k&xn2eJ5t#nCBx?c?hEb7a)NQna zZr`HIOG~cLclJx}rsir-#v+T*^&|=c z=x0RO%^@{j!3cbd#3*y-3hrZ~!7&Ak7tR_pTcv?StB*_OkG%NLrlbj^7Tw+5#R@O` zVh0;yaLeBqKLMntu_4Ei!F#F^F>L$dQyuU@@pBo4* z4J3c|H|)IrH zM}b$e#zYMbcYp6**R(iXp8bTn^J$u1M}{*j*h>Ij*o7&H?%}_}FHF1J?{R-JGwSea zZ~W`mubc00_e@6Jeiwgx`H<)sVoov+WsnIJ8Ho|#xtDDb2fkbtvSdvd-$K7e1WutEW62vBB3&FiJT_)4R-SYY6J z8%iqj@t9f>CL;N03ED9{!t-=a42>)nIcK1BC@P0;PJkv1t!R{`x#A+JSvu8BCx}3 z^gG=3z9GM7PS?*aZEFA=uj841mS6XYUgy%P!XMw$e~(F#k{)R8b+dZ{7IT0{{un#@ z`))K_hvvp2EW#vQmc#`78cs$Rl*e8~UCHb^ySuC7+&=(_xn62Wh~&)on4F={7{9em z##vGO5kWAKX^N%xr>o7`ife4gYYP=x(Ah*kRERe*F(RHUO}XKJzfP}JJ(<_YCr;PO zsDz-I$8LpXYKE0jmiuGCQ%U9=JK1lF9Rmvwj6-c48*UAs^Ssp%6|@+FfR3CFC1rEC z7u0l&2VJh7Q~og&Cy0$>T78o2oN@U2oEsFSg{jDiHAo-46BY> z)15=V!H<=clH8U!qlIgTG5<^moAO0dO>G|iPDn>hZHE*KnQ;6Eqh7b&C{3wi*X>#U z=S_&6Fty}skC)4E$)kt%YKj$L(Fy>;;-by#^|R`)Y!Bz@@wqJ;Z>T^ShNN>qg)Vx3 z!#@FC2ea<0Z_qx(w8zHMldajtNI6?B4JJ=a+Q5YU0>{80_WfEhr<+JabGY9QiN+9y zhXo+&i%-F0M9Vdf6BmP z7pGg@Yy$c6Gc%HpOIJ}Ymc4jzWqDSycHN+fbapcCi|?KrZK-=FMvgojfY-m}uT%o+ z+L&zUXlSuebH&hra5(NCsEp~yZG{EL z@4E1yw~c5^K|$xM#;Q~3n{#uR1_4^^UY-lC_&+`%7__ich-qHYlesMX$wIjwZ$|xG z2s0`@e>K@brOOsk8HXK#8Y|&|3-{GgwQD@d-+tU-m6@wCuf##+bsSLEHu{Cq+*4Te2tM0{x5e@CVXaA8GY+x99za>V%f}bG0+%v-E*~9 zV8@H*U|;M7$5YL#{f_I>-qB{K8+lB433mNVBDYd{{P4q+#vGJkC4-5ot0Zu1rchMU z(sDn8aPH~pDMc}>QK5Od)@2J_p8c0~*ffgrb_=10j?{l4F=v5Jm-}#v2@wMKXz<;mY%?W;&h&F(<$CQOcd05fm+pe|%B;W%X|lE>)QSul zYHQ5P!BlkaBJP?symu=?F!ixzj!%Jcg(sWvFJ{lP)o$nh)u$)j-wvLh_fk?qZL==8 z5}igi76e2@M6vOah=o@}B}+#jqA-F)I28g3l<)XuH7YEGLd!jYiZhQbn=iU3Y-vOj z3-qaPgK9?4sey09cN;2zF{3^|QyqA?m9^}KMQOOiaUxtZ+i9jnup=++#~q{AE}x#q z|K8S+gvF^`d)>i^c|Q4G!GVk#tP8za{o zHNzsX{9AOCFG5#N4%7PntaM;>>m+(ub^o}xNeTcMi-b*}=KP~5a)k+6d}>-Ugasrf zKup1mhJQ#XY^SZ06SV=uu5c{CAZFnHJ&gzk0;UWifF0}=G&k}qLzp0;cP0;@69r&K zPD1qFhZlxHhFqUF$Z7)&E=+q-r%BR57$g8J6l;Z^{TSu;=y3c95S z8U~=2ydXNYIQ2*yliZX&WGmxsV@k}ib6>T#I^X*B=~n1{(5UORbZIJ!#}XE?3tY)JT=8V6xf1*`ye+lsf4%xz9!Ik1CPW-j#f|u;1*o;-EX#Pk3+kpZ& z*Xdlr+pXsoRCA6=heifDal0QvFPFyJ!$HX{od5tLdl0n=f&vWZfQT{tS3i64;(y3y z3YaFWcC6XTuXB^Wu~zHaeyw}kVbX1qq8d6vLMPXCL)S}RA{pY^{&IijOPAI&RV?PZ zDqe4Qw`E8u>!~5HvJ>GnND9?2E{;KRAdD<)Zq^ikdPY$7fQJP4f&pHFxXt#{8SBb= zdYOtV3j*|4OS91&@W#eEtxhIeADZoHu%hbJn{qp4{Oelm*YIK`eLH*up4T>cD}l60 zg!}<74;RAkP$_#?m+w84y|LJQDl;1Rj8!<9`4Kj8ao#bE z%0YMxqOSdPUu|@;(U$)sKN$sBU8^mHvCm}m-OYXxbp$ADugy&35E^Irk6qS6D&Gqp zKFq!-<+=NZYLK5!7!`th$zH$bx7WfE2{saB!ROyC_|v(r`xEKttB%;ZD0mFau$t8% zAWl{Gk|7`BFK{?`ANFU8m6#pC=}L%LGy@f$l{^Gu%}jtf$TA?p43sV;hsf%~eSz^+ z83owYh5X$=bHWbAY>cFa1aq4s00`ton8oQ$Fd{|(C_4G5)ml$J1zC~Z=>6Td=cV0eT7 zVW=sy1r(EF%UP|{#e&5Q@3V&9b?q=d$r%feXjK!|Mx6)&n}Rz%&vOJam*8N6;x@K) znzmFhG3Kla2>*2X&o43(7uK~0ZA{&7@_RLOjpG9I!sIkxJl|gfzp+nzBqMt}3~8Cp z?MMrf1{=RFFy(Z6oc_zdk82g?Kx=sKpMHl@*rEHX+boM{X7ibs@pH*gyRM#VpyR=; z=bK+uQ-6BxLu)HrgMZ4g;9|MBl%5=NpQ7#b@PQuL`~Fh*+hPBD%bFent5x{D-Da!# zc!_qULtf2s2-lleVb?f3raH=x%7^3kt-wm_G$DTem&<3zTp>>uM&Kvt-Aw0pdAwdh z?}pn-^56c;?fkC1!j)**K_s)m5lauQzbe?_#ejyiEOuH(sjDr-B-t?QQadA zuz>JpTEI|~t?EG^@raXO(EXHQjWvvJGYSrVa%0t2wHl8rUB;Liuc{(oOhRF@&UJ0k z%Pt7NzJ?++AE~Xum8_828U3_rJ!Xg8oTLMu4 zQgI|fl@X4M^0+lw;9UuK3dY|u^|DM0UoQ+nH3O&xot7fMf2A(e9 z2n)im^!uJ!k65OEiUk2=2}Cevso4nAJKV+{DoUovi0y4Imy@j*768lgS zCF&i(_sHt1SQug6Y_p=|)0hEl_gQUrB2q%a^s~;yrF05Ws4ok&R0)C*#`|szYdq0C zuc`boc08T>hfANhH-qqzmtiNMlR~S;5X!66tu@}bqMA&tjw9|&_R=``<`T@>Zo7_! zttLx?|8)yCJJ^p*T(aZ!v ze~n&WR&1AQNO;}U#S744-?t8g0?+@_FIDSncbH6eKYz5dDN~}}9v1XFUye#5!X;eH zA%$Yr6@B7pv>P`vpJyd|d;6)PHbj5F|BY@2FQyA9<(Wb>StI%B`T6+RGne1{O0CN0 z@t+0;CUK)l=F%5^y>1UUZT7$gegHuy3xSh@m>wNX*q(Tn&0NCYR;jPE9@Ek=mtmu=(5HsKe9tB&aBOFJ~_kd-GFlH3Oe7X?`ghwfdpa*1Im9 z)ts(BU0(KA+1uwU54gTyBtkJzd^{QK>DhAKWrq^~<8k2I!pKM&&I6`zF)M-aq8R$g zdbj&{6#8bz`htMvR(*+q9uR_=KC$k5w%KGp`5dwFkC*`Z)Q|6TAEt!f#~qVCPjMV2 z$AMvA`(xU6hH$2G${mwm_Fw?!IM%&bY7 zUe^PkLr;E#yOhVbo|kxBy(v&nL3t?8rz&t2h8^Mp_aWxG@4Y^DAb* zX*Y>0!PzVY6q!gbXyXb1sI*|X-og3DocAr zeLhfwIr!o&9Y5@RvtjM|*U0ki5QxSkDq-FmmW(rNO4psh!j~$3QIr zm?v2b>I6;*Q_A;km_I=++onj~l|cweAJW{##lr}Ps7J~TG1vjE(uBwi`t^+sw?88B zz+C$OH{0|=v(4kK6{P^lT)K?8DY`=CERX_6gdE3oY=~$WZ>jB&S8OUaX(^=KI7&vs zou4H+F&3g@w3erxpVyn5t1y~VoV>b;0|j$HD?5gxloCMs@`j}y17YkUd`K@;YepA4 zihi=SWdewlC=~v)v5~VG6iJaNZC^=^7rWe?Ep1s6P!iVkdvOs)*wd41m!{f9tG1)V zw>Q|ksj11kx82WF1{1EDDk^2NLHBuDf39M~<_a&w7EZULqyQqm8 zD}82Wn^afb3Ex9AtHvqI>CshoO*c>Fy`r>rqf>kXGIbrzm8i?TAY7pA3*@dZBb#u zMdaK=K{^=Nou(-msgX*%vp+5`pgEnKE~%c{+X#= zfrnL7K+EAN`{&%3DaW@xEdi*_)?L%r{{l%hFdK4S6xr==w_~dd6@a{!mZkbiC+<$? zouRtAU1-mVlx(_ockn4?;E{~SWp}ymYtl~%oh4-^NL*|)eanx5AK{FJxAGM?$IXd) z6)jaBpUx_uEu=C{1+;jx4KoKsfq*N+IvV8kFck}0h-%{)=QE*-uyA9v&C08h+=1qw zeWZW#;}MPrY9R0D_-CBxIc!P-ad9U!k5val9`CEK_%U+!uZhbi_IjK~9WFY3y;XWG zIRehR55A7(Gp4n|$*jtl=r`{AT|O$mPKaFt?&dA=Co;1M&Rl^iLwZ!gU*JXaN8*~s zr$~Jd4igIrYpXpz)aW;jMgK#VEFk7`R zphgo{Ozu+H#HfjH&|!bjHZFjuPp5%sjR+5$D4?fl#0#w+&PiQn*}oo1u_I%AQw}?h z(jGEOO&=a~Gae-QNJ^+^VXEL7QWuAo$w|uhvOma{yNuz-C)oLqDc7*kOXPVkAZ*TK zXKW>DSdGBi+PXW!HfFs)~ah1gq&ay>DmBW)*6~hF)hXjEM$d$l=Yw z<=Vb{&~xm93TFMsnPX0hL2e`mr!AXz>N^G7r8PvUW7#8w=Eq>E^NI>R?WMpHu7zwc z#CiH#-4*wX<9a`$6#AXZ!(fo|0Z zM?Nii!Cs=MF5kq=EY#blln^-#9s!|iI;PR=@mu59jxmGQw#T7Z;z0fzn(I?f^~3;C zs+9AU21}^@ER_PU%kIc*p%e-nZ5nv{{VKI5RDYlj9D+eVnl?4&Ug2oaG5UE8BxkLW zoNrS<5T8qzC{0Q7!RYJ+Z|3ks?zkg80oND=Rx1Cf%5pj}X7SvB9A1ql3MS`+urx5cq5CPnAYIe!`%dPsx zUA4jTl|vw75mV-asufVh5-f8bEC3Q?yEwIz^CmZp!%It2!Gdc>{U*9U41srD=41^b z!ovYE?!wd)z_53TH$G-IU$him#9cQ_^>J(|EBJci*TLf4NXCF6%27N{9%qO#!nBQF z$HV7o&_4#e3t(b`iD69_kg>{teKLCXxqc}*F!C}nvy=ZCnobDqcT>I3zrXaf;xzWU zuoz2+&#eQD!#-s~SB>xXYdm^SYVcF@U7QZu=0>~gaMKSVa^`P{R*o)dL7fqn%x0Lg zr`1zaQ;K+$IDmO%sU^n^my>>>EFpi^D74GE#HOC*9;@D$oGoI*5}5x@Ngs1-J{(Nv z-iPi$o7)U}&8V2rXGj+K0yQ3kE>*`a@7E`H$Cg5=c$i;M7XI#LfB>2mokq-BI5K7} z-#>jmXAT_XVjnCMq$81TpKaai9E%SP9BaBn2F}NYKPH5S+!T~-Tg9Y>KNV|$9mG3d zT-s9YU>vLL;XL*9cjOfdIkDh>|v#5sv3_Z=MH+wwBAT?#Ani<2K3 z42*+{ZML6;H?|GDhL)BVwlZl;b);2q9?Cb7Jtnm`)4jpipr9bnP5=8@sgJisR8_b6 zViEB0@O0U$jb*I5C11pn07x8MOzuRa0MQ@`h3J?B7y#E#Tx@!4tF(pPiGLl7dNN}O zw7=uZ|Dd8kTPngcB&3AVG-_&g#DF5z^6+r@^WU6~8=d~ocmJtqBw8#2+!i`H{8QT_ zRwLxAQ0;f^v#-ZEC|Dcroi4Jd_&Z8p^ zR)hJuA)M2qbF|00VQHd?b-tD}-Il|a>7E4VZ^_Ouk|sEK>VVLkKu@^bpzl|3Ga_u| zqD}Oi?4buk|BhpwmdQx%c-&j5*|4_dkBa0pFq|i!7V=jT&C%$;WOpTy$bQl3qU_}U z8MQ9r|GHz_n{3^tnSblYacnuwt{bI5Se=-nr>Q#&Vawa}GQ{`4Hw7aUrqoL5@-nl1EB4kwanLxRjPoqIa~dko0y<|IObUt8l7293@# zaPhWtRj(1C!VfB>GAN1|!=-nhouP2GcQ?HcS1js3?C!+3JG7uELm?Pel;3IQ=Gzcg z_m(LlAUN&Y@81-(mq|PS)jIXlFdnbhtkG_>U*kOA{7+~8VJ9A)gu51+USf&4BKi{% z$h^QzBw~tucgcC>6+-|(@OR&F37}XKU*Az;axWedhNurEo1bl&^Mc)Q?P=|Lr>)HR zFr!C2w%ONzOxD_Gu3Nre_RN~HR@EQyM#B)k-MbumlcZKvFdrf}B(`qM9n(23@>nBD z09;4ghK$5iI>j(mFd~P#K6Em!gtt$1K>b52_FUi#W5kW)DcN)>9_c8x5WNx&L%p#> z{CIc*mEb+dKQeKrH;Jkc71=oz--{%G>QSyW*fsTtk&ONw!x~rJt~tJ!-|nZNN)jOT zlFIPLflTzRB>>6^OS)-5>GDpuZ4Pu?VE#BNz%aQE0l2Fv?`uSk2}FXpdAO4ZUTwjm z^8)}8@qWuI3n~OjzAdWNKmf8%wRQ1iFR_?T1u{9zV4uW}S?4IgkQv;D@`Hf@ZP?xo z(fIB~M}!XqPKP9s>w*3e9ovY_o)H_!%1>mw<(p~}03Ftv%~O_waa8>c7)U;#qoec1 zso_)^@o z?il>FISPZfsk&n^{t9BykVR!fd(9GIKL#v4lZ?aCSt+SsK!yE=Iz;edn_0WQw z5PVnkc-frNVNL+(Y&8ZtrJRa;4=$!?PPlE~#m4K3mSZ5_8J(i6g zaZEELE3Q9lY^+gE697cJdd`(ghgqzq4O7c)hywNS+|kX6_ccU)>fz70+~}hqgS*n!0VmDvv<1yKTrzLw~XqUAfP13)AR!YL$A&ICG6u?i%<3`LW>P`*b zg0jCvrSZJpafR`JP1(y|OWMoK@d55laxgE`+Fm=|t9kFV$8|Z2 zKrkqx@A2%cvjfwq2&ON^{sqwwu^kcr)tM>Ykq7 zg7Q&|KuEvbXR+uR4)=~&0VC=|S7Fx?&%Uf4Fql%<9`^nGqOcP5&S=Ikr18V^U)UM- zJ&q*NBQ>`RtQhd?b(^)W&+mi5dHO1b&x~q!XDe|OP^m6ilNcc%uC3Lf5KHn7DU`+`R7h014<4cCnpUs_R8Ieeu_0=IBfOsBmtE@fqvJ{z@!MFE~~y)_Oq+3BhN&bW;x{zOLjH1yO9_RVmFG6DhunF@{7W ztqbUF?nvIpA-<@{d>;F&d-NSk>r0J!-j%KZhzy{fC{?)5snUQ&%p?~6|DD6|KYwMW zzLmkij7=69AAcxV#MoGommH&#thTcpzjG(^N-% z7nz?dJ)-|OYIs#e?U6DGHOF%=#zo=2yjj53sw}eRrA7wIXvw9L!#h&}+auk0p&UIZ zXqS|dYA6|<2nh)}a^uB9YO5xWmBeJpR#dt)Lumm6abkEeNz0>PAUoViKQ!I;+0TeK zR#$!g{VqS%YZiK3KX3AMfZBt^Ntj0CF(OBV0H8x1J}4lHJNuIM7gwKPtg5$`#ign# zaHT{T0~m(phBn%Y0Z#}Yl!uBysVIsHmi_==-E_A`Fl(%SrdoKa*|X4TH^Dw94jAA) zDX~S*wHp^wrkm2mI0l!_IIBoAA2jSlHH-jH6LwR4Vg8;>I-RcO+*V}bF|@DDO7$@y z18`bF$Pf&Lu*32}kX>z3WIVN9&V1+gTd0 zzW+14L(3*A*97u+dubnrbZ-C0yP@oO%I`Dt^9{z28ANg^PrrZMc$nOHe1K=|tAmr% zvO-;4oP^4S`a9biHoRWZ1zbftvFKq#~>}t zuMq_l3%SBb#4u^!7D`#LF|QRqnQnR8*YZY=A2Ey2fwFjnX#UgF6SQ0uU5HmUts@}X zzqoM9|1DUD?SlZnDR{H-HC|9vkJ$hHA_D41TdkMZL~c9hF_P0@5m1F|+vBu0BP>c} zZu)$s6cdGOuJ*Xs>|NLY=-72i2}+3Wmcle!b7GcuU7SFP5N%=lqiF~H1>(E1p5{dQ zi!(oltJNQ#Dd=%C@@?O;`+lcK@b*&bw1v+enITIp9;`n@ir5!7zqZ!lFd3e5+Oq7t z!`Xt!dSNC7bAkXU`pRRd-znm8j9pMnW9PX9+a0JBl3;gN zUXfwgzGcV?v8nQt#fwwSY7aD@y76YqwIX~iH)G|&Lxgf35!s+NREl&>N+fF(+v&H_ zk*aj6*cGvZ0Ao8l1|wY+6#75ogRW#ZQ+yGUQdqusXNK&7y##w11P2{JfR5yslOznP4$TB+j#C${`{k%hd9Rrq*8o?!aAG13Q~UB~V@? zFH}#ZeQok9RJGls@oECosi3$s<$F6yuVws4?#jNM=Kv+AA+D|#P#|;qCN? z7z{coK9-}xE>+C(vpYW%AL?4*X}37_N7*1+T3d3wrR_(Kn7BTe#y9i7_#bZ9 z<>e#R_a+8oopv|tEf0_e);$+A)jM(X|B&5SU%j>%rZUP;*h+|O^mB1Nx-%cLOG=y# zm-hi?2-vm7# z{1wscH=Df=&YqjH+XD7Z(&bWdxeZzz8MPa3RzAK@;Qc%L$L;j@*JNfDAUraQ*PWLr z?qs$5bt~eIiuI`X?JV&ja{S`%S&G&4Y@>^d+R;b`;1I7T6{>a!#4M8siR?+*V!B{mBa99^C?x8y)_Wk zI#I1uQHmnbV68>!qYpT}8R5AjVyE!SFTV3YtLV0`_cR zn4uKYM*TrMk0$qfqhqvXP2~uj13Z_5o7;LZ|4tinv(H)@EC7*ZrwU-ePcJ zg6Y?V7mclY#V&d`rG2++5Vu2uf;HBGK)PT0FaPp zs#-Y`-b~!`qzB)Jl_5I~Igo(Fh^ju|1HQ4@E8^oQAI8MSy*zPEpRad)-3eFly%+e4 zNG%y&4K)~d9=I0s?42n~xSj3)$UaE+dX}VgEdQ^6hh{BYckG`(_A>~8LHSE@K!KTn zJR3_?uJquGA@d%N!lfx1Wv+lJxDDNw)^>ZGudhu=`q(Yc^1CQi;cXvD@^ zV_=CUXIMd5XTa^5zmV#G#LCvSleQTGvs2t;@icSkbj2w=&qyIJ15wzP)7d(UokiW< zoJr2mBQe{3I-tla;AWx$N`Ri`zn+L@>Q0O2%UKqMcVYR>!|k){cQ1CFDavXmj2wjK zth@P23J?c>Dg(`gDI*`pWu2dqxar#%=gFoTBB3_lqBWyYXJ+n5YA@ULtaANMLUjh* z3*68qdu{HL83xxvNm(f0<&x3Oa=Aa6Fn?;Tf$ED_NZa&d;rQ0e4ebC#~OhYe{S`W zxSqZfx;r*{ceR}+AYPn0UGq3y*8Ny@bbKoK0yW#JHu_qt53U!5o=Wk4r*K@brOcW{ zc5)neJjs>*$ftDXXk~|!*nPJ4vGZE;xT={wd=~ZWPrC`!w*U6_R*`za<#D?=1Ytgd z{U_lB(|@&S9{lQ-n{2fNOLcVG&@iERCrO%ej6^?Y&T}k0yuG(qgh}1$8(}|4hnN@J z>BosS)=%_`7#p7Tf)3JgjyBbo5^x^0QCn183`cDhNjU;gdi`r{u=To>-y0IQPiTBw1gq-u$=N>X;WgFZ zjxqbR&A)U@Xct)=uGx@9+2}~Ddb`4V*^gAcb3s9IKr!OS3#FO}*V)#cT#YH}KK9A> zc3w{yDaLt#ID?B}D^p0DRblj!gshu28QXPlv^y z6wmBsUrcPp@fq<`qDxw6f;g8s#EDHS(%l7FKe5#hj06Wd_xq{gYNsJI?Vu|Nppzjw7DgN9sj>D!K~v?2FhT$4St{-j~^xGl6;;MyIA zXgQ`QDsSmiB#uByRsGN;;l}$K|Nao4E?;u7cG8D*)A#N!jOD)k98KeCm*E&uyon*j z{JhIG05nFkkew9t=evM%=gEGQpi2Jach&DJ64^r2v+zG(2@*CKuIg2nGV-Uxc5cgX zLN!d_hCu|nr1Y@AiKRRufusItYv zYKnmUdeocQGTQeNYouXs{w=TX_nRj<%97mOG_6*vbT79N_f^^WkNrUh^SV@X8E`0+ zMIE{OD41|ggWoiMn8yxAFpjT62R4=Ms|SI%N^uFqKYe}Pu?3)~@?7=$TBEjEkz?fT zhJCGSO2|mJVe5y7Wz?%<#Xy8^L)t*s<0;aHe+AR;*Fw}-Ncgb4xkz*!*EbkY;SJI- z9P#)jOmO9>IR=$13orq{6B#g%8%L;<0N2Y&Ql{prH4)3c2sX8$U2!cOKk`H!W){v@ z=7h`AYLBtiND()~a@^~P9NC7umN2t&qfC^R7tv*;VzNV$Wlhk%e+X}+>|pYYZ;lJi zf^6xk)$f;WK<5#R5AwmyTa;D5<7!UjMSq@3+HNme&7jc@GhC>q6*%#WS#Ys`Kt)x@ ziR*DmCOpVU8#Ol*BQC=w-xw_$5z@nvX23&%4+)o{-EDQ?%_buyy)})syhEh;Y#_&y zl5pL}l0ZMq*ZlHEg`NOyYgy)a?;pg?_Y*Q1^y$FY@G;Lm+0q<5f&&>dAP0Og31nXFe2;)7Xd3=SNFc4>1HR0!1WH^c zJ9{{M!({diN8QNM4(my4+7O;7?vcQN;p!mlMA93R8UNyCp=2fOKtdjgwZ!(>k`ll3 z#hnNMLLDh?n)LU;OdK=@NvZ<+K2WfF=|O%$l}=xNZcUc5U_oe|34tO6Q6O+;jwS=A{&M?Nk_1Nfo+Nb{}I-94xVZE6e4f2ifOMVC&uo0M*1pqmNR6_Gr zgW6J0D+jl^b*%fx|ES--;)89gd3UHcF=;CYBp`5;6>syKFR6=}j6IJn2lfhZTM3?O ztWy+`MH%}{tDW9o?Uzlt{5x(J>6yAPTejMI9#;CouA#6}D^ev01IWw@Qvk3)vlO8# zeRyWqMUkD%?s2-_pR9`5it^%X*|r?NU=g^g4lzC!LIXh zv%V}|&uw`axUYpz(zHC6;uMz^Mz25pLZY>>5D7TL!;6nLH;(TQH8%)T3C;^Ng=@yFx;k#F9mHbTNMz<8sqWFx?m1te^wbN zU7Ph*qr1~&!@VC#j?OT{tCp>xQ+)D9&|ELYN*TWi`!D9l$J}z&2M3O-$elh|Im4L} ztj(oAonKFhBW1kxfg%{LZG<|i3J)id@c66wxfIaYlp4R&?uTetB2>}hXdbq6qBW9{ zCV^9Cci}FWvB0L)mGq6isq}9QwPa>MO4~5YQf|aSb0j;*R7NjB+10#eeLPJo{;ro9 z3@0Uaf(bTO2{#mzva%E&f6|**rgmsdni4bOcbd2BZQJ` znnc2{SpFK=?Y`2CW5+Z>%HuqJ8LnVw=JC8MCqbu3`SiUm{(oXRHCburw!`l2>1a$Q zEor#3A9+)iKFS;_d4sZ_kB-k|O6NDCJso)7Te0CjMdEZlQo!OdTr=r}=sP^VmtLhO zCUf^u{DRpnK>7<;5XVL4qT=FBzty2xgZRD0je!4N1CFTkN5WQ9C<{FyN`53YP~3Qy zq5vRt`gUnd32QJdrJc}-0rrB=Y$LTW3;>f@+t?5h6NeDOjYmfYL4~KtzVbNE^$3c( zE^ko&XWXl-#@-D^x)*fu>|K)v0|ttI;Ur=a>yM`JsWHy#VH#>%TqegSm?{InAPVVZ zBh?QSZY>|pA0tOQwD(h^J`;&;Rz<6-PFb?9agj0bz2-L5=jL0b)mhF#5Yt@ghV8IA z0b*kCriX2G}X=;tLtP(^yN<|Nh4HJKP(e3-LRuAdxr zGS6E-iy)@P>bR0aM!5F9Pltw5OQRpG1w#T*c+tlcxj9Qs zi{_hbQdS>~XL6qd1r#pwUkQkbf9fk`Y{FM%0TKcHEY1#d2~(zsQ~o>do< zxaGKj(ET)=#?%wN)ul=~7v1aFZj07av#A6XX{Ilcdi)t<>G=UgTo)VzmDeNMxSCn3 zVUZfFFsG8f#y7keuVM{e8)Ts%iM?bJ?(SLF(}b%I1x?Ze)5?Y48M`|>u1k;dF{hS8 z0+B`kU+bu?RQ|Mr;2lUGmiUunT6+` zi#0y`6#Pdn4GqtdJ+Uu|^S|jb+#aHY-BXU#;D;MHp8OmzfHRM4y*ppT0pg~!|9JNPLt!o%$ESDT%H?5W;mnYy7CPSlU&t{Lqr_Ko zYk09IC*6Y#e>27I} zZplTuk?w8*mj;m%1QkU(rD2zDSh_o)| z9dm3@T|0;2#3%PbNOOl*?|8=AUTEVDf9d<1Qnpai&#c|Vx*IW%@HAg;vF*egOB7Tf z2n%}3%0#QEsyY>T-#>Yp7x{nRl!yDbeBZ{G#Ool=7x-}F+s*6VJj{tbz4lYIW{8_j z#zvMLl>ruCIo^&D+dH@Uw~;zYjJeyqIi(m+h-kEVaDpDVcf;w=8-3#cJ!hWqGRNv9 z(}=@c*9o?rbD3N8mFBG*FcqIz+VimX=?;9Y1aLn%7fmY>sO1138cXfhC+$`B2vCy! zx+lHXexC(B-rY$WznSKlKfeMF$LAJJGYdbplMiYBeX5`APi}j4ll$7-#Pk9mzU)n(c6RURCi4u|Uby5e8jg>nmQ9ktNvUDsS$B}hL9jpf$o@vzGiX59PO;t^r? zzSL%B)Ee<&g4JrVp7Os|e;a`x`M44)CPK%WQC4wLu~Yjz5T0>HjDh!oU=)=X1BAzz z%u9#G$gmytvzm*X2d!NYNjhlzz4GPbQ(A_=5J>mO8PS!@lMJdZ1gLq+c8wgF;+N_g zc&iXQmZE<* zB`FEdag(5;4PJ{P1sbQS8k-s}ctl{R*IXyAG$#Wn$m7I1qgL(*csPHvpk~-Grb47%M#MJulj#$kJR*|}B2$>`A(q47fTRG!(N}2(0gf42i?!l= zaZzi1>#n$a)0RGI=*@cIsFs!dTcZDcEl-tV^a>SMwehyw3Wpio>8#Fy+O%~|;F=L^ z%q|dwTd37WDEG4gwo)br>ZWQx_FydoEa2W5=J7Z}H;XTxrlF~p6ZACt{Wj~BCMSrP zxvQ4wt=ZVAr`SWhovo>%;irSWcal$1#Ee=JvE6vLF?12%s&*5Tg8V!joLpQ+j22pt z45!7Yr8Am!-g2XcM##&y1AXHh9k8jXZ7;yI!CyV-_~g7`@;_VkpU?fI)>ie0w9owp z2u=^LPVSa1LZ1L$Akh8*)V*3=MjA9Z(k*VWYx*16Qs zAvQYg*XdNW=BgDm2Lz_4r&orj4_12%Jiap{)%w4?LbR2EPy!467Q&JGUF$~2H`F1- zX>xEYE8foD{=5JldUURc3zI3|2qbxJ8IKYDZd&>yNbYLHq&X{IwTE$#F)qQ~V)iDK z0**(B4?$;1gvo+lgDk5}9OjyReM{IR&PE?Ung!Sd*!m1!9f}-E^9o6&tCR**Mq%H& zR)2`;VpN4Fbw$Wz(xEc&a+d;rTN-&%2tMTP)r$xFCg}}woR%2)Dfk48czEHQgT|i2 zM^gY}>w@fcJR}^Cj&eFePe^}R3)07Yadjn@C;+SM<3t9eZ$Jbw$O6A7cGKNN$LGJO z{B{t8Z)*xo)0EP;Ps|y&16P=yk z_uJO&FM0v)q|N(;zAj5mm)qH{dq+oCB0gST2TRS@)q~v6X+|sA9GiBwwj=wt#ycsR zHLpFfPRl6l|I~@jtpBw?Zx|K_k+PM{lYDsyxSoT%_nPa=mEt>N`Lsu zb^8MB{t{-ItBOTlKWRJ`RDa}=#=sEiDtU1jQDtJ~uUpf=Zp2X6Abj5H8~np7nN38a zBS&>uBjHzzmW7yyrNder?Dn?A=lGSlK=5#Q_O)rCiDqCnR}^Vp;9V6c`C=oj=4H>QG7KpJtKnz}OrMJGpiY%DLbG*quxC^T zatd}S4Dmo8d>Jz{O;}cxweC9re943;)g}C2WyDWyD-@7zz+i5;p+xqN&X>FEzO2Pd_ za5}T?Ib#BQo>_)H|02x|T(6o~bF=n$mtVyn15oyP_qPJ+Zc=G@_=1K%_ghS@45q-1 zeh_qqlys6_cjFnMUz3pjer3jJND$YI_mb>)Jd7xVs;1MJndxaCz_<6s#no-7%L?>y zBJb$}ShsUZeK1|Ao{tKXem5vdz{P`ZPKNJs$*e#@*Uc-zhqb97pQA@|Bmtj9%5k?sDarjk(TK6`So>Z@Syn+>eIal6vHAJ!JWD4r1trq zXQanKV(@Arc4o*IH1!H+kf(_>V~lZ-&u1FbwzSg9>GSl;`LkmcQl^BVHho_?*MMRV zV|H1XGa}$;D6;Gq>^mvp>o7#TAWKiJnJX3iZWcX)D zPRykx1Tbyw{basE$QEg?x<7Sa9{Q9ODiQRTJflO^&TgeJyuemh!0 z_;LC8(KP%W*=Mjxz{40V^zQCGpKv&i1W2|=o|R%Bq98j#$f7Jkzu{)?6|~(JSpMD- zhv%niH>Ui8RaqHVJ?wZI7kVwT{0D^o1u!77{F7f(TYI@xr9(p6ha3f}GKaL88(Jd5$B3ZGoB=|~g?6)}y z-#9HO5x`0g9CQfEZ}S)!Y_DmbPZjf7l6Keztm!nSEgUGBbbg~%eU@ifVQ5XsEvHt` zlmwKUeQ6r^%bPg|5$VdW`xujk;P8sToF6%8xjV03@Njar!Mf2wlzgCq0^1-X7gyH` z8^cSHRk7i^+FE@DJy50*I0$Gp6ZSKW2%QH%Oh4afKK*GoAijQ*-weKYmF^2iLIY%X z5CAOe@w~r0Te0Pixfu^~c679B({6!XUT__>`HA`%SM|Ny(DUt-^sDQvDVNfJ?^njV z<`Vq6E=CJ+g2!>%hmx-uuDB2+8S?}Cd)LRS=gH+UTMUD3?VOBt9S_iy6{9jlTF z`6x-2P#+gt7Z>g{6@R+ut(4QkNoGC_bQgfFXPcvLRsO(PdBNsi?C< z@As3)8}z=q;^L~{1UPo@jE&#DQ(>Mcl4Hmoe&ISdVvSTO$LX#l(JRloh)VXxGR85+ zh=->F4rGr&FX#m5p|?8^GLHGR3d~6or}yMy{BvWr?$EKKb(-uA z2!9D!;I5bc!<1OMt`e78uXl`Zyqb4(zJD zmki%p{EeIEw{pLHzbvR3pEuX?SnXIbdsRdGOqXhhK?Xv`i_7kBxmZ`NUuxpnIq6UR zjj=d^FX?eGWi56)9>){BnofBhKk*w5w(c5ENYI3HGL_HP0%`)l;I)YHy}Q^AQqEoA z-R_E+F)~_^+e{dYS5jmxNO8Eg_r(Z|Y~Cd@&c6yHYXjW25P4FR zKIeU#w6FCxdJrEGO}rkJmN*N?4Fa_hYD&Bf5c*#Ji5WM!xEiJm^7_+#)Vf0#_8(|} zu9}JNv#MyK@88k$df0RD#KZ(3pWAL;Sy6!&28xokaCi6j^Sc@SzC+zxBCYQk`|l%p z@BJRkv*vb!Me+D<$PN~;{m1&HlhO6^?buZJ+|-m_<2R8%%I0rmb36MM*7bSV(6YbM ziKp+R*Md(o_K=L+uqBe5a1q$v*UB-Ohh<_%()GwlxLa&C?wENQLoa(AU0Q8IC7=&C zR!KCHXP-XMO?(-3oDIC%%?oV2!I?1&T!KEXZ`m;`m`Of0xnshm&ILr8tDk#C)Hn}gEsBs8w(k`hKDOipg@*ixH@Nu}2PzkLj6sd*1Sx8wI|m*c^lKF5-1R#jwS zfyb@a`zwBK-rkp=kf~rd>v5@8b|j>v3-j|*7vFIK%{Y`cU`pgCNqtc(0Cdtw(lfZ9 zIR7uw1m3`yMwd&Ke_XXXzHIgQ8Sz6ETVfTR_lc3w}B+?j} z!0=c4WA*5X4g%-hOfKPu+*{1xv98KWppU)F>?=u&kPGyV7Yj{?%9~wVUXUlFBgrW z(w=v&iuAXNwQ`=2v}@92>iCeS^J{P4KUC5In=!p1o1(d@S7K-NJn6(QYZ9zQT&-#4 z{TcCAuDL-$q@huh3v}wg9u84AH!fQ`FK=9Noz&3Fo&6boFZ^F6%6CPegzY{AxQ`WX zfd8AIfA%HVITR&{K--TsX@1^-J&G8Z3jby5Ifm|-8!W7v_=aBb!0w1y@;q^53N|;4 zcn*El3yW)6F+uI~226AeEnSX(|6cfy7MoIKcA(JvhMgIQJ$reb1Zli6P(f6=2dUU1C_~+v??j7nkze=C=zZvXhe6bwxS1vSp_bxzn zkj9^^`7(T<6zYKgTP zv5+kH+%UD|ZVIMPId`x?vRh2vR6+bSAer^E!?X2N@5bT!qBVWkxUkzQNW@Vd&}%OJ z^itENb2**etYYMjB6f#3LTdtZW}xtWoDM!&CS6VCQM!hMm>a!@66uXZk)I3a6h%bxm-F`DEsLfE_g@H+H(exI{&wTLXB(U zJtYKc5WAT8F0(9G@3M54_xuQJ4swKHG|Z_jDzm^IwxokjPcU%nlJP-Zb-A46o^7;`js*Zuze zKdiuZ?h^BXQ$oP0-$y^hTVe9ES`RHt8erpkztm4RmGWT99_HtimLg@v?6o(qIInl& z{c%klFI}*cNMM?z%BSCzP~Jf~e3wQEQj3qH&P(uMEAJMMxqF&+{jOT6Z}g}aB5-u% z8>zMkvaGb`&+@fC{pb51&Z-jKoD7~`TO-Uic{61VGqJn&9D zGb4kvW{xw4OLy|;k1>0b8klhhAGb;cCl-{H&sf)#@o4yS5}ck&fEcE?cJpb%ZM~I` z@njmkR|*>R;6s=qFSmN@Pb3?!EgXeM_?9`1`BS|pjq!h145F08sZs=tJ7` z$-^3>*`wz3p=0Rfwb8v%8Ri5-ySGO_?B@LWtUlBm*4rZlEg@20E($%DtlWpK!p3Hv zc{aO9ISDb+K56sFTi~QWm1I0{bM1)c8-KZp@W|fz+ab<@ys^GPy_e)XIo~}LWOm21 zvTHL^UyD`T9xA36pm2Gml}rx6;+!0BmTtPpK@>~*|9!v=G=nIoN)~Kt=6FI*6{5)pQ|ct`f`&S3hr4hFq(f-90f%1FrSlWeAu0#Y`bfJ6z1Sj?fh2zN(Dk z=mAS^A7_#Cs|#P(LW3u5KVrtwAPKl@fso@4)l5lwF4p4CdQ#b9jb+bW%NF9^tH#Xx zAKHxE&oZqzx>h+GrV}=&|I9l7dUGV!q8=v@p8h%;uJBtK37W;~%MIGwskQIzHyx8Y zNg+G4B|vx+^l(y=Up_fts>hQilG4LYyf*;vCV4nihdL|Ne)>}||G711t6;!Lu8Z{B zo*rwc9To$FT0xx|W$||spo#tEP?t~T@@Zm#YlnhR2ZoEUT>GikMREE(a+<2ag)ii( z7N4y*t5kJKf{$4C`}21qsEGk_6(pIAp|}FJn=CS`?|#tqsod;S_%{tHCI(v zuWe&4w~cJFDIeJWnYZ$9x#{&opJlr95YY+woZ!f=jhE@LhZ^Y@$>Jj7;u4e33$RB` z{q2KK#xmyI6>)|~{t3})yO452r;eWXW_RWL`kY!~WK zOUEO(7=(Pbo{lXiLqkm?U3$%V zjZi)^hzT*G>f=Y$zTu&fksquNfgmF~SM^lxo@B~5os1ubBwk5!$fm{Zy7zz3lw%#~ zylnP8VofS$%|!KthAW!>v+l|JXOF5m3J{^!`}(xpomnt)Bgl%L&MS)(#YewWYV!&a zp0%CVI`rX_jkbtX6CDp2_P$^C?B*uL(^`TFWztm!6=`=NnMZQDiH$ZOD0-r|7C~ZFpAEU2r@v=Wjo^cKK+_KP*JOO#7x>e9T3(P4j?LRL9rVKY| zOj^;f(W)@ik&QcrT66xO&Tus|H660%@9xaz5dcdyXyglmB7DB(a{-3{#n}EYakoy9nvOHW^|Obz25ougOT}rnSsgWLz~xsiOU|ygY1u zLR^H{rxM$q4Z_wR`FL|s%%+eYpe@ad{}S~SNXR*ik1Ib z7X?H>!g8JWJdyW!`}}k-9kl*i;QaLHXk=*Mf;Q;(THv-t(PxUlI$<%QekCN3lO*t@ z9c^Ow4-c+9Lkgg*4M-xmhTbm-VKsa&6s91${QD!ht6Sv2G1jW2VKw_WQkX`Kl>mC5 z>gz&%@ak@SEl)$@J$@bb1)!lf&WGJ*b31Ykr9J(Gj7!AX+eLtZL067IPJr%TX$W{G zNYJ4(q8b-rV6`(cf@16j|pi5NagY+!8B@^CY2 z-+tP`^*Z|@IiVK+1d9Y!jSWJIEluz`_LTYSp>uR4fhKQ!8*e!KraOVKoGz8x8*7Lp zyMb)9e0=3=Hm683Sp{r5Q#PE2^>>*b2Ei3BQ$F-~VwPhbl4?)mzH}C$GGyt=(ySf4 za_eXo0_5lzX+g~lbX8hj2kA9hkRYr26>nyK1-kXD@q~^2?GTGC(UgH=6erQP(>DiVWEr#Q!amg8%ljIrsP>a(?v z(!`AyZvcmH{BWBZ5NG-`uVT>YJR?ImLLV<7208vIE{Dda+7dc#sMxXb;Xj!t(Uys_ zvS_+cUOohnChZ?q&uKdLG;45x69e!uEG(>EQ!#B;B8gd9^*87PMMX)ECu5pRo+qpA zA8@5^%$ZY9^{m2ZgD=nu-PnZBKe}8@dDGLskrP?+8}8G(?p$dIJ-8@TtONC$N*>X# zMvKJ_P$(UoRD^B|+(r>51wY^WgDs<68(fZ`D6hTi>VFaR_d=iZ#{_=24Ott{a&fJh zJx_kbZb5_Iwg$CdHupGY^*N?A%bt{udW$UjonIPwcfTVY8Iic0)X8VjB<@Og>vauz z60y~!y4y|FwEbDzTC)*))A3c>?_!Mcw{68@>-AOkHIA2DP1Ytn_vsd}qdf@6gDkpr z;)s~GHVaHwH}IYV;vJf5YWLMXK(uGW?OgNOuKJ+7JlC3EuTNZ!o`1SvExa8XLWpjw z)9;4I2I@k@>Y<_9qdhoGh&)td_qPpV$mMjJtNv9l>2>i}H4%+@Vrik)X2E77pYZhp z1*LdRb;GK5@&;#XD1~_!r9%BKIJnu{H7ZdBu@8QcS?crLY$?zR`4vmSSVH7x> zPjZQRcT)wr8qh;B+SbArAwE9*q~Gym(N*ZdCfxzX)sTbm_VAQ&s9T>0X5&P@m1@l5=*fw>Z=fjA30b34 zfWz6)o&$kpj16uAHsCug7x9`gN0|lK2v7B*$=Dovp*q>|*q7HYaIZc_aA|z!4cPMv z62jo4HVK~by)qQGnJDUyiAnQQ(JCFF{bjt&SRIXuHg9BPDt_u90X=B`L`ss8g{_7c z@)sH1+UIzlD=Y1^Y7CLC2RJ7#?#0KBaM|x}+ZSL78lHiPh%lRD=$QwI#Z>$-!1VCI zu(>;SjPEJ)PvnOpU4VDx;O}n;?t5VxJiOesJ~=r_q$*1E!nAd#TEK2(h|4Y@@U9;B zx<1m`2QyWmX2SEl90dHge<_cSaZ%KnQvvX)Digngic4_Ix$efeCcQzB{=eyk$mpy) zHUwif54v+?CUKgd|IsYy_9RXX)iM(Cn~=91bs#Wh{DIXJmqx-5m_&e`XW)P6QwTQ< z9sFLu^Q7906FdpF_B-*FZ!xXo_l-Hq>d0cIn0oF3%GIVW2 zVO0=L$jc61#QSZe8QO-+^qtVV$;E>UVgu9LQ_aQ8C09u|ORtWXot2zGY$b!uV3EAA zymVed`*5-k^E*@QH3L%4ey`ka0;{E<^QYh$)9V@*nmC@3W7uhv=Cc2e+4EFrQm=st zrZ8DK`>)J|Oo~HMd!sRXYZuENr0}5vb))FJCffoQo!uZoiucreB=`{eHy=8Y9v>ek z2nlbUJpA0udlRVvOL~GYu}~+F6%T>%`S6Y|Q{-l5o?uZ@P*Ks)yvxZ3 zU?w<`Ms==p^8}c-T3+}1a_j;S`gGVhWjKaYN>*N8UUoLp>LwPQqrJVoo12@1lVFBA z?u)DAS@$^(MKk!;c~#wXL~6E#fP>QAqZHJO)4(l~Q=H5`E}Dgf*%+Y@`^A;(yr6`% zWSUwRQeD3_e=gYB`A~yigwF?)=uvut#65bEZ9^A71T_A7m_U*WOkVpFJjH5B4 zllQw;AMH!jx%8!(34Aj{u!+g3Yu=ACO5|!4yHh(+P6B*-{trmN%|T`Fr44i4tM7AK zfKJsNZuFYDn}l&bgRSDx$)!c#bit721O+8T1tlazgt9b#H#ar;9L)NkT^eKz8Tw>#j@n&Oe|34j3k3 zt}dss;p^Gd`5DgSFlPDG_3q5jGQ>Z9G=SK=ak(vn+r$x2L3P47@hI;#&? zWn7OJFE-a7sKO?qib}2z4(+j6);V6_*46oP{?3T668pgw7Z-Qyw0)5^cDNvH78pD; z-m(7h5X79y#~UIgFO^}a-etl^gHmR0ZT{`};9%%rGxH5eCth6gVaj{QV?m~VIVwv| zao)+5S`VK9507~ylf=LG4r2VUjoq-2w^-mIySBs_eg@e`Y>NO zSc2b@Gt6_&ngP1$X8tm_!DA*dd@aTA;b&XT>bB>tUr<&QiGFHxxu52<*7M$8>?}fD z{7V{;@Rinc+gxMOTt5bBdK>rr(b|?kQCb=??q~0orRMUB5`s5lV6^bm7*TVhaPv0) zEU!=WVR}2!)rWsJaslu$fBJiA)E@yka}lo2Ui@J&x0Ja4beyj zkVlEH{<8ib>zx+#l~_FV+k}KMu+A43b9PjI9g`7HZ+!WtZ6*$Vyjlo-*r3h8 zP8w$I!4dJIlsY*K?VG3ncyKnj8g6bhIXpkaz@S@&nU$&F*sq77DQlnP9nG+o731E? zT}t;M(uqOU+~=r?ANe{PSE91zkG*+g0okCqfWUbw?NeLj4o7-$HkxbTigUb-QJ|es z-2lq(6VGpC+{9oMC~v3Y{N>C$$C@d45uTScJ7bYqey;k_;isGFbM3=woCc8gh- zg_|`$w$ef$eO`~!bC~}WqBzh0FxUv!JH&ThZt-t`iDV28189J*jDY4qu)TwAF6YDq z^&peM8A^slR42Wanpg@GF9u&<5+%uOyNoE+H)o^hde2P&h5tZ8!869%s@bOLO?;HU#QcGX$Vv1 z#Onehl34-lN`jcr*}ma!tVEb#<}5HiKK({}Qm;wU9?J`P)lzhjI3D)Ly*o>^4=Tz0I;!vOm@2yvU!wIM_3*DVtTsvv3Jvli9P zNFE9AclY*yhL;NzG;1>xG{IM7YMCgH`2Cx6_M}5jHzV1NJ2@Q!@as80&1h1=fgId% z`j>W5zd-)h>|Z6pCjWDNNY#h$$u`>k4Q3w(!Roe(0UdSD5{Av!Gt4Z?!f6^^89brz z#I0Y6jki(O>^po>h~Z~yR%C79WOH70nJ-YN`LuuJw|`1uKU&;pq2s!0BHUZDWLoqY zw33onie|Tj1fh+5s5wsT-f$DN9a~Z=B<`}p?V7srceS9;{npU)ug9f2uM=ognn*KGy*t;?Jr-gn(T(Y&`MllwSvzZYmdY!`1f^r`(PjHC zjC?!kOY=0HqMq%t9QtSnJ869L(bVtY)NFQ$bu;f~cjLFh&ci0)AyPCY5!KkSVv^n$ zQ&=lR5LerJGd@^WR+cU7JP!?S*6E{?b{}9Qe1Wo%zji8Mhn%ZhI<61nqzBR{W|p3f zlK0ZhhH}Qer(N&B8ZON(6)%a7cH_6Si)5K`BGDDVBpK;&?(UEG_uLEP^-WD*G#iAh z9M!DYc**`)jrCpp{R897g}JTI=SH_Eysnp5{)`bateu`z(oM*X%>CuO8;w z9d3MQ*2o?CLH`dIzqr_LxRL}*9!Gs-LC z<6Qo2kSQKAMwb;&o*MHbL#B)m*gO)lN7H?i!va0-CN?Prr@nD1l4QT)9HEeUgJMt&{&W6m06A2 zt_1kZNMvUK{IBBawlFq&oN3Q%Eg<&nrFk2Z`I(Z8 zr3^M79kXr5fu491*&Zt`@;7r%0^t)(lvnN!5A&^os2^WwYO0g-sX4vIT)(@sEPHE^ ztX8&t9<|10q?sp$(I!Jzj#Axa_%2@h>;hLh5Z;~?yguR@vVrhhh6yNKvpVFfnKO`* zbTT*ePo?RaqAGF+UT(I$bm?m_Qn)kFPp`g>m^Ee2b~rvlLIGVxy~h_e|LE^z8DQyg zfK-35;Jf7gG1vc98n~ad1#I^`KJJdsRuo>>b~OX$XjU1dR{o_8gtSPEc^svsrIE4- z+&Avk?TJ_11V0H**TKY?i8}yxO!br)p9~$+j_B^un6vL8O4M&l8r6@BhDXL+^GOXy z+|<>~z#wp;WAUefy9EYfrX0KSA|)nDe1p))Xpj_qXzU6s0p&!-uL3dey10Hgao}&K z!z0B+(Hy04HcPY%9$sSKO+rK>L&Bw@g5%W6J=kIX3I@wbMB99%j|R1IZB~~Les^OM zXbw9gijcP+AgIABv?LQ3pQmf7ZW`c~`wZWa6n3r}+47r?kcsTSvFfmXXlMEX7F^%* zEZ@gWA&SyKf}q66Ge*gdc+a_e&b6?tW5i=9Wa2v?LwYwt=!9ZDytc|L3pZVk4j|+{ z8zsu&2$5MC1{1-ptz~6)sr^4;&9EC9J?y}(r40CviN$mL0sO=g5qZaBu4wm`!{?~w z;cQSdN0go>6-=URXlQqK&CHD0)lYgxQ{OF?q0bNEWUQ)7S7Y92q9Lr=M>S%w_C6Vqc$`B^{QL`_XK?DL!Br|b{`GW~T)*n!_}R0ir;iO}&R79jg1+a&9qZy3uJ z`{fpsfr4LDw+)G3E+-GSGxd?rd1JR<$D41zgr=oMlTA(BsQ{-E0Jul~b7F2_a4+

1Rf$YY?cMV>v2dRjcQjL9HmFAT%b{=y6O-)4N}Av*PW7gs%n~XZigGGw|F&{c!AQ;~z1yAf-{VPNyrLW;PJ#7`1}%D`l%PRGcpVBQC`i zpil(Apg@l&Tw3+D z&}jeMji;L5tVv9{GHo+7!Vq|sRj#&3e0?1?2Ky%keD%)99ic=adJr!d?PI>B5IZJhsv0e`m|fY~R|Z6~n`gOsDgr+ZwiFfxBXg5)LACDp z(xWhiwgaEntwesu^vMW-d>2pNn+wC0XB&>69OMI0%jl0;TwB9-?NAj1d*&ncd%a+A z`9G@>x|hpuhkI$Y7Lg5|Zxg+M&gu9(=Y%MVS>S3?<7q&*7Nq;U$tJ2B4Q|?9?+&Cv zKC!e30~GEu-Jhdh-cnE?@X@x z9#u5PqsxD2CR}W^n+$}x`&s|7kZodYW_uyQ?iv}n1c0>8<3v4A?SR9PkIMGA0>5>I znO#wNJbf3pJ(=jo$s2xwYrEe>Sltw>mBA!`+)z=#^F0+JC5p?HR>p;i{^c?f&5XrEwRI{amcE4 zLr^>AsZ)5vak;KpF&F=DP^mYsxc)8O%&$)-Z7uP|<)p*6cyH6Y=b%{>j*M$%5-tFG+Ja13!tiIq9eMj#BL`P_7XfP0DBw~E6 z-g0`)6*>gC_#N)gy8+45LxZ=2wOwos-Uhqye(W{3lO$BIr3-P_`TkG@b7y=w^YQ%5 zMPDGyt{L^(OEFB}O6Z;7<#~9_Dlgakn!)t^Wfl#wovm#HG&Hgc$)fN6I`Dmt$;lB4 z)V>WDb-Y!*T^Zo)rV!TZViY;^*tRf%HP|dibs_-lF*5 z6%?2gM*r2i(Tyo}^z_5;Mahr&L<9tmR|rbjv5MUI6-TR%cQ_<%-mfc;8ZZO;K3iUA z^J8&=Fd~!`Ces&Sv!y>#*i^Y>i$5)=>sO_CXPn&5Ue3OIUQJI5;1S}z#Uz};1~=NR z;tI1-UIZUp_;Dhjzeisv9Ok~Fg^M2{pS7R$EO0UVJpNwa-zSmJ!v4_aZIV)9sXV@e1Lj~NbET)p z9K#qUeJ?B}ERs*{8P~0db;%f#z9ceNszkf}wiiseOXr#lB6q?_et}R!Ytc7OsUmfE z-s{==IpT&lF_M?;>{aHp{7()G+)%InwqRHyExEw4Vlxdtd|S0&t7*u=`H@Tf4qVnTzk@+?$BQ$dfGj_>5$=#&IbaWx{6Py0Q$(pyiLj(u}$W!Ru)wvCrm(9i0f| z61CV7epuMBDK|*BwmTW!#dhiIro~;0b5?l?Y!Oc74c#$NRksv#%)7$*p*C0R*0yg& zDLpAg1&PU<+qhf*zOTOgu~-M+T9oDz$T1F#6Q#+SXiu zl1-FhMjc9RzQ?nep|%}0x7M`CPLFwesL;QO2OpvRnvAo6+;dVPr|`BOo^6$q&Fem* z^l;ipqFNo0x?m7R0()mU#ZI+J!ZWa(B*=0%@MeXJSX0CvmPT_yh-v<2{$sM=y9=k;>zSN?Rk%)2!$rRl=Z0PDBT#56uQHC5wqgrhO2m9(yXh&mM! zaW$McJwGMz^7^?zmm3j5i#q3Q4pD&yt4l?JKS{6{b=&vT0I4y{a!F!}T)WD!j;L+A z)BkS2UZQbl90#ryOl8hicwtf7#~0VhljYmIv$uX`!N>E!nhi8HCl?pWWMdNV=i%1x zUJjdmqB3xJBt*?m9iFaE^&QWxTU25w^>nW@Rsz%1|Dz5lN%-v>3JN^ID(y!{)z%|Mz@<@c!?R_GDudErx>bg zsQMlo{p+pWD;g9qCM@9L0xV`q9PQnz0V+*u+Tjge9rxhpgeF=J>*_IjexgK>{SQ4d zK@#3))uRVjMaFld6@K-P^GbXjsWkrvWuW=KO30kzTKG{$JbGyhs|UU-oN@f+s`F*n z4S-7SRsbzV*14pEgOUgmJ<#>a9aFxZfWy9`Os{AH1_pEIbCg(H;kZZ-`k~*3N&t@7H5w0Pj`NL#ANdBqk5F68$1+Yu;#YzdJzcuk| zlXc&{ynWDU{0Qqmi?Us*s;lbUBYBTNe(7m>Jto>(<^Q)xwE+XKM~qHCnIw5_M#E2mV_KA)ihVOdfjeMwxu zprHEceV%fXl#s2MiW*DOOc(9#JuUHBq+r;;1&EGtxFP5p9DHM`Gl!&B+?qJU@KSt{tO^#?#<#B>)goazh`^J)Q`uaiCa)0RBc`X#lKW!REnB5@|_rMA|T8{4n3k-QRn%!grN9fUa!z! zz{;~lr;0I#m{(D~+$rWbGgngoswNCc?I3E8dpuu{LM$j9UV<|DeNw2e&fr5yyp7h= z-H9~y7C^~yc77~v&ZcD$VsM7bHH0Y#vxXPofwD>O=nvn%35ksz2d9-jKD{?}mBADn zV`LmZ;+>lf>L~$&rZo_+)n_kHg#fhjal)Cyh&M|%Dkf%fY>Xx10_lI-3Dj zKXF+HH#av)$#>Mzvo^4ePCgVr@g2oabG0DQV=Vw4>DMo2j17UX;b0XM^beAfiDD&& zQ$(2h9BC#U&A$$ljiGzzV9%s`URC@kt z<+3Ws^5Js9JCCTJ8}z1h7>H`A6|e%p@xzmy_s7x-z99_5PqvuwVW^l4>yL#46&Zg9 zadtgnpm9t}0Qq(<&PS z`r3p^E(3(qkMmEf3)6WNCYF^-u?0*Z|M#Hn09uOEv12~$T~*!S3Pa@@RZRF;bRop| zC8)ZW5*%eewA_ri)2C~&H5Jj7F$>8sjB#Xh($DxGdgElyiSPyPzbvi3q;2p@C_9@)OsM^uK3~ORrMOUsO6QCX$gZq)M0B%j+eR zaG=i&Ybu97JkETj(wUo{LN|v~%<|m=8X^UQoDE1OC|f9C7PoB}EGj?7&E3cS=j(er z1J9`WFBW>hRL^SX^Fhq(LFe0+1tKJ}CA6QSBpfyyl=Je~C@}GG#00MeIniv4%0PAd zjL*Q0ZCFyIv(f$`v0k_doFF~qD(D}8cZ}{1jf21>%-|e^`-PVfU6W~HwUum~cHWsn zpF~qDEzp^bP+fr{P>(zX*RP_P2Lh=T+zftaTsDdtl2`qnEH-Gy2n)uD?3MwojeaYv zl+E#1Y@FkzYBj3O;?I)03dX18G5Hu|WaNNjL%xVoI|6bnsv1>nGxHTE-QlWFNe<+3 zu_-0yMn0Tb^55c7%QT07PGi={co5BKa%|=OFI>W5C{Yp~>}zsya$-k`w@c#PFA09g zuv)RyC!34dAEw)@YSU&fURxej~?YDn#t-zohS0T;G z$s==dEOe)0y@w(*6Y#lTkE4LmAt0cp5ueE9aR(q!L~FArK$vmyrK_@`5L(&Q8hn3! zFb#<7CG%vzZRerz-=Dysz-di(J`OCzRJ|VW3ChUbwm&|l=xMH>U-ZJX#t>RtTBqWZ z@Lv5P089nY=995J?&CsT1*^*nDp$!}zj$p&aVgwfBkS~e63^fK5=CjK=PWXo*^~e7 zan}P=?~Eh}_>^eko;&bJh+oXI?T0jsSf{Q2GbJ`rg4Doa4@NRNfb(ZigwhiSg31OYT|JsBM zkKOP-8CA2szQpA%`za+kxv+3A2p_A@o^jj_9|J1o2Pw8@G7mPJUs%d*@K8-2Fm*oP z`{df;xm&UNyP#4|FNU~X@E0Zm!(gfV{(PJr$(rNsn9fVy6;KbZ#rQn;?qxrm6j+%+ zEjH9!SsFb?&FPzd`02UkPx5m8|7iLOpeVb(?`2uKQ;=|hrMtV4?(Pn0q`M?VT0k0< z?v@UbE~QfhVd<8x@AA3dj~Rv;W|tk#xz735FYMT#?p_7>qiF6+ttN`!{O-S2EoBrU z_24Evzw5ojCz)7C1IPOKy%V+9=D>ky=EI z=ZiT*hQ9PTJnX;6zPaemQ0%@&K-J&cdiX6lvLvTCf8#!Dw(ul#5w&BA(c-^ktI`~R za6aFPKih$%EchveZ#Qe<>6ZA;#2n>Yy1%PHGBJ`eOzEgl8Pk*(GW+|->leL|*S`Se zgKB~1zc@IeZc|al-8?!va5v|*Zp3IYzvt@fsT{-4CBOO#y;#*K4#oVNcWBG>nY0>b z7HUk!VlHocZ`hwUrxIQX3d}BKhjHMV8RletQdZ~b^TMyuFk{sD*4%uLL;1AzsI@cI zNwE|4)K}P&PLJdLs%!clADTY;aH#C+T8o&culzJ&_H1dTXXjv_KnnO8ed(tG5s-ri zGKilyo*zS~v&?na!8Y4$?7EHXFdGiJFbe>@)eIomI+I#2ga3W-(Bglk@`f!z4sV+3=a}IJmIW^a zfdno4be*ac-85c~$d6|bgEZIIlA?VcO8aMQhRn^~fneTyua`hCs?WUa{q^hw>vuHE za$&C7=jE|l0jcD6ILdL)j#}{PJDx)fQEmfn*?J?%C5?*mLOybAEF7H56#j(Pt!;tm z_xPMp6&0+X%fsgDxUq)m*&6M9BHch;158cJJyT|tjA+b-{}i~I-+VWzPL^jtruv*y z8-b6~@pF1np8yL78#9$DRbcb)2W4Nc5-o+PkNvo1UARiZ2`eNUQ?;0;okD*fOP}wK z*~RaFKeW{sQXxQU!XYiV5xz^hNuN}f#&h7@EhcEt@&YY^i9O1v= zaQM6i$Kr7FJHuG=+h+&}cwoBb~6-fF!=6s<`?XOu>u z0)At2A?w8rIK5Jblbt&M_urbgN*Wp(LR`2q4-XG>a&iC=P?^(-ezFp3O{G-3;w%R1 zq>5(75<1FLrD2do$_u16V~>YIIm$yZl4%)IB;1~|kNeZ4z)foL8a2(Dj@dJ`j)fT? z-Ht2sL#xu6zLHVL*K}JXn{B4|$V|1phYy-unak~7%r;utsdnM0O+4Z~pOoqNotLVV z*_NoVk5(7kctq0`iz60kuOb9VEbCqN_kq+>sf4?xf(Ac!{$`^5Mi&v(3p7BuGFLnO z8o3yqRKfQZ$MO?q=rJ0lClp^j_xzoyB~&TUL7t4g#_`id}r%*b5aQ~ zuv)yrD?P7Jm#Ov{~?foSp>Xakf<>o-g21>M9>m~GwsWz zVMQWtmqfjoB!1ecNX+t&hv+pt`;n2-WPHx4driePh`jFCKsa%8R1Zg=lL5>ze<^K*$pfRonZj5yAu41*xlx0YMGkHJ|Fgci)Q z6GCpDB2UCTXYm4;xL;yNA2>Q1Tp*H_7yYSEWD7j~UQcso-QXIy0q|*88Y~_?rXD7B z5TYSX&i9k=P;e&4S)F@auUl9T%tDT_j(uv-`@jb1DOGi-@`SopO7}Gs*B_K8?OIzJ8@baI zglmk97XH2vzppe$&P+R2y~!XIhzrz`Rh?-PNZ}LswOz#*HiLG0q==5%+ST0L*5>E$ z-c(XD*oY-aU9Uf`)70F|%g;X*pEx@^8}X$XWV`^}gmf9L|HYE*c>Vwjwm&0EGRg9= zWQO4vjqj@%Usq)^em{`2l3oJ>@7*0)h%yWx!N&~Eg{q<9+o=AAxZh^1Re(>_h)PGy zz-CaFp7_J*x2+q(;ZaJvF@*h<-M5s`k$ zdTt`$6R5!kWurqbR_tnw4;mTn_M%_4%*$eXi@R5~@a29`QNd;#A7aso%BC_BC{n%a z@=usvi+FCSDj;}HI$UtU&d8~cZHlwb7kz49Xw?|ga&Cg>w zN~ajrghbKjhd2O~REP(HC_$nVC;<$u!KciJKVv-|zI$0pc&(Wpc7S~I-Bte67CUX4 zz~k|4cQRuSj`^5Mm(SG)y`9**ti6#~KwfTdy4ke%g4!Z9mW6>upvrBy<;#RzyINCI zt=3#@ePM=~pED;ScD<|TFgDNJ5wti8Kcq^Ve}%2ni8GQ6dRsp=BYx&(*BI{)ur83v zH9vxoWh#gk3tiD3fWnS! zMZ*M{Z88eomR%mk@}EAHussh|veQ;YVMekLfzXXmiM3~`^iBN`I;;`R1&UILs<_c< zqKKsaub0$C|L*ZIE)Js^>r#iwJgvC67{ID!&q}_t@bX<(1_8zHf&!$n2$JEXS^wKG z&zdo!)3e&T(skhUmlundal;HD0`5Enf%H8(?mQv%eVx))3RX;YupbCJ@?uS0(=y|e zwv;>`xu)(S8?}>x`2DS#oh`ko-!={lTP30dF0sDiAroFX(!xBfm+`%SP@Oh8dWW^2 z9$s0hNkbvTy(>(5BQ2~LTe|D*gyv8jDSiWYOel<+QTOcb@b!=}dnOyG?c_KhDhkDz z)(n&A^ifIyq#l-nO5@l~9}_M}_zyO+Qz{Egy(15uXkWkj`1RFq``Fg7&dXnmqivEu z;>^znjn8E@E&mU*0bEj0>bX!Hr_*YD#t(`mNvvf2CheYR=_WR24p@lRBuO${x8ojD zPfKcfPmTE^kK>I!V}kXBvYn{rOvS1(rr6?8j!bt)0ny6si1o^d^qGXvS5~AQvMao0txqrCz{_h)GxWSqw%nDQB5U$NS) z>#fIt&i1t@Y?2s{1VgLp?4Pj9nKX`Ht1A-I);xZ5zZ@PO#>u;Ty2=#>MZLM+cuW+3_&pGp{{-g;%nsxE0YFtODm24s3_E0JC6QmI zYQ5*L6w?vvQglZmg`oTHT4QO*Wu+MauY5c$KjGt_|JLx;c|G$3tH^VC{C{I z%VS5e{RIr~S^((==Ot_I%j+vxsy(gl)#K}AFHY7~OgxDg2KpYR;2X8u5~P2)ihntC zKWp(DLroX4M&pY7<|tFD+qEKJuCseRv)hCG=ea+LqUZN|PwL&C7T93TdT5+SV_d9^ zm61_VP4IemP=3hGuV_2fQUM_zk>$nd)7}lMj62h6tjL5{tvhiz>tX@$oKTWs^~L6w z#^H|9ulyeBJ35wwA6LxSSq8Mon;j0Wj>LVR4i8?4-!2yJW@FHwJZ-DBIIRhsZ#Pv; zs;T|GHc`=n9#GS-2%RbBgNAdJqLbz+1Wi=zz; zd92lN9s8$$EO3qS(IA@nmPW3tL#^>5qvwX2WL$Hfd*4+MvT4Bdyx8-&>_6lcm7|9L&4|Gt{(^=bqr7YtCE3|PPaH9kH* zl1i`V&6He$4zMt~TUu<3dh%xx1ZbAUMcVJfx^x+qSG5<^tr9W+g47q;extP1d7T7x zI0#;T>v?vtu$9uNEhs5Tf5UT^>neJ;Hl$WmhAqKL|M^m7lpCVIk1u0!;5=Gn1#GGr z&Zt8Wjd`a&+@MV4jbEmA)$ecJznePyY>FW&Bm=hwy3H8%%FhWpI~kH~=IM$v;RW?H zi}g(H2Qp&{*^Y06W|~dF%ahYRqfy)ogwx4uGM;$FMMe$KualV#>afiQ zr?Ztj2O8-y&V$jB?0mV%CF7*`HpP8m;=jxAHGW`}G`Z=vHaX;1j+Xcj=9-6uTP4-b zhxPh?q{xnu-$)#M3cG^q8U!lbKDMf^=a=Dm8FaVzs_$pg2-ng*th$&g%fk4nM-a&(Fu3#Cgo9($&|MoqhClUOMxeg7jIK zZDZs51pA*So^;;Yi9hc7swXhjJ_cj zjK#_t;Jx0{?XgBy+r`C6p=5CPQ!Za5xGI1%^Ce~Ove(JlO2_=aOb%8);0y1X@=u@$ z;yH?tyQ_UI#q-LiR)v;TXMSt?;lb5R@Np;PX$L2NH~3=9Na-vxQT+Ds=a*a$`fWR81`~hXj5juG)EIe{Fm`*%a=j|NpC|9_@ZHDu zS(Ir)M$e6EjK3-E&Q8Y$>&}~|s_rXuwZ}v^cV%6CjUazuZFs)dQiPTyX<^$y+-yAuK5|>vJGSq^{T7yY+5o7-d zth6|>Ugu$6U9+>X&GOCam51u2r>8rRy>!(?!HSyO2;xQnI2U^7jvQ4Kb3vk`Jw~4< z=%4R&Q%nqZ0$N){4f>{oCMNMeUaFPXoiZ=XJBUqe*X0BRNE_W%8m%(fG;&lgRrq@4 z|Bly~^K|EC=vv`-8pePZS16y=&Ivc<>^#H=ve`^_;aQDdPMX-u|B1gE0s!;%tE=OT zX_T^6>RvH1431Ii;)DZa!;SVYJq#HSZEdaub{A!XoiO?let$!j*wj32UO=!>tC-Vp z{Y=w#{@>k;`SsIulNf(YOjUQw92;0?-W;JVtGRMx@D#p+r1`_=yi^?)8Wcwp1UzmO zqbl`CTX}Lsl;x5>x~z&#`hT~W{MjEF;WpW$IpELs(;Tbz58qs%&L`Z}^W*uL)oDVFdJ zL78pw8dDm!u}O>G+;JiQjzO}kY9!;21~EohwYNZidFWGi*KT)AK$PkplC!W+6NOj-F(PnUW42FgJ?IJ{LAi(8z2HoOvyM?p{%0 zTA~>_`0&3rVE_Aqlowy4YL}8IR@n>_(oZL7e$=oiHPK4go9?Ia1?}8H1D#LF8Qf5w zzAKhq+vu22_y9<2?Twb#F*jGs-)Ly3s?P1bhNnUUC@`9RTz;5MdLWHIk{&AHir>9l zP&0b-zr<;|!Ezv;NTw}-vs5rT;h+gWFheH-MA*u?I~u6T^r3Vt=7C+&)wm^^NDU>a zYirxnJ1q-!e~CA}IS*5|K^rZV(Z?&tajeIR)~9whE?Uuar`xI`>*(TKlm;9C51Y>{ zWa{&HCkAfAh7)ijFmQYa{Z2tTa=+RcyB_*7UApJff8p3>QNFgno@o=5@e%ZfwkJ_1 z5~?$eH4=0uQv|+Yv@QODQvEHLy|w?A%dvn-=jUndCjhxTHQ-x zEMV@KnSJRvX5{J{>1at8sPnkG>1id>$3*>olr$By3=xjFpjcGAM!_H~1%sXT~t# zqb*^w&zz2`_DISv$S^jJnwb5pwN100!SNI4oOduQG!N;_=M6J5%#yOG)O1;{<=J0J zdI0o?wV9zd4lY$+kDMAJ)5Jr7W3rH!eObgjjZP)c1_&mg#1HAjzx=$9?l%}Pd8yl& zZJ0KxJ-2F2&0l4Ez!+@uUYu2(kB#czX>3LERZ>A=%+mVYxXuLw78$Tkx0zTBlz6TN-xN4YXmkL%kS`%GXT zy!IMhQ&*qCZB{Rd;p3-~z5QthMct&LUb8F5|I2x-etPXPNhL zvy~!i4);sWK0Vxa@Ua}y#Hk0lsT0xrT!VzUmJZhDS%+rgV;E}aVhjaD;*eps+uHVf zE>5ye+9E^6n#wXGxdtfX<%zL)?){3ASeP8-2X+z3`_^96X|1g} zCNKTe>V*j)NaI1IrL<85#!q~TCL}%+9tV5+uOpfDhHNkM)-H@CTP$^jYOAHG1@0E; zR_do>qLIi$%7tn^4b*y;%7lj7u(7e(Hu^y=B~bBW7^UQH`M}iz!K)F*;Gvz@u}FB2 z!OunuH340w(E_lwmyIl>omXjjs~Li?UC^f!NHyBsB~D%>PDm_4x4Pj`o2!gwjPcW= z*Fu-q$PcbvUK1x5g$?S2C#$6|yiM6DXz|f}$Vjb@tp+}43xfLxqwXACyKlQrf=-0j zZ&l6A@-2LgR)pD1+Kb*wm3onsmp@&7?J*%jEBv#E{K5L?$KE77YK1iG7cq$4=8K@v zBFzuKHXGD=Vm}jZvn{T>GtaHQozWu9M8s3-NzXFKQYpUo(T@F*Nxr|BhN`Xz1=Ubr zQYYD0{LJ>!dzZ}p+h-OFmQ*G~QeF@)PWg~$8TlijY!8KE`0y3!_uSjS9pN6z)%Ch{ z6tntpbf&U`aP)3;S~@TSd=e_KqAVW* z&8a*@Ra?l^76NXAT=MM*(L=GiOi#x59SYT<)IIAsF`@WcDfIkkSCM6|T;K)P!k_b2yLVHOTQSn3Z z+(5r!A2sA-BV!7bjcIF~(1(j4Vai|xJo<7>NQ4j`ND&!dAvT;!9-56H&jBJ$7J|1R zgeKD^Dd@1_Blbz)qowspF9pFuX`E+eGV*)F!C@0NZ!pHv#6hm8V!^K z!gY?egP?xzMFv492F7hb62ppCh%uEtq@n7U5Hu(&hOM2QV11}uDo8?^n#NocT}Gmg zL{Y*3{_^7ofngX2a*D#ro+uiyL~e zHNzMxRqAgNPsIfjt5_w_-s0&;SmtOTq|C{U91p<>gw|V}sv(EMhB{-NkOBa*VSz-UtgliHya}DTExhQg5zma;)8w? zOxc#8=<>JP%24BxwH{+=F90B6nfZkUq?=I3AvMYc&(l2?EWS`#5Go%%orIiCJ>b3! ziYb89;1w9YQjCj&S@xnzf)P_XJ5f3M1Q6UBaM40Z!X!+<9JsqFFvJnU3$R!0a16-C zUY8sE{hWzDSpZiN&B05`Tqba5^a~-W%n0by=lQk;9T^a#bAYYwAQaSmj3B^H^}AO_ zLs}B5$_J81b4b?#uDEqyAd6I4!y$Un{of<}L_&o|(yy1mGte@h*_4$)z7`XYj=y^9 z>E*?{Fz1jnI5cEuyCdBv5zzFdrX)SNi^AyWl`m%dGa%rOam6PB zcW3HSS5J;dMag9d&_2kVXsngA3uP-NU|+e?Yp(tgUH7Od zWU6vZQl*WOmE~`7|IquR>FauTEQIEZS}C#KiqW!JP%&Pls37YK(vftsb z1uVeNKG^fH95l`2y&cb=Z0-(UI(+DaccK8yUb-D(nhp7;u3|d;Nn8AP@6_-|a!N=| z)_z7sy?nQEb8|C(2nKN^t*jX32i}o^7=MCZ&qxryCI*2-%zRrqTcM!1vsrt)AX~Yi zJn)QAyMMy0o+J4Xunu;1G#lzh8sL8E$1|DmWV$c{4wD^G(Y0n?ovf1L&Dhc}G)60L z%@^^G1eXIpDa;LWnj1PcJ@?N)A|{zV@LMNmv+L;TWJ6R52BHx zKho0D_14%R$BJ(h6cowoI&7&tm>OasbP!NtO-C^ycBJsvaoAUJWNiuG$Z{5snrgf@n6zzPcB|AktU)z#0GWP^wPYc5yfQ5A3$^R%vv{c zOy{MTRaHlaF#f6Sd0Tz#NZPha{#3y2G#Rjk3fq9_irKDnhsQOsG>|*gVv7Vi?QsA8 zp#UX}{p;Bh-QVX)S86{QtT`a+_3||;YO*@?rE_g<&8V%W)tCMQBTmbdgXI<1o(Tmp z(p*zRQ<+=-9g^YP4*Vh#=c+thbq>T*rZ-6dJQ?*}#$@1wnHO#Hj9 zuhxV|$RngT;)0w3gF(im;O-q`6R*YJfMwp0W&I#cbXR+Qdnbnomn{Zr1hq+p8h3qR z--bT5>uz3Ncg=oQ%w)4N7q5^=T^og7*Po`hPJvHLjV&!LfClmxLIMH;ZjU%c;}7*M z3A^=hJ#~GhsTYHCbYT{9K6=v9xzYWLi!_88@KErj?jJ^bJIZ^s zwMTF*#pVrOoqhVG0X>k;_VSuY2+0 z#gMEmi2_tMG-~qxZj%116#%8)kS_y{PP>3GeryC}g3|yT@$TtOQ&%CM=J56_QGar^ zqHntCG7Ie9_rA44G7f`yujV7q&URF`@ z(*J>jG;t@EhX1bpbGi2U3_T4({&k=l#?H7DW2Q`sd z-OcV3MLAM?z|_Kmt5Z#F?tw^0ck`{U+s)sRRA*<_Ysz0JQDJo6YHOGBZY{M@{^f9+wvg%?&@TDrT_ zN0(Y2x7Rw#WxmPeFPz}D-0szeq>ku=v3`6=I8NBQTORIx_or0#tBf$^8rcuhdD#aw zWXi#>src$3YV~BlDWo^xK=& zx4F-?_gY7Qa9TaQbj*2mv8?%rk&({htzAAE6~w>t0B3Fv;u>R9 zCB0g1thZRJcrV_akP)BNLE+C6rLqV(nwr0TV+0 z568#HTkv!BXHhe+AiIE%ghb(}$(&#``2zcw2S*M{Fw1;wEPlROA1|-fHXnNS;<^bn zK#ekcLh1AkB^DaM>e2mrUaFcC)6%lq7Lt(!yl#MBcNSG}<=5@5Y0qFi+hz!tn|OCF zVG3Hbhca~Tl;N`JMD*B3HS!kzG@J%~jt!IiY@!AQLaF=CSONW6OMEN8SR?GHicj6w1NW ztC2rLahESzIK&^f#+>+E!+$Q$YU>d^oX;dW$9@+`wzQ-|6XX}=7JQn0QL%rp=c2!& z>f}1B>z9wYn&?`pGZE+R>$&+iv+?NX(>+cE?t20}RR2IzD<@})7(_1U#{&Bge!R{W z!B^vFUx4IPdD=@a z=HA{P=dwsb3CZ_Ek`(AyzT^UsTpY^39+Q)oEpxoHwa#|oeZN!X5*zlcR73zx%A3g`oN0wS3E&RYV9Y% zeRU&d3?7H9va~MI-=lrT5nnq!p62>v!?@#0LCyLopnl`043V#wQcsU@2CBJ#CEckU z7)=>eSzbQF+d{Fjx~#ekiuY=kC#J@akz&i3+QBHnlyT^!6yxShpV(@=4R>BqAHT2p zhNkEVW``mqId6OH<+QanD1Mu?NvXFMN|#3o-`JNQcHEt^Ek$ozeRv3j?(&Pgq{29P>yLen|07i9}*U2<;R|HJy`vi$YkHO+p@{DOP54wnNyis!5`Prj87m8p?lXsTN zxM&}WOor{aGsn07D)VJm*VdwdE+>6O@>Ub>AA_kPY7U+x3e1;prF56mEqovQLhZBK zj9v7|5TB9?1Nt}WYU(pb&YbuxfliFpZ@RJEa!(j}$syhwy)6)$bKY8PkXe|1_r0wHkB4hQ3ehvXD14gUtgC!^L z;J1Ql+y{;at?2V0&<6r87_|;L^Nbq^N{;;&tCYU9A#;3iu!M0`BUp)(7wO!*b=2+@ z5Xn%c|gT;P`DGG-UHUIl8bojS_pM~}BvnH&@rzVLS+HP-eC7|XcrZ5nGo3T(f z`__0k4Ol{I*mk`=?l#d!}^df z;@7}LPDx71L5o~jAsWRe&Tm6(7hU)^jWZLJD;6Y_K4QH&qAe-GL4HO8D5;6$s`=9HuJ$vz{%}XW?|bW^}73S-h%R6UPLQ4a3xK50rh3`cWj8ik zCojR#ar4DeeezK3VoQ7!%slHSphvvwJN#P{u|78@V>ZCRGpoQ7rY7^(S41r2ba3Sa zx`?X9cADCes%*)g)5&tmEF!C+FsgRT>W>l z`o`Odn~#%s=I4yv&czosWc=((E3K#b;|0|hn*B6ZVKj((n`=;unG{| z+-$1ckQCD7L*-2qU9ylQHK_@{p9{|HT)V>p>{vOR;%&4yQu?r6Efx08d2MoopV!ASpm&OZixjwRykZIB#)Nd_284G|Yn3@q1cYWq0{G52VJ=)&? zualA01?LY;Y=NG(gqU7^=l;UQ^UqC4RK9`1`QpA8a1}IUP}1<*327vP(IAK9q=L;T zsDB{B3`?l<+rRRR9>CgRfanT*H!EPVs42rOwC*MfMx5O&M(&fO1<(d(v@8^!>LaI> z`V!e(1UaRN^=Z_mB6Y_+xrIoZ6kJd{gm-^5Wt~EFxftoA>`SU^Ur{mF-ee9%Ub!o* z8^eDm=He8^8vXVIer!W zoxTF-Q8C@%V5GhO_Tv&D5kkNAJm#LD!EI2p(ys1A=M%=!%(Lhmgt$~&mKO-X@(41$ zUN{mj(exQjp^D|xrd$yENqP_{rRh+LDZ9h#-n-Bo0feFi{mib(q{QHWh7Upo4|)}W zsnHmBc3uE_L@P$B-ag)|Pv_}L_@Vqfd~>sN>s>*WNh-Mq2chm#shfVnp)i>es>~Q` zqH#~5wU7gkv2jIQi4Vg&XTDdXLn>JeTON^tHD$ERw}CAr3CVq`6FcfN8_Rc5K?h%C z%(HBna(e=H;!+@?xIY+%lwV{2CV3}98RXcZzV2Jr(mna8O078^(~`Hh|E9}hXL9wS zF>+ww%~&NM+1sCrvFCaBvwEspDoP#yO?Y6iv|UsmFJMA|Y&hShJ%SJVgpgzhQ<>qx zeD{{(5G=QMvAbe~w9yJKClu-ZG3{e5%z%w&2P&kLH?2i^s|KuW7h zbzB&dyUMlsysfIXwp(fpEeOP|qP5^vvS*F~87cfUj99!(4|0RqXO-BJD9>o%+U=hC zTAn#e!D4d`Ky=CG-xUR8P$5*{P-v$Hq3Zezoh8U0n9|H58MAqCM_ zr#cA%g=Ib;j{~ju6EY}A9(9G3iFHWm%TJ*EK_^2y1=g5?cRjWBJI{nR-O96#2DVIS z=SMBRwDC_R1`9uKSaP*F%@ANa!$}c!+=9ZXBY7fc4k6DVJj0aEtglb%R?wGrqcDLti_JM)HD@? zK}k88Sjiqk5<E=&RhrTSEoHfc*4be5S_3hIE%{|hH7gC{-} z9VX`3ZqXJHhFtyqU6yP1v&~%6HIwxecjWVF(c`rg8Mq=v-Rl6LPjdDl2nDyu#(=N>E~+GSo$Q>HvX=6F1RiI$BDaKY z0~ui-NvivhqrG*?tL( zuut(B#-%6raZT;W=oNBGYOX3B3z#-c0*Z?nkrS9#-0?}nXyG@OFW^I+0C}j7*YPsx z(wZAO^#@?Ka-%SUUf#rYJqq+U4AXl&(yVEU$w{TEx##zflC}Kgr`&{R*3?#;u`QDl zy)RBl$&&@;){2~7mGC%_Wg`Z=a{P{SrJo2F@PI4#y5@-Zmofb|Fk6SYN$DBVDf9t) z`<)2DYa|>11Jsv*EMWj@$Z@YS(k)hFJu+Ps-amj?m%<=R2AwHS+zJn@eIh0qX#5NUt{u}s99V(AW zTShej@`envc=6j|`q6c^ohF8E zJv=;|k1oqPk)amDOcsdNQIbhK_CLT#4Q`e1Hd13ttmw1%D*#hUqi<(g&Za$6C;veI z)MK(-_#@ULc6}p_xla-dhu6rw%0pIu1g4@Z8yLvi?5@?dLdUis8P!o*ogGA<>Wb(3 zj~br@JV3Ah!npi&jZ*EVp>UJQH6zY#wc0wr`_)EMFcX#?! z1^Y^|OXMzK6{?&5p<`4IEQZOzswmi8L46PT2>%rB_IjGL6g~FQQ!hrZ4TW{q> zRpa~JwgZgsm#?0e@^>sNUp=0`0O5xUqR^|GGS7Lu_hUvth$WiBjZwF0WDY`IEAC)V zAEl+1fY$W*?|WJ-ZNi+Ot5y;kZc_h&9InZVzr^cDX$jep0gI*`FU?+ z#SxOS8G8sB8Vri7cZXWg#B%WSPv0B4g?KXh2?u=Iy6rl;e!Se`QN2{u7>^!66x6?9#( zeg2^i0QmrIBY_xY%bA9Vi`TM@5S6Kkw4#qWrX=L#hxXzDN6XDXRVZ*HkUOC?JIf}u z2wxp{vK!k=PY3$27Z`CD5vaomDj?j@h&ROJeM5kOx1GzXZ?==8i;GL317!{?R#H-? zaz*Lag&)v7c}y&kNoy{?`sA}Sz_p5uKOu|(#y4LiA(N((u57_BYGyA;wzljqARw?f z7ksrGe-O@)sr-UPVcZ~?!4Hc-LVPC;ph=gT9oqm0g*VWQF|=MlZL+4Zbj5H= zRb{d)E7clA96I{_eM#D@z}58`fB0ZLS&Q@3_#{1DKBH=wBGw1}B0+`v?)GWz7v1jL z=p0|v8LDcl97)$*5HAm%Ym-R<73Vw0LMSg>izK6(IgPer(va*Bbr`_RCUBZZn(nmjc;}2YwG(KhL=H}!;+%>r{l&~?e zf$38B<-C_g_Pa!sdA|8qIHLB6-o_o?tQiWJSlBj=f^aOFFbJ-`m=;XR#)G86QAdX=!-?5IZ?V{AFo6xWDWg&B%q2uQ4bXt6IBbbq{4;Z@65ozy*q{;$W>>Hp`xr+*>xTNofd z{kX5!15CzHhT!JlwvqU+uBHLNY$5C$VUn_4dfo)~UvWdD6{vZNy>^k6z}WUejgsJt zaSZnfdFx)0b^o1`O#L2@dkiu!_v(e$W=rS7v<2~b5Mc?p$SnJ(o{dz@c~z zuO9aD^TdKu5IOAawLwj2eOv<}_bWZ?i>t&pyvFV@cOwjdj1&Au^ws_4P|tW{XngxU z4iJP?3F7#>`;`+#!es6dtu1bNpN zl(@=#^A(8-iturbug$|Yw6~T2cQXbZHKp+M-_gSTdz@EW&TL7{h6s3g(z4{Cpdy*1 z@3_gSAW#@NNFY{|%g_z!8_QD#r8T4%M)ld#(pQktDXpjI0;-7i5|s3(Q4*xN?cg;> z4&#EtU@XL?qu>7A*!fQKv7 z%c}Q7g}bb(+6fIGVQ%KAFayCD8m+Z`5tH;nu9C7kqV^ORp28pku4L?lcZ}|tY-^(SID!AsT*3B(eQC!iGR3~2U!gO`@Jb^VC)W*;w#n<72Tnsz!V zfbhuv9=mS36A_eDMD9zTsRY6;Aa@RZq4W_ILJs=WgjcUU`pdnXff#1a_T;`7Z1V-b z)e6ZCsp)Nmg3QZ9R7~v@>2eccAYl8fndR7Qb8BG9hfDmR1>;trsmMT-NAU1sz9-BL zmeW(hHh^*S@9g+>PUi0|N9qiJe+8WU6CP`G+5mNj!nRWOOwBl2>hEspU!ne$b8rM~ zTAueypKoUp#mQe%ruD+Ou(W@>W6QTUArwo{g{67pfC|XDkf7A=P-tO}3aW_I?Nqq- z^`B#tRO(O;jMI!)!H2~>G*F2V5*i}!1YGr?{vs=41S#3!$sFg)8!%o^@SXm}*56K< zCUq@WJM$OZMF@ySV45G=w&LKh_g*`*W~4o6>M4IV84*6klPB@mb0_gZ!6@1&pp;}? z+|B;P;Jq{hn8GkjsCird8`_JVBtdb2#sWOl)aNxVT1P|s!IwYTxzd|rhX@#%C?ya$ zA$tOyp>)P(-Jc^!=@kmVIT{+M`Ya2!s{GJHxtf}PrFz+Kn*w6}p{AoifT9XkU(O=n zK+Zh^+EIp4Nioi%q!e7vL;-0a3Fij{i3)~dMGJNEi_g>$m$ayl!a>O}dC=s>S1q3M z8-j@A(z>dLR%3OgJ5d1xL!Vjlk9xtk=){+b2L9SXJE2`HKa&XQ&!84k^(hI_4=8)dRR!tVZY(kO9`W-jcuhtjejCg!3&8wkg9lK?vZ%zSt|N8b0EQ zK%2uV5rqtPZv`luD0!Tq3oh)~A>wzszHvLmg`S>9vM322JH zI3jQLNRaSL_&7iHZ~qcedaShM*^GRjAFo6H_Fo4B-%lHlA>%+aSY^<_2QRhFj&250 zhQp_q|0LhKu66n!EzixRMQVv=B;*S`hCCmw!&MqS?HHR~$Nx(2xj%W{KD>>yTM?>2 zM8L9k^0;lB63fmuYC%{L7OFMvpgbq@<`Wd0e!jIcgBKINF%G`abX-|-2#G25?e65^ z?pTn_oB= ze_9E#7xBK^By*kDl|#=Qf7kJq@-I#}KxhzuOkT}--u;7vp)5Qz|MpSSHAtr}NofsW z=l6ZFD6A6iZQ)<|+R_@E2Z!7KKbpQVypHaBJ4w^Hjh)7}ZQFKFY};02+ezcbw(Z7h zY+LUC>6>e7KF_*lq}wa&+TBj44-0#E;Pu5u&J>0ik69My9lmAFU1J6##kO1RpWCy4 zXj>u9mo~cBeyacPwEnHXIvnejn+_rY?IGy(Q5q9VO0=^`J_+B0t6MArhDVyJg4HqX zp&CTchDB^!Y&Bi~+R}u!D1&YsFC-Q1P|px$8dy`wrQB*8^S$sxAvj$2`e_5gIX3|B z?jj){9Cnob_Myi2-6O*1&)Q(4?u)M-7$tuFXAj-YK^)@qaG1IwvFF$kj-?wk`(vgx3Q>7sz6gqPmk`LzTxmiG2b}UDL&M>U=NS zhV=^CCD3MDIfuew;!`*OXf%`)9;DTd_+*Etd0m~wSBW$#K3-P6f%_0WI6Z#Q2?9`O z6y)>l){D0DgcBKUR_Ts}|1AlO?pIBsF`S$KLjGeRLB}WJtCoo3U<;IZ4;s&RGVo@| z_QxxSrDMBX3of4eKWOJC6;*IG)ix;oCH<3`xVY-{UU&9J%(eHYh@XQqceg$pThAAd zpC8?yvF^m39UU)Gi2Q%*juR(cIP@=N20)KcW=OTqk}Awy@1xKJK!dA7W2?n+f2^P9 z@nXHrXK%T7iC#alsyaq)vI1ra0WvQRvh&cO-L)mRWSQpGRRKN(+eV;~VorxbZ4L(?~ce=WA+`1pnKVRJZniE2c_ZJcQ zpYq~le)tqWS8>pna%d1B#27^|!E?6%S@^r5p@FWiwMIIGgUDf5ZTH+UG}&AO_@VkS zG9h1KJzI-Ipx7H`kuVk5%bhniZ>3|myk9TJEtQ%%mB@^tkbxgkkN-%Lz0X_wt=D1y zaWXg+=4QuGW!sd+1uaGD`hQq}_fEI&SB1~J+tC#^(v~Of%R!k#=!;D)9W8#3hY7dM zchskqX#i6r(?}L<@3PU@Y$q?!^?p51RI`*gu48|t@qcI6V6WG%T%1Z(9RF9{$6zQc1G?}l8b4x_OY0l%6T3ddI7yEtHqzbu; zTGOwSmZXx&r+hT2loW>Co z9qEM)6*?6nj15k7t2$)P|G+=3 zzI}s+-N|d~bf@lo`}1}VW&dBPkl?$)8rr{yAU;YcQTE)|?NkCFA^tpB)zRR8x+>hL zAyk)E50lqaMDNxv`WKGc#V8CnBV%tHFJX_FAa)o7Kyf4*C(|kr&iN^Ss6IrC%2u#INUxZv_`=JWrmX4NA!cutlHkJH9 z0KuKN#w91q-TUyOlLvwqbsDwzWa##=EtfBoriGu5KUd+<+r;Cik4dqr?Xzn6&HH~4 zTWWWzq-3UJ?j=4+H$KTS``3xoq+-mZax@D^uI;lIAB|Qo83~P6+hl#FvkkM6VzYQA z!bh!|w(h-#z(6Eykvb$&5}Q<-UR!)PA|fNfczec7xy+_ry1*6%%eI8FzG4z7Fhl8^ z>Wn zb@42{9~$^5?azcagC)6`OU4GM{WN`;E2+yGjwob?p$pg}E;O1)f)D_URue zc|4pqGnTU@Rwxa-F5A$&E(3=-lHP79W5V3&$LvdQ8jL8PhZQr1Pu0wG16La|i25;7w=i)JLS)d0DQPY+1d$rxoxT%F6zRDyIGwNxDRB zi4~8}+Z<#J*2^Bk$I9<;qbk#Lxo^ltpE4&-Oh}hag+{xEdxg?>y*vDQ!tn71_)L5r z_`M2z9L&TQW;r##hlD$7ovzf2EpEPK#BG%d=xA!|ZDr(Se17D6$NN{2AzLz)kW(bgAsDU|r1`!K?twIG*zK8j| z-27uDi5r1W;M38ri4g6;p)pw&3yfYoWKl`H{228r*99{qlp#1Ado284j^Es`QJwp0 zNpi7pa4|m{gui~xL5O4tLv6aZME%&43*$CeWJszM>1aLG`S@kwubM%(=l#Z|PjgN4 zV=)#b3LpOTa~DvI{Q@?E1??PyxuGNi=-SnYOM}UR!+^c{elAvMSXo+veQzK<^Vx~2 zkb}&~@4gm#6@jqYmqf;do@u$TTV!gCp1-~|B*8|pbRq6 z|G_=#-MqT^f`f_PxZe}JUqK6AAqsd2q|$Ue*%VKr5TW*1mJ1^Twz0QV_2;A4qnm){ zMs-CM&w8B9Fb4>Br^wBrz)6aVtiVuQO|BoY+s((*^usLoeTe+IDl_#f^OAPO1m8qM z2po86!k!fP>XaSynSSrA33WoSA(6o191cP#p!5AcAYLPw)Pt2ySA&m@lg-#0dSgi= zB`q8`QhsvM{d(c1K_UKmU2jmzcvY2YcP_0F6D~WmFkiNLsbBfuXp}W1hnt$%Wr;qI zF$viCurImXb^tV(AAaZKN~wXjg;0-F+jo?Sw=jSPpGYE?I`#Pe1PB)#^}bIFw8K*` z^FSc44O6AMUIr_kgRA}4IvfF)PR$Ank4v7oO2`aB^!tbclzlL#-9Dd61up#TKX0cm zNTdR+r=hQKGk6ZbROM0r<1nN`gSe!(gy1Ep-bEu^J{uz`>7-we7-N2j-oADM$ES@> zH=4|3XpfRJ9ot#I0T=FXV8M_XCG|@N-HPb2$oc8x)ir#%Iq4H7bm%^XdtLc;!AK6- z^$+zF97`AkQ+z6yNiB!)o$X?Hg6YSjsCNb(SDv5eYtPnD4~TO+nj%H0r>fUN3p!FL z*x;Y1m<;?~E&w=2N^pJxpof~)uP|@z7_OusKwo@0cI4AzCKZPLg&wWZ?rA2$9`}Cd zO}siNv^=~9f$WbQ&z+<69P}e5$g`gNi@;TTWs~MFFy7vtVt7pat+woO^o6cI4~pNS z+!t_YKV0LanGB@uIDZ?sqDV>)@(5W*z(mYYMS%yPE@5I3izpjMD_igc%(31^!B5w( zyz$eCR%7GW(b1$T1^Bu-KpQsWW{_;Ze{Cvdkeycz8g31U6U5eytb3;(PmP2XuI`~X zS%cz~(s6YdST`zTCFB$O{BIl^ZE7Q0dCQZw`&$++9vB3oJk+IB_SYS{ChfyT>s%)? z%ehtuy1Y+t$enLx=lYxL0pny;TIYUxAqyU(!Zg75`kZ;%7=nw}uet0bcn6d0`mcuK zy+gCa5CQZ;nZR`B_luWa{LTLztB~){_0>}R2F5xxRdF{>+46rAAG805c^(?$ zmtxAY)a{2Mja0nxa8-66P*x$AWN-ZAFX2+4~P^e4?{DH&qSx7Z4%r*lof8+H4RZe#unE6wrbY{v{Tp z#y5!y{Ge|YW2!OtX|@MvRZM9=Bsib4hPB1RE?x}O+V6$ouH>I%i(u-HIXghC-lm&>m= z>CXD6)Y82!cWqrAs4z^h%dx10y3&V>y`#3vqR=+~XTICr@%m$p>)(Y%(k3?!#pFb3 z6Khz2sDX!#KbC-TW9h07;3Ny;rn~0Wbu}~+x9Vgw?_($RTJPEn$X_ov(0_j=S)_>h z5n)q}dQ-9*ZjsSMcx>rQs5|diU}9yYC335%7j2Uv45J()q>9G8W<$D`j#~Ai>ll+_&!)AIuldw-Aqn~ zo#LDrrCgqkvgn*vK&5@VZ)~#Ab_{E9izjg%N_~!v{vk(F;$W0}3MLpx(m$5-Uqa(f zB<+f1`sC@^hEpLN3Ke1O2xt#t$UxjY#F#rdvBbG|{dsb>hQM}pbz{XVpT~}lR*;=! zXqMoKWhXG5)7Q(u#>R!dxIE@rLi+B`x-LJ^++-9Tmf0EF^y2-~*qD^!&akDty{fp0 zI%#SEI!wG7g&fR&^7VmdOT3Q);0c-LLW|^2J*2diE0X^pz`@y+-ec=Ox`(8 z4|_-3L#}vWk#=zk;uJXy41mGMO;FgoPMI(z*0F_vjt(l&egrH#PjMaOSlV8>D=I2n zY>g8lW_LcSs%+#Cn;OJo**NkUOG}5$%+!832WY#HuH*8d#y}>nWK~n0Ai#^4udWs@ zE~>SvIyztNxH5}n)Rfj}L$-u*+C)3~J%PiF^bcavlS!ko`+Qj;sk}}|v0I{XIT3@Gc z`3ege!11j+H$2a7CoVDQMVeYm!f5a=~oSyUgUDu^gZ?xVYkBz~BG3J?%9~)a+ zIgyYHCHAPoJv^I5VYp=TZrq&hpix*5VUB>45+j=nYFiuKE_J^wHb&`XXJ?lLd$Zp< z)5HFDs(ZcM;(3@F%(+cX;eWvcQ}sF*1-(Br7v@bL^0-~-ipYsV*j812-0a5Idt{!X zNWm>wFNk~yZnfr!ENWnE z`t7gAfSKo|qDxwQYirltjzsR>%~waU{?(soJvBc{P7$#gPN&9@_Qp$`Du8mw#|RNJ zR=*-8D<@-|%wVQI*XqWa+vmfEGdl+h3jhU1hJ~EEcOGU3)h?U5sJYv3{{->SlP02= znI4em@>jtf)!aAK*jQJRRvI#La$ZrUMmD>OT3J~UZ1@#9aBrE|aW5q6cl z*xvRxd)G}(DeZ`%*PmvWow;Y6)!x&Wqm`#?Jzj_qN8qIqFC2wx%#kvJkP0Q3q=O^cmem>%?DTofD4(bo zFjm`Wt8NEpC5@j@AVgYpsjrha-S15HxWO`?rs;gW7)ISi&A>~L(`mLPIPdnlD59#r zp4nkh9^tB@qfo#f{~k+8cQB#DYc`dI_`L&K?jU zcS1U!m(wu$Hwn(E2fEBxs4)3#?pELPv6$#{iI^WI66_q!Iouw+0_=6}&viCDuJ~N; zMVijdb{h*EoX#%`qxV)z*0y#6&Sz;T#PjQpCwtcsXWL_J6>A)}yzYC!!OKl9^V{1! zqq`Ib4Gq?74Xb`AvIr8~ptZkrg@yfqp|n1$-{wP=4nMcId!G~YBweWj9>MhYO0D`5 zzz}usilVx-e=z^hP-9g=GMChIl+xx?7s zG0`>>F*%CTEWr^2rk0d)d%75N5jI6!79Dr_`VnG);R3oU99j9@VS0Ho6?HI?_SO=?Y8XqMzo{~xDxqYZrs^*ysa0<~gEb=)(*qRa8B2YQLO(YR|dcw96PE8-IHLVRK2!5WTR@7w(# z`ifD`uoZt>jHp5710gk{A(583i~ig!*+a`eyZeY8x8qygg}rbvK8kFVP+NT}?St2~ zwP%w~m@?I&GumD???-ZiEO_~i8CdV1AGdx9X8|BgdOd!>9XRBGWYQ`jS%<$bjRfzI zGZ|#HT`rxc0`KpiWY~m6&L5Kmofx|)q7ZFQy9DoqLu885fBxzp57a8idpn|0Mhzep zJ!czq8U;25l8PWL@BCXf-?{v}^dlrLc#8GA`!Y%v4jCj9vi^CS+dU2m9YJ$A<($I@ zJ7*{q7^0qsCA87D@@~-O^{t36h436>xPE0;=a8WdKk@G6a^v~q@nf_bZgsB?x%_FU z`+Y`o)9>=)vHKA*a6E>WFoLi&@Axoyis0j_bGMrTIX;xMDj1A4hJerYZ5er9d=w{X zjueL0<#?f%#4H9sasm6x_R(Bt4({hZK@EgrxZkJO7J4O#iRfJy7?P+9c+_+ZaS?uF z?K~Yu)V?nfD3tnu!m;%&WGZv=Hib4zfMaWeFMgbA^yV^wbZHO$amXE=54LTi_wF-{~^gh`O7Ol zMq6E_%Gbf8+a<{r>AV*8_>8qJhcKuJ9h8U9iPQwVF;Uo_T%~v~RzD z_!a(Ta;nu#Qbtiyv1hJo?8z8i{ycOvO1K}9^mTV8O_wLps4QpX zBqk=F9CBi@XNjkU*-kl7C8ko)YgMLAsoprtZg!u1d%k_S*d)|*jA{i=+r_sp9j#SP zn6w>ijKVXkPFjiNWkl&P166eC#(i8)`_i?Vc<0Ce{V(wO*7I^Rscr87;AeMS=saRD zSi9ksB}EQ`VglCWZZ#(K{@8mvk0ecCwp5%B)$wH}UTc2ONY=G)NPw(Z<9nSc|ak7Q+aHA5;qF?ISNs(`{kv(Eja;~5@sJ7lU*_9e~* zDp@Dl_3j?)5mX!sq7#gE9_Ouw`9y4x^%+>x2RFq40iNDqod{G{Tt*ci?&9WxaXWa* z;rVc}kr~P5QFD}@hheH9Xi>GJR8Z^!oId68E8>KHp)14Q{)#N*dpw}kZ8Y5M=xXPH zWbL7=k}ESlTd$=V^v;3BFZzXCKNfA`6asoY?`oA%=Zx8HEsnU(BHsCK7A8K)F?h>@ z2M!kbEX;4v;rn_~JvhEA|7)W|0_?5y?|KqG-Uu@)bfy^{9bH74Q`362&w&nMqt2K@ z@U$MM{O8qCMpKDwNf{>-e>;~?VQC_Xg{M?|$lHKuZ4t6v_F9tUki14rv6q{tq*uvQ zb*398oq6ma-dCs!Wg>%-=&e?#Vecq#K2GKo+nmLdK@_Ia_KJE&CB{0kf8VXhmg)8> z96b!0G-(g|zF4T^vsg(cA|T(J(X#EhOk65YUaP9MJM($+M3p5HYa}&A5=IT8{3Ndu zwhVG$o#T_}8-ed(>g~XS@f8{D@VE^o;jlRl=Xtq%mWjj@=%RoZV8*QV2eEIq4hi?e z5TTE;w;fc8Lzm4?>7j7luJh z7(oP0+xNh%tNhHe)lwPs8J!LGxP5;h2YH{Yiy4I&o$%MYPSQ+<=CKq>i9gi;akmXPVzv$3%m$7FCO zKC>OA&yX@#mIMZCj+&uNLyra* zA=`Pmm<#oi;f~&Lbi$3Y8p0l`GrB)L*UP1UJvWCEc9?vzU*KjAuv!MHmyun@dyZ zdtQHyCS`%oCZWm~41g{fH{4-VA;vIUQnQWVk`fh%h4^Hn1w;*WsQ)+y1edldQWniy z6znkONJ_CJLK1=v3f;{!{`wt3wR`CYqfrY*NEsgvKvOXdyyLfP38&{tjd0LCX}~8l z8j@;7%T=i@YiLE7v!#=kZ`W^Fs9O)P(;38w;}yv?TFJoOkw|5sj_M|~IY~iG5^fDO z1`W>X&n0_MP~(YgAb~TO`ik%%%|X*`gl~61ymFauI@c%i`&duU-%%1odi3m{O{e_qcq9qTm_n7zFZsk-SQSw4<)dUW`=_+-87WV z=l+a`+NO@hgY792C&!K&-UVn##qSJ(!YiO0^7Z?|}sRZ2-x<*t49l_!s1Zk%}@n%V8k;H$_MW<73V{)9s= zWWz;=l(>|P6FN|E7vlOqEPxi@Mzl{a!uM5uy4EsLsM#%&!}Rq2RAv_FouhYu9c9kx zbHAVaktUtZZGN}6xxghwd10PQ-YBgndcE{Aw846;y%trt)N+4mU#G27J3k~+)4p+u z-bE}iT$-69da~U}d8Fpqmh>wi}8yy*iH#i{J~>o5ZJo7U{LW(U@2O!x!F*)XA&Y94le&$oWWb&z~~ z;mFSzlQ}6w!X5J@R)_o~f@za}W_(fy=J=%eJxZL_c8A@e$u&1__R1|F#>58^U?fAs zd;=5G#(P-WK|pu!H9^X+3H+*O8TL50Oxq%t#H4QH}owZ$K)K zKFwSSK8s0`52So*)E0qiTaxB(d5N3>A)wM+A1@5H48SSnBDer#N3@NEsWU%C5uW%xi1+hFis5Jmoriw#A03e}a zZ^gg@6+tLTvV=1Z=2a0p{u`jth(QxtdOl_);ooN>MKU2ET8u>I6262!DRNLhNf7$8 zXc`U2M}(v$s_Q)EQp*_xdV>LG)Zo#)u^+X$`i60d9MboAnjZaHw5Y-Dfh1^&7iKhF zjR8`m-a7VRdQfe7$7@Q9pO=KGWHDpgKMv)s>sP!@8#%?x`PD}h#YuITek-%4x z`aqgA9ki`Uj0{iV8bI?4ENa^;zxO*#Alpb01O4G-89W>3cdK26-(rJ?!5@ z!VHp#81khBhoyOzWV#&Ka(Cyr=q2jD#hnD*A1_-KAJy1e7`AWCO6rJ)|bZ? zgnQ~!d_M=Y881NYM z7Sg4eEM?bcX@5P>+lJ?kOJu1xRu>R@O}`#N3z7nMtU#V1>ga4O3llG2>UOsLcMi=q z^!0&6o;F>U9CIQY;13=c_Z^>g?Fhw0C!Z>m&eNrnr@hf5_ci2^YrwO#SM$Wv&>FOI zC)-Ms{X@B8>4Kk4vUrfLR@ZDWSE5b?NNGQZV`5^GBR5~}Pv?*CPSB%tGUhtpzeM7} zaJg-E$Z|J=_HX{9U~nF_(OYS^xJxl-PZ=u*1W}ksh|7rCZG_&J$$H-36w@7x+WF-Y z{{%(B9od%>9iak?}mON3KBAjx)mFRcv><&n(MsF);)L6Yz6JW>#^jONQia(5_!38bVb8zq&bnHI= z4!Z)7W_n#f-rl-Y==_r(B!Nx^oGW8w~pR1xaKM3ong3O5n5;g8>z(1L%cdPI|5 zk2?t`ofWV;Ej#S)@f~%9nSSZQJnSW6i!6#!3Z*P9y*6;fTWl3Y`S&uq zCn)Q(I_?rfV&ROVZ3aeLV6r@c^=1{6bSpH7UiJGaX=E7`Rx+~Rv_RZq3JHmlznNGE z3}l)Mz1*yCZl7kAqcW2cV=9*?8Fw7xHl!8);L__sjQAK`2J>?BN)QLqk4nC?^Bw+x z-W@=se5I<1>t0$Kmj+kd^NLl^u!q~nP{`tJ^iX+adF=+%Yx6wUemHTNp4X_^T3_C* zt*v@LS)7VmYE#JN>?|uhUwOCEO0CxG^nJEOtc8M1(?l&Yx6KA9rw_)uXhZcttXe<= zU|g=e_|P}9zTfK;3V=EjYK3tA%+{LQ-d%)neviFst?`MU>8G~(;MC~`X3aVhV>~%5 zt)TOz{?^+we6GgRGgHsf(VmRyl-NGXUOWu}n3xiI^12LDRoXmlv=!<@n}up2{ewo+$uy2hRi z2W+L&F|l%#@hz_Bt0~boxT~vxd)nswyp8LwTABOFLK$7=YOcpmKYk9+kBMttUqH>l zD#0Wh4L;P`BD>jSEyh?EgQ6~#L7$TP(`Cf=c%e*Xjor@s9pPI!gZgM6}h7kz&Oet zL65U$Hf0N-21lCf*pA5OTl9G!DtQ2extgqiuh`*2!i=ayI2xxv%ja@4krLBn_E57} zzPz&HalHBs1i^$%)x5Cc;&U4M?fe|KTDR5leD!{}l))M>O8>n5?@r@e>q*H zl-!;)qf=Z$A5W62e5&0`d z9<^h0bMyK3@Z9SGUKplEv(5gn1O$>~L}qPZ_ZdVW?lqoB?a4d%y8_rcz~^d@vV@AC zVaQM@WNpsPKD>PlCuy|!cE0{g#+?}|*MJd3sbyL*T%tRGdWFtlDa)_z5YNIQw3*YY zS5gCqR4<-8PAqM*we-7qen)Msbt)AY-FLcn*A)pIown?QI6E9OLE%73 zMlIz4p2#)&^INC$KGb$_rp<(G+{9jMTg9ok#M%q+3M({f9`5favsq5X*O|6VkOcm) z8*{Ux#r948^R38|_pm|))q3aTn>}{lgxe{phsfsWbG+yb-1pV-)1tflO{KoGtFD1d zU!@r?okKTgzV8iUE~Yj{S?o7nl2^MeG#@uZo>Mk*H)h<(N-s(>ZP5_Nshl`_~^^rg1lSsePLJcAE$Y>xh2?HaLsj*Pb{8@!y z-2>S~-1i8>snPj)dS~`dnZXzXdJ<$N5F)8iaFhjQ^oWVYk$?t9BTR zLP_$-+6m=8covKTL;I0TLyCJv<0_*kAw<%CM+aDyGkhVOVE2bPftIQ7i*1r28Y2Bg zQewak_@)sm=or>hW-tzkU?Q zZp!J)1TH9?&RHVsyG}3QiUrF&Ov@W-5g3d+!1nms+szm3pJE>kswl`I^vvk*RU@32 z1SjLtrORkf)j6*fAx%#vl%xkY9AJY1rY9j?L&_D;UnA&12y9t}VPT!{Jg8wv@B2=F zws4q%n1{3m3cNosYv1FHp(NwM8fNg3k>SbZuF~=-we=z5Dq##^#DsGH!08p>L@UcV zfCZy~P40w;gnl6h-{m7^6+#A&!+}7!0rUR~Up6kz-8({2Phc_j)Tw>%UAP5uVp?{u zI6mS~4Z9IFR*_ub7n<)UU&W&ZPBFKW2o&>G5`V1e&6cyRi6-~^zU7;b9vhvIM)&9k zS=plf)zXMX%BiTJj6m^zIYNU@T;xiA7h9PiSvL+};PRMkT#^rS%wl={2NaJm2)1^D zzXoZBi3cJ`WFo1U?#tr+&VO%?z5LOWvHiRn-{U|Y?!W)~9ZN-2j!;h2bc^flo10`D zM1WQtIin;swan1OszT7eSH(2LY)Fi&@Gtu1D>v2!=c+%nK&(?Xot6L4wh5SVPv5MAZ~TU1nWz))+L0Ri*G{Wde~9iCMx|Q#Y?3 zQU3xQ*a&_gU^z)JqQ-|_K3Xj>=Oc8Tq2Y6xc=W+WB81g~1uJ!aW&Xlg;qB-CCoV%P zA%5j2szB3?R@#C24H+6FIrfy#n|}1!6n0s7BRLYhoEixRp+IDok|FlmDq%~-Rp#W* z?}I;6U`qVcrmU8tQ27j@gr@Xsu}{pLgue}!gYScl>K9Xj| zKABO_sv%F9xN70<+c+h4t>Mg#Cx^()WhQb)Y9p-F)h({v-XE$&f2jv;pNM$aVfn>Hj>Qp`<@Kk8p|H98 zl~LIO>^6F_?cYCy#d`fYm5&zhe?!QyXG8;6thY5{0;oJoyxRp^)tfcd;ubu?my$(d zEZ+)pI{M`pXGeEiQrBK*W{0h;U~+GK7EwJkXMO1jGI1BbRclE=v1Hn;w{eqv%IrVO<~^k~6=67VNeGF^F%;BV@z{uW?->S~FZbvcD>ZZA|vR z*=pPBstT(J%xt$S`i1|6L*m=@v$7<9T;oa`1l2CwP&99t^T0%eeGf%`6H$%AN)0_e`;%J-&E=AF zQo#xri=VB^#as*}xm;QVxrp<+Y$jw;cTKRwk__g=!setos-@g9r>WdZJE$~XJp@)Z zLu<*)QjJaZTalElk`=Y`y}!3x=8!=&Rj#jU8eH*Qk&#h?m)wos?>Ch%=&l?k==04r zuyS!{MRUMdub?B)Ar@RH%9u3G&j{y7d|^zh7U_-eJ_`y=*ub5Z+4YF(`WIyzF$&+hG~x%uvl zW@s34<=>(FUWp8?#n4EYnsTc+WeX~m+|>tm?)edPld)@-h*Y`7r}U28Q|2GZTh8>| zTQcO>74w*77xTI6f>3PlD@Q&5%`6rRbY(MdG3X z0>7yjS2>eTXSLyp(k34?-XF*A7P+HkQbeM0s9dUgqB*v3&d0y{{xMHtBku@h&3qB# zvys*GOWv2cH&r!^v-t1nVD`Z_XALwz=?)HUh#=VDg1O{U!A&tl)5bzU)aXH@eoFO0 z!utkKtEWC@khj8u&z*!l;ZO)h|Df0je`|qNNKVC@>zvTel4!XUu%3h+J+ESX z9nK;1?8!6qGgw-4vP6PvtgVu3mAu22!y#={&)?q4w0do&l`6bOYlKb*l?)pv5QB41 z7_&-U6FMXuI}@wYIP`r69lH2=xleq!Dko#9te~pF`1jx!{j^)sMOm+=wQ(O-WKdfw zA<^D^gRxiarZTvM6{m`=rasNlmilk&cAApa4&j~qmOhq^1e?cxb@HzIkK%7Jw=_(C zcoLW-#LIjOp|r=!orCAi0yozd=IoVGFRQlLB~MxUZ-cy9gfiP5ahz%?JP-|e z$T!e95axjEP7_e|ujcaKgZ#Y7J$ibsfH?!I5<1{dMw|VX3uiR)at|{|n2^c6 z%FS+jmALyiO+XgMQk9WQ)bSKZ^uJk^Z}vrwBt`&m0k?*#=zP*QAI5KU`5cw7!BAi` z6O}?gZ5kmoIuCGL-Tp3nKDmi*6ZRh%XfY8@_~b6Pdo4}T?f7u5gInNv@s^jTiSq57 zdO~WjQLiHI(^om$FOsst20o0lw}k-%^X@8h3GjA~>JO{%s8cLV^R>6Bn%=4dyWLik z^{n?OTyk%%B|7#HwWxHf&00xu)S=2YHA<6`pv;3{Mm`L;_2nLldiLH(?yN78aHU?m z=Vq4HfM>Zp4U3%Jn$)}1361kaX$B;`+3Y{ED2xeoUAQ0N)NjkL%+xNt^M2Q$hcdz5TR;W-}7V*S6fZ;o%E_NVV$wD zc>Gr%MCjD!)b$fsQOHnW;O4s#kv$Fhrds;(=wx@Wuw+UT#F8=?-+Fo>**B}i?QFeS zwUCrt^6bEhdLfW-kdjd`=Z6^1wz}QMSJM>9QisVMo7>vu@jr`d?EHz)LdtZ!ssCk) zuHti+TbA_bu$dP^g+0=75dE&vUXAoRzS2)L4kIex47VyGh`^X3+$S~gd>;4Fh#}~3 zE2In#1PhMF{x5($N(4n8iPv(O`jbo?A^W zG-l034RHX@rPcFUSx4vfZBWL!%>H5^Uxa`leI_QbFu2}$G2?T`Pn*T=Z{=)!fJoT) zYymH!+J0gpyjH8jp&vTz%@Y}X%U+M$0bA=W9^=K3qXQ}m)@w~BPQ)v$%Wh`!i5>mg zV1xmhj+dLm1Y7GVvO|}5u~Va@^)m?hNE8%ZR>J@og}te13je74@j773`Ei|^P^xyX zi*L=%`dG4PE~wM(kd@0#6F#)ux-gZX-K)M`@OWexX#C*dua<~UCqd_QLG?%a+mO60;ZL8L3tt&gzjH49l#S0OiciH2-;+Ot$uzo@af_lu!AwMw6Zd2FwZ~5P{`)> zz1tNb`>G+CP*+aSX1h5W1)SLV>V5j_v?eX&re*{`dmlgESEG60BEOz*JWjAEoSBXo zd>m}-QZP`qqy7ZekqT!;=Yg)$hqHCplc!k@U)`>v0uEU?5z|o|TCBO-f%rlSEX}PZA=GnE@5`JG zo>+YTC%}Cs`WJ%V`Luq{V%AyPY@&r_`r4}O>SD0TxW>aHBF2!D*W=k{z1Ly92*Kx7 zBLPCJv4O|_c$FMgn2f3In0-&fkb`*JfeUEkOE`P`48`^Slnr_fKuAOL-iQ-Rl&>4f9E`Dwnj#g*p+g-uR} zmx-LtSA3oNjJzJuW6p`Mxs8++5hjBUJ3sf`Ji+Ihj@>(NDnrN3&3F}^MK9qbpz!vx z`PlUcyzp2|dfk>SXMz7V3$7v2v+ode@7GHm`NcaVuVj9zL@WaLH96Su=9bSYE!vOoLw=kUagmz%`(B5)&&^xDuWMT$4}gL0?ZeuoznIMc67ihqxO&Z} z!N=04Tm-4)L4E($`v;D_&yUopEUb}n9=F%Er5fE9>%sI2 zp8;*lOIMAy5Vi<$Sm4Z2o=X=gp&$w(jSln`D`= zwqb7WRfQj~M~Sfl)XKL&EqJNM;2{2vGG5-)wV{UJ>n^A;EVeDF#nH`qGSp=#lO%o0 zdZp=px%Y7@m+M&RU(bKn@^N_{+ZW*o1!{FXucN)F&;n!Q6aW8O)skRc3FOwMt)rD> zcgyCkm5j5@Znnhp4FE9EC~T9@@1Q}V?6K~$<~7r`+Z+b24nLS=d0p`3G~ZL6!DJInMyqifOH~1%y*+r=bDhvhfeJGx@i}F)0CE|G5OhHrk~y?6fOc?Ou-v-g|pUF#$5Yc@T|;7%yL zQn&e+v>fZ#3?L>k)y$s>XYzd*1q3$EJKubImB&D5Yt4>-G7S>B2OuIz4i9|8V9>C( z5M4y{9`Z$MLWC>}jDn*J**S#@2uY|5d&+NWCz0FbYN899aF1%6b$TZ8OOb)c)-sbo zXQdQA=u^6ssToJ3GAsUhySHo&d^|i$5Lvf9pc1Zvbkw?Pi^EP&6fy-i`}WONzCfAj z%=Z?B>BKR9I$uRq%1k?5hx=pf-9`(gCdzw)A6ph={kp63VyX?{hz_F-bz?F`vrO&( zo$6_n5y@lt1W=Fu2t{K zR|&c5`i;W@+2CdDe_>Mbc=(f_FT`lGhI0SZjD1N2z#4uslU5Zc2>BLdnU++a!4|{X z7Jfh1#7Yi4!cBg-Qw#-{;iC&i#G6%XzFw!Y);MSuev`%?e2kaAgQ~U*P%Gp*cLupH z!>qNev-LahKaMmG-Y$JyHTJZLJ+0<%7@Rf+`5{ra7#*@+?NeYkgzDxXzqw#4m&}z` z`ebT#KJ4i{cugZ}xEj%=s(rQjvvCUhP%R=u_A$Eb6W&P@xgErsMrvrpYRrRBBy1`u zwf=77sq?hir<`VnAjXe3nQ6eB>8siD(!^YLib}Ki9x7X>E3`}=kw%5EC(9IB{$p5} zQbucvuwg(zDqCqY$GVgAKe9Tbg3=wLZ2R2I(v*sC-Vfo+wfo;RYw1^gTYcS3ww{-@ zOE5NkJ8O@#M8=#H1h}?H)o~*e1HoNiH{8$@nLuy^W3W@&8L?qvRWH9`nq&^EuF~ok z(3LcYJrR67Yp=A0lL3fD^}(rpy%-LogPBP^P;hz;DmISadb5E(-$ZI^ik&j0;vup| za*WK|qK~P)E{C-bplU@SBzs~ag)8Cbk;a9Gg^`~J<74K0Q>$kYre$IB{J$mrZ+*8X zN2?9}FNuaV-BlAw$@Y+sV=X5Z%-kvkl4RN+$1CC7uErZLvBeEKJc5o*kd;$XEvb;+ z>r(R1>mv&FPDA9N#rdjqDGYdcXLN~)T%79k?{&;4iSY#XNGqkV=;W(^41JNqNv_a# zF1RA-f{n;|$KE|^R@>aB-D0#}VBH>oTl=GIEXd8qrc9MKZAEv)S_*rk9$Ne0_5JX- z+G2$s95g3v6yN06r8U{E`M&Akv;toT_88E6n@CdbH~M~i(o9XpVa*+&-1u#sAj=-; zP8oEX^-pgXB|1pfY(M-;=kVMBG{OnU!Z5G^90U#ekAU`z>2BR_z5gDd!l#>-v~6DY z16njXt{z$_h~@_bYf8lM;qSX`^4^ML3Et#)M@w}ekb|(`PrT22xj9mZ7AZs#lrIAk z>fG=Js$^*k&;V*YNX$2zt8YO0W*#Dbr6oVgX@ZrC?<=D{CnWod`1`*)^?wG5z0fMJ zSbbcHH#dC`=a^KDbT?DT@uW|TGvnpd&R1tBg{t#M7!E|Co>8Z{Q0jXKghsyU0G}3< z-Kk#fT|6S^Sy(DZ5Z1+BGm3h-Hl?)9N{#z&fL5p56|u(PRdWxp`?8nYOLt1n2)T$! zGn6k+f~~Q!n^lUwwEx>V!^*2YRwtuM$NpZz9~$`pi=N%_e66C;%Cj)Qs6JX@=Svn}vu}3E=@&AIacRk(E4HCHPU2tTMu!Upbc;0c zHd!2vKx6)*#~s1v%H{2!+efF{lVx+4b6U3}kIC`WxN&LsvmRWb_p%#lrThx#-UQ37 zTucU0EDr8o4jqo?S>BW>-&|ag!<~K~t_3qH9AA(z=>x~SP0shNYkD>%+wHRCqV25R z(|kRofg{rV?Wi1%GR9Ksz0t!FfSlP}7S#4Ssjfd(r+?_5^exw_E~}}5QK`oeWVyU~ z+#2h*rd-%!{N_}foJ4RyvU0Ea)RM(Gd4()qd8|$%bx$P+R3F%ai#@iPor-zG2DVcR z1#%xLHH?w4>eo|g7bR{yM+7V4NA6ThaI3#l#N#7Vd^gQ6>xm@bJ0YPEqszuNJ+sTbbY1cbqH0daE5|Zo7-quFD<%fIh zla@@4iOfznHH-86YVzN4>$RRw6YEbo>(#lfa4&(d1XGeRnPVT;i(I}f*LtRmva|Bp z+Zjs5i$;-$%Gs{EoNScJ7949#W_NnY|BhIZYBuh5eS8Wbw+-^!e~(u;jy6509b@(|1J> zVFchw3J93wGu-tS9Aiaj(mAl_D}h*ux-hkgAMf)X#uYlB=QWxW1)c6K7Rxu%h;4wc ztN=D<751mgH{49=d)8g-q_{LQiVK0Nv$&0+{z%Z(998{7$tGlR)f4CElG7bQdL}zD ze8G=jF3|U1B=J6yvS&oC6b3KW)Y#cCY&6Pn?cnU2)9H?m{PoGJhMD4&C>(;qmoAt+qO^BBCQaHMb5W=A{{8G zs?7gf=tg5(MM#(W&%GL%lmUmIkFXC>H;NAfZ6 zHCPE@ANhb?SNh1uHZ;vz4S3Kd-kD9;#k?Sd<>F`WU#kn{ozAV2ELvBP|B3IC$yB!no5~tnWsr+Eat_F%a}|m(h4yLQ1Vk`#AUN| z;)%v9GLtGr7k0_n)r0+WyHOJ)f?;oDay}uFHP9lG!CG$)0`-5CcQBh%5=pxhmLtVo zIoege3fLQcKT{0N7Qg`}LUD6Rv)^W9AkVe}Y~``+vqQsUaU!+M{E)S2_GFkD`>`+9 z%}Nlp@5=A0%~I_t0&r8eCMN*J6+d`7tqw+;%!_=iuU|vc@KIr4t0kcrGfQ?T8N%T2V?AEMU>%97(|Q3m7x||BJTs%FD}J|BCxt1l{+#+aC^r-Cr1t z?%Rzc4f=sAWqs8O!|cxy5)#79iL8!TA@Y!-?)RyVAM>R2vs^*^6$odk4OZOL{f%No zut+4K5`y%GT?Qvf7~VBeK+h*sy)#jnRuaKO^+Y6^s0`w4r^j9X3_(;aa4Mk}#W~y?5F4+G7J#;S+TKc%!PZGPFBOXz>nmM- zZ$=1^A-vHXFPA6SjVJ{P)nHADCr9K5M4JPz`m!K0w;%lSph3DxYY^o7Rl zde8SQyOT>FER@0N?6oCp@c1hD_AiO4ote$yeL)_~e`^m*tOR$myZPN4n1p_0C z8--ipWaZ-ijJQ@O9l_(=`?|YTqf&1XqCAl0{%Gm=LE0+kZo>22d&>Ex5Xb1ecGAho zY0OH7_dgkx|K_SzLx3g;{l(gI`1TsR4m`xG3A_v7g%2`iJ1#9UL{2>)VZiH=XC* zW2NtnY`=>8?*9f-nr9@iKnR4Z*SD||zQ)9si(I0pFsJc82!23Ut57)}yM4hVdmhnQ z5Tg)`Y(5uIizEr{ zf~$R`E0?1EG4L;72oA+wgjJnUCzTlgdA-ty!Du385Gd1?R=$qjz22zr*L&-zk7dsD zOzRj9anL2--mMc4@8aUiLSln!t;So7T>cbw`fQ5YKy_%z9_BjQnFOjIrb0=aHVJL* z?A}(mI&NnH-o!O`gect64>9O>FMErfiL4EdliN(bHyMJ@gXi*cQI)!!zE1ZWoo|;G zWbtlh0AmZ^T~9TJ;Qf8?CX;UMYdCc_Nt@@>-`!I2<+}Z0>g=_a+q?5lPtTk>^Yb%1 zuT#3W=SsmQwk>+Ch8pITx7kJvkBiT(uIH~S8XeaP0LzixFt&E4fY0CS=~$UWf}tc$ zGbtVsO|-E8t;v$-x3*@O)A(j@%^Kqxxg&}}n&H@ZJpK!6t2}N1R-4vNxK>myuB^Ag z^Vj3Pz92t<`sU!Qu%4`!|tDn;5hd1V4}bZ=NL`-$>+PFRu&eP z8WwFldSGavu$V-~`Yt})bkBCVVf(szljP|>5XtJa+1c~#L~s?@oTYl5;5rk2KGwPU z`CZ}bXx!IUJ73S`zv7D(`f(@d(NSwn(*;@s89G_Mwpo)zR$;3MG8C2PwTF=r*B)wqbW?C-tG_~m*UR-$N92UL3Y+xLEopy$ds}3 z(+Q*;W`TvxO#^^hRXO6?^=<1--d>;iYK$`Re~z+Iwb(;K9da7yoiRD28o_cXkhh#K zr(UE_nz}ewJ0z{aFKj^&?hB7t{tI{6o%>c@mK4Xrf0l|K^VQ+8T0GE(LB#X!+s*Ty zhFu^o8(s98*f2#SEIXSuG~-I{ zIfrSf?~JqAft^VHj+k@)f1LrVe3Tj@724Z(1I^Mn1xyUWl0Fjl1u`IJ8YAk1c7Lm$ zT6IlHGD8%y=ralcC5TD{1{bBJQM#A4JEpXBUMFSN?oTXNeY-Iiy{}?`t%L?w>gYOIqpzU!9tQqI#}ob=UqTcR&u0hl{SAyB4=gbtAi{W+k9CB=4y_G zt>1iT{d#d7O!&veFC*5wWlX(0+3>DEhlR?ut$!T}wcf}s@I|gf5;Qs;t^C)j_&~h# z@N1+?v}}0tyDUUIu!j%mw|r&oXU02dnNP{2u~}`aIIX6n#_-*t^%E<1m4?9UM#aS7 zM)vQHrnu_0j@jo&m21^Nx#o0eaKfYzIz^*(-^LzVvSVUmft%cGwI@)o-^VXdfcawo zp4hKHZ$?OeN&>LCP&}vwDbiy%>(Kw%K&sH8>^^5#gUWrji$BjuTERgQi`lY;xtDHX zL_AaiVfbR4>c9z36HWyshnvj)Rx5=y7TutayZC(zh4 zi7g%Q@#y#^#rHt~#;^n_fOHGVC^|JH$f5G5;LF-X{fA0@LL#OnGKqg+oL-oIa4k5( z9;Y8sS%~+y{q>aaCDkq8>sW`Jhy9<8N#EN@y2Vz13J?`DH~}nLL$!b^DQqDn@^9XU%JLqSj*0#GA2PAi|@1?b|hR#++84n zJJa=d`|D2Kw@(-n_d>ma2ui_m$ppXv$>H7@MFIHu;^M-|aTHG0x|$DpYzRCxH^&}4 zh=84-NCDzL`Jpa>YZMSpizPQ@kC#70x`3ny{-YEi&8rKx3~nMT8OGEe{EU%9!wSOU zk(^p-`pL}FJqIyP`GeU7iHPFd#P$m@loJ1LM7&rLRD>wE*9L&NwyIzie}Qwhe^Lu& zw2FeXMUYN;6eE8Ak^)H;BOzVJ&43j#^V6`Yx}$>}nm~vt$^=sO`c=0F{=_Cut1UMG zk~&q0S{?U)?Us>dvsE;sH5^{NbA^_-tE;O@bbNP2!6=Y&q8@~Ae{i@)*zwAzCOo5fzDKNCOl|IP-(kQe|$#x^fVZ6G%a5GXOdm_pf}bm!5SzGpCFR zF(Zgr%EEi%&3;aRm{18YECOmrR`LV(m|&yRP*F;QNT$@ZuyCnR^lVUcJy7rk~R%Rr4W$W!a#QQZ!9BZ{R)3K92x5~l6Djru`&+pO{Ng>d3qIdl3?g%Ih5XD=-&SAE(5s@tn+3>)Vl)DK>{#`uKZ5>}d41F=B#|BD77bx0ygm@!<%d@v$Ry9=7asnn z?ImIhtcZLv%j6f~RlWd8wsZ-J0j$l(g~rb3%Mb7Fg^7uR0MLfrW(xq?KweYBk&3^D zO4&nenL0s1kl1p44&O5yl?}O{rL^U%5@yNDKF2z~50|4%3Zkb*X)HYm`7fuIG;u7h zT%V{<2!Ib1^m}nEs`6-b!>XPq#d7n^Ukx|+63+irBG2efr`$YY??fheJ0m+EZqGLxP0K|`j9EB27v|kBhT;J|=y%&B6E~~O?u>W|@tB~cVywiE zA3xHY4dPlGyfBN)Xba|{!6sO2AAc$9!2r{+VvrCMe3IcrXC2Yi zCv;--(8YBLN(&7Q9UK@SaL%-VXmi}xO*{ST(X0Iuw8Lpd`mhjQ+bP2 ze6?YwGrJ{BZL)59^_{EY7N*42sm z+q3G((d?R^TK6r$>NM*gZi$E@1TWqVV2HY!?|dEUv9MC<^`uq+`nl%-ncrI$lPVrK z7b4g67bUM2gb13NyfY8MhjtN$T5Z&%hx%|~XU^N4Mz)MO2Huo; zyzpY2<*DWRr1M{@YX_mJQZUWOp=0*If?Up)UKpfaVU%8sS~7lsN+JLD2@UfYnA-RR z4Y*!Azg>H6-g_V0`GH{c1)H`GhDY04x2R5&ibdA?F~d(Bd^|3dkgyjL(u!nvJMU|K z8(r@vy1Ru3qpm*+y0>j>8j}eK3f}B~`0!Cdj@M5n1ZdrUxvS&iSkQ_Yp=a1=d7NK= zBBxD!-h4e0oIt(NyT7D+w%i?&m5@=v=M zs_20eua-4ZnAQn!LY7x&r`MOYH5Qo#c^wb0Hs$!T8d&I~rVbPI*7a=TEgZhyN*gPG zVM8)}*9D65qj^`WH^!Igr8i9On0EUH(6@JC@1z2-Qh&}Hg zG&F=$<5g#UA0M&}UT*}^_}vU6r6e`(Jo!&E2$XSGwRAO`uKQC|RImY{$a8D|?sMn! zBOT7)ms)e}n2+DW2GMf*gAXS=@2#$<>2tYPRxV#l?$VErX$$@W-H>lADrWV&;_}uYp3+DK+-l_wbF53E(|ebc`cbo$B&UjA7E_81+2$wT@~U^+{^fo)L(x7k zV(N35ko~;A`O>RQ<@;=M_eAou#r0|+B|A1A^YhzQn(g>_;Ikhl4)Sn}NobHE2rQX1 z=4Z_4eE4ov#M% zpVp#H!DZ)z$K6pCHb-8dcJEPROa9pYQn|YXGQ@)iNedY&IX8ZX;D8`-aIAO%?^xSs zSKW`smS3$?s7M53@(Ct!3WGq%G?pBK8>IBJg;-)Bj|ht;=R z!5~bYw+qR}^R>sDKfRjVs*HOu(EB4VwtH8ltokPf-&?k+WGE=?KGNX%FRn5kuvqo? z`WSta8hOA8TBzXK#0Hl9boxrv_d90|g-3bU$zfjr)1_>#hjfT_Eg7;ejONXjzF>zy z8~0nMAns~uCQ~Ik)8HGRssVNjc6;|5A_5x+5(+~`eP77D(@;3TcJ3@w$}G?fHIC*d zwq0cM6|lcUXD;k)x?Tjh#VRseGe@R8iJ#!@^zNTG8CUSsbMJX^VL>MKXEE`@^8%2u zpVE`^d>@^ewYx27s!M8XuO4?N5zeo*@Lu^^z3wYtP43Q!M`PK_%O&9}44Rz}kAH+I z@Vk55|6>desM=Hi0YQ@gDj`BP(^hhE{X|V>AulJ-OZLmqS!6AVQ1m_WxD5( z&DK?)3zx;fV(;&nyLAMvZ%M=Qf!;4v7+;BjM@m;DGT{0%h?ff@0^>bANtLrlAh+>8`iO$0< zQuHpbd9`Y$ajyS|1rQW)9l=6^#|==Xh>s|MVugRmc4#==%_PxcvDO<>EjUPA20fKn`0;^b0Vb2FG2Kx|J2 zFYT9F1Iv3f`DnT57}S>?b6>t^!>04Sa$vL%+3;`u+&ec*o4Wa1SR$}a8nD0GMDB^h z+Hft`3l2T?Dm%^{*V$rXx_O>mw#5hTmS+t~X_5u9pLhx2~%4;0OF33b-lG?b+_%~)1T`cGQoZ#OyT6LI_5 zUwm&{4RDFbRqBv~iTn%^YkhjXQ;_h2{lk2o6jFv#nzzYdg;=}PzwFK$yqaBQZe=A< zzg<_pK5ooUGJ0K@;`c_3f2d^W(sq-n2edwMextMMAM|NRX##F-;m^EgSNr9`+-v>N z(+eEEVrdM_rfa0N0`bN&y=L?#dRfH91=>W~t42q(y@z{Er)X^+iCc#k+X15nIoIOa zT^sksCc~yaU0c5%3gi#kyQeF7AIaO=${AWQW=$9OZ{IlSXgeQo?0jFT1$o?$UVL9J z)ma)f8%_Xm(Ef}-Fsxq~rdsuY@U4tEhMgyb?iTJuS$$>iGA% zc~Ir5#Dtv!qBZED**@ZFuqKns%*+hnRgp%OVyO%B{lxl+ngo?;cI*Je+3+Fu``)F4 zgt+kGFjA>QR937h6>X_ZOI&|pME@}gP*Q+baO2rhd-*4h);J$-PUqV-pz}X7tCCQy zN=s`KNZ0R>YiCjO(ii$bHa0$fy5z}?`J-navop)C&7}aT|Ck_#h?}*xYKW%M)~bqr zR0-8j_^8GX&g^|x=X1ZHbt&arCjnU#AtK}tuP8^%j|FZ4xTrs-X&1J_f$PNs7`}E7 zfrr9g0`EA!e+`%&TJ?7xyf7u-7FaF15wBW+{%!3)W-4WJ=-GtIV;taliGZ$j2%9%I z?-qU8;lOUKTiO1vj?0bJOWqvnq(c;4!uB{aB5;2GC6#Yy)4h@tKYiXv-Y!U(^g7hb zid|b`I$Ud3tLb`0^8zs>lHdT+GPR)gcIv$8gM?tp0vr%D(8Rg1vr+(leZ9SCBEQVY z=i67>gkf?>z;=!*4JMKBh=er62>c0>TA|lc9Fc~-PCTg4A(_m<>2jgS+2d)UaXylI6vPANyszz#kXH0_#r!@k1Q~^Pok8jKGq9 zMsc~Cm-iRj(emRFr2M>B$D6MWiq%>{2E&RSra(C%JB+4|pYz@78DJBig$#wj`Wbag zPu4Uh0mgI%*9K`IP~O*G<(ND?5Ct4at?`{m=ew@?{mq@G9zWPBMtne^Az52P=`G*} z)G}(rKp6#s#Ndt85EG-xXY)f6hD{3MT!{u(5BOY#7hEzv5`N|X(jkOak$pU1tTc|T zlEFwI3^no=tx04`Jl!Yo>|mYS@i7ChDIRs`G(DKYh;GGE4~G?pk;Nlw;fhD}A5B@- zC$@GkQi!nPFS2TtzXvfBp~tI{{#vQqtWZ@YiTlWHgc&ZTS|B%eO_2s4e^a;KV8A#1C(D|M&6nyU0 zk%Bk=rP%--PgNSccdV&9AU_6F(0@r1CmCMQs|nesA2Td}fviSwJm$vx=E0mYt%_x* zFxS*y2o{Hw1`*EB1O0_w6>S}fH0p0LP@2!+j^(sVytkN^omm*iQh(qC7&OFYiq*zqk_t=%29&h;bhD}GPYLt4+gLO`<>547x7B6crkGm-UwK%EO;&ALY!gO zDSns>1q@x*fI4bb-Om`x?MWKS)XN7i`n1RH17ye_@f=EQ5LS5Koi{u3AAcC6AQ0;k zJQO}KSXY_=nMP!bjpS#C>($Doa(Wju4pm|(abMt|ecqSNZYvy{nq=e5Kv!1?X{;Cz z--IHW`}<{+&za9X9*o~su*hdts6S?8&_vrJFdsi})9^PlMi*Na1nOELA6cZ{AtJFdX<*wCQ<1kyQF{QiJ7?M5A8& zaA0gdPXeQuTV0ZS;r0}-O2*31KfmazQ!TA%?5pbi;h7i)WM~40g^~K9#eC>v+2K9{ z3lA0RrcZ-_6nW(QhENgrRyz3L)FhhMJS);BB`UtYesuSS(HzVmzzk1=poyuX`zg|F zPY<G5i?OQjmbII#=d9pSil(8O4@nisxVESK_@^C#*A+BJ(^p$ z)^JRV3hc@AcVdpTQbWZHrv!jY4}Wfjs^j>*TlN7BR&6T$a3Q>{gS6SAw)E1PAaFn$ z8ij^%A(sXXGm$4C%=U6B;EK4w@aNMxC~i6`WL)iPHw_AcoS0Qz&MSl%Yv$vtpV zd6r)<@J%1%!om^;qyiIc|2!KkCuwWZ&2}69{dEzlZ`0PL2WR&Ku?s;PK`G?CLP8A* za-whdNrQhYdB+OIOak1U&js7pMI$5?`~x#iiwuse_VolQm=5hw=Gt!exJDAb$<80P zdc!f-a30-;=^U2Z(vS)f1qGipB=$In9^3n)~&TV-_{?SI}-MKx%u3O1iuB}fpAAF zBl%`1vdF6j@|$vjk2)A&*#{O4;4nm{WJoz+g+M4kY;>U8dEMP82*4!|97eVPtUpj} zSotrLw`k}vIa3QAE$j0gp7q=BwABjXZd+vtTm@jEkU)D}$Jd9Ny~GJ{R>NSuf>t$y zauyj#>Cv`peg99iSu_I%oAuWaZQ`Jh#A5K?B-DzLJXJ#LB=%@LYqvK%y~x&H^_Ib( zY4`?R?TFqHfd34uHEV+Gi0V0`6OaGxeR!o+fk;*Tt4qlFfUS@@7)JlAdg}1*i*B=1 zwPuB%onn|2ES#TqIR(XH@L}t+y=G^YrtODH$a78 zN&Uq*zP8q!TvQMKAjORIivv+C!GKpwqM~Diq+pQYkr02_`j3qT24FW+)4CmsN*59Ebw@m%!9fFD?=~CLB33*hG?Ssezx9v+$Er7|5my z4vHslkKuN(4mLiO=Z2~2u$35~-338!9>8IE@BBnQ6-0@Vkt<{;kD3aZ#UW1@OtqUK zfD$kviRO2?bWT%C)!x>2$FCRFTdUqEA*l27^Pm09*jJh?Rb~rYf|ozPqZ*p(fO{H` zi%OUHI0Vz^15!5h$hLtW9>YTP!(*?PA-S+pSQLW7NVg1B&C{T&Wdka8&PnCSfeqT? zAgEznl+cirjJlc#7%>bHc^!pR8c`@lOag2M`@W(XCIQ?Z1H8)sVh|~a!puKs6&H#a z2LlrXmLyRTpRZ6~ovsps#wLi2VnE%TTDF5>ue%8og?{r}qQEQR@_FBp!vOy|Py`!9 zu^N<5ghxo2K>zovXlWe|Gw3KDS3vr9oCVh5+uhAgo>ofKt;@$|_id|clVy6T5WFBp zhGiuhFkuutDQuice-1=3HpRLz*v|$9o=_gdn&yX_H^tf|^{3!Ulnmrz9_rNP_!W+l z0`6f)coI=aG2O=u2TE)x0T+h~butx&pu7*`P?yC54n z1J4rg{F4G3*y)qm8}MGr&uX?wr@|9$R;B7`dP=j?R#{j6iaiEXRPB-1)2NTe8` z5Vx|HEda8tZgzPylYxPd7Gq*P=ix`?ugmO6pq{ONo<$Qkn+GZ`!-;<-_XRPbXKSAS zEIe-KN@iUvHisC^m51G|J#~yle=84dc5*RyRw0sJnM{W7wKQ3V=`yl0Gso$VA%AUX zXgme-i8wY64(GeqDwE$VRS+}JCl(c4__Ga4NUO^!Po3lFymg9>4#4}i_DPCAExpOt z4iVhoS`RQG<6r@z#DDS*9{?;enr>&-=C2Z18b=#c4!>1uCT6{&x~Uj=U03?1Q&hC{gJ{|E@x z;Mb7mPic;szyz~L+kA6>+u)D9LO~_dO0g;~oit}FB+d7WiMb`_b+yz-+aOGq>k5Do zE&SeA#MSuI=lvBU;F@bfTGrD4+3Z}}X(&pjwmH30Q_@ip zXVBs8X~?32nznVeorxChT*!{Ryw>jD(O-bdlCNA6V?0Pe_#@b>;qA7|p^yeOFR#>g zbg!@~=;@9GRWRMk)_T3wk@`n^mWR{%D;$U!bsOO3{c$+a9;#4Ct&qv%zP&KsFZVgC z?AV5PWlqt6e&=kxm8&xjiBMwk;_PvtCGP7U+p;H|_hG-dydVgXG^W1sbfa1{F5QEb z#myD5;Dl#meM-?Bk1_MG-xS2AX0SQ$L!TtGgF@NM+(ySJ5(lCY*U->l#r%{kb4SAW z(>*;#g8aPQ^Kk|U4#|KwNN8tBVf3~&zP~PEYph&%IKi@Lg-I+4IU@R$EE0lLKgHWC zQ}D}AgcMFwYuBjxgd6Y2bQt9C&ai7<;Sf%(EBqNlz9Z^SIm(q4_1F}V63Wf0NZ0dC zo>L|Lqi*)!zTVQ}7_dIS|`ZTTho0p@t%AlJ!qmL`VOg=)V3`iImr_np*E+daUoGZObTl;; zPg;ITg%g)Ec)Hrcg@%NNl3xVDwh7v&Xc6-`LDe>w76t9EEmqG?bzlq+w(e)ss0(rS zCw&8=T+jK52(slUk5{}LDcW@r)3&zT^|WtmWJhIXWmg;Qkb*@NvgI}jo2=<1>9~U+ z028dIi;bj9VbRC+R^M#2D=}9)6;WDx(v(qD)HPWqv-sR<()o=ItrWQ=X*u5#5d<#3ml0k&k5}r?y+*~UAaI!r=OY~pB z2-Bg#49)wB=dijLlADXbIkI4U8vn0XdHKhdo4L8Ui_hNexb<|1Jn%$8TXFb@XM8-J z+_SE=ofVAB>m9Me5azrLuIS8>4{wj|4TkZEk@l*dB&i|a+l~*Esa|$Zc{3W>U0oUs zfH+eKPgdgT>D!)Tn%)xzk)>^`&qKPcj&UA2pl(yG+Iw+UQSKO-w(DNfZUI6Bv*gdp zGI(a7BH<1c78dpgf3`2Xh~Mf)z{ne(3qQ*mca%*lcqnXH%Ir8EmDS34CkWg zKHBgvINTg1g%rxmqeUC|Tm(Ce(4ZP!pU@{Nf9QDTNg39}doPjSdSyXXd*6Qw<6C)5 zNO-o^lBJ_vhaaZ2I@Y(nrOD36&NgV`=RyTbibhTr9X?1NdUqCHa1`;`6JSNHt*sr7 z!)K9KwDK}>`v>60(}0s(@9x4nBlpZoe6tgFoCmK)tVS?m zm&qkXcKr9*LV4LM$Kn%U`*>oI3}D~1^RMgdeW;baf(0pIA-;=}J9yLufzoF*DKRuE z^-HW*=9ZddvM}*ee_q5-=XU3|xLx;M+%`X4%Qa{&rnYz=o4L3*k6{G?-N=LI9xsE< zOgIGeI%3;qmzUoKzhgc}3`k0Z!jp(4+yG0FzMuSlwzgeQa&Lyso6Bx~p0ai2_Q`$8 zciW-tyCdLqg@-a+y&KOy^3`<$pS#{#V>u4^TFb$Qt6T#&+>eQHs6q`Y4=0f*yGGL@ zDkDcHt&Zkbd*hjzr@&nI$e!(c5&yNdwYFx`?_0H-D=~&&7r1PB?EmSX{=z_Rg=`wQ8E5I zFfSW}e|~zlJo9X_cUTul6#zAjjYY@ccT@ZjdwWZik!hVinv6BPl&Qmo^02%9bV~x9 zXNQNvaId!QrJlcXo8vmpH{lfx-lYw@{`^uvV#>2YVaL@Oi%Nz})!k*tjAy5drPZDP zhETz2LU~{r_lfyGW8^al#R+>Hpqljq1mCWe)nRNbS16XHsj!tt;&Zn|7zG=d?LMOi z%FPwXApt57!6ILM1Z+Q}3vrw?#~F&ilaUD--F^0#Do~z|Bl5BDRHOm=wg94Pe0&_# zC6}b!tV{Dnjz2y=>?S^T?b^(Epv4dpsVag-yfP{5ik}C{<Fb|^8z!~{_sDd#F4OfOj!XcBl{(_~2ni(q;)H^}sh zlsvm#O4HbcpvD)=;%`me-@6qQ)5V7NDiJT{sM03WsxP}h#Bvp@=oT*jyBz303d0kF zut7vvWpG~5qNC#;Oh*%oCPkVk1-{`A<>T_>KcwS~lOp;Z#2c*%zEPmTn{*lF<_vmk z#qx3o{B&qeWlE-|$_^4Con`+q9K>!9{BkPO*yN;+Rq!x@SY1<^+8=_ApYUl^pIlNLg4259bQ~(%g?P%Kc&U;R&+*Cy8I73JeXnq<7MW>dkQ;W zr6_tPwB21kgo@;vX#MxDGc(Hc?`PRH0Gikm)oL20O$c(Zx|M8SPhl>hDNd9k{SL#Z zw)O<0Y0b1-nLrXVI2KKCsx)+)M&h5FChhzE_wQ{&@l06NNRWmHV3%fW~f$@cu8q}d5B6^qzdE?mxpqU zX<>v_SbFn<sVq5EIY*!F@- zE10>GhOdc$$)~(zh0wfxZ_!(|w?{2dPs4bzll3pq_ylSNcc`#JqTkj#Y`?pgD3@^e zXadZ|Hk+I;CS+KwM1w9w2ti;W9Qa(sU^!DAAuGzhrWm$AaAZe8m{@+8cw#g*JnHX~ z$c3uFY1Q-cm>P$R~q zaz*ZuCfx!)Ecw1k-%CtkM381uP~745->IRzhKi3xx^2F9>So`EyQgReSPf~gT<8gj zbo`*dj7qS=2Qb-&^b_B2W$qFgk&4ma{7!6Js-OvC#d^IDSUMNw&2x0-)Ux8b$ z_GTzPbG0#;0?&|%vFNM31jY+57N^as15v0FH+=*y9YJJH@o$E(B;_bX8hmN<53KU; zCqu8NtBsySaaGE-+uYE1$mqdnrmN_m7F-4ExirJ*l<;swn~fxUNKG=G6cE1N;5(Ov z;X;jQY`9~}i55IcwRN!9<%zppchl4fJnAtq$+`1ElsTxuc^7V%&x6?zF_52{aBPYY z==YMTj;z3*tk8t=23D@vaeSbTtLkUw^NA1Tr!yW9zob$oa;#{H?R~*llO(=a)Y$ov zTEoO1#QHj-je2icsjBe0rk}TUJc!CjUnTOUHUB0ktgZNs%%FZ>CWD9liYH+`3d7G6 zRuh-S(nGqdInn_71y!sa-6pLDgCuel#d@YSlfIalsfm2b`#SsGreN*6uEyYwM6)O+ zx+KFW>|in;m3l!9Zg@Y0f>pXMW#%f7d@E0n8bi$i&h>oKJo8$)!F{!%|8po&t1=Bv z(v7`aU3uvstp+^8nRe1ILCH#9Nr4AZa42{gxj;;^7T+(;b~Rbq>a#16B*70M7*GyG zqPU>Ibg`kFR2sgxf0t5EPUQ^Xvkb%I6diHV`~*q>eQqj3h5RY+&B#aA3OLciI9{0>0OG9be6I2e3zjj2`$zVO!oSW!c6B7I%`-_!lA z{@l^y8HDLv9C|Hoq#710{%JU*UnUD1ZPzg;%~STwZN*m`?d+_qTD&eq zI(CT8&$L!m7SUn9k0#r`{zwNAbE+{*laj|vc~~lu6k!h4&^CKr{OylS-UayyTwzNL z0unP+AL|>44s7gNuW!V~!BA^A6dMitUerL~fm9--C~6Y5HijN+QkgVy%~c#78B<_D z#M`rRG;wLF6{=&nkqEcvo$h(Dcs2oH_$xK0vW~Wum}x_Y=bf7lJ5#Ts41uDtr!t=b z2ILmETq8^f%3OMI9Ei+twIYXIk$@*W)8(;Pwalx+ZTpz%R3_&CarM<o<(_TFEA zZ>)E%xKvC@Mrt865D_zdz}N$UfsdzGw^0>sKRB^KM!ckf&WgzgU_iTSexiBfw7Y_{ zTi;&w!ESR|moyl3-1B(yloh`({QEbB8GT)LPIPG=9-fbaY@7nskRiQVAz@*|R$w%L zcz8s6s1v+XQ#8VM;Lu_x6^e{ie}qZgYD<+<68f<-nX#+mmvo*q6kk2KBWJre4JP<~ zfEa}F@)4&^${4;*ainA+Nk*mh3+YKrZtIZr*t|^$BZyr)TcOqrkfVtp=Yf%-`I<&R zK+=Fy5vri<&6s5HE~Pp{@XOHuhGnCVpI^bK#SaQm_Ly!Uq7ULl5x{HQFETT3?6i4J<_?rg*psUa7ot`9UKT zl<~U2cAiy;RxB;$>sK2w$NJos1D9pz?$8d&T2OB}?8@`x*R20Gq@8KZ|GO!kQd1bv*e=u@=^t}i2;Xe zz_HhIsZoFyXSzq`rAPohTe=R%TxDGzpcZm8+amBNVt8ceGLlJ)KF|mgQkV#6FhZH3 zqH$2@Pha!|--0vcQPag?+w`phIst)PC=`tytGxKF2G21X2=?vuvjFxy(_}3FAfj;Q z4*2#R>LwoU;BH3-#xyeM;MyDu@4D-h&xJ#|$ds9hLn*?b>P4iu+?&`*J z8V1!zgHiHxOeZskiz07NdT$rrqLntUnKy5hFahZ> z_h;gKC?^+hzFbU5+=sw7TkoY3CN(k!BiBhIuBb3%<`L90QyPZDgCfibI;$vI5`ge} z{D`>kwRn2<^B-F)qYBJpYFP*BGt<7$pnO_|Tnn=^$W!>L}UV)g&*GU1LXY z@7jHb{09jL*WRCXh_>f)MjaUA zVg*H%T-5?0?aZ@%rb*$CUb_9@1pJF!1{r#YMbgK|7^CP@1#}|xqK9)XQw2E@MeNg<*NFzUM*nYP0lPTlk^z<|!-q=Yfp%Hmy zGgWKb^U-ecvE{+`bJxSKn3}!AdbQ}hps={7vYjS>dZKHnMI7BsO--E~wG8D&qCf2+ zGHMag0$T{p@P_%ioA7SN;J3YYyc+Rnk5eWG7i`Wpe*wvRjQ20K%no-3lJW>2nUz%Ww#`rk*a+c4(lWdnTb^9}oE4y#PE;~m-Ff>Uc0n0$m37h-!4E2T)$9ZeyLk-t+jr+@~Qt2DMC~U!Zy2}fwcRo|nR2D^ zGk8^uBsMfHEybw7@p$Eq>1j{Gn$t^E;TP@FMSr)C@3{r;HPCWmqfPLb06Ejj4*<6o z&ra%x4a&0c?!5iWnLkNMo!S-pyL(Mur8O4|J#_G@9(j475*-Bc!}2vKT4)vcy|mQo zx-3Xj;v75z2i8V?*Qv`RR!_aj>-_9mx9#yQ%?-$0wy=ZCf#Ui+ZUqzx0c4IL`>*lvln$Y*G+mc zo|mg3-RgJ42d&aY6u3bYaqsgCBr#xU=;%X3qKi`rF99TD@ka|eF92Le!8|*$yu3W? zvb`U&MZuTcr$Mck4N6cL3!+gY`VN-P<#Wdf^bxr|KB&D}lTN)PxN zJQ;bdnm{9XvAz?RRC{%P5fTs}M$;pHw=S*H(ub=n zDD`gI$LY`A&$r=8Or4#%NIBMP`^9}rvD;O7y11$yIkrPW3BXUqp@&eLs)gdK?}g&1 zWuxJT7B*bp%?hx7*(|OlWio0Z#x)03(rlRnqvbIlLYG-~s)Qw+09@WG?bUSm>zd=o z9#K(@nWfqjP0~2Wa~O)mTu%#3`Yr{OaPmFH>6G)mc^d&wmdLUX(-+D}1P!~nTPeS_ z{wjE$ug^<_9)Nie-%-5lL=Rffr!0&)-3H+47@+SiiVoFfAI@ZKOIuH=r#=rn?OTy^ zCoOM^)FTwA{(XP2lux{ekgC%COitkoVG+PPIQgbRFQuU<>7+``d7&(6zLES)r9m1E z^wRq~Qu~N?YB|T6tKDCX7bKwlhp7?mt-fwaHv1!7Ovo#6JH6JJBroG`?ni>2bJ(SQ zqI1uSfdzhDfd%C$*z3GdN{mdz5 zf$06YXFf=Qie4cj39E)Y5Dg?Lgol}Q-Ms(#S(8b&{4W}7#Nv;pKOLZ6a4$MFNR12a z&Azwe3UWsgSJs29XAKK&#cIPD)zx{Nn>qcd*f+wrHqpmD;%o>Yn zr1b)I94|Ul=Zn2HQ!BMH7elz1N!+O4J$b?`FiV$=QPk z077!8o66?V|C}xK=7!(%)<{E|(ROl(UvY?xLn4zMDZZ_f?@F;M7_HsNh3WcgMD|Xm zxA=R^QZ~A^vb$2im{ra`nI`1*iO+6?w9GxX(rOUOPa~nJ@UX)7U>Ju ze(OlKeOSVvW=Re~U=OI+ zt`T+O!=h}O?AkEVD}}pqcGCvm-3Ap$)FVD~H!;{~c~D(#rjhB!Y)oovg<00{mWEuc zH*nmS%4+ivwSF1g7Ag0LgL;kmPvWAi>J!tP$x}c-m8FKZ#mVo!*cDKCe(DiyD&CdW zcH+xquZs3>`lUKhGX5v~j)47ptQea+)n0sjZl`LdC<|F&hx{WB($K=PYEpVe_c;wZ z!kVvO2UK{9!962c=`w$b{}Fbp6m+9l@TZ<|@r2fm=7kEA@XFuR`x0$4PI0V-u8fAg zjklzGv6117$m=dhfg*HnMKX?vn#L_uj+zA@FwpQ8QpA5*@WVS;hT8OR?x3O5uez zkAXCZIf&ir8B~);+8#E&vD@gqC;rH)M!Dwf&B)!>;dcxOghOsF z+^?eE0K@;c1fnYKTyX0FJ*nMH=>=kAa_BC&4l8g4?^c@x{kTE?xoUM~Qmxh5BUDLA?kzTfEpxo!cu(Dx*T><@*(RmCHkp@MZvupId6xR? zY)&2%X1(dFmYL)4CYH3+sVwE)y7JE`8~%Tc))m{1h^*adTE7K0Bl^(B zYqws9@rd%TE+8-;=|kn!;Q3=KBV+|8C{mgTZ>mpadRpq+*7p_N;L7E*?kqln;zVAw zTuBJ}RFTfOc}hORXkJv@8&G(osi>*1kUr9G043^vvKO*Nd%G}zk4)E-`F>xhb&%}+ zKrA=-Vo%@XYp0~(FFZ~j;`+X~#rN8_?zz#DWyTSeyH7PshIB%L30{W@vOLzN4iow% zkg7UZ%t5s_EAlFaw(VAf4%c>X-Tc_Vq_R@{J*b|K4J9_&N;9~h&M%ldGT0mX5>dMR z4);yn>HdeCeXk-d#8o0z=43!ch=$+?JGiK|6QinW`pB?3m*+THKCj&uuHhzX zVJ<=UZXN&c@jLyvLhg|?JE>9-aV+GkRk4nIV_Vy5f6h@)@ls?oQsU>ba)*Jr`Foqi zTCe#e);?U}`WxJ1?4QY3u+nSczqzE=PoAI2l~=UUHG&KN(W6UhC+EoUnB zf8Pf)t|FHtGL*Vlnzr)hOu|YA0BQ{&Q)_KwLX$8jR0#M@p`gL7{B^`Hs zV==<~hmQuHNR$HLqLB#1cB_r4)79E_?$+29!7I^bt1Q&ofYzSbYp=MaGQo%L+(I0+ zqh}WvtW?{+TAapHq5aAR)e*%>N6a(s zC8h-_bmk9*NiOY8a8J}ZB9pDQyBkmh3V#vQM>y|PN+H-k%+?9TuhugVsC>0>D0eay z9Zj?M#;#-St9Q&+-nB|yu6~{V%y%bQw|zgELhvP}Y84+zhEULi-0j_8ER&55JdC}} zUlr1lWqxliDJ9ME_->uhM^xD9Y;EuluAE}_k{+C@dd}YeD(A`&-L}63J^Qk6#CzA^ zvoDo+EE>cQC?4{8=yVS({!4Mm!ByyOi|bOu=M(7$*qGKDkBbwLUA@{^gRa)FcB5_` zF)IVUrdN+^q>N-0m$k8R4vb6Mc~UPzQ=F2j$>tI)S;=MXxTXk)A(D^r#m@J7=^>B6 z(7s48Gl+uJ_gLQ?)^n~uNsJ*nGIM4&EV?JVr++iOrNiz#AL$RwOS_ zEIwc@Y!8Y!tW8QXh`LA2-|JM)GMK!GXQR=?#d(~Ph^Yw1eU*_(kctWNGvZY+>}TN1 zmL&TIhO`@zwtpYLz%dO2!s%arN@D+f_~j`k2fkml?g+E5)A2by&ea~Fv{8l#S_``+ zJ1}ba=WqW`PUcS#y;F{k@w1e^k)r- zET~m7yI1lXwTF!-CWrzb%V>i6GH~nb0}3?fU%FV|#y!!|l`)MO<)fkrFhJjSSa1{V z)*ieWVvxee)!BN5WeS*4>YNgol@Bd)NXE;Rl=_V`K&k&6kALr59{(f5NB%EOu9Twf z2+ZY-3KBf7-J1Ost*th9)USB%_f=hwmIQ>H?nmygd4hX>wOyWa;4vjxU8_pm4-B;3 zov(I2ILf?QQLwf)VPuF++ANT`zdbvFv-x5((El9kkbc=>`rc`}T+Ax{%+!&?BzuS` zY1QN)gDy)fBQ%6F?-u2OLEd)Rm@~uQ0SLb#Y7gxJoll^m?DRBGFM!Kbz6TEArlT&> z6AXq2RlU7eJAP}(^{O3()n-N`Obt+L3xnMw-XcPRWxkMZk%AFVhh5Bo%X3#l{YC-D z^X<`?r9Ip77)_;y2<`HBZJ(!C$*UgV92c)l@A+Iu_)JM?*+_%Dug9}mDf|Ckjvi>3 zkYIeD)7&%cpQ*LpP5!T7bp`=9B*N4EXRv~3+{+~ZipHnOPI|aV^)+h* zN7%Z^L6VuIXtHdLaQ|!$IZ?} z{7(-;0-4Oam5Hcy;pBmZ`NgK1*5TN%BTp9_L;#c^I!3l+S9G+Hd8_Ldt-#T2we|ck zjdsPNs7tBsbo2rGM10?t}wgm^F88OSW1oG-o;$4NIh9sTS~-HDv&vzFxTg%-neFC5-_RmB<}wt4YnJf#XA z^*;Yyo}rg&>E`BUYU5AQUbAraQZO^^M% zHbg~C!QPiqiR*2si5HEkSxnp6p@@MzxA)c6f-xfF&!WW54&7~X8!AL8D7B%j?SA+I z58pA{sDQH0)9p^4sgC;&@$&%o{40+h&*3EXN}|ls_nI{sPASOAxT-{uc0%O~priI; zqDO>z6K!4>nt8F>?19gkMCOvKQvF2g>V$-yU0poAHo{FJ%FD}d$2v0Rg@x@Gr)DQV zU9D-VM{E5^&!3%L)O`2a=W~Iok`8;~&q$(C)D2Q#CsphQ2|(}m`xB{ZWjalAk#ypP z5j`GFO=F*SHqOzlBq$ly`k>2YR96dQODKgdS>!o%tATs9Bgt*xz0Zk>y$bi2joIKy|2clU_H8|CP|Mv_b9Qt>be@8?)B zF(^&|Z2l$(YWAYk#mxR3?mbg||1-1NCu(Z1#^!T|8K*KcyT1{LMA8($!^qsXFZ+{r ziz_l@`h&YV;Trx&33+be#uCGpV%nmc#l|D?KR~jTxJy@r*Wpipp7KG_44ynP=5^#^ zDknMxE$yA9X{Gg_U#Odh=3B#oQhgbCH=~^)F~iY>K6iZ&L|jCx?Uysh>AvnA9v<%R zd+wFhL9jRYaTBlbuEk0c)Rl@R-7kx9mCNtZl}%Tu-1a?Ci!}DlPBWZlKpLAwt&2Ow z!;%4WZ`Rn{>UMJ|J-K4>dRC84;fL)f!wUPe{oUZuSWx8!R$16jV~YZ|2X;%8v>9&q zf2spIP13J#7-Z|_7rJK3_As>V31WuVn@vx*M?1+S#6De@0b&(=o>$inOYLr?Dz&uY zPUoHD1tVFg{o&(9;bqT=$EvpTT)e)d)t}=3khrp1RQYugD~_BUM&d^=HV8QEA$n(+ z1242DDTR3k888{pbQsZeif-am2wN}9)pv1v27N;*{D?>>vn6wi5arNFyLZ>ku|ot8 zALf1PKFn-IOij5BwmEJn329SOP-pZ7wJeHd2HNEX)FPmwn}#ipPtyBFhBv|)K;TzG zV#y=q_77t|itgNq(gxS2D&?7&0P*^rsvLzjM7%;_r0 zz*b64-I+}_<;Mz5bnx_imdv>TG;Sz8v;45X;agb?3+09VLVer-dBrm2tJ?2tZwTK% zAr1rJtm;JxoMdXRd1Nmnl%C`zs1JGA=r9ZeI}IgqKYh=k-Y=amk9`NIJ-b`6{*m=W zYoZf6i}<0V%GLR$y-GNpb3=xi7>7)T@mt;iQ`};!*9|9I^u_r(XZm~Z%VO|~x z+r=5Wlc&nhUkJ!g2C8a%msBm~AKd^U*5&3Rp8gI~*8HsnWR1rQ$S}F`sO674{V#o& z)616}%#i%{(5=sJN#eE3^}y@#&hKZkQq3;( z+6(0gHyf%x?%o4Q4@8P9xRIaA(sA6PeR-9~Ak-B(9Wgg9mCQ zL7k9-2nTG0$1BiTHKSg%BAau}G!@%J9V44>!K(bXn*N$LmDjX(v``|RGl4vvo;%5= zE5;+!qEmfE%Lq-)Kn0u82m_FHu5K&YK)Eh`eR2#p8F`vmYq_XxbCG4l2gr>!Hni3K z?jRu{VFlmRIU!wKGsQ1nGh~Uo?1y6UTtCJCR?$|_)O2<(Qhh3$V6Mq}IlJ>H!<`4eEF!aBbL&|J ztPcA>EC6@T-`Us{8!m;ACHQw-Zf{eQ``ir}qNb00{&Z`ri6Vw%XfgV)Q>T zqDX>~wh< z*J&~Anf998nf8ow>#+CB3SsMwSY31gWPz*^w?%z7n}N}q1sn<6ov+_H>~#ptO`R>0 zh8!-mAa`rBTkn=MI0S`+l5m>U`4(zCjwVZz>ueSlk|GUg=?J=Y5xVMSuB_uyefx9O zH`ncx&+8#+mCaCmuAiJEv{?1c(q$tMKD#IZyx`b$MiVrqb#Lk!CCh*=-`kVXrlwl^ z9m@zG#LpY8mDYr?-rW8S3=ntJ@Y54KRY~br$NJH10!{RI&`@^Dm>#7(TUA?Y6(~Ih*V_Vf@r7Z1)Xmnp(ci?SFNTidrJte|98>^(K4~4vCNN;l5=|5w8*H0GCl>2Eu zYi@z^Obp&_^H(jSBEzkmG)*r+hW4X!hluN{u4O>jNs*TbQe4E13BOaHD0gS2=s?8$ zm%JG1O=DAs`Y7gx0s$I7Pg%Rz)G0~K+k+9wK-s$k!@F6rKK1#jqtfoTiRy$&5kx89 zoKSP^el1NdN8T<*Mk`|*C3HUStL;B!Dtw$h_|mWTtE(Gq`r7Z;<5i=n7m(N5ZE*wn zBR=k|=_8QQmE)Omu?MWLz1$AtV~}(+$c^bXJAVL0D$97!3@ERJWIKeIVowzS!Y^M=jgQ|0B0ZmNl)G=YX96kGGX`j0+*VjYp>NrZB^=-#kBA(-0q#ezB-??`O#64`$N@M2?61o>a4cM>VfeG_kTj} z52}~%_ZQgc9=#w6SA_;Q6SKVY*tjA`+T513U5?s>DUlLBaW}eOs%Um7Y^u&RebZ~k z<8{3xQm`~4Cv>~=K!P_upzW%=4S1fO%<&oRN!x8dyv4({C!UJ+@S*))`0cK6o4A*E z3Q#G}^x!qM*u>J3-9I$+_xs8A=C(h|TK;$}Uynr>2&GCR7LgK-&z_$g9C=q1R1ekV zg~7Aq*;JRBKWn!qEx3nVJR?25)MTRJ{`Dq)ee!hBE4XiU_>~79Ko2F~dLzdoW z`lZaaWWPq8F!^Qx^Y_@8vEo*cm=ikjDQ2*&C>#@y{(B0RC}t*%7!3E*9v<~|Q%9#i zehQCeS)H38B{oz-58RiTA;D|s1-T`mQ80osDbVS;OmVcpNsNpgTA{tf&@K$3(nrMk zjsT_rMpjEN^9st9&6Whw^_Q9;W}G69d!h;@0D+A`;9zSDa<~y;Fi}+?2u~KjP-rCY zOZ1u|j!{4;BpwzF0bo?8SnCmJHW$iTe~pb|#TYhH<1j2}EFh@}KgMskwwe;~lP8Tm zBI|K(8&Tu@@Zk;WDb>LI^1%b#|Omu7=qC(Jxao7FInQg?o60f8Vc*LkyU7~B(v!zVU+`0i!$O-grgVU&T%fk zAEzWyr6vd}ym%$U5R;jX*vSELO%X2?A2TK?0*RtLU0>fkeWu*JBpc*NJk`*wJB@VKtPK_AB$$-ehs~xh3)j*!On=bZ!Pdsh zn=Z-tz(V%c=a36jdY8xnbl$3lb_DY=*`BvTs?r0P(ZG00i`v47IQfNE?m^Eua^V8Ua`8VgV@Bh`9v88XQ?;P~adqU@+zNgGi!ALYhBQc_PRz96< zn=4EO%b@I^6TLUrad2{lR8CK~oor`g92B`hFqvfP?pL$ClZPMEcRaH%n>_NdjFHSj)7_{2TE z7MmO-H|x9`LduhwCUc1%@-OP`D*I@90Yjaa;7I#{-L&Wy%$R&a&fpF* zB~uBqp`$o%#p+v-;ca`bhoFPX+XLvpT<%Z9)iAHxz z-Ra0KjSy1Lda1kv)Q@`PU-FUvz6I~kD`kVcpy}gPGj`Ga5m$hLE-TdAM>}%>M+rF| ztiBsj{*qw=_2~pGjXF;>#1h37fBzsl!mztJc+v>Z}zdhH8 zpMq-jVjvy7Nxb+f694}6cNJVfb^d8)=h4>P=QHFk=dS$^Ve?Q80eQ@Cr@GB}4c)fz zmB~|opD2%!0mTmsZFk$qc)7|-)YS-=&hok+wKB3T3q}6+?)x*h{J$>*-fA8DaB6oa z5c|RiQkrC_`M)m})qJphn%`8|Z6(Z=#R7K|SMU;1Hj4m(W|e|8Xc@yWjt9yDWmDN?NN)y2sdK?!m+`x^qeE zUG;>5L&iHZpBgQBEJcu8TGiG_OlJXvo+xYzVsv@>1^B|Bfi_p4`vHt~R%p`(d zL;Ru2Yn?49xr@es4c_Gol!jU1P`=Wj5oipVI9IQ~IMB!{`C%rD>6_FU#l2 zc`?0*jF31M@LuRG(MKPH0Z+MqCnRqwv?uXVN74+j9uX+;w+t8u%n)JLP`ApA0bRG9 zn;D-5WXL<185tR!;pCy3xC6K)AgIp%N;OEN&CS;=j++!h5Rx@%4Vw-QOG? zEwl`foG?euFm~!nKjFRK^10vW>Po;$!ynfOL4g7pTiKRTQ_~NyxnrWDaOxRo#~}Ph z#Gz>PP&+w{hMQ}22nOac9cr~MzHeRiZu1BVcJ6WHs1YU8!OLHT(e5sB;#b1s3S*8u zZ>%vew0q=wut23X0L4hg@L(9YP&CIyMeyz3_tw@h?#=7%xJQiWf^_{R8Q=4#=*f%U zxudxLS;Bc!!q~I&*K{H4>wR_Jw}0sFCu8of_h+k98vxu>Q`_B1%>B{w-H@ueg~dRS z(tw$Ww8mY-wh2+u!3QqTM|p;ieXMq>${?LOiyF$%`g6i6$N;e$aV0n`9d4``XcQJm z#LpEGlyY6o3kkkZ05dcQ^-al3)*rGw*=n2F6^OKYOSV4T5&9H$OD-Yeyi3}5+wESo zaOSxGi^Z$mmrIk_s?bhs<+H_gsKgrEr9_vyHy5V`vF4qIjn{T~-gTs%^Ud?R$>-n* z13c7dnVoZq-r}xSUE^e#9o@=QfxOr92>L#bIh&jXD;xeTpf&n<{Kd0p2~R`)c-F=4 z*GfVj9vWLP;Ux*v7 zO$vj@y*>Kc>Uj|SRC{2ZsV7DZxd1Ppf(i*B(+yGAa~D)1NIEan#k4SUZnysRZO|vS zWp80HdV3uyV9r)F-;Qa6FDH4tuVyW>P%ElFyn+I#0mIuVTrLkl_DqIL{*g8`hz98}Ig0szC&u}P zzAhw2g3!17kynEJ6-*MF&p%8tshcvK(hYr?m?Arb-v2ikT}kRW<7W_B59m@I6QVxo z|2UiNS2A%sLpMI0Wq>ggCcPb2bBy8QflVvD91NI+1fnwHAVR;U`#$DOJ}7A{+lmrO z%he;xPf7L_0|V?5q@T{kfCre5nOK1{{%CviU*xet?8NQlIiXL9q{(2FdA+wwy0y~K zU=S!<)~*2#?f>X;K1QY6f;RQr2yIVEhf5-V1RVq6NBCy^9Sh}IDctG&MDDA z>M-%ubd(cg3(x%)sVP*~CgC5w>}9)3%6XU=Uu_T*Uues497%hPlvrrkLINowSYQmW zGP#o#9alxpEO2oQftB-TS$v|WpH5`EFdC+~{IC+s%-;Y19hf|jnH%FYyxWzbd$H8y zNS&>3X|iLLz)e~C3f>=FW3roNVNO&8W*B_ zFMiupj0aeAoWcy8?a|nE-l1nS=7sWg;P z<-%k&lTrFd^u6`>e_N;uRyHN#>$(W=xuFno@}Za_*;5Ab59|XaTm$>ON9Yq<1EPaH zEMxpvRgC?yW$Keb{@Lx&`mZ%v4F`557*?BDyI_vUz5QUBM8 zspn`Q^d1jk`HxXO&b*e-L`_~%1Ouw? z=|h`m-HpOxI*%;>wbQI<;83#BrJ;EyTQ8DSb;vqeC`)Y6@%0RPVYG8l7z=TjQ!Nrn z9QE+s1rc%50Kv||feN$wtc?(UCqU^+D__-mRG|JOCljC|JbkMS^$#rsB&C2I?djR9 z#tyTdmX?+gG=dvjQ07oUfoe|=clW>#R#wH?HL_=-X#_#_d;QkkpooC_pEtUp0Y-w0 zFjy{A)<{fD4D#GbQ-%SCz%9+Ou~Jpl=1A<{ifGW!E|;w(o0_BVCC3c{N&%*)MWf=K=78C)hR6P>1$bO@GZ_~NsnUuyKaq{*4eY-l0 z#5#_l2d@S2J9ka9u-^E7J`)lh;UvX%`%xoq5JE<415fGhuYN_j1SD2D;+r`_NOX1u z3>c;~ZR^_FB;pjlkMGcftw!*&PZjH}$uXr~X1r#NN1;%Z5B}Y>MS?06cIuHg+T$!t z)6ufiCxvvojd45b@jO$#3$fja@(WYlSV{LzhpSw>B&M|RT`4SP`l{eOcxzucO$nlH zmB}tY?vXUZp{jJ8D)*`&}N^pZMYkY*eOGelx?``X#ZY5;*<}nwVIZ~V4ptFh5Aj-?*=Kxwn*t%q^5%zI z&q|JL`K9C}4;eL>$V0axVg^*Il23tth*=iLmxNl8NZBJG&Q~q4t_@Z*Ma+c-0h8wb@q2#i-S{) za$3bIZ&CwB>~>~jPs0k>3d-M6-JY*WzJ%;`ubjCqUG~F=4zw#C-u{`-dKBa;zIawP zZKw$GIiC%Rx8P1-N6@RTF8!geS#8nKnh%p>lz+ItRA;U2;ZZW<;BK>C{mG&NAAjn= z1}&8HZB_rDr_K2nIz)^eYfo!a$~nWv28qZBVH!Md;y*(0qUBLCH;6!~yN)2`DP1zK z2e-f0J9hG@M=!gEOg}18>w9E7D&T$DBtQK4CgmCu>0utnzC@g8s09wH$4UyIr*{w( z-UVh24zn>eU0)YzdvH9*Sz))-d;$DupCPJ4w}|6UPaDHVjH818-~6%pM#)|^8G6`m zP0=m~i|Y{W>>M2K9J+6&T6kaYxfbB%f!=(f=eSBS^;*BF6GcVww56PGoNjGNyg1{F zbz3N?WcPRAqP;=XwZdx(&c?dW=sbDX_=GN;06evfzG8yI2ZAXQm_NP0QgkE>b}5@y zJ6R^A_$Jxp0cLwOrt0@Ewm)krDIt5sjip$OKeDUmk*LtD?U~>Ff`Uv&L_`pr@6>w^ zO~ZIFjZUl@Ct|maN!a(fdN+Dy2!wvgKei{JN}8QHQK{i&(S@MY^^LI3TAgZ3`oJU5 z*O$)Ms&}T=K0XrJ3>4i`M(RaS@hq&%yR*`Nmw{3lT3|l~JgtG=m*s`G>2l@}4ZFF- z?SB7OvK{HHhfB|Rg*ZSy+O{|O=f?YN_<>x?ijHx+&5o_tpNp+|Fa0Lb<0j?pZ#$H~_4@uU!%r{ZJu|>n z6erV^X=@**q8in5bLi388IPaUZ{5?oOQLdk)QqO_a3_ZX%8bs_A?DT6=_;?c78m8! z@=0Kv1rHLym@Z_$e`FKU!>Glctg=kgC?+P1hxWYc!@y>F`LG+8+|+v2&g$yw?5s|x z!s3sVg3P7X_cB}EpDY4?V|SsbJhFCZyPMrgZgKljU~l4Xx4Rf`@rv1b zeZ|k-8+CDyMBYD7+W<%r+B|RDXRGNR#wG%6s289CxSx(XH4pegxeYA#~%d+ z*2qH(AP2xN5MRe=Yg?A5zB4tnC;m}Yb)?*&sr+5TGckb&T@(&@lq19WO&-^OMinSF z!*a@=9(3zZ1)}}5cU*gQre!&d^s+u^W|vz;Zv=hmTUX|30nGpvokBlY8K-oY4IB7N z{tcHDU33!=lth%req(v8(#6E+=;8Lv*UQ- z*lVxanoQuOjQ{S_M+gY;PK^6BlsmF0sc;yQHYXKXW@b~1fY@tHw2HU;;n?rRlh#bT znC^C88#|ZN;ooIc!ojMU?sVSYt1H;jBI%wxtn^>0j(8v6_5MCi^wT5J*<%ylUq1-p;Y~B`*Cwdb6*xPZtCpp?8#1Q1V)|tr@XtncdVgI2aY4z7=A4`&OX@$P!t*%KT`<_;x5Uas)k#Xc%@^sSi>U0Ch z|NnT2WnS}t?J%XXw}~DbHK`8?WTti{3>II3n0bG-N4OvmG!ovo83J3qx^=Nfxw{t} z^>+f*PYQURwKm$$*KWno?(gn%hm)V}ySaQNYK%975#z1dh_k2iQby$Iwzx6A6ylc_ z0^N{h$}&%mr76j^o$sC*x^FyU%4At)Qmp-<^{|SSJ0g}mMqcl2pS8Aq1O*0dddghY zhwx3Fn%ND4481XOc} zOEh7>ZbRj(Kc#+tQ}6(}1sKR(-GZexgEo89KZ;7)uE);U=G32RW0z+D!jKwCXU+nl z3lF)!oPN75{&%BVsT7qaE0ZM*Z{l@>_?pb#9yRE_%?@Na4Kuyj%w$iME=m|C$KoS% zGqzu%Zp=vRl#iwrtD>sCM96E)KKE@yNHtWlxN@#BK9;RV75cg-PqlPfxcZ zYEJf&y+7_EU%d9(nM@9(IQt_NcxsAw3M;HN=0nquVwfE(a$R~%{k8Jg0B1}++lc*; zyEiwBdin?K8cXAS&!^o>1rghx<%6-3u8F7c;*$pp=r_wih3po*0X91pqgEoVV z?*;yIq6{g}A@v$RrpuR?>V5VQEgo5%ZWcVt;9l`2lO8zAV4c#IvF>2sW5t?Z%7H7=2bfF3gs6p+(AO1|#Af<|bNIx4QS>kT*`nsqvo${^SB2qbT zE#R>FzXPe~+xtlh<3)gvDghQ{R4Kuic>MPVjL%9UgzrU}a!x!!g>cZXr6T@ROdplUy zihPhPF5!wiG-yY-*!Hd~Pxp*xcmK@o;WliHO@=9R8K1(;9ie=GQ{Ja|A<NIo zB)$xo_OP;!7&bd!&U|7$4zXW8_6qMIN#-G>w612EU#zPssCKq8R%btL#|zbZ>h85W z6&IH@1rw!haJGGx+}Gs)1beA#{d~CK+8BT{fqeDmh0YPSgQT6&!w5uD7XW-9;-klC z%PXs{sL)_1eF^-VBc84xcntC7<+8Q3RbeKU8m5rRjUU$&vJe0E?4_fg(E{C|v~t-S zJ&6IURA-x(f=qAoRO|O;AFQo86Di5#K6+nv(g9pO=KyHQc3Z_6@GJ8R3OaZ(6(WM! z|F>c%<`|4*;>wp)J~gQI%8=AhST5bChpgf}$M5u`n#PlQLKO8Ymk185G{o5m&;8*< zNHq(`8~pWyg*zuzJu4fl!?hLI8%97DO4W98Do)AC38~*GsQzqYN;dHGWT!WyaS8&c zm$jKm{+&Cg`1HBR&i+0ZXF3>dWSrx#vl_yhLGs}O^&A!TRYGV5} zHuYt9apLIa?%-#7eEeH7TKM$h_<_g5+}POiGTq7NW5mkDdIFjr0|{x9i11}x@A<1a z|8T#Z!$bbO+l;vPCM~jTuNqQ1J7!cYEP4h9UkT)CZYT2iIA4hYt_(3|H$7PYILv@F zRuOu4J8~Zx>V2NuCduNv{?2)alW#dfF1Ie&4#UmOkMqA{2ie}v>1aIdw`CVYjxYQB z)lwkmo+S59b`qajD~cPxin#-f!=Q~9=8I|5E4+@=&ih7>n5dIahSo?FPAd&!flA)- z3!kO0X4ls=ZS-%^0d!C$q%RF@qyuc0TfNN9%lpg?YH@HdlR_6TAA_5e<#~ZzaQq`y zCLM{xhFbheKVJ$+#p!xDiK&o{Ws^7RTFJ!W{}grR z-)vw}91oIKCzMnzwFIektZ9Q&TQy>9XjMh5wWN&_jD2qzwMA?dTiT*Jqm={|EoqEp ztQ|rv6H*LHEhDCjs47Y)Kh(_q1Kv5`_ue`8ozMN=clU$ zEX+>*qb6N2hSPWocerl_WBY1+Zok0rJ*fg)lp1@)T-9rXY4Y;d73@W2oy+H^+dbp; zCwj9Y+s6kzjfY3>`9~_97ACoiz`m#yD)dX$DSVYW#kc7df1{U7;t#=epxzfaJK|oE z@C^QDshnS&Uak4|%{ej{GZKIyrqh880%BJ|FXD-uIjs=7&1D*v zSe|`iLoIc#16tL-LDZ(UjDBJvQhw0MLJAs?HOK-)L2*1DvnrOl!hNO+1E-2MQeDv+ zWcDhFfE!aj6vYk-R&&@amG$|Gi7;Uk3fAW_-zs^|$&41a`rG!M9F}f!$BvfOPWw`# zSCVX*HPq@>uHP7*?r^3gBFU&dZ3VOJ-n{EzMb8mMptA45!fX`G&clV(O8u=HA-1gZ zGH-%8LD!m&U_@M&jfwFyEurrerX$x_>sb9V;j3Y7tm49&^-SH{Hf7)p*Lh<&d`!?( zJw9GEjK`V-AT=&}H7-E`0Rb4F|Cy!5N4V#$wZYxb3YlcKSm%Ak2PR@R^cy-Ks;Sa5 zovW9O5tv&fHk@i}?G_T%*JI?JC%rd0TBX4%rLYcAqN(MAfw=JjqW@@+Tzx@H-8*Oi zE#s0*?1*R5!g$~GSCu+*>6>GJdK^$_%l8^m1fKP{D3&*YKu}w(b`$5i=#}Dx6gJyG zm%8d>UnIp1744x|2U`F0nPzJg?5;i8jNFL+-S(52VnDL}G~X%oIx%T5$B|V?CqOqn_&tmXIAw zI~YXH%+&N=J?BmPNU+Hv$%R5C{1fqyqIw>$v68jLGlBHtBtO-eY!Hn3__ilcG%G4Y{~mI2yLJIzEm+1;1IH-;aQ3Vh;f{bi zs*nd>kZRZa4%3;urV7~jV-oYdeM<}jO9urV^L9YHzcpbJ|M2vxcs#DF=WBjK6C_g`ca?%aoaHNPP z?dZ%3KwC6$hYp-98;m;^4+NzTT0y8J{tShFLl_vjSzaoB?*E)Q4gS(QP&fC64i zc%YADJ7!TO6zYdxK7YN5N`DNFt}{Iu*?E=qANNp)oRHLIbytKnpb{daIPY7f2K_;v zcM5be&_vPe16I7r8Xq-;m0!fU)7~Tv*Cu3ewUn$aRf1YK^|F%Q-J0BCxAp0RzxZtP z^|3(>x@j3E3mMF|G>Gu?(Z_x-C`^K3P0NWYxzko7Ao<|mFwo8EWL?-2Eh0y>gj6)S z?Mn*;rK04~4-qXa&L-hA!@mCaQ1WDuA}|K5IUfbyKGV~u6hpx>DxXrnKP@fJm3k0j zl|0_xZX2cD0j~c=F02@BOydVzWCtj}mmnNeamnFp-{Crsl%5Tk`Ad;x6grn$3{*=8 z(YeCz%<3u|%t)&#?w5i6pJSv9i$G}kS+9=^T^x&j*#6exiI9W#^Z#+1+2noMAw|n} zF2L&IgV(p;cBE{x(!>C%y~D%=fYW!O#fcmqIQ*4w5J>Wb?p+r~EJAEg_BekX{kwkC z9d!1?Ed;FsyT5c$PmW4)UZv9)4w1URkk9`n6bHIE5c+O=93Th>n@gw$q+i;<0}m6s literal 0 HcmV?d00001 diff --git a/docs/image/ClickHouse/gluten-debug-clion-toolchains.png b/docs/image/ClickHouse/gluten-debug-clion-toolchains.png new file mode 100644 index 0000000000000000000000000000000000000000..a90e474463cbd0b047897b80723d35b83fdd479a GIT binary patch literal 164977 zcmYJaWmwzI^F18g-L1GovEmNFinbIh4#A2RcXto&DPG*6xO)r5-L1I0J!$Xn@Bi>3 zTuC-Nv$L}^vuDmHLQO>u1CSm-BV_9h+Be=trT<)i?W zW90h)02M$1DEY}f>!{r#o?E%w)@*BM9>%}YbOyUrJWtSZ~G z_t~$H&^O-IUm4}TNYALZUtGNIue~bUmOPHXF0I)u)pHTpT^)S>;l26#)bwhL_V$Ti z{Bb`ZC$jV9u+#TXx8kqIyRAfQ&$q89oowe`ez$I~ z=PPb&G3qji1GFmY!}TFpPsg2)r<-zXueYyTYu`*FwI@=>v3J z(h^Q6O0s4{gbxf3&*jf9j&-mqh_3aVm^N*shlh&v9LS7|kZ+SgF zD#}dPIShpQo#j(vfoLE{f`o_rS&M4YW4B6wko=w!qg9t~-y9xyOfGY#>kWA11X&1} z_PcEhUX@!8?F-1Chsek@V8r6^LM^fSsFHXop)g5tf`%w`ay2mmf zBBhM5!~m2zT>xxiIYNH|i*?32v5U-$g(#QA*dy>{^Y&Mf@S_cSUQ zqwBGH&j7vf09gqm4at87cMS_iC5U->d2trYmbXGaRV_Sa@76OhkD2i%3pDQFXZXIgCIs0DehnW0@}~ zjimEwrb_&IJJHg`g(iLfP!Tv~7G-iI~`Kaa{Z6!g~XzwLxbv>~T-h09Y} z+;td!j}DjLAWfHA>AP&3oSdXqH0>yeX8mWyOxjHV{Hio+D^YKeB_JL^qthNF4-8x# zuI`OLT!_X%Nqb&gTrADV7!~zFP9Do>x*IO;;J>XbneDgb(?Uq2A^P}l)i;IJa~}*0 z-oy>6mwb95LO+h#=Fgk3+>lCJ=hVl{|GV~)AJ|`kNlv*!g!v05+lBEx`xzU1m^1Ig z;WyFyQ1NUs&}p-BvAd|h!uyrm;3w+;omZy0ooUhO%t!o9++#inDW zUWk@vtXYX+D{HhWqh>Z#ih7*PWhUxv*k8lp?;J!=3tx0bf07kL^c~u7W5!abTKp6ZgPQm|uGIEB zT!^%{q}s^A#5$)1K?*x@lr912G4(6NOBtqTZq(ZZ@p=ydoe9f@8ZiHMtqM)+xjc?< zm((?HF^XkGS&jhe%_r~PaYy}IY!ky@FXB(ddGj*O0{oDq3ofS^bZLh;&XNa3D_J<} zfVRD(yYFm1R*F7<5Y74Tn1p>xp^9{LV%E#RY(Rxdxv%}OA^lI9^zQpB z&cDMp5Z7dJ`cfr?6?wsHKJ;C}1*)~37Hk#e$7#R8AKP&fB|rAF_@grs0ZhPrsQ8D+g z(2+SxEM!?{gx!2CUV04In6=qYadTUzLtAnK(VzZ##ow7bq#P%VQRSRl5;Q*y?6OF0 z{EAHe_D|=(=J&?=bowuY{97~rPHY#MOWcE&R?@_e5vN=}s(%jgcXSg?>DT}FD`=pS z<7}aqcddMUfZb&tnM>@^h6_;*4mJIMa+|M7WZ;M88nCH|7jfW1->ru@TJZEavv7om zbLG_#6Z|I+YEqZbNUA=7JYblGf<}rC5ZoV*4aSoLHtP!=sh>ClgOQ~;g9g;IR_;nM zq@y_j3_Jr+C{fc_%P_D@?1?-aon#gaL`Qra4B1tzC z7AIH$AC-4@P9|^TsovGNX%!j9C;_MQu+eUfc_{?G(;}f{lp@<|4P(uR^@DS{Jg{5) z15gdrc_Blhd0gu6{dex@2Xxn+>jw^M)Bj8|avYdS0Sf~lJ+PKI=?QWADdMQfllI3P z8Wxt%w~N13H1P8BUSqfry=~lcy;vX6damY-*|`^ZTu;Qru}7zU?nl?5riZt*T@x7i z0Ew1>H%^B9w`WNrMGh(DbmRy72cR3di1T66NA(YCoCwERhbJeoKpI0^U)iAWQ1-z) zR30a}?oWMk(X+2H!61>y$?6>*enHj$DQN3I{G8e^Nc7YL3RR}%sj<95lYvqD78HJJ zx8Pxp!%Tihkl{n{ZF4uKSU!=8Wa_OPbAfv%xn`Lb5wCnS#oPtS|9oQ1*s?k*&}CW% zi*SPeBoa27*nh>bj2w}2=E%q~saSU>E)$lhKCWvc5?l`XywkGLT^tQl#03$efz{ql zah?4aC!wx(pVm{kt;X*dvp1;cSQMa#6kC}hbC#4oVTKh-=v3pkZueQG=eR|O_%(HNgq;Mq{*H`p*dO{j~&gMI&8mN=izs?CjpMveV~5lNcJ&n2U=GK?~XM-*F=p z{l8q_V$Wk@!!DRO%!`EPtHTJQ!W=2cB8w%jy3DPpXO@bIyWaf_VT}zXx1ud%Pt43L zr3u44K8+LBI$B`=DgE+rf9+t}s@D#Kj6%(6NrHZbWB3!f~BMyT>L)i&{|LvA$FPUx| zc3MhG%D}*Yda=rNzX@Q3mF2U=PnIf6ai?8$0`0}PX~|C{ACjzP<&n|hy1H~wL~OVL zI9$j(VHVxmwMG+Bxt24xIo_L(KRpd_!puJ=xrE=xS7I}y#t6X6TW2cTfnA}=`Hou7&7Hq7fa6Pr=A!4Z&rlJpDS*n2$!ZO5;!$bDG&HT?L0f2CQ7ncU7 zw|u`TuNFoOX;=-Ja=HizZmmK7Rj=*pXfNb?iC{=uyWH}BtibMi`(NIUG|8kisTkRWN3LTtt zDr{qI0ODAe02h6snct0;cqLn0+l^|c@Yk#FUHsLOBxUXMS1A#4<9bS?M$Msmm0j_{ z^5|#)LL&6^s1zc;r@Q>MEI)f|1!N#!x{|?ui}%scglMbSA(?!FJ0@RdIVTQ*!COdJ z&l(J_HZ@*(FXtwk9yYIF>x+CA4)#|ytZpCjlpk8~f*cl&kD{wQe{}y1n1yQqw0>4NK6qLRJOdnuHO_fc#=4W&IcUNgdY`B0Lg9Dk;hAM9v6% z0X$1Q1h)}vvs8u%Yc~OhYB( z-Tu}7lm`MG6P9q6sEg-hfkI8C9r~CqW<+1Q8%N%=lne=nloaB&#_VjWMv{81HC4{H z$&3L7^&*w;$1BG)+wv0RCg!=(C@{#>GCegpV{WV7bq$CR-$t~LqG_e!{={#( z5Hb4*t!Tv*-g^HWGO7g42WyuTK%PLMLER$4dy`7BB3+CoCRm&#+=wPM`}_ycwWRwlAFL4935rOTK#~JkXI$E_ZUIbdnPs#rSa+sF;~y!Ka$j=)G~Y z2l0FP8S;kua2*AQ8V=s7whKLyYrw5iNd2<50iNvVOC`}`Hric>cjxf>X89y2(gEg-)* z5ih(0gDpoFO6B0QqJP@{Ja+YE^?LtpN_-mjI`SAYQZhQ=jJ%vS4xSKJ!NZ7<7?>9A zyZ2w!02}RK_V=&Q>BP}O&L{&%+d<8jXYj`}Iy}XZ(Z#sl$&qEL{r7SfB5+5C7?&nc z_Oo`}fOZO%6cY!Bxa*OnF<*>-glUcTVL3^_Ph4#^S_>Y-@1Eab^O)vBiIF3ysTZ&J z$1qOYA=L!dnicLgzuHJD)D?SLdH9|zUi;%l72CC5?;&w-gt@#XCK=!O)t^317{2Z{ zby&=n>+n=W3JVK!_^s;37ZB9);p;THUgtHt$>mS5);Q1e4D7X8K){~#QO5pVkGH-z z&FZmk1-h?-Hh zF&p>e&5`GR4AwkrHlOua{#u)#l*@Jg+S7pe(+NTV+w!s!dGxE(h3AD> z+oeP?T5{jB+_2Lev69lfz6>=tt-ajmR#)$Oo_#ok)qDh_;7gCnsxpbajfoMpiF7on{q7JSVT~UmxZ=KUY7zSWQ_>YbFIF>+9+yOyS1}7%qsb0M97}j)Ml%oZ>yYFW$~Jmz0_o{TC@?;gC+KKc2&ae zVpd3Ryd?g+&VH@KHnXXZXAFjO_OtKJf<+eBis*Ca-H!d^Rz#U^RF3b%zTwj*QO7A< zC=-Vo=obz~TDqan{SLq3d%TH@M@nC#Uy(Oxd%m}sh66mC*cUk5eD#*AfsKKl^DQoi z5|cP=9;Yph7xEij?qX1F}raAh&#`6!~sZnp6j_t zsT?}QuIC^c1AQG+Md2MLD?I_rs;6X zad|7VUl$sqht}2xp8NWYjSViJ^?e$YZwe6;=#yPWXUl2DJFKRP@MYlOg7uI!ws}o) z8#j>@*(b#)1-yIQH!M2pD$RS66fl&F#|;f?+m2tZv4ZuvXP2^BvwaTRvB*8xD|kpt z*)0;_6sF%P{dj$8h`Dy@e0@ghkYA0CHRj=Kab74Lq|`Rqcu;>k>a>-#Y;Ip}% za9^RE$e@{;p-wzZwsFame@kYR`jZgFORb+Bb`~Y#MS5K5KI1vR4 zUSFIVDubK6`w!&i*!hlxJRezD%4RF{bx0fSJ07{1L@F8@Ix0sLZ;Ek;Ya?3-Smzd% zo_g7ZaT9QGOo*cT6UJNyH)TwG3GR@F{q)`%wB1z` z9q($|5A6-^5{om?Ob<1}*jN^R6^j5-oEHjQO5Nj+06`(U9(jgFw2H%;;fCxt_A`Q&3DM zcerP+ngIF9euT_>RtA;yo9QOX=bb8FCoyP}<%F^6o%(E{QjS>zK|h(uAh11CKebsk z{H*!B-PEa}u9NU*|0nuYvPM$jlo&!;_f=x?U(jHLLcvvqYaWrGk55+G7}je*#U>+O zn1wYE{$>;~mMt81zMxe9P8F6*VfIcNnJ(n*&5yiVHr$B%rlz>Hre|oDqATj`?5wRl zcyA$>Yfj6#qmlgl33iyv2HC8_r_P5EgKzTlM_%JD``f^L#tfOwnS@^4a}x0N?3G znVtKhLUHH|Av)ae`_b(4q))-UGczt)o=7Q;?3nLw?bODa?KgG=&1`Ea4)DaZ zFJx8iwU_Jn3zxg%_jilwx68u!TX1`etrsfv!Ac4qGYO4_k~wQ@9i4?P16U9b?}2ii zsSGSV5hrfK@lPppN@?!|zg}ETGsifTYz?YbYs1w1a5M0`AKJ)Qefh%(Jz}D@hg1Bk z?xw3F_99Q~aK92W-qRQpm1s{lM(c0x?7g_W?(G#TS>I1tP-cpH-K5weJ@5d5RJXI% zo#OSc>j!e3OS6r4m9w+9OO1u{076&$_WOZ=Ek+8RaQ;wwV*>`Str-57i}4rli98v| z$*Ov_F!oqlxCoW@O^;;OdY7>RpH!XJM}s;Ct4!waVd76Orc>rCJRTPTquHXiwwLVp znS56bb8FK!mUvcwvizrLD{6-me%DZYoM6WvYlqx)fsGTEvNZ19K=X6E|j zQRmA6E7C0`>d8y=?7nW34PlD1Zj*1DnY4m}LP6U@`l+w=XBW%K;Lxbz2PG9E5Dm2{ z-SdNWmACiZq%>BAUv-7cZuBfxBjwwe6d(<_{?gE6CmF3+g&2+zKDcV_SL#3Hbi{V- zYzR)K0HA($dgc@rs?8i6PIsst=k$ts>aAb);y^I?o#?HerLFDp{=RDmgvefGJX6T! z;CyQk26?%rH_T!z%j4R0s5mmi7tDn;)P12&`P3_`F{a zAWU}`Q31?fJKx&PpKa8weTIe!Pv^BUxJS&Q?=6*;HRbQfc`u;#7Oasw{92SOEq!4F zK?8wV;aeZ$aSMpR9;zi@a@$9|L zIelNRWRo`i$?xUKDlkZZRcv0|E7u1_YQwS*jidl|EkQ=^QJU3XTY|YZxdDTK9%nq? z0^&1^?0tdm9SNkNH{&*vpWmtMAsLtk?G#*0a;F{$ z*86dwWDB~TFDJW?_O&qMv5Y|kAMQ({ZhvKq+s+=XYTp)=I@DHOPVb|sya!01RTKy& zwz)TZjntFSmSM2V94doE|xB4SX!0Hx$h#0yGQl_@aUyX6!YmLoTX|Tz=)S6 zm#IWF91ig?tQ{NZeed~LB2o1A&7BO+aK|%=%((4(LorlE&ihV$9k;CvE z0EL)AdLy)@<-jotG?%05CIs;|eU6r2twgfXXYjj~23KQvIILdml)bUZ)YT-tW;UTW zeL8Vq>Vr=muS)A;2oLrFPZ4nl{)|il(rRvs>JxFesAP)WBZ)sGUNXjdUrg*KGOCy{ zp-T`0P5ZF^wNzy8g%S@B(qm&|Ge(Xa({65D_pn+HuyIHZksa zI6Aa8qF*d6&uo5h%+4(>@%!>%>uJF6pPZch`J-KJBq;{8lQG+=x0;fgu!I(~hN@V7 zcO8Czep5@!d?Lr8&ZbpQvZf<0q!g&O@AW!m{P00X_jXU*qo}lKw=#X|hFk!hSZZh< z+rLfLA8eOsjDY#AtGY+tm6J4!*PWdhM&~QlSW67Lfo$`{c6Shxq35#?A*$4Xl!$3L z@APjqi|7C~AAQqzq%yETsp<~Ma{FPOGtJ2m?H#qaSiF2lDU-b)xV2jHtP~fncHlFu zl*9eCt3M|RN-Xu8eMjWxCfnJD!J&3oWVj)LjUz5|kVn|GhiBAd`@GV zDYLUmd1deD7~{yJt_K%b&4?4IK1YJmNCtRjCA>rN@C3qXXlOFepI2JNNaTg68Aif6 zp5Y>%P!$6G1-@gTKdTH?j}OcsUgpO8tOp>i`n^6&B7n4&3ICi+pFD=Q=X3S7CjkU` zMr@YJjTKF(@24Fxt&mvLspx{-D7%tMDy(Nk*9)s`p3d4?ebRYZQ<627z-#(6BRY)K zza)j>19EmD(a}_*T^1tL9g8_Xmn## zN~?D0&bz-PeE04hxv=Ze(Lx;_gHjrwsVrZzUYv(^)6BnnZO`u}30DW*4pF?r42fdX(lw0v9Bb0mSWbWJu z92p7>jW(vBNpTOMiul99)0d-3-=M978ic^?Gn;< z40Z(2h6hxiPLfbZPS2`R7!lKyC{iY@{->C@NVvv zlb7(AD_&3!25iydC(R}PG>V`8RsbRop^#EyAjROw5#Q@>7*&kl7x1cWZ-0Jo?`N3c zRQPxnge5lLa-5$HIXNK| z%8J*T2zAD!NN#f@@MAkP)L`bS7mvF+J8RBNztwMGa^T#t+6qF-gcr*!3FD3Rb zC@y9M6Q8jaevJ5~%jbL6jU?P?^XR;1;S&K0y@;H3@Mg4Ht0AhivZR9>kEFNgp2tKNqZ0jyDO!mrO{p`9*MNUw8i&nCA4`NreRJWee*e7I~6 zA9()j)Z=repJ>|~8bTTxOFFuxByioqWPEshFIIFccjHnS2V^4Shfx^LgQ&I=0|z05 z3E9Z(dT+$vb+N;iiu-Mz+Q;W%USy`UEG=zA1_yaw#-(?mEI^V@uOm%^nlgZQ~pln z0=&Ukayxxqzx`6)I>JiSFm!5mcz>T%F{cL`pX0U$EU~sWTNLoqbJc4J?G7H=5GrW*AzK zyM?LfA?KYX_^$&5f=_g-ooCz3@FN4GWr`jz_M28c854);9v;M&>MV{&c39`i^;?{+ zM$#?cwV95F-tya`!8V|J#EHFBBp;R(l#6^xuuz2vIlbs*{DPhfO=mM(tRTGB1g_?|zdJ4w8A&&0zC zq8ixFHh~96bjKn{ookPG=tFob(mK7b7ej1WvrkiHj%F(;+w1muXu<^&DT*?^ZIMZa z3&wLqS3a(^*gf+TvFSiB%IBj01^rsCx%+b#P;E zukkCw>MX_?D10u4S05?0+g+cfSv4!A>L(L~eYdanorprdvx4JgFg=G&Zbj+;`isOLXRFj!Lp3Z45sk1wa1a# zfqU}UPh+rK@JdCM9lLR_+hGw(o>V&rWh2DrC#&;BKD+-x;2oi$W63NhnvX{DzHr!J zZ8r0YU9YjoF&stK`vK_j*)oXU#un{rUf5%q6`7w(W zIT(vO1&CgZazqOfMez!yKl{Ow>#z@X^~^&SRfvf9-S2?+)7>uC8pR(g!&*yUQ`>9|1q(k&NU%V&>C!Hz+3t9@ zzJca|Z*6W#HRH;d#u(SM4_O8t-Z;={`<~$^16%dFv)XDF=`YQvmmJn}XJv(< zI^eLop@qd`O}^ai$%745IxcF^lrFy=`JZE3U4i17NhZ zT(T722>JeLzg<-Zc|(rEPVJF1kxpzYWBTzz)gj*X3-phvP3)zWrAiomH%DuAW}_*= zSP*m|CbS|mUue%owTE7c;Ue&vlpHhx41CmbKy2yFA*XE;99k77I#9Ht?aGyJTa)Q< z-o3I*12`Odi8-Pq=IjBUQPFG;nVKR?IiSZGLx-)N88BD0yUj+|)?S>jX6LGX?=rUe z`iR4DS;f(ort+9Hw}OcBdqluT+L^qPK|T6u*IfHjdD0LwYtZ=sBF z!p&0u1XddiCRc_IJd8xDL4nrCTv4(n-_2gcubVTpKmJ|u+^7Uk-DrLJ8--_);JINf zgi{=|#imi8tGLoT?z+D1%Ld3Ld@YhaeB({iG|W@1Eavut9{0H2)t50~;RI#0C`KD? zZZj9FeZx6v?m(LBM;u&>uipcS~OJM|}xFUY#)X6SF_7A56 z+%O*)N!s*#Y)UH)Kzlj?st%I1n%9soH3giUlAauQY>=gJu6dCdmauVFeA?Z-Uih>3uV8v=5UDQF(*#xy zO25kM0nw}dqrU(ibGZ|%=vRv`0|<&Y^OV)nTo^1Ofb{`O&X z{8;GBLk;f!qws~i02>M{PY($yiU*wyxOPE6fI%6DiyCK&+ia3j^wDUFBhcV%HF zF&9_P6{I6Y&JQGkYL^gn+oqmZb2`y=Fl zGO^>!FlgS>vVjz`2#^ZU%jM_6F6Qzv2s;StOb0>oS=0cVkczw5wxkF^_yxcpKY1sI z7W6$h0Q)1|5jGC@kf>X8aTjNswSH*nzxw&2!T22xh$$<^)T5n7{`V4HBJc1XRW(Qj z%sD4Ou@iHp%v}q=aKen?2Kvb4w;cx)&yzRno9ig&V2UA)Y|@X_2q*0WrrKsoLzjeA z{VsVHbMz+)z{Nne#M?UX%byw-eNuwTSlSBLzjpWm-`tczIUR(aE#T53o`aaDBS+UP zf(_~ej1vb?DFG8=rjGXE}Det+r!m~3MlnsM;8 z?LKXs2>0p9>fh^MfM68M2dX*2aXvvo;gm!5rxHwAuCZVzhN((EEQ~^@%8~H zQ5JDPku)$~;&F(-t((k%?f0BRZ`cN9-P3-ke^xA^v7-(+JJ?@Om|um>k;@G6PzS&* zmD{$aj+zkJaVEo;<;-h#l8>VOsyEM4u`McVsx*e z7NCxk3J6nAx z;0WX($Lg=})Y>3)J)_00cgvwB;d;FFCjHtl!~?{5m|pbMTg8uVM=$P6jd9)}4CByb zdV!UIc->xY%T~E&OXluwT!D$PA|vuzRBvP7i)6L|M0P>1*|hDYr+J9K%}}*OOCU;z z00J8OWyE~jH?cq8A|-$-@o|sR3}jY_z;362O$-Tnv#3jD{W;5iNMxvg{q2Cezb&fwz{Sa#hlScdojbusS>m-IA zX3%n?x5jN*$kuopk15^{0(NYju+LUzSmwJD=b(A2b+iDvbSiU@}iI@SXFrW0~L%Z(#aqiz^ z#mlFsjBerlCh@*Tnn&oYMjT%TI zUUec{m>TW(BtH!ay@JKtSJF_@x2_d+Ww|=Jfi9e~a3AxC?Br?U@=p5siJ7 zGFbJY@Q)ok8F*-kz+(uaI|a?)KL8|MJ1W&qCu~0u zNB=apfO-F8VLG<*^i6sU<&g1;<$3qW>{-j{-5$6H?eUm#r(Y6rAo#AK9(yZSf6?#i z>recUBuu0Z`b$>*kKY#Qc(}1MJ5SZQ+o`6AQh-f}jIPcHi}4BdLy~`B#5q+#`2Rn0 zG@${WXbNqc{EvR9i^@S6Wcg7iv}rnr2jEImnaA%@pq!QT@I#=Ow*7ZAFhV0nB@MSzVC6I z1*7wpIh||#|9{n@Bcc4^UNszGGu#9(_kUwYY=i%OiYyz%62jyL`~3Xe9m?naKQ;KI zWa6u^3hV^zF)r}7gakfym|G@7X}y1!VCPo(HW2^#8kh4)%LF?=+^U}0H+lJ7^1<|~ z;<7c8RG(-U^r`GCNd2Mnmr$y@= zZ9YCP(o2J!`>dgHC?O!}iP~?edWf`VWOi!+p_r^Gl@^og_Ld3u2VZTK1><{deg=8J z=(1=(1D~f&sf``k9Nw{-!}(M^oB;5*&;UornCX5drJ z5cfL?4*f!FC$jJswO624X`rWR2-|(zhlF`8nk`%fpS6i&(KA^$>&g0_(6UT=`>u+c z_V1oJPBC+r_zfz#`ieHc4_~%XjUvWfcE@8P?G`24Z99|PMnS9H@<0Qtai++IZy*dJ zwKv*$;RJP$>BZkUQ~>FSU7R|`8AE{}wLaW#CZBHb9O`=4Q!vutWls(82zOV18@|JrC0MW!oN*Gnd2|fmP zPedUIpW^#P?o+aU`TPFt(2K0cCOsyMer6z9CTrWTY&c#qAf~}r62Yg}f`i>Qx36^} zYby(z+q1mpE1EP+vjLMiE4JtV_0kB5KS=Npo4up?caF3S0=plF z-t?rEV7%cu*omATooPWOzS=u8izBI`2?Rh<*dMO}4X5q51~fUi;E9vRb^r8eBKG?U ztXHEi`uBxs7EnlKoc=kQR@jmd2Jx#JnN9_-9!$B+kV%cH|^8qrXtLi9$c94cuo=@GG2xf1$e)6?@YF`WZHG+d}-Y zBwSmY#?gTZThjOC65=;Q0*}}qo0THOw4rZCSF#-I-A)iMGWXj~E;*&c2orSFz~?sL z^S9PG!T&=@y!#tNN5nBv2(gJEKaxsPd@|5`t*Wxz^PyE$^tDHE7*u-8uHljyQwspeNsuGBG`(^89I&^PfD(jC*s9*tc;w7f@i5-B zO54Cbd4cLVqmeO{p!sl~9@s5psT^AFkMWdpHi(5704jBjxw9eDI4$|t2wEhv&*p4~ zb#G@Ee6N0Q6yW5ly+}}Vdd%>;@a`C=l7?S*N2ScDX4U#`_7#;*x~=)})6-jR5$p!T zBQo&Z;S(dVa;9Vf*kp&0*^r}^)Rq>O;E_EzZ5D_E=@m}x^QZK<*>2#uv7&P_6*C|%;DEgw!z=IE zeRymnE|`%RtUc6=y7hqp4by532F*Irb`+ee9{x4QXcwsA%cSE&zf23HQ`(d_tB+0a z4f8#Z?R?NX(hQ0J2KeTfA?0Ah5HR1wR?Vkhsss7j_Ji^_SBLwn_@6=OEL72%D~}ZF zw#;nyTy~WqLmF(7KsvB|+gT5qs_6Cf)Y{`o6h~W@Jl%ztT&#P#$c4>P;3FINplVKQ zS)s>7&po42kWcE=`7D z@FLuwr=vZ@qnDOfV#z709xYuClvS~5*Vr8|Hq0dN!1GCGhoF*B!})i4xgC2y2F3ZH zvg@{A1kBi0s57PpOP@!>cwFv{8MHWm^|5uytGfNYs@ilq-*1JPgF{{JcSjZ{exq$j zWk7sxlC4_}o&dXD9BAGccCvWv{kZ5)dGb$}bq0P_8v)N6B2X5JYBpb8 z$>LplpTv8X!wk(T-IaN@+s8S+`)c;7;@R1hC6Gw+S=pD&*>YQ($|~>c>!t*~*+!dt zT5CdSZvFQ2$dOu&vZcHaZI`QqDeY=Z_>n+Q6SzWp*UjzS!5sH1C?!%))zD;YdBMd+ z!+>RjadSeRDfxYfRIWT7?jeOQ=rG!B*0f1Q2DFge?Jlxa;w`X&K#yZX#z!2~XBZS|tcv9wDU@*2yMak6Lzoeob~ z_@Ivd;pa`Ojs;4a6&fpY$IBOakEIPwnz2&q)L|#=9~@VeGa0s*vXh>#dfuw|! zR~{En>%OxRr{afAIH^!NxGu(4(87q^EZe*72ldLf+AsGPjRP%0sD z)c1EKKssCnhN*?0Ww00YI6zY_{wodlZ2Z-$zt6r(YbFHqEhG8?WtUTzt&uy(&P#iuiGNx8r%!mDbqRpS03{dnpjy`k&J_B zJEQY`SkXEQLI$Hno>&Z*XjVm0nJykvFZ5H6KH>lZs+4lGVD7a<^*vo6R6`3vZTGvG z;!hh`Z`{`d_}9D>)j(s0$RMH08QskY0Z<{K4%$%|FpDoOzd%Lkv)j_W)A({#bhx`>#4ZTIek z!uaXY@z>Ff#MELXtCtC$xD}^y(WSOR5D*7*JDV$F4X45e^UyquU)eYxXC0)1#A!o3IRX2s2Rt)-RY1lzrw7E8kLG?|An zj-%Oaz;x}w^~<;0hB*U4zh=B`ghLA{vBztmMlxZ4?)m#2T+_7uLn+-*-gmh;I|*^ z`G4mxXV22Mgl?{GV~+d)Vsn4|`0?UTew?ezwat?QPah`}(LDS|huhLpmLgS1%CIw~ zs3-ngm%E`3nOi<(h+vuLp-LymaUBmV>sYvIpy{BB;ej&W%qLaLQfTYG4JNUgfVz8P zt611`lR%@fB)4YpqmB+qgx7Am1$mxSbyD!WrIn05cVoHjel3Bm2ACX-(YBZRzoW#^?jb( zsRe|qEJs+gBQ0j{bZ<>ZONfzdq8|zDRbKiZwxyz4xI)OmkW|E=QdBTe; zp0(Gvc>uBks@#i7%i5-KfaFW}$#Q=zdYc;EQAo4^G{oGum^qE)8_Q9%#_Fju;gb^o z5%oCuxh;q>W%2Vznn?N^$)x||0<5X2lbm)sTx=`ku>d5Izb`Hx)L!64YJ zZj_&%9UsaI-f=!X?TK@cCp;fnh_}X)BTj$POiE2nAog!H^_c@pIcUnrAjA+;qAEd~ z&@5Dj+#!^Y#)tZ0NQcbYB`h?~>?<+rLsDJ?hjuefg!!(>#L}#sd;8|)AlUEg?6$_2 z=VPeqZ)Wlz({L?}ulV)Z{yycpJcgcb!}KQs{RUF!7dLu1xZX@W1@T89cz#P%vp~Mc zk1W<5WTGtjYAj+1qA}T}3PizJ!MVF7_h{S3Ej;R!1qQM@@`>l`S5!xdX(z(IMQ^31 zp?#za(Q4KlpbvyTR9+16C1vC2xHAy}leTe!k&z^g;P6ttPP0D{@)~2l-l_T}aMPpH z%7={p(&s61I~7wSKAHilEsOcdgebSO?cE+mzzMuS-Ld^TF0;};cJZWzB`R@PY=`Ew;>YAqE!QCae z28ZAf+}&kx_dsxWch_Kn;4rwmySuvw2p-(N$@Sdt_jA^ob=HyI-Mg!+YD-PamsaI) znhwp90IqyLX14U-Mv(!}AFJN;J7239wr~BXCUt6p0WFu0N!2gE1BBhw=y@qIHQ<2HG`HX->A!q{tpCSt{nDNLGTyKviD<(P~a$bav$!3JTU*<)hQaC;X>e8S| z(ojh(V=e^E5L{eW`5W}#?esIVAnhgQ9Il*^m1on;%*@i^05k=(; zaAR!3Qq&db3WNb9gjvVw)&?Jd=<`cTj*sYFHps<8V23MUu#JFnPcJ?(?h6T2nfCl? zFZ!s77{DN5CWwk`q)v^(bNWb-EFL|fHF`o6mAwP%od{h95o&n zV<5qp(^f^oy2NM_%>Z<%1(GYarbqv-@`)G&T`dr{2VQbo8eMTR65f=={lw_UZPGbi z5xhl8N?q8hPM$;~3X>;8K{TORxUqz`_UgFMqoBC|?NU}51d?Ae^zfL(53kiqf%(*X7L+c!-B=V&$^hq8bC;z{OX2= zXotJII_2dy6lU#hRsN0khSc!5m3k_M+knHpqCWCwId^w=-qgz2A3~2aV&~qdb+XzE zXXCJuRTsPG*Bw^&yftQ`RH0O09w*ypkzzD1-;37_!!BZ zOgC767{N!GAV~|J+*PiK@M6gTI2Thxxy2|}*Ei?SG)gYXD6Mc2^&e;KfRSMbz6@rv zG+6I_1Q%!51H#f?w2%eBz0|WQV_F zsPi|`eZw73#Z7?RKAb9XB~_R(B7*UI0RiNIF58!MUiXG#3?+rHE4b1ciTzTLM# z2ra9VvZbBbWnn_;yuvF+3hg)mL2^o<(A?^gB=(2`ZWJ~Zl^GdrgY8W59AyJW3Y!EZ zI*>Myu0sQvJR}xvSp&{+S=S-0wH6cXiebj~G%H1lqb&@pqiq!tP%NLp)>?(Qx{%fF z* z=;=0>+FK1pjT#c2VCET73%+n}b(RFWcOWv^|JBleY5hl47JnkPXwB1;4_@_W7gqF| z05yTp0;xET3d1(PvjT=?A&bh%zwRGr(|)UqSzp~AuLfe#aGNb3wx)$PpF$6+!tv^u zn3&4mCt@<{4Snwx&uo8kCYKD4GX%KUPKKnGYhJ$lZ1=cgk7ug^E3{g@w)QxQn4<^A z_z`*ec&7|o7UnOBd7OSnqKq%wuGKkB9+AP*?{!#CzoWTdFk_n5Vv2FNFYUy|p)$Px zch||eZhvP}WU!cuVB}9<1W5(mEN%`eb7Ty=*PO~)CD6z9;Y_G5E)_`l9gOCdQgumu zofeB}3a%q)-eQlZ4Xv%4{)l8mK<`5CNPC@;^EePSkDXO_+d0*G6ebWK6$g8Dy+5A? z!?-qQE*4$U)6$AKp~CNb^U(#gWuA_Wjgh~hh=4#05%QM#FVt(eYyW;o^JF{(;oCL4 zZAeBqxd)U$hCB_%CKyMQ2!j0>a2I{JxqJ=_58Mk|0lND()05dl`eQS2hP|Q#h^2=e z?#Q16#CMJIA9JR)g-34rFC)Xbg<7oqJY&GLq8BiotAVm`hS&~hd^0d$G3`k0>2*2$ zVHgR(C@i!q(puv&+fTUZM-xc9$4+6Ub@C%;aU7`ATx6m8dGU~JyE}#$IQ+E?Z+~TL zd{RAp&~5}Rrv+R?nlC%}N-LkX;ZnK(?(97sifdLk8rax;ih{@6DP~|25GWfPd+ehc zpT6YjMN~bPaT^{xgB0?;?#nnQy_DA$`3!;%{hT~H<{@9#X-UZJdvh&Cw~Y?x?SA=N zsi{)C(ZtDDKP33#qUwDsr}Xh$Gt2McohFN`%069LC@*I&L)}JAO^N~|f!v&8I!VO@ z>ceY1)qVTJY(5O^4S{lpRVI|&wzjd5A+N)x99Bn#)x#1(uFZyM`n|^TnC2uWZjs@~ zqEad6XD~NYo}l)mGuL-5_q%SV(^Fv^f{JNdJKxLmb3*btEr9Yw{AJOw^X_74GRs2o zCx_=)1DKil=Izeetmpl{YPn>)=F?hsD0+K0V=*KS5>e8$1ZUkkaPF5CLI3O0#AVRY zav9yZIf_j9XSJ2`=Wngfn>-m;&O|)-!#LWI00`A4GsaptPI_%Nh?X)XJ?5b(OSKCHM1lNo-&o(P?F{tNjfR*u|55difcEOnis@S*l1lBae6M#jgF@R z4;+H#U(jH#)f(s&+!8r>adpUpHPgS6!>3XVUiLPAMNX+AYN9F63=jSU2rqld!<3kO7H@O?CD_c^OSq_&($ARohioDTo)M5Ux`qMI~7%i38$=_ zR^e_vVc~F4mA`O~lX||E<-`@h3HhUmjHwagT1=HQq%eBos=x#zv|6R*<%Q(sC{Ix2 z599&zTj)~M=27px;eDg*Sr9R2e|0zW{k)iF4uuR*J_*fF5_oi|mzIⅆjw>F9b^B1JbC5`_bO)|Cm zcmJ13wc)r!>J^qyFKlSlf}e(@h9`08JWt0wRV;Y$zfjMeGVdSWDLf3BNHU z0D~95whqBa1QVy6MLQ~xgPsnO(F9i#0h*J5h0(^2bw24+({QkPZ3Qm;e%?{=<+rZM zfIy`hNr_&;L9IctFm=c%y7Yh=MS!HQC?Fs&h0u!tLU(UQGXfj7Yt}~&9SML(4vkS8 zaE$@=Og;*pWF<`BdXE*ptY8e!1uCndO5-<2GcirFP$iwc+PGcbz=R&dWtA)Z zjv%7`-HM-pOPiR!zpFczjB8{nN1**c>cA_8I7-?gXErDo zeFzJD{Jno)g0HBN*3~R__QjjIs;QhXO`}eqPn5l^NoBwNhu7Tjy?$)&6 zNdW;V$?<0omO^x3O;H2@HqE)-5!*7DivYlbBhw339(=092r$M;GrE|59hN22?=bRH zyFC<*(u$DN!soI_Yom?AMuMgyafz7JZm5`2#Gcy8*7q=ml8!&9Qylc4M+;t1S$#yl3MD!Eok)rd-D*KX1Xr) zR>;hI9&h-j@{-|e_~&`yINe++RdQ6+fUujckjqNa$J7G@2FNXP@pf8WAV5}b28deEFACX1w{5XC6Zu|~cI z#cq3E^sSl_mk2T^`$?=Yd8hTs5FrE_3rqr(#i4T`MWM~Z?h+u;f5njH6yuqw;7%?3 zaN)<*%eMZlu@+`U>PHVEU81I~SFaEp73bJ-Fp%$$r-;(Jo`t}M#znSCFh)TxqlbCQ z1g2`mB9^1oD6z_gkyRjKs&Xy;s&Kn=_IKI)8#%@PGWLCL9=kWL=9w>n+QjPXipsU* z#CD#saFhsyVuZALgaG0QTs|Bbi)f)N8Fjt`UnB~x$?ceB7$-ddDF%jJqR@(f7}du& z3PIxbVr&eZJPa`frUsr#e?rG)ZGgZl^%!na88H^gScXKa&tz7)3EN95ybBoPz@$SZ z{|Ewz!N4=2LQ1RH8hhXLw>x;E0J0o-5t)#;^D3aTkY33_)^w0bGwv4BE_hV1WQY_> zVtEm44A7z@iy8}o0pq0N#GvK4JQ9g_ zsvjR+h~%G!bc`J*VO~T>D)wQ(U{AEmive?nPKM2X9lr!uaH&p);_{M)mti z^m0{W*GoGZ+0(;SskY(;tEApBIbiku;OyzOK!)? zGms%oprsI8eCpqxfA9&m%`6ZkcqOjP;V@`^(ajIpd_nbZIO~}+M-yms6*K40lzsTg z>*#ZN@iRl1L-p-6_ucUxh^_uMM~TYKJ3Jz=P2H z-=1Bp>{~8Z@PY~7&nax#Cr&w5WIDY41)=N_s3BeebhgkS5$y6_`U6)ysXxE6ggu0Z zQp$++v(ad^Eu+4;jIKMAEQ(N0(g-`1i_Lf|YEsQ|@?pir-QB5z&9I~tq%5xdjgqte zN99kp{XnY@ido0QKf4d!?d?oTI&CIm{SDt{Z)8J%{ONGI*n!*J+#KiX)FO!`dHUPk z_l(amVOZz0)j>z<-(-4Asnl#>*Wu;DE&qM!hK71F-4JNUzwT+heW{ySRXowaBd0O_ zlaG%e-o>pcm*-ug_^uGctb;C^WNnVcR<@x-6hp?%fXDrJrb(^|2X`SZtL^x18eVzf3~-1PZ8t?ZlL5IxPE zHHFmp%{YjrpJWEJ+VxTaXNa3X&W5IvJnI3@rcK9LG-Og2wgQ>0H> zHKDEt9-o{t4>ixq_p3hx7)T8&_TWna57?VLFmtvkcE5$_?1uo&Z@z! zdq*Y^_|!~uj2aMF|0P8Qh>mkAHD$@@5`?d=JJpai`J|T)A6|)bFFZok3q!lWcWi>_^tclqhxbrcvtLzw^|j+) z+mPbv4ncK}l=stK?>3h()81aOahK!CPzV}tPTcbN<>kKfwezycu-j|r3PH_fp1!oT z=u~OeM*Vg7M=gS_zZvzI`nH$dFZhGh{Lhc3O5fMpz)MTNT`-r= z@`#dG6AXK=b#t28+8f{al}wO+VZI=f&-h_Xq!b=8vNnd@*d;wD)^-0%m*@ zbvy`k90w!M?RPN-5)Y^MAebNbKfyjA{h-6>`-`>NrBuCU>p|yf{<~Ylu8c8Zpi-`& zuhBFr1l4?C(l@(ro|m0yY$shA^Tl?n9bMILr@f{nbw=giIammU1p*KtdQs7^oeoA4 z_g@}B*7?jRmJp8rYK|{rBHaZvlLVU%4_HTn*tTAkjeb^X&{R{@F?N3^R?ufIr)gV} z#r5)d9Xu-7?KIaXb%%u*I$JhrJ$k^^2M*$%ABho%+QGV)g#TC+G&2R^o1`eUAt`2A4PUN@usK5))<9SA^ zyMZMYg}vIXlf+A_Kz>$tAaTp(L`tb4K`Fd&jL&9&*dse)cygieW&O5=KKA|NWYg4b1$TtV zkcHr1{hydW@KASp%wg{S)Uc0Mo}gsSA-neCZA}G*y@xT8I?WJ^@wx zBow@A3fCO^=qE7!Zsl@cJ@-{Uj#V zC)R}EFuc@bvF~j-Nj{kgWD#~%($u{zLP7vVYc30Pn}ffDVdGsU*j}wD+Cm{yKx)c0 zO;f$n%j)_^yO5yUV?wB#tD2kn7~c9f{${=1w`k={*;G(-?N zF!demA>!ZEW~|!LVg7V;2-mgYc)GG1Fy{6d>(=mXs{vn4cgh{GSMtouw^olmsTw1^ zsZT8Ma~aUktQa-Qm$9K2k68ZYXk)iXX zcfUkcKn)BZp8F0I0s~CcIqU6)g2PdLm*y4*#d^ir&1!n^R7yx%6a8(5ow|vQ|7Ee_7Tu8fUAB zRgjoJV=A01_0!WOS!pj0XxEqIe61}xpMdBgvN`+i@po4njnP_(s4fEid&1W&TY;oB z@P(OdNDPR}a@Mwqscvj<^p4gd1FO_FfRKIb#36{`j4xb@i&D9SJ}@l z;n#iRQz2jvJsd3R3Z2H&o!($jo5`Uy@Z0zAL;@$5=tH*9t>3hItwW7=Q3#u53^aT0 z-xC=4h`8k(FTz-{-7TnxH<84oC4k+pr#%z+$+>(ip$gmuSl0kdz#1gU0&ElN@EhJUB|*gqx2l6-f6T} zm$%AeqB9VfBiGfye>DKw=O1QqS8TKyV-zBP{-w@O097Z?1@d)N5dFHoU3NuLt5rui zJMYfTq9I5Y^VU$#YwmGWtpt?W7ZOU=EXw6 zI$ab8W{{CQn`VAKdwN3%&C^N+cXul1@!$HSThGs;JX2K^J(G%r>+#188YL<^>bNj( zxoow7wSIyOY^8BzYPiuZ$t*CiWo!zfCK+2CI}#Ka%0>y-2~pn+<)8a!DnnDBbHt|; zgTqO5I#2U}3=qPHGkId2qvBH-W)-X=c2T0xfhx3d&>{&P-J@TvX$Pg%x@zTN!a)*5 zaqV&WdU&R$HM2DbvS;wm(b=l|T!#y^A>Aly3Jig=A-Y}6B?a3QPF$hJ+mtqVp`{+) zf&iKUE3TTO&2#Vfzr#XX;9(0iei%?xp}+95>+$V~9nRhFuLv4Y@Zsn1?E^cM{8P)W zagQ?vr1&ol>>;Rhb&e6c?f2bQq{_4o7As&ecj6%%Lw1&T`B4G(rqa=!53?lO0ZKxB zE+zm+^cPt5q_0hKdC{U6(0D&jOv-84maxkPgB4`P%RoC0l!WeqTtrFE@!oA`Dv9^) z+D|6=;(uCeYYfn_z)eitjBx@(0Fop+Rv_=c#Q!plA#0(jnyN?NEM5I4<-vi<={J~O z#MH7VYj@B7jjm!F0QwiM5EW9mo2-MIO5}{TPjSMT<3ID8KCa%;<~B8@DGT#Q?h9k2 z*wukf#z&#z*a5HxGmA@eU+qWvnQSuIoUOerovD*qNzgJZut_;$4tajh_tI2?3XO2I zkR0%e(OY8@5R*32V+F`iFp#MBS&_>V&iPz+kj`81fQGw?QQPY>1{EHaAdg+yh}1qk z@&OQ(F;Xhv@8_g&G$OrPYpMWBI5J02 zu2N{1)4!A=Wd^0hO>xBN%c?5jyk8!CXIEEOFM$sa4?_@uCD<}`P@7%812ljVe*RXu zrK6X}HU9F&phGm0hXx^V9CiQSGJYWpwN$wn1eGZKbhugRx=%wQfeZ+pjA}e9p3?K_ zda|}x3I~-{=dRAV=?~s2sP`gr(U;|byWBQo4NXf`4471rNV3pI=z^s4Z_!r89UXOz zWw}hOK@r7)PH{(Yve3bLAXQ29*)E&OW6Ll4oUZqEqj$jJ2Y=X6$XasKRtQ~WbFf?x zC~vzNvvj5zDd1sXtFK_cjDrY?_shyk(<}}TV$Pu~OdW4k-5xI5P+MbbWBpU(8}v9j zvof2xM7wks6$r!y@NWF39ruH| zvV1I`3#JI-NivV*;Ca)>wgxt1%V5?0&ug&)nx#+Ih6nYjiZ+tmM%4e~0(9J5)!ts; z)RDgZ_U^Ci-%z^xb(S1$_*3mS*zmk$BHM;XM_;Y`v3F37;MCpk_5JQ%e5gMXi>9f< zAh~wV<4JbF!92tL?*fk@mHbjpbzE|!0ggedTYuz!_{(B>Oa@sIz-Gs=#bGycl0nmc zZEax#s0Jiq{|;S~G-_6?@KeXoa61d0cdZ?F+VEBQmWmMiB*wq&grIhmdAr464x9dMj%v>gQ{8{ptPu&Wg!IS_BrdZmEALZ()q-=+E?7x$b=EElw>6z zf$Vc)`^}D@)AFR?o4usBw>$rZ%IC3~JmQcQyh&&QO&#Xkj7|@^+;7q@4VvjLRPJ2d z`pl==x2JLeMgphTYHL1?_PcwWYu**j=h<+iI$YvTN#nvn0kgS^u6kW7&pk<#6y^c~ zqoKFBL**tk-Mr?X#Ux>41;VVwogkAj%gwP_?uxx%GMNo^YpevwbAEr1^Y+UQ8Ypru zDn^2o)y+CT0jDI5dzX^2#F_W#H{!`wVJme!lzi?)VLM666w=GsS)AuHwIF0;#$v4S z{dLJN7_R96J#zlvI9R+{F^9*$y*KrK7WokXxHR+d@9B_a-YxCC$-Prx`hJ`+5pBM` zTKM7G5ypUc)PYu05N^NbV_@^zKx1rP6617L$*&8m@zA17x#j#aul z_;tvHJy;&lXNg)jjL2k#kSq+fvH<_(i_S6^xBXM|Ebc0lI$ul0Tp_Q$Bt#q-oG$;# z@4=gs=_!NLg)e7E)7~ekLa2;X-rkv2dK=fX-e8j-viJeNM)&yQBnDEQCa=k`o@qUP z_nUzeqzb`aM1G_KqXZXsy%(1kDcxme!=~?L)!7$HIb8`g12H>~vtoeb$w5%br-lua)SdDG8qCos-OnvB{Nt8w>oa9PzH zQnQ4M21-x`WREfO>T0YqFqs&KX+!(Ymdl1+K1p>5LF8jY4MHZ~D0L*{#c!-K_|VWW zbvb`?dD@ua|0E4Yo3PF=Zr1uwhrj8c=QznHtNisMU&^UNbE;RLJfykm6 zXUsC86aPZXhIrfJd%{lvZ@1C-FZyjO%gGfLjHE9*9OW(#_efHV*OhpTlG|rTr$isM z($cp=NRkP8KKp8356*f*Oeqm~dos#uDqM4&mA``IlOjY(RbUX($#b3okWm2nnhFXQ zGEM>jJS}WQNyh8t)dG#`Yy?}`+LZy+uRL?;F?zBv1yG>>s1*S){<7CqFzy6UAUha;mfVZ9H-d<$bC5_N)Cu9;sGt!H zULzE-X3`)Wb8KlQ2C8?DhghT(u_=6~okxbGSPl)4JM4u!FV24ss zJf|1{t!`|LCwe6?>0rFbNu+gzIS1jXGUx$ExSft^mR*xP%iBeTt|C>po%CtzHdGrf zpts75^37nPG6i3bY!~8FlMmj)o^Le%3JI*c2En-F=B4}oC87ni8hrsC3J^G!LGL66 z9Te7CL8S!D6UFwDa9ml|xdAA5$+sY3gwB6gPT%+0xj>XVvM~xhqKC%9f8!d=p@_gA zk)PU?Ab=kdA(Iv%lNEVlUXg5!BNfA_{568fQcO76LWvXM*#nth_owe zh9gdb5sAoyAMxMJRw~NGObhL;XQt5$YTL>RG;=i#pWu#2s*ZeBfJkdrGY@w|Ntvs=Xd?lbCNG6PA~v%)3A_^jnYncPI}Faob4^bzD0x3_mf?T2UATo7nIdCcG24 z%~)Az6WXxI$SB&V1Vp16F^R2Hp%LL^LYjjIf^F?hN^BCz(eM~zK8fXnq4u=X3C}}EuH_f|Z#mIgQQsFw-jmS|8p{SWY~T;Msexl`IvPlAv-J@P=i zB?(9vs_GwCtSb@cG|0^}2Wh>qhE<6+gkF-AD92GYEj+y^=juV|i6HDhsDsgp#(_yC z7-8)N9ZzOfPQUVfhXvEGlQWk)2`#vu#*#Ch|YjP{c8_l57CKAj9EalM7$q@{d^h4mD#tFsSzEi;LlBL!KDsHolu52`ilYOc2( zS`BxKDt>&Cttej5R}xS1)^>0tQ>9SVQbk9M)Pc4Zq5kxT#jv7<9zAuifZ6xq35;2Q z7~_ffd=9!_=Y`N}13>h$0s*Z7N>Mm+xO$yy$mZ0U+|HmYW3L%u;=y<-T-MmyS470`cu$MP8(RRsQK z`}NL{-7MuHr|0!e=oM;K*olhVq*&-_mmyxLg9owEub6$-_u_BaX0G4Dic8~m3PV18 zup%z8LZr+&mhMuOZtLT6w^u=B z*a_S|%)_0bs_)TL!_w2v`!bg!ZqIwZ_FL&Uja4c>u?IQDuZuZ?w$$wL)kuVDX$^b@ zh~$o|tN#I7m!iS_YN$Bo@uf~E_#lW;psF<$1|SvhBc?E!SY((LRcxqcg|y&;^Zr>b z=3&Uj06*_QTCAfM?al57W(WfGy}@%Y8t!DS?{m-AKf*kK$)N7hds+yCedP^oC*^$~ z6ob4J77`33haCvvq1-_T#7MxUlj_U%fu2Om``X+c1S>95A&PF*0zCyti52bJ`SQdP zYlnW+qG=2x2A_s~Qo&4NZeF{06@h4x0I|BanWEhYV)Uwf8ngF%Lb9mJ{p#7@2CRN7 z>3$g`?B2S5>ScsMRDzI17ZgWS;xOT_zY65BR3d&k9cNaTK?qO*28c$-%mx_wL`8te zNdx-ojkmr4m4YL5(3*bVQAt7-;Euo)f5A5%*mX!L^r!)g#4`BYve#{(9pg&>FI5>C z8Rhsci@%ff^4)hWId1Sa&k~2_eW&9tVl8w&p7gxWC>f)3{$1`1c2oBZkWRQYffVZ# zj|5@J8CXfOn?RV+S8i9S!`grFnzu9x11_KV*f5?nO;&vs?gFHqGpRr1q^$-Vs zXA~2^pmyDi$$E+Tuy)U_U4@q zyGL)HvYDSeSWNxPtK+im_ThILP{2jpZM+sH+QF9p8%f7ST3U9}k)uCjUdu z+Tp1HvQY0R{X)dutR@ozM*-LmTrPFN>q4-c7kG6mDJ|5^qa4m>ncM5u&=63J;Q zo9hz&H$h>vO8suP&C|@WmfDQ_Glh0AmD>HCzP9N=h=!$cbE`kMZjJ?x*!_--l4kGaC@nla<^Q0fHzfOg#ef z9@`Ac*Zt*U(T1zSu<^)i-k#KZFL9V;`+iHe+s>x-b)`V}{ia0dY5ODoV`DYd2nCgR zKo)ld;l=N?GzNoM?n z1XOpw-0JjwQN;O-Dd2uE$J2$-1;Yh5tiWafmGNI~b7ykbKYhsiGCJ6TXItP{zZ`0| z!mrThkH{gP@xB+$yfApMQOo>swLAPR77iP0W(~zA*Ymzi1`FhcwOJbl0{im`bh-Mf zt9=5;t9pr2j+Q=&8XLRCWMonkm%gHTFwgBLg^F?$GqtWt`}Eg=rdXt7g=Sf5T>UPW zUbT38R`gUd0@iB}{*2$_h{coH*BiIf%gamR@Xkk5nJjrx;D+wm=9F6Kb!30r`?S_+ zi~wg5BcBJ7jJ#yi)u$fWw+ntpZ;J|&ZmG)ecVeZkmm1il%=5uZS4u>wmzN_QKCE~V zn<7A7ifwWBXXe*$Itmg`$yp?8*Y8=mm#eN#*Eh^y`B>USFxo^^0W&B=-Cjve?qzVP z>deJ^QQUjqHBvPd*_&|M$IY9%s85d~CgyUnGUq+@58J&)~mKr68vT0%j;q z0=JaIcEUmI$4l|9^|A>nls57iyo<@|<3;tPT@QBX;!J~#`PXAnoHiQ$ z7TeF0)zi5`Lf`IRN#AD2(Y_eefWLKG+LF4zg{XwjX@ksiRiVX1x`;I)b2bt**^SY* z|K&sYmxi~ia3MXm&gT)Zkm}v&;rY%c*Wc`Lp)7ga)KMt5)BC~aeK04=qXR^7-}$MV zy)4h}`GBso^HW3$9%<*pMXf0fiB|1N=y8rggVoe7)Sad727SXm37DDm8502*D^79k z;nTuxw56c{fG>W0(?gGeS2wb@u{mQf^?#jnH++64)sE81hgVq);XNQG4xtquz{E0p z%72>nKR&^qFZSTlA3UL?QOsMgacVIa7uRAGW0Du1UH9Z6+U9t$AyfU2P6-}f>JQqE zjysQEut;2QAyr<#^{^=XWVP8ZW@>ly$B)qBTErp%jhnKf8h9S!03f_2p|IkI$IEpv z-##(x=QMfjZNQPV%y}32i7NQpj~5LwL|Fq-_Ja%Snhtedg6A3bRg@NN+SW3F_hVx{ ztWf@wZQ%ajzBEb#kN&eV;Vu6hz*=Uohwb;@)ghG4P9SHZOfq$jtqisjK|gf9uwY&o zs-dj=*DeDWkHQ!cKdNzlF((b&u_vs_L`axrmnvrx!5g)FUGQvc`R`r-KZ<7R); z%D3xrCEXiQcWEG%sc0=CosF@mJk+%;;;|SvGrPk^th6I?B1`xW-FNqYXvDuAA*x+F z<(vYci0nWf3XL3@fcK4AQ?>V9TjLp*5XGr)-HTipXCa`XQ3{nrv1HzOKy9k{`DLDQ zR*gonN}u0$r~{5(IryjyqI~Q@Sr5QkEzJl4U^oK9xxYfXP{Jq2T|09oMGb z+wDUBGbWhAk*%(&R3Y|PM5Grc%q&97yHFYt{0)10e6zjZahBG)-gCaGQ44+inXfI? ztO*ElRss(0EQbU@CIo3MCRg;C#vA(h*oe=+pc4O+IW*TCREs88!|V>{5U# zZtV>LlyIFay2%gXqxbzP=O6zOOa8r~ab(8gEmVAXyI%OD<^Wo`SdaSNcT7KRhte*wDbyYw+0%bc9L#xc3OhDL+74Mvt- zI$Euiz7gj9JQ;C3z}F6VVq)T@Q!qjO;}4C{$%(?ELMD6<+Y`YN!(K_Q@5OF-vjjci ziq(A4*^erNaLN8NDRBoL6C!tZiFrkG_;iSJS>%4q_HU;# zANSoKq*NnrF77}1d`~A0-#cngK!>uqk3Z3b&Xt~&?DcOKOxIt(e-n@iUZCmx<85`?|D%_t5k<}aX1;soyY)*rDY20E?pa4dTs0QZ*2>C3 z68--r%1Rc!HxaY7^>zEAsdXc0V|Z*xicpn6D?r`E=}WH9>vcnrfi~=4N@+LsvjON zkjivac%c-`H9>h@Dp%*(V!D^>Wi3LB3 zKH^tQfI6NmA{F9SlU_B83BSA4%haN(xAp1w?lh9DA~tfd8de#I5r+)W&}RuSyO*Jt z%yc1%K+Zr|5+O%Rv`4u#P?Ce-t-kyDdY}yyzwNg_ah(jYl2pQyluLnnukS92K0Ij@ z{Li^r&d=p!Zm=uI%-sA*`P941 z!!b0j0^}6Sqo6#Nlk0Y(Ttj!Cn_}T8g*Z=p(yDdC+nj8rU z>qn^lh~x-wJxmm208yh-;Z+LA0AQa-DIjKu%VCr z(uyOGHke`9SHnG$g@foHH-L^Lah+RqUqF7Di7`p)8U9lx?M++dijP5^lB00SL``0% zeqsiy{bpPJ>1v49XMgbJe^u@)k*V>Y;@G4?M7f%M&cuzObh5k&17$OFTKIjr?<%2n zHO70=j0Cq7Tv?B&SMU0{{)cJ{^blh+eO;}M7aaz{ID^S-QyDBR37Z-KC~_D&q?kT< z{E|-hjkkm#V|~cjitCE zgIi~nLlhBSEC#W!BDX-DPjm??+|ir#aQwE0$Fq z*=T2#=uKv_Oe+ep<>o=yGfb$AUPnzO43l}WLuSp<(m^KT-$WW*;ovsf!p9Q`h75Mk zm4h}?o_@^;YKSNv?v9?j+|@p15Uy{0;|>2b+}T_4P0j3ka`s+{`D=YXAs_38&YLk# zQ7T#0tAKtEa9-f2Y3015E*&HJSGE68Q!qOskvtL3zK8@HCF_P85*C?M&qp5_e{Yw@vYfHV4d|3dDv55X;dQDDh8nrVpL*iMW99usk0)2Qj|) zuu8&JUVL5mbP)T}!kjmGMJ*U{-cU@>Ck?LK=oG3%P!o49|0`kpdsBEKQ5x=WCO@q0 zp17+AM^@<&p;|gr#UiXm{#1{+7dM=mJ>Ku`_hE371;-h<%p@HW#Z`w=AdU18yh!Vm zh#siIz{m7vn9E+Mlsa$B2rKUq5}W@u<>~C{M6sEAE(RT79{Mq1Q$&k#Vn%}XX;dWf ze3but$M)=?OvMuLaS!Z)O5z}H5&~Z)S@d6pC&q<&>$VF+;DIF(x9>;$@X&Sa4)v}f z4UAO!XfjRo$YAuRV6w+z!o&ZiG&x3{hTq`R^0+(yP00>MlbN}>%TBc@@Z+qYTScH_ zS+PJz&PVAg1!yrzfhIk~79QA8#99wus`Avu<7cN#I-Zz`;`jz?=ni`}1>0^+_}OQ5 z^_xkvU=Qwz#vQBK-8N^4uj4wHQ1#rA$a7IrE~fZ!b+YLk_r+*nvaP_kxx(6b+ODU>VXd%w?q)7O z{}@kjIPd33mX4S*e6TTp`-3+KsS5x6#`FUbx9T&-Q z=MsR^eXdlW`0nWwopUS2Q+daUbT9_b&fq9BP-Dhk^8O9Gq{d&aq7?7H2s zVW~uz!i3cSO2IZfh(NO3m_&;LZ$1tHAoy&H4iGO^?nhA-*rp*s2h=Fj^UemzSVaVd zMWEZEeN5PX(YeKQQ^1Z}B955z>L0sp{XV3%^{0W+B|Ch)Ie2&g*j^4w=Ma6M(Ccf5UcE z69I#82%^OHgBcQ(RwW_t7@2Xyz4-)xxW-ld{2zCPfmXzs-C31_Dl~v&LvbhR0nDV;h=z;^v~dI`QfQ(~ zkjfu2sB;xjE8*I}`r!Yor-q{9yB&P5?zm^rrT>o$@W75rb*hET4~fen9l>dNrksmT zCiEH8B;i80xz7Bo;B2`viVUwPYl>J}CKcJcTvOoh&RnSJr}+P?FfGlr0|D(cdaOeh z*uQ4rvcgi9vuDHr@U`d+j&`xKjAbm}p#w|adz58HDOA^4o{C62t`1Cf$4swFKY5>o zGrvvxKX4-*OBbrDge@fz;jl!$ zZE{^4bPDi3Z8{3l2q%c8j$f;qRWF*=i8ND915u089PHwrE8yxR?rUs6^$MeQ3WB@) zL+f~W>)1;%u?1QjFHy{b9$WOBjULF-pGStfCy-Jr4}?XT7XPVO^p}mI`2ml zk}s(%NL8cQ_x(5|Wy}`h&c`o+WK*`O_{eJB;`h9$gOQ_6M(B*jKxbs2A7=o&YZw?{ zWC@tM**K3IzK3^Lm6m!uXxek+@Kpy;1;m^Nyf?F)s@9+brUpkD)&xbd>^V29!PV>Q zPy{Bl!Drmf4vRZ5V>+894>rlx{@at!cA%JL)kGHGM@cpR?aOwT@8jb)RSgda>DXqA z#mrA6h7a~aFUv#a#AHe$0C<8UMM}0^LcK<#NBH&t78?HJF$Uq*UpP-LHV@wP8s?m} zB{?47`=bUA9Z@Up@9sD`xi7Xl+wZLTqJu_6BjwX=Ndi=029iYH5yLq{H7jV<>C3}3 zImu8yyTq!n5(FevE6{!7fCex|^gv>_>8nyf2LO)F+NuAqf^>a3V%7tYHTc2jwYKtX zE-Sg0vzUofN|X0TH#QZ6p2LMao|d~k8Gb5$LPQ?$)`b0Pc}R{6`5LoQQA8WiV*!OT zsx2&yAXyox1cI(v5m(xlBK1`lD!UlyHSJTOnq#Sfhg1sPfe8o^Ggd8tD941hhBOHX zg_o6218P-bGE#WHjG)&iv-^X?dH7tiEaXvCCZaRy^hfu{UQiiU94XJ$YN`dxbS7_T zK`p?b0~983dzFUrX*%a*{1KuyiLN@Q=G*vE`lx!^AZ?7K2niiBTSyNQ68himYQOH#EiFicbVzr1H$zI7v~<@1lF|**ozl|X0wUd=(%pPl-`~fdux72{-sd^7 z&pvx!w?~w##R468ai0?wA=}CMayJMQ`9d}T9~=OJ!}d&n{9wev0(wt-60QIFzhdBoAq<|7zoe?Fqp;dg zBds^eFM<%>gj8egljxemH$nzcwbJPfhmPlqQReoii&4OG#ezPGzTVbj@|V{B(+B9OONN>{6q1#PFQ>Xm6$Ru(PZKb_qSkFqY07PKyr z+-?a41O?ew4mFjP%gyP+*(j;xu5j96$h~01J@h^}^mY6boQNfpq`7!`gPv9bm^oU! zP32~@?-QnlHV-NYV8ToL>~fsGgR;krC>_ztr`JOgi$lQ%IEgj6NtjEziw`zC?}6}5 z4g8<4V`VHY_d1_X(1}^0&mo@9S{2%CwQY>6OU{RVxq_{?R>($mPS+#A63kaRW`?sL zm4)Zex5Boz$MxvvPT9bn65!belY`YvEF5+s3TI}(r_TqsDdXSdQ)2^clrpdAC=IMs zWpx~s+kSlibdsrq5R)kr*qtkRx%)*%tyH4^J6Y=O?u6mpw7PO4o>W53c!G4tqt_re zS5h=J2Bb>@nL^nA7#9L`VAAd9iFg;umD(LX=Xn>KJw<<9mzwNaT}pFHO3uc({bqmt zIy*1c>sy^#a%NV=r&8+Y|5K%ZAQ|Z2T$NwE<-lcEM#TC%hRW}yoO?X;`K6rL8Fd2^xTx~r=q7)k{i?*$YBmaR(2A*(o*%~M&H55RzZZ)a-{abCVi#N+`THB69M^C5U?qv=(?|&q0(_ zJjbfhXlrF5iWuBdfv>?f0`_X7A0b$gUnp|VHV4b9r%Cwz4{B9tJ=kn=SfL``zU6Fk z;ruAjV7m}+%4ykxmAjc&JA-0Dx^jEn<~b!E(l5Gcw$(67>R-?MHd3*a2<_dQkZkXpkWHEW3IB&vk7We!^0^A_MzhJLrA~WmJDm=bYx(Mqm$lAkefotz zqW8wby*V$r+%Rn_Z>72=irOkX4wln~v!aW3HA&&kkdnz&l)I6|k(H0O&%uJkOiPVE zyPm7@xTA(1*Osr>8EbB?u9coXGx6q;x5tZ~g;iE&*H^9#DQ|d^Le3Rk#?oD5m{VJ} z{@BgwCNOAzcswv{zglW~QJLGS)a!IS&a--S_xIk={qq9}o2w$zUHRneztnL2Jg*Yt4h+mLE;I+2&iFkxt}aU$=ge~Z1#o2AiQ~s{k59=jV#R(mass-C$d1tA?t&NUJ^*f`a!!hh?ppW#}WX|L(DHr=hkkpZASzBcKB2*YS9E zKZ!l?yt&7Wfz46*C$}srXEitCA=~$6p}%fT?*fYm2~X>Teq)b{$==*j=Yqdi&o*6k z=grUU19s{dVdt8vH`@%`k;>e2MEpv*GIBcmyBkb@v&jz_Sg->^?pZ`cgdB=kG2sUo zj0-l%hLui99^?}nCcoF+uc!MM#SGr!>+K}oI?FiHYF3a*|7V*Fp=)sx8XohJME~bGf5p#%-_H|PfDOZf)>eL^n}80AbLh7a zD9m8mnbldx^JN|R+=1~d!iJX`OEKp{Fpx{v%x$Jky2&b%kfEqjpS5ORiaz)JQag}WF#*x zmh?}%ink_)T=+Q@bN+H%5SEp0l>zzoJxe|VLu3TXleXS*X+oLHz3y{E@_b(nIDasz zP1ipSEYn}YrYbf>qs*>Ws&jGhuQ=XP`n*MnfccK2oYh&Ba(ghng>UK2Hz9t@gLL*3 zA>Ee!8q?6uKL^oN34WBi1uSx_zyxQmQisz^19wRyFeU8T1z3rE8>bU9_Eqg-<@KdH zIhl(TPDnCBPC5|>SNQcxeM0aVwM&I7s`fW+6mnFFvB@*k<<2wB88@15uk7UK;|n}e z|M^}?1ILA<=F2SKd&{iXjpoG!3l$eLi`lndu*zv$yq|)Vb0LuU-+2`IU$vM)-vg;r z*t&H5wLaVGSXqgx4kt_qy#gMFMYPZja56Y}Nh!%{lvo{Z$LYv)m>_vuOnhLnTJ^tb z`V>Re!9AwIZ6GSGo92MqcgN@*^=ISn6eNGzy{@f)6)9VAke0PL?~Q*dFnYQ>@1_HW zux`hTg|T|S@P-G!La1w@=z*7|n877_zUqD2zZKri6aqR<@DucXxWdc!%ap)ZfHkBe zx*GWTlUo}_4FsJB$QxtSPrzrGA#i<$=|NW1Qf?bwars*g>4%UB3JSJ6>hW4YPdRqM ztCd#t6oB$3zuTRH8xW#r+iE@?vnvZ3;Ll>yDBT5n{Qg1#m)rx?!PPI@z}lIOo9z;O zhR=QTb+aSU_4V}*+*2R6asx*@FtW}{^;vuM8ce|(Hxz665#HCz?;8s_ZS0wSrefcb zWh1e%wm+KO^l+XFu~7uSAN-K|I^Ta9!yvH9LWiICeSt1MC>#$oj~7NjK>?_F<&g@! z?9JDX!a}N6%K%T7J0=5gCx9&jp79;EwY7zXJLT0KSogUk7$DR`l(=Jep{@uJEp1fW z&6Jibkj&Qpe5Hb~*P2mqO;reeM%jQ(3S3%T^zz!GR%3CWght=AieS%*yt-yb?3mY} zr_1@gGel2yJ#Q)7(9q!Oe(g2c^DbtLJnzuabU;&L;nePvtx|Z6N(lJS{WsL@2mGL4 zFuJ*eK#KM2LzN39$1ReePvUW`vfqr7#2$!dji*Gk!Pv1BmpDS!GG2y zYiVi03hx`b_&uBaC3*LcB2%U7*s;Fj((EL53=JSIB!MI0zDlV(^k2Uo^UcFSVdgC! z9s=`<=US`IB;ens_t^~;BuEDG$yjXkwb6fWuv=<+;Bu9&h{%wDfX)P}>*}0=mF^l3 z$j~Q$*6AEH21-j&tCKc3XN$3V9BPW>>GP=uQu`(M&AZ96q)V*|17XYIEoj1+fGPj(5Lu%Un@r?R}ii4~u&zV?$Rj0Smqs zud!NbGemfTAL^M2fAi=GcR_(k|5}6PD*z%HzAj~x-rK(xw)_DNM3Kx)X7G6&%2DFT z75Kn~I{?~Ofcmzu$6YKsmWseZN@6IQqos5!Sf|G2cC(=MTK>#EA@aPmaO z>#vSzYzc60Na0plF}0$xvCfX^CEiyO@Jl~;cA>;XW_#PlFwI<*@6*Uk%O^?jyhs=> z<5`YqV`MCHH%*c@quctgng&8Hxd=c6$t0gjQneblVq4UXf+Ry*&~bX7bx+S*iA3QXoAG8R#c;K|L=D}OzU?Z%ylyW)Mi>foJX$nP`^9XAJ6IH1 z){@v8b0S!PP2~@gG-`&1&n>rMKc5~R;8Ivti*@MkTF)`fc~gXi%o2tpG%eQo;)US3 znpv8VC%Hq+G?!1#%($|4ypnWir{7`XXl~`%)_Qq0+zX8RQ?N?TwbK%r-pNbR9Vbra zoG7qx#jg{5gWCmShI>}qi-+2mE4RkZ=SSfRy#}9g$j;~-Pz$=$a}o2n2=w{!^o1wV zdXhH)WizQ=GptB=S-Qf43=d9`#-Hm4WudkEyDghn<4Nml>~rnG{>wd$5His-F~s3N zXmgEzb*~FF*(HMR*H8WOt!~HB{pt)UjJnNUN!k0Am4e=f+qWWo*i#Sm!V(NmXL`oSpcFwqUn;0V=BHq__QUja=LuO*;5@!!X`%TEee) zn>*znd>^h?_1B8LmQtAYYJ3!R*e0E;Zmtg0i+z^7jZWJ65 z&XYrLXQ}+JH6RjIfl;q7!FWd5fmV5IR?!dW4M{LYuYY^Ckh6qpj*j6^ol_vp^16SE zE=!}EpN$O*LJdD2lVNS>A%g3#x)2e;QDYP}tb9KiMo_lcA~S1oHg?~}4?=BWUH~y|YXvy-zouq% zK7QnRy8WODn8bcyq-%GEap5^ZZrA_T-`~&uvZ5JFHk5)Bl&HE^qP15Md}_!hc$c_P@ENrwTMn zhnHBvcd}d4Sw9ADc$fRlLckB&&1aiORNEY&`f;vhw;z))TcbDWC~|EDA7CR4Bl@tA zslX?cEk7EsRY6!Um9ne(EH{9`=zSt<{@x_VxSxa*nDKT8dZE%6l6eA38bR@ z+s&@ZHR)Z(Wx(Gw`W%oGRdEB$AdsC?rXU36*OKY^fJuz_#gayz3X!fC0i*`Lp+ha0 z>8d;Z4eLR;u0)F1q7@sv56)@ zS7Qg|rdd0b!Ezo3<&3-~!-Y`6mh&=-cp*uI&}l;(lT(-E8nJ~^ToPkqv#sgBPMq!@ z|HVFm>7+0j+~?!NBq&FMX2}hWq@acmhwjtQn_S<>f{;Ey*d^~Ye#f5$+Q3XZFekE# zln65<QpyD2ArN+5`pAPs6WRzZw^74Xl$F=lsLiNMLDcI60o*D3H-2LMsVxI1fFD z20svg=nMKS&FhWmWbi#L=UMpOuNUJG zt7H^%*>Q;CGPxLW@t=xb&OQ(P4M@@6K@c2XC=%WoGk67L&D=iNFYr}o9Tw?yi;LCV zAl^%vvnNsk6x8c`ap=M^6!OM2e(Y%p6nviztAh$3Zc2w;CI*=Y0Tt1o(eKO-w@C=Y zzW;lA8y*EiobXPOvpcK<4@<$fsr(j2sBV_1=Ha{cdj!_pb^}UL9Bz&cRPq_cHe0th2Av(_)jAine^xZ6pFEIwXGL z|1qK?BKn1`{-Q0bAPa)f zSF`>1F@O9b%=u;=9*wqFF@~UQA%*~q79!Rh$R5aVCPKLUkSKRApwhH0<8^n^_+J;j z5o;3^j;c}qqh`aMxfkz5n^MN>+Nz|Tx4Ex5d(W;q{W@}aSvr9sJY}gV63Ua~-IJ~) zFBaFscZTAtl1~;{%&%T7>ng958K&h)Nyn#OUD(g?159_s5 zeYUf+)85`L`Yw5cwswt`X6P6TbrS6Axcu%*0@7=2+Z$e@XlZl`=)^DrLc&2ng$d|i z`LWzKuw?#+c}r{jWe+#`+6s_3eEQV&y;t@0bNI3ks+;qb9YXn4Z%wPcZ65E`7%L7; zo^U1t5PmI(FjygJcNKwt4^aegd%`yzyU{`zD!NJ`VOSmQF4v2{bm;F-zyEuBZwAFj zSxiKrPXa?Ymg8FhHK;C02oDiD=xBAqA;b21Ayn#Y=3j9bx@F+R{PP##^vGED?{6`L zI8&}XUpCvsKY zr}a;9g$Jb~QpcvwFL9-1T+0)W-`dxtV@AIHM@VxV9473JvvEK*Pe7gNJrfr@slX0~ z;YMH(5j%kJ$eAU|gagb_E(@2PhQ-B)2QS*A?@98|!P=s*XjWJ%fsVM!#4v~k=xs6{ zhts8CuxW%`gdmquEnR18)1H_37SPnImBN-|BLHDCn)HkFU$`4cA$L$uUEH;{Jj0bY z&IsSg7z0HmQ11$QR6wvL@sd#9t6R$U6$GkR;LDla7LR!3FEz0({v5mGZW zJlzxp0$8uU@{)&{LBE3&jq@g(y}%!|hN|DMmx0&jC!4oCw0V@=YF9<&A~4Z%P424n z|8A|o87h~S{qX9L1!(zj^yQlqghfPRw{$Q2o3OiEU=2J$3{6IQy0sE9uisJB9i(2C z3U#z>1e7{lR@Q?JQ5E(@TPG_C#tool3<}GM=-Ire*(#Us{gyx-RoUzz_(@zsC4*QF zgTzEsqK_Un5CWeh=PCe;I!;9h{6q2P@}dPG(Nv5lt>0H`)&dR(7v>1)K3azUW9DYS zwF!;)-nn%T@%ZneJl*{;jU$7mF53K!#k}drmL*5k_G#fxHlavEO95Z~df4*O2T>f- z@D5HV7!C)ER0JZ1%)%&&Bh~e`ARyKpu{>TE67ZwE(i3QA)TQ?~#6y|hQN0aQ9mPsc|{q9I1kn zBqS(M@R+TwtN?3^fl_whfQLd{_F+H;DG0WF)i+XA5#4g6qKV5Y>2l4|aK0AFMu-9N z6t4v1Xe(4fo1FGU32Yve2R}EiI@6n!|EoNr1Ncjk@~jjwGKKui036V-yW|=e73%Rl z*XT^$90|?*#y+)?Ma0E<-V%N>1#?`VJf0GF{$lgrfGAE>+mv-a%r2ewGX?#Ggj{L@ zU}0_T$BG*_cl~yi$NokK-(H)!s&5}HmUPq8Z3wx9&VT-t%^!`35R3a%QQW$Fm%Gw> zdvjNjbB;YIT;t~pC_~HU)ivzYF*h2NwVu4KAYp?K4pd0v;>1A?FDt`}l`G`q=ieL4 z&{EO9dcNILBI0j!>wWk<50gOd0NfW&PR^?BqTUS;TGg42JC9*p61RW%<2=l<_#5;H zDn1z6L{`wio7KGA;0FrTPMk@#V zqy6v{O$~(%0in(NG*OgN+R$Q!456Z0Ml)Mw3LO=-TKntThS8Mosd+uMS_WQ?USg7M z#qIfp&EHwlWhgTlhk6GCPF=3{C(GVaf^d!`BX^l}`K(>QvW}C};n>s4)CuQZ%Ee{C zP0v*S3Ct}xFreWm!C-Q2TANt#h-bFzf=?Gz6&87iBR(Po>{DZ6bn<> zf7U=C6cG#vEgW?`G=+6uKR}}xGgkDi^IZyj4jmM1JhQBBFSADC`tB+N7XXITs=n4u z2(9=Roea}tc~~DzTdC8W%xkG@t6+SUs%!xKzJHvIz)@gP#pHK52MK&crDfIE)=q8J z>V_3z4ARxr1#n#CyveQ;nmpjgTgc!q#M=rxAx;G)dPUu^uj3%-M!w|z)K{8ZWsf9{OH}dO@?C5_!#Yk{Sp;SwoYdCwp0zcc6voXA zVOy-Z2rTP&)S5&6Qk#pE@h}{T_gOAYP1D?sO6eRM*W7Cr{D$72*0O**R4gH~tMsvY z3(7=9N?~1L)I$pKJBh8ne(!zx+}jWOyPhODO6Wk6dh$5Xf}vX5liMUD4X_k^$G3W{ z^h20Zkdpzevb+1f$w9O(|CCW8BKNe*=z`M`D&W5eces2n6KmwSns1}@^Z&U3)Y#`5 z%Ie}PJG;fUg6sV%Kcu42!{LoNp2 z{#<%piMs%(8**%J=;nF6!4FKZa2FV2XWd4@#=txrz>X8Ahq>Vf5`|k#V9WU#auOa$)a9`9&8)`{BWi=u^zyN$`vT>>DCOi7=uykRC%UDy2B`cCxFoJ8iP)4Fx)|`Zm4fha zkJcJVLx`|`WtEkt%R?6cXPr8viMZwS#!>S@`5v==gZ&`{92ZMo0HVHI`F0F=nd@}e zp0|8eu3Wm)h^IhZQ!Y;rF9UlZGyA?Z2^pk^$H^ozqnG~bart!)yVOj1Su+_A$;6IW zAJ+0gU^`RT+x}v^bl@@!@wZ6&SXlphuOA&*2vIV8Ne_MXwwTz57{KU$(N-j^ zH-SR|Fpws_3+)~8^GUN7Z?2C^%&ilJi9POofeDN#NjO05QY&<*^ePlE>Hd&5G^qq zhJ`K{>7y{om7D&(C{$KfThJHj?ta4W>9Ut~>XPFTyk-18O@Tzv#|$VykU^O;5BmE0 zicH0w!s;+;aARprMi<+|Ln|8;@U5jo z!d$s#PO=e{1d$(kTv%s_(B*lE;q$gpqUSRGuXpgIlX084IH`u;7XcPiMOf`A)dn@& zI`~u`*6(`GWEYCV0^W6~j-KH%nt?nHy^+E1>5dlKm1TqTpfw=~fvTcVe)x7pgCmDwu>wWAxBG4m`a2F&hL|x zqqiSs7!3VhnqTi?*7{5h3UcloUJnm6*cwg$q|9aLZl0UjTM<$~@D}f`)WP{u_1xU* z>jju_31x8WSZ4pij0Z<9nsU334FU*XBmYer<3~75TWxF*qZo40&$7tc8Y2jrJcR5m zrhEliEWx#o4>ekJEh6sQl)eNcO4GiU^WkDGCPSC!`X6^c4mpnsGT5JSn84Nlm zYqv3^Leujq%6VmSO=ROKq zGZ+Taw6McNK^`z8EO$M9+F@qUtk!As@G}l$Htcv9=g$s16n>d9JiXh^FfC?INlX&H zogR2MRUDQr;CD4NUfmh-cqlDr;CDb%R}{q@;c@P9+8q??ezK@#pnG%4?GFsBnxvz% zE<(Z)<>ah~gCprl4I6CTl5SI8Ura+uZ!lF&;OXL%6(Dj&QKXm58C-4GvkuIS8Jsb~ zLN#m`!=b0tjNw?AbofT{I2t(_O-3)jLoS(p{Y_@04*Y!{zXjvMp}=jB9cGgKmqyc< zohXM?s-q~A!ytFskr)9{Z^A4H@xPQIq8cRzbIRP}(lU5*t#lUm#Ry+do5qY?w5Gq$ zdMo)}=9);@k)DoB6Qauo!hZ`JX`~0IVxkH%87e_G>MEc|o=D?IC)Ih?^$J1^2;>3; zE|-0L|214-06`qU0G->pyWawZFtC-Dtw0_0$qtVYKob0d5kc=ix|jNGrm}!N)MVMY zeo_P_-o<%r_iWS+|0yj&tNUz->8J)e_sDB(bQ_O)w|2ai5`Q){qPivZb^dWov-~K8 zBE!IsizMdt!T>{7mg4H( z#^*!YYGTGPv@ o*VrsC=HLt{;%A0Wj?_2T6XMq3J`KXa#{b}_g(W*Raa-v^O^;Y ziAVd1?p6cCT-C>e{i9mBgAG12NDC^-oUoHo5(_>U?|FHGTu9qrKZwxw@m^~7v(Z2__V^76X& zyzgC8)7K9zryY+c)1?HYUvS8VZ@MCuk8?g}l|ggygQ&jL*13@gISG9xfE8XiWkBd-kJnn3qxyi@SIWNcL=Y(G)-NH4PP~ z;fPl(R0SWt(x=zR^kL{_Jh?s2ep_l}dlE5&EfB%~!l9yH*a7i@PxLmRpuh#oj^?a> z=g8S2&zb0#l8_imCREuGITeqiL;f{!NK9Wd3cP9(k5rs2vvBOMx|%v+D0-mVD64 z=}S3f=zJu@_Q&V?|LXdO_MX4rbOW$XD=0~41G&JfGL2QqRE6m(d{F#{yke_BuK?JH z?k_8#G#X?SggEcN(0Jk(}%9QQ_+!Nq|#JE;4n%B!%Czn7@af`AyJ&cBc*x zYYaHB$R;8jG&~=k-w$;)L4pRYxoqcIimRF_#W;&?d_6!|;S$oyPcfaZx5>M2Aoz;+ zwzRr{mr@Q-b;FH>;@AN@=cR?B=Rxxxe>(qRl>r*Xkx30cyR^LAVyh`?XGJ}2+(m+> z(=t(@5CemwN`EFfBnSEfMq-j^EDH(@q_#^vR@w|a09N*RC`HS*d%4`X1ZX@siBbKM z1r=>=9UUYAtkmWS&=%T7(}{;Qre#vVZqwdt{~=0{hTr|DVaN5Xp5MqA%>Q~I)CnZ9 z?g+2N#O6{sRvvf>+B2!E=TsUcv#_&ig;Ap0P^X){PvZNOD$%d)9s zzDHmOS1-jLW6)z_raY;SB~kCjz^&ccxNY>?fE+KetTz`=_J2?LGX>(;qg#oK3cTZ< zdqGPbUW(i!weA4Aq%crOL|1vUN~c?;=RqlF1I?SJTbFwrYPp7wowM_c%`d}vozK?Z zWliO6l)7`j<3_SlhbT0()eLlrPCi*#+u7Xs3S%KVTRd2|tZL5vs(l$QZnS&0)u=b? zO*gk3Cp!LGXv@J55h+cV6z`WbXBK|EavxO6ZdI|QGaF{0rA-y`^br>i3eftHMat@P zJ(BckI1I4;?6~>$Vw1uw*lxdbzJ)Mh*gTLcBM~w^(89-T&{l6YlWv1{oi8UPz~#Fc z&DLlmVRhj z4EhhjCPkzReL6eSgNff)%oOmvNW-m*CgI;?Kw<4oDd_aP+%3-b+Dz$uGVfz&n)~x| zH|`x?K!8gsg`v4Q2?h)M?ljkKesajWojzCJ`;hsbmKK(C^YO2?`aR9u(}h;&g=7qg zVB6l3U`H)NY(xOuPYku1ZN~zSkJo**@~#rsf!bD-SUN2(2X~I9qCV{}7YL+2(~GmX zrHt7?&#~3aZYMc83C2|~R`fAh;mbEM5`J}hjD{)hSZul8v7CDy1-20%Wm_#zU#2+M zQj&4MX0x0x7mkt~+IgV`Cy^Ne6!0HjQvXgGkA1VmNl+KMd*8CKU|1%80@(xEF&2TF zMEKSRr^rb`(8utPgZU8*agXmMMWJKJv#2$gXw5`@>Gg4jWKc9jai&c!ISR|fjHxg< z9%IeS)o34~HSuBTnCQES>@;vPO<>5(5+t_OB?4k~A?PAV(CBh#YaER%uZAvzwaQ+Np&sE$BZ_RS6XrunaX92uyvvG9KoD_R^LrQiga7_X z=#vQS?NDAi5^I^?x*7j|C&04&CCEtRIxWL3^%h5*lW=`~$op^^>tJY4usIN>{~!7@ z_lGZIZ<=kD?Q8`D|2fELW#ekiJtET&GI_(jMY%VzUT#Vri`;uJdcxQ3 zo<0}t-dGDjnO*oCHnBHwx;I+0_>iZEkIj6TwG;3A2$!SP`&#Pa^Qg9u3yEokAIW!ZW&`%W&nf*Hirq|d|b_PL!>1H zHW$I=K}soeGKZF0s+vwDz#JPwpvb0Sgp+wJA#wM1$`^)#DYn)U7o_03n*a%dZ<_AJ zfH5(GqIKl*v`jUu!(i}cFH%n9DZ6@FIIZU3utd|u>+|NuCu01nu$$<#_82@W5A1%R zBf{)L7d!uZXh1jH*OfU-`qQqBdaVzM?ek|H!8hihJTj9IT0(M|Fwuy9hyZQyJQm73 zXl;{%O>739_E;(jiS&G|bXD|&OFUSIEHUVSB;10y%{jk|&3e@hK$X2@chfb)5dym! z@ghphjp{_M3WgvNF#uEPjXLB~2<%8Q#u0(vtC(#!`GiK&+nXB~*FnkdilM}FLY^<( z&$KcM8xLs8O7zkWqcb*W>2mUEU1Fjqc28CB`=H@e?e8H+)mH0_e`(0x%InFU46A3$ zn~Y69pS^ZW=d;lTOgp|MK8lZ9ysSEL%4I75BS1xy;tBt1YyN8L@F7Z9O|AAk{93Qc z{_8dM7v#_2Shf8H%WQ+@Pg2U>Q>1>6k0+}IAY^QM+NYVdPF4L~;wSspVm){I8jL!K zHYmf<=kx3q6j=t02_3!xA8T+IoFrBLU^H{2n{pM2`>qL0Z5&=jYzJcW(Ow0xf6itQ zCG<%52kKnFVfCpwa#xV6td`;^E&67b0dM?dwY9U7KP?>lo&75)G_U`4EBnRIEb#ek z#eF=p(>iG0G_S(e`sPq$K@1H-A^{FZ#ZWVWB2*+OpydhKsv?I4bH1z7L1cd2PC7DB z9Hb~MBiM`)de|32n*v_&#V1eL5dw5;mLYvi;{TORv5u!x#P&w{EL{@YkA8r$Vv%A- z-sC`{6tZDGfoX~*$`dTk030Hf!mFQX*O z9%c&)HWpN5WelT=jSTflXG=e_cUWq)TN+>S?&SB-4uPyDYWG=c2Lb#a75RtDu1=%a ziy>uiHHf^*Ta(cZ718qvgVzK32|-?~k6_h#_JQ&eB2M@5qwfe`maSh^)GSEhM^MS{ z;OR62W3dp9gUZ>hL-4ne$is4EV_1&ShQ}(EUUh%;^t!`n;$uKHH!C3{vu(>x2{BN_ zlgIb`jT|Hi3@UOG;&citzL3i|x$JDHUuIu^bqM!%S^aNfhcCn(4T}lx+Z6oU00ED} z;Bi5D&HJ=n+kv9BwYu*cgv`~|^?K}hCV$j&B3;w_0VfPsz-E_2+)v%qSG?C%K7`C+ z>TXd{2tOFZ+~VtIo)rLVD6l+W=+wcp8_yY0@+rjzth$> z-3)FBTnJ`$YwIOo58&-~tXNoq4!fHUVN9E`nN`f-Fh~FdwMcjz%Zm7orK4yGb+D}L zP8(@q|L$&tu65qgscNzf?Ef{t3)kb{?0lRMzNS7jc*()I6qqzt~+%o3=dELP5wv<%{+pixA*9aen| zLl1z&mqvpXSN5fMb-u>*eH>z}q46Pg?6Svd=ZfD6b$7GJ@-ti_ zud3=pngRuro+Gw>5t0~|3vpzBqwT^4U})~Y8IGpc=%a*BWvBcv!5ErL9V&bW{J0*L${DMRYh!ax}TLPsr=^+teOmS^41WilMNu_svfS_pHMP zGTacK8uQD8t>Q{8T@79~BD%AFzl&iS*OYN=RV+*idMFXbfKQGK@r9#;Ur!g$XXX%; zlv4!`9JmCzJs+-yv8_KC*nsZIo;NR*t(9ggd>;B|MpG2C+3vee9IzK?O~iq{n#oUz zcJeEQC_`bsA`4oz)Gmb2rwes7@9^_Ah1M=~#`*zWBMM2xP&kz&W}y|IU5T%!xFN`Z zs(}7}_uBn+WBcQq=ya~fhU^!!#l-N0^1;j!s)$v^6oc0_V-1zN5n}dQ#)=?Ono(zDmM#q{lnA-g~;k*<#mp z+_=)X;7P5o|RYRueGZ)KO=i>kr&p6|g zu=8O(xWl8NG`Ar?ZzPID@X$|=6P z2hpfV*zPUG8&G$Px}7yUs!l#Ge{}1>|FzeSK89jl2bqLgDawbx2f!=bvBa=rV*H1MXRjylgtb4t8)(RH=m~>*r5}NJX{)OZ`Y&K zA8ZTXeA#kevR|@m{rA3Z>>TRYKp8cp@b$yG-*;0D< zEyBRtZZijVkQ%H>Nk>V*@aL17g8hSVDzy%&UQ-$+p(kI#1ZE=<8AfdApvTzI8%hdmYcAvDO^ z=mmkUbIQ`)M<@yGITtb>*q|54KtD76gn~5sgG|Ee)?iW>bmcwANqi`pNzKb*(4GXE zL|+Gdv9W>eNF>mgZDFBMBYny>IT)wMBW4??^6ln zdyOK~CMg^(G)&w+Ls$sa=V#;z=)R}v6$OAzc3BQO^!aa37ak8$CMweP1#K1+ec#c9 zVJ$B5d9|1pN=mYmWT6ut*bULO2tw&EQ!>fsUIsS>ziI$aX!D8 ziK`nN({E{ePfim(cSWrRnA&eiC4Cd}F0HIAET_@b^+0(RpQgz;Q}0;~Qw9If#xpT| zQc=02@=YPjdeFD4{3ghj_rqr(bD`23U5P;0=_*XEhIo!J9i$x?LPyWQPLFJZ^w!58 z^;kOif%|p^fnIi9Yf}XTc~e;@3Vh<5)dK6Nw|3mSZSLbqgA)Zh{I(tjxsg3#Ma!Bl zzuj6_xe@VteEL~=Wl`oB`2So0bT+P}idHAr-%qkO+(Lr>{koEO$4xK47YOAM?gMat zZr-}M5(&6mgQtjcQInvsd>&x{$=jiRIy4N5rM6)~_&1w;cskqY)c~Ova z4C%P9F^;EOEr6zzaQZ#XV-H5(cPLAwPw=ylA)9u^_zt%yh#Ac6t4<9>3AkLyXcnb?Cml|3N14|rY9aTlHz%VPIf_J|*Le93TUz<9Binl*bJZx8B8lt++mxPH zNgkyHJee$;24;oAcFh0ZXhRR9x_9;@fMLi?0{~)aGs2r6v*Sw66gAP>h$4%V-)t_N z>Q!?S)P9|-o-_E7ya^ph9#T`YvUYU0eMzj9-pc5mEAyigFr{rm!HAsz{oNGE0`)B} zE*hw)uvlm`XGZ~EBDRBBL1iNtk+EID)53rKc@0lW9FA`Ul&)Ne=lwrj4rWtHFa<)X zX$CWTq8f)y7-x&8*AOEeTIH2DoF5B!)OoMP4a}bW=+a$UUG-}z;#$o(<8b|aTBUcm z7SFnQx?FQSR9Yw4p{1gc8FmY3XKUw+dYh(&C+0rhlH6@x+I_A3Miep7y7=Z~?XsK3 zkh<@E&6;0B!_Cu9D(8ye!FIl_Y5@PkF@He!Xexa{{Ihpg_9LfE=$5e<6|LHd&IRZ@ za)S9CYVzo7mWj^zL$$qX@Gy}b{;F!%M8x)PhoRy$a&gN}&3NA*1_@8be!!KZG zqWuP3f9q_AA#~V-IyPV#@MW0|`o39LjE-<5h7Pt)z3oooVYt*&v$Lf_Cd(|w&LJ2H zGbEDx`I?Yf=u!e!_IG_uxEWyo%z#>_)8R3>eORNZ*6a3I80te7Y=#wTU%>n?^i$@OyS36K2$$K@7@ayxHF{9ehK|ap=7*u0cz=&05gj zu+zEbnY|IJ;lk6x6iwP?#K6mP@{O{x*D-~R!sK+gI9sRwlc?%^(6zU>clF1P9}c(C zUmgZ1rE$%sp<~J3L^HD9LuMPfTpGnIAR!`$>^=oBAbeXHNUDVVnXIJo-~Ic?g+IV% zIYo%*NQv@UQmg{C|08K<>*_pxA2P%``ZXjZc9=^Qg0CXwAIy{kD#cGCL+D-B@{MIg z1}5*gxSBD9bq*6Bz#&LAMIb~0P+9l!MeG~9QQ5R%flEXz4IdbzB#nmeEPQ@Xci8+L zI7`f)EPLA8qTbReF7+^2u&&7DXWZsXC_MPaqN4R)%c;KJ9RMLt{8iZd^qzXqvEZ=- zQ)fkRwF~fAbPv9Tk;ihOC@s{?OVG<-5wtwrd-QybZ=G0mcD<3&m1PK@c9NY80EuEm zpMf?IQGcQiK;f^kPfe{PVcv>s-d<^=c0p08-h1C+2+n|Y(q>k`adDUq7^S*u5=pxf z;*##l%EmSd1=c!oQ#D+rFm-m}%jZ<>_L}y|hEe%~bYFwf8f|=Ku~UNGcX; ztqRidy7y^^x~+F2M8-5T??^pR6}k%wsZN5G?2TI7Q7+7*QYH_kn#pBLlFhF2((Mbs zFS!_}{)(@dEhY?Qk&c*5KqRy(zGx1t*lY>t!$;QWOdqFdC|EI^saUXBA89$M8Id`* z8#%fJ<$7!oZ^I=vlrDh-5;O{>2I&82I?J%Ox~6RhEn1|w28z2=in|wgcZ$2aySr;~ zFYfM8ptuz;Uc6ZNcJJr?_>&*Wk&x`QXU&?q=Dak^&MVU*8pRaNBLY9+A|}*}LBmHX zIbg*P3W0`qJ=sl&w(P8?9Yknw=yZ&!CJCUyafpd4Q=`uU3@!Wt$VVJA_`-BxGBdc0 zggG!re-|9`o`g;|0j!OYZVIg^I4vTAVW8ia)K`(byV8aS9j`pW`G=ShZxqYkJ?*ED zC?ZSfilIivq+Le<)t6*;85%~_vPe}`W7FD}FzVTXQ}fMqDT9=JqCj!Z*O>g^J?=#j zjmzi#ZQ?buUgF2%B23Vn(q|VJyFxmH7b^SXLx`FMxZ}JUmWJ14rkU^aLnq5^$B#|J z!rMRRlAV=@C!qFt-GVdtBBTc#@gPub5?7Ob+G1N%OVjILcQ97=uFjx0PaMSZ&U`XWp10MuqMGJw@ACa> zS@=nv0XKp6en@Xh^yH-fb#d>xZTDmN!&Y@>uH9sF)%`7O%aj!VqYVy&=52C5Lv3yK$+?}N=kDKBdJWC7p(uUt zyUx|Pk=N%3WH?@T!DhSFm#AxFHsR4%Z^`$*^D8TO9WoYTvKkEb)^^9|(pHMy&PNN@ zX|r{eG{H{ZquqP?zD`Yr(#E%I4_jj{@0(l9A77#xF~1cQziu_mzTdLxDi$6Grj65% zZTS9N2`K3`yWIg8)d|_%3e~qo`t&#-w@)ZdQ><6BY?y>-Kxl>`;!+X@xrk-L^fn^)NY-vy-rDp2NNGMP z&(9z80HBT6`{-oAnaB2vgNu%9Cay?5OCjxQlgT=A=Mu^D*}}T*xLQ^I--`#x)A@Gy z-9fB6d6VyR8h>Ytzu&%t_2j0RHTTQn!`92MR>4lUJvVjw;y?^6F{)p6rGCqwpld~G z`M|VwG>!WGQ-)IzcsA>fHbpI@=B9q9I-r=&HZD+&*NeTo8hHMN+5ff5o7QRj!;)Oz zRm;+0yub?!=O;~-&fnv?>)yN<`W+4e#qfhB&Ledn_W#9}_#=#Qg|Dk>^Dq^39xVBNWT|gu<8qd>WNIS#@(W`(>0nFQ4E0uzOuvEdi;5p0^lg z^W(!;Ybm5e71Smfy9`j{#6R#LARd#&VV;(}fN29xC&vqAy~h5ebBIaNdA}_4>O_@B$8<7H zh%_w0aeLfzx;WC1jPSeS7z|HVjQFzGIzx|R#kH9~J)R~rdsi1WD>U4&U@jP zkbb(+=&Gd}y{ldNI@9r-_PN*wI^}pR<5;XzGc)l94`=75ns?n-So{KAoMOVk=O3}0aRvnJ6( zVKIZGXfZ?}5d-B&G{YG#ca3XtY1!Q{6$!hKa_FUn{+e@+y|0I{>wxz6q{iAnm(Rbl z^sc88BxW3rqm~jX9G1>#u(h;1umGK#9?IS4eG-7fvFhvng1%aoYbn#!**t1L0pQP|^!g%yn=*ww}Xs&7JTVZurc zmiBfo3|I#B9*@a7%K&aISC|jOQs?@8{N0aRR`9>eb0x0nxW_4rDSm+QV}f$xXbUQ5Oe<)ZYcu(8o?>-yJ~|8P9c<1~?n`tThE{5qCpAQ*rfJM%bJpyP zm?u)k%LA zmQcSnE6cl-6AG`A%kcKkfn%lpYkzisvqxglZf>d&GQzuD?z~XU0L8EU%wdI}jO3P$ z(oTN{#I$W#iH(J@*&I|5k{q9TqtR!SQhUP!ucz-c;94OvT0vGE4HFbGV0Pi36JG!B z;{*~yl|x}(3vazpLYJZ2G~fm3@>x$uN46Sikcc6Z*SesJk%;K`$+@)ock_NWa<+Qb zB)Dehjt(#7V>t@DX~iJ0qOkL<&Fi%w54z2;}&lTj-xLevit zM6x&if9<6$9lpL3XuYH;85FZy6`l=ge^>4G@L#NtM{fv}S5Dn(2K8=6FlN2<>d@mBG8q18J_ICX#Ox6 zbKEq)TCJms(Knuj&=ATRmZ$l-w`ND;t*=0>7#1E&SnY1g^9GmTgHF5KYfk1O=_@3- zx@{%4{5N4$W2Kn*!u1K`)ChxCPjy8F>!P5Ls$4+yCAx9wINH!*qdRQL5Yw_`B(}uj z;p9j^BR&R*!n;%Y{9BCi1d+M*`a>iFBLbN8k=wajKdNFe%s=?;_hjaAydura}smsJv z#@9k59j}gfS_imzRCzKa z?ZGg>`L(9`QPBUn^C=q@DrzQ|Urh_sX6^oGTzBbe|Ng7DJpZOIFdi8tgrUggee!$m zky)_)>v~+6AQ7MEOF8jbP1Smfo5Jh zt?k+Lbd}p37t&T2D#4leos91ud{bn~KCyrSaRC;Va#2TPmz%`hs6{+IgB%D0!^*Dv zVdw+Bf%iG5vUgq0U9h{~f(O6_f@_H)U|ziZU2ov780EvXY@HlkG;{2P0s8fPx(i)2(5D zZ;FAHmVYE=Ke;{oU!e44ppc0|ese*_E)lJbRW&f6%AJ{wLDs0$WxKYdmNH5h8_JwR zgX4eQpjVmy;XuX4HsVaq7ZC2E&)4HdweU6`JI5;w`R>6%9f8UYfYgX?YQn;FUdOi@ z8cr*f0njs1fBD?-^JFgcxPH0B}qT9szR`E zcXbsq&_{+aMW`?-;dI!TQUWjtnqyEv!hEfIyIJrm1dc%z6#PFAmjn44e8g3p_Y!Vt zJ@#5^bhvF^&SxRZhIH6To#1M);{`k-jApTMBzumc1Ij=W7oCU=CzYHSK1i`e%Cx=B z?&hg%dG_sK_RZZrOC(BiSus*GR=c<`5IaH95JQsBDgE*BSgWNsK>mJ8H z%HF}Z4UmtZ2eFAvETUl|U8mDf&mQLf-_B1tAn|p^7aNH; z7nz7k?ZC*y|4(EXFWd+_5x!{t-?ov`4@cQx0jquV2we|1fZDz#{Ao8q@^5a;xMi7| zT`+e#f1ad83@J5j1)k}KHB?j4A8IK&^YSj_aIAMiR)~tQS<9EOFx6AAoR8)FY-A{5 z-#Q<6&8#?J!3ZHvmlFIKLZKO)_3agn*Mu*F^rh$5J9##8CpnEs#8&F&+71+*CwyB-#Z3 zrpD*_$cfh~|Os1hezhkzi|7ZjPX^h|w>1EB_k#G5`Uk^1! zCE78iq8bY2d}kx{UgYK%ml}y%)0m!E*J{FJgi%SqBaCpl$ah0~S3WgA>b>Ws9)$Zl zA6v)dlZpkMKODYb@RS|E=0)r<%DL2H*v5HO)#o;>zkCFa%whNs>?= zgH4Chh)~)}p?OWThi~y0CBJa3NUgzuX}=pt5D|*&e`;;}h!d&erz z&poLspFBsTw)>022M#}ej2Ig@!0Qaq!5t9|WyH;K*~h=B%_+Q&NBfP9%ke2uVmu-c zO>kJ0lDr||rdV1ImLkY7XfN!J>@zF&XJ!dNi01@HB=NV!Nt&kkJ;7`;n_7i(e27B$ z)r%S1qYK_FR#GSGd)aWKX)`a8o(_WYX#56H6+s9(GFSY*dPJUeC9JU;;#h|iDkYWg zEm$BF3X7s7$=mJevhcqx*xwwBX=Qn7$xq8422QCKjly0EFwxX&FcyDJzf0zGQ(l_? zeVcam*(g_7LWIeLg4>MIMKeOm$UFp{KB9ni8UauBJHP~mYnrfMGU#1cI_8*n3U7?W zozCZWcN3oj3NughZgPSV>=Qgld20JhEl&JUNGbD3yneh&xpAN^<*?!6~)VY#yh z)(coAY(I<${S;MF>+A6D29f4d9+EB!owCwx%&!{?-l(^c}=&}y}!jaXLK_592 zFr4)ghGe(@p4u~KtG!%wUh3P*@^?Qx!l|+SnEN?zCAv$-s;M{}lXcniq}}eqx&4=% z(^4`Wu?0*C(6*VbN~ zoHZxw51ZNzOy;y4><5)ip+55`bQUae#W4chP;0pdsUj24dtu7 zdRm4w`=1*;|9jvcu5(7=AKejorpzZ|`RpDzvN8q=35lqzteii-H8mC@LURckQydYl zT`-yE!UvPz+}_OA`WbtgH6S8n%B`%fdY;d>12eh(@JrL-xylTNv>-zc3Wu*+!#SQe z^n0(-N3)%4ZYmM6=EdTELaQJid_FaEylhnT`Hb<8Llala2G3hrl+GyC1LJMl*pEQZ z%AnT|23_=v{g5U)SRyn+Od6qDj-tdUB`_)bUb21LIP<4xp(+g7K@Nlup_&W?PLra5 z?Bu`oG*Upe>>)R9uc%6tAF$HXW^42m@lSO=`QN9vuA@u;My%>+L{JiAPefPc1YwU{ zV~uoK7WRZEWvonYdl|icK#tmz2z2?p34EpE9>_-K0!;AN#6CTtbSIH&RrI`K8!STv zDND_j_u^!X6txUyOrw)`o;0N!i|#BZVdykqa422?nUZSUd*2J~guQc!%NprL(=QtT zDbxku8rC8>kXWE}3dG=T@6sG+?|d6_vQ|R{Ey`MG90Np)mrT)iN~i2y)nzxIuqJB#OY?wR>|ZVC#%QecSBNS$y?S+qrGJd7ESRnGB+X*} z*afh7+nmg{r8xKUauKEktY-ifW2D;3CijCLlBReYqN)taJd(h36UvL>M0tzFdF*|5^a4 zcVy@`0STrh&mZ%5cXp{5@ZsK3jD2f75xW+LRfi!nrr|)wQ-UAGDokX+BBRElFvTAr zz)T!K_=IaK49ZAdWDEIOk|{(aMKJL-uEGi>O%)4KWJ-@Fp-&7qST(T9 zj+^3SiwQ$kq=>79|D}z0(w@YogQVIN z7J^P7WK_id1#)@`Y;68^pQ6c9$G2=OON{1%R4&D65HN`r>aH9trnYmbdllKGzg0?P z{O>efM?AjF;#=;Lfx(x;`UgZk<~%E?qOJY3q+?$SKW``N=y+va*PSLuwe5M7`(h_X zQ5oRK&+C6N)enr!>OSpXU7z zEPp5XcHmzU=>|AZmWX`j?YP{sBqfN9){7XuKU-_?+?)_2pUWhQdH2iVXF%~Hg@#f* zRM5&K?riT3M3{^!A_%Slcq^aNVT*T9&wh_LZ@wAf{7>2ZGg;eU7CUMv}BEkP5(e>b%y9*wJtiPsp9#^3j{`Xbx6!c55*2^`U&9)1-UsGYm z3+F0mJr5s`(wH-p)YLIyuWD;+f!`&E&-KeZj@75KlOnJz6pd!!wFb|#+uih0X7uV^kQ4VNy$8ShrhMC#W{azN$_1IIy};w& zzuSBsuX8+`k#QKnsP>!Hf`6_crrU;L#^9}bKikbEdG>E=rAg!TxL-)0UrMIg-d$1W z`&x|`J+@aRc6nUwHhdhVoQh&g*%WUX(%Xc2d)|I4_BV7$+V=hzDAUxlkVtkEQ*ng`re?UPcZ1E>{gS2 zR0VQ^%%?_kXYHo4=dV4V1+8Uvw$|4BJ_v5L3KC}3thaREUL&(tO+0yZL{HGUoq0aB zQ@4&%Fk)42D?gmjZ)XX1CRJlcYfgn2GeTFH{v>FIelHwVJ!d1tHSw${Pn8V#&k^hK ztE&}NlOuymhSJV?d!BcciGFg~DEv!x)v*&wRJm0CqYuY*7MFR{;O7f~Ul{LG3L0V`dqZ2x-k zP}dFVUN-Q*L-YUp3*f7U?bftz$&}6)`!YVJW&)+jb61@O)rka%>U#fRc#Uf&ags*= zkMjC_C;bNw<_}4J>DHNE6^mV6OIHd$1UxfuLPO^A#Ds!@ zD(LkV97cFYCKrZ@$vn-Rft{LazuFbc>ahQdI{b@nh2HnS)5`|B-)#UpRRb=LRuiZ`=IXydHv-QFosGHdwL_L<(ZMJ*+ zOSK1Alvn%Qq%npmhog~9-93g<%BC}AxgYB6 z7IE<6`A_!~^J(wRU~hJ>;Z{2CZ1)@F*&6tFH`o zmp`Pd_Z1>bufZhfvVSeKbP#zgN!sInH@`?c>+3DKxl|gBLn7$McS^*eF5vSPOy8OX z3ss@S(nSg!QUA3K|Jwxj{`_+Cs!%-vmW0=N^W`g$6txu>@44H*38cy#Z4q%XT9~)y zB%hssaeD|69{!%mJ`pu(@N)0&cSm}8XVH|Rk~>C<@mdAPX-$jkD@3HsQryu|9l^2a zeYz%Z$xsytcho47GdL)n{N5g2raJT1kS*EqOM+@;zNT7ptNX8C)cHM+8nY?bxCWg* z7xTmPT7aPV34pPSB{l#ihwvDPa3KlNGEp)U<<%Y4T`TY13_dvn__-j}-#>-I#Y(sxui3^{ zW%88~VOM;(K@%mj@cxD}F%sc31Bs-X@YxLI4~90YwcG3YqJaI_%U)@u!agA-T~$@> zqH&t25OJx;wPrwd-7;NUo5ktuWv#<*QtmeO{aCHkERto>5rD-D{aD#p`zcWGuodIk zRGCF+|C9TBt$wTBP9t{r_{?gmFxRTzjhl)TBAJ^BBKz?-Z(?U0rnpg)%K43@JRBxE zI<65_4UP1)G=ed}i3Q&Zx;n-MWA|W~@>niO6~t;q*3w?X9}QO0#e$uCSSkM~ z>8{NKwi@ID7SH{i57Jbojjf9K#l}xGdP$DnoG+U5k&>(pChOgkbbGfG|GW4W6nmeA zuLnLiYhBIYIWi87THU8QwD+_biAHAN zkrI?#PG|Ezl!N01a~y2!kJz3c23Wd$Ai7>)+ej$Z)&8wO)gp!ttgJ~1$MVV~DE#** z6?pTyx!M{6VdMLC=Ltq!l?1i;j?Kh!`?}&Vr!td}Xxu5< zNWxS(Nekw8jsexE%sITg851dhgssm{DqQNYX2zU!AdxC0C&^IgDyl~BNKoUF{wH^HL&X%K zv6X};c;zJa7(MJfIT~pA4^kXY-_SAI%N^k5(MQhzg>gZ>Z&UIAcca~aMqGnX4z$pP zzZUrad3_Ls9nrMWSbR~AJv1+iyPjt|A_k)_nm<1eHeOF;t*uqvaica?YXRnK2U0%z zi_i#a44F~am2DfxmtD);hV(M@8upsbfB!fJb679&ATZ!hFH#~#YT=`k?R5+>af;~G zBnpi$>Nd;b%&sc6RtT;-_nuz7tgR{tJl}R+k`n*6x>P_-OCMS|zHRszt}<}tp=fGq z$RbCX>j9$sbSiYd)sWeSA>B(+Eft5#U~0v z9_3Z#NEd@bHW{ZW{De`h^E%X|u!#N9`{>IaAg?hgO&u53(a#z)Ul@(|YP6iKS}s<)y*9lfHSii?O;V*9!L>gS2W;FW z0fJ!nO$GG=Fv0aS6?&8IcD~i)`=CO|y=+FKkA>GvUJ#i z8hV2ZzFAzbR?6O?5tIHt?{?XVUUjWHy+7M*d;Mx{+tK?_Cir$| zNab4v8JQ=xZarIRGJUNTi$Tz$gr`7ZmzQT`|5mrz^pI(*!wvJaVDNHrQ9yZw3qrfS z%j0#nx=+@*=erpg(VN8`Hi1%ZJ+bHD;Eays`JmW8F;cpySzlV<^i5{Ib-drxc;TKymNS<3PAP7y zKx|Uu{HHP<0>W(VETGyvQ@hGu<$k%EjUnLB0RayC%;&3Z-j_GoZUUstQNtXrrz6el z@-<|`?8koq9FFFnDXtIkcN_M-&Sm&CIOWbhZFo3wbc8&yd{AP2iWds^fYGeq{XOTp z+l79wSy^)nN55oT?w0>=6q@VQ73i`^HO%_yFnM`-C zO175&Hsk$5BmJnlccfGQDF)+@P7OmolsMe~1v!UAFkrR2Qiqi*`dher&E$9#wK1e2 zh#sHY*tdDU0&|s{fSVQ^b$mWb1b4N*l9F;NqOJRLci70c>P98R0uje(Jn6ry{qeFM zvqfv|udmW+e^x9clg3Rqzi|TfV_AeJZ?B8iEopO2WAUH>L)#8gMY1+S^x}aMvSC1k z6aeJ}jEnKJYvpdY-CM(7s0s?` z=;VEVRxUv>{qx%*za;5ghpRf5wI(a^_>_?4M~3MaUq6bO)#fUVwDdG9uFWlAH8reS zmH!xgCn%4n-)!oqBrUEW=}k^Dq^u=jNsCUtbXKt^IiT8TcbaxGF|X>sy*POkN5Yec zt&BTbU^A|19L7Q+N>!5Kq}%sH5&l_vla4qU$$Djbfj^;^A&xqOhJR&y%kNUc_w}*q z2!27U5N`+|sI}F#uE#Rjm6vrk^$M1XWg+JrU!F73Wm08MrMK1+e{ukuHQ`(Z=MzP- zGw1Wx)Cpos-Tyq29O%!p)!|a8V~~A#$nSKy|DCvpFPnw}qfw{3*6q4Fb^RAtqqY>j z=@&WGq^JoZNH#<cV70MtpQIJvk zo2IaB056;0%fX*ku-R#KU^gkMHg3j)&m$olQ1*EA>;TeP13h{Q_v$aD)TUD0rqx{C zZ$KpaSJGdG`qxo}^MCIgw7*CiAtfe}9vn$QI5ck2s1|?rTCU+gh&PwpCYfva(2ptC#zC(2#ZSLB%@UxcN$PG0E^VhBm4|W zK^IyjK!s7Fbkd+=q7BS!@R$bJHD2Vax0i3*y{;vnO7Ic^rxeE{#~JAv8D)> zS_Z?gk+E^R+tw7R|J!;N{@;UHzqv)ughFH45*tJ#-RdCr91sPQ76|_f$RXcDYEYYM z1wUFiIs9@mKKVJ-%tcZx2qhXaX042pYKOmw4QekcF^(9n2&e0$ zYl8-z8RMX+>h2>*<%Us0u{darBFR#{QCESerkdpp-Gr&dped1HIqWZp2A!%J8kkd| zpw&SGhTzjAX~m5GK}9pD5aYRRbm$L&=sXfCvT}0HFo*`UCnu`H|g5Q-yG8r9R_d~Q7860k#ycxH_GD<1Oz1Kcm7F_m9M_~afD_I_(&L3*zJ9ue>J z177yn1si6Md}1EyXDW;(pZTycc;U`rLqv|2a+G_v5qOl=Y&i*5H6(SoC1+zEe!GQi};)ntH&}234?HM(`#En~T zOHy@5^@uhwJbDD7y}qE#>Il7sC~890z6?h-+Q+9dJau_4_pPxN0w-qFOCLXlm>iQM zJ|4StB}TMu1I_TyeeJtF#cN_;D3;JMB10?D1OMLFoQCtPdmPvxEEQfU*K&Yir_ck; zeq#+ji57p6M8GHiqUx%OBHn+8@2!gz#0m^z)6A^8yx|@xknp?Blc(tr;gC$EM?vO- zdSX|dc7|tdr&?L61x-k<+s1LJNxJ9uuK2K$eUxz^JxJYTD$kvSP0isRY5scqPqYA# z4GC~dvZ^6I{5jUGt`=q2JlG=m^Yr)EAlB;S-yg>anLZNHn-Fm9@faYre5i^*pBi(M zNPi=!%bk0US?0zp(#800*(J0kGj=Db);qy@TlZ@Hw$^+9@4@T#R758Z>*f%cb!{Zs zV-Nfg67fwjexPaAM9a@CDM0o)x}L zSBR!D1FFX*{Z>tc*9*D6fQCTw2oaR2LLRI1f2y8HfVf61v?2iG-yI6|u07+N_FdxU3jh z+FUBx%p9!ov%~BtW(j!YYS^5g#o@ICl0<0ke}wJ)!2!_({b3$gR3%mkOZ5$ei9ZUA zsfCW)X_|iF=Sw$bJ#%ncoGtUR`fS`WIprUe2=>g-5@V6b`mz;~a3I3Kx<))0@=lNPzd{6r z4S%6Fn@=o6ngT=N?}eyBKgl8@+i_d)-20%bU1kz4rjHiIH0Y(G`@yD2$?{ZIMg#7o zKS<3`9vpp3+mamd()2kPaQ)68F;IxtWYhi(Fc;R5S&yH|@W9F~jx=~m_(dBi`^@8% z`&V*(4!)rSpW=2p51?Tw+(}&(RHII;aK9G<@W{bkHYx)&aYT?P>r0@FY1nxw5@kY? zCVAA)+%Rz(77#9~B-{gJ01uY96?39kG(`m)xBrj8hs4S@CiDz7EcIgF5&g_U#LFp} zfJQb^Lv&0-1ZX&qk{fbNZ;Y51bP0iI=|b298bjDVgrgAemasAY_HQAqjMVDo3n=eE z<#hBPq%1Qqef(VhcZns6VbuF{DW%6~M8?eb_gej=>1> zJWF#VDO51Is{ax@x&w>fA93}?ij2c42=q%Z>q8KUiqKzDY=9L%tn8KW4b_$bEKi*F z)C;NgN995Ss~PO}`-j@cozEBMx~DWIW^&r}mP6$Skw1l?i-*${vJ_vh?CXQfps~ z#3tv!`K>ROosaU2P|l?Wg>4nm*`UQ~^0`Fjo$OQcrhxFs9)8g;_!2OBr(!@(Z34DX z3M0*gJk3h@Piz^qP*5!L6~#s$Mm6ky<+p`;O<~5IX?ofq4HQNl!wLiEm*iD zXQi#wj=-X~%fae=4V!qAAIQ3wb*)h788C(?^`9@?8}-};wjTV{_tHFasNd70u(#N* zWkkHgoyjG1b>j-dW!83hgLLcKSQn;V88$PI!ZaCU+l-?FtYTO!P1u=sbHhR|hiwj| z|EmY)A%by3Wj?~ZeVtD(h)lmHwUo!eAaT0Dzg3wQB~IW?g$ARD>tlT>__#p5b_Ycj zJzpP)D(F^?V>#vP6n?b((d9e-=d}+k5+eg*-D*yO)5TlFt<4n$U~%HU-)tl8S8O=BnLP>^K{F*YF*n0oZV-^DWyk4-L83Y~;er2KRFZ#*CjiV5Hd z%H7UZJMCUO0BOUyI`!JZORbRC<8bhqjAp}PIW3lOsjjNB(r|ps-dezMcWAh(RWTtd zDVp4fY(_jz>;h3evN9w25cvzD_~_w9V$!6Mk+NnLpkN+1Wpnq&62hzhHL#zqYgU^m z&HCsg=lGS|;3HCFL~07wY`%{3%kO_iz%WWHy7$%yV$CZ>fgkhPGG{Zel*bndU;>ho zivBHozAgiX?|q$yQqH2>>OVG;Q*feemkFsF&Xq}~^$6f90o^|-P~)`|;d#c=f)|jSGp1J6aG3(Sm)>dxG}*zlPBO)mWr}GlQ}HRU_hbD>zVFoh%}|f<2HEMhGLEpb(UJAHu1G z;YemihBNmDxqJYX6&ET^j)4M381aQcec(wtIarMPN+hzdjTg^-cKegD z?L{~uCl+@|P}^L~H(qsVBLuQwF=@o=|34ZCn-!`oWrqbaRQ*(`#w>=Y#!#J$&wjs@ zn}$NoFv=K1Y2HYoMUH?kgkgRUn7KsWLbFC^l=fNv=EE(4Rzqhci`MK3uZe9{LQgaj zk`SedVdL=*kV<95S2jtY#1@eSr_e;%2m$|@r!QYp%ww3Eo1FgL`&jyg9)>iJ4U)gDM z$N#00DLc>RL(BhP3m`_TNMmN8q~}`KY+Dkcd3zXa5fRVZU^r1Z7?W1+QD7`y632D? zN_u`Kma?!+FIA4OwP?TLbBvLD`!6rm5Rs0IoVxE|yg8hC+O&ss?@0KDVIuYTTwxZa z`pld+*=}{=+rK>>wc2fsk%KZPahYj9$+EKb$1wuN(*Fkg?0fwm{`9{t*KLmlWq8i2 zDAek;dTDF%RB>=}>jT!Mk*QTjGr8HQ3l?8;`MmZc!j`g%_yI>?{pOO6N)!Zb?JVGE zTn|~2S2OzXueo%2a^iJ~_Rcxi94g`yg2%ssChpq0TAFtM*7H8dD{3sGT$kBV+&|KC zd2IkH8o=o}h&Fin1GH4U9{?%RMRMI7ww##EcbDPomcz8IE&ZtJm$~@2!;9D(NdaDl z6q>s7m+^SWQ?i0U`~L<8sZU#`${EH{Zb{x=evUh9L|Cpb)l@C>v357+*c|>&oCP{J z7O9b6@8-qbvkASDeNAI9mzi|`4GdP+R?E0G=kqzfOr+8am4DlB)-afK;kwxFHK1v+ ze#qmTJkm%_4U&_5?k>(JwzRSNZdx$*z>+CPH__h^d%A41-q=uC{atX?-SDmO8!sx& z*oIcO*LfblH=XQmX5fA^JuWb3j%gmb0_?*k2NS7kl*3`)Z~WGG2E*;SIS3=1>g_f= zYO=Z1G}JJ&0H}aj?QH_WdtF?`X44 zzb+)uSp2VNTh#fU_d|uVaei)X=EoDnoeyp;TkS4)USX5gxJ+`C>uA2W zU#QA;d;MMs?R%cG4c?BL*B{VJ(c^y1p@6oUo9s>xW5eH{U7r2k`ff7X+WT-==l|6E zoWxI`K1KNZ^6zx`WrdZ7LOwb>+*A(XUqvfv2pvepjh6(eJfwz!Igi`oa8=<8!mbEw zs8NH2DEZ!g17Zu%W$IQC^xiv5R5+;{ijsW-(DE!dNy9hr@4U8n>J5yN`8R{WC1Pft z$1gk!aYRq5YFL4l1V7rF%6L1UaTpCAZ;|U1xqJ`2e8a{tfmz2OFqikmM*qeAgMgnP zVbuoU2WoU+yKec;v+0Fz<)NOYix2N~qWE;GYXUZ+vZnc># z&)k$LaBh5Ms?BJ4INn^Ko6w>BPUPI!5q5imYr&-7(bv#Oyx#hiLg=HGwYS&v(Gt+? znl)vIakfyka{J?_VPOgFAO7$!ud^`L-G{;rA;#)Comk{eHk*1PG5ONa3M^~$=`4&7 zjN~02$I;2u3Vx~RQQ>YuF)gbk(-GNp_5+F^OL+Rbgd{t7iikmu2cXd{BahJ>(5dFy zb29zpwZ~uc$yhJGyk1x=VZeq;-0@6CCs^o)Bq4_@Bsp~K+__aLg+s$ZEF!h7yu@Yi zoBYuiVgbi}!>B62Ew@bL4KTj4UABLPLMk@WaQ6n_yx8(7=>j)DRnlkjW$#h-d)GUC zhCkdKO?Lw}%plEve&PqaMFZl3%IsD`$l*KxX0zwGr}axrQ1hU-TxCPie3CrjlCR(X zuVLgParq+%%-ap=+t;0-HG14mV2V-lsAiqJnt=M^weIcy4H_CX!To}7rcVTL$pJ+l z&XP}aaB+qctVsP_FbierD@lvMj0|D@c3`)@+3C*M5DFC~4K;o5=E}9zULyi-z-q$! z$)q`EtV3uI6No8Q+OV_qcylC*TFcp-t^wE;HFfkrvMN?1v`v{ErcM%s1hlwPnfilu^UqC zO`SK-J*|Rgd7J#t3THg_o7vo1b7MrY9BeK{VnJX;$pA|a`!(M0tp{v-Td!nQl~u$) zSUAXcou5FfT8W8?N{iSS+!wF<^3V7*F^vJJJga{+gCd9kQ;O6&-#bacCnW`UZ@@$H zvfCX9)l={TnhZG6!%nz>Uk*G{L0K7fs?342+5Y6dNU(5W!Qk1S==P;$pt=v_X`5?z zG@4=CJ-(e46~|=DduSJkZ1mxiBIBQ#uo#ZYB$4DRTv>2TX~v@>!48R7N`>Sw_O+ex ziraxOW}>y0wRO^wjZUPYt=YbNtM*4nB1pR=?r(CNtO97SCJmYb;ihoXu31%j_q}KB z)&AM0tytIQh|09g^o&uXyzA?QQ&!M>-0lzOD;@kKv5&_T!gv#OB7C1&k3IA`2#E8g z@Qe2%#W)BUu{?|s`X79)j?XqbJaza8CjTC9JamP`V(tdy?lwlv?-tc{{THL(-3oV zjz!`BKDD>AgH)yWE;{Y)mfn|t<)1=W;{+;F!I1xlD zeL@jJl&KbzDRF@PvYD0oOh}Nf&R{f+#REAPgBB|1!bD0hpZAzofcs;7HQ%_KM04k| zp{ynr(40kCcs-5qX`C zwKcZP7&`yC#U4grAsBlqIz2uC{gZ+Yuk*EIR+NRw%5N%>TU%S9s@0W}@g%Xs#z1*X z5nah$VYy*Xl5{J57s4+L%I&Gl-C2|!^TMFN>Z;oQK?_sZqRYYv74->vIiU)_L-^nc&aIcGidVqVOw zSwGFC@3lXBJ9By%o5UP>7FW9v=9oc?^<XB1t>uP1HD77jOPk*i9 z61PRu@$eQ_B`HO195k-=fkmTA$X56{K0BJy!zTrB4bfg;tvBD3qt1|{QZtu5p#OC% zg=|4A1?_1EQwRBM9(y7z^Y$lB_g z#3Tnbm+#pd2vg_Pjk<~nFrK}L~>@<_4-3!U77i*j<;bR zmW2w-b^VW=%a85xVGChiCz^J?qy-7rU{Ro1(e)_ym{Pa9-Y=E)IEoZ9Lt2 zyXWDt)Am1@rIv6um2Ud4Z6}6-!otFESq!4_nLo|t2nVwY3i2CL*Gn*K++FSTgY|^7 ziIGL$zkd%yB~6tkZ+6@I{!H|a&OZU!2QNq^MD@A6S>)(G0*#%7Ceoo8*k0_nuatlv zc@kVG)^!4tK_a6+T2U3Pkx~vNvZj&wOwO&4WTVAYrfWcyV%qfH=d^PFz0P?jdM{e! z!}iTkq`nY{`HB6DW$DO+Zf_$@SRPqOr%;yX8msF>F>6fLf{45QUqiB6pR!w2`nVL5 z$U!H=ctmyn93_Lc4gGc_3p>R|T3BoCLGmKjqA2heNwWGa;4f;sn%agyd1Tpl!1$S& zYg3y47+LX8_3cewPXk}=*>e!XFK=86>lfD`ahxJNTn}bP61p^Km_quw#lYsfJ<8Fj zMeKe$gNZ+SdwT=B0t36=1Akn*4E0QW^kVYKDSV}&h7KsP-f{2q*&=!9)!Vv(ydTkT zG-t}lFRtVor=ckr0fJ>bL3Ie~!DK(>t`NlBrH@Dryifb%l&aUhkj4!WO+44{cU9_u zmKH#zG`a%qzLE4qMYb5m0A!$EMVSGcpkVfh>EW$h&gvI3=>7RB9qGhlD|;IF3DWljXs<8 zzrT!ehEfZp91TXr7d<`XfbiSfoU<(7@(FMEou#?d>nqKP1Gi zEMX6%Q;EUo#%Jamf>R#w-nSPc)c2B}9Pd%-Hh=%U`?g4Uj8C%xkwY4LfwQ`o+>NM8 z7`>`_pvw*!zs#4x)S5* z^a2@FXfQXUlCfjSwPeK1os4CC&97J3ptnM%&?tuKu+SkG0?~L00Hflbc?r@B<={T; zM)G^I1~jkRuExemQv5qOJzAK5X6wGprDj1A$say-iw*o~rH`u@QRMn^K3OYtxnDskT&|!o=D&9+dc6Kt zAAwR6GP*V#0!pf?D}zx>DZpNpM27}VX8PEVKX6+eb4!@!VKd4G6S4@gdlefsUG0TJ z2{Hk@fO*p|S@xZe`}aa{xz^_-3`|UJ`ndVCv$LyQgrU!H>88iC2VIw8S@d#PZT)y9 zs&4w59aZ;A-nXU{)D9uyX?WEYK33T{i&?S2lm6QxxJ+3o4y(NDZ)i2%cMnsryiTbnKX=*Gm>pPh^xd1^A zXI3o^PpND)7*XKXv|~aDqJWT1k8)?vXmKkn^3yV5M_)fCHBfCP$8jY>k%6IttZWj|1cP}msiYhF_!J~jSW9kzgSY`{BCi`3< z0e`XrIfdEFIFgqRX)ot8|3}O&T{7mmB^B@4&vuhLK-2n@?rwKZcT>p8laUf)7uLW- z43DD%(1Uoidyv|+W~WN5(>)7zd_Ipz(i&Dv5T$%8l%;lM#UD&BhYg1%P%fk*Lj{oc z*}5a@En@q4T(ij@r`2Lk;d2fHW-$U&L9=J#tLlHvZ!0HMi+?4%N%wf88 z+jaLVQQ7P5oXsTz@AJcRO%My#=4ks*K6gt!i5TT* zhv8torH1Bab|nSecq+8@hG;mZx88R0wv^1&&iD~wYcZgzek||SC8o&Zc6;9k_rdc) zV<${r2!LDW^I`dxIh*g>hJE+_DY(xICQM8x_e8YO?rWC~GRW}{ZD zOcEy2>O5SP^Ab5XG9XPGd$)NkxZtw$bd+abv30Zg!~Nam@e$0kcij$S;*;xz2do(^UXCp7W!inJaYu`k+?nsDi|4UUm6gt@ozIc%%2ASr_*!F zB5#daUD$V)miZsQ+wGfNTo5EH5eEL}4-J2gg$Rs_WR>URY_2s0P|?z`ADiC4_5n%4 z=?po$%dz<1MkSCZ8}>+*Hhbs-IC-VGU^MUo~E9!*Ey=H%JPTI;ZEvbwmq#U zwex_IDd6c0*jO*}+LzbYH}jkdxEAE|td_NLDDOC!&fS%`8?tqoIOVKmO&Gq-Jzr0H|uUQfN~+coPwiN6PT| zb8Fs|TOc}gKqAIb4vZc;+|W!k8Z>$!=Aqh5#cw*%-4cERugzYSfu8G#^m*011q)t8 zO%F&`g_iSbGV+?Ta!4`>RndY>td@vBiKKoX-m~s|In92TNfz5^fP(;ix#IT+M?mioVqCkEe^zp7Jro$ICdx+cfY&Xdl%K#k{BT+2 z|Euc|_v6005dRY$#%RsHjWegiS7@{wWIc&bVibw6@jG+@;Xe^DR20OZDOuYI?k_i5 z_jp9!WZ4r~&+vDBat1~*Y91BJwvh)RfSf2Yp@oQU+X4hFnVp3Jd|44gpqd&aE_7K; zRRT66w?SH&r(J51*UfU$j<>MXwF#?O0;gO3l5N2FPa}a4Jt(bhh_8>IATjd7sj`fQ z|NVz?#yU!@L3eU2I1``4Lyw(GrQU*+JtB(8iJj zqL4ROv4JMPF=OcKzhB$AO2XY&b-GlZ?rnCZFvvfgZy~q(wKp;x{0VJbFx+f>dtMl$ zeHkh*XUJP=o?MtgI+s;LP!$CDhzx&%XHd`y=AXSpzbk=B;jWjTl1Knek={YKI&nwQBR%@oH=)QJ0;v%&&XS