From 2290de009c41e7037edf42a388f68a52e4177a57 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 30 Jun 2020 10:35:35 -0700 Subject: [PATCH 01/33] initial commit --- .../hyperspace/actions/CreateAction.scala | 14 +++++++---- .../hyperspace/actions/CreateActionBase.scala | 23 +++++++++++++++---- .../hyperspace/util/IndexNameUtils.scala | 19 +++++++++++++++ 3 files changed, 47 insertions(+), 9 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index b5954ff0e..d1181f7b8 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -17,12 +17,13 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.types.StructType - import com.microsoft.hyperspace.HyperspaceException + import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.util.LogicalPlanUtils +import com.microsoft.hyperspace.util.{IndexNameUtils, LogicalPlanUtils} class CreateAction( spark: SparkSession, @@ -33,7 +34,7 @@ class CreateAction( extends CreateActionBase(dataManager) with Action { final override lazy val logEntry: LogEntry = - getIndexLogEntry(spark, df, indexConfig, indexDataPath, sourceFiles(df)) + getIndexLogEntry(spark, df, indexConfig, indexDataPath, sourceFiles(df)) final override val transientState: String = CREATING @@ -65,8 +66,11 @@ class CreateAction( val validColumnNames = schema.fieldNames val indexedColumns = indexConfig.indexedColumns val includedColumns = indexConfig.includedColumns - indexedColumns.forall(validColumnNames.contains) && includedColumns.forall( - validColumnNames.contains) + + IndexNameUtils.resolve(spark, indexedColumns ++ includedColumns, validColumnNames) + + // indexedColumns.forall(validColumnNames.contains) && includedColumns.forall( + // validColumnNames.contains) } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 0477ce2ae..ba8c93d8b 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -20,9 +20,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer import com.microsoft.hyperspace.index.serde.LogicalPlanSerDeUtils +import com.microsoft.hyperspace.util.IndexNameUtils /** * CreateActionBase provides functionality to write dataframe as covering index. @@ -102,13 +104,17 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) IndexConstants.INDEX_NUM_BUCKETS, IndexConstants.INDEX_NUM_BUCKETS_DEFAULT.toString) .toInt - val selectedColumns = indexConfig.indexedColumns ++ indexConfig.includedColumns + + val dfColumnNames = df.schema.fieldNames + val resolvedIndexedColumns = indexConfig.indexedColumns.map(resolve(spark, _, dfColumnNames)) + val resolvedIncludedColumns = + indexConfig.includedColumns.map(resolve(spark, _, dfColumnNames)) + val selectedColumns = resolvedIndexedColumns ++ resolvedIncludedColumns val indexDataFrame = df.select(selectedColumns.head, selectedColumns.tail: _*) - val indexColNames = indexConfig.indexedColumns // run job val repartitionedIndexDataFrame = - indexDataFrame.repartition(numBuckets, indexColNames.map(df(_)): _*) + indexDataFrame.repartition(numBuckets, resolvedIndexedColumns.map(df(_)): _*) // Save the index with the number of buckets specified. repartitionedIndexDataFrame.write @@ -116,6 +122,15 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) repartitionedIndexDataFrame, indexDataPath.toString, numBuckets, - indexColNames) + resolvedIndexedColumns) + } + + private def resolve(spark: SparkSession, dst: String, srcs: Array[String]): String = { + srcs + .find(src => IndexNameUtils.resolve(spark, dst, src)) + .getOrElse { + throw HyperspaceException( + s"Column $dst could not be resolved from available columns $srcs") + } } } diff --git a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala index c509ea2ea..c4b683178 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala @@ -16,6 +16,9 @@ package com.microsoft.hyperspace.util +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.Resolver + /** * [[IndexNameUtils]] provides utility functions to format the index name as need. */ @@ -31,4 +34,20 @@ object IndexNameUtils { def normalizeIndexName(indexName: String): String = { indexName.trim.replaceAll("\\s+", "_") } + + def resolve(spark: SparkSession, firstString: String, secondString: String): Boolean = { + val resolver: Resolver = spark.sessionState.conf.resolver + resolver(firstString, secondString) + } + + def resolve(spark: SparkSession, firstString: String, secondStrings: Seq[String]): Boolean = { + secondStrings.exists(resolve(spark, firstString, _)) + } + + def resolve( + spark: SparkSession, + firstStrings: Seq[String], + secondStrings: Seq[String]): Boolean = { + firstStrings.forall(resolve(spark, _, secondStrings)) + } } From 8193f525dd0e81cb4122b01b379e921aa05e9c62 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 30 Jun 2020 11:05:41 -0700 Subject: [PATCH 02/33] added test cases for creation with different case --- .../hyperspace/actions/CreateActionBase.scala | 2 +- .../hyperspace/index/CreateIndexTests.scala | 21 +++++++++++++++---- .../hyperspace/index/HyperspaceSuite.scala | 10 +++++++++ 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index ba8c93d8b..0c7602e12 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -130,7 +130,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) .find(src => IndexNameUtils.resolve(spark, dst, src)) .getOrElse { throw HyperspaceException( - s"Column $dst could not be resolved from available columns $srcs") + s"Unexpected Exception: Column $dst could not be resolved from available columns $srcs") } } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala index 4b4f6d95e..8031ab433 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala @@ -17,14 +17,12 @@ package com.microsoft.hyperspace.index import org.apache.hadoop.fs.Path -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.DataFrame -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData, SparkInvolvedSuite} +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} import com.microsoft.hyperspace.util.FileUtils -class CreateIndexTests extends SparkFunSuite with SparkInvolvedSuite { - +class CreateIndexTests extends HyperspaceSuite { private val sampleData = SampleData.testData private val sampleParquetDataLocation = "src/test/resources/sampleparquet" private val indexSystemPath = "src/test/resources/indexLocation" @@ -87,6 +85,21 @@ class CreateIndexTests extends SparkFunSuite with SparkInvolvedSuite { assert(exception.getMessage.contains("Index config is not applicable to dataframe schema")) } + test("Index creation passes with columns of different case if case-sensitivity is false.") { + hyperspace.createIndex(df, IndexConfig("index1", Seq("qUeRy"), Seq("ImpRS"))) + val count = hyperspace.indexes.where(s"""name = "${indexConfig1.indexName}" """).count + assert(count == 1) + } + + test("Index creation fails with columns of different case if case-sensitivity is true.") { + withSparkConf("spark.sql.caseSensitive", true) { + val exception = intercept[HyperspaceException] { + hyperspace.createIndex(df, IndexConfig("index1", Seq("qUeRy"), Seq("ImpRS"))) + } + assert(exception.getMessage.contains("Index config is not applicable to dataframe schema.")) + } + } + test("Index creation fails since the dataframe has a filter node.") { val dfWithFilter = df.filter("Query='facebook'") val exception = intercept[HyperspaceException] { diff --git a/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala index a6ff09060..c80e96897 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala @@ -62,4 +62,14 @@ trait HyperspaceSuite extends SparkFunSuite with SparkInvolvedSuite { } } } + + protected def withSparkConf(confName: String, confValue: Any)(f: => Unit): Unit = { + val original = spark.conf.get(confName) + try { + spark.conf.set(confName, confValue.toString) + f + } finally { + spark.conf.set(confName, original) + } + } } From c4651a1946b0c504fcf635a69827b6502465878d Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 30 Jun 2020 12:52:14 -0700 Subject: [PATCH 03/33] improve tests for case insensitive index creation --- .../hyperspace/actions/CreateAction.scala | 6 +----- .../hyperspace/actions/CreateActionBase.scala | 15 +++++++++++++-- .../hyperspace/index/CreateIndexTests.scala | 12 ++++++++++-- 3 files changed, 24 insertions(+), 9 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index d1181f7b8..5960959bd 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -17,10 +17,9 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.types.StructType -import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.util.{IndexNameUtils, LogicalPlanUtils} @@ -68,9 +67,6 @@ class CreateAction( val includedColumns = indexConfig.includedColumns IndexNameUtils.resolve(spark, indexedColumns ++ includedColumns, validColumnNames) - - // indexedColumns.forall(validColumnNames.contains) && includedColumns.forall( - // validColumnNames.contains) } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 0c7602e12..d36290aba 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -51,8 +51,19 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val signatureProvider = LogicalPlanSignatureProvider.create() + val (resolvedIndexedColumns, resolvedIncludedColumns) = { + val dfColumnNames = df.schema.fieldNames + try { + // Try creating log entry with resolved column names. + (indexConfig.indexedColumns.map(resolve(spark, _, dfColumnNames)), + indexConfig.includedColumns.map(resolve(spark, _, dfColumnNames))) + } catch { + // Try creating index log entry with whatever the user passed. + case _: Exception => (indexConfig.indexedColumns, indexConfig.includedColumns) + } + } val schema = { - val allColumns = indexConfig.indexedColumns ++ indexConfig.includedColumns + val allColumns = resolvedIndexedColumns ++ resolvedIncludedColumns df.select(allColumns.head, allColumns.tail: _*).schema } @@ -80,7 +91,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) CoveringIndex( CoveringIndex.Properties( CoveringIndex.Properties - .Columns(indexConfig.indexedColumns, indexConfig.includedColumns), + .Columns(resolvedIndexedColumns, resolvedIncludedColumns), IndexLogEntry.schemaString(schema), numBuckets)), Content(path.toString, Seq()), diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala index 8031ab433..13788595e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala @@ -16,6 +16,8 @@ package com.microsoft.hyperspace.index +import scala.collection.mutable.WrappedArray + import org.apache.hadoop.fs.Path import org.apache.spark.sql.DataFrame @@ -87,8 +89,14 @@ class CreateIndexTests extends HyperspaceSuite { test("Index creation passes with columns of different case if case-sensitivity is false.") { hyperspace.createIndex(df, IndexConfig("index1", Seq("qUeRy"), Seq("ImpRS"))) - val count = hyperspace.indexes.where(s"""name = "${indexConfig1.indexName}" """).count - assert(count == 1) + val indexes = hyperspace.indexes.where(s"""name = "${indexConfig1.indexName}" """) + assert(indexes.count == 1) + assert( + indexes.head.getAs[WrappedArray[String]]("indexedColumns").head.equals("Query"), + "Indexed columns with wrong case are stored in metadata") + assert( + indexes.head.getAs[WrappedArray[String]]("includedColumns").head.equals("imprs"), + "Included columns with wrong case are stored in metadata") } test("Index creation fails with columns of different case if case-sensitivity is true.") { From 918a443f7cff378363856696c00a0acf47b7a60b Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 30 Jun 2020 15:15:21 -0700 Subject: [PATCH 04/33] Filter rule now supports case-insensitive matching --- .../index/rules/FilterIndexRule.scala | 37 ++++++++++++++++--- .../index/E2EHyperspaceRulesTests.scala | 12 ++++++ 2 files changed, 44 insertions(+), 5 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 d247f6897..8b737ae1f 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.CleanupAliases -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources._ @@ -59,12 +59,22 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { // such that its children would not have any Alias expressions. // Calling "references" on the expression in projectList ensures // we will get the correct (original) column names. - val projectColumnNames = CleanupAliases(project) + val baseRelationColumns: Seq[AttributeReference] = logicalRelation.output + val projectColumns: Seq[Option[AttributeReference]] = CleanupAliases(project) .asInstanceOf[Project] .projectList - .map(_.references.map(_.asInstanceOf[AttributeReference].name)) + .map(_.references.map(_.asInstanceOf[AttributeReference])) .flatMap(_.toSeq) - val filterColumnNames = condition.references.map(_.name).toSeq + .map(findBaseReference(_, baseRelationColumns)) + val filterColumns: Seq[Option[AttributeReference]] = condition.references.toSeq + .map(findBaseReference(_, baseRelationColumns)) + + if (projectColumns.exists(_.isEmpty) || filterColumns.exists(_.isEmpty)) { + return project + } + + val projectColumnNames = projectColumns.map(_.get.name) + val filterColumnNames = filterColumns.map(_.get.name) replaceWithIndexIfPlanCovered( project, @@ -81,6 +91,24 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { } } + /** + * This method is useful in finding an attribute from a list, which is semantically equal to + * a given attribute. This is required for e.g. to find the base attribute name for an + * attribute. + * For e.g. if the data source has column names in lower case e.g. c1, c2, c3... + * and a case-insensitive query looks for column C1, we can use this method to find which of the + * base columns (c1, c2, c3...) matches with the query column C1. + * + * @param attribute attribute searched for. + * @param baseReferences available set of base attributes to look from. + * @return optional matching attribute from base references. + */ + private def findBaseReference( + attribute: Attribute, + baseReferences: Seq[AttributeReference]): Option[AttributeReference] = { + baseReferences.find(_.semanticEquals(attribute)) + } + /** * For a given relation, check its available indexes and replace it with the top-ranked index * (according to cost model). @@ -184,7 +212,6 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { index.includedColumns, fsRelation.fileFormat) } - } /** diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 61f3cab69..4cba3ae07 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -95,6 +95,18 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite { verifyIndexUsage(query, Seq(getIndexFilesPath(indexConfig.indexName))) } + test("E2E test for case insensitive filter query utilizing indexes.") { + val df = spark.read.parquet(sampleParquetDataLocation) + val indexConfig = IndexConfig("filterIndex", Seq("C3"), Seq("C1")) + + hyperspace.createIndex(df, indexConfig) + + def query(): DataFrame = df.filter("C3 == 'facebook'").select("C3", "c1") + + // verify if case-insensitive index works with case-insensitive query + verifyIndexUsage(query, Seq(getIndexFilesPath(indexConfig.indexName))) + } + test("E2E test for join query.") { val leftDf = spark.read.parquet(sampleParquetDataLocation) val leftDfIndexConfig = IndexConfig("leftIndex", Seq("c3"), Seq("c1")) From c1bb94c8ad06a0346bbae69aa896b2768df83761 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 30 Jun 2020 16:59:11 -0700 Subject: [PATCH 05/33] revert FilterRule to go for resolver based implementation --- .../index/rules/FilterIndexRule.scala | 37 +++---------------- .../index/rules/FilterIndexRuleTest.scala | 13 +++++++ 2 files changed, 18 insertions(+), 32 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 8b737ae1f..d247f6897 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.CleanupAliases -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources._ @@ -59,22 +59,12 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { // such that its children would not have any Alias expressions. // Calling "references" on the expression in projectList ensures // we will get the correct (original) column names. - val baseRelationColumns: Seq[AttributeReference] = logicalRelation.output - val projectColumns: Seq[Option[AttributeReference]] = CleanupAliases(project) + val projectColumnNames = CleanupAliases(project) .asInstanceOf[Project] .projectList - .map(_.references.map(_.asInstanceOf[AttributeReference])) + .map(_.references.map(_.asInstanceOf[AttributeReference].name)) .flatMap(_.toSeq) - .map(findBaseReference(_, baseRelationColumns)) - val filterColumns: Seq[Option[AttributeReference]] = condition.references.toSeq - .map(findBaseReference(_, baseRelationColumns)) - - if (projectColumns.exists(_.isEmpty) || filterColumns.exists(_.isEmpty)) { - return project - } - - val projectColumnNames = projectColumns.map(_.get.name) - val filterColumnNames = filterColumns.map(_.get.name) + val filterColumnNames = condition.references.map(_.name).toSeq replaceWithIndexIfPlanCovered( project, @@ -91,24 +81,6 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { } } - /** - * This method is useful in finding an attribute from a list, which is semantically equal to - * a given attribute. This is required for e.g. to find the base attribute name for an - * attribute. - * For e.g. if the data source has column names in lower case e.g. c1, c2, c3... - * and a case-insensitive query looks for column C1, we can use this method to find which of the - * base columns (c1, c2, c3...) matches with the query column C1. - * - * @param attribute attribute searched for. - * @param baseReferences available set of base attributes to look from. - * @return optional matching attribute from base references. - */ - private def findBaseReference( - attribute: Attribute, - baseReferences: Seq[AttributeReference]): Option[AttributeReference] = { - baseReferences.find(_.semanticEquals(attribute)) - } - /** * For a given relation, check its available indexes and replace it with the top-ranked index * (according to cost model). @@ -212,6 +184,7 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { index.includedColumns, fsRelation.fileFormat) } + } /** diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala index 5f31a808f..5c423931a 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala @@ -115,6 +115,19 @@ class FilterIndexRuleTest extends HyperspaceSuite { verifyTransformedPlan(transformedPlan) } + test("Verify FilterIndex rule is applied correctly for case insensitive query.") { + val c2Caps = c2.withName("C2") + val c3Caps = c3.withName("C3") + val filterCondition = And(IsNotNull(c3Caps), EqualTo(c3Caps, Literal("facebook"))) + val filterNode = Filter(filterCondition, scanNode) + + val originalPlan = Project(Seq(c2Caps, c3Caps), filterNode) + val transformedPlan = FilterIndexRule(originalPlan) + + assert(!transformedPlan.equals(originalPlan), "No plan transformation.") + verifyTransformedPlan(transformedPlan) + } + test("Verify FilterIndex rule is applied correctly to plans with alias.") { val aliasExpr = Alias(c3, "QueryAlias")().asInstanceOf[NamedExpression] val filterCondition = And(IsNotNull(aliasExpr), EqualTo(aliasExpr, Literal("facebook"))) From c8900560f58fe34d55dad138a26ca99221dd70ec Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 09:30:52 -0700 Subject: [PATCH 06/33] Update Rules to use spark resolver and added some tests --- .../index/rules/FilterIndexRule.scala | 7 +++- .../hyperspace/util/IndexNameUtils.scala | 38 +++++++++++++++---- .../index/E2EHyperspaceRulesTests.scala | 20 ++++++++++ .../index/rules/JoinIndexRuleTest.scala | 12 ++++++ 4 files changed, 68 insertions(+), 9 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 d247f6897..326f32cb2 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.types.StructType import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexLogEntry, LogicalPlanSignatureProvider} +import com.microsoft.hyperspace.util.IndexNameUtils /** * FilterIndex rule looks for opportunities in a logical plan to replace @@ -178,6 +179,7 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { index.created && signatureValid(index) && indexCoversPlan( + fsRelation.sparkSession, projectColumns, filterColumns, index.indexedColumns, @@ -201,6 +203,7 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { * 2. Filter predicate contains first column in index's 'indexed' columns. */ private def indexCoversPlan( + spark: SparkSession, projectColumns: Seq[String], filterColumns: Seq[String], indexedColumns: Seq[String], @@ -210,8 +213,8 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { val allColumnsInIndex = indexedColumns ++ includedColumns // TODO: Normalize predicates into CNF and incorporate more conditions. - filterColumns.contains(indexedColumns.head) && - allColumnsInPlan.forall(allColumnsInIndex.contains) + IndexNameUtils.resolve(spark, indexedColumns.head, filterColumns) && + IndexNameUtils.resolve(spark, allColumnsInPlan, allColumnsInIndex) } /** diff --git a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala index c4b683178..23774440d 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala @@ -35,19 +35,43 @@ object IndexNameUtils { indexName.trim.replaceAll("\\s+", "_") } - def resolve(spark: SparkSession, firstString: String, secondString: String): Boolean = { + def resolve(spark: SparkSession, requiredString: String, availableStrings: String): Boolean = { val resolver: Resolver = spark.sessionState.conf.resolver - resolver(firstString, secondString) + resolver(requiredString, availableStrings) } - def resolve(spark: SparkSession, firstString: String, secondStrings: Seq[String]): Boolean = { - secondStrings.exists(resolve(spark, firstString, _)) + def resolve( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Boolean = { + availableStrings.exists(resolve(spark, requiredString, _)) } def resolve( spark: SparkSession, - firstStrings: Seq[String], - secondStrings: Seq[String]): Boolean = { - firstStrings.forall(resolve(spark, _, secondStrings)) + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Boolean = { + requiredStrings.forall(resolve(spark, _, availableStrings)) + } + + def resolvedString( + spark: SparkSession, + firstString: String, + secondString: String): Option[String] = { + if (resolve(spark, firstString, secondString)) Some(secondString) else None + } + + def resolvedString( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Option[String] = { + availableStrings.find(resolve(spark, requiredString, _)) + } + + def resolvedString( + spark: SparkSession, + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Iterable[Option[String]] = { + requiredStrings.map(resolvedString(spark, _, availableStrings)) } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 4cba3ae07..ba4bf5a49 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -128,6 +128,26 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite { getIndexFilesPath(rightDfIndexConfig.indexName))) } + test("E2E test for join query with case-insensitive column names.") { + val leftDf = spark.read.parquet(sampleParquetDataLocation) + val leftDfIndexConfig = IndexConfig("leftIndex", Seq("C3"), Seq("c1")) + hyperspace.createIndex(leftDf, leftDfIndexConfig) + + val rightDf = spark.read.parquet(sampleParquetDataLocation) + val rightDfIndexConfig = IndexConfig("rightIndex", Seq("c3"), Seq("C4")) + hyperspace.createIndex(rightDf, rightDfIndexConfig) + + def query(): DataFrame = { + leftDf.join(rightDf, leftDf("c3") === rightDf("C3")).select(leftDf("C1"), rightDf("c4")) + } + + verifyIndexUsage( + query, + Seq( + getIndexFilesPath(leftDfIndexConfig.indexName), + getIndexFilesPath(rightDfIndexConfig.indexName))) + } + test("E2E test for join query on catalog temp tables/views") { withView("t1", "t2") { val leftDf = spark.read.parquet(sampleParquetDataLocation) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala index 4680d53a7..c81637559 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala @@ -125,6 +125,18 @@ class JoinIndexRuleTest extends HyperspaceSuite { verifyUpdatedIndex(originalPlan, updatedPlan, indexPaths) } + test("Join rule works if indexes exist for case insensitive index and query") { + val t1c1Caps = t1c1.withName("T1C1") + + val joinCondition = EqualTo(t1c1Caps, t2c1) + val originalPlan = Join(t1ProjectNode, t2ProjectNode, JoinType("inner"), Some(joinCondition)) + val updatedPlan = JoinIndexRule(originalPlan) + assert(!updatedPlan.equals(originalPlan)) + + val indexPaths = Seq(getIndexDataFilesPath("t1i1"), getIndexDataFilesPath("t2i1")) + verifyUpdatedIndex(originalPlan, updatedPlan, indexPaths) + } + test("Join rule does not update plan if index location is not set") { spark.conf.unset(IndexConstants.INDEX_SYSTEM_PATH) From ff63d6177eb860ebd675fc1b3919d8bbc9f531b0 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 09:31:40 -0700 Subject: [PATCH 07/33] Update JoinRule to use resolver for case-sensitivity --- .../index/rules/JoinIndexRule.scala | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) 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 fb3d0f43b..55764a52c 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -34,6 +34,7 @@ import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.rankers.JoinIndexRanker +import com.microsoft.hyperspace.util.IndexNameUtils /** * Rule to optimize a join between two indexed dataframes. @@ -377,8 +378,9 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { val rRequiredAllCols = allRequiredCols(right) // Make sure required indexed columns are subset of all required columns for a subplan - require(lRequiredIndexedCols.forall(lRequiredAllCols.contains)) - require(rRequiredIndexedCols.forall(rRequiredAllCols.contains)) + val spark = SparkSession.getActiveSession.get + require(IndexNameUtils.resolve(spark, lRequiredIndexedCols, lRequiredAllCols)) + require(IndexNameUtils.resolve(spark, rRequiredIndexedCols, rRequiredAllCols)) val lUsable = getUsableIndexes(lIndexes, lRequiredIndexedCols, lRequiredAllCols) val rUsable = getUsableIndexes(rIndexes, rRequiredIndexedCols, rRequiredAllCols) @@ -516,10 +518,15 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { indexes: Seq[IndexLogEntry], requiredIndexCols: Seq[String], allRequiredCols: Seq[String]): Seq[IndexLogEntry] = { + val spark = SparkSession.getActiveSession.get indexes.filter { idx => val allCols = idx.indexedColumns ++ idx.includedColumns - requiredIndexCols.toSet.equals(idx.indexedColumns.toSet) && - allRequiredCols.forall(allCols.contains) + + // All required index columns should match one-to-one with all indexed columns and + // vice-versa. All required columns must be present in the available index columns. + IndexNameUtils.resolve(spark, requiredIndexCols, idx.indexedColumns) && + requiredIndexCols.distinct.size == idx.indexedColumns.distinct.size && + IndexNameUtils.resolve(spark, allRequiredCols, allCols) } } @@ -586,10 +593,13 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { - require(columnMapping.keys.toSet.equals(lIndex.indexedColumns.toSet)) - require(columnMapping.values.toSet.equals(rIndex.indexedColumns.toSet)) + val spark = SparkSession.getActiveSession.get + require(IndexNameUtils.resolve(spark, columnMapping.keys, lIndex.indexedColumns)) + require(IndexNameUtils.resolve(spark, columnMapping.values, rIndex.indexedColumns)) - val requiredRightIndexedCols = lIndex.indexedColumns.map(columnMapping) + val requiredRightIndexedCols = lIndex.indexedColumns.map(c => + columnMapping(IndexNameUtils.resolvedString(spark, c, columnMapping.keys).get)) + //IndexNameUtils.resolve(spark, rIndex.indexedColumns, requiredRightIndexedCols) rIndex.indexedColumns.equals(requiredRightIndexedCols) } } From ed05730accd6d81f5001f382a242d4cacffedefe Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 11:29:59 -0700 Subject: [PATCH 08/33] code cleanup and refactoring --- .../hyperspace/actions/CreateAction.scala | 4 +- .../hyperspace/actions/CreateActionBase.scala | 39 +++-- .../index/rules/FilterIndexRule.scala | 6 +- .../index/rules/JoinIndexRule.scala | 35 ++--- .../hyperspace/util/IndexNameUtils.scala | 77 ---------- .../hyperspace/util/ResolverUtils.scala | 137 ++++++++++++++++++ ...lsTests.scala => ResolverUtilsTests.scala} | 4 +- 7 files changed, 182 insertions(+), 120 deletions(-) delete mode 100644 src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala rename src/test/scala/com/microsoft/hyperspace/util/{IndexNameUtilsTests.scala => ResolverUtilsTests.scala} (85%) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index 5960959bd..8805a7115 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.types.StructType import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.util.{IndexNameUtils, LogicalPlanUtils} +import com.microsoft.hyperspace.util.{LogicalPlanUtils, ResolverUtils} class CreateAction( spark: SparkSession, @@ -66,7 +66,7 @@ class CreateAction( val indexedColumns = indexConfig.indexedColumns val includedColumns = indexConfig.includedColumns - IndexNameUtils.resolve(spark, indexedColumns ++ includedColumns, validColumnNames) + ResolverUtils.isResolved(spark, indexedColumns ++ includedColumns, validColumnNames) } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index d36290aba..421645e93 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -24,7 +24,7 @@ import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer import com.microsoft.hyperspace.index.serde.LogicalPlanSerDeUtils -import com.microsoft.hyperspace.util.IndexNameUtils +import com.microsoft.hyperspace.util.ResolverUtils /** * CreateActionBase provides functionality to write dataframe as covering index. @@ -51,14 +51,13 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val signatureProvider = LogicalPlanSignatureProvider.create() + // Resolve the passed column names with existing column names from the dataframe if possible. val (resolvedIndexedColumns, resolvedIncludedColumns) = { - val dfColumnNames = df.schema.fieldNames try { // Try creating log entry with resolved column names. - (indexConfig.indexedColumns.map(resolve(spark, _, dfColumnNames)), - indexConfig.includedColumns.map(resolve(spark, _, dfColumnNames))) + resolveConfig(df, indexConfig) } catch { - // Try creating index log entry with whatever the user passed. + // Creating index log entry with whatever the user passed. case _: Exception => (indexConfig.indexedColumns, indexConfig.includedColumns) } } @@ -116,10 +115,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) IndexConstants.INDEX_NUM_BUCKETS_DEFAULT.toString) .toInt - val dfColumnNames = df.schema.fieldNames - val resolvedIndexedColumns = indexConfig.indexedColumns.map(resolve(spark, _, dfColumnNames)) - val resolvedIncludedColumns = - indexConfig.includedColumns.map(resolve(spark, _, dfColumnNames)) + val (resolvedIndexedColumns, resolvedIncludedColumns) = resolveConfig(df, indexConfig) val selectedColumns = resolvedIndexedColumns ++ resolvedIncludedColumns val indexDataFrame = df.select(selectedColumns.head, selectedColumns.tail: _*) @@ -136,12 +132,23 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) resolvedIndexedColumns) } - private def resolve(spark: SparkSession, dst: String, srcs: Array[String]): String = { - srcs - .find(src => IndexNameUtils.resolve(spark, dst, src)) - .getOrElse { - throw HyperspaceException( - s"Unexpected Exception: Column $dst could not be resolved from available columns $srcs") - } + private def resolveConfig( + df: DataFrame, + indexConfig: IndexConfig): (Seq[String], Seq[String]) = { + val spark = df.sparkSession + val dfColumnNames = df.schema.fieldNames + val resolvedIndexedColumns = + ResolverUtils.resolve(spark, indexConfig.indexedColumns, dfColumnNames) + val resolvedIncludedColumns = + ResolverUtils.resolve(spark, indexConfig.includedColumns, dfColumnNames) + + val allColumns = resolvedIncludedColumns ++ resolvedIncludedColumns + if ((resolvedIncludedColumns ++ resolvedIncludedColumns).exists(_.isEmpty)) { + throw HyperspaceException( + s"Unexpected Exception: Some of the columns from $allColumns could not be resolved from " + + s"available source columns $dfColumnNames") + } + + (resolvedIndexedColumns.map(_.get), resolvedIncludedColumns.map(_.get)) } } 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 326f32cb2..582eb3354 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.types.StructType import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexLogEntry, LogicalPlanSignatureProvider} -import com.microsoft.hyperspace.util.IndexNameUtils +import com.microsoft.hyperspace.util.ResolverUtils /** * FilterIndex rule looks for opportunities in a logical plan to replace @@ -213,8 +213,8 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { val allColumnsInIndex = indexedColumns ++ includedColumns // TODO: Normalize predicates into CNF and incorporate more conditions. - IndexNameUtils.resolve(spark, indexedColumns.head, filterColumns) && - IndexNameUtils.resolve(spark, allColumnsInPlan, allColumnsInIndex) + ResolverUtils.isResolved(spark, indexedColumns.head, filterColumns) && + ResolverUtils.isResolved(spark, allColumnsInPlan, allColumnsInIndex) } /** 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 55764a52c..bfadb724b 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -30,11 +30,11 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFil import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types.StructType -import com.microsoft.hyperspace.Hyperspace +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.rankers.JoinIndexRanker -import com.microsoft.hyperspace.util.IndexNameUtils +import com.microsoft.hyperspace.util.ResolverUtils._ /** * Rule to optimize a join between two indexed dataframes. @@ -89,7 +89,7 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { right: LogicalPlan, condition: Expression): Option[(IndexLogEntry, IndexLogEntry)] = { val allIndexes = Hyperspace - .getContext(SparkSession.getActiveSession.get) + .getContext(spark) .indexCollectionManager .getIndexes(Seq(Constants.States.ACTIVE)) @@ -127,11 +127,6 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { numBuckets = index.numBuckets, bucketColumnNames = index.indexedColumns, sortColumnNames = index.indexedColumns) - - val spark = SparkSession.getActiveSession.getOrElse { - throw new IllegalArgumentException("Could not find active SparkSession") - } - val location = new InMemoryFileIndex(spark, Seq(new Path(index.content.root)), Map(), None) val relation = HadoopFsRelation( location, @@ -378,9 +373,8 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { val rRequiredAllCols = allRequiredCols(right) // Make sure required indexed columns are subset of all required columns for a subplan - val spark = SparkSession.getActiveSession.get - require(IndexNameUtils.resolve(spark, lRequiredIndexedCols, lRequiredAllCols)) - require(IndexNameUtils.resolve(spark, rRequiredIndexedCols, rRequiredAllCols)) + require(isResolved(spark, lRequiredIndexedCols, lRequiredAllCols)) + require(isResolved(spark, rRequiredIndexedCols, rRequiredAllCols)) val lUsable = getUsableIndexes(lIndexes, lRequiredIndexedCols, lRequiredAllCols) val rUsable = getUsableIndexes(rIndexes, rRequiredIndexedCols, rRequiredAllCols) @@ -518,15 +512,14 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { indexes: Seq[IndexLogEntry], requiredIndexCols: Seq[String], allRequiredCols: Seq[String]): Seq[IndexLogEntry] = { - val spark = SparkSession.getActiveSession.get indexes.filter { idx => val allCols = idx.indexedColumns ++ idx.includedColumns // All required index columns should match one-to-one with all indexed columns and // vice-versa. All required columns must be present in the available index columns. - IndexNameUtils.resolve(spark, requiredIndexCols, idx.indexedColumns) && + isResolved(spark, requiredIndexCols, idx.indexedColumns) && requiredIndexCols.distinct.size == idx.indexedColumns.distinct.size && - IndexNameUtils.resolve(spark, allRequiredCols, allCols) + isResolved(spark, allRequiredCols, allCols) } } @@ -593,13 +586,15 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { - val spark = SparkSession.getActiveSession.get - require(IndexNameUtils.resolve(spark, columnMapping.keys, lIndex.indexedColumns)) - require(IndexNameUtils.resolve(spark, columnMapping.values, rIndex.indexedColumns)) + require(isResolved(spark, columnMapping.keys, lIndex.indexedColumns)) + require(isResolved(spark, columnMapping.values, rIndex.indexedColumns)) - val requiredRightIndexedCols = lIndex.indexedColumns.map(c => - columnMapping(IndexNameUtils.resolvedString(spark, c, columnMapping.keys).get)) - //IndexNameUtils.resolve(spark, rIndex.indexedColumns, requiredRightIndexedCols) + val requiredRightIndexedCols = + lIndex.indexedColumns.map(c => columnMapping(resolve(spark, c, columnMapping.keys).get)) rIndex.indexedColumns.equals(requiredRightIndexedCols) } + + private lazy val spark: SparkSession = SparkSession.getActiveSession.getOrElse { + throw HyperspaceException("Could not find active SparkSession") + } } diff --git a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala deleted file mode 100644 index 23774440d..000000000 --- a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.util - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.analysis.Resolver - -/** - * [[IndexNameUtils]] provides utility functions to format the index name as need. - */ -object IndexNameUtils { - - /** - * Normalize the index name by trimming space(s) at both ends, and replacing internal space(s) - * with an underscore. - * - * @param indexName the name of index passed in by user. - * @return a normalized index name. - */ - def normalizeIndexName(indexName: String): String = { - indexName.trim.replaceAll("\\s+", "_") - } - - def resolve(spark: SparkSession, requiredString: String, availableStrings: String): Boolean = { - val resolver: Resolver = spark.sessionState.conf.resolver - resolver(requiredString, availableStrings) - } - - def resolve( - spark: SparkSession, - requiredString: String, - availableStrings: Iterable[String]): Boolean = { - availableStrings.exists(resolve(spark, requiredString, _)) - } - - def resolve( - spark: SparkSession, - requiredStrings: Iterable[String], - availableStrings: Iterable[String]): Boolean = { - requiredStrings.forall(resolve(spark, _, availableStrings)) - } - - def resolvedString( - spark: SparkSession, - firstString: String, - secondString: String): Option[String] = { - if (resolve(spark, firstString, secondString)) Some(secondString) else None - } - - def resolvedString( - spark: SparkSession, - requiredString: String, - availableStrings: Iterable[String]): Option[String] = { - availableStrings.find(resolve(spark, requiredString, _)) - } - - def resolvedString( - spark: SparkSession, - requiredStrings: Iterable[String], - availableStrings: Iterable[String]): Iterable[Option[String]] = { - requiredStrings.map(resolvedString(spark, _, availableStrings)) - } -} diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala new file mode 100644 index 000000000..debb8b6e4 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -0,0 +1,137 @@ +/* + * 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.util + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.Resolver + +/** + * [[ResolverUtils]] provides utility functions to + * - format the index name as needed. + * - resolve strings based on spark's resolver. + */ +object ResolverUtils { + + /** + * Normalize the index name by trimming space(s) at both ends, and replacing internal space(s) + * with an underscore. + * + * @param indexName the name of index passed in by user. + * @return a normalized index name. + */ + def normalizeIndexName(indexName: String): String = { + indexName.trim.replaceAll("\\s+", "_") + } + + /** + * Resolve two strings based on spark's resolver. Returns true if both are resolved. + * + * @param spark Spark session. + * @param requiredString First string. + * @param availableString Second string. + * @return True if both are equivalent based on the current spark resolver. + */ + def isResolved( + spark: SparkSession, + requiredString: String, + availableString: String): Boolean = { + val resolver: Resolver = spark.sessionState.conf.resolver + resolver(requiredString, availableString) + } + + /** + * Resolve first string from available collection of strings based on spark's resolver. Returns + * true if any resolved string is found in the collection. + * + * @param spark Spark session. + * @param requiredString String to resolve. + * @param availableStrings Available strings. + * @return True if any of the available strings can be resolved with the passed string. + */ + def isResolved( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Boolean = { + availableStrings.exists(isResolved(spark, requiredString, _)) + } + + /** + * Resolves ALL of the requiredStrings with availableStrings based on spark's resolver. Returns + * true if all the requiredStrings can be resolved with some string from availableStrings. + * Returns false otherwise. + * + * @param spark Spark session. + * @param requiredStrings List of strings to resolve. + * @param availableStrings List of availble strings to resolve from. + * @return True if every one of requiredStrings can be resolved from availableStrings. False + * otherwise. + */ + def isResolved( + spark: SparkSession, + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Boolean = { + requiredStrings.forall(isResolved(spark, _, availableStrings)) + } + + /** + * Return available string if required string can be resolved with it, based on spark resolver. + * + * @param spark Spark Session. + * @param requiredString The string that requires resolution. + * @param availableString Available list of strings to resolve from. + * @return Optional available string if resolution is successful, else None + */ + def resolve( + spark: SparkSession, + requiredString: String, + availableString: String): Option[String] = { + if (isResolved(spark, requiredString, availableString)) Some(availableString) else None + } + + /** + * Finds the first matching resolved string from the list of availableStrings, when resolving + * for requiredString. If no matching string found, return None. + * + * @param spark Spark session. + * @param requiredString The string that requires resolution. + * @param availableStrings All available strings to find a match from. + * @return First matching (i.e. resolved) string from availableStrings. If no match is found, + * return None. + */ + def resolve( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Option[String] = { + availableStrings.find(isResolved(spark, requiredString, _)) + } + + /** + * Finds all resolved strings for requiredStrings, from the list of availableStrings. Returns a + * sequence of Optional values for matches, None for unmatched strings. + * + * @param spark Spark session. + * @param requiredStrings List of strings to resolve. + * @param availableStrings List of available strings to resolve from. + * @return Sequence of optional values of resolved strings or None's. + */ + def resolve( + spark: SparkSession, + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Seq[Option[String]] = { + requiredStrings.map(resolve(spark, _, availableStrings)).toSeq + } +} diff --git a/src/test/scala/com/microsoft/hyperspace/util/IndexNameUtilsTests.scala b/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala similarity index 85% rename from src/test/scala/com/microsoft/hyperspace/util/IndexNameUtilsTests.scala rename to src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala index b0cccf5ef..15a18402d 100644 --- a/src/test/scala/com/microsoft/hyperspace/util/IndexNameUtilsTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala @@ -18,10 +18,10 @@ package com.microsoft.hyperspace.util import org.apache.spark.SparkFunSuite -class IndexNameUtilsTests extends SparkFunSuite { +class ResolverUtilsTests extends SparkFunSuite { test("Test normalizeIndexName() function.") { val indexName = " my index 1 " val expectedIndexNameAfterNorm = "my_index_1" - assert(IndexNameUtils.normalizeIndexName(indexName).equals(expectedIndexNameAfterNorm)) + assert(ResolverUtils.normalizeIndexName(indexName).equals(expectedIndexNameAfterNorm)) } } From 2f26efce7ca302d5a3bd6a804c0e641debaef757 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 12:06:12 -0700 Subject: [PATCH 09/33] fix for join rule tests failures --- .../com/microsoft/hyperspace/index/rules/JoinIndexRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bfadb724b..40d80de8d 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -594,7 +594,7 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { rIndex.indexedColumns.equals(requiredRightIndexedCols) } - private lazy val spark: SparkSession = SparkSession.getActiveSession.getOrElse { + private def spark: SparkSession = SparkSession.getActiveSession.getOrElse { throw HyperspaceException("Could not find active SparkSession") } } From 902aa8b67438ed78da816795ca71c982b06806b0 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 30 Jun 2020 12:52:14 -0700 Subject: [PATCH 10/33] improve tests for case insensitive index creation --- .../hyperspace/actions/CreateAction.scala | 6 +----- .../hyperspace/actions/CreateActionBase.scala | 15 +++++++++++++-- .../hyperspace/index/CreateIndexTests.scala | 12 ++++++++++-- .../index/E2EHyperspaceRulesTests.scala | 12 ++++++++++++ .../index/rules/FilterIndexRuleTest.scala | 13 +++++++++++++ 5 files changed, 49 insertions(+), 9 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index d1181f7b8..5960959bd 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -17,10 +17,9 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.types.StructType -import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.util.{IndexNameUtils, LogicalPlanUtils} @@ -68,9 +67,6 @@ class CreateAction( val includedColumns = indexConfig.includedColumns IndexNameUtils.resolve(spark, indexedColumns ++ includedColumns, validColumnNames) - - // indexedColumns.forall(validColumnNames.contains) && includedColumns.forall( - // validColumnNames.contains) } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 0c7602e12..d36290aba 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -51,8 +51,19 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val signatureProvider = LogicalPlanSignatureProvider.create() + val (resolvedIndexedColumns, resolvedIncludedColumns) = { + val dfColumnNames = df.schema.fieldNames + try { + // Try creating log entry with resolved column names. + (indexConfig.indexedColumns.map(resolve(spark, _, dfColumnNames)), + indexConfig.includedColumns.map(resolve(spark, _, dfColumnNames))) + } catch { + // Try creating index log entry with whatever the user passed. + case _: Exception => (indexConfig.indexedColumns, indexConfig.includedColumns) + } + } val schema = { - val allColumns = indexConfig.indexedColumns ++ indexConfig.includedColumns + val allColumns = resolvedIndexedColumns ++ resolvedIncludedColumns df.select(allColumns.head, allColumns.tail: _*).schema } @@ -80,7 +91,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) CoveringIndex( CoveringIndex.Properties( CoveringIndex.Properties - .Columns(indexConfig.indexedColumns, indexConfig.includedColumns), + .Columns(resolvedIndexedColumns, resolvedIncludedColumns), IndexLogEntry.schemaString(schema), numBuckets)), Content(path.toString, Seq()), diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala index 8031ab433..13788595e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala @@ -16,6 +16,8 @@ package com.microsoft.hyperspace.index +import scala.collection.mutable.WrappedArray + import org.apache.hadoop.fs.Path import org.apache.spark.sql.DataFrame @@ -87,8 +89,14 @@ class CreateIndexTests extends HyperspaceSuite { test("Index creation passes with columns of different case if case-sensitivity is false.") { hyperspace.createIndex(df, IndexConfig("index1", Seq("qUeRy"), Seq("ImpRS"))) - val count = hyperspace.indexes.where(s"""name = "${indexConfig1.indexName}" """).count - assert(count == 1) + val indexes = hyperspace.indexes.where(s"""name = "${indexConfig1.indexName}" """) + assert(indexes.count == 1) + assert( + indexes.head.getAs[WrappedArray[String]]("indexedColumns").head.equals("Query"), + "Indexed columns with wrong case are stored in metadata") + assert( + indexes.head.getAs[WrappedArray[String]]("includedColumns").head.equals("imprs"), + "Included columns with wrong case are stored in metadata") } test("Index creation fails with columns of different case if case-sensitivity is true.") { diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 61f3cab69..4cba3ae07 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -95,6 +95,18 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite { verifyIndexUsage(query, Seq(getIndexFilesPath(indexConfig.indexName))) } + test("E2E test for case insensitive filter query utilizing indexes.") { + val df = spark.read.parquet(sampleParquetDataLocation) + val indexConfig = IndexConfig("filterIndex", Seq("C3"), Seq("C1")) + + hyperspace.createIndex(df, indexConfig) + + def query(): DataFrame = df.filter("C3 == 'facebook'").select("C3", "c1") + + // verify if case-insensitive index works with case-insensitive query + verifyIndexUsage(query, Seq(getIndexFilesPath(indexConfig.indexName))) + } + test("E2E test for join query.") { val leftDf = spark.read.parquet(sampleParquetDataLocation) val leftDfIndexConfig = IndexConfig("leftIndex", Seq("c3"), Seq("c1")) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala index 5f31a808f..5c423931a 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala @@ -115,6 +115,19 @@ class FilterIndexRuleTest extends HyperspaceSuite { verifyTransformedPlan(transformedPlan) } + test("Verify FilterIndex rule is applied correctly for case insensitive query.") { + val c2Caps = c2.withName("C2") + val c3Caps = c3.withName("C3") + val filterCondition = And(IsNotNull(c3Caps), EqualTo(c3Caps, Literal("facebook"))) + val filterNode = Filter(filterCondition, scanNode) + + val originalPlan = Project(Seq(c2Caps, c3Caps), filterNode) + val transformedPlan = FilterIndexRule(originalPlan) + + assert(!transformedPlan.equals(originalPlan), "No plan transformation.") + verifyTransformedPlan(transformedPlan) + } + test("Verify FilterIndex rule is applied correctly to plans with alias.") { val aliasExpr = Alias(c3, "QueryAlias")().asInstanceOf[NamedExpression] val filterCondition = And(IsNotNull(aliasExpr), EqualTo(aliasExpr, Literal("facebook"))) From 6d9d620624fcdc75e96b2bc5c72e1858e95eff1c Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 09:30:52 -0700 Subject: [PATCH 11/33] Update Rules to use spark resolver and added some tests --- .../index/rules/FilterIndexRule.scala | 7 +++- .../hyperspace/util/IndexNameUtils.scala | 38 +++++++++++++++---- .../index/E2EHyperspaceRulesTests.scala | 20 ++++++++++ .../index/rules/JoinIndexRuleTest.scala | 12 ++++++ 4 files changed, 68 insertions(+), 9 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 d247f6897..326f32cb2 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.types.StructType import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexLogEntry, LogicalPlanSignatureProvider} +import com.microsoft.hyperspace.util.IndexNameUtils /** * FilterIndex rule looks for opportunities in a logical plan to replace @@ -178,6 +179,7 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { index.created && signatureValid(index) && indexCoversPlan( + fsRelation.sparkSession, projectColumns, filterColumns, index.indexedColumns, @@ -201,6 +203,7 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { * 2. Filter predicate contains first column in index's 'indexed' columns. */ private def indexCoversPlan( + spark: SparkSession, projectColumns: Seq[String], filterColumns: Seq[String], indexedColumns: Seq[String], @@ -210,8 +213,8 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { val allColumnsInIndex = indexedColumns ++ includedColumns // TODO: Normalize predicates into CNF and incorporate more conditions. - filterColumns.contains(indexedColumns.head) && - allColumnsInPlan.forall(allColumnsInIndex.contains) + IndexNameUtils.resolve(spark, indexedColumns.head, filterColumns) && + IndexNameUtils.resolve(spark, allColumnsInPlan, allColumnsInIndex) } /** diff --git a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala index c4b683178..23774440d 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala @@ -35,19 +35,43 @@ object IndexNameUtils { indexName.trim.replaceAll("\\s+", "_") } - def resolve(spark: SparkSession, firstString: String, secondString: String): Boolean = { + def resolve(spark: SparkSession, requiredString: String, availableStrings: String): Boolean = { val resolver: Resolver = spark.sessionState.conf.resolver - resolver(firstString, secondString) + resolver(requiredString, availableStrings) } - def resolve(spark: SparkSession, firstString: String, secondStrings: Seq[String]): Boolean = { - secondStrings.exists(resolve(spark, firstString, _)) + def resolve( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Boolean = { + availableStrings.exists(resolve(spark, requiredString, _)) } def resolve( spark: SparkSession, - firstStrings: Seq[String], - secondStrings: Seq[String]): Boolean = { - firstStrings.forall(resolve(spark, _, secondStrings)) + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Boolean = { + requiredStrings.forall(resolve(spark, _, availableStrings)) + } + + def resolvedString( + spark: SparkSession, + firstString: String, + secondString: String): Option[String] = { + if (resolve(spark, firstString, secondString)) Some(secondString) else None + } + + def resolvedString( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Option[String] = { + availableStrings.find(resolve(spark, requiredString, _)) + } + + def resolvedString( + spark: SparkSession, + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Iterable[Option[String]] = { + requiredStrings.map(resolvedString(spark, _, availableStrings)) } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 4cba3ae07..ba4bf5a49 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -128,6 +128,26 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite { getIndexFilesPath(rightDfIndexConfig.indexName))) } + test("E2E test for join query with case-insensitive column names.") { + val leftDf = spark.read.parquet(sampleParquetDataLocation) + val leftDfIndexConfig = IndexConfig("leftIndex", Seq("C3"), Seq("c1")) + hyperspace.createIndex(leftDf, leftDfIndexConfig) + + val rightDf = spark.read.parquet(sampleParquetDataLocation) + val rightDfIndexConfig = IndexConfig("rightIndex", Seq("c3"), Seq("C4")) + hyperspace.createIndex(rightDf, rightDfIndexConfig) + + def query(): DataFrame = { + leftDf.join(rightDf, leftDf("c3") === rightDf("C3")).select(leftDf("C1"), rightDf("c4")) + } + + verifyIndexUsage( + query, + Seq( + getIndexFilesPath(leftDfIndexConfig.indexName), + getIndexFilesPath(rightDfIndexConfig.indexName))) + } + test("E2E test for join query on catalog temp tables/views") { withView("t1", "t2") { val leftDf = spark.read.parquet(sampleParquetDataLocation) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala index 4680d53a7..c81637559 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala @@ -125,6 +125,18 @@ class JoinIndexRuleTest extends HyperspaceSuite { verifyUpdatedIndex(originalPlan, updatedPlan, indexPaths) } + test("Join rule works if indexes exist for case insensitive index and query") { + val t1c1Caps = t1c1.withName("T1C1") + + val joinCondition = EqualTo(t1c1Caps, t2c1) + val originalPlan = Join(t1ProjectNode, t2ProjectNode, JoinType("inner"), Some(joinCondition)) + val updatedPlan = JoinIndexRule(originalPlan) + assert(!updatedPlan.equals(originalPlan)) + + val indexPaths = Seq(getIndexDataFilesPath("t1i1"), getIndexDataFilesPath("t2i1")) + verifyUpdatedIndex(originalPlan, updatedPlan, indexPaths) + } + test("Join rule does not update plan if index location is not set") { spark.conf.unset(IndexConstants.INDEX_SYSTEM_PATH) From b733a03a402ab42366df967035d20efcf5b8cdc1 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 09:31:40 -0700 Subject: [PATCH 12/33] Update JoinRule to use resolver for case-sensitivity --- .../index/rules/JoinIndexRule.scala | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) 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 fb3d0f43b..55764a52c 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -34,6 +34,7 @@ import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.rankers.JoinIndexRanker +import com.microsoft.hyperspace.util.IndexNameUtils /** * Rule to optimize a join between two indexed dataframes. @@ -377,8 +378,9 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { val rRequiredAllCols = allRequiredCols(right) // Make sure required indexed columns are subset of all required columns for a subplan - require(lRequiredIndexedCols.forall(lRequiredAllCols.contains)) - require(rRequiredIndexedCols.forall(rRequiredAllCols.contains)) + val spark = SparkSession.getActiveSession.get + require(IndexNameUtils.resolve(spark, lRequiredIndexedCols, lRequiredAllCols)) + require(IndexNameUtils.resolve(spark, rRequiredIndexedCols, rRequiredAllCols)) val lUsable = getUsableIndexes(lIndexes, lRequiredIndexedCols, lRequiredAllCols) val rUsable = getUsableIndexes(rIndexes, rRequiredIndexedCols, rRequiredAllCols) @@ -516,10 +518,15 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { indexes: Seq[IndexLogEntry], requiredIndexCols: Seq[String], allRequiredCols: Seq[String]): Seq[IndexLogEntry] = { + val spark = SparkSession.getActiveSession.get indexes.filter { idx => val allCols = idx.indexedColumns ++ idx.includedColumns - requiredIndexCols.toSet.equals(idx.indexedColumns.toSet) && - allRequiredCols.forall(allCols.contains) + + // All required index columns should match one-to-one with all indexed columns and + // vice-versa. All required columns must be present in the available index columns. + IndexNameUtils.resolve(spark, requiredIndexCols, idx.indexedColumns) && + requiredIndexCols.distinct.size == idx.indexedColumns.distinct.size && + IndexNameUtils.resolve(spark, allRequiredCols, allCols) } } @@ -586,10 +593,13 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { - require(columnMapping.keys.toSet.equals(lIndex.indexedColumns.toSet)) - require(columnMapping.values.toSet.equals(rIndex.indexedColumns.toSet)) + val spark = SparkSession.getActiveSession.get + require(IndexNameUtils.resolve(spark, columnMapping.keys, lIndex.indexedColumns)) + require(IndexNameUtils.resolve(spark, columnMapping.values, rIndex.indexedColumns)) - val requiredRightIndexedCols = lIndex.indexedColumns.map(columnMapping) + val requiredRightIndexedCols = lIndex.indexedColumns.map(c => + columnMapping(IndexNameUtils.resolvedString(spark, c, columnMapping.keys).get)) + //IndexNameUtils.resolve(spark, rIndex.indexedColumns, requiredRightIndexedCols) rIndex.indexedColumns.equals(requiredRightIndexedCols) } } From fdeaa2587baccecf01f65954ad7fb472cd7b39e3 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 11:29:59 -0700 Subject: [PATCH 13/33] code cleanup and refactoring --- .../hyperspace/actions/CreateAction.scala | 4 +- .../hyperspace/actions/CreateActionBase.scala | 39 +++-- .../index/rules/FilterIndexRule.scala | 6 +- .../index/rules/JoinIndexRule.scala | 35 ++--- .../hyperspace/util/IndexNameUtils.scala | 77 ---------- .../hyperspace/util/ResolverUtils.scala | 137 ++++++++++++++++++ ...lsTests.scala => ResolverUtilsTests.scala} | 4 +- 7 files changed, 182 insertions(+), 120 deletions(-) delete mode 100644 src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala rename src/test/scala/com/microsoft/hyperspace/util/{IndexNameUtilsTests.scala => ResolverUtilsTests.scala} (85%) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index 5960959bd..8805a7115 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.types.StructType import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, CREATING, DOESNOTEXIST} import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.util.{IndexNameUtils, LogicalPlanUtils} +import com.microsoft.hyperspace.util.{LogicalPlanUtils, ResolverUtils} class CreateAction( spark: SparkSession, @@ -66,7 +66,7 @@ class CreateAction( val indexedColumns = indexConfig.indexedColumns val includedColumns = indexConfig.includedColumns - IndexNameUtils.resolve(spark, indexedColumns ++ includedColumns, validColumnNames) + ResolverUtils.isResolved(spark, indexedColumns ++ includedColumns, validColumnNames) } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index d36290aba..421645e93 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -24,7 +24,7 @@ import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer import com.microsoft.hyperspace.index.serde.LogicalPlanSerDeUtils -import com.microsoft.hyperspace.util.IndexNameUtils +import com.microsoft.hyperspace.util.ResolverUtils /** * CreateActionBase provides functionality to write dataframe as covering index. @@ -51,14 +51,13 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val signatureProvider = LogicalPlanSignatureProvider.create() + // Resolve the passed column names with existing column names from the dataframe if possible. val (resolvedIndexedColumns, resolvedIncludedColumns) = { - val dfColumnNames = df.schema.fieldNames try { // Try creating log entry with resolved column names. - (indexConfig.indexedColumns.map(resolve(spark, _, dfColumnNames)), - indexConfig.includedColumns.map(resolve(spark, _, dfColumnNames))) + resolveConfig(df, indexConfig) } catch { - // Try creating index log entry with whatever the user passed. + // Creating index log entry with whatever the user passed. case _: Exception => (indexConfig.indexedColumns, indexConfig.includedColumns) } } @@ -116,10 +115,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) IndexConstants.INDEX_NUM_BUCKETS_DEFAULT.toString) .toInt - val dfColumnNames = df.schema.fieldNames - val resolvedIndexedColumns = indexConfig.indexedColumns.map(resolve(spark, _, dfColumnNames)) - val resolvedIncludedColumns = - indexConfig.includedColumns.map(resolve(spark, _, dfColumnNames)) + val (resolvedIndexedColumns, resolvedIncludedColumns) = resolveConfig(df, indexConfig) val selectedColumns = resolvedIndexedColumns ++ resolvedIncludedColumns val indexDataFrame = df.select(selectedColumns.head, selectedColumns.tail: _*) @@ -136,12 +132,23 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) resolvedIndexedColumns) } - private def resolve(spark: SparkSession, dst: String, srcs: Array[String]): String = { - srcs - .find(src => IndexNameUtils.resolve(spark, dst, src)) - .getOrElse { - throw HyperspaceException( - s"Unexpected Exception: Column $dst could not be resolved from available columns $srcs") - } + private def resolveConfig( + df: DataFrame, + indexConfig: IndexConfig): (Seq[String], Seq[String]) = { + val spark = df.sparkSession + val dfColumnNames = df.schema.fieldNames + val resolvedIndexedColumns = + ResolverUtils.resolve(spark, indexConfig.indexedColumns, dfColumnNames) + val resolvedIncludedColumns = + ResolverUtils.resolve(spark, indexConfig.includedColumns, dfColumnNames) + + val allColumns = resolvedIncludedColumns ++ resolvedIncludedColumns + if ((resolvedIncludedColumns ++ resolvedIncludedColumns).exists(_.isEmpty)) { + throw HyperspaceException( + s"Unexpected Exception: Some of the columns from $allColumns could not be resolved from " + + s"available source columns $dfColumnNames") + } + + (resolvedIndexedColumns.map(_.get), resolvedIncludedColumns.map(_.get)) } } 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 326f32cb2..582eb3354 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.types.StructType import com.microsoft.hyperspace.Hyperspace import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexLogEntry, LogicalPlanSignatureProvider} -import com.microsoft.hyperspace.util.IndexNameUtils +import com.microsoft.hyperspace.util.ResolverUtils /** * FilterIndex rule looks for opportunities in a logical plan to replace @@ -213,8 +213,8 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { val allColumnsInIndex = indexedColumns ++ includedColumns // TODO: Normalize predicates into CNF and incorporate more conditions. - IndexNameUtils.resolve(spark, indexedColumns.head, filterColumns) && - IndexNameUtils.resolve(spark, allColumnsInPlan, allColumnsInIndex) + ResolverUtils.isResolved(spark, indexedColumns.head, filterColumns) && + ResolverUtils.isResolved(spark, allColumnsInPlan, allColumnsInIndex) } /** 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 55764a52c..bfadb724b 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -30,11 +30,11 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFil import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types.StructType -import com.microsoft.hyperspace.Hyperspace +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException} import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.rankers.JoinIndexRanker -import com.microsoft.hyperspace.util.IndexNameUtils +import com.microsoft.hyperspace.util.ResolverUtils._ /** * Rule to optimize a join between two indexed dataframes. @@ -89,7 +89,7 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { right: LogicalPlan, condition: Expression): Option[(IndexLogEntry, IndexLogEntry)] = { val allIndexes = Hyperspace - .getContext(SparkSession.getActiveSession.get) + .getContext(spark) .indexCollectionManager .getIndexes(Seq(Constants.States.ACTIVE)) @@ -127,11 +127,6 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { numBuckets = index.numBuckets, bucketColumnNames = index.indexedColumns, sortColumnNames = index.indexedColumns) - - val spark = SparkSession.getActiveSession.getOrElse { - throw new IllegalArgumentException("Could not find active SparkSession") - } - val location = new InMemoryFileIndex(spark, Seq(new Path(index.content.root)), Map(), None) val relation = HadoopFsRelation( location, @@ -378,9 +373,8 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { val rRequiredAllCols = allRequiredCols(right) // Make sure required indexed columns are subset of all required columns for a subplan - val spark = SparkSession.getActiveSession.get - require(IndexNameUtils.resolve(spark, lRequiredIndexedCols, lRequiredAllCols)) - require(IndexNameUtils.resolve(spark, rRequiredIndexedCols, rRequiredAllCols)) + require(isResolved(spark, lRequiredIndexedCols, lRequiredAllCols)) + require(isResolved(spark, rRequiredIndexedCols, rRequiredAllCols)) val lUsable = getUsableIndexes(lIndexes, lRequiredIndexedCols, lRequiredAllCols) val rUsable = getUsableIndexes(rIndexes, rRequiredIndexedCols, rRequiredAllCols) @@ -518,15 +512,14 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { indexes: Seq[IndexLogEntry], requiredIndexCols: Seq[String], allRequiredCols: Seq[String]): Seq[IndexLogEntry] = { - val spark = SparkSession.getActiveSession.get indexes.filter { idx => val allCols = idx.indexedColumns ++ idx.includedColumns // All required index columns should match one-to-one with all indexed columns and // vice-versa. All required columns must be present in the available index columns. - IndexNameUtils.resolve(spark, requiredIndexCols, idx.indexedColumns) && + isResolved(spark, requiredIndexCols, idx.indexedColumns) && requiredIndexCols.distinct.size == idx.indexedColumns.distinct.size && - IndexNameUtils.resolve(spark, allRequiredCols, allCols) + isResolved(spark, allRequiredCols, allCols) } } @@ -593,13 +586,15 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { - val spark = SparkSession.getActiveSession.get - require(IndexNameUtils.resolve(spark, columnMapping.keys, lIndex.indexedColumns)) - require(IndexNameUtils.resolve(spark, columnMapping.values, rIndex.indexedColumns)) + require(isResolved(spark, columnMapping.keys, lIndex.indexedColumns)) + require(isResolved(spark, columnMapping.values, rIndex.indexedColumns)) - val requiredRightIndexedCols = lIndex.indexedColumns.map(c => - columnMapping(IndexNameUtils.resolvedString(spark, c, columnMapping.keys).get)) - //IndexNameUtils.resolve(spark, rIndex.indexedColumns, requiredRightIndexedCols) + val requiredRightIndexedCols = + lIndex.indexedColumns.map(c => columnMapping(resolve(spark, c, columnMapping.keys).get)) rIndex.indexedColumns.equals(requiredRightIndexedCols) } + + private lazy val spark: SparkSession = SparkSession.getActiveSession.getOrElse { + throw HyperspaceException("Could not find active SparkSession") + } } diff --git a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala deleted file mode 100644 index 23774440d..000000000 --- a/src/main/scala/com/microsoft/hyperspace/util/IndexNameUtils.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.util - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.analysis.Resolver - -/** - * [[IndexNameUtils]] provides utility functions to format the index name as need. - */ -object IndexNameUtils { - - /** - * Normalize the index name by trimming space(s) at both ends, and replacing internal space(s) - * with an underscore. - * - * @param indexName the name of index passed in by user. - * @return a normalized index name. - */ - def normalizeIndexName(indexName: String): String = { - indexName.trim.replaceAll("\\s+", "_") - } - - def resolve(spark: SparkSession, requiredString: String, availableStrings: String): Boolean = { - val resolver: Resolver = spark.sessionState.conf.resolver - resolver(requiredString, availableStrings) - } - - def resolve( - spark: SparkSession, - requiredString: String, - availableStrings: Iterable[String]): Boolean = { - availableStrings.exists(resolve(spark, requiredString, _)) - } - - def resolve( - spark: SparkSession, - requiredStrings: Iterable[String], - availableStrings: Iterable[String]): Boolean = { - requiredStrings.forall(resolve(spark, _, availableStrings)) - } - - def resolvedString( - spark: SparkSession, - firstString: String, - secondString: String): Option[String] = { - if (resolve(spark, firstString, secondString)) Some(secondString) else None - } - - def resolvedString( - spark: SparkSession, - requiredString: String, - availableStrings: Iterable[String]): Option[String] = { - availableStrings.find(resolve(spark, requiredString, _)) - } - - def resolvedString( - spark: SparkSession, - requiredStrings: Iterable[String], - availableStrings: Iterable[String]): Iterable[Option[String]] = { - requiredStrings.map(resolvedString(spark, _, availableStrings)) - } -} diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala new file mode 100644 index 000000000..debb8b6e4 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -0,0 +1,137 @@ +/* + * 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.util + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.Resolver + +/** + * [[ResolverUtils]] provides utility functions to + * - format the index name as needed. + * - resolve strings based on spark's resolver. + */ +object ResolverUtils { + + /** + * Normalize the index name by trimming space(s) at both ends, and replacing internal space(s) + * with an underscore. + * + * @param indexName the name of index passed in by user. + * @return a normalized index name. + */ + def normalizeIndexName(indexName: String): String = { + indexName.trim.replaceAll("\\s+", "_") + } + + /** + * Resolve two strings based on spark's resolver. Returns true if both are resolved. + * + * @param spark Spark session. + * @param requiredString First string. + * @param availableString Second string. + * @return True if both are equivalent based on the current spark resolver. + */ + def isResolved( + spark: SparkSession, + requiredString: String, + availableString: String): Boolean = { + val resolver: Resolver = spark.sessionState.conf.resolver + resolver(requiredString, availableString) + } + + /** + * Resolve first string from available collection of strings based on spark's resolver. Returns + * true if any resolved string is found in the collection. + * + * @param spark Spark session. + * @param requiredString String to resolve. + * @param availableStrings Available strings. + * @return True if any of the available strings can be resolved with the passed string. + */ + def isResolved( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Boolean = { + availableStrings.exists(isResolved(spark, requiredString, _)) + } + + /** + * Resolves ALL of the requiredStrings with availableStrings based on spark's resolver. Returns + * true if all the requiredStrings can be resolved with some string from availableStrings. + * Returns false otherwise. + * + * @param spark Spark session. + * @param requiredStrings List of strings to resolve. + * @param availableStrings List of availble strings to resolve from. + * @return True if every one of requiredStrings can be resolved from availableStrings. False + * otherwise. + */ + def isResolved( + spark: SparkSession, + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Boolean = { + requiredStrings.forall(isResolved(spark, _, availableStrings)) + } + + /** + * Return available string if required string can be resolved with it, based on spark resolver. + * + * @param spark Spark Session. + * @param requiredString The string that requires resolution. + * @param availableString Available list of strings to resolve from. + * @return Optional available string if resolution is successful, else None + */ + def resolve( + spark: SparkSession, + requiredString: String, + availableString: String): Option[String] = { + if (isResolved(spark, requiredString, availableString)) Some(availableString) else None + } + + /** + * Finds the first matching resolved string from the list of availableStrings, when resolving + * for requiredString. If no matching string found, return None. + * + * @param spark Spark session. + * @param requiredString The string that requires resolution. + * @param availableStrings All available strings to find a match from. + * @return First matching (i.e. resolved) string from availableStrings. If no match is found, + * return None. + */ + def resolve( + spark: SparkSession, + requiredString: String, + availableStrings: Iterable[String]): Option[String] = { + availableStrings.find(isResolved(spark, requiredString, _)) + } + + /** + * Finds all resolved strings for requiredStrings, from the list of availableStrings. Returns a + * sequence of Optional values for matches, None for unmatched strings. + * + * @param spark Spark session. + * @param requiredStrings List of strings to resolve. + * @param availableStrings List of available strings to resolve from. + * @return Sequence of optional values of resolved strings or None's. + */ + def resolve( + spark: SparkSession, + requiredStrings: Iterable[String], + availableStrings: Iterable[String]): Seq[Option[String]] = { + requiredStrings.map(resolve(spark, _, availableStrings)).toSeq + } +} diff --git a/src/test/scala/com/microsoft/hyperspace/util/IndexNameUtilsTests.scala b/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala similarity index 85% rename from src/test/scala/com/microsoft/hyperspace/util/IndexNameUtilsTests.scala rename to src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala index b0cccf5ef..15a18402d 100644 --- a/src/test/scala/com/microsoft/hyperspace/util/IndexNameUtilsTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala @@ -18,10 +18,10 @@ package com.microsoft.hyperspace.util import org.apache.spark.SparkFunSuite -class IndexNameUtilsTests extends SparkFunSuite { +class ResolverUtilsTests extends SparkFunSuite { test("Test normalizeIndexName() function.") { val indexName = " my index 1 " val expectedIndexNameAfterNorm = "my_index_1" - assert(IndexNameUtils.normalizeIndexName(indexName).equals(expectedIndexNameAfterNorm)) + assert(ResolverUtils.normalizeIndexName(indexName).equals(expectedIndexNameAfterNorm)) } } From c9d2113a72c55aa9c5c575b9fa9ef09edb262d2b Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 1 Jul 2020 12:06:12 -0700 Subject: [PATCH 14/33] fix for join rule tests failures --- .../com/microsoft/hyperspace/index/rules/JoinIndexRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bfadb724b..40d80de8d 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -594,7 +594,7 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { rIndex.indexedColumns.equals(requiredRightIndexedCols) } - private lazy val spark: SparkSession = SparkSession.getActiveSession.getOrElse { + private def spark: SparkSession = SparkSession.getActiveSession.getOrElse { throw HyperspaceException("Could not find active SparkSession") } } From a0fbf774767c3dcf88ded2a3f5672dcc074a2ca2 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 16 Jul 2020 17:46:10 -0700 Subject: [PATCH 15/33] review comments and remove isResolve methods --- .../hyperspace/actions/CreateAction.scala | 2 +- .../hyperspace/actions/CreateActionBase.scala | 15 ++-- .../index/rules/FilterIndexRule.scala | 4 +- .../index/rules/JoinIndexRule.scala | 12 +-- .../hyperspace/util/ResolverUtils.scala | 78 ++----------------- .../hyperspace/util/ResolverUtilsTests.scala | 27 ------- 6 files changed, 25 insertions(+), 113 deletions(-) delete mode 100644 src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index 8805a7115..ad2438628 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -66,7 +66,7 @@ class CreateAction( val indexedColumns = indexConfig.indexedColumns val includedColumns = indexConfig.includedColumns - ResolverUtils.isResolved(spark, indexedColumns ++ includedColumns, validColumnNames) + ResolverUtils.resolve(spark, indexedColumns ++ includedColumns, validColumnNames).isDefined } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 421645e93..f363c87a8 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -142,13 +142,14 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val resolvedIncludedColumns = ResolverUtils.resolve(spark, indexConfig.includedColumns, dfColumnNames) - val allColumns = resolvedIncludedColumns ++ resolvedIncludedColumns - if ((resolvedIncludedColumns ++ resolvedIncludedColumns).exists(_.isEmpty)) { - throw HyperspaceException( - s"Unexpected Exception: Some of the columns from $allColumns could not be resolved from " + - s"available source columns $dfColumnNames") + val allColumns = resolvedIndexedColumns ++ resolvedIncludedColumns + + (resolvedIndexedColumns, resolvedIncludedColumns) match { + case (Some(indexed), Some(included)) => (indexed, included) + case _ => + throw HyperspaceException( + s"Unexpected Exception: Some of the columns from $allColumns could not be resolved " + + s"from available source columns $dfColumnNames") } - - (resolvedIndexedColumns.map(_.get), resolvedIncludedColumns.map(_.get)) } } 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 582eb3354..e4e77849f 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/FilterIndexRule.scala @@ -213,8 +213,8 @@ object FilterIndexRule extends Rule[LogicalPlan] with Logging { val allColumnsInIndex = indexedColumns ++ includedColumns // TODO: Normalize predicates into CNF and incorporate more conditions. - ResolverUtils.isResolved(spark, indexedColumns.head, filterColumns) && - ResolverUtils.isResolved(spark, allColumnsInPlan, allColumnsInIndex) + ResolverUtils.resolve(spark, indexedColumns.head, filterColumns).isDefined && + ResolverUtils.resolve(spark, allColumnsInPlan, allColumnsInIndex).isDefined } /** 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 40d80de8d..da2a3af27 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -373,8 +373,8 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { val rRequiredAllCols = allRequiredCols(right) // Make sure required indexed columns are subset of all required columns for a subplan - require(isResolved(spark, lRequiredIndexedCols, lRequiredAllCols)) - require(isResolved(spark, rRequiredIndexedCols, rRequiredAllCols)) + require(resolve(spark, lRequiredIndexedCols, lRequiredAllCols).isDefined) + require(resolve(spark, rRequiredIndexedCols, rRequiredAllCols).isDefined) val lUsable = getUsableIndexes(lIndexes, lRequiredIndexedCols, lRequiredAllCols) val rUsable = getUsableIndexes(rIndexes, rRequiredIndexedCols, rRequiredAllCols) @@ -517,9 +517,9 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { // All required index columns should match one-to-one with all indexed columns and // vice-versa. All required columns must be present in the available index columns. - isResolved(spark, requiredIndexCols, idx.indexedColumns) && + resolve(spark, requiredIndexCols, idx.indexedColumns).isDefined && requiredIndexCols.distinct.size == idx.indexedColumns.distinct.size && - isResolved(spark, allRequiredCols, allCols) + resolve(spark, allRequiredCols, allCols).isDefined } } @@ -586,8 +586,8 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { - require(isResolved(spark, columnMapping.keys, lIndex.indexedColumns)) - require(isResolved(spark, columnMapping.values, rIndex.indexedColumns)) + require(resolve(spark, columnMapping.keys, lIndex.indexedColumns).isDefined) + require(resolve(spark, columnMapping.values, rIndex.indexedColumns).isDefined) val requiredRightIndexedCols = lIndex.indexedColumns.map(c => columnMapping(resolve(spark, c, columnMapping.keys).get)) diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala index debb8b6e4..0b6283696 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -20,86 +20,23 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.Resolver /** - * [[ResolverUtils]] provides utility functions to - * - format the index name as needed. - * - resolve strings based on spark's resolver. + * [[ResolverUtils]] provides utility functions to resolve strings based on spark's resolver. */ object ResolverUtils { - /** - * Normalize the index name by trimming space(s) at both ends, and replacing internal space(s) - * with an underscore. - * - * @param indexName the name of index passed in by user. - * @return a normalized index name. - */ - def normalizeIndexName(indexName: String): String = { - indexName.trim.replaceAll("\\s+", "_") - } - - /** - * Resolve two strings based on spark's resolver. Returns true if both are resolved. - * - * @param spark Spark session. - * @param requiredString First string. - * @param availableString Second string. - * @return True if both are equivalent based on the current spark resolver. - */ - def isResolved( - spark: SparkSession, - requiredString: String, - availableString: String): Boolean = { - val resolver: Resolver = spark.sessionState.conf.resolver - resolver(requiredString, availableString) - } - - /** - * Resolve first string from available collection of strings based on spark's resolver. Returns - * true if any resolved string is found in the collection. - * - * @param spark Spark session. - * @param requiredString String to resolve. - * @param availableStrings Available strings. - * @return True if any of the available strings can be resolved with the passed string. - */ - def isResolved( - spark: SparkSession, - requiredString: String, - availableStrings: Iterable[String]): Boolean = { - availableStrings.exists(isResolved(spark, requiredString, _)) - } - - /** - * Resolves ALL of the requiredStrings with availableStrings based on spark's resolver. Returns - * true if all the requiredStrings can be resolved with some string from availableStrings. - * Returns false otherwise. - * - * @param spark Spark session. - * @param requiredStrings List of strings to resolve. - * @param availableStrings List of availble strings to resolve from. - * @return True if every one of requiredStrings can be resolved from availableStrings. False - * otherwise. - */ - def isResolved( - spark: SparkSession, - requiredStrings: Iterable[String], - availableStrings: Iterable[String]): Boolean = { - requiredStrings.forall(isResolved(spark, _, availableStrings)) - } - /** * Return available string if required string can be resolved with it, based on spark resolver. * - * @param spark Spark Session. + * @param resolver Resolver. * @param requiredString The string that requires resolution. * @param availableString Available list of strings to resolve from. * @return Optional available string if resolution is successful, else None */ def resolve( - spark: SparkSession, + resolver: Resolver, requiredString: String, availableString: String): Option[String] = { - if (isResolved(spark, requiredString, availableString)) Some(availableString) else None + if (resolver(requiredString, availableString)) Some(availableString) else None } /** @@ -116,7 +53,7 @@ object ResolverUtils { spark: SparkSession, requiredString: String, availableStrings: Iterable[String]): Option[String] = { - availableStrings.find(isResolved(spark, requiredString, _)) + availableStrings.find(resolve(spark.sessionState.conf.resolver, requiredString, _).isDefined) } /** @@ -131,7 +68,8 @@ object ResolverUtils { def resolve( spark: SparkSession, requiredStrings: Iterable[String], - availableStrings: Iterable[String]): Seq[Option[String]] = { - requiredStrings.map(resolve(spark, _, availableStrings)).toSeq + availableStrings: Iterable[String]): Option[Seq[String]] = { + val resolved = requiredStrings.map(resolve(spark, _, availableStrings)) + if (resolved.forall(_.nonEmpty)) Some(resolved.map(_.get).toSeq) else None } } diff --git a/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala b/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala deleted file mode 100644 index 15a18402d..000000000 --- a/src/test/scala/com/microsoft/hyperspace/util/ResolverUtilsTests.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.util - -import org.apache.spark.SparkFunSuite - -class ResolverUtilsTests extends SparkFunSuite { - test("Test normalizeIndexName() function.") { - val indexName = " my index 1 " - val expectedIndexNameAfterNorm = "my_index_1" - assert(ResolverUtils.normalizeIndexName(indexName).equals(expectedIndexNameAfterNorm)) - } -} From f8cee6d717ec39ca5a6cd25022e39988a8f050c0 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 16 Jul 2020 17:54:02 -0700 Subject: [PATCH 16/33] replace iterable with Seq --- .../microsoft/hyperspace/index/rules/JoinIndexRule.scala | 8 +++++--- .../com/microsoft/hyperspace/util/ResolverUtils.scala | 8 ++++---- 2 files changed, 9 insertions(+), 7 deletions(-) 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 da2a3af27..441214899 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -586,11 +586,13 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { - require(resolve(spark, columnMapping.keys, lIndex.indexedColumns).isDefined) - require(resolve(spark, columnMapping.values, rIndex.indexedColumns).isDefined) + require(resolve(spark, columnMapping.keys.toSeq, lIndex.indexedColumns).isDefined) + require(resolve(spark, columnMapping.values.toSeq, rIndex.indexedColumns).isDefined) val requiredRightIndexedCols = - lIndex.indexedColumns.map(c => columnMapping(resolve(spark, c, columnMapping.keys).get)) + lIndex.indexedColumns.map { c => + columnMapping(resolve(spark, c, columnMapping.keys.toSeq).get) + } rIndex.indexedColumns.equals(requiredRightIndexedCols) } diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala index 0b6283696..7e6f6ffb7 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -52,7 +52,7 @@ object ResolverUtils { def resolve( spark: SparkSession, requiredString: String, - availableStrings: Iterable[String]): Option[String] = { + availableStrings: Seq[String]): Option[String] = { availableStrings.find(resolve(spark.sessionState.conf.resolver, requiredString, _).isDefined) } @@ -67,9 +67,9 @@ object ResolverUtils { */ def resolve( spark: SparkSession, - requiredStrings: Iterable[String], - availableStrings: Iterable[String]): Option[Seq[String]] = { + requiredStrings: Seq[String], + availableStrings: Seq[String]): Option[Seq[String]] = { val resolved = requiredStrings.map(resolve(spark, _, availableStrings)) - if (resolved.forall(_.nonEmpty)) Some(resolved.map(_.get).toSeq) else None + if (resolved.forall(_.nonEmpty)) Some(resolved.map(_.get)) else None } } From 2372ae9a2c5c4e8b882bc3a877b75506456ade5c Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 17 Jul 2020 14:44:31 -0700 Subject: [PATCH 17/33] Lots of major changes and bug fixes 1. ensureAttributeRequirements now compares base relation columns instead of join condition columns. (for case-insensitive support) 2. getBestIndexPair works on resolved columns from base relation instead of join condition columns. (for case-insensitive support) 3. simplification of some checks --- .../index/rules/JoinIndexRule.scala | 128 +++++++++--------- .../index/rules/JoinIndexRuleTest.scala | 17 +++ 2 files changed, 81 insertions(+), 64 deletions(-) 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 441214899..bb6915406 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -17,6 +17,7 @@ package com.microsoft.hyperspace.index.rules import scala.collection.mutable +import scala.util.Try import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging @@ -206,6 +207,10 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { private def isPlanLinear(plan: LogicalPlan): Boolean = plan.children.length <= 1 && plan.children.forall(isPlanLinear) + private def find(attributes: Seq[Attribute], column: Expression): Option[Attribute] = { + attributes.find(_.semanticEquals(column)) + } + /** * Requirements to support join optimizations using join indexes are as follows: * @@ -222,9 +227,6 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { * WHERE newT1.aliasCol = T2.b * Here, aliasCol is not directly from the base relation T1 * - * TODO: add alias resolver for supporting aliases in join condition. Until then, - * make sure this scenario isn't supported - * * 2. For each equality condition in the join predicate, one attribute must belong to the left * subplan, and another from right subplan. * E.g. A = B => A should come from left and B should come from right or vice versa. @@ -275,35 +277,49 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { l: LogicalPlan, r: LogicalPlan, condition: Expression): Boolean = { - + val conditions = extractConditions(condition) // Output attributes from base relations. Join condition attributes must belong to these // attributes val lBaseAttrs = l.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output) val rBaseAttrs = r.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output) + val allBaseAttrs = lBaseAttrs ++ rBaseAttrs + val base = new mutable.HashMap[Expression, Expression]() + + conditions.foreach { + case EqualTo(c1, c2) => + (find(allBaseAttrs, c1), find(allBaseAttrs, c2)) match { + case (Some(b1), Some(b2)) => + base.put(c1, b1) + base.put(c2, b2) + case _ => + // Found a join column not present in any base relation. + return false + } + } def fromDifferentBaseRelations(c1: Expression, c2: Expression): Boolean = { - def contains(attributes: Seq[Attribute], column: Expression): Boolean = { - attributes.exists(_.semanticEquals(column)) - } - (contains(lBaseAttrs, c1) && contains(rBaseAttrs, c2)) || - (contains(lBaseAttrs, c2) && contains(rBaseAttrs, c1)) + (lBaseAttrs.contains(base(c1)) && rBaseAttrs.contains(base(c2))) || + (lBaseAttrs.contains(base(c2)) && rBaseAttrs.contains(base(c1))) } - // Map to maintain and check one-to-one relation between join condition attributes + // Map to maintain and check one-to-one relation between join condition attributes. For join + // condition attributes, we store their corresponding base relation attributes in the map. + // This ensures uniqueness of attributes in case of case-insensitive system. E.g. We want to + // store just one copy of column 'A' when join condition contains column 'A' as well as 'a'. val attrMap = new mutable.HashMap[Expression, Expression]() - extractConditions(condition).forall { + conditions.forall { case EqualTo(c1, c2) => // c1 and c2 should belong to l and r respectively, or r and l respectively. if (!fromDifferentBaseRelations(c1, c2)) { return false } // The following validates that c1 is compared only against c2 and vice versa - if (attrMap.contains(c1) && attrMap.contains(c2)) { - attrMap(c1).equals(c2) && attrMap(c2).equals(c1) - } else if (!attrMap.contains(c1) && !attrMap.contains(c2)) { - attrMap.put(c1, c2) - attrMap.put(c2, c1) + if (attrMap.contains(base(c1)) && attrMap.contains(base(c2))) { + attrMap(base(c1)).equals(base(c2)) && attrMap(base(c2)).equals(base(c1)) + } else if (!attrMap.contains(base(c1)) && !attrMap.contains(base(c2))) { + attrMap.put(base(c1), base(c2)) + attrMap.put(base(c2), base(c1)) true } else { false @@ -364,13 +380,20 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { joinCondition: Expression, lIndexes: Seq[IndexLogEntry], rIndexes: Seq[IndexLogEntry]): Option[(IndexLogEntry, IndexLogEntry)] = { + val lBaseAttrs = + left.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output).map(_.name) + val rBaseAttrs = + right.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output).map(_.name) - val lRequiredIndexedCols = requiredIndexedCols(left, joinCondition) - val rRequiredIndexedCols = requiredIndexedCols(right, joinCondition) - val lRMap = getLRColumnMapping(lRequiredIndexedCols, rRequiredIndexedCols, joinCondition) + // Map of left resolved columns with their corresponding right resolved + // columns from condition. + val lRMap = getLRColumnMapping(lBaseAttrs, rBaseAttrs, joinCondition) + val lRequiredIndexedCols = lRMap.keys.toSeq + val rRequiredIndexedCols = lRMap.values.toSeq - val lRequiredAllCols = allRequiredCols(left) - val rRequiredAllCols = allRequiredCols(right) + // All required columns resolved with base relation. + val lRequiredAllCols: Seq[String] = resolve(spark, allRequiredCols(left), lBaseAttrs).get + val rRequiredAllCols: Seq[String] = resolve(spark, allRequiredCols(right), rBaseAttrs).get // Make sure required indexed columns are subset of all required columns for a subplan require(resolve(spark, lRequiredIndexedCols, lRequiredAllCols).isDefined) @@ -383,27 +406,6 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { compatibleIndexPairs.map(indexPairs => JoinIndexRanker.rank(indexPairs).head) } - /** - * Returns list of column names which must be in the indexed columns config of a selected - * index. - * - * @param plan logical plan - * @param condition join condition used to pull out required indexed columns in the plan - * @return list of column names from this plan which must be part of indexed columns in a chosen - * index - */ - private def requiredIndexedCols(plan: LogicalPlan, condition: Expression): Seq[String] = { - val cleanedPlan = CleanupAliases(plan) - CleanupAliases - .trimNonTopLevelAliases(condition) - .references - .collect { - case column if cleanedPlan.references.contains(column) => column.name - } - .toSeq - .distinct - } - /** * Returns list of column names which must be present in either the indexed or the included * columns list of a selected index. For this, collect all columns referenced in the plan @@ -471,16 +473,18 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { rRequiredIndexedCols: Seq[String], condition: Expression): Map[String, String] = { extractConditions(condition).map { - case EqualTo(attr1: AttributeReference, attr2: AttributeReference) - if lRequiredIndexedCols.contains(attr1.name) && rRequiredIndexedCols.contains( - attr2.name) => - (attr1.name, attr2.name) - case EqualTo(attr1: AttributeReference, attr2: AttributeReference) - if lRequiredIndexedCols.contains(attr2.name) && rRequiredIndexedCols.contains( - attr1.name) => - (attr2.name, attr1.name) - case _ => - throw new IllegalStateException("Unexpected exception while using join rule") + case EqualTo(attr1: AttributeReference, attr2: AttributeReference) => + Try { + (resolve(spark, attr1.name, lRequiredIndexedCols).get, + resolve(spark, attr2.name, rRequiredIndexedCols).get) + }.getOrElse { + Try { + (resolve(spark, attr2.name, lRequiredIndexedCols).get, + resolve(spark, attr1.name, rRequiredIndexedCols).get) + }.getOrElse { + throw new IllegalStateException("Unexpected exception while using join rule") + } + } }.toMap } @@ -502,9 +506,12 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { /** * Get usable indexes which satisfy indexed and included column requirements. * + * Pre-requisite: the indexed and included columns required must be already resolved with their + * corresponding base relation columns at this point. + * * @param indexes All available indexes for the logical plan - * @param requiredIndexCols required indexed columns - * @param allRequiredCols required included columns + * @param requiredIndexCols required indexed columns resolved with their base relation column. + * @param allRequiredCols required included columns resolved with their base relation column. * @return Indexes which satisfy the indexed and covering column requirements from the logical * plan and join condition */ @@ -517,9 +524,8 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { // All required index columns should match one-to-one with all indexed columns and // vice-versa. All required columns must be present in the available index columns. - resolve(spark, requiredIndexCols, idx.indexedColumns).isDefined && - requiredIndexCols.distinct.size == idx.indexedColumns.distinct.size && - resolve(spark, allRequiredCols, allCols).isDefined + requiredIndexCols.toSet.equals(idx.indexedColumns.toSet) && + allRequiredCols.forall(allCols.contains) } } @@ -586,13 +592,7 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { - require(resolve(spark, columnMapping.keys.toSeq, lIndex.indexedColumns).isDefined) - require(resolve(spark, columnMapping.values.toSeq, rIndex.indexedColumns).isDefined) - - val requiredRightIndexedCols = - lIndex.indexedColumns.map { c => - columnMapping(resolve(spark, c, columnMapping.keys.toSeq).get) - } + val requiredRightIndexedCols = lIndex.indexedColumns.map(columnMapping) rIndex.indexedColumns.equals(requiredRightIndexedCols) } diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala index c81637559..40a259029 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala @@ -347,6 +347,23 @@ class JoinIndexRuleTest extends HyperspaceSuite { } } + test("Join rule doesn't update plan if columns don't have one-to-one mapping case insensitive") { + val t1ProjectNode = Project(Seq(t1c1, t1c3), t1FilterNode) + val t2ProjectNode = Project(Seq(t2c1, t2c3), t2FilterNode) + + val t1c1Caps = t1c1.withName("T1C1") + val t2c1Caps = t2c1.withName("T2C1") + + val joinCondition = And(EqualTo(t1c1, t2c1), EqualTo(t1c1Caps, t2c1Caps)) + val originalPlan = Join(t1ProjectNode, t2ProjectNode, JoinType("inner"), Some(joinCondition)) + val updatedPlan = JoinIndexRule(originalPlan) + assert(!updatedPlan.equals(originalPlan)) + + val indexPaths = Seq(getIndexDataFilesPath("t1i1"), getIndexDataFilesPath("t2i1")) + verifyUpdatedIndex(originalPlan, updatedPlan, indexPaths) + } + + test("Join rule updates plan for composite query for repeated predicates") { val t1ProjectNode = Project(Seq(t1c1, t1c2, t1c3), t1FilterNode) val t2ProjectNode = Project(Seq(t2c1, t2c2, t2c3), t2FilterNode) From ed77af3ddda103dbd95d7581bdc1a288aa5a8b62 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 20 Jul 2020 11:07:37 -0700 Subject: [PATCH 18/33] minor variable name fix --- .../hyperspace/index/rules/JoinIndexRule.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) 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 bb6915406..145618d24 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -461,26 +461,26 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { * * This mapping is used to find compatible indexes of T1 and T2. * - * @param lRequiredIndexedCols required indexed columns from left plan - * @param rRequiredIndexedCols required indexed columns from right plan + * @param leftBaseAttrs required indexed columns from left plan + * @param rightBaseAttrs required indexed columns from right plan * @param condition join condition which will be used to find the left-right column mapping * @return Mapping of corresponding columns from left and right, depending on the join * condition. The keys represent columns from left subplan. The values are columns from * right subplan. */ private def getLRColumnMapping( - lRequiredIndexedCols: Seq[String], - rRequiredIndexedCols: Seq[String], + leftBaseAttrs: Seq[String], + rightBaseAttrs: Seq[String], condition: Expression): Map[String, String] = { extractConditions(condition).map { case EqualTo(attr1: AttributeReference, attr2: AttributeReference) => Try { - (resolve(spark, attr1.name, lRequiredIndexedCols).get, - resolve(spark, attr2.name, rRequiredIndexedCols).get) + (resolve(spark, attr1.name, leftBaseAttrs).get, + resolve(spark, attr2.name, rightBaseAttrs).get) }.getOrElse { Try { - (resolve(spark, attr2.name, lRequiredIndexedCols).get, - resolve(spark, attr1.name, rRequiredIndexedCols).get) + (resolve(spark, attr2.name, leftBaseAttrs).get, + resolve(spark, attr1.name, rightBaseAttrs).get) }.getOrElse { throw new IllegalStateException("Unexpected exception while using join rule") } From b769c6092952b0dde9c82facccc4b6cf8fab3bc5 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 20 Jul 2020 11:53:39 -0700 Subject: [PATCH 19/33] code cleanup to use canonicalized --- .../index/rules/JoinIndexRule.scala | 55 +++++++++---------- 1 file changed, 25 insertions(+), 30 deletions(-) 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 145618d24..e08d09828 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -24,7 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.CleanupAliases import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, AttributeSet, EqualTo, Expression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference, AttributeSet, EqualTo, Expression} import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} @@ -207,10 +207,6 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { private def isPlanLinear(plan: LogicalPlan): Boolean = plan.children.length <= 1 && plan.children.forall(isPlanLinear) - private def find(attributes: Seq[Attribute], column: Expression): Option[Attribute] = { - attributes.find(_.semanticEquals(column)) - } - /** * Requirements to support join optimizations using join indexes are as follows: * @@ -279,27 +275,21 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { condition: Expression): Boolean = { val conditions = extractConditions(condition) // Output attributes from base relations. Join condition attributes must belong to these - // attributes - val lBaseAttrs = l.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output) - val rBaseAttrs = r.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output) - val allBaseAttrs = lBaseAttrs ++ rBaseAttrs - val base = new mutable.HashMap[Expression, Expression]() - - conditions.foreach { - case EqualTo(c1, c2) => - (find(allBaseAttrs, c1), find(allBaseAttrs, c2)) match { - case (Some(b1), Some(b2)) => - base.put(c1, b1) - base.put(c2, b2) - case _ => - // Found a join column not present in any base relation. - return false - } - } + // attributes. We work on canonicalized forms to make sure we support case-sensitivity. + val lBaseAttrs = l + .collectLeaves() + .filter(_.isInstanceOf[LogicalRelation]) + .flatMap(_.output) + .map(_.canonicalized) + val rBaseAttrs = r + .collectLeaves() + .filter(_.isInstanceOf[LogicalRelation]) + .flatMap(_.output) + .map(_.canonicalized) def fromDifferentBaseRelations(c1: Expression, c2: Expression): Boolean = { - (lBaseAttrs.contains(base(c1)) && rBaseAttrs.contains(base(c2))) || - (lBaseAttrs.contains(base(c2)) && rBaseAttrs.contains(base(c1))) + (lBaseAttrs.contains(c1) && rBaseAttrs.contains(c2)) || + (lBaseAttrs.contains(c2) && rBaseAttrs.contains(c1)) } // Map to maintain and check one-to-one relation between join condition attributes. For join @@ -308,18 +298,19 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { // store just one copy of column 'A' when join condition contains column 'A' as well as 'a'. val attrMap = new mutable.HashMap[Expression, Expression]() + // TODO: Check 2: Exclusivity Checks. conditions.forall { - case EqualTo(c1, c2) => + case EqualTo(ExtractCanonicalized(c1), ExtractCanonicalized(c2)) => // c1 and c2 should belong to l and r respectively, or r and l respectively. if (!fromDifferentBaseRelations(c1, c2)) { return false } // The following validates that c1 is compared only against c2 and vice versa - if (attrMap.contains(base(c1)) && attrMap.contains(base(c2))) { - attrMap(base(c1)).equals(base(c2)) && attrMap(base(c2)).equals(base(c1)) - } else if (!attrMap.contains(base(c1)) && !attrMap.contains(base(c2))) { - attrMap.put(base(c1), base(c2)) - attrMap.put(base(c2), base(c1)) + if (attrMap.contains(c1) && attrMap.contains(c2)) { + attrMap(c1).equals(c2) && attrMap(c2).equals(c1) + } else if (!attrMap.contains(c1) && !attrMap.contains(c2)) { + attrMap.put(c1, c2) + attrMap.put(c2, c1) true } else { false @@ -600,3 +591,7 @@ object JoinIndexRule extends Rule[LogicalPlan] with Logging { throw HyperspaceException("Could not find active SparkSession") } } + +private[rules] object ExtractCanonicalized { + def unapply(expr: Expression): Option[Expression] = Some(expr.canonicalized) +} From 603008e8021c9acf949c093eb573f9fb574512ca Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 20 Jul 2020 13:59:11 -0700 Subject: [PATCH 20/33] git commit -m merge errors fixed --- .../com/microsoft/hyperspace/index/CreateIndexTests.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala index 13788595e..e2af5d35f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala @@ -25,9 +25,9 @@ import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} import com.microsoft.hyperspace.util.FileUtils class CreateIndexTests extends HyperspaceSuite { + override val systemPath = new Path("src/test/resources/indexLocation") private val sampleData = SampleData.testData private val sampleParquetDataLocation = "src/test/resources/sampleparquet" - private val indexSystemPath = "src/test/resources/indexLocation" private val indexConfig1 = IndexConfig("index1", Seq("RGUID"), Seq("Date")) private val indexConfig2 = IndexConfig("index2", Seq("Query"), Seq("imprs")) private var df: DataFrame = _ @@ -39,14 +39,12 @@ class CreateIndexTests extends HyperspaceSuite { val sparkSession = spark import sparkSession.implicits._ hyperspace = new Hyperspace(sparkSession) - FileUtils.delete(new Path(indexSystemPath)) FileUtils.delete(new Path(sampleParquetDataLocation)) val dfFromSample = sampleData.toDF("Date", "RGUID", "Query", "imprs", "clicks") dfFromSample.write.parquet(sampleParquetDataLocation) df = spark.read.parquet(sampleParquetDataLocation) - spark.conf.set(IndexConstants.INDEX_SYSTEM_PATH, indexSystemPath) } override def afterAll(): Unit = { @@ -55,7 +53,7 @@ class CreateIndexTests extends HyperspaceSuite { } after { - FileUtils.delete(new Path(indexSystemPath)) + FileUtils.delete(systemPath) } test("Creating one index.") { From a95a54436818aef8d02a712a41f5714b15c3077d Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 22 Jul 2020 11:48:38 -0700 Subject: [PATCH 21/33] simplify code --- .../hyperspace/actions/CreateActionBase.scala | 12 ++------- .../index/rules/JoinIndexRule.scala | 25 ++++++++----------- 2 files changed, 13 insertions(+), 24 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index f363c87a8..636ba4a53 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -51,16 +51,8 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) val signatureProvider = LogicalPlanSignatureProvider.create() - // Resolve the passed column names with existing column names from the dataframe if possible. - val (resolvedIndexedColumns, resolvedIncludedColumns) = { - try { - // Try creating log entry with resolved column names. - resolveConfig(df, indexConfig) - } catch { - // Creating index log entry with whatever the user passed. - case _: Exception => (indexConfig.indexedColumns, indexConfig.includedColumns) - } - } + // Resolve the passed column names with existing column names from the dataframe. + val (resolvedIndexedColumns, resolvedIncludedColumns) = resolveConfig(df, indexConfig) val schema = { val allColumns = resolvedIndexedColumns ++ resolvedIncludedColumns df.select(allColumns.head, allColumns.tail: _*).schema 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 88b744125..b26ea61a1 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -305,19 +305,20 @@ object JoinIndexRule // store just one copy of column 'A' when join condition contains column 'A' as well as 'a'. val attrMap = new mutable.HashMap[Expression, Expression]() - // TODO: Check 2: Exclusivity Checks. conditions.forall { - case EqualTo(ExtractCanonicalized(c1), ExtractCanonicalized(c2)) => - // c1 and c2 should belong to l and r respectively, or r and l respectively. - if (!fromDifferentBaseRelations(c1, c2)) { + case EqualTo(c1, c2) => + val (c1Canonicalized, c2Canonicalized) = (c1.canonicalized, c2.canonicalized) + // Check 1: c1 and c2 should belong to l and r respectively, or r and l respectively. + if (!fromDifferentBaseRelations(c1Canonicalized, c2Canonicalized)) { return false } - // The following validates that c1 is compared only against c2 and vice versa - if (attrMap.contains(c1) && attrMap.contains(c2)) { - attrMap(c1).equals(c2) && attrMap(c2).equals(c1) - } else if (!attrMap.contains(c1) && !attrMap.contains(c2)) { - attrMap.put(c1, c2) - attrMap.put(c2, c1) + // Check 2: c1 is compared only against c2 and vice versa. + if (attrMap.contains(c1Canonicalized) && attrMap.contains(c2Canonicalized)) { + attrMap(c1Canonicalized).equals(c2Canonicalized) && + attrMap(c2Canonicalized).equals(c1Canonicalized) + } else if (!attrMap.contains(c1Canonicalized) && !attrMap.contains(c2Canonicalized)) { + attrMap.put(c1Canonicalized, c2Canonicalized) + attrMap.put(c2Canonicalized, c1Canonicalized) true } else { false @@ -558,7 +559,3 @@ object JoinIndexRule rIndex.indexedColumns.equals(requiredRightIndexedCols) } } - -private[rules] object ExtractCanonicalized { - def unapply(expr: Expression): Option[Expression] = Some(expr.canonicalized) -} From 090a3af2f46dd9de2bac2bc5dda2953294514ac6 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 22 Jul 2020 12:13:59 -0700 Subject: [PATCH 22/33] review comments --- .../microsoft/hyperspace/actions/CreateAction.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala index 4ca2b2374..47c958c9c 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateAction.scala @@ -64,12 +64,11 @@ class CreateAction( } } - private def isValidIndexSchema(indexConfig: IndexConfig, schema: StructType): Boolean = { - val validColumnNames = schema.fieldNames - val indexedColumns = indexConfig.indexedColumns - val includedColumns = indexConfig.includedColumns - - ResolverUtils.resolve(spark, indexedColumns ++ includedColumns, validColumnNames).isDefined + private def isValidIndexSchema(config: IndexConfig, schema: StructType): Boolean = { + // Resolve index config columns from available column names present in the schema. + ResolverUtils + .resolve(spark, config.indexedColumns ++ config.includedColumns, schema.fieldNames) + .isDefined } // TODO: The following should be protected, but RefreshAction is calling CreateAction.op(). From 489908cea58ac09bdb724fe08080c66e6af3742f Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 22 Jul 2020 12:41:05 -0700 Subject: [PATCH 23/33] added comments --- .../com/microsoft/hyperspace/index/HyperspaceSuite.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala index b350a0707..678e31d17 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala @@ -73,6 +73,10 @@ trait HyperspaceSuite extends SparkFunSuite with SparkInvolvedSuite { } } + /** + * This method is not thread safe. + * Reverts spark config for variable after calling f + */ protected def withSparkConf(confName: String, confValue: Any)(f: => Unit): Unit = { val original = spark.conf.get(confName) try { From fca13ae28e8c479a100ca096feb67c5be8730e23 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 22 Jul 2020 14:16:57 -0700 Subject: [PATCH 24/33] show unresolved columns in output during create --- .../hyperspace/actions/CreateActionBase.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 636ba4a53..f90bc3836 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -129,18 +129,19 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) indexConfig: IndexConfig): (Seq[String], Seq[String]) = { val spark = df.sparkSession val dfColumnNames = df.schema.fieldNames - val resolvedIndexedColumns = - ResolverUtils.resolve(spark, indexConfig.indexedColumns, dfColumnNames) - val resolvedIncludedColumns = - ResolverUtils.resolve(spark, indexConfig.includedColumns, dfColumnNames) - - val allColumns = resolvedIndexedColumns ++ resolvedIncludedColumns + val indexedColumns = indexConfig.indexedColumns + val includedColumns = indexConfig.includedColumns + val resolvedIndexedColumns = ResolverUtils.resolve(spark, indexedColumns, dfColumnNames) + val resolvedIncludedColumns = ResolverUtils.resolve(spark, includedColumns, dfColumnNames) (resolvedIndexedColumns, resolvedIncludedColumns) match { case (Some(indexed), Some(included)) => (indexed, included) case _ => + val unresolvedColumns = (indexedColumns ++ includedColumns) + .map(c => (c, ResolverUtils.resolve(spark, c, dfColumnNames))) + .collect { case c if c._2.isEmpty => c._1 } throw HyperspaceException( - s"Unexpected Exception: Some of the columns from $allColumns could not be resolved " + + s"Unexpected Exception: Columns $unresolvedColumns could not be resolved " + s"from available source columns $dfColumnNames") } } From ee30227aa3668d701cd2908a4ce82684fa162e0a Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 22 Jul 2020 20:31:42 -0700 Subject: [PATCH 25/33] review comment --- .../scala/com/microsoft/hyperspace/util/ResolverUtils.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala index 7e6f6ffb7..1b0b92cba 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -29,7 +29,7 @@ object ResolverUtils { * * @param resolver Resolver. * @param requiredString The string that requires resolution. - * @param availableString Available list of strings to resolve from. + * @param availableString Available string to resolve from. * @return Optional available string if resolution is successful, else None */ def resolve( @@ -45,7 +45,7 @@ object ResolverUtils { * * @param spark Spark session. * @param requiredString The string that requires resolution. - * @param availableStrings All available strings to find a match from. + * @param availableStrings All available strings to resolve from. * @return First matching (i.e. resolved) string from availableStrings. If no match is found, * return None. */ @@ -62,7 +62,7 @@ object ResolverUtils { * * @param spark Spark session. * @param requiredStrings List of strings to resolve. - * @param availableStrings List of available strings to resolve from. + * @param availableStrings All available strings to resolve from. * @return Sequence of optional values of resolved strings or None's. */ def resolve( From ce72622065d2dc21562bb95beab0dc5a82ebe6ca Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 23 Jul 2020 11:18:47 -0700 Subject: [PATCH 26/33] comments cleaned up --- .../scala/com/microsoft/hyperspace/util/ResolverUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala index 1b0b92cba..36f1fce8c 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -63,7 +63,7 @@ object ResolverUtils { * @param spark Spark session. * @param requiredStrings List of strings to resolve. * @param availableStrings All available strings to resolve from. - * @return Sequence of optional values of resolved strings or None's. + * @return Optional Seq of resolved strings if all required strings are resolved. Else, None. */ def resolve( spark: SparkSession, From a29b3d78363739f1c65c750bcf638dcc54ab5912 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 27 Jul 2020 09:49:03 -0700 Subject: [PATCH 27/33] review comments --- .../hyperspace/actions/CreateActionBase.scala | 2 +- .../microsoft/hyperspace/util/ResolverUtils.scala | 7 +++---- .../hyperspace/index/CreateIndexTests.scala | 13 +++++++------ .../hyperspace/index/E2EHyperspaceRulesTests.scala | 2 +- .../hyperspace/index/HyperspaceSuite.scala | 14 -------------- .../hyperspace/index/rules/JoinIndexRuleTest.scala | 4 ++-- 6 files changed, 14 insertions(+), 28 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 0ca4fddd8..4b977b624 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -142,7 +142,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) .map(c => (c, ResolverUtils.resolve(spark, c, dfColumnNames))) .collect { case c if c._2.isEmpty => c._1 } throw HyperspaceException( - s"Unexpected Exception: Columns $unresolvedColumns could not be resolved " + + s"Columns $unresolvedColumns could not be resolved " + s"from available source columns $dfColumnNames") } } diff --git a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala index 36f1fce8c..19bda600e 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/ResolverUtils.scala @@ -57,8 +57,8 @@ object ResolverUtils { } /** - * Finds all resolved strings for requiredStrings, from the list of availableStrings. Returns a - * sequence of Optional values for matches, None for unmatched strings. + * Finds all resolved strings for requiredStrings, from the list of availableStrings. Returns + * optional seq of resolved strings if all required strings are resolved, otherwise None. * * @param spark Spark session. * @param requiredStrings List of strings to resolve. @@ -69,7 +69,6 @@ object ResolverUtils { spark: SparkSession, requiredStrings: Seq[String], availableStrings: Seq[String]): Option[Seq[String]] = { - val resolved = requiredStrings.map(resolve(spark, _, availableStrings)) - if (resolved.forall(_.nonEmpty)) Some(resolved.map(_.get)) else None + Some(requiredStrings.map(resolve(spark, _, availableStrings).getOrElse { return None })) } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala index e2af5d35f..1a40186ed 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/CreateIndexTests.scala @@ -20,11 +20,12 @@ import scala.collection.mutable.WrappedArray import org.apache.hadoop.fs.Path import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.plans.SQLHelper import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} import com.microsoft.hyperspace.util.FileUtils -class CreateIndexTests extends HyperspaceSuite { +class CreateIndexTests extends HyperspaceSuite with SQLHelper { override val systemPath = new Path("src/test/resources/indexLocation") private val sampleData = SampleData.testData private val sampleParquetDataLocation = "src/test/resources/sampleparquet" @@ -58,7 +59,7 @@ class CreateIndexTests extends HyperspaceSuite { test("Creating one index.") { hyperspace.createIndex(df, indexConfig1) - val count = hyperspace.indexes.where(s"""name = "${indexConfig1.indexName}" """).count + val count = hyperspace.indexes.where(s"name = '${indexConfig1.indexName}' ").count assert(count == 1) } @@ -87,18 +88,18 @@ class CreateIndexTests extends HyperspaceSuite { test("Index creation passes with columns of different case if case-sensitivity is false.") { hyperspace.createIndex(df, IndexConfig("index1", Seq("qUeRy"), Seq("ImpRS"))) - val indexes = hyperspace.indexes.where(s"""name = "${indexConfig1.indexName}" """) + val indexes = hyperspace.indexes.where(s"name = '${indexConfig1.indexName}' ") assert(indexes.count == 1) assert( - indexes.head.getAs[WrappedArray[String]]("indexedColumns").head.equals("Query"), + indexes.head.getAs[WrappedArray[String]]("indexedColumns").head == "Query", "Indexed columns with wrong case are stored in metadata") assert( - indexes.head.getAs[WrappedArray[String]]("includedColumns").head.equals("imprs"), + indexes.head.getAs[WrappedArray[String]]("includedColumns").head == "imprs", "Included columns with wrong case are stored in metadata") } test("Index creation fails with columns of different case if case-sensitivity is true.") { - withSparkConf("spark.sql.caseSensitive", true) { + withSQLConf("spark.sql.caseSensitive" -> "true") { val exception = intercept[HyperspaceException] { hyperspace.createIndex(df, IndexConfig("index1", Seq("qUeRy"), Seq("ImpRS"))) } diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index cf56e4c0f..135b58097 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -99,7 +99,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite { def query(): DataFrame = df.filter("C3 == 'facebook'").select("C3", "c1") - // verify if case-insensitive index works with case-insensitive query + // Verify if case-insensitive index works with case-insensitive query. verifyIndexUsage(query, Seq(getIndexFilesPath(indexConfig.indexName))) } diff --git a/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala index 678e31d17..bc2b53e8f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/HyperspaceSuite.scala @@ -72,18 +72,4 @@ trait HyperspaceSuite extends SparkFunSuite with SparkInvolvedSuite { } } } - - /** - * This method is not thread safe. - * Reverts spark config for variable after calling f - */ - protected def withSparkConf(confName: String, confValue: Any)(f: => Unit): Unit = { - val original = spark.conf.get(confName) - try { - spark.conf.set(confName, confValue.toString) - f - } finally { - spark.conf.set(confName, original) - } - } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala index 2077743fc..fa6975f18 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala @@ -332,7 +332,8 @@ class JoinIndexRuleTest extends HyperspaceRuleTestSuite with SQLHelper { } } - test("Join rule doesn't update plan if columns don't have one-to-one mapping case insensitive") { + test("Join rule updates plan if columns have one-to-one mapping with repeated " + + "case-insensitive predicates") { val t1ProjectNode = Project(Seq(t1c1, t1c3), t1FilterNode) val t2ProjectNode = Project(Seq(t2c1, t2c3), t2FilterNode) @@ -348,7 +349,6 @@ class JoinIndexRuleTest extends HyperspaceRuleTestSuite with SQLHelper { verifyUpdatedIndex(originalPlan, updatedPlan, indexPaths) } - test("Join rule updates plan for composite query for repeated predicates") { val t1ProjectNode = Project(Seq(t1c1, t1c2, t1c3), t1FilterNode) val t2ProjectNode = Project(Seq(t2c1, t2c2, t2c3), t2FilterNode) From a602ee3342a56b7ac35de6a5129d8ba481deceaf Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 3 Aug 2020 10:10:47 -0700 Subject: [PATCH 28/33] some review comments --- .../index/rules/JoinIndexRule.scala | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) 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 57e0ba44b..ac5b52174 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -236,6 +236,9 @@ object JoinIndexRule * WHERE newT1.aliasCol = T2.b * Here, aliasCol is not directly from the base relation T1 * + * TODO: add alias resolver for supporting aliases in join condition. Until then, + * make sure this scenario isn't supported + * * 2. For each equality condition in the join predicate, one attribute must belong to the left * subplan, and another from right subplan. * E.g. A = B => A should come from left and B should come from right or vice versa. @@ -286,7 +289,6 @@ object JoinIndexRule l: LogicalPlan, r: LogicalPlan, condition: Expression): Boolean = { - val conditions = extractConditions(condition) // Output attributes from base relations. Join condition attributes must belong to these // attributes. We work on canonicalized forms to make sure we support case-sensitivity. val lBaseAttrs = l @@ -311,20 +313,20 @@ object JoinIndexRule // store just one copy of column 'A' when join condition contains column 'A' as well as 'a'. val attrMap = new mutable.HashMap[Expression, Expression]() - conditions.forall { - case EqualTo(c1, c2) => - val (c1Canonicalized, c2Canonicalized) = (c1.canonicalized, c2.canonicalized) + extractConditions(condition).forall { + case EqualTo(e1, e2) => + val (c1, c2) = (e1.canonicalized, e2.canonicalized) // Check 1: c1 and c2 should belong to l and r respectively, or r and l respectively. - if (!fromDifferentBaseRelations(c1Canonicalized, c2Canonicalized)) { + if (!fromDifferentBaseRelations(c1, c2)) { return false } // Check 2: c1 is compared only against c2 and vice versa. - if (attrMap.contains(c1Canonicalized) && attrMap.contains(c2Canonicalized)) { - attrMap(c1Canonicalized).equals(c2Canonicalized) && - attrMap(c2Canonicalized).equals(c1Canonicalized) - } else if (!attrMap.contains(c1Canonicalized) && !attrMap.contains(c2Canonicalized)) { - attrMap.put(c1Canonicalized, c2Canonicalized) - attrMap.put(c2Canonicalized, c1Canonicalized) + if (attrMap.contains(c1) && attrMap.contains(c2)) { + attrMap(c1).equals(c2) && + attrMap(c2).equals(c1) + } else if (!attrMap.contains(c1) && !attrMap.contains(c2)) { + attrMap.put(c1, c2) + attrMap.put(c2, c1) true } else { false @@ -361,8 +363,8 @@ object JoinIndexRule val rRequiredIndexedCols = lRMap.values.toSeq // All required columns resolved with base relation. - val lRequiredAllCols: Seq[String] = resolve(spark, allRequiredCols(left), lBaseAttrs).get - val rRequiredAllCols: Seq[String] = resolve(spark, allRequiredCols(right), rBaseAttrs).get + val lRequiredAllCols = resolve(spark, allRequiredCols(left), lBaseAttrs).get + val rRequiredAllCols = resolve(spark, allRequiredCols(right), rBaseAttrs).get // Make sure required indexed columns are subset of all required columns for a subplan require(resolve(spark, lRequiredIndexedCols, lRequiredAllCols).isDefined) From 5671e4a6aa779b129678eaa0b8200689eaf9fb5e Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 3 Aug 2020 10:25:54 -0700 Subject: [PATCH 29/33] more review comments --- .../index/rules/JoinIndexRule.scala | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) 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 ac5b52174..7b508352d 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.CleanupAliases import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{Alias, And, AttributeReference, AttributeSet, EqualTo, Expression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, AttributeSet, EqualTo, Expression} import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} @@ -291,16 +291,8 @@ object JoinIndexRule condition: Expression): Boolean = { // Output attributes from base relations. Join condition attributes must belong to these // attributes. We work on canonicalized forms to make sure we support case-sensitivity. - val lBaseAttrs = l - .collectLeaves() - .filter(_.isInstanceOf[LogicalRelation]) - .flatMap(_.output) - .map(_.canonicalized) - val rBaseAttrs = r - .collectLeaves() - .filter(_.isInstanceOf[LogicalRelation]) - .flatMap(_.output) - .map(_.canonicalized) + val lBaseAttrs = relationOutputs(l).map(_.canonicalized) + val rBaseAttrs = relationOutputs(r).map(_.canonicalized) def fromDifferentBaseRelations(c1: Expression, c2: Expression): Boolean = { (lBaseAttrs.contains(c1) && rBaseAttrs.contains(c2)) || @@ -351,10 +343,8 @@ object JoinIndexRule joinCondition: Expression, lIndexes: Seq[IndexLogEntry], rIndexes: Seq[IndexLogEntry]): Option[(IndexLogEntry, IndexLogEntry)] = { - val lBaseAttrs = - left.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output).map(_.name) - val rBaseAttrs = - right.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output).map(_.name) + val lBaseAttrs = relationOutputs(left).map(_.name) + val rBaseAttrs = relationOutputs(right).map(_.name) // Map of left resolved columns with their corresponding right resolved // columns from condition. @@ -377,6 +367,10 @@ object JoinIndexRule compatibleIndexPairs.map(indexPairs => JoinIndexRanker.rank(indexPairs).head) } + private def relationOutputs(l: LogicalPlan): Seq[Attribute] = { + l.collectLeaves().filter(_.isInstanceOf[LogicalRelation]).flatMap(_.output) + } + /** * Returns list of column names which must be present in either the indexed or the included * columns list of a selected index. For this, collect all columns referenced in the plan From ef60e784e995182ed0a65fd9336cabde7c185e81 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 3 Aug 2020 11:57:02 -0700 Subject: [PATCH 30/33] revert removed check --- .../com/microsoft/hyperspace/index/rules/JoinIndexRule.scala | 3 +++ 1 file changed, 3 insertions(+) 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 7b508352d..aaa7d85b5 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -557,6 +557,9 @@ object JoinIndexRule lIndex: IndexLogEntry, rIndex: IndexLogEntry, columnMapping: Map[String, String]): Boolean = { + require(columnMapping.keys.toSet.equals(lIndex.indexedColumns.toSet)) + require(columnMapping.values.toSet.equals(rIndex.indexedColumns.toSet)) + val requiredRightIndexedCols = lIndex.indexedColumns.map(columnMapping) rIndex.indexedColumns.equals(requiredRightIndexedCols) } From 04c410e97411205589009211d7dea96b26a0d0f9 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 3 Aug 2020 18:07:34 -0700 Subject: [PATCH 31/33] review comments --- .../index/rules/JoinIndexRule.scala | 3 +-- .../index/E2EHyperspaceRulesTests.scala | 23 +++++++++++++++++-- 2 files changed, 22 insertions(+), 4 deletions(-) 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 aaa7d85b5..026ba484d 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/JoinIndexRule.scala @@ -314,8 +314,7 @@ object JoinIndexRule } // Check 2: c1 is compared only against c2 and vice versa. if (attrMap.contains(c1) && attrMap.contains(c2)) { - attrMap(c1).equals(c2) && - attrMap(c2).equals(c1) + attrMap(c1).equals(c2) && attrMap(c2).equals(c1) } else if (!attrMap.contains(c1) && !attrMap.contains(c2)) { attrMap.put(c1, c2) attrMap.put(c2, c1) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index a2ea3c370..afc016d86 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -18,14 +18,15 @@ package com.microsoft.hyperspace.index import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, Row} +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData} import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule} -class E2EHyperspaceRulesTests extends HyperspaceSuite { +class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { private val sampleData = SampleData.testData private val testDir = "src/test/resources/e2eTests/" private val sampleParquetDataLocation = testDir + "sampleparquet" @@ -103,6 +104,24 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite { verifyIndexUsage(query, Seq(getIndexFilesPath(indexConfig.indexName))) } + test("E2E test for case sensitive filter query where changing conf changes behavior.") { + val df = spark.read.parquet(sampleParquetDataLocation) + val indexConfig = IndexConfig("filterIndex", Seq("c3"), Seq("c1")) + + hyperspace.createIndex(df, indexConfig) + def query(): DataFrame = df.filter("C3 == 'facebook'").select("C3", "c1") + + withSQLConf("spark.sql.caseSensitive" -> "true") { + intercept[AnalysisException] { + query().show + } + } + + withSQLConf("spark.sql.caseSensitive" -> "false") { + verifyIndexUsage(query, Seq(getIndexFilesPath(indexConfig.indexName))) + } + } + test("E2E test for filter query when all columns are selected.") { val df = spark.read.parquet(sampleParquetDataLocation) val indexConfig = IndexConfig("filterIndex", Seq("c4", "c3"), Seq("c1", "c2", "c5")) From 12235d7c1005be656427e82259c0d94e7f181087 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 3 Aug 2020 18:25:31 -0700 Subject: [PATCH 32/33] added missing test cases --- .../hyperspace/actions/CreateActionBase.scala | 4 ++-- .../hyperspace/actions/CreateActionTest.scala | 15 +++++++++++++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 253bb2375..90cd0f79a 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -157,8 +157,8 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) .map(c => (c, ResolverUtils.resolve(spark, c, dfColumnNames))) .collect { case c if c._2.isEmpty => c._1 } throw HyperspaceException( - s"Columns $unresolvedColumns could not be resolved " + - s"from available source columns $dfColumnNames") + s"Columns '${unresolvedColumns.mkString(",")}' could not be resolved " + + s"from available source columns '${dfColumnNames.mkString(",")}'") } } } diff --git a/src/test/scala/com/microsoft/hyperspace/actions/CreateActionTest.scala b/src/test/scala/com/microsoft/hyperspace/actions/CreateActionTest.scala index afe83259c..29539415e 100644 --- a/src/test/scala/com/microsoft/hyperspace/actions/CreateActionTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/actions/CreateActionTest.scala @@ -19,6 +19,7 @@ package com.microsoft.hyperspace.actions import org.apache.hadoop.fs.Path import org.apache.spark.SparkFunSuite import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.mockito.Mockito._ import com.microsoft.hyperspace.{HyperspaceException, SampleData, SparkInvolvedSuite} @@ -26,7 +27,7 @@ import com.microsoft.hyperspace.actions.Constants.States._ import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.util.FileUtils -class CreateActionTest extends SparkFunSuite with SparkInvolvedSuite { +class CreateActionTest extends SparkFunSuite with SparkInvolvedSuite with SQLHelper { private val indexSystemPath = "src/test/resources/indexLocation" private val sampleData = SampleData.testData private val sampleParquetDataLocation = "src/test/resources/sampleparquet" @@ -99,9 +100,19 @@ class CreateActionTest extends SparkFunSuite with SparkInvolvedSuite { test("validate() fails if old index logs found with non-DOESNOTEXIST state") { when(mockLogManager.getLatestLog()).thenReturn(Some(TestLogEntry(ACTIVE))) val action = new CreateAction(spark, df, indexConfig, mockLogManager, mockDataManager) - // No exception thrown is considered a pass val ex = intercept[HyperspaceException](action.validate()) assert( ex.getMessage.contains(s"Another Index with name ${indexConfig.indexName} already exists")) } + + test("op() fails if index config is of wrong case and spark is case-sensitive") { + when(mockLogManager.getLatestLog()).thenReturn(Some(TestLogEntry(ACTIVE))) + val indexConfig = IndexConfig("index1", Seq("rgUID"), Seq("dATE")) + val action = new CreateAction(spark, df, indexConfig, mockLogManager, mockDataManager) + withSQLConf("spark.sql.caseSensitive" -> "true") { + val ex = intercept[HyperspaceException](action.op()) + assert(ex.getMessage.contains("Columns 'rgUID,dATE' could not be resolved from available " + + "source columns 'Date,RGUID,Query,imprs,clicks'")) + } + } } From e895dcd7783118288ee996b789dbd262a42adff5 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Mon, 3 Aug 2020 19:28:55 -0700 Subject: [PATCH 33/33] removed empty line --- .../com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index afc016d86..3b8f09ab7 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -176,7 +176,6 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { Seq( getIndexFilesPath(leftDfIndexConfig.indexName), getIndexFilesPath(rightDfIndexConfig.indexName))) - } test("E2E test for join query with alias columns is not supported.") {