From 45236257ddd7b4f2a5fdbe90c9cf77a3dd9ae526 Mon Sep 17 00:00:00 2001 From: Andrei Ionescu Date: Sat, 17 Apr 2021 15:32:23 +0300 Subject: [PATCH 1/7] Support filter on indexes with nested fields --- .../{RuleUtils.scala => BaseRuleHelper.scala} | 56 +- .../index/rules/FilterIndexRule.scala | 93 ++-- .../index/rules/JoinIndexRule.scala | 31 +- .../index/rules/NestedRuleHelper.scala | 494 ++++++++++++++++++ .../hyperspace/index/rules/PlanUtils.scala | 198 +++++++ .../hyperspace/util/ResolverUtils.scala | 229 ++++---- .../index/E2EHyperspaceRulesTest.scala | 41 +- .../index/rules/RuleUtilsTest.scala | 36 +- 8 files changed, 993 insertions(+), 185 deletions(-) rename src/main/scala/com/microsoft/hyperspace/index/rules/{RuleUtils.scala => BaseRuleHelper.scala} (96%) create mode 100644 src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelper.scala similarity index 96% rename from src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala rename to src/main/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelper.scala index d9e0f6bf8..99ce39dab 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelper.scala @@ -35,7 +35,7 @@ import com.microsoft.hyperspace.index.plans.logical.{BucketUnion, IndexHadoopFsR import com.microsoft.hyperspace.index.sources.FileBasedRelation import com.microsoft.hyperspace.util.HyperspaceConf -object RuleUtils { +class BaseRuleHelper(spark: SparkSession) { /** * Filter the given candidate indexes by matching signatures and index status. @@ -44,13 +44,11 @@ object RuleUtils { * index source files and the input files of the given plan. If there are some common * files, the index is considered as a candidate. * - * @param spark Spark Session. * @param indexes List of available indexes. * @param relation Relation with which given indexes are compared. * @return Active indexes built for this plan. */ def getCandidateIndexes( - spark: SparkSession, indexes: Seq[IndexLogEntry], relation: FileBasedRelation): Seq[IndexLogEntry] = { // Map of a signature provider to a signature generated for the given plan. @@ -131,7 +129,6 @@ object RuleUtils { } def prepareHybridScanCandidateSelection( - spark: SparkSession, plan: LogicalPlan, indexes: Seq[IndexLogEntry]): Unit = { assert(HyperspaceConf.hybridScanEnabled(spark)) @@ -153,7 +150,7 @@ object RuleUtils { // TODO: Duplicate listing files for the given relation as in // [[transformPlanToUseHybridScan]] // See https://github.com/microsoft/hyperspace/issues/160 - prepareHybridScanCandidateSelection(spark, relation.plan, indexes) + prepareHybridScanCandidateSelection(relation.plan, indexes) indexes.filter(_.created).flatMap(getHybridScanCandidate) } else { @@ -161,17 +158,6 @@ object RuleUtils { } } - /** - * Check if an index was applied the given relation or not. - * This can be determined by an identifier in [[FileBasedRelation]]'s options. - * - * @param relation FileBasedRelation to check if an index is already applied. - * @return true if an index is applied to the given relation. Otherwise false. - */ - def isIndexApplied(relation: FileBasedRelation): Boolean = { - relation.options.exists(_.equals(IndexConstants.INDEX_RELATION_IDENTIFIER)) - } - /** * Transform the current plan to utilize the given index. * @@ -183,7 +169,6 @@ object RuleUtils { * - We know for sure the index which can be used to transform the plan. * - The plan should be linear and include one supported relation. * - * @param spark Spark session. * @param index Index used in transformation of plan. * @param plan Current logical plan. * @param useBucketSpec Option whether to use BucketSpec for reading index data. @@ -191,13 +176,12 @@ object RuleUtils { * @return Transformed plan. */ def transformPlanToUseIndex( - spark: SparkSession, index: IndexLogEntry, plan: LogicalPlan, useBucketSpec: Boolean, useBucketUnionForAppended: Boolean): LogicalPlan = { // Check pre-requisite. - val relation = getRelation(spark, plan) + val relation = getRelation(plan) assert(relation.isDefined) // If there is no change in source data files, the index can be applied by @@ -212,9 +196,9 @@ object RuleUtils { lazy val isSourceUpdated = index.hasSourceUpdate val transformed = if (hybridScanRequired || isSourceUpdated) { - transformPlanToUseHybridScan(spark, index, plan, useBucketSpec, useBucketUnionForAppended) + transformPlanToUseHybridScan(index, plan, useBucketSpec, useBucketUnionForAppended) } else { - transformPlanToUseIndexOnlyScan(spark, index, plan, useBucketSpec) + transformPlanToUseIndexOnlyScan(index, plan, useBucketSpec) } assert(!transformed.equals(plan)) transformed @@ -227,7 +211,7 @@ object RuleUtils { * @return If the plan is linear and the relation node is supported, the [[FileBasedRelation]] * object that wraps the relation node. Otherwise None. */ - def getRelation(spark: SparkSession, plan: LogicalPlan): Option[FileBasedRelation] = { + def getRelation(plan: LogicalPlan): Option[FileBasedRelation] = { val provider = Hyperspace.getContext(spark).sourceProviderManager val leaves = plan.collectLeaves() if (leaves.size == 1 && provider.isSupportedRelation(leaves.head)) { @@ -244,14 +228,12 @@ object RuleUtils { * * NOTE: This method currently only supports transformation of nodes with supported relations. * - * @param spark Spark session. * @param index Index used in transformation of plan. * @param plan Current logical plan. * @param useBucketSpec Option whether to use BucketSpec for reading index data. * @return Transformed logical plan that leverages an index. */ - private def transformPlanToUseIndexOnlyScan( - spark: SparkSession, + protected[rules] def transformPlanToUseIndexOnlyScan( index: IndexLogEntry, plan: LogicalPlan, useBucketSpec: Boolean): LogicalPlan = { @@ -290,15 +272,13 @@ object RuleUtils { * eligible and we reconstruct new plans for the appended files so as to merge with * bucketed index data correctly. * - * @param spark Spark session. * @param index Index used in transformation of plan. * @param plan Current logical plan. * @param useBucketSpec Option whether to use BucketSpec for reading index data. * @param useBucketUnionForAppended Option whether to use BucketUnion to merge appended data. * @return Transformed logical plan that leverages an index and merges appended data. */ - private def transformPlanToUseHybridScan( - spark: SparkSession, + protected[rules] def transformPlanToUseHybridScan( index: IndexLogEntry, plan: LogicalPlan, useBucketSpec: Boolean, @@ -416,7 +396,7 @@ object RuleUtils { // For more details, see https://github.com/microsoft/hyperspace/issues/150. val planForAppended = - transformPlanToReadAppendedFiles(spark, index, plan, unhandledAppendedFiles) + transformPlanToReadAppendedFiles(index, plan, unhandledAppendedFiles) if (useBucketUnionForAppended && useBucketSpec) { // If Bucketing information of the index is used to read the index data, we need to // shuffle the appended data in the same way to correctly merge with bucketed index data. @@ -446,14 +426,12 @@ object RuleUtils { * The result will be merged with the plan which is reading index data * by using [[BucketUnion]] or [[Union]]. * - * @param spark Spark session. * @param index Index used in transformation of plan. * @param originalPlan Original plan. * @param filesAppended Appended files to the source relation. * @return Transformed linear logical plan for appended files. */ - private def transformPlanToReadAppendedFiles( - spark: SparkSession, + protected[rules] def transformPlanToReadAppendedFiles( index: IndexLogEntry, originalPlan: LogicalPlan, filesAppended: Seq[Path]): LogicalPlan = { @@ -567,3 +545,17 @@ object RuleUtils { shuffled } } + +object BaseRuleHelper extends Serializable { + + /** + * Check if an index was applied the given relation or not. + * This can be determined by an identifier in [[FileBasedRelation]]'s options. + * + * @param relation FileBasedRelation to check if an index is already applied. + * @return true if an index is applied to the given relation. Otherwise false. + */ + def isIndexApplied(relation: FileBasedRelation): Boolean = { + relation.options.exists(_.equals(IndexConstants.INDEX_RELATION_IDENTIFIER)) + } +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala index 945105924..0b04e5015 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -18,7 +18,7 @@ package com.microsoft.hyperspace.index.rules import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.CleanupAliases -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule @@ -26,6 +26,7 @@ import com.microsoft.hyperspace.{ActiveSparkSession, Hyperspace} import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.IndexLogEntry import com.microsoft.hyperspace.index.rankers.FilterIndexRanker +import com.microsoft.hyperspace.index.rules.PlanUtils._ import com.microsoft.hyperspace.index.sources.FileBasedRelation import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEventLogging, HyperspaceIndexUsageEvent} import com.microsoft.hyperspace.util.{HyperspaceConf, ResolverUtils} @@ -53,19 +54,24 @@ object FilterIndexRule case ExtractFilterNode(originalPlan, filter, outputColumns, filterColumns) => try { val candidateIndexes = - findCoveringIndexes(filter, outputColumns, filterColumns) + findCoveringIndexes(filter, outputColumns, filterColumns, plan) FilterIndexRanker.rank(spark, filter, candidateIndexes) match { case Some(index) => - // As FilterIndexRule is not intended to support bucketed scan, we set - // useBucketUnionForAppended as false. If it's true, Hybrid Scan can cause - // unnecessary shuffle for appended data to apply BucketUnion for merging data. - val transformedPlan = - RuleUtils.transformPlanToUseIndex( - spark, - index, - originalPlan, - useBucketSpec = HyperspaceConf.useBucketSpecForFilterRule(spark), - useBucketUnionForAppended = false) + // Detect which whether the index contains nested fields. + val indexHasNestedColumns = (index.indexedColumns ++ index.includedColumns) + .exists(ResolverUtils.ResolvedColumn(_).isNested) + val ruleHelper = if (indexHasNestedColumns) { + new NestedRuleHelper(spark) + } else { + new BaseRuleHelper(spark) + } + + val transformedPlan = ruleHelper.transformPlanToUseIndex( + index, + originalPlan, + useBucketSpec = HyperspaceConf.useBucketSpecForFilterRule(spark), + useBucketUnionForAppended = false) + logEvent( HyperspaceIndexUsageEvent( AppInfo( @@ -99,8 +105,10 @@ object FilterIndexRule private def findCoveringIndexes( filter: Filter, outputColumns: Seq[String], - filterColumns: Seq[String]): Seq[IndexLogEntry] = { - RuleUtils.getRelation(spark, filter) match { + filterColumns: Seq[String], + plan: LogicalPlan): Seq[IndexLogEntry] = { + val ruleHelper = new BaseRuleHelper(spark) + ruleHelper.getRelation(filter) match { case Some(r) => val indexManager = Hyperspace .getContext(spark) @@ -111,20 +119,40 @@ object FilterIndexRule // See https://github.com/microsoft/hyperspace/issues/65 val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) - val candidateIndexes = allIndexes.filter { index => - indexCoversPlan( - outputColumns, - filterColumns, - index.indexedColumns, - index.includedColumns) + val resolvedOutputColumnsOpt = ResolverUtils.resolve( + spark, + outputColumns, + plan, + ResolverUtils.resolveWithChildren, + throwIfNotInSchema = false) + val resolvedFilterColumnsOpt = ResolverUtils.resolve( + spark, + filterColumns, + plan, + ResolverUtils.resolveWithChildren, + throwIfNotInSchema = false) + + (resolvedOutputColumnsOpt, resolvedFilterColumnsOpt) match { + case (Some(resolvedOutputColumns), Some(resolvedFilterColumns)) => + val candidateIndexes = allIndexes.filter { index => + indexCoversPlan( + resolvedOutputColumns.map(_.normalizedName), + resolvedFilterColumns.map(_.normalizedName), + index.indexedColumns, + index.includedColumns) + } + + // Get candidate via file-level metadata validation. This is performed after pruning + // by column schema, as this might be expensive when there are numerous files in the + // relation or many indexes to be checked. + ruleHelper.getCandidateIndexes(candidateIndexes, r) + + case _ => + Seq.empty } - - // Get candidate via file-level metadata validation. This is performed after pruning - // by column schema, as this might be expensive when there are numerous files in the - // relation or many indexes to be checked. - RuleUtils.getCandidateIndexes(spark, candidateIndexes, r) - - case None => Nil // There is zero or more than one supported relations in Filter's sub-plan. + case _ => + // There is zero or more than one supported relations in Filter's sub-plan. + Seq.empty } } @@ -136,7 +164,6 @@ object FilterIndexRule * @param filterColumns List of columns in filter predicate. * @param indexedColumns List of indexed columns (e.g. from an index being checked) * @param includedColumns List of included columns (e.g. from an index being checked) - * @param fileFormat FileFormat for input relation in original logical plan. * @return 'true' if * 1. Index fully covers output and filter columns, and * 2. Filter predicate contains first column in index's 'indexed' columns. @@ -164,19 +191,19 @@ object ExtractFilterNode { def unapply(plan: LogicalPlan): Option[returnType] = plan match { case project @ Project(_, filter @ Filter(condition: Expression, ExtractRelation(relation))) - if !RuleUtils.isIndexApplied(relation) => + if !BaseRuleHelper.isIndexApplied(relation) => val projectColumnNames = CleanupAliases(project) .asInstanceOf[Project] .projectList - .map(_.references.map(_.asInstanceOf[AttributeReference].name)) + .map(i => extractNamesFromExpression(i).toKeep) .flatMap(_.toSeq) - val filterColumnNames = condition.references.map(_.name).toSeq + val filterColumnNames = extractNamesFromExpression(condition).toKeep.toSeq Some(project, filter, projectColumnNames, filterColumnNames) case filter @ Filter(condition: Expression, ExtractRelation(relation)) - if !RuleUtils.isIndexApplied(relation) => - val relationColumnsName = relation.output.map(_.name) + if !BaseRuleHelper.isIndexApplied(relation) => + val relationColumnsName = relation.plan.output.map(_.name) val filterColumnNames = condition.references.map(_.name).toSeq Some(filter, filter, relationColumnsName, filterColumnNames) diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala index 0a0d25272..3db907829 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -61,17 +61,16 @@ object JoinIndexRule getBestIndexPair(l, r, condition) .map { case (lIndex, rIndex) => + val ruleHelper = new BaseRuleHelper(spark) val updatedPlan = join .copy( - left = RuleUtils.transformPlanToUseIndex( - spark, + left = ruleHelper.transformPlanToUseIndex( lIndex, l, useBucketSpec = true, useBucketUnionForAppended = true), - right = RuleUtils.transformPlanToUseIndex( - spark, + right = ruleHelper.transformPlanToUseIndex( rIndex, r, useBucketSpec = true, @@ -109,11 +108,12 @@ object JoinIndexRule private def isApplicable(l: LogicalPlan, r: LogicalPlan, condition: Expression): Boolean = { // The given plan is eligible if it is supported and index has not been applied. def isEligible(optRel: Option[FileBasedRelation]): Boolean = { - optRel.map(!RuleUtils.isIndexApplied(_)).getOrElse(false) + optRel.exists(!BaseRuleHelper.isIndexApplied(_)) } - lazy val optLeftRel = RuleUtils.getRelation(spark, l) - lazy val optRightRel = RuleUtils.getRelation(spark, r) + lazy val ruleHelper = new BaseRuleHelper(spark) + lazy val optLeftRel = ruleHelper.getRelation(l) + lazy val optRightRel = ruleHelper.getRelation(r) isJoinConditionSupported(condition) && isPlanLinear(l) && isPlanLinear(r) && @@ -285,6 +285,7 @@ object JoinIndexRule right: LogicalPlan, joinCondition: Expression): Option[(IndexLogEntry, IndexLogEntry)] = { val indexManager = Hyperspace.getContext(spark).indexCollectionManager + val ruleHelper = new BaseRuleHelper(spark) // TODO: the following check only considers indexes in ACTIVE state for usage. Update // the code to support indexes in transitioning states as well. @@ -294,10 +295,10 @@ object JoinIndexRule // TODO: we can write an extractor that applies `isApplicable` so that we don't have to // get relations twice. Note that `getRelation` should always succeed since this has // been already checked in `isApplicable`. - val leftRelation = RuleUtils.getRelation(spark, left).get - val rightRelation = RuleUtils.getRelation(spark, right).get - val lBaseAttrs = leftRelation.output.map(_.name) - val rBaseAttrs = rightRelation.output.map(_.name) + val leftRelation = ruleHelper.getRelation(left).get + val rightRelation = ruleHelper.getRelation(right).get + val lBaseAttrs = leftRelation.plan.output.map(_.name) + val rBaseAttrs = rightRelation.plan.output.map(_.name) // Map of left resolved columns with their corresponding right resolved // columns from condition. @@ -316,14 +317,14 @@ object JoinIndexRule val lUsable = getUsableIndexes(allIndexes, lRequiredIndexedCols, lRequiredAllCols) val rUsable = getUsableIndexes(allIndexes, rRequiredIndexedCols, rRequiredAllCols) - val leftRel = RuleUtils.getRelation(spark, left).get - val rightRel = RuleUtils.getRelation(spark, right).get + val leftRel = ruleHelper.getRelation(left).get + val rightRel = ruleHelper.getRelation(right).get // Get candidate via file-level metadata validation. This is performed after pruning // by column schema, as this might be expensive when there are numerous files in the // relation or many indexes to be checked. - val lIndexes = RuleUtils.getCandidateIndexes(spark, lUsable, leftRel) - val rIndexes = RuleUtils.getCandidateIndexes(spark, rUsable, rightRel) + val lIndexes = ruleHelper.getCandidateIndexes(lUsable, leftRel) + val rIndexes = ruleHelper.getCandidateIndexes(rUsable, rightRel) val compatibleIndexPairs = getCompatibleIndexPairs(lIndexes, rIndexes, lRMap) diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala new file mode 100644 index 000000000..73c882d82 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala @@ -0,0 +1,494 @@ +/* + * Copyright (2021) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index.rules + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BinaryOperator, ExprId, GetStructField, In, IsNotNull, Literal, NamedExpression, Not} +import org.apache.spark.sql.catalyst.optimizer.OptimizeIn +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project, Union} +import org.apache.spark.sql.execution.datasources.InMemoryFileIndex +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.types.{LongType, StructType} + +import com.microsoft.hyperspace.Hyperspace +import com.microsoft.hyperspace.index.{FileInfo, IndexConstants, IndexLogEntry, IndexLogEntryTags} +import com.microsoft.hyperspace.index.plans.logical.{BucketUnion, IndexHadoopFsRelation} +import com.microsoft.hyperspace.index.rules.PlanUtils._ +import com.microsoft.hyperspace.util.{HyperspaceConf, ResolverUtils} + +class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { + + /** + * Transform the current plan to utilize index. + * The transformed plan reads data from indexes instead of the source relations. + * Bucketing information of the index is retained if useBucketSpec is true. + * + * NOTE: This method currently only supports transformation of nodes with supported relations. + * + * @param index Index used in transformation of plan. + * @param plan Current logical plan. + * @param useBucketSpec Option whether to use BucketSpec for reading index data. + * @return Transformed logical plan that leverages an index. + */ + override protected[rules] def transformPlanToUseIndexOnlyScan( + index: IndexLogEntry, + plan: LogicalPlan, + useBucketSpec: Boolean): LogicalPlan = { + val provider = Hyperspace.getContext(spark).sourceProviderManager + // Note that depending on the case we transform only the base relation + // and sometimes other portions of the plan (e.g., filters). For instance, + // given the following input plan: + // Project(A,B) -> Filter(C = 10) -> Scan (A,B,C,D,E) + // in the presence of a suitable index, we will transform to: + // Project(A,B) -> Filter(C = 10) -> Index Scan (A,B,C) + // In the case of nested fields we will transform the project and + // filter nodes too. + plan transformUp { + case l: LeafNode if provider.isSupportedRelation(l) => + val relation = provider.getRelation(l) + val location = index.withCachedTag(IndexLogEntryTags.INMEMORYFILEINDEX_INDEX_ONLY) { + new InMemoryFileIndex(spark, index.content.files, Map(), None) + } + + val newSchema = StructType( + index.schema.filter(i => relation.plan.schema.exists(j => i.name.contains(j.name)))) + + val indexFsRelation = new IndexHadoopFsRelation( + location, + new StructType(), + newSchema, + if (useBucketSpec) Some(index.bucketSpec) else None, + new ParquetFileFormat, + Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark, index) + + val resolvedFields = ResolverUtils.resolve( + spark, + (index.indexedColumns ++ index.includedColumns) + .map(ResolverUtils.ResolvedColumn(_).name), + relation.plan) + val updatedOutput = + if (resolvedFields.isDefined && resolvedFields.get.exists(_.isNested)) { + indexFsRelation.schema.flatMap { s => + relation.plan.output + .find { a => + ResolverUtils.ResolvedColumn(s.name).name.startsWith(a.name) + } + .map { a => + AttributeReference(s.name, s.dataType, a.nullable, a.metadata)( + NamedExpression.newExprId, + a.qualifier) + } + } + } else { + relation.plan.output + .filter(attr => indexFsRelation.schema.fieldNames.contains(attr.name)) + .map(_.asInstanceOf[AttributeReference]) + } + relation.createLogicalRelation(indexFsRelation, updatedOutput) + + // Given that the index may have top level field for a nested one + // it is needed to transform the projection to use that index field + case p: Project if hasNestedColumns(p, index) => + transformProject(p) + + // Given that the index may have top level field for a nested one + // it is needed to transform the filter to use that index field + case f: Filter if hasNestedColumns(f, index) => + transformFilter(f) + } + } + + /** + * Transform the current plan to utilize the given index along with newly appended source files. + * + * With HybridScan, indexes with newly appended files to its source relation are also + * eligible and we reconstruct new plans for the appended files so as to merge with + * bucketed index data correctly. + * + * @param index Index used in transformation of plan. + * @param plan Current logical plan. + * @param useBucketSpec Option whether to use BucketSpec for reading index data. + * @param useBucketUnionForAppended Option whether to use BucketUnion to merge appended data. + * @return Transformed logical plan that leverages an index and merges appended data. + */ + override protected[rules] def transformPlanToUseHybridScan( + index: IndexLogEntry, + plan: LogicalPlan, + useBucketSpec: Boolean, + useBucketUnionForAppended: Boolean): LogicalPlan = { + val provider = Hyperspace.getContext(spark).sourceProviderManager + var unhandledAppendedFiles: Seq[Path] = Nil + // Get transformed plan with index data and appended files if applicable. + val indexPlan = plan transformUp { + // Use transformUp here as currently one relation is allowed (pre-requisite). + // The transformed plan will have LogicalRelation as a child; for example, LogicalRelation + // can be transformed to 'Project -> Filter -> LogicalRelation'. Thus, with transformDown, + // it will be matched again and transformed recursively which causes stack overflow exception. + case l: LeafNode if provider.isSupportedRelation(l) => + val relation = provider.getRelation(l) + val (filesDeleted, filesAppended) = + if (!HyperspaceConf.hybridScanEnabled(spark) && index.hasSourceUpdate) { + // If the index contains the source update info, it means the index was validated + // with the latest signature including appended files and deleted files, but + // index data is not updated with those files. Therefore, we need to handle + // appendedFiles and deletedFiles in IndexLogEntry. + (index.deletedFiles, index.appendedFiles.map(f => new Path(f.name)).toSeq) + } else { + val curFiles = relation.allFiles.map(f => + FileInfo(f, index.fileIdTracker.addFile(f), asFullPath = true)) + if (HyperspaceConf.hybridScanDeleteEnabled(spark) && index.hasLineageColumn) { + val (exist, nonExist) = curFiles.partition(index.sourceFileInfoSet.contains) + val filesAppended = nonExist.map(f => new Path(f.name)) + if (exist.length < index.sourceFileInfoSet.size) { + (index.sourceFileInfoSet -- exist, filesAppended) + } else { + (Nil, filesAppended) + } + } else { + // Append-only implementation of getting appended files for efficiency. + // It is guaranteed that there is no deleted files via the condition + // 'deletedCnt == 0 && commonCnt > 0' in isHybridScanCandidate function. + ( + Nil, + curFiles.filterNot(index.sourceFileInfoSet.contains).map(f => new Path(f.name))) + } + } + + val filesToRead = { + if (useBucketSpec || !index.hasParquetAsSourceFormat || filesDeleted.nonEmpty || + relation.partitionSchema.nonEmpty) { + // Since the index data is in "parquet" format, we cannot read source files + // in formats other than "parquet" using one FileScan node as the operator requires + // files in one homogenous format. To address this, we need to read the appended + // source files using another FileScan node injected into the plan and subsequently + // merge the data into the index data. Please refer below [[Union]] operation. + // In case there are both deleted and appended files, we cannot handle the appended + // files along with deleted files as source files do not have the lineage column which + // is required for excluding the index data from deleted files. + // If the source relation is partitioned, we cannot read the appended files with the + // index data as the schema of partitioned files are not equivalent to the index data. + unhandledAppendedFiles = filesAppended + index.content.files + } else { + // If BucketSpec of index data isn't used (e.g., in the case of FilterIndex currently) + // and the source format is parquet, we could read the appended files along + // with the index data. + index.content.files ++ filesAppended + } + } + + // In order to handle deleted files, read index data with the lineage column so that + // we could inject Filter-Not-In conditions on the lineage column to exclude the indexed + // rows from the deleted files. + val newSchema = StructType( + index.schema.filter(s => + relation.plan.schema.contains(s) || (filesDeleted.nonEmpty && s.name.equals( + IndexConstants.DATA_FILE_NAME_ID)))) + + def fileIndex: InMemoryFileIndex = { + new InMemoryFileIndex(spark, filesToRead, Map(), None) + } + + val newLocation = if (filesToRead.length == index.content.files.size) { + index.withCachedTag(IndexLogEntryTags.INMEMORYFILEINDEX_INDEX_ONLY)(fileIndex) + } else { + index.withCachedTag(plan, IndexLogEntryTags.INMEMORYFILEINDEX_HYBRID_SCAN)(fileIndex) + } + + val indexFsRelation = new IndexHadoopFsRelation( + newLocation, + new StructType(), + newSchema, + if (useBucketSpec) Some(index.bucketSpec) else None, + new ParquetFileFormat, + Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark, index) + + val updatedOutput = relation.plan.output + .filter(attr => indexFsRelation.schema.fieldNames.contains(attr.name)) + .map(_.asInstanceOf[AttributeReference]) + + if (filesDeleted.isEmpty) { + relation.createLogicalRelation(indexFsRelation, updatedOutput) + } else { + val lineageAttr = AttributeReference(IndexConstants.DATA_FILE_NAME_ID, LongType)() + val deletedFileIds = filesDeleted.map(f => Literal(f.id)).toArray + val rel = + relation.createLogicalRelation(indexFsRelation, updatedOutput ++ Seq(lineageAttr)) + val filterForDeleted = Filter(Not(In(lineageAttr, deletedFileIds)), rel) + Project(updatedOutput, OptimizeIn(filterForDeleted)) + } + } + + if (unhandledAppendedFiles.nonEmpty) { + // If there are unhandled appended files, we need to create additional plans + // by the following steps: + // Step 1) Generate a plan (planForAppended) from the current plan to read + // the appended files similar to indexPlan. + // Step 2) If Shuffle is required, perform shuffle for the plan. + // Step 3) Merge both indexPlan and planForAppended by using [[BucketUnion]] or [[Union]]. + // For more details, see https://github.com/microsoft/hyperspace/issues/150. + + val planForAppended = + transformPlanToReadAppendedFiles(index, plan, unhandledAppendedFiles) + if (useBucketUnionForAppended && useBucketSpec) { + // If Bucketing information of the index is used to read the index data, we need to + // shuffle the appended data in the same way to correctly merge with bucketed index data. + + // Although only numBuckets of BucketSpec is used in BucketUnion*, bucketColumnNames + // and sortColumnNames are shown in plan string. So remove sortColumnNames to avoid + // misunderstanding. + val bucketSpec = index.bucketSpec.copy(sortColumnNames = Nil) + + // Merge index plan & newly shuffled plan by using bucket-aware union. + BucketUnion( + Seq(indexPlan, transformPlanToShuffleUsingBucketSpec(bucketSpec, planForAppended)), + bucketSpec) + } else { + // If bucketing is not necessary (e.g. FilterIndexRule), we use [[Union]] to merge + // the appended data without additional shuffle. + Union(indexPlan, planForAppended) + } + } else { + indexPlan + } + } + + /** + * Transform the current plan to read the given appended source files. + * + * The result will be merged with the plan which is reading index data + * by using [[BucketUnion]] or [[Union]]. + * + * @param index Index used in transformation of plan. + * @param originalPlan Original plan. + * @param filesAppended Appended files to the source relation. + * @return Transformed linear logical plan for appended files. + */ + override protected[rules] def transformPlanToReadAppendedFiles( + index: IndexLogEntry, + originalPlan: LogicalPlan, + filesAppended: Seq[Path]): LogicalPlan = { + val provider = Hyperspace.getContext(spark).sourceProviderManager + // Transform the relation node to include appended files. + val planForAppended = originalPlan transformDown { + case l: LeafNode if provider.isSupportedRelation(l) => + val relation = provider.getRelation(l) + val options = relation.partitionBasePath + .map { basePath => + // Set "basePath" so that partitioned columns are also included in the output schema. + Map("basePath" -> basePath) + } + .getOrElse(Map()) + + val newLocation = index.withCachedTag( + originalPlan, + IndexLogEntryTags.INMEMORYFILEINDEX_HYBRID_SCAN_APPENDED) { + new InMemoryFileIndex(spark, filesAppended, options, None) + } + // Set the same output schema with the index plan to merge them using BucketUnion. + // Include partition columns for data loading. + val partitionColumns = relation.partitionSchema.map(_.name) + val updatedSchema = StructType(relation.plan.schema.filter(col => + index.schema.contains(col) || relation.partitionSchema.contains(col))) + val updatedOutput = relation.plan.output + .filter(attr => + index.schema.fieldNames.contains(attr.name) || partitionColumns.contains(attr.name)) + .map(_.asInstanceOf[AttributeReference]) + val newRelation = relation.createHadoopFsRelation( + newLocation, + updatedSchema, + relation.options + IndexConstants.INDEX_RELATION_IDENTIFIER) + relation.createLogicalRelation(newRelation, updatedOutput) + } + assert(!originalPlan.equals(planForAppended)) + planForAppended + } + + /** + * The method transforms the project part of a plan to support indexes on + * nested fields. + * + * For example, given the following query: + * {{{ + * df + * .filter("nested.leaf.cnt > 10 and nested.leaf.id == 'leaf_id9'") + * .select("Date", "nested.leaf.cnt") + * }}} + * + * Having this simple projection: + * {{{ + * Project [Date#100, nested#102.leaf.cnt] + * }}} + * + * The projection part should become: + * {{{ + * Project [Date#330, __hs_nested.nested.leaf.cnt#335] + * }}} + * + * @param project The project that needs to be transformed. + * @return The transformed project with support for nested indexed fields. + */ + protected[rules] def transformProject(project: Project): Project = { + val projectedFields = project.projectList.map { exp => + val fieldName = extractNamesFromExpression(exp).toKeep.head + val escapedFieldName = PlanUtils.prefixNestedField(fieldName) + val attr = extractAttributeRef(exp, fieldName) + val fieldType = extractTypeFromExpression(exp, fieldName) + // Try to find it in the project transformed child. + getExprId(project.child, escapedFieldName) match { + case Some(exprId) => + attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( + exprId, + attr.qualifier) + case _ => + attr + } + } + project.copy(projectList = projectedFields) + } + + /** + * The method transforms the filter part of a plan to support indexes on + * nested fields. The process is to go through all expression nodes and + * do the following things: + * - Replace retrieval of nested values with index ones. + * - In some specific cases remove the `isnotnull` check because that + * is used on the root of the nested field (ie: `isnotnull(nested#102)` + * does not makes any sense when using the index field). + * + * For example, given the following query: + * {{{ + * df + * .filter("nested.leaf.cnt > 10 and nested.leaf.id == 'leaf_id9'") + * .select("Date", "nested.leaf.cnt") + * }}} + * + * Having this simple filter: + * {{{ + * Filter (isnotnull(nested#102) && (nested#102.leaf.cnt > 10) && + * (nested#102.leaf.id = leaf_id9)) + * }}} + * + * The filter part should become: + * {{{ + * Filter ((__hs_nested.nested.leaf.cnt#335 > 10) && + * (__hs_nested.nested#.leaf.id#336 = leaf_id9)) + * }}} + * + * @param filter The filter that needs to be transformed. + * @return The transformed filter with support for nested indexed fields. + */ + protected[rules] def transformFilter(filter: Filter): Filter = { + val names = extractNamesFromExpression(filter.condition) + val transformedCondition = filter.condition.transformDown { + case bo @ BinaryOperator(IsNotNull(AttributeReference(name, _, _, _)), other) => + if (names.toDiscard.contains(name)) { + other + } else { + bo + } + case bo @ BinaryOperator(other, IsNotNull(AttributeReference(name, _, _, _))) => + if (names.toDiscard.contains(name)) { + other + } else { + bo + } + case g: GetStructField => + val n = getChildNameFromStruct(g) + if (names.toKeep.contains(n)) { + val escapedFieldName = PlanUtils.prefixNestedField(n) + getExprId(filter, escapedFieldName) match { + case Some(exprId) => + val fieldType = extractTypeFromExpression(g, n) + val attr = extractAttributeRef(g, n) + attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( + exprId, + attr.qualifier) + case _ => + g + } + } else { + g + } + case o => + o + } + filter.copy(condition = transformedCondition) + } + + /** + * Returns true if the given project is a supported project. If all of the registered + * providers return None, this returns false. + * + * @param project Project to check if it's supported. + * @return True if the given project is a supported relation. + */ + protected[rules] def hasNestedColumns(project: Project, index: IndexLogEntry): Boolean = { + val indexCols = + (index.indexedColumns ++ index.includedColumns).map(i => ResolverUtils.ResolvedColumn(i)) + val hasNestedCols = indexCols.exists(_.isNested) + if (hasNestedCols) { + val projectListFields = project.projectList.flatMap(extractNamesFromExpression(_).toKeep) + val containsNestedFields = + projectListFields.exists(i => indexCols.exists(j => j.isNested && j.name == i)) + var containsNestedChildren = false + project.child.foreach { + case f: Filter => + val filterSupported = hasNestedColumns(f, index) + containsNestedChildren = containsNestedChildren || filterSupported + case _ => + } + containsNestedFields || containsNestedChildren + } else { + false + } + } + + /** + * Returns true if the given filter has nested columns. + * + * @param filter Filter to check if it's supported. + * @return True if the given project is a supported relation. + */ + protected[rules] def hasNestedColumns(filter: Filter, index: IndexLogEntry): Boolean = { + val indexCols = + (index.indexedColumns ++ index.includedColumns).map(i => ResolverUtils.ResolvedColumn(i)) + val hasNestedCols = indexCols.exists(_.isNested) + if (hasNestedCols) { + val filterFields = extractNamesFromExpression(filter.condition).toKeep.toSeq + val resolvedFilterFields = filterFields.map(ResolverUtils.ResolvedColumn(_)) + resolvedFilterFields.exists(i => indexCols.exists(j => j == i || j.name == i.name)) + } else { + false + } + } + + /** + * The method retrieves the expression id for a given field name. + * + * This method should be mainly used when transforming plans and the + * leaves are already transformed. + * + * @param plan The logical plan from which to get the expression id. + * @param fieldName The name of the field to search for. + * @return An [[ExprId]] if that could be found in the plan otherwise [[None]]. + */ + private def getExprId(plan: LogicalPlan, fieldName: String): Option[ExprId] = { + plan.output.find(a => a.name.equalsIgnoreCase(fieldName)).map(_.exprId) + } +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala new file mode 100644 index 000000000..52ce1ce13 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala @@ -0,0 +1,198 @@ +/* + * Copyright (2021) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index.rules + +import scala.util.Try + +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, BinaryExpression, Expression, GetStructField, IsNotNull, UnaryExpression} +import org.apache.spark.sql.types.{DataType, StructType} + +import com.microsoft.hyperspace.util.ResolverUtils + +object PlanUtils { + + /** + * The method extract field names from a Spark Catalyst [[Expression]]. + * + * @param exp The Spark Catalyst expression from which to extract names. + * @return A set of distinct field names. + */ + def extractNamesFromExpression(exp: Expression): ExtractedNames = { + + def extractNames( + e: Expression, + prevExpStrTypes: Seq[String] = Seq.empty): Set[(String, Seq[String])] = { + e match { + case g: GetStructField => + Set((s"${getChildNameFromStruct(g)}", prevExpStrTypes :+ "getStructField")) + case AttributeReference(name, _, _, _) => + Set((s"$name", prevExpStrTypes :+ "attrRef")) + case Alias(child, _) => + extractNames(child, prevExpStrTypes :+ "alias") + case b: BinaryExpression => + val leftFields = extractNames(b.left, prevExpStrTypes :+ "binaryLeft") + val rightFields = extractNames(b.right, prevExpStrTypes :+ "binaryRight") + leftFields ++ rightFields + case u: IsNotNull => + extractNames(u.child, prevExpStrTypes :+ "isNotNull") + case u: UnaryExpression => + extractNames(u.child, prevExpStrTypes :+ "unary") + case _ => + Set.empty[(String, Seq[String])] + } + } + + var toRemove = Seq.empty[String] + val toKeep = extractNames(exp).toSeq + .sortBy(-_._1.length) + .foldLeft(Seq.empty[String]) { (acc, e) => + val (fieldName, expStrType) = e + if (expStrType.contains("isNotNull") && acc.exists(i => i.startsWith(fieldName))) { + toRemove :+= fieldName + acc + } else { + acc :+ fieldName + } + } + ExtractedNames(toKeep.toSet, toRemove.toSet) + } + + /** + * Given a [[GetStructField]] expression for a nested field (aka a struct) + * the method will extract the full field `.` (dot) separated name. + * + * @param field The [[GetStructField]] field from which we want to extract + * the name. + * @return A field name `.` (dot) separated if nested. + */ + def getChildNameFromStruct(field: GetStructField): String = { + field.child match { + case f: GetStructField => + s"${getChildNameFromStruct(f)}.${field.name.get}" + case a: AttributeReference => + s"${a.name}.${field.name.get}" + case _ => + s"${field.name.get}" + } + } + + /** + * Given an Spark Catalyst [[Expression]] and a field name the method extracts + * the parent search expression and the expression that contains the field name + * + * @param exp The Spark Catalyst [[Expression]] to extract from. + * @param name The field name to search for. + * @return A tuple with the parent expression and the leaf expression that + * contains the given name. + */ + def extractSearchQuery(exp: Expression, name: String): (Expression, Expression) = { + val splits = name.split("\\.") + val expFound = exp.find { + case a: AttributeReference if splits.forall(s => a.name.contains(s)) => true + case f: GetStructField if splits.forall(s => f.toString().contains(s)) => true + case _ => false + }.get + val parent = exp.find { + case e: Expression if e.containsChild.contains(expFound) => true + case _ => false + }.get + (parent, expFound) + } + + /** + * Given an Spark Catalyst [[Expression]], a needle [[Expression]] and a replace + * [[Expression]] the method will replace the needle with the replacement into + * the parent expression. + * + * @param parent The parent Spark Catalyst [[Expression]] into which to replace. + * @param needle The Spark Catalyst [[Expression]] needle to search for. + * @param repl The replacement Spark Catalyst [[Expression]]. + * @return A new Spark Catalyst [[Expression]]. + */ + def replaceInSearchQuery( + parent: Expression, + needle: Expression, + repl: Expression): Expression = { + parent.mapChildren { c => + if (c == needle) { + repl + } else { + c + } + } + } + + /** + * Given an Spark Catalyst [[Expression]] and a field name the method + * extracts the [[AttributeReference]] for that field name. + * + * @param exp The Spark Catalyst [[Expression]] to extract from. + * @param name The field name for which to extract the attribute reference. + * @return A Spark Catalyst [[AttributeReference]] pointing to the field name. + */ + def extractAttributeRef(exp: Expression, name: String): AttributeReference = { + val splits = name.split("\\.") + val elem = exp.find { + case a: AttributeReference if splits.contains(a.name) => true + case _ => false + } + elem.get.asInstanceOf[AttributeReference] + } + + /** + * Given a Spark Catalyst [[Expression]] and a field name the method + * extracts the type of the field as a Spark SQL [[DataType]]. + * + * @param exp The Spark Catalyst [[Expression]] from which to extract the type. + * @param name The field name for which we need to get the type. + * @return A Spark SQL [[DataType]] of the given field name. + */ + def extractTypeFromExpression(exp: Expression, name: String): DataType = { + val splits = name.split("\\.") + val elem = exp.flatMap { + case a: AttributeReference => + if (splits.forall(s => a.name == s)) { + Some((name, a.dataType)) + } else { + Try({ + val h :: t = splits.toList + if (a.name == h && a.dataType.isInstanceOf[StructType]) { + val itFields = t.flatMap { i => + a.dataType + .asInstanceOf[StructType] + .find(_.name.equalsIgnoreCase(i)) + .map(j => (i, j.dataType)) + } + Some(itFields.last) + } else { + None + } + }).getOrElse(None) + } + case f: GetStructField if splits.forall(s => f.toString().contains(s)) => + Some((name, f.dataType)) + case _ => None + } + elem.find(e => e._1 == name || e._1 == splits.last).get._2 + } + + def prefixNestedField(fieldName: String): String = { + ResolverUtils.ResolvedColumn(fieldName, fieldName.contains(".")).normalizedName + } + + private[hyperspace] case class ExtractedNames(toKeep: Set[String], toDiscard: Set[String]) +} diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala index b198711b7..77a83b549 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -17,8 +17,8 @@ package com.microsoft.hyperspace.util import org.apache.spark.sql.{Column, SparkSession} -import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, ExtractValue, GetArrayStructFields, GetMapValue, GetStructField} +import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, ExtractValue, GetArrayStructFields, GetMapValue, GetStructField, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.functions.col import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} @@ -31,76 +31,6 @@ import com.microsoft.hyperspace.util.ResolverUtils.ResolvedColumn.NESTED_FIELD_P */ object ResolverUtils { - /** - * [[ResolvedColumn]] stores information when a column name is resolved against the - * analyzed plan and its schema. - * - * Outside unit tests, this object should not be created directly, but via the `resolve` function, - * or `ResolvedColumn.apply` with a normalized name. - * - * @param name The column name resolved from an analyzed plan. - * @param isNested Flag to denote if this column is nested or not. - */ - private[hyperspace] case class ResolvedColumn(name: String, isNested: Boolean) { - assert(!isNested || (name.contains(".") && !name.startsWith(NESTED_FIELD_PREFIX))) - // Quotes will be removed from `resolve` and nested columns with quotes (e.g., "a.`b.c`.d") - // are not supported. - assert(!name.contains("`")) - - // For nested fields, the column name is prefixed with `NESTED_FIELD_PREFIX`. - lazy val normalizedName = { - if (isNested) { - s"$NESTED_FIELD_PREFIX$name" - } else { - name - } - } - - /** - * Create a column using the resolved name. Top level column names are quoted, and - * nested column names are aliased with normalized names. - * - * @return [[Column]] object created using the resolved name. - */ - def toColumn: Column = { - if (isNested) { - // No need to quote the string for "as" even if it contains dots. - col(name).as(normalizedName) - } else { - col(quote(name)) - } - } - - /** - * Create a column using the normalized name. Since the normalized name is already flattened - * with "dots", it is quoted. - * - * @return [[Column]] object create using the normalized name. - */ - def toNormalizedColumn: Column = col(quote(normalizedName)) - - private def quote(name: String) = s"`$name`" - } - - private[hyperspace] object ResolvedColumn { - private val NESTED_FIELD_PREFIX = "__hs_nested." - - /** - * Given a normalized column name, create [[ResolvedColumn]] after handling the prefix - * for nested columns. - * - * @param normalizedColumnName Normalized column name. - * @return [[ResolvedColumn]] created from the given normalized column name. - */ - def apply(normalizedColumnName: String): ResolvedColumn = { - if (normalizedColumnName.startsWith(NESTED_FIELD_PREFIX)) { - ResolvedColumn(normalizedColumnName.substring(NESTED_FIELD_PREFIX.length), isNested = true) - } else { - ResolvedColumn(normalizedColumnName, isNested = false) - } - } - } - /** * Return available string if required string can be resolved with it, based on spark resolver. * @@ -162,16 +92,19 @@ object ResolverUtils { def resolve( spark: SparkSession, requiredStrings: Seq[String], - plan: LogicalPlan): Option[Seq[ResolvedColumn]] = { + plan: LogicalPlan, + resolverMethod: (String, LogicalPlan, Resolver) => Option[Expression] = + defaultResolverMethod, + throwIfNotInSchema: Boolean = true): Option[Seq[ResolvedColumn]] = { val schema = plan.schema val resolver = spark.sessionState.conf.resolver val resolved = requiredStrings.map { requiredField => - plan - .resolveQuoted(requiredField, resolver) + resolverMethod(requiredField, plan, resolver) .map { expr => val resolvedColNameParts = extractColumnName(expr) validateResolvedColumnName(requiredField, resolvedColNameParts) - val origColNameParts = getColumnNameFromSchema(schema, resolvedColNameParts, resolver) + val origColNameParts = + getColumnNameFromSchema(schema, resolvedColNameParts, resolver, throwIfNotInSchema) ResolvedColumn(origColNameParts.mkString("."), origColNameParts.length > 1) } .getOrElse { return None } @@ -179,6 +112,36 @@ object ResolverUtils { Some(resolved) } + /** + * The default way of resolving field names. + * + * @param fieldName The field name to resolve. + * @param plan The Logical Plan that contains information about the field. + * @param resolver The resolver to use to resolve the field. + * @return A optional [[Expression]] that is the resolved field name. + */ + protected[hyperspace] def defaultResolverMethod( + fieldName: String, + plan: LogicalPlan, + resolver: Resolver): Option[Expression] = { + plan.resolveQuoted(fieldName, resolver) + } + + /** + * The resolve children way of resolving field names. + * + * @param fieldName The field name to resolve. + * @param plan The Logical Plan that contains information about the field. + * @param resolver The resolver to use to resolve the field. + * @return A optional [[Expression]] that is the resolved field name. + */ + protected[hyperspace] def resolveWithChildren( + fieldName: String, + plan: LogicalPlan, + resolver: Resolver): Option[NamedExpression] = { + plan.resolveChildren(UnresolvedAttribute.parseAttributeName(fieldName), resolver) + } + // Extracts the parts of a nested field access path from an expression. private def extractColumnName(expr: Expression): Seq[String] = { expr match { @@ -188,9 +151,11 @@ object ResolverUtils { extractColumnName(child) :+ name case _: GetArrayStructFields => // TODO: Nested arrays will be supported later + // See https://github.com/microsoft/hyperspace/issues/372 throw HyperspaceException("Array types are not supported.") case _: GetMapValue => // TODO: Nested maps will be supported later + // See https://github.com/microsoft/hyperspace/issues/411 throw HyperspaceException("Map types are not supported.") case Alias(nested: ExtractValue, _) => extractColumnName(nested) @@ -212,19 +177,107 @@ object ResolverUtils { private def getColumnNameFromSchema( schema: StructType, resolvedColNameParts: Seq[String], - resolver: Resolver): Seq[String] = resolvedColNameParts match { + resolver: Resolver, + throwIfNotInSchema: Boolean = true): Seq[String] = resolvedColNameParts match { case h :: tail => - val field = schema.find(f => resolver(f.name, h)).get - field match { - case StructField(name, s: StructType, _, _) => - name +: getColumnNameFromSchema(s, tail, resolver) - case StructField(_, _: ArrayType, _, _) => - // TODO: Nested arrays will be supported later - throw HyperspaceException("Array types are not supported.") - case StructField(_, _: MapType, _, _) => - // TODO: Nested maps will be supported later - throw HyperspaceException("Map types are not supported") - case f => Seq(f.name) + val fieldOpt = schema.find(f => resolver(f.name, h)) + fieldOpt match { + case Some(field) => + field match { + case StructField(name, s: StructType, _, _) => + name +: getColumnNameFromSchema(s, tail, resolver) + case StructField(_, _: ArrayType, _, _) => + // TODO: Nested arrays will be supported later + // See https://github.com/microsoft/hyperspace/issues/372 + throw HyperspaceException("Array types are not supported.") + case StructField(_, _: MapType, _, _) => + // TODO: Nested maps will be supported later + // See https://github.com/microsoft/hyperspace/issues/411 + throw HyperspaceException("Map types are not supported") + case f => Seq(f.name) + } + case _ => + if (throwIfNotInSchema) { + throw HyperspaceException(s"Hyperspace cannot not find $h in schema") + } else { + if (tail.nonEmpty) { + h +: getColumnNameFromSchema(schema, tail, resolver, throwIfNotInSchema) + } else { + Seq(h) + } + } + } + } + + /** + * [[ResolvedColumn]] stores information when a column name is resolved against the + * analyzed plan and its schema. + * + * Outside unit tests, this object should not be created directly, but via the `resolve` function, + * or `ResolvedColumn.apply` with a normalized name. + * + * @param name The column name resolved from an analyzed plan. + * @param isNested Flag to denote if this column is nested or not. + */ + private[hyperspace] case class ResolvedColumn(name: String, isNested: Boolean) { + assert(!isNested || (name.contains(".") && !name.startsWith(NESTED_FIELD_PREFIX))) + // Quotes will be removed from `resolve` and nested columns with quotes (e.g., "a.`b.c`.d") + // are not supported. + assert(!name.contains("`")) + + // For nested fields, the column name is prefixed with `NESTED_FIELD_PREFIX`. + lazy val normalizedName = { + if (isNested) { + s"$NESTED_FIELD_PREFIX$name" + } else { + name + } + } + + /** + * Create a column using the resolved name. Top level column names are quoted, and + * nested column names are aliased with normalized names. + * + * @return [[Column]] object created using the resolved name. + */ + def toColumn: Column = { + if (isNested) { + // No need to quote the string for "as" even if it contains dots. + col(name).as(normalizedName) + } else { + col(quote(name)) } + } + + /** + * Create a column using the normalized name. Since the normalized name is already flattened + * with "dots", it is quoted. + * + * @return [[Column]] object create using the normalized name. + */ + def toNormalizedColumn: Column = col(quote(normalizedName)) + + private def quote(name: String) = s"`$name`" + } + + private[hyperspace] object ResolvedColumn { + private val NESTED_FIELD_PREFIX = "__hs_nested." + + /** + * Given a normalized column name, create [[ResolvedColumn]] after handling the prefix + * for nested columns. + * + * @param normalizedColumnName Normalized column name. + * @return [[ResolvedColumn]] created from the given normalized column name. + */ + def apply(normalizedColumnName: String): ResolvedColumn = { + if (normalizedColumnName.startsWith(NESTED_FIELD_PREFIX)) { + ResolvedColumn( + normalizedColumnName.substring(NESTED_FIELD_PREFIX.length), + isNested = true) + } else { + ResolvedColumn(normalizedColumnName, isNested = false) + } + } } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala index f60d270c1..b05f31c35 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala @@ -24,11 +24,10 @@ import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData, TestConfig, TestUtils} +import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData, SampleNestedData, TestConfig, TestUtils} import com.microsoft.hyperspace.index.IndexConstants.{GLOBBING_PATTERN_KEY, REFRESH_MODE_INCREMENTAL, REFRESH_MODE_QUICK} import com.microsoft.hyperspace.index.IndexLogEntryTags._ import com.microsoft.hyperspace.index.execution.BucketUnionStrategy -import com.microsoft.hyperspace.index.plans.logical.IndexHadoopFsRelation import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule} import com.microsoft.hyperspace.util.PathUtils @@ -118,6 +117,44 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite { } } + test( + "E2E test for filter query on nested columns") { + val loc = testDir + "samplenestedparquet" + val dataColumns = Seq("Date", "RGUID", "Query", "imprs", "clicks", "nested") + SampleNestedData.save(spark, loc, dataColumns) + + Seq(true, false).foreach { enableLineage => + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> enableLineage.toString) { + withIndex("filterNestedIndex") { + val df = spark.read.parquet(loc) + val indexConfig = IndexConfig("filterNestedIndex", + Seq("nested.leaf.id", "nested.leaf.cnt"), Seq("Date")) + + hyperspace.createIndex(df, indexConfig) + + def query(): DataFrame = + df.filter("nested.leaf.cnt > 10 and nested.leaf.id == 'leaf_id9'") + .select("Date", "nested.leaf.cnt", "nested.leaf.id") + + verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName)) + + val res = query().collect().sortWith((r1, r2) => r1.getInt(1) < r2.getInt(1)) + + assert(res.length == 3) + assert(res(0).getString(0) == "2019-10-03") + assert(res(0).getInt(1) == 12) + assert(res(0).getString(2) == "leaf_id9") + assert(res(1).getString(0) == "2019-10-03") + assert(res(1).getInt(1) == 21) + assert(res(1).getString(2) == "leaf_id9") + assert(res(2).getString(0) == "2019-10-03") + assert(res(2).getInt(1) == 22) + assert(res(2).getString(2) == "leaf_id9") + } + } + } + } + test("E2E test for case insensitive filter query utilizing indexes.") { val df = spark.read.parquet(nonPartitionedDataPath) val indexConfig = IndexConfig("filterIndex", Seq("C3"), Seq("C1")) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index 8d4ae62d4..a8b970826 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -91,16 +91,17 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { val indexManager = IndexCollectionManager(spark) val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) - val t1Relation = RuleUtils.getRelation(spark, t1ProjectNode).get - val t2Relation = RuleUtils.getRelation(spark, t2ProjectNode).get - assert(RuleUtils.getCandidateIndexes(spark, allIndexes, t1Relation).length === 3) - assert(RuleUtils.getCandidateIndexes(spark, allIndexes, t2Relation).length === 2) + val ruleHelper = new BaseRuleHelper(spark) + val t1Relation = ruleHelper.getRelation(t1ProjectNode).get + val t2Relation = ruleHelper.getRelation(t2ProjectNode).get + assert(ruleHelper.getCandidateIndexes(allIndexes, t1Relation).length === 3) + assert(ruleHelper.getCandidateIndexes(allIndexes, t2Relation).length === 2) // Delete an index for t1ProjectNode indexManager.delete("t1i1") val allIndexes2 = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) - assert(RuleUtils.getCandidateIndexes(spark, allIndexes2, t1Relation).length === 2) + assert(ruleHelper.getCandidateIndexes(allIndexes2, t1Relation).length === 2) } test("Verify get logical relation for single logical relation node plan.") { @@ -113,7 +114,8 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { test("Verify get logical relation for non-linear plan.") { val joinNode = JoinWithoutHint(t1ProjectNode, t2ProjectNode, JoinType("inner"), None) - val r = RuleUtils.getRelation(spark, Project(Seq(t1c3, t2c3), joinNode)) + val ruleHelper = new BaseRuleHelper(spark) + val r = ruleHelper.getRelation(Project(Seq(t1c3, t2c3), joinNode)) assert(r.isEmpty) } @@ -149,9 +151,10 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.99", IndexConstants.INDEX_HYBRID_SCAN_DELETED_RATIO_THRESHOLD -> (if (hybridScanDeleteEnabled) "0.99" else "0")) { - val relation = RuleUtils.getRelation(spark, plan).get - val indexes = RuleUtils - .getCandidateIndexes(spark, allIndexes, relation) + val ruleHelper = new BaseRuleHelper(spark) + val relation = ruleHelper.getRelation(plan).get + val indexes = ruleHelper + .getCandidateIndexes(allIndexes, relation) if (expectedCandidateIndexes.nonEmpty) { assert(indexes.length === expectedCandidateIndexes.length) assert(indexes.map(_.name).toSet.equals(expectedCandidateIndexes.toSet)) @@ -281,11 +284,12 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) df.limit(5).write.mode("append").parquet(dataPath) val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan - val relation = RuleUtils.getRelation(spark, optimizedPlan).get + val ruleHelper = new BaseRuleHelper(spark) + val relation = ruleHelper.getRelation(optimizedPlan).get withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true") { withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.99") { - val indexes = RuleUtils.getCandidateIndexes(spark, allIndexes, relation) + val indexes = ruleHelper.getCandidateIndexes(allIndexes, relation) assert( indexes.head .getTagValue(relation.plan, IndexLogEntryTags.IS_HYBRIDSCAN_CANDIDATE) @@ -297,7 +301,7 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { } withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.2") { - val indexes = RuleUtils.getCandidateIndexes(spark, allIndexes, relation) + val indexes = ruleHelper.getCandidateIndexes(allIndexes, relation) assert(indexes.isEmpty) assert( !allIndexes.head @@ -326,7 +330,8 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { val df = spark.read.parquet(dataPath) val query = df.filter(df("id") >= 3).select("id", "name") val bucketSpec = BucketSpec(100, Seq("id"), Seq()) - val shuffled = RuleUtils.transformPlanToShuffleUsingBucketSpec( + val ruleHelper = new BaseRuleHelper(spark) + val shuffled = ruleHelper.transformPlanToShuffleUsingBucketSpec( bucketSpec, query.queryExecution.optimizedPlan) @@ -360,7 +365,7 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { val bucketSpec2 = BucketSpec(100, Seq("age"), Seq()) val query2 = df.filter(df("id") <= 3).select("id", "name") val shuffled2 = - RuleUtils.transformPlanToShuffleUsingBucketSpec( + ruleHelper.transformPlanToShuffleUsingBucketSpec( bucketSpec2, query2.queryExecution.optimizedPlan) assert(shuffled2.collect { @@ -379,7 +384,8 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { } private def validateLogicalRelation(plan: LogicalPlan, expected: LogicalRelation): Unit = { - val r = RuleUtils.getRelation(spark, plan) + val ruleHelper = new BaseRuleHelper(spark) + val r = ruleHelper.getRelation(plan) assert(r.isDefined) assert(r.get.plan.equals(expected)) } From f960e4a3ead1add68b9e9e7127dfe0033b8416f2 Mon Sep 17 00:00:00 2001 From: Andrei Ionescu Date: Sat, 17 Apr 2021 21:52:17 +0300 Subject: [PATCH 2/7] Add aliasing from index nested field back back to original name --- .../index/rules/NestedRuleHelper.scala | 11 +++++++++-- .../hyperspace/util/ResolverUtils.scala | 16 +++++++++++++++- .../hyperspace/util/ResolverUtilsTest.scala | 3 +++ 3 files changed, 27 insertions(+), 3 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala index 73c882d82..49e477eb2 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala @@ -18,7 +18,7 @@ package com.microsoft.hyperspace.index.rules import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BinaryOperator, ExprId, GetStructField, In, IsNotNull, Literal, NamedExpression, Not} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, BinaryOperator, ExprId, GetStructField, In, IsNotNull, Literal, NamedExpression, Not} import org.apache.spark.sql.catalyst.optimizer.OptimizeIn import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project, Union} import org.apache.spark.sql.execution.datasources.InMemoryFileIndex @@ -347,14 +347,21 @@ class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { val projectedFields = project.projectList.map { exp => val fieldName = extractNamesFromExpression(exp).toKeep.head val escapedFieldName = PlanUtils.prefixNestedField(fieldName) + val resolvedField = ResolverUtils.ResolvedColumn(escapedFieldName) val attr = extractAttributeRef(exp, fieldName) val fieldType = extractTypeFromExpression(exp, fieldName) // Try to find it in the project transformed child. getExprId(project.child, escapedFieldName) match { case Some(exprId) => - attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( + val newAttr = attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( exprId, attr.qualifier) + resolvedField.projectName match { + case Some(projectName) => + Alias(newAttr, projectName)() + case None => + newAttr + } case _ => attr } diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala index 77a83b549..90d1625ca 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -226,7 +226,7 @@ object ResolverUtils { assert(!name.contains("`")) // For nested fields, the column name is prefixed with `NESTED_FIELD_PREFIX`. - lazy val normalizedName = { + lazy val normalizedName: String = { if (isNested) { s"$NESTED_FIELD_PREFIX$name" } else { @@ -257,6 +257,20 @@ object ResolverUtils { */ def toNormalizedColumn: Column = col(quote(normalizedName)) + /** + * Returns the simple name that is required in the projection. + * + * Given a nested column `nested.nested.fieldA` when doing select over it the + * returned column name by Spark will be `fieldA`. + * + * @return The name for projection if it's a nested field, otherwise [[None]]. + */ + def projectName: Option[String] = if (isNested && name.contains(".")) { + Some(name.split("\\.").last) + } else { + None + } + private def quote(name: String) = s"`$name`" } diff --git a/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTest.scala index 7f0ba8c4a..663528d63 100644 --- a/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTest.scala @@ -148,6 +148,7 @@ class ResolverUtilsTest extends SparkFunSuite with SparkInvolvedSuite { // `Column` object correctly. assert(col1.toColumn.toString === "`a.b.c`") assert(col1.toNormalizedColumn.toString === "`a.b.c`") + assert(col1.projectName.isEmpty) val col2 = ResolvedColumn("abc", isNested = false) assert(col2.name == "abc") @@ -155,6 +156,7 @@ class ResolverUtilsTest extends SparkFunSuite with SparkInvolvedSuite { assert(col2 === ResolvedColumn(col2.normalizedName)) assert(col2.toColumn.toString === "abc") assert(col2.toNormalizedColumn.toString === "abc") + assert(col2.projectName.isEmpty) val col3 = ResolvedColumn("a.b.c", isNested = true) assert(col3.name == "a.b.c") @@ -162,6 +164,7 @@ class ResolverUtilsTest extends SparkFunSuite with SparkInvolvedSuite { assert(col3 === ResolvedColumn(col3.normalizedName)) assert(col3.toColumn.toString === "a.b.c AS `__hs_nested.a.b.c`") assert(col3.toNormalizedColumn.toString === "`__hs_nested.a.b.c`") + assert(col3.projectName.contains("c")) } } From 671c4b0f6509e531e7da921fa02978629c363043 Mon Sep 17 00:00:00 2001 From: Andrei Ionescu Date: Mon, 19 Apr 2021 15:55:43 +0300 Subject: [PATCH 3/7] Integrated feedback --- .../index/rules/BaseRuleHelper.scala | 85 ++-- .../index/rules/FilterIndexRule.scala | 9 +- .../index/rules/JoinIndexRule.scala | 20 +- .../index/rules/NestedRuleHelper.scala | 409 ++++-------------- .../hyperspace/index/rules/RuleUtils.scala | 56 +++ .../index/rules/BaseRuleHelperTest.scala | 345 +++++++++++++++ .../index/rules/RuleUtilsTest.scala | 14 +- 7 files changed, 563 insertions(+), 375 deletions(-) create mode 100644 src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala create mode 100644 src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelper.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelper.scala index 99ce39dab..6dcb168ad 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelper.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelper.scala @@ -32,7 +32,7 @@ import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.IndexLogEntryTags.{HYBRIDSCAN_RELATED_CONFIGS, IS_HYBRIDSCAN_CANDIDATE} import com.microsoft.hyperspace.index.plans.logical.{BucketUnion, IndexHadoopFsRelation} -import com.microsoft.hyperspace.index.sources.FileBasedRelation +import com.microsoft.hyperspace.index.sources.{FileBasedRelation, FileBasedSourceProviderManager} import com.microsoft.hyperspace.util.HyperspaceConf class BaseRuleHelper(spark: SparkSession) { @@ -181,7 +181,7 @@ class BaseRuleHelper(spark: SparkSession) { useBucketSpec: Boolean, useBucketUnionForAppended: Boolean): LogicalPlan = { // Check pre-requisite. - val relation = getRelation(plan) + val relation = RuleUtils.getRelation(plan) assert(relation.isDefined) // If there is no change in source data files, the index can be applied by @@ -205,20 +205,31 @@ class BaseRuleHelper(spark: SparkSession) { } /** - * Extract the relation node if the given logical plan is linear. + * Returns the updated output attributes given a relation and schema. * - * @param plan Logical plan to extract a relation node from. - * @return If the plan is linear and the relation node is supported, the [[FileBasedRelation]] - * object that wraps the relation node. Otherwise None. + * @param relation The [[FileBasedRelation]] to extract from. + * @param schema The given schema to compare with. + * @return A collection of [[AttributeReference]] that will be plan's output. */ - def getRelation(plan: LogicalPlan): Option[FileBasedRelation] = { - val provider = Hyperspace.getContext(spark).sourceProviderManager - val leaves = plan.collectLeaves() - if (leaves.size == 1 && provider.isSupportedRelation(leaves.head)) { - Some(provider.getRelation(leaves.head)) - } else { - None - } + protected[rules] def getUpdatedOutput( + relation: FileBasedRelation, + schema: StructType): Seq[AttributeReference] = { + relation.plan.output + .filter(attr => schema.fieldNames.contains(attr.name)) + .map(_.asInstanceOf[AttributeReference]) + } + + /** + * Returns the output schema given a relation and an index. + * + * @param relation The [[FileBasedRelation]] to extract from. + * @param index Index used in transformation of plan. + * @return + */ + protected[rules] def getFsRelationSchema( + relation: FileBasedRelation, + index: IndexLogEntry): StructType = { + StructType(index.schema.filter(relation.plan.schema.contains(_))) } /** @@ -471,6 +482,38 @@ class BaseRuleHelper(spark: SparkSession) { planForAppended } + /** + * The method transforms the relation part of a plan to support indexes. + * + * @param index Index used in transformation of plan. + * @param provider The source provider. + * @param plan Original plan. + * @param useBucketSpec Option whether to use BucketSpec for reading index data. + * @return A transformed [[LogicalRelation]]. + */ + private[rules] def transformRelation( + index: IndexLogEntry, + provider: FileBasedSourceProviderManager, + plan: LogicalPlan, + useBucketSpec: Boolean): LogicalRelation = { + val relation = provider.getRelation(plan) + val location = index.withCachedTag(IndexLogEntryTags.INMEMORYFILEINDEX_INDEX_ONLY) { + new InMemoryFileIndex(spark, index.content.files, Map(), None) + } + + val indexFsRelation = new IndexHadoopFsRelation( + location, + new StructType(), + getFsRelationSchema(relation, index), + if (useBucketSpec) Some(index.bucketSpec) else None, + new ParquetFileFormat, + Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark, index) + + val updatedOutput = getUpdatedOutput(relation, indexFsRelation.schema) + + relation.createLogicalRelation(indexFsRelation, updatedOutput) + } + /** * Transform the plan to perform on-the-fly Shuffle the data based on bucketSpec. * Note that this extractor depends on [[LogicalRelation]] with [[HadoopFsRelation]] because @@ -545,17 +588,3 @@ class BaseRuleHelper(spark: SparkSession) { shuffled } } - -object BaseRuleHelper extends Serializable { - - /** - * Check if an index was applied the given relation or not. - * This can be determined by an identifier in [[FileBasedRelation]]'s options. - * - * @param relation FileBasedRelation to check if an index is already applied. - * @return true if an index is applied to the given relation. Otherwise false. - */ - def isIndexApplied(relation: FileBasedRelation): Boolean = { - relation.options.exists(_.equals(IndexConstants.INDEX_RELATION_IDENTIFIER)) - } -} diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala index 0b04e5015..3e68ac2a3 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -107,8 +107,7 @@ object FilterIndexRule outputColumns: Seq[String], filterColumns: Seq[String], plan: LogicalPlan): Seq[IndexLogEntry] = { - val ruleHelper = new BaseRuleHelper(spark) - ruleHelper.getRelation(filter) match { + RuleUtils.getRelation(filter) match { case Some(r) => val indexManager = Hyperspace .getContext(spark) @@ -145,7 +144,7 @@ object FilterIndexRule // Get candidate via file-level metadata validation. This is performed after pruning // by column schema, as this might be expensive when there are numerous files in the // relation or many indexes to be checked. - ruleHelper.getCandidateIndexes(candidateIndexes, r) + new BaseRuleHelper(spark).getCandidateIndexes(candidateIndexes, r) case _ => Seq.empty @@ -191,7 +190,7 @@ object ExtractFilterNode { def unapply(plan: LogicalPlan): Option[returnType] = plan match { case project @ Project(_, filter @ Filter(condition: Expression, ExtractRelation(relation))) - if !BaseRuleHelper.isIndexApplied(relation) => + if !RuleUtils.isIndexApplied(relation) => val projectColumnNames = CleanupAliases(project) .asInstanceOf[Project] .projectList @@ -202,7 +201,7 @@ object ExtractFilterNode { Some(project, filter, projectColumnNames, filterColumnNames) case filter @ Filter(condition: Expression, ExtractRelation(relation)) - if !BaseRuleHelper.isIndexApplied(relation) => + if !RuleUtils.isIndexApplied(relation) => val relationColumnsName = relation.plan.output.map(_.name) val filterColumnNames = condition.references.map(_.name).toSeq diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala index 3db907829..3483f56ed 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -108,12 +108,11 @@ object JoinIndexRule private def isApplicable(l: LogicalPlan, r: LogicalPlan, condition: Expression): Boolean = { // The given plan is eligible if it is supported and index has not been applied. def isEligible(optRel: Option[FileBasedRelation]): Boolean = { - optRel.exists(!BaseRuleHelper.isIndexApplied(_)) + optRel.exists(!RuleUtils.isIndexApplied(_)) } - lazy val ruleHelper = new BaseRuleHelper(spark) - lazy val optLeftRel = ruleHelper.getRelation(l) - lazy val optRightRel = ruleHelper.getRelation(r) + lazy val optLeftRel = RuleUtils.getRelation(l) + lazy val optRightRel = RuleUtils.getRelation(r) isJoinConditionSupported(condition) && isPlanLinear(l) && isPlanLinear(r) && @@ -284,8 +283,8 @@ object JoinIndexRule left: LogicalPlan, right: LogicalPlan, joinCondition: Expression): Option[(IndexLogEntry, IndexLogEntry)] = { - val indexManager = Hyperspace.getContext(spark).indexCollectionManager - val ruleHelper = new BaseRuleHelper(spark) + val hyperspaceContext = Hyperspace.getContext(spark) + val indexManager = hyperspaceContext.indexCollectionManager // TODO: the following check only considers indexes in ACTIVE state for usage. Update // the code to support indexes in transitioning states as well. @@ -295,8 +294,8 @@ object JoinIndexRule // TODO: we can write an extractor that applies `isApplicable` so that we don't have to // get relations twice. Note that `getRelation` should always succeed since this has // been already checked in `isApplicable`. - val leftRelation = ruleHelper.getRelation(left).get - val rightRelation = ruleHelper.getRelation(right).get + val leftRelation = RuleUtils.getRelation(left).get + val rightRelation = RuleUtils.getRelation(right).get val lBaseAttrs = leftRelation.plan.output.map(_.name) val rBaseAttrs = rightRelation.plan.output.map(_.name) @@ -317,12 +316,13 @@ object JoinIndexRule val lUsable = getUsableIndexes(allIndexes, lRequiredIndexedCols, lRequiredAllCols) val rUsable = getUsableIndexes(allIndexes, rRequiredIndexedCols, rRequiredAllCols) - val leftRel = ruleHelper.getRelation(left).get - val rightRel = ruleHelper.getRelation(right).get + val leftRel = RuleUtils.getRelation(left).get + val rightRel = RuleUtils.getRelation(right).get // Get candidate via file-level metadata validation. This is performed after pruning // by column schema, as this might be expensive when there are numerous files in the // relation or many indexes to be checked. + val ruleHelper = new BaseRuleHelper(spark) val lIndexes = ruleHelper.getCandidateIndexes(lUsable, leftRel) val rIndexes = ruleHelper.getCandidateIndexes(rUsable, rightRel) diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala index 49e477eb2..93964b53b 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala @@ -16,90 +16,57 @@ package com.microsoft.hyperspace.index.rules -import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, BinaryOperator, ExprId, GetStructField, In, IsNotNull, Literal, NamedExpression, Not} -import org.apache.spark.sql.catalyst.optimizer.OptimizeIn -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project, Union} -import org.apache.spark.sql.execution.datasources.InMemoryFileIndex -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.types.{LongType, StructType} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, BinaryOperator, ExprId, GetStructField, IsNotNull, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} +import org.apache.spark.sql.types.StructType import com.microsoft.hyperspace.Hyperspace -import com.microsoft.hyperspace.index.{FileInfo, IndexConstants, IndexLogEntry, IndexLogEntryTags} -import com.microsoft.hyperspace.index.plans.logical.{BucketUnion, IndexHadoopFsRelation} +import com.microsoft.hyperspace.index.IndexLogEntry import com.microsoft.hyperspace.index.rules.PlanUtils._ -import com.microsoft.hyperspace.util.{HyperspaceConf, ResolverUtils} +import com.microsoft.hyperspace.index.sources.FileBasedRelation +import com.microsoft.hyperspace.util.ResolverUtils class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { /** - * Transform the current plan to utilize index. - * The transformed plan reads data from indexes instead of the source relations. - * Bucketing information of the index is retained if useBucketSpec is true. - * - * NOTE: This method currently only supports transformation of nodes with supported relations. - * - * @param index Index used in transformation of plan. - * @param plan Current logical plan. - * @param useBucketSpec Option whether to use BucketSpec for reading index data. - * @return Transformed logical plan that leverages an index. + * @inheritdoc + */ + override protected[rules] def getUpdatedOutput( + relation: FileBasedRelation, + schema: StructType): Seq[AttributeReference] = { + schema.flatMap { s => + relation.plan.output + .find(a => ResolverUtils.ResolvedColumn(s.name).name.startsWith(a.name)) + .map { a => + AttributeReference(s.name, s.dataType, a.nullable, a.metadata)( + NamedExpression.newExprId, + a.qualifier) + } + } + } + + /** + * @inheritdoc + */ + override protected[rules] def getFsRelationSchema( + relation: FileBasedRelation, + index: IndexLogEntry): StructType = { + StructType( + index.schema.filter(i => relation.plan.schema.exists(j => i.name.contains(j.name)))) + } + + /** + * @inheritdoc */ override protected[rules] def transformPlanToUseIndexOnlyScan( index: IndexLogEntry, plan: LogicalPlan, useBucketSpec: Boolean): LogicalPlan = { val provider = Hyperspace.getContext(spark).sourceProviderManager - // Note that depending on the case we transform only the base relation - // and sometimes other portions of the plan (e.g., filters). For instance, - // given the following input plan: - // Project(A,B) -> Filter(C = 10) -> Scan (A,B,C,D,E) - // in the presence of a suitable index, we will transform to: - // Project(A,B) -> Filter(C = 10) -> Index Scan (A,B,C) - // In the case of nested fields we will transform the project and - // filter nodes too. plan transformUp { case l: LeafNode if provider.isSupportedRelation(l) => - val relation = provider.getRelation(l) - val location = index.withCachedTag(IndexLogEntryTags.INMEMORYFILEINDEX_INDEX_ONLY) { - new InMemoryFileIndex(spark, index.content.files, Map(), None) - } - - val newSchema = StructType( - index.schema.filter(i => relation.plan.schema.exists(j => i.name.contains(j.name)))) - - val indexFsRelation = new IndexHadoopFsRelation( - location, - new StructType(), - newSchema, - if (useBucketSpec) Some(index.bucketSpec) else None, - new ParquetFileFormat, - Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark, index) - - val resolvedFields = ResolverUtils.resolve( - spark, - (index.indexedColumns ++ index.includedColumns) - .map(ResolverUtils.ResolvedColumn(_).name), - relation.plan) - val updatedOutput = - if (resolvedFields.isDefined && resolvedFields.get.exists(_.isNested)) { - indexFsRelation.schema.flatMap { s => - relation.plan.output - .find { a => - ResolverUtils.ResolvedColumn(s.name).name.startsWith(a.name) - } - .map { a => - AttributeReference(s.name, s.dataType, a.nullable, a.metadata)( - NamedExpression.newExprId, - a.qualifier) - } - } - } else { - relation.plan.output - .filter(attr => indexFsRelation.schema.fieldNames.contains(attr.name)) - .map(_.asInstanceOf[AttributeReference]) - } - relation.createLogicalRelation(indexFsRelation, updatedOutput) + transformRelation(index, provider, l, useBucketSpec) // Given that the index may have top level field for a nested one // it is needed to transform the projection to use that index field @@ -113,262 +80,6 @@ class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { } } - /** - * Transform the current plan to utilize the given index along with newly appended source files. - * - * With HybridScan, indexes with newly appended files to its source relation are also - * eligible and we reconstruct new plans for the appended files so as to merge with - * bucketed index data correctly. - * - * @param index Index used in transformation of plan. - * @param plan Current logical plan. - * @param useBucketSpec Option whether to use BucketSpec for reading index data. - * @param useBucketUnionForAppended Option whether to use BucketUnion to merge appended data. - * @return Transformed logical plan that leverages an index and merges appended data. - */ - override protected[rules] def transformPlanToUseHybridScan( - index: IndexLogEntry, - plan: LogicalPlan, - useBucketSpec: Boolean, - useBucketUnionForAppended: Boolean): LogicalPlan = { - val provider = Hyperspace.getContext(spark).sourceProviderManager - var unhandledAppendedFiles: Seq[Path] = Nil - // Get transformed plan with index data and appended files if applicable. - val indexPlan = plan transformUp { - // Use transformUp here as currently one relation is allowed (pre-requisite). - // The transformed plan will have LogicalRelation as a child; for example, LogicalRelation - // can be transformed to 'Project -> Filter -> LogicalRelation'. Thus, with transformDown, - // it will be matched again and transformed recursively which causes stack overflow exception. - case l: LeafNode if provider.isSupportedRelation(l) => - val relation = provider.getRelation(l) - val (filesDeleted, filesAppended) = - if (!HyperspaceConf.hybridScanEnabled(spark) && index.hasSourceUpdate) { - // If the index contains the source update info, it means the index was validated - // with the latest signature including appended files and deleted files, but - // index data is not updated with those files. Therefore, we need to handle - // appendedFiles and deletedFiles in IndexLogEntry. - (index.deletedFiles, index.appendedFiles.map(f => new Path(f.name)).toSeq) - } else { - val curFiles = relation.allFiles.map(f => - FileInfo(f, index.fileIdTracker.addFile(f), asFullPath = true)) - if (HyperspaceConf.hybridScanDeleteEnabled(spark) && index.hasLineageColumn) { - val (exist, nonExist) = curFiles.partition(index.sourceFileInfoSet.contains) - val filesAppended = nonExist.map(f => new Path(f.name)) - if (exist.length < index.sourceFileInfoSet.size) { - (index.sourceFileInfoSet -- exist, filesAppended) - } else { - (Nil, filesAppended) - } - } else { - // Append-only implementation of getting appended files for efficiency. - // It is guaranteed that there is no deleted files via the condition - // 'deletedCnt == 0 && commonCnt > 0' in isHybridScanCandidate function. - ( - Nil, - curFiles.filterNot(index.sourceFileInfoSet.contains).map(f => new Path(f.name))) - } - } - - val filesToRead = { - if (useBucketSpec || !index.hasParquetAsSourceFormat || filesDeleted.nonEmpty || - relation.partitionSchema.nonEmpty) { - // Since the index data is in "parquet" format, we cannot read source files - // in formats other than "parquet" using one FileScan node as the operator requires - // files in one homogenous format. To address this, we need to read the appended - // source files using another FileScan node injected into the plan and subsequently - // merge the data into the index data. Please refer below [[Union]] operation. - // In case there are both deleted and appended files, we cannot handle the appended - // files along with deleted files as source files do not have the lineage column which - // is required for excluding the index data from deleted files. - // If the source relation is partitioned, we cannot read the appended files with the - // index data as the schema of partitioned files are not equivalent to the index data. - unhandledAppendedFiles = filesAppended - index.content.files - } else { - // If BucketSpec of index data isn't used (e.g., in the case of FilterIndex currently) - // and the source format is parquet, we could read the appended files along - // with the index data. - index.content.files ++ filesAppended - } - } - - // In order to handle deleted files, read index data with the lineage column so that - // we could inject Filter-Not-In conditions on the lineage column to exclude the indexed - // rows from the deleted files. - val newSchema = StructType( - index.schema.filter(s => - relation.plan.schema.contains(s) || (filesDeleted.nonEmpty && s.name.equals( - IndexConstants.DATA_FILE_NAME_ID)))) - - def fileIndex: InMemoryFileIndex = { - new InMemoryFileIndex(spark, filesToRead, Map(), None) - } - - val newLocation = if (filesToRead.length == index.content.files.size) { - index.withCachedTag(IndexLogEntryTags.INMEMORYFILEINDEX_INDEX_ONLY)(fileIndex) - } else { - index.withCachedTag(plan, IndexLogEntryTags.INMEMORYFILEINDEX_HYBRID_SCAN)(fileIndex) - } - - val indexFsRelation = new IndexHadoopFsRelation( - newLocation, - new StructType(), - newSchema, - if (useBucketSpec) Some(index.bucketSpec) else None, - new ParquetFileFormat, - Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark, index) - - val updatedOutput = relation.plan.output - .filter(attr => indexFsRelation.schema.fieldNames.contains(attr.name)) - .map(_.asInstanceOf[AttributeReference]) - - if (filesDeleted.isEmpty) { - relation.createLogicalRelation(indexFsRelation, updatedOutput) - } else { - val lineageAttr = AttributeReference(IndexConstants.DATA_FILE_NAME_ID, LongType)() - val deletedFileIds = filesDeleted.map(f => Literal(f.id)).toArray - val rel = - relation.createLogicalRelation(indexFsRelation, updatedOutput ++ Seq(lineageAttr)) - val filterForDeleted = Filter(Not(In(lineageAttr, deletedFileIds)), rel) - Project(updatedOutput, OptimizeIn(filterForDeleted)) - } - } - - if (unhandledAppendedFiles.nonEmpty) { - // If there are unhandled appended files, we need to create additional plans - // by the following steps: - // Step 1) Generate a plan (planForAppended) from the current plan to read - // the appended files similar to indexPlan. - // Step 2) If Shuffle is required, perform shuffle for the plan. - // Step 3) Merge both indexPlan and planForAppended by using [[BucketUnion]] or [[Union]]. - // For more details, see https://github.com/microsoft/hyperspace/issues/150. - - val planForAppended = - transformPlanToReadAppendedFiles(index, plan, unhandledAppendedFiles) - if (useBucketUnionForAppended && useBucketSpec) { - // If Bucketing information of the index is used to read the index data, we need to - // shuffle the appended data in the same way to correctly merge with bucketed index data. - - // Although only numBuckets of BucketSpec is used in BucketUnion*, bucketColumnNames - // and sortColumnNames are shown in plan string. So remove sortColumnNames to avoid - // misunderstanding. - val bucketSpec = index.bucketSpec.copy(sortColumnNames = Nil) - - // Merge index plan & newly shuffled plan by using bucket-aware union. - BucketUnion( - Seq(indexPlan, transformPlanToShuffleUsingBucketSpec(bucketSpec, planForAppended)), - bucketSpec) - } else { - // If bucketing is not necessary (e.g. FilterIndexRule), we use [[Union]] to merge - // the appended data without additional shuffle. - Union(indexPlan, planForAppended) - } - } else { - indexPlan - } - } - - /** - * Transform the current plan to read the given appended source files. - * - * The result will be merged with the plan which is reading index data - * by using [[BucketUnion]] or [[Union]]. - * - * @param index Index used in transformation of plan. - * @param originalPlan Original plan. - * @param filesAppended Appended files to the source relation. - * @return Transformed linear logical plan for appended files. - */ - override protected[rules] def transformPlanToReadAppendedFiles( - index: IndexLogEntry, - originalPlan: LogicalPlan, - filesAppended: Seq[Path]): LogicalPlan = { - val provider = Hyperspace.getContext(spark).sourceProviderManager - // Transform the relation node to include appended files. - val planForAppended = originalPlan transformDown { - case l: LeafNode if provider.isSupportedRelation(l) => - val relation = provider.getRelation(l) - val options = relation.partitionBasePath - .map { basePath => - // Set "basePath" so that partitioned columns are also included in the output schema. - Map("basePath" -> basePath) - } - .getOrElse(Map()) - - val newLocation = index.withCachedTag( - originalPlan, - IndexLogEntryTags.INMEMORYFILEINDEX_HYBRID_SCAN_APPENDED) { - new InMemoryFileIndex(spark, filesAppended, options, None) - } - // Set the same output schema with the index plan to merge them using BucketUnion. - // Include partition columns for data loading. - val partitionColumns = relation.partitionSchema.map(_.name) - val updatedSchema = StructType(relation.plan.schema.filter(col => - index.schema.contains(col) || relation.partitionSchema.contains(col))) - val updatedOutput = relation.plan.output - .filter(attr => - index.schema.fieldNames.contains(attr.name) || partitionColumns.contains(attr.name)) - .map(_.asInstanceOf[AttributeReference]) - val newRelation = relation.createHadoopFsRelation( - newLocation, - updatedSchema, - relation.options + IndexConstants.INDEX_RELATION_IDENTIFIER) - relation.createLogicalRelation(newRelation, updatedOutput) - } - assert(!originalPlan.equals(planForAppended)) - planForAppended - } - - /** - * The method transforms the project part of a plan to support indexes on - * nested fields. - * - * For example, given the following query: - * {{{ - * df - * .filter("nested.leaf.cnt > 10 and nested.leaf.id == 'leaf_id9'") - * .select("Date", "nested.leaf.cnt") - * }}} - * - * Having this simple projection: - * {{{ - * Project [Date#100, nested#102.leaf.cnt] - * }}} - * - * The projection part should become: - * {{{ - * Project [Date#330, __hs_nested.nested.leaf.cnt#335] - * }}} - * - * @param project The project that needs to be transformed. - * @return The transformed project with support for nested indexed fields. - */ - protected[rules] def transformProject(project: Project): Project = { - val projectedFields = project.projectList.map { exp => - val fieldName = extractNamesFromExpression(exp).toKeep.head - val escapedFieldName = PlanUtils.prefixNestedField(fieldName) - val resolvedField = ResolverUtils.ResolvedColumn(escapedFieldName) - val attr = extractAttributeRef(exp, fieldName) - val fieldType = extractTypeFromExpression(exp, fieldName) - // Try to find it in the project transformed child. - getExprId(project.child, escapedFieldName) match { - case Some(exprId) => - val newAttr = attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( - exprId, - attr.qualifier) - resolvedField.projectName match { - case Some(projectName) => - Alias(newAttr, projectName)() - case None => - newAttr - } - case _ => - attr - } - } - project.copy(projectList = projectedFields) - } - /** * The method transforms the filter part of a plan to support indexes on * nested fields. The process is to go through all expression nodes and @@ -438,6 +149,56 @@ class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { filter.copy(condition = transformedCondition) } + /** + * The method transforms the project part of a plan to support indexes on + * nested fields. + * + * For example, given the following query: + * {{{ + * df + * .filter("nested.leaf.cnt > 10 and nested.leaf.id == 'leaf_id9'") + * .select("Date", "nested.leaf.cnt") + * }}} + * + * Having this simple projection: + * {{{ + * Project [Date#100, nested#102.leaf.cnt] + * }}} + * + * The projection part should become: + * {{{ + * Project [Date#330, __hs_nested.nested.leaf.cnt#335] + * }}} + * + * @param project The project that needs to be transformed. + * @return The transformed project with support for nested indexed fields. + */ + protected[rules] def transformProject(project: Project): Project = { + val projectedFields = project.projectList.map { exp => + val fieldName = extractNamesFromExpression(exp).toKeep.head + val escapedFieldName = PlanUtils.prefixNestedField(fieldName) + val resolvedField = ResolverUtils.ResolvedColumn(escapedFieldName) + val attr = extractAttributeRef(exp, fieldName) + val fieldType = extractTypeFromExpression(exp, fieldName) + // Try to find it in the project transformed child. + getExprId(project.child, escapedFieldName) match { + case Some(exprId) => + val newAttr = attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( + exprId, + attr.qualifier) + resolvedField.projectName match { + case Some(projectName) => + Alias(newAttr, projectName)() + case None => + newAttr + } + case _ => + attr + } + } + project.copy(projectList = projectedFields) + } + /** * Returns true if the given project is a supported project. If all of the registered * providers return None, this returns false. diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala new file mode 100644 index 000000000..48df35957 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala @@ -0,0 +1,56 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index.rules + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +import com.microsoft.hyperspace.{ActiveSparkSession, Hyperspace} +import com.microsoft.hyperspace.index.IndexConstants +import com.microsoft.hyperspace.index.sources.FileBasedRelation + +object RuleUtils extends ActiveSparkSession { + + /** + * Extract the relation node if the given logical plan is linear. + * + * @param plan Logical plan to extract a relation node from. + * @return If the plan is linear and the relation node is supported, the [[FileBasedRelation]] + * object that wraps the relation node. Otherwise None. + */ + def getRelation( + plan: LogicalPlan): Option[FileBasedRelation] = { + val provider = Hyperspace.getContext(spark).sourceProviderManager + val leaves = plan.collectLeaves() + if (leaves.size == 1 && provider.isSupportedRelation(leaves.head)) { + Some(provider.getRelation(leaves.head)) + } else { + None + } + } + + /** + * Check if an index was applied the given relation or not. + * This can be determined by an identifier in [[FileBasedRelation]]'s options. + * + * @param relation FileBasedRelation to check if an index is already applied. + * @return true if an index is applied to the given relation. Otherwise false. + */ + def isIndexApplied(relation: FileBasedRelation): Boolean = { + relation.options.exists(_.equals(IndexConstants.INDEX_RELATION_IDENTIFIER)) + } + +} diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala new file mode 100644 index 000000000..30f82d395 --- /dev/null +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala @@ -0,0 +1,345 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index.rules + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, IsNotNull} +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, NoopCache} +import org.apache.spark.sql.types.{IntegerType, StringType} + +import com.microsoft.hyperspace.Hyperspace +import com.microsoft.hyperspace.actions.Constants +import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants, IndexLogEntryTags} +import com.microsoft.hyperspace.util.{FileUtils, PathUtils} + +class BaseRuleHelperTest extends HyperspaceRuleSuite with SQLHelper { + override val systemPath = PathUtils.makeAbsolute("src/test/resources/ruleUtilsTest") + + val t1c1 = AttributeReference("t1c1", IntegerType)() + val t1c2 = AttributeReference("t1c2", StringType)() + val t1c3 = AttributeReference("t1c3", IntegerType)() + val t1c4 = AttributeReference("t1c4", StringType)() + val t2c1 = AttributeReference("t2c1", IntegerType)() + val t2c2 = AttributeReference("t2c2", StringType)() + val t2c3 = AttributeReference("t2c3", IntegerType)() + val t2c4 = AttributeReference("t2c4", StringType)() + + val t1Schema = schemaFromAttributes(t1c1, t1c2, t1c3, t1c4) + val t2Schema = schemaFromAttributes(t2c1, t2c2, t2c3, t2c4) + + var t1Relation: HadoopFsRelation = _ + var t2Relation: HadoopFsRelation = _ + var t1ScanNode: LogicalRelation = _ + var t2ScanNode: LogicalRelation = _ + var t1FilterNode: Filter = _ + var t2FilterNode: Filter = _ + var t1ProjectNode: Project = _ + var t2ProjectNode: Project = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val t1Location = + new InMemoryFileIndex(spark, Seq(new Path("t1")), Map.empty, Some(t1Schema), NoopCache) + val t2Location = + new InMemoryFileIndex(spark, Seq(new Path("t2")), Map.empty, Some(t2Schema), NoopCache) + + t1Relation = baseRelation(t1Location, t1Schema) + t2Relation = baseRelation(t2Location, t2Schema) + + t1ScanNode = LogicalRelation(t1Relation, Seq(t1c1, t1c2, t1c3, t1c4), None, false) + t2ScanNode = LogicalRelation(t2Relation, Seq(t2c1, t2c2, t2c3, t2c4), None, false) + + t1FilterNode = Filter(IsNotNull(t1c1), t1ScanNode) + t2FilterNode = Filter(IsNotNull(t2c1), t2ScanNode) + + t1ProjectNode = Project(Seq(t1c1, t1c3), t1FilterNode) + // Project [t1c1#0, t1c3#2] + // +- Filter isnotnull(t1c1#0) + // +- Relation[t1c1#0,t1c2#1,t1c3#2,t1c4#3] parquet + + t2ProjectNode = Project(Seq(t2c1, t2c3), t2FilterNode) + // Project [t2c1#4, t2c3#6] + // +- Filter isnotnull(t2c1#4) + // +- Relation[t2c1#4,t2c2#5,t2c3#6,t2c4#7] parquet + + createIndexLogEntry("t1i1", Seq(t1c1), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t1i2", Seq(t1c1, t1c2), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t1i3", Seq(t1c2), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t2i1", Seq(t2c1), Seq(t2c3), t2ProjectNode) + createIndexLogEntry("t2i2", Seq(t2c1, t2c2), Seq(t2c3), t2ProjectNode) + } + + test("Verify getCandidateIndex for hybrid scan") { + withTempPath { tempPath => + val indexManager = IndexCollectionManager(spark) + val df = spark.range(1, 5).toDF("id") + val dataPath = tempPath.getAbsolutePath + df.write.parquet(dataPath) + + val indexNameWithLineage = "index1" + val indexNameWithoutLineage = "index2" + withIndex(indexNameWithLineage, indexNameWithoutLineage) { + val readDf = spark.read.parquet(dataPath) + val expectedCommonSourceBytes = FileUtils.getDirectorySize(new Path(dataPath)) + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { + indexManager.create(readDf, IndexConfig(indexNameWithLineage, Seq("id"))) + } + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "false") { + indexManager.create(readDf, IndexConfig(indexNameWithoutLineage, Seq("id"))) + } + val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) + + def verify( + plan: LogicalPlan, + hybridScanEnabled: Boolean, + hybridScanDeleteEnabled: Boolean, + expectedCandidateIndexes: Seq[String], + expectedHybridScanTag: Option[Boolean], + expectedCommonSourceBytes: Option[Long]): Unit = { + withSQLConf( + IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> hybridScanEnabled.toString, + IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.99", + IndexConstants.INDEX_HYBRID_SCAN_DELETED_RATIO_THRESHOLD -> + (if (hybridScanDeleteEnabled) "0.99" else "0")) { + val relation = RuleUtils.getRelation(plan).get + val indexes = new BaseRuleHelper(spark) + .getCandidateIndexes(allIndexes, relation) + if (expectedCandidateIndexes.nonEmpty) { + assert(indexes.length === expectedCandidateIndexes.length) + assert(indexes.map(_.name).toSet.equals(expectedCandidateIndexes.toSet)) + indexes.foreach { index => + assert( + index.getTagValue(plan, IndexLogEntryTags.HYBRIDSCAN_REQUIRED) + === expectedHybridScanTag) + assert( + index.getTagValue(plan, IndexLogEntryTags.COMMON_SOURCE_SIZE_IN_BYTES) + === expectedCommonSourceBytes) + } + } else { + assert(indexes.isEmpty) + } + } + } + + // Verify that a candidate index is returned with the unmodified data files whether + // hybrid scan is enabled or not. + { + val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan + verify( + optimizedPlan, + hybridScanEnabled = false, + hybridScanDeleteEnabled = false, + expectedCandidateIndexes = Seq(indexNameWithLineage, indexNameWithoutLineage), + expectedHybridScanTag = None, + expectedCommonSourceBytes = None) + verify( + optimizedPlan, + hybridScanEnabled = true, + hybridScanDeleteEnabled = false, + expectedCandidateIndexes = Seq(indexNameWithLineage, indexNameWithoutLineage), + expectedHybridScanTag = Some(false), + expectedCommonSourceBytes = Some(expectedCommonSourceBytes)) + } + + // Scenario #1: Append new files. + df.write.mode("append").parquet(dataPath) + + { + val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan + verify( + optimizedPlan, + hybridScanEnabled = false, + hybridScanDeleteEnabled = false, + expectedCandidateIndexes = Seq(), + expectedHybridScanTag = None, + expectedCommonSourceBytes = None) + verify( + optimizedPlan, + hybridScanEnabled = true, + hybridScanDeleteEnabled = false, + expectedCandidateIndexes = Seq(indexNameWithLineage, indexNameWithoutLineage), + expectedHybridScanTag = Some(true), + expectedCommonSourceBytes = Some(expectedCommonSourceBytes)) + } + + // Scenario #2: Delete 1 file. + val deleteFilePath = new Path(readDf.inputFiles.head) + val updatedExpectedCommonSourceBytes = expectedCommonSourceBytes - FileUtils + .getDirectorySize(deleteFilePath) + FileUtils.delete(deleteFilePath) + + { + val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan + verify( + optimizedPlan, + hybridScanEnabled = false, + hybridScanDeleteEnabled = false, + expectedCandidateIndexes = Seq(), + expectedHybridScanTag = None, + expectedCommonSourceBytes = None) + verify( + optimizedPlan, + hybridScanEnabled = true, + hybridScanDeleteEnabled = false, + expectedCandidateIndexes = Seq(), + expectedHybridScanTag = None, + expectedCommonSourceBytes = None) + verify( + optimizedPlan, + hybridScanEnabled = true, + hybridScanDeleteEnabled = true, + expectedCandidateIndexes = Seq(indexNameWithLineage), + expectedHybridScanTag = Some(true), + expectedCommonSourceBytes = Some(updatedExpectedCommonSourceBytes)) + } + + // Scenario #3: Replace all files. + df.write.mode("overwrite").parquet(dataPath) + + { + val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan + verify( + optimizedPlan, + hybridScanEnabled = false, + hybridScanDeleteEnabled = false, + expectedCandidateIndexes = Seq(), + expectedHybridScanTag = None, + expectedCommonSourceBytes = None) + verify( + optimizedPlan, + hybridScanEnabled = true, + hybridScanDeleteEnabled = true, + expectedCandidateIndexes = Seq(), + expectedHybridScanTag = None, + expectedCommonSourceBytes = None) + } + } + } + } + + test("Verify Hybrid Scan candidate tags work as expected.") { + withTempPath { tempPath => + val indexManager = IndexCollectionManager(spark) + val df = spark.range(1, 10).toDF("id") + val dataPath = tempPath.getAbsolutePath + df.write.parquet(dataPath) + + withIndex("index1") { + val readDf = spark.read.parquet(dataPath) + val expectedCommonSourceBytes = FileUtils.getDirectorySize(new Path(dataPath)) + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { + indexManager.create(readDf, IndexConfig("index1", Seq("id"))) + } + + val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) + df.limit(5).write.mode("append").parquet(dataPath) + val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan + val relation = RuleUtils.getRelation(optimizedPlan).get + val ruleHelper = new BaseRuleHelper(spark) + + withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true") { + withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.99") { + val indexes = ruleHelper.getCandidateIndexes(allIndexes, relation) + assert( + indexes.head + .getTagValue(relation.plan, IndexLogEntryTags.IS_HYBRIDSCAN_CANDIDATE) + .get) + assert( + indexes.head + .getTagValue(relation.plan, IndexLogEntryTags.HYBRIDSCAN_RELATED_CONFIGS) + .get == Seq("0.99", "0.2")) + } + + withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.2") { + val indexes = ruleHelper.getCandidateIndexes(allIndexes, relation) + assert(indexes.isEmpty) + assert( + !allIndexes.head + .getTagValue(relation.plan, IndexLogEntryTags.IS_HYBRIDSCAN_CANDIDATE) + .get) + assert( + allIndexes.head + .getTagValue(relation.plan, IndexLogEntryTags.HYBRIDSCAN_RELATED_CONFIGS) + .get == Seq("0.2", "0.2")) + } + } + } + } + } + + test("Verify the location of injected shuffle for Hybrid Scan.") { + withTempPath { tempPath => + val dataPath = tempPath.getAbsolutePath + import spark.implicits._ + Seq((1, "name1", 12), (2, "name2", 10)) + .toDF("id", "name", "age") + .write + .mode("overwrite") + .parquet(dataPath) + + val df = spark.read.parquet(dataPath) + val query = df.filter(df("id") >= 3).select("id", "name") + val bucketSpec = BucketSpec(100, Seq("id"), Seq()) + val ruleHelper = new BaseRuleHelper(spark) + val shuffled = ruleHelper.transformPlanToShuffleUsingBucketSpec( + bucketSpec, + query.queryExecution.optimizedPlan) + + // Plan: Project ("id", "name") -> Filter ("id") -> Relation + // should be transformed to: + // Shuffle ("id") -> Project("id", "name") -> Filter ("id") -> Relation + assert(shuffled.collect { + case RepartitionByExpression(attrs, p: Project, numBuckets) => + assert(numBuckets == 100) + assert(attrs.size == 1) + assert(attrs.head.asInstanceOf[Attribute].name.contains("id")) + assert( + p.projectList.exists(_.name.equals("id")) && p.projectList.exists( + _.name.equals("name"))) + true + }.length == 1) + + // Check if the shuffle node should be injected where all bucket columns + // are available as its input. + // For example, + // Plan: Project ("id", "name") -> Filter ("id") -> Relation + // should be transformed: + // Project ("id", "name") -> Shuffle ("age") -> Filter ("id") -> Relation + // , NOT: + // Shuffle ("age") -> Project("id", "name") -> Filter ("id") -> Relation + // since Project doesn't include "age" column; Shuffle will be RoundRobinPartitioning + + val bucketSpec2 = BucketSpec(100, Seq("age"), Seq()) + val query2 = df.filter(df("id") <= 3).select("id", "name") + val shuffled2 = + ruleHelper.transformPlanToShuffleUsingBucketSpec( + bucketSpec2, + query2.queryExecution.optimizedPlan) + assert(shuffled2.collect { + case Project(_, RepartitionByExpression(attrs, _: Filter, numBuckets)) => + assert(numBuckets == 100) + assert(attrs.size == 1) + assert(attrs.head.asInstanceOf[Attribute].name.contains("age")) + true + }.length == 1) + } + } +} diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index a8b970826..73b5b8dc9 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -17,13 +17,13 @@ package com.microsoft.hyperspace.index.rules import org.apache.hadoop.fs.Path -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, IsNotNull} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, IsNotNull} import org.apache.spark.sql.catalyst.plans.{JoinType, SQLHelper} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, NoopCache} import org.apache.spark.sql.types.{IntegerType, StringType} +import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants, IndexLogEntryTags} import com.microsoft.hyperspace.shim.{JoinWithoutHint, RepartitionByExpressionWithOptionalNumPartitions} @@ -91,9 +91,9 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { val indexManager = IndexCollectionManager(spark) val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) + val t1Relation = RuleUtils.getRelation(t1ProjectNode).get + val t2Relation = RuleUtils.getRelation(t2ProjectNode).get val ruleHelper = new BaseRuleHelper(spark) - val t1Relation = ruleHelper.getRelation(t1ProjectNode).get - val t2Relation = ruleHelper.getRelation(t2ProjectNode).get assert(ruleHelper.getCandidateIndexes(allIndexes, t1Relation).length === 3) assert(ruleHelper.getCandidateIndexes(allIndexes, t2Relation).length === 2) @@ -114,8 +114,7 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { test("Verify get logical relation for non-linear plan.") { val joinNode = JoinWithoutHint(t1ProjectNode, t2ProjectNode, JoinType("inner"), None) - val ruleHelper = new BaseRuleHelper(spark) - val r = ruleHelper.getRelation(Project(Seq(t1c3, t2c3), joinNode)) + val r = RuleUtils.getRelation(Project(Seq(t1c3, t2c3), joinNode)) assert(r.isEmpty) } @@ -384,8 +383,7 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { } private def validateLogicalRelation(plan: LogicalPlan, expected: LogicalRelation): Unit = { - val ruleHelper = new BaseRuleHelper(spark) - val r = ruleHelper.getRelation(plan) + val r = RuleUtils.getRelation(plan) assert(r.isDefined) assert(r.get.plan.equals(expected)) } From 5f5034dfc01846bc482bb7a0a82586d7353165bd Mon Sep 17 00:00:00 2001 From: Andrei Ionescu Date: Fri, 7 May 2021 22:01:20 +0300 Subject: [PATCH 4/7] Add Spark 3 support related changes --- .../index/rules/FilterIndexRule.scala | 39 +++++++++---------- .../hyperspace/index/rules/PlanUtils.scala | 7 ++-- .../index/rules/BaseRuleHelperTest.scala | 2 +- .../index/rules/RuleUtilsTest.scala | 7 ++-- 4 files changed, 28 insertions(+), 27 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala index 3e68ac2a3..4cbe83ca3 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -118,37 +118,36 @@ object FilterIndexRule // See https://github.com/microsoft/hyperspace/issues/65 val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) - val resolvedOutputColumnsOpt = ResolverUtils.resolve( + val resolvedOutputColumns = ResolverUtils.resolve( spark, outputColumns, plan, ResolverUtils.resolveWithChildren, throwIfNotInSchema = false) - val resolvedFilterColumnsOpt = ResolverUtils.resolve( + .map(_.map(_.normalizedName)) + .getOrElse(outputColumns) + val resolvedFilterColumns = ResolverUtils.resolve( spark, filterColumns, plan, ResolverUtils.resolveWithChildren, throwIfNotInSchema = false) - - (resolvedOutputColumnsOpt, resolvedFilterColumnsOpt) match { - case (Some(resolvedOutputColumns), Some(resolvedFilterColumns)) => - val candidateIndexes = allIndexes.filter { index => - indexCoversPlan( - resolvedOutputColumns.map(_.normalizedName), - resolvedFilterColumns.map(_.normalizedName), - index.indexedColumns, - index.includedColumns) - } - - // Get candidate via file-level metadata validation. This is performed after pruning - // by column schema, as this might be expensive when there are numerous files in the - // relation or many indexes to be checked. - new BaseRuleHelper(spark).getCandidateIndexes(candidateIndexes, r) - - case _ => - Seq.empty + .map(_.map(_.normalizedName)) + .getOrElse(filterColumns) + + val candidateIndexes = allIndexes.filter { index => + indexCoversPlan( + resolvedOutputColumns, + resolvedFilterColumns, + index.indexedColumns, + index.includedColumns) } + + // Get candidate via file-level metadata validation. This is performed after pruning + // by column schema, as this might be expensive when there are numerous files in the + // relation or many indexes to be checked. + new BaseRuleHelper(spark).getCandidateIndexes(candidateIndexes, r) + case _ => // There is zero or more than one supported relations in Filter's sub-plan. Seq.empty diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala index 52ce1ce13..d17588a97 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala @@ -80,13 +80,14 @@ object PlanUtils { * @return A field name `.` (dot) separated if nested. */ def getChildNameFromStruct(field: GetStructField): String = { + val name = field.name.getOrElse(field.childSchema(field.ordinal).name) field.child match { case f: GetStructField => - s"${getChildNameFromStruct(f)}.${field.name.get}" + s"${getChildNameFromStruct(f)}.$name" case a: AttributeReference => - s"${a.name}.${field.name.get}" + s"${a.name}.$name" case _ => - s"${field.name.get}" + name } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala index 30f82d395..07dfc552f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala @@ -30,7 +30,7 @@ import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, Inde import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class BaseRuleHelperTest extends HyperspaceRuleSuite with SQLHelper { - override val systemPath = PathUtils.makeAbsolute("src/test/resources/ruleUtilsTest") + override val indexLocationDirName = "ruleUtilsTest" val t1c1 = AttributeReference("t1c1", IntegerType)() val t1c2 = AttributeReference("t1c2", StringType)() diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index 73b5b8dc9..e9b58a690 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -17,7 +17,8 @@ package com.microsoft.hyperspace.index.rules import org.apache.hadoop.fs.Path -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, IsNotNull} +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, IsNotNull} import org.apache.spark.sql.catalyst.plans.{JoinType, SQLHelper} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, NoopCache} @@ -151,7 +152,7 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { IndexConstants.INDEX_HYBRID_SCAN_DELETED_RATIO_THRESHOLD -> (if (hybridScanDeleteEnabled) "0.99" else "0")) { val ruleHelper = new BaseRuleHelper(spark) - val relation = ruleHelper.getRelation(plan).get + val relation = RuleUtils.getRelation(plan).get val indexes = ruleHelper .getCandidateIndexes(allIndexes, relation) if (expectedCandidateIndexes.nonEmpty) { @@ -284,7 +285,7 @@ class RuleUtilsTest extends HyperspaceRuleSuite with SQLHelper { df.limit(5).write.mode("append").parquet(dataPath) val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan val ruleHelper = new BaseRuleHelper(spark) - val relation = ruleHelper.getRelation(optimizedPlan).get + val relation = RuleUtils.getRelation(optimizedPlan).get withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true") { withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.99") { From d3fe005da9970e7028bed41242611c268a426aba Mon Sep 17 00:00:00 2001 From: Andrei Ionescu Date: Fri, 14 May 2021 09:56:19 +0300 Subject: [PATCH 5/7] Integrate feedback (2) --- .../index/rules/FilterIndexRule.scala | 2 +- .../index/rules/NestedRuleHelper.scala | 55 +++- .../hyperspace/index/rules/PlanUtils.scala | 37 ++- .../index/E2EHyperspaceRulesTest.scala | 3 +- .../index/rules/BaseRuleHelperTest.scala | 16 +- .../index/rules/HyperspaceRuleSuite.scala | 57 ++++ .../index/rules/NestedRuleHelperTest.scala | 302 ++++++++++++++++++ .../index/rules/PlanUtilsTest.scala | 197 ++++++++++++ 8 files changed, 640 insertions(+), 29 deletions(-) create mode 100644 src/test/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelperTest.scala create mode 100644 src/test/scala/com/microsoft/hyperspace/index/rules/PlanUtilsTest.scala diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala index 4cbe83ca3..92b398fbb 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -57,7 +57,7 @@ object FilterIndexRule findCoveringIndexes(filter, outputColumns, filterColumns, plan) FilterIndexRanker.rank(spark, filter, candidateIndexes) match { case Some(index) => - // Detect which whether the index contains nested fields. + // Detect whether the index contains nested fields. val indexHasNestedColumns = (index.indexedColumns ++ index.includedColumns) .exists(ResolverUtils.ResolvedColumn(_).isNested) val ruleHelper = if (indexHasNestedColumns) { diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala index 93964b53b..d39b1b600 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelper.scala @@ -17,7 +17,7 @@ package com.microsoft.hyperspace.index.rules import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, BinaryOperator, ExprId, GetStructField, IsNotNull, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference, BinaryOperator, Expression, ExprId, GetStructField, IsNotNull, NamedExpression, UnaryExpression} import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.types.StructType @@ -104,16 +104,28 @@ class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { * * The filter part should become: * {{{ - * Filter ((__hs_nested.nested.leaf.cnt#335 > 10) && - * (__hs_nested.nested#.leaf.id#336 = leaf_id9)) + * Filter (isnotnull(__hs_nested.nested.leaf.cnt#335) && + * isnotnull(__hs_nested.nested.leaf.id#335) && + * __hs_nested.nested.leaf.cnt#335 > 10) && + * __hs_nested.nested#.leaf.id#336 = leaf_id9)) * }}} * * @param filter The filter that needs to be transformed. * @return The transformed filter with support for nested indexed fields. */ protected[rules] def transformFilter(filter: Filter): Filter = { + + def transformGetStructField(g: GetStructField, n: String, e: ExprId) = { + val fieldType = extractTypeFromExpression(g, n) + val attr = extractAttributeRef(g, n) + attr.copy(PlanUtils.prefixNestedField(n), fieldType, attr.nullable, attr.metadata)( + e, attr.qualifier) + } + val names = extractNamesFromExpression(filter.condition) - val transformedCondition = filter.condition.transformDown { + + // remove IsNotNull operators on root of a nested field + val cleanedCondition = filter.condition.transformDown { case bo @ BinaryOperator(IsNotNull(AttributeReference(name, _, _, _)), other) => if (names.toDiscard.contains(name)) { other @@ -126,17 +138,19 @@ class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { } else { bo } + } + var transformedAttributes = Seq.empty[AttributeReference] + // modify nodes from GetStructField to Attribute access + val transformedCondition = cleanedCondition.transformDown { case g: GetStructField => val n = getChildNameFromStruct(g) if (names.toKeep.contains(n)) { val escapedFieldName = PlanUtils.prefixNestedField(n) getExprId(filter, escapedFieldName) match { case Some(exprId) => - val fieldType = extractTypeFromExpression(g, n) - val attr = extractAttributeRef(g, n) - attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( - exprId, - attr.qualifier) + val attr = transformGetStructField(g, n, exprId) + transformedAttributes = transformedAttributes :+ attr + attr case _ => g } @@ -146,7 +160,11 @@ class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { case o => o } - filter.copy(condition = transformedCondition) + // and IsNotNull guardrails for all indexed columns + val guardedCondition = transformedAttributes.foldRight(transformedCondition) { (attr, cond) => + And(IsNotNull(attr), cond) + } + filter.copy(condition = guardedCondition) } /** @@ -186,14 +204,19 @@ class NestedRuleHelper(spark: SparkSession) extends BaseRuleHelper(spark) { val newAttr = attr.copy(escapedFieldName, fieldType, attr.nullable, attr.metadata)( exprId, attr.qualifier) - resolvedField.projectName match { - case Some(projectName) => - Alias(newAttr, projectName)() - case None => - newAttr + exp match { + case Alias(_, aliasName) => + Alias(newAttr, aliasName)() + case _ => + resolvedField.projectName match { + case Some(projectedName) => + Alias(newAttr, projectedName)() + case _ => + newAttr + } } case _ => - attr + exp } } project.copy(projectList = projectedFields) diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala index d17588a97..1635b037e 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/PlanUtils.scala @@ -26,10 +26,33 @@ import com.microsoft.hyperspace.util.ResolverUtils object PlanUtils { /** - * The method extract field names from a Spark Catalyst [[Expression]]. + * The method extract field names from a Spark Catalyst [[Expression]] and + * returns them in the [[ExtractedNames]] form so we could know which field names + * to keep and which to remove. + * + * Given a plan like this: + * {{{ + * Project [Date#89, nested#94.leaf.cnt AS cnt#336, nested#94.leaf.id AS id#337] + * +- Filter ((isnotnull(nested#94) && (nested#94.leaf.cnt > 10)) && + * (nested#94.leaf.id = leaf_id9)) + * +- Relation[Date#89,RGUID#90,Query#91,imprs#92,clicks#93,nested#94] parquet + * }}} + * + * We need to extract the field names that needs to be transformed and which + * should be removed. For example the `isnotnull` is not longer needed because: + * - The nested field name is not part of the index. + * - The `isnotnull` construct checks for a nested field to not be null because + * trying to access the leaves when it's null would end up in exceptions. + * - The values stored in the index are not nested, they are flat. + * + * Executing this on the filter above will result in: + * {{{ + * ExtractedNames(Set(nested.leaf.cnt, nested.leaf.id), Set(nested)) + * }}} * * @param exp The Spark Catalyst expression from which to extract names. - * @return A set of distinct field names. + * @return An [[ExtractedNames]] object containing the extracted field names + * to keep and the ones to remove. */ def extractNamesFromExpression(exp: Expression): ExtractedNames = { @@ -51,6 +74,8 @@ object PlanUtils { extractNames(u.child, prevExpStrTypes :+ "isNotNull") case u: UnaryExpression => extractNames(u.child, prevExpStrTypes :+ "unary") + case e: Expression => + e.children.flatMap(i => extractNames(i, prevExpStrTypes :+ s"${e.nodeName}")).toSet case _ => Set.empty[(String, Seq[String])] } @@ -121,16 +146,16 @@ object PlanUtils { * * @param parent The parent Spark Catalyst [[Expression]] into which to replace. * @param needle The Spark Catalyst [[Expression]] needle to search for. - * @param repl The replacement Spark Catalyst [[Expression]]. + * @param replacement The replacement Spark Catalyst [[Expression]]. * @return A new Spark Catalyst [[Expression]]. */ - def replaceInSearchQuery( + def replaceExpression( parent: Expression, needle: Expression, - repl: Expression): Expression = { + replacement: Expression): Expression = { parent.mapChildren { c => if (c == needle) { - repl + replacement } else { c } diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala index b05f31c35..4614de96c 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTest.scala @@ -117,8 +117,7 @@ class E2EHyperspaceRulesTest extends QueryTest with HyperspaceSuite { } } - test( - "E2E test for filter query on nested columns") { + test("E2E test for filter query on nested columns") { val loc = testDir + "samplenestedparquet" val dataColumns = Seq("Date", "RGUID", "Query", "imprs", "clicks", "nested") SampleNestedData.save(spark, loc, dataColumns) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala index 07dfc552f..b515926f9 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/BaseRuleHelperTest.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, NoopCache} import org.apache.spark.sql.types.{IntegerType, StringType} -import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants, IndexLogEntryTags} -import com.microsoft.hyperspace.util.{FileUtils, PathUtils} +import com.microsoft.hyperspace.shim.{JoinWithoutHint, RepartitionByExpressionWithOptionalNumPartitions} +import com.microsoft.hyperspace.util.FileUtils class BaseRuleHelperTest extends HyperspaceRuleSuite with SQLHelper { override val indexLocationDirName = "ruleUtilsTest" @@ -307,7 +307,10 @@ class BaseRuleHelperTest extends HyperspaceRuleSuite with SQLHelper { // should be transformed to: // Shuffle ("id") -> Project("id", "name") -> Filter ("id") -> Relation assert(shuffled.collect { - case RepartitionByExpression(attrs, p: Project, numBuckets) => + case RepartitionByExpressionWithOptionalNumPartitions( + attrs, + p: Project, + Some(numBuckets)) => assert(numBuckets == 100) assert(attrs.size == 1) assert(attrs.head.asInstanceOf[Attribute].name.contains("id")) @@ -334,7 +337,12 @@ class BaseRuleHelperTest extends HyperspaceRuleSuite with SQLHelper { bucketSpec2, query2.queryExecution.optimizedPlan) assert(shuffled2.collect { - case Project(_, RepartitionByExpression(attrs, _: Filter, numBuckets)) => + case Project( + _, + RepartitionByExpressionWithOptionalNumPartitions( + attrs, + _: Filter, + Some(numBuckets))) => assert(numBuckets == 100) assert(attrs.size == 1) assert(attrs.head.asInstanceOf[Attribute].name.contains("age")) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala index fc2cc1c87..724db9e5e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleSuite.scala @@ -85,6 +85,63 @@ trait HyperspaceRuleSuite extends HyperspaceSuite { } } + def createIndexLogEntryFromStringsAndSchemas( + name: String, + indexCols: Seq[String], + includedCols: Seq[String], + indexColsSchema: Seq[StructField], + includedColsSchema: Seq[StructField], + plan: LogicalPlan, + numBuckets: Int = 10, + inputFiles: Seq[FileInfo] = Seq(), + writeLog: Boolean = true, + filenames: Seq[String] = defaultFileNames): IndexLogEntry = { + val signClass = new RuleTestHelper.TestSignatureProvider().getClass.getName + + LogicalPlanSignatureProvider.create(signClass).signature(plan) match { + case Some(s) => + val sourcePlanProperties = SparkPlan.Properties( + Seq( + Relation( + Seq("dummy"), + Hdfs(Properties(Content(Directory("/", files = inputFiles)))), + "schema", + "format", + Map())), + null, + null, + LogicalPlanFingerprint(LogicalPlanFingerprint.Properties(Seq(Signature(signClass, s))))) + + val indexFiles = getIndexDataFilesPaths(name, filenames).map { path => + new FileStatus(10, false, 1, 10, 10, path) + } + + val indexLogEntry = IndexLogEntry.create( + name, + CoveringIndex( + CoveringIndex.Properties( + CoveringIndex.Properties + .Columns(indexCols, includedCols), + IndexLogEntry.schemaString( + StructType(indexColsSchema ++ includedColsSchema) + ), + numBuckets, + Map())), + Content.fromLeafFiles(indexFiles, new FileIdTracker).get, + Source(SparkPlan(sourcePlanProperties)), + Map()) + + val logManager = new IndexLogManagerImpl(getIndexRootPath(name)) + indexLogEntry.state = Constants.States.ACTIVE + if (writeLog) { + assert(logManager.writeLog(0, indexLogEntry)) + } + indexLogEntry + + case None => throw HyperspaceException("Invalid plan for index dataFrame.") + } + } + def getIndexDataFilesPaths( indexName: String, filenames: Seq[String] = defaultFileNames): Seq[Path] = diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelperTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelperTest.scala new file mode 100644 index 000000000..288f4e4ad --- /dev/null +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/NestedRuleHelperTest.scala @@ -0,0 +1,302 @@ +/* + * Copyright (2021) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index.rules + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference, Cast, EqualTo, GetStructField, GreaterThan, IsNotNull, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Project} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, NoopCache} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +import com.microsoft.hyperspace.index.IndexLogEntry +import com.microsoft.hyperspace.index.sources.default.DefaultFileBasedRelation + +class NestedRuleHelperTest extends HyperspaceRuleSuite { + + override val indexLocationDirName = "ruleUtilsTest" + + val simpleAttr1 = AttributeReference("simpleAttr", StringType)() + val nestedStructSchema = StructType( + Seq( + StructField( + "leaf", + StructType( + Seq(StructField("id", StringType, true), StructField("cnt", IntegerType, true))), + true))) + val nestedAttrParent = AttributeReference("nestedAttr", nestedStructSchema)() + val nestedField_Id = GetStructField(GetStructField(nestedAttrParent, 0), 0) + val nestedField_Cnt = GetStructField(GetStructField(nestedAttrParent, 0), 1) + + val nestedField_Id_Transformed_4_Rel = + AttributeReference("__hs_nested.nestedAttr.leaf.id", StringType)() + val nestedField_Cnt_Transformed_4_Rel = + AttributeReference("__hs_nested.nestedAttr.leaf.cnt", IntegerType)() + + val fullSchema = schemaFromAttributes(simpleAttr1, nestedAttrParent) + val fullSchemaTransformed = schemaFromAttributes( + simpleAttr1, + nestedField_Id_Transformed_4_Rel, + nestedField_Cnt_Transformed_4_Rel) + + var rel1: HadoopFsRelation = _ + var rel1Transformed: HadoopFsRelation = _ + var scanNode: LogicalRelation = _ + var scanNodeTransformed: LogicalRelation = _ + var filterNode: Filter = _ + var projectNode: Project = _ + + var nestedIndexLogEntry: IndexLogEntry = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val loc1 = + new InMemoryFileIndex(spark, Seq(new Path("t1")), Map.empty, Some(fullSchema), NoopCache) + + rel1 = baseRelation(loc1, fullSchema) + scanNode = LogicalRelation(rel1, Seq(simpleAttr1, nestedAttrParent), None, false) + + rel1Transformed = baseRelation(loc1, fullSchemaTransformed) + scanNodeTransformed = LogicalRelation( + rel1, + Seq(simpleAttr1, nestedField_Id_Transformed_4_Rel, nestedField_Cnt_Transformed_4_Rel), + None, + false) + + /** + * Filter ((isnotnull(nested#106) AND (nested#106.leaf.cnt > 10)) AND + * (nested#106.leaf.id = leaf_id9)) + */ + filterNode = Filter( + And( + And(IsNotNull(nestedAttrParent), GreaterThan(nestedField_Cnt, Literal(10))), + EqualTo(nestedField_Id, Literal("leaf_id9"))), + scanNodeTransformed) + + /** + * Project [simpleAttr#101, nested#106.leaf.cnt AS cnt#1030, nested#106.leaf.id AS id#1031] + * +- Filter ((isnotnull(nested#106) AND (nested#106.leaf.cnt > 10)) AND + * (nested#106.leaf.id = leaf_id9)) + */ + projectNode = Project( + Seq(simpleAttr1, Alias(nestedField_Cnt, "cnt")(), Alias(nestedField_Id, "id")()), + filterNode) + + nestedIndexLogEntry = createIndexLogEntryFromStringsAndSchemas( + "q1", + Seq("__hs_nested.nestedAttr.leaf.cnt"), + Seq("__hs_nested.nestedAttr.leaf.id"), + Seq(StructField("__hs_nested.nestedAttr.leaf.cnt", IntegerType)), + Seq(StructField("__hs_nested.nestedAttr.leaf.id", StringType)), + projectNode) + } + + test("testTransformPlanToUseIndexOnlyScan") { + val nestedRuleHelper = new NestedRuleHelper(spark) + val r = + nestedRuleHelper.transformPlanToUseIndexOnlyScan(nestedIndexLogEntry, projectNode, false) + val expected = "\\!Project \\[simpleAttr#\\d+, " + + "nestedAttr#\\d+\\.leaf\\.cnt AS cnt#\\d+, nestedAttr#\\d+\\.leaf\\.id AS id#\\d+]\n\\" + + "+\\- \\!Filter \\(\\(nestedAttr#\\d+\\.leaf\\.cnt > 10\\) (AND|&&) " + + "\\(nestedAttr#\\d+\\.leaf\\.id = leaf_id9\\)\\)\n" + + " \\+\\- Relation\\[] Hyperspace\\(Type: CI, Name: q1, LogVersion: 0\\)" + assert(expected.r.findFirstIn(r.treeString).isDefined) + } + + test("testGetFsRelationSchema - general use case") { + val nestedRuleHelper = new NestedRuleHelper(spark) + val fileBasedRelation = new DefaultFileBasedRelation(spark, scanNodeTransformed) + val r1 = nestedRuleHelper.getFsRelationSchema(fileBasedRelation, nestedIndexLogEntry) + assert(r1.fieldNames.toSeq == + Seq("__hs_nested.nestedAttr.leaf.cnt", "__hs_nested.nestedAttr.leaf.id")) + } + + test("testGetFsRelationSchema - empty") { + val nestedRuleHelper = new NestedRuleHelper(spark) + val fileBasedRelation = + new DefaultFileBasedRelation(spark, LogicalRelation(rel1, Seq(simpleAttr1), None, false)) + val r1 = nestedRuleHelper.getFsRelationSchema(fileBasedRelation, nestedIndexLogEntry) + assert(r1.fieldNames.isEmpty) + } + + test("testHasNestedColumns - filter w/ nested fields") { + val nestedRuleHelper = new NestedRuleHelper(spark) + assert(nestedRuleHelper.hasNestedColumns(filterNode, nestedIndexLogEntry)) + } + + test("testHasNestedColumns - filter w/o nested fields") { + + /** + * Filter (isnotnull(simpleAttr#1) AND (simpleAttr#1 = a)) + */ + val filterNode2 = Filter( + And(IsNotNull(simpleAttr1), EqualTo(simpleAttr1, Literal("a"))), + LogicalRelation(rel1, Seq(simpleAttr1), None, false)) + + /** + * Project [simpleAttr#101, nested#106.leaf.cnt AS cnt#1030, nested#106.leaf.id AS id#1031] + * +- Filter ((isnotnull(nested#106) AND (nested#106.leaf.cnt > 10)) AND + * (nested#106.leaf.id = leaf_id9)) + */ + val projectNode2 = Project(Seq(simpleAttr1, Alias(simpleAttr1, "another")()), filterNode2) + + val nestedIndexLogEntry2 = + createIndexLogEntry("q2", Seq(simpleAttr1), Seq(simpleAttr1), projectNode2) + + val nestedRuleHelper = new NestedRuleHelper(spark) + assert(!nestedRuleHelper.hasNestedColumns(filterNode2, nestedIndexLogEntry2)) + } + + test("testHasNestedColumns - project w/ nested fields") { + val nestedRuleHelper = new NestedRuleHelper(spark) + assert(nestedRuleHelper.hasNestedColumns(projectNode, nestedIndexLogEntry)) + } + + test("testHasNestedColumns - project w/o nested fields") { + + /** + * Filter (isnotnull(simpleAttr#1) AND (simpleAttr#1 = a)) + */ + val filterNode2 = Filter( + And(IsNotNull(simpleAttr1), EqualTo(simpleAttr1, Literal("a"))), + LogicalRelation(rel1, Seq(simpleAttr1), None, false)) + + /** + * Project [simpleAttr#101, nested#106.leaf.cnt AS cnt#1030, nested#106.leaf.id AS id#1031] + * +- Filter ((isnotnull(nested#106) AND (nested#106.leaf.cnt > 10)) AND + * (nested#106.leaf.id = leaf_id9)) + */ + val projectNode2 = Project(Seq(simpleAttr1, Alias(simpleAttr1, "another")()), filterNode2) + + nestedIndexLogEntry = + createIndexLogEntry("q3", Seq(simpleAttr1), Seq(simpleAttr1), projectNode2) + + val nestedRuleHelper = new NestedRuleHelper(spark) + assert(!nestedRuleHelper.hasNestedColumns(projectNode2, nestedIndexLogEntry)) + } + + test("testTransformFilter - general use case") { + val nestedRuleHelper = new NestedRuleHelper(spark) + val newFilter = nestedRuleHelper.transformFilter(filterNode) + + val expectedTreeString = + "Filter \\(isnotnull\\(__hs_nested.nestedAttr.leaf.cnt#\\d+\\) (AND|&&) " + + "\\(isnotnull\\(__hs_nested\\.nestedAttr\\.leaf\\.id#\\d+\\) (AND|&&) " + + "\\(\\(__hs_nested\\.nestedAttr\\.leaf\\.cnt#\\d+ > 10\\) (AND|&&) " + + "\\(__hs_nested\\.nestedAttr\\.leaf\\.id#\\d+ = leaf_id9\\)\\)\\)\\)" + assert(expectedTreeString.r.findFirstIn(newFilter.treeString).isDefined) + } + + test("testTransformFilter - two column equality and cast") { + + /** + * Filter (isnotnull(nestedAttr#1) AND + * (cast(nestedAttr#1.leaf.id as string) = nestedAttr#1.leaf.cnt)) + */ + val filterNode2 = Filter( + And( + IsNotNull(nestedAttrParent), + EqualTo(Cast(nestedField_Id, StringType), nestedField_Cnt)), + scanNodeTransformed) + + /** + * Project [simpleAttr#101, nested#106.leaf.cnt AS cnt#1030, nested#106.leaf.id AS id#1031] + * +- Filter ((isnotnull(nested#106) AND (nested#106.leaf.cnt > 10)) AND + * (nested#106.leaf.id = leaf_id9)) + */ + val projectNode2 = Project( + Seq(simpleAttr1, Alias(nestedField_Cnt, "cnt")(), Alias(nestedField_Id, "id")()), + filterNode2) + + createIndexLogEntryFromStringsAndSchemas( + "q4", + Seq("__hs_nested.nestedAttr.leaf.cnt"), + Seq("__hs_nested.nestedAttr.leaf.id"), + Seq(StructField("__hs_nested.nestedAttr.leaf.cnt", IntegerType)), + Seq(StructField("__hs_nested.nestedAttr.leaf.id", StringType)), + projectNode2) + + val nestedRuleHelper = new NestedRuleHelper(spark) + val newFilter = nestedRuleHelper.transformFilter(filterNode2) + + val expectedTreeString = + "'Filter \\(isnotnull\\(__hs_nested\\.nestedAttr\\.leaf\\.id#\\d+\\) (AND|&&) " + + "\\(isnotnull\\(__hs_nested\\.nestedAttr\\.leaf\\.cnt#\\d+\\) (AND|&&) " + + "\\(cast\\(__hs_nested\\.nestedAttr\\.leaf\\.id#\\d+ as string\\) = " + + "__hs_nested\\.nestedAttr\\.leaf\\.cnt#\\d+\\)\\)\\)" + assert(expectedTreeString.r.findFirstIn(newFilter.treeString).isDefined) + } + + test("testGetUpdatedOutput") {} + + test("testTransformProject - general use case") { + val nestedRuleHelper = new NestedRuleHelper(spark) + val newProject = nestedRuleHelper.transformProject(projectNode) + + val expectedTreeString = "Project \\[simpleAttr#\\d+, " + + "__hs_nested\\.nestedAttr\\.leaf\\.cnt#\\d+ AS cnt#\\d+, " + + "__hs_nested\\.nestedAttr\\.leaf\\.id#\\d+ AS id#\\d+]" + assert(expectedTreeString.r.findFirstIn(newProject.treeString).isDefined) + } + + test("testTransformProject - different alias use case") { + + /** + * Project [simpleAttr#101, nested#106.leaf.cnt AS cnt_2#1030, nested#106.leaf.id AS id_2#1031] + * +- Filter ((isnotnull(nested#106) AND (nested#106.leaf.cnt > 10)) AND + * (nested#106.leaf.id = leaf_id9)) + */ + val projectNode2 = Project( + Seq(simpleAttr1, Alias(nestedField_Cnt, "cnt_2")(), Alias(nestedField_Id, "id_2")()), + filterNode) + + createIndexLogEntryFromStringsAndSchemas( + "q5", + Seq("__hs_nested.nestedAttr.leaf.cnt"), + Seq("__hs_nested.nestedAttr.leaf.id"), + Seq(StructField("__hs_nested.nestedAttr.leaf.cnt", IntegerType)), + Seq(StructField("__hs_nested.nestedAttr.leaf.id", StringType)), + projectNode2) + + val nestedRuleHelper = new NestedRuleHelper(spark) + val newProject = nestedRuleHelper.transformProject(projectNode2) + + val expectedTreeString = "Project \\[simpleAttr#\\d+, " + + "__hs_nested\\.nestedAttr\\.leaf\\.cnt#\\d+ AS cnt_2#\\d+, " + + "__hs_nested\\.nestedAttr\\.leaf\\.id#\\d+ AS id_2#\\d+]" + assert(expectedTreeString.r.findFirstIn(newProject.treeString).isDefined) + } + + test("testTransformProject - no nested fields") { + + /** + * Project [simpleAttr#101, nested#106.leaf.cnt AS cnt#1030, nested#106.leaf.id AS id#1031] + * +- Filter ((isnotnull(nested#106) AND (nested#106.leaf.cnt > 10)) AND + * (nested#106.leaf.id = leaf_id9)) + */ + val projectNode2 = Project(Seq(simpleAttr1, Alias(simpleAttr1, "another")()), filterNode) + + nestedIndexLogEntry = + createIndexLogEntry("q6", Seq(simpleAttr1), Seq(simpleAttr1), projectNode2) + + val nestedRuleHelper = new NestedRuleHelper(spark) + val newProject = nestedRuleHelper.transformProject(projectNode2) + + val expectedTreeString = "Project \\[simpleAttr#\\d+, simpleAttr#\\d+ AS another#\\d+]" + assert(expectedTreeString.r.findFirstIn(newProject.treeString).isDefined) + } +} diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/PlanUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/PlanUtilsTest.scala new file mode 100644 index 000000000..4da60df51 --- /dev/null +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/PlanUtilsTest.scala @@ -0,0 +1,197 @@ +/* + * Copyright (2021) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index.rules + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.expressions.GetStructField +import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.functions.{col, concat, lit} +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + +import com.microsoft.hyperspace.SampleNestedData +import com.microsoft.hyperspace.index.HyperspaceSuite +import com.microsoft.hyperspace.util.PathUtils + +class PlanUtilsTest extends HyperspaceSuite { + override val indexLocationDirName = "planUtilsTest/samplenestedplanutils" + var df: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1) + val fileSystem = new Path(indexLocationDirName).getFileSystem(new Configuration) + val dataColumns = Seq("Date", "RGUID", "Query", "imprs", "clicks", "nested") + fileSystem.delete(new Path(indexLocationDirName), true) + SampleNestedData.save(spark, indexLocationDirName, dataColumns) + df = spark.read.parquet(indexLocationDirName) + } + + before { + clearCache() + } + + test("testPrefixNestedField") { + assert(PlanUtils.prefixNestedField("a") === "a") + assert(PlanUtils.prefixNestedField("a.b.c") === "__hs_nested.a.b.c") + intercept[AssertionError] { + assert(PlanUtils.prefixNestedField("__hs_nested.d.e.f") === "") + } + } + + test("testExtractTypeFromExpression") { + val query = df + .filter("nested.leaf.cnt * 10 > 100 and nested.leaf.id == 'leaf_id9' and clicks > 0") + .select(col("Date"), concat(col("nested.leaf.cnt"), col("nested.leaf.id")).as("cnctstr")) + val condition = + query.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition + val res1 = PlanUtils.extractTypeFromExpression(condition, "nested.leaf.cnt") + assert(res1.isInstanceOf[IntegerType]) + val res2 = PlanUtils.extractTypeFromExpression(condition, "nested.leaf.id") + assert(res2.isInstanceOf[StringType]) + val res3 = PlanUtils.extractTypeFromExpression(condition, "clicks") + assert(res3.isInstanceOf[IntegerType]) + + intercept[NoSuchElementException] { + PlanUtils.extractTypeFromExpression(condition, "NotThere") + } + } + + test("testExtractAttributeRef") { + val query = df + .filter("nested.leaf.cnt * 10 > 100 and nested.leaf.id == 'leaf_id9' and clicks > 0") + .select(col("Date"), concat(col("nested.leaf.cnt"), col("nested.leaf.id")).as("cnctstr")) + val condition = + query.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition + val res1 = PlanUtils.extractAttributeRef(condition, "nested.leaf.cnt") + assert(res1.dataType.isInstanceOf[StructType]) + assert(res1.name === "nested") + val res2 = PlanUtils.extractAttributeRef(condition, "nested.leaf.id") + assert(res2.dataType.isInstanceOf[StructType]) + assert(res2.name === "nested") + val res3 = PlanUtils.extractAttributeRef(condition, "clicks") + assert(res3.dataType.isInstanceOf[IntegerType]) + assert(res3.name === "clicks") + + intercept[NoSuchElementException] { + PlanUtils.extractAttributeRef(condition, "NotThere") + } + } + + test("testGetChildNameFromStruct") { + val query = df + .filter("nested.leaf.cnt * 10 > 100 and nested.leaf.id == 'leaf_id9' and clicks > 0") + .select(col("Date"), concat(col("nested.leaf.cnt"), col("nested.leaf.id")).as("cnctstr")) + var gStructElems = Seq.empty[GetStructField] + query.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition.foreach { + case g: GetStructField => gStructElems :+= g + case _ => + } + assert(PlanUtils.getChildNameFromStruct(gStructElems(0)) === "nested.leaf.cnt") + assert(PlanUtils.getChildNameFromStruct(gStructElems(1)) === "nested.leaf") + assert(PlanUtils.getChildNameFromStruct(gStructElems(2)) === "nested.leaf.id") + assert(PlanUtils.getChildNameFromStruct(gStructElems(3)) === "nested.leaf") + } + + test("testExtractSearchQuery") { + val query = df + .filter("nested.leaf.cnt * 10 > 100 and nested.leaf.id == 'leaf_id9'") + .select(col("Date"), concat(col("nested.leaf.cnt"), col("nested.leaf.id")).as("cnctstr")) + val (parent, exp) = PlanUtils.extractSearchQuery( + query.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition, + "nested.leaf.id") + assert(parent.toString().matches("\\(nested#\\d+\\.leaf\\.id = leaf_id9\\)")) + assert(exp.toString().matches("nested#\\d+\\.leaf\\.id")) + + val query2 = df + .select( + col("Date"), + concat(col("nested.leaf.cnt"), lit("-"), col("nested.leaf.id")).as("cnctstr")) + .filter("cnctstr == 'leaf_id9-21'") + val condition2 = + query2.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition + val (parent2, exp2) = PlanUtils.extractSearchQuery(condition2, "nested.leaf.id") + assert( + parent2 + .toString() + .matches( + "concat\\(cast\\(nested#\\d+\\.leaf\\.cnt as string\\), \\-, nested#\\d+\\.leaf\\.id\\)")) + assert(exp2.toString().matches("nested#\\d+\\.leaf\\.id")) + val (parent3, exp3) = PlanUtils.extractSearchQuery(condition2, "nested.leaf.cnt") + assert(parent3.toString().matches("cast\\(nested#\\d+\\.leaf\\.cnt as string\\)")) + assert(exp3.toString().matches("nested#\\d+\\.leaf\\.cnt")) + } + + test("testExtractNamesFromExpression") { + // Filter with `and` + val query1 = df + .filter("nested.leaf.cnt > 10 and nested.leaf.id == 'leaf_id9'") + .select("Date", "nested.leaf.cnt", "nested.leaf.id") + val res1 = + PlanUtils.extractNamesFromExpression( + query1.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition) + assert(res1.toKeep.nonEmpty) + assert(res1.toKeep === Set("nested.leaf.cnt", "nested.leaf.id")) + assert(res1.toDiscard.nonEmpty) + assert(res1.toDiscard === Set("nested")) + // Filter with multiplication in filter and to support `concat` in select + val query2 = df + .filter("nested.leaf.cnt * 10 > 100 and nested.leaf.id == 'leaf_id9'") + .select(col("Date"), concat(col("nested.leaf.cnt"), col("nested.leaf.id")).as("cnctstr")) + val res2 = + PlanUtils.extractNamesFromExpression( + query2.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition) + assert(res2.toKeep.nonEmpty) + assert(res2.toKeep === Set("nested.leaf.cnt", "nested.leaf.id")) + assert(res2.toDiscard.nonEmpty) + assert(res2.toDiscard === Set("nested")) + // Filter with `concat` + val query3 = df + .select( + col("Date"), + concat(col("nested.leaf.cnt"), lit("-"), col("nested.leaf.id")).as("cnctstr")) + .filter("cnctstr == 'leaf_id9-21'") + val res3 = + PlanUtils.extractNamesFromExpression( + query3.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition) + assert(res3.toKeep.nonEmpty) + assert(res3.toKeep === Set("nested.leaf.cnt", "nested.leaf.id")) + assert(res3.toDiscard.isEmpty) + } + + test("testReplaceExpression") { + val query = df + .select( + col("Date"), + concat(col("nested.leaf.cnt"), lit("-"), col("nested.leaf.id")).as("cnctstr")) + .filter("cnctstr == 'leaf_id9-21'") + val condition = + query.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition + val (parent, exp) = PlanUtils.extractSearchQuery(condition, "nested.leaf.id") + val res = PlanUtils.replaceExpression(parent, exp, lit("REPLACED").expr) + assert( + res + .toString() + .matches("concat\\(cast\\(nested#\\d+\\.leaf\\.cnt as string\\), \\-, REPLACED\\)")) + val condition2 = + query.queryExecution.optimizedPlan.children.head.asInstanceOf[Filter].condition + val (parent2, exp2) = PlanUtils.extractSearchQuery(condition2, "nested.leaf.cnt") + val res2 = PlanUtils.replaceExpression(parent2, exp2, lit("REPLACED").expr) + assert(res2.toString().matches("cast\\(REPLACED as string\\)")) + } +} From 374fbb38cf82504dba34641344f596ad9f0142f5 Mon Sep 17 00:00:00 2001 From: Andrei Ionescu Date: Mon, 17 May 2021 16:54:06 +0300 Subject: [PATCH 6/7] Integrate feedback (3) --- .../index/rules/FilterIndexRule.scala | 10 +----- .../index/rules/IndexPlanApplyHelper.scala | 34 +++++++++++++++++++ 2 files changed, 35 insertions(+), 9 deletions(-) create mode 100644 src/main/scala/com/microsoft/hyperspace/index/rules/IndexPlanApplyHelper.scala diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala index 92b398fbb..30d5f7092 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -57,15 +57,7 @@ object FilterIndexRule findCoveringIndexes(filter, outputColumns, filterColumns, plan) FilterIndexRanker.rank(spark, filter, candidateIndexes) match { case Some(index) => - // Detect whether the index contains nested fields. - val indexHasNestedColumns = (index.indexedColumns ++ index.includedColumns) - .exists(ResolverUtils.ResolvedColumn(_).isNested) - val ruleHelper = if (indexHasNestedColumns) { - new NestedRuleHelper(spark) - } else { - new BaseRuleHelper(spark) - } - + val ruleHelper = IndexPlanApplyHelper(index) val transformedPlan = ruleHelper.transformPlanToUseIndex( index, originalPlan, diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/IndexPlanApplyHelper.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/IndexPlanApplyHelper.scala new file mode 100644 index 000000000..90d9fe849 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/IndexPlanApplyHelper.scala @@ -0,0 +1,34 @@ +/* + * Copyright (2021) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index.rules + +import com.microsoft.hyperspace.index.IndexLogEntry +import com.microsoft.hyperspace.index.rules.FilterIndexRule.spark +import com.microsoft.hyperspace.util.ResolverUtils + +object IndexPlanApplyHelper { + def apply(indexLogEntry: IndexLogEntry): BaseRuleHelper = { + // Detect whether the index contains nested fields. + val indexHasNestedColumns = (indexLogEntry.indexedColumns ++ indexLogEntry.includedColumns) + .exists(ResolverUtils.ResolvedColumn(_).isNested) + if (indexHasNestedColumns) { + new NestedRuleHelper(spark) + } else { + new BaseRuleHelper(spark) + } + } +} From 3c476abbcbdbf21f8a4a2be59b8549576bcfac92 Mon Sep 17 00:00:00 2001 From: Andrei Ionescu Date: Wed, 19 May 2021 10:55:42 +0300 Subject: [PATCH 7/7] Rebase over master --- .../hyperspace/index/rules/CandidateIndexCollectorTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/CandidateIndexCollectorTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/CandidateIndexCollectorTest.scala index b46098324..0699b13dc 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/CandidateIndexCollectorTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/CandidateIndexCollectorTest.scala @@ -270,7 +270,7 @@ class CandidateIndexCollectorTest extends HyperspaceRuleSuite with SQLHelper { val allIndexes = Seq(latestIndexLogEntry(systemPath, "index1")) df.limit(5).write.mode("append").parquet(dataPath) val optimizedPlan = spark.read.parquet(dataPath).queryExecution.optimizedPlan - val relation = RuleUtils.getRelation(spark, optimizedPlan).get + val relation = RuleUtils.getRelation(optimizedPlan).get withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true") { withSQLConf(IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "0.99") {