From 4d002cd96976b8dc54c6643d61ed863f5d480797 Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Tue, 25 Aug 2020 22:07:07 -0700 Subject: [PATCH 01/86] add delete support for index refresh --- .../actions/RefreshDeleteAction.scala | 116 ++++++++++++++++++ .../index/IndexCollectionManager.scala | 8 +- .../hyperspace/index/IndexConstants.scala | 2 + .../index/RefreshIndexDeleteTests.scala | 106 ++++++++++++++++ 4 files changed, 231 insertions(+), 1 deletion(-) create mode 100644 src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala create mode 100644 src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala new file mode 100644 index 000000000..cf58d52d8 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -0,0 +1,116 @@ +/* + * 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.actions + +import scala.util.{Success, Try} + +import org.apache.commons.io.FilenameUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} +import org.apache.spark.sql.execution.datasources.{InMemoryFileIndex, PartitioningUtils} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DataType, StructType} + +import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, REFRESHING} +import com.microsoft.hyperspace.index._ +import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshActionEvent} + +class RefreshDeleteAction( + spark: SparkSession, + final override protected val logManager: IndexLogManager, + dataManager: IndexDataManager) + extends CreateActionBase(dataManager) + with Action { + private lazy val previousLogEntry: LogEntry = { + logManager.getLog(baseId).getOrElse { + throw HyperspaceException("LogEntry must exist for refresh delete operation.") + } + } + + private lazy val previousIndexLogEntry = previousLogEntry.asInstanceOf[IndexLogEntry] + + private lazy val indexConfig: IndexConfig = { + val ddColumns = previousIndexLogEntry.derivedDataset.properties.columns + IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) + } + + // Reconstruct a df from schema + private lazy val df = { + val rels = previousIndexLogEntry.relations + // Currently we only support to create an index on a LogicalRelation. + assert(rels.size == 1) + val dataSchema = DataType.fromJson(rels.head.dataSchemaJson).asInstanceOf[StructType] + spark.read + .schema(dataSchema) + .format(rels.head.fileFormat) + .options(rels.head.options) + .load(rels.head.rootPaths: _*) + } + + final override lazy val logEntry: LogEntry = + getIndexLogEntry(spark, df, indexConfig, indexDataPath) + + final override def op(): Unit = { + val diff = getDeletedFiles.map(FilenameUtils.getBaseName(_)) + if (diff.nonEmpty) { + val indexDF = spark.read.parquet(previousIndexLogEntry.content.root) + + // import spark.implicits._ + indexDF.filter(col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(diff: _*)).show() + + // PartitioningUtils.parsePathFragmentAsSeq() + } + } + + private def getDeletedFiles: Seq[String] = { + val rels = previousIndexLogEntry.relations + // Currently we only support to create an index on a LogicalRelation. + assert(rels.size == 1) + // pouriap fix this after recent checkins + // rels.head.data.properties.content.directories.flatMap(_.files) + val previousFiles = Seq[String]() + + // val fs = PathResolver(spark.sessionState.conf).systemPath.getFileSystem(new Configuration) + // val currentFile = rels.head.rootPaths + // .flatMap(path => fs.listStatus(new Path(path)).map(_.getPath.toString)) + + val currentFiles = + new InMemoryFileIndex(spark, rels.head.rootPaths.map(new Path(_)), Map(), None) + .allFiles() + .map(_.getPath.toString) + + previousFiles diff currentFiles + } + + // private def getPartitionKeyValues(path: String): (String, String) = { + // + // } + + final override val transientState: String = REFRESHING + + final override val finalState: String = ACTIVE + + final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { + // TODO Change event type + RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) + } + + // TODO Common code + // private val fileName = udf((fullFilePath: String) => FilenameUtils.getBaseName(fullFilePath)) +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index 201123cef..09ffff4ed 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -66,7 +66,13 @@ class IndexCollectionManager( withLogManager(indexName) { logManager => val indexPath = PathResolver(spark.sessionState.conf).getIndexPath(indexName) val dataManager = indexDataManagerFactory.create(indexPath) - new RefreshAction(spark, logManager, dataManager).run() + if (spark.sessionState.conf + .getConfString(IndexConstants.REFRESH_DELETE_ENABLED, "false") + .toBoolean) { + new RefreshDeleteAction(spark, logManager, dataManager).run() + } else { + new RefreshAction(spark, logManager, dataManager).run() + } } } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala index 7e288e253..e5fe74de2 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala @@ -51,4 +51,6 @@ object IndexConstants { private[hyperspace] val DATA_FILE_NAME_COLUMN = "_data_file_name" val INDEX_LINEAGE_ENABLED = "spark.hyperspace.index.lineage.enabled" val INDEX_LINEAGE_ENABLED_DEFAULT = "false" + + val REFRESH_DELETE_ENABLED = "spark.hyperspace.index.refresh.delete.enabled" } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala new file mode 100644 index 000000000..400eb1a0a --- /dev/null +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala @@ -0,0 +1,106 @@ +/* + * Copyright (2020) The Hyperspace Project Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.microsoft.hyperspace.index + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.datasources.PartitioningUtils + +import com.microsoft.hyperspace.{Hyperspace, SampleData} +import com.microsoft.hyperspace.util.FileUtils + +class RefreshIndexDeleteTests extends HyperspaceSuite { + override val systemPath = new Path("src/test/resources/indexLocation") + private val sampleData = SampleData.testData + private val sampleDataLocation = "src/test/resources/sampleParquet" + private val partitionKey1 = "Date" + private val partitionKey2 = "Query" + private val indexConfig = IndexConfig("index1", Seq("Query"), Seq("imprs")) + private var dataDF: DataFrame = _ + private var hyperspace: Hyperspace = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val sparkSession = spark + import sparkSession.implicits._ + hyperspace = new Hyperspace(sparkSession) + FileUtils.delete(new Path(sampleDataLocation)) + + // save test data partitioned. + // `Date` is the first partition key and `Query` is the second partition key. + val dfFromSample = sampleData.toDF("Date", "RGUID", "Query", "imprs", "clicks") + dfFromSample.select(partitionKey1).distinct().collect().foreach { d => + val date = d.get(0) + dfFromSample + .filter($"date" === date) + .select(partitionKey2) + .distinct() + .collect() + .foreach { q => + val query = q.get(0) + val partitionPath = + s"$sampleDataLocation/$partitionKey1=$date/$partitionKey2=$query" + dfFromSample + .filter($"date" === date && $"Query" === query) + .select("RGUID", "imprs", "clicks") + .write + .parquet(partitionPath) + } + } + dataDF = spark.read.parquet(sampleDataLocation) + } + + override def afterAll(): Unit = { + FileUtils.delete(new Path(sampleDataLocation)) + super.afterAll() + } + + after { + FileUtils.delete(systemPath) + } + + test("Validate refresh index with deleted files.") { + // scalastyle:off + + // val df = spark.read.parquet("C:\\Users\\pouriap\\Desktop\\scratch\\samplepartitionedparquet") + // df.show(false) + + val pCols = Seq("Date", "Query", "ColA", "ColB", "ColC") + + // val p = "C:/Users/pouriap/Desktop/scratch/samplepartitionedparquet/Date=2017-09-03/Query=donde" + val p = "C:/Users/pouriap/Desktop/scratch/samplepartitionedparquet/Date=2017-09-03/Query=donde/ColA=12/ColB=pirz/ColC=Hello/f1.parquet" + // val s = p.split("/").map { kv => kv.split("=", 2) } + //val p = "Date=2017-09-03/Query=donde" + //val x = PartitioningUtils.parsePathFragmentAsSeq(p) + + + val x = p.split("/").map { kv => + kv.split("=", 2) + }.filter(k => pCols.contains(k(0))).map(t => (t(0), t(1))) + x.foreach(t => println(s"${t._1} , ${t._2}")) + + // spark.conf.set(IndexConstants.REFRESH_DELETE_ENABLED, "true") + // hyperspace.createIndex(dataDF, indexConfig) + // + // FileUtils.delete(new Path(s"$sampleDataLocation/$partitionKey1=2018-09-03")) + // + // hyperspace.refreshIndex(indexConfig.indexName) + + // scalastyle:on + } +} From bf984259a331ba3ff1a409e762e7abea8dd9e78c Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Wed, 2 Sep 2020 14:44:33 -0700 Subject: [PATCH 02/86] add delete support to refresh index --- .../hyperspace/actions/CreateActionBase.scala | 11 +- .../actions/RefreshDeleteAction.scala | 65 ++++----- .../index/RefreshIndexDeleteTests.scala | 133 ++++++++++-------- 3 files changed, 112 insertions(+), 97 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index 28a6cc55b..a2ea2e2ef 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -22,7 +22,8 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} -import org.apache.spark.sql.functions.input_file_name +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions.{input_file_name, udf} import org.apache.spark.sql.sources.DataSourceRegister import com.microsoft.hyperspace.HyperspaceException @@ -131,7 +132,8 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) IndexConstants.INDEX_NUM_BUCKETS_DEFAULT.toString) .toInt - val (indexDataFrame, resolvedIndexedColumns, _) = prepareIndexDataFrame(spark, df, indexConfig) + val (indexDataFrame, resolvedIndexedColumns, _) = + prepareIndexDataFrame(spark, df, indexConfig) // run job val repartitionedIndexDataFrame = @@ -205,7 +207,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) ResolverUtils.resolve(spark, _, columnsFromIndexConfig).isEmpty) val allIndexColumns = columnsFromIndexConfig ++ missingPartitionColumns df.select(allIndexColumns.head, allIndexColumns.tail: _*) - .withColumn(IndexConstants.DATA_FILE_NAME_COLUMN, input_file_name()) + .withColumn(IndexConstants.DATA_FILE_NAME_COLUMN, absolutePath(input_file_name())) } else { df.select(columnsFromIndexConfig.head, columnsFromIndexConfig.tail: _*) } @@ -223,4 +225,7 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) assert(partitionSchemas.length == 1) partitionSchemas.head.map(_.name) } + + private val absolutePath: UserDefinedFunction = udf( + (filePath: String) => PathUtils.makeAbsolute(filePath).toString) } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index cf58d52d8..c5ce04fef 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -16,30 +16,27 @@ package com.microsoft.hyperspace.actions -import scala.util.{Success, Try} - -import org.apache.commons.io.FilenameUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} -import org.apache.spark.sql.execution.datasources.{InMemoryFileIndex, PartitioningUtils} +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, REFRESHING} import com.microsoft.hyperspace.index._ +import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshActionEvent} class RefreshDeleteAction( - spark: SparkSession, - final override protected val logManager: IndexLogManager, - dataManager: IndexDataManager) - extends CreateActionBase(dataManager) + spark: SparkSession, + final override protected val logManager: IndexLogManager, + dataManager: IndexDataManager) + extends CreateActionBase(dataManager) with Action { private lazy val previousLogEntry: LogEntry = { logManager.getLog(baseId).getOrElse { - throw HyperspaceException("LogEntry must exist for refresh delete operation.") + throw HyperspaceException("LogEntry must exist for refresh operation.") } } @@ -50,7 +47,6 @@ class RefreshDeleteAction( IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) } - // Reconstruct a df from schema private lazy val df = { val rels = previousIndexLogEntry.relations // Currently we only support to create an index on a LogicalRelation. @@ -67,14 +63,17 @@ class RefreshDeleteAction( getIndexLogEntry(spark, df, indexConfig, indexDataPath) final override def op(): Unit = { - val diff = getDeletedFiles.map(FilenameUtils.getBaseName(_)) + val diff = getDeletedFiles if (diff.nonEmpty) { val indexDF = spark.read.parquet(previousIndexLogEntry.content.root) - - // import spark.implicits._ - indexDF.filter(col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(diff: _*)).show() - - // PartitioningUtils.parsePathFragmentAsSeq() + val refreshDF = + indexDF.filter(!col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(diff: _*)) + + refreshDF.write.saveWithBuckets( + refreshDF, + indexDataPath.toString, + logEntry.asInstanceOf[IndexLogEntry].numBuckets, + indexConfig.indexedColumns) } } @@ -82,35 +81,31 @@ class RefreshDeleteAction( val rels = previousIndexLogEntry.relations // Currently we only support to create an index on a LogicalRelation. assert(rels.size == 1) - // pouriap fix this after recent checkins - // rels.head.data.properties.content.directories.flatMap(_.files) - val previousFiles = Seq[String]() - // val fs = PathResolver(spark.sessionState.conf).systemPath.getFileSystem(new Configuration) - // val currentFile = rels.head.rootPaths - // .flatMap(path => fs.listStatus(new Path(path)).map(_.getPath.toString)) + val previousFiles = + rels.head.data.properties.content.directories.flatMap(_.files).map(_.name) - val currentFiles = - new InMemoryFileIndex(spark, rels.head.rootPaths.map(new Path(_)), Map(), None) - .allFiles() - .map(_.getPath.toString) + var currentFiles = Seq[String]() + rels.head.rootPaths.foreach { r => + val path = new Path(r) + val fs = path.getFileSystem(new Configuration) + currentFiles ++= listLeafFiles(path, fs) + } previousFiles diff currentFiles } - // private def getPartitionKeyValues(path: String): (String, String) = { - // - // } - final override val transientState: String = REFRESHING final override val finalState: String = ACTIVE final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { - // TODO Change event type RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) } - // TODO Common code - // private val fileName = udf((fullFilePath: String) => FilenameUtils.getBaseName(fullFilePath)) + private def listLeafFiles(path: Path, fs: FileSystem): Seq[String] = { + val (files, directories) = fs.listStatus(path).partition(_.isFile) + files.map(_.getPath.toString) ++ + directories.flatMap(d => listLeafFiles(d.getPath, fs)) + } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala index 400eb1a0a..c475c4a5c 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala @@ -16,57 +16,51 @@ package com.microsoft.hyperspace.index +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.catalyst.plans.SQLHelper import com.microsoft.hyperspace.{Hyperspace, SampleData} import com.microsoft.hyperspace.util.FileUtils -class RefreshIndexDeleteTests extends HyperspaceSuite { +class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { override val systemPath = new Path("src/test/resources/indexLocation") - private val sampleData = SampleData.testData - private val sampleDataLocation = "src/test/resources/sampleParquet" - private val partitionKey1 = "Date" - private val partitionKey2 = "Query" + private val testDir = "src/test/resources/RefreshIndexDeleteTests/" + private val nonPartitionedDataPath = testDir + "nonpartitioned" + private val partitionedDataPath = testDir + "partitioned" private val indexConfig = IndexConfig("index1", Seq("Query"), Seq("imprs")) - private var dataDF: DataFrame = _ + private var nonPartitionedDataDF: DataFrame = _ + private var partitionedDataDF: DataFrame = _ private var hyperspace: Hyperspace = _ override def beforeAll(): Unit = { super.beforeAll() val sparkSession = spark - import sparkSession.implicits._ hyperspace = new Hyperspace(sparkSession) - FileUtils.delete(new Path(sampleDataLocation)) + FileUtils.delete(new Path(testDir)) + + // save test data non-partitioned + SampleData.save( + spark, + nonPartitionedDataPath, + Seq("Date", "RGUID", "Query", "imprs", "clicks")) + + nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) // save test data partitioned. - // `Date` is the first partition key and `Query` is the second partition key. - val dfFromSample = sampleData.toDF("Date", "RGUID", "Query", "imprs", "clicks") - dfFromSample.select(partitionKey1).distinct().collect().foreach { d => - val date = d.get(0) - dfFromSample - .filter($"date" === date) - .select(partitionKey2) - .distinct() - .collect() - .foreach { q => - val query = q.get(0) - val partitionPath = - s"$sampleDataLocation/$partitionKey1=$date/$partitionKey2=$query" - dfFromSample - .filter($"date" === date && $"Query" === query) - .select("RGUID", "imprs", "clicks") - .write - .parquet(partitionPath) - } - } - dataDF = spark.read.parquet(sampleDataLocation) + SampleData.save( + spark, + partitionedDataPath, + Seq("Date", "RGUID", "Query", "imprs", "clicks"), + Some(Seq("Date", "Query"))) + + partitionedDataDF = spark.read.parquet(partitionedDataPath) } override def afterAll(): Unit = { - FileUtils.delete(new Path(sampleDataLocation)) + FileUtils.delete(new Path(testDir)) super.afterAll() } @@ -74,33 +68,54 @@ class RefreshIndexDeleteTests extends HyperspaceSuite { FileUtils.delete(systemPath) } - test("Validate refresh index with deleted files.") { - // scalastyle:off - - // val df = spark.read.parquet("C:\\Users\\pouriap\\Desktop\\scratch\\samplepartitionedparquet") - // df.show(false) - - val pCols = Seq("Date", "Query", "ColA", "ColB", "ColC") - - // val p = "C:/Users/pouriap/Desktop/scratch/samplepartitionedparquet/Date=2017-09-03/Query=donde" - val p = "C:/Users/pouriap/Desktop/scratch/samplepartitionedparquet/Date=2017-09-03/Query=donde/ColA=12/ColB=pirz/ColC=Hello/f1.parquet" - // val s = p.split("/").map { kv => kv.split("=", 2) } - //val p = "Date=2017-09-03/Query=donde" - //val x = PartitioningUtils.parsePathFragmentAsSeq(p) - - - val x = p.split("/").map { kv => - kv.split("=", 2) - }.filter(k => pCols.contains(k(0))).map(t => (t(0), t(1))) - x.foreach(t => println(s"${t._1} , ${t._2}")) - - // spark.conf.set(IndexConstants.REFRESH_DELETE_ENABLED, "true") - // hyperspace.createIndex(dataDF, indexConfig) - // - // FileUtils.delete(new Path(s"$sampleDataLocation/$partitionKey1=2018-09-03")) - // - // hyperspace.refreshIndex(indexConfig.indexName) - - // scalastyle:on + test("validate refresg index when some file gets deleted.") { + Seq(nonPartitionedDataPath, partitionedDataPath).foreach { loc => + withSQLConf( + IndexConstants.INDEX_LINEAGE_ENABLED -> "true", + IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withIndex(indexConfig.indexName) { + val dfToIndex = + if (loc.equals(nonPartitionedDataPath)) nonPartitionedDataDF else partitionedDataDF + hyperspace.createIndex(dfToIndex, indexConfig) + + // delete some source data file. + val dataPath = + if (loc.equals(nonPartitionedDataPath)) new Path(nonPartitionedDataPath, "*parquet") + else new Path(partitionedDataPath + "/Date=2018-09-03/Query=ibraco", "*parquet") + + val dataFileNames = dataPath + .getFileSystem(new Configuration) + .globStatus(dataPath) + .map(_.getPath) + + assert(dataFileNames.length > 0) + val fileToDelete = dataFileNames.head + FileUtils.delete(fileToDelete) + + val originalIndexDF = spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") + val delCount = + originalIndexDF + .filter(s"""${IndexConstants.DATA_FILE_NAME_COLUMN} == "$fileToDelete"""") + .count() + + hyperspace.refreshIndex(indexConfig.indexName) + + val refreshedIndexDF = spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + + // validate only index records whose lineage is the deleted file are removed. + assert(refreshedIndexDF.count() == (originalIndexDF.count() - delCount)) + + val lineageFileNames = refreshedIndexDF + .select(IndexConstants.DATA_FILE_NAME_COLUMN) + .distinct() + .collect() + .map(r => new Path(r.getString(0))) + + assert(!lineageFileNames.contains(fileToDelete)) + } + } + } } } From fcb4f956e666e014b2eb039646f5e37f79981d24 Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Wed, 2 Sep 2020 14:55:05 -0700 Subject: [PATCH 03/86] add delete support to refresh index --- .../actions/RefreshDeleteAction.scala | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index c5ce04fef..6fd725f63 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -63,18 +63,15 @@ class RefreshDeleteAction( getIndexLogEntry(spark, df, indexConfig, indexDataPath) final override def op(): Unit = { - val diff = getDeletedFiles - if (diff.nonEmpty) { - val indexDF = spark.read.parquet(previousIndexLogEntry.content.root) - val refreshDF = - indexDF.filter(!col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(diff: _*)) - - refreshDF.write.saveWithBuckets( - refreshDF, - indexDataPath.toString, - logEntry.asInstanceOf[IndexLogEntry].numBuckets, - indexConfig.indexedColumns) - } + val indexDF = spark.read.parquet(previousIndexLogEntry.content.root) + val refreshDF = + indexDF.filter(!col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(getDeletedFiles: _*)) + + refreshDF.write.saveWithBuckets( + refreshDF, + indexDataPath.toString, + logEntry.asInstanceOf[IndexLogEntry].numBuckets, + indexConfig.indexedColumns) } private def getDeletedFiles: Seq[String] = { @@ -82,7 +79,7 @@ class RefreshDeleteAction( // Currently we only support to create an index on a LogicalRelation. assert(rels.size == 1) - val previousFiles = + val originalFiles = rels.head.data.properties.content.directories.flatMap(_.files).map(_.name) var currentFiles = Seq[String]() @@ -92,7 +89,7 @@ class RefreshDeleteAction( currentFiles ++= listLeafFiles(path, fs) } - previousFiles diff currentFiles + originalFiles diff currentFiles } final override val transientState: String = REFRESHING From ba55f30e54ebe172733e0f23d43645ff5e806ad0 Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Wed, 2 Sep 2020 14:57:47 -0700 Subject: [PATCH 04/86] add delete support to refresh index --- .../hyperspace/actions/RefreshDeleteAction.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index 6fd725f63..c5f63c4c1 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -74,6 +74,14 @@ class RefreshDeleteAction( indexConfig.indexedColumns) } + final override val transientState: String = REFRESHING + + final override val finalState: String = ACTIVE + + final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { + RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) + } + private def getDeletedFiles: Seq[String] = { val rels = previousIndexLogEntry.relations // Currently we only support to create an index on a LogicalRelation. @@ -92,14 +100,6 @@ class RefreshDeleteAction( originalFiles diff currentFiles } - final override val transientState: String = REFRESHING - - final override val finalState: String = ACTIVE - - final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { - RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) - } - private def listLeafFiles(path: Path, fs: FileSystem): Seq[String] = { val (files, directories) = fs.listStatus(path).partition(_.isFile) files.map(_.getPath.toString) ++ From a15b71d63e9494d5e9a4e0ba9c7688b73f93f0f9 Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Wed, 2 Sep 2020 14:59:43 -0700 Subject: [PATCH 05/86] add delete support to refresh index --- .../microsoft/hyperspace/index/RefreshIndexDeleteTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala index c475c4a5c..4c294d1b4 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala @@ -41,7 +41,7 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { hyperspace = new Hyperspace(sparkSession) FileUtils.delete(new Path(testDir)) - // save test data non-partitioned + // save test data non-partitioned. SampleData.save( spark, nonPartitionedDataPath, From 70d941619b4e35115065edb0df24e775576ecdda Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Thu, 3 Sep 2020 09:56:56 -0700 Subject: [PATCH 06/86] fix index manager test case --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 2677b2035..003e2d18a 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -69,7 +69,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { StructType(Seq(StructField("RGUID", StringType), StructField("Date", StringType))) if (enableLineage) { expectedSchema = expectedSchema.add( - StructField(IndexConstants.DATA_FILE_NAME_COLUMN, StringType, false)) + StructField(IndexConstants.DATA_FILE_NAME_COLUMN, StringType)) } val expected = new IndexSummary( indexConfig1.indexName, From 0b26209c339c6d3dc69bbd663b8dd6b7de37d85f Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Thu, 3 Sep 2020 11:41:07 -0700 Subject: [PATCH 07/86] fix index content refresh --- .../com/microsoft/hyperspace/actions/RefreshDeleteAction.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index c5f63c4c1..75e643513 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -59,8 +59,7 @@ class RefreshDeleteAction( .load(rels.head.rootPaths: _*) } - final override lazy val logEntry: LogEntry = - getIndexLogEntry(spark, df, indexConfig, indexDataPath) + final override def logEntry: LogEntry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) final override def op(): Unit = { val indexDF = spark.read.parquet(previousIndexLogEntry.content.root) From 66fa8fb70f238e7e37e6fcfc019be3d6e0057d10 Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Wed, 9 Sep 2020 17:27:35 -0700 Subject: [PATCH 08/86] fix merge conflicts --- .../hyperspace/actions/Constants.scala | 6 +++++ .../actions/RefreshDeleteAction.scala | 24 +++++++------------ 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala b/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala index c2a0cb34d..8c1423dd5 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala @@ -30,4 +30,10 @@ object Constants { } val STABLE_STATES: Set[String] = Set(States.ACTIVE, States.DELETED, States.DOESNOTEXIST) + + object RefreshModes { + val FULL = "FULL" + val QUICK = "QUICK" + val SMART = "SMART" + } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index 75e643513..1c1b6f4e7 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -16,8 +16,7 @@ package com.microsoft.hyperspace.actions -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} @@ -62,7 +61,7 @@ class RefreshDeleteAction( final override def logEntry: LogEntry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) final override def op(): Unit = { - val indexDF = spark.read.parquet(previousIndexLogEntry.content.root) + val indexDF = spark.read.parquet(previousIndexLogEntry.content.files.map(_.toString): _*) val refreshDF = indexDF.filter(!col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(getDeletedFiles: _*)) @@ -86,22 +85,15 @@ class RefreshDeleteAction( // Currently we only support to create an index on a LogicalRelation. assert(rels.size == 1) - val originalFiles = - rels.head.data.properties.content.directories.flatMap(_.files).map(_.name) - + val originalFiles = rels.head.data.properties.content.files.map(_.toString) var currentFiles = Seq[String]() - rels.head.rootPaths.foreach { r => - val path = new Path(r) - val fs = path.getFileSystem(new Configuration) - currentFiles ++= listLeafFiles(path, fs) + rels.head.rootPaths.foreach { p => + currentFiles ++= Content + .fromDirectory(path = new Path(p)) + .files + .map(_.toString) } originalFiles diff currentFiles } - - private def listLeafFiles(path: Path, fs: FileSystem): Seq[String] = { - val (files, directories) = fs.listStatus(path).partition(_.isFile) - files.map(_.getPath.toString) ++ - directories.flatMap(d => listLeafFiles(d.getPath, fs)) - } } From 2a62d1d8ea0fb6b6d8ddab83331cb46153a3d7ff Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Thu, 10 Sep 2020 11:40:36 -0700 Subject: [PATCH 09/86] refactor refresh code and add refresh delete --- .../hyperspace/actions/Constants.scala | 6 -- .../hyperspace/actions/RefreshAction.scala | 55 +------------ .../actions/RefreshActionBase.scala | 79 +++++++++++++++++++ .../actions/RefreshDeleteAction.scala | 43 +--------- 4 files changed, 83 insertions(+), 100 deletions(-) create mode 100644 src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala diff --git a/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala b/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala index 8c1423dd5..c2a0cb34d 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/Constants.scala @@ -30,10 +30,4 @@ object Constants { } val STABLE_STATES: Set[String] = Set(States.ACTIVE, States.DELETED, States.DOESNOTEXIST) - - object RefreshModes { - val FULL = "FULL" - val QUICK = "QUICK" - val SMART = "SMART" - } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala index 6b0086cbb..9eb730f8f 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala @@ -17,61 +17,14 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.types.{DataType, StructType} -import com.microsoft.hyperspace.HyperspaceException -import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, REFRESHING} import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshActionEvent} -// TODO: This class depends directly on LogEntry. This should be updated such that -// it works with IndexLogEntry only. (for example, this class can take in -// derivedDataset specific logic for refreshing). class RefreshAction( spark: SparkSession, - final override protected val logManager: IndexLogManager, + logManager: IndexLogManager, dataManager: IndexDataManager) - extends CreateActionBase(dataManager) - with Action { - private lazy val previousLogEntry: LogEntry = { - logManager.getLog(baseId).getOrElse { - throw HyperspaceException("LogEntry must exist for refresh operation") - } - } - - private lazy val previousIndexLogEntry = previousLogEntry.asInstanceOf[IndexLogEntry] - - // Reconstruct a df from schema - private lazy val df = { - val rels = previousIndexLogEntry.relations - // Currently we only support to create an index on a LogicalRelation. - assert(rels.size == 1) - val dataSchema = DataType.fromJson(rels.head.dataSchemaJson).asInstanceOf[StructType] - spark.read - .schema(dataSchema) - .format(rels.head.fileFormat) - .options(rels.head.options) - .load(rels.head.rootPaths: _*) - } - - private lazy val indexConfig: IndexConfig = { - val ddColumns = previousIndexLogEntry.derivedDataset.properties.columns - IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) - } - - final override def logEntry: LogEntry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) - - final override val transientState: String = REFRESHING - - final override val finalState: String = ACTIVE - - final override def validate(): Unit = { - if (!previousIndexLogEntry.state.equalsIgnoreCase(ACTIVE)) { - throw HyperspaceException( - s"Refresh is only supported in $ACTIVE state. " + - s"Current index state is ${previousIndexLogEntry.state}") - } - } + extends RefreshActionBase(spark, logManager, dataManager) { final override def op(): Unit = { // TODO: The current implementation picks the number of buckets from session config. @@ -79,8 +32,4 @@ class RefreshAction( // in the index log entry. write(spark, df, indexConfig) } - - final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { - RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) - } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala new file mode 100644 index 000000000..31fefed46 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala @@ -0,0 +1,79 @@ +/* + * 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.actions + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.{DataType, StructType} + +import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, REFRESHING} +import com.microsoft.hyperspace.index._ +import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshActionEvent} + +// TODO: This class depends directly on LogEntry. This should be updated such that +// it works with IndexLogEntry only. (for example, this class can take in +// derivedDataset specific logic for refreshing). +private[actions] abstract class RefreshActionBase( + spark: SparkSession, + final override protected val logManager: IndexLogManager, + dataManager: IndexDataManager) + extends CreateActionBase(dataManager) + with Action { + private lazy val previousLogEntry: LogEntry = { + logManager.getLog(baseId).getOrElse { + throw HyperspaceException("LogEntry must exist for refresh operation") + } + } + + protected lazy val previousIndexLogEntry = previousLogEntry.asInstanceOf[IndexLogEntry] + + // Reconstruct a df from schema + protected lazy val df = { + val rels = previousIndexLogEntry.relations + // Currently we only support to create an index on a LogicalRelation. + assert(rels.size == 1) + val dataSchema = DataType.fromJson(rels.head.dataSchemaJson).asInstanceOf[StructType] + spark.read + .schema(dataSchema) + .format(rels.head.fileFormat) + .options(rels.head.options) + .load(rels.head.rootPaths: _*) + } + + protected lazy val indexConfig: IndexConfig = { + val ddColumns = previousIndexLogEntry.derivedDataset.properties.columns + IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) + } + + final override def logEntry: LogEntry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) + + final override val transientState: String = REFRESHING + + final override val finalState: String = ACTIVE + + final override def validate(): Unit = { + if (!previousIndexLogEntry.state.equalsIgnoreCase(ACTIVE)) { + throw HyperspaceException( + s"Refresh is only supported in $ACTIVE state. " + + s"Current index state is ${previousIndexLogEntry.state}") + } + } + + final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { + RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) + } +} diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index 1c1b6f4e7..feb04febb 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -19,46 +19,15 @@ package com.microsoft.hyperspace.actions import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DataType, StructType} -import com.microsoft.hyperspace.HyperspaceException -import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, REFRESHING} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer -import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshActionEvent} class RefreshDeleteAction( spark: SparkSession, - final override protected val logManager: IndexLogManager, + logManager: IndexLogManager, dataManager: IndexDataManager) - extends CreateActionBase(dataManager) - with Action { - private lazy val previousLogEntry: LogEntry = { - logManager.getLog(baseId).getOrElse { - throw HyperspaceException("LogEntry must exist for refresh operation.") - } - } - - private lazy val previousIndexLogEntry = previousLogEntry.asInstanceOf[IndexLogEntry] - - private lazy val indexConfig: IndexConfig = { - val ddColumns = previousIndexLogEntry.derivedDataset.properties.columns - IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) - } - - private lazy val df = { - val rels = previousIndexLogEntry.relations - // Currently we only support to create an index on a LogicalRelation. - assert(rels.size == 1) - val dataSchema = DataType.fromJson(rels.head.dataSchemaJson).asInstanceOf[StructType] - spark.read - .schema(dataSchema) - .format(rels.head.fileFormat) - .options(rels.head.options) - .load(rels.head.rootPaths: _*) - } - - final override def logEntry: LogEntry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) + extends RefreshActionBase(spark, logManager, dataManager) { final override def op(): Unit = { val indexDF = spark.read.parquet(previousIndexLogEntry.content.files.map(_.toString): _*) @@ -72,14 +41,6 @@ class RefreshDeleteAction( indexConfig.indexedColumns) } - final override val transientState: String = REFRESHING - - final override val finalState: String = ACTIVE - - final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { - RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) - } - private def getDeletedFiles: Seq[String] = { val rels = previousIndexLogEntry.relations // Currently we only support to create an index on a LogicalRelation. From 9916c2588b81cdaaff2a66b3ae192b3dd444e288 Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Thu, 10 Sep 2020 12:19:16 -0700 Subject: [PATCH 10/86] check for lineage during refresh delete. --- .../actions/RefreshDeleteAction.scala | 26 +++++++++++++------ .../index/RefreshIndexDeleteTests.scala | 15 +++++++++-- 2 files changed, 31 insertions(+), 10 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index feb04febb..c07427fc4 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -20,8 +20,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer +import com.microsoft.hyperspace.util.ResolverUtils class RefreshDeleteAction( spark: SparkSession, @@ -31,14 +33,22 @@ class RefreshDeleteAction( final override def op(): Unit = { val indexDF = spark.read.parquet(previousIndexLogEntry.content.files.map(_.toString): _*) - val refreshDF = - indexDF.filter(!col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(getDeletedFiles: _*)) - - refreshDF.write.saveWithBuckets( - refreshDF, - indexDataPath.toString, - logEntry.asInstanceOf[IndexLogEntry].numBuckets, - indexConfig.indexedColumns) + + ResolverUtils + .resolve(spark, IndexConstants.DATA_FILE_NAME_COLUMN, indexDF.schema.fieldNames) match { + case Some(_) => + val refreshDF = + indexDF.filter(!col(s"${IndexConstants.DATA_FILE_NAME_COLUMN}").isin(getDeletedFiles: _*)) + + refreshDF.write.saveWithBuckets( + refreshDF, + indexDataPath.toString, + logEntry.asInstanceOf[IndexLogEntry].numBuckets, + indexConfig.indexedColumns) + + case None => + throw HyperspaceException(s"Refresh delete is only supported on an index with lineage.") + } } private def getDeletedFiles: Seq[String] = { diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala index 4c294d1b4..bf1e7fbea 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala @@ -21,7 +21,7 @@ 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, SampleData} +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} import com.microsoft.hyperspace.util.FileUtils class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { @@ -68,7 +68,7 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { FileUtils.delete(systemPath) } - test("validate refresg index when some file gets deleted.") { + test("Validate refresh index when some file gets deleted.") { Seq(nonPartitionedDataPath, partitionedDataPath).foreach { loc => withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "true", @@ -118,4 +118,15 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { } } } + + test("Validate refresh delete fails as expected on an index without lineage.") { + withSQLConf( + IndexConstants.INDEX_LINEAGE_ENABLED -> "false", + IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + hyperspace.createIndex(nonPartitionedDataDF, indexConfig) + + val ex = intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) + assert(ex.getMessage.contains("Refresh delete is only supported on an index with lineage.")) + } + } } From d808a13b3c85b83b8a52984ee13fbf0f84f0a661 Mon Sep 17 00:00:00 2001 From: Pouria Pirzadeh Date: Thu, 10 Sep 2020 13:41:27 -0700 Subject: [PATCH 11/86] fix refresh delete test --- .../index/RefreshIndexDeleteTests.scala | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala index bf1e7fbea..daed66ef0 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala @@ -18,7 +18,6 @@ package com.microsoft.hyperspace.index import org.apache.hadoop.conf.Configuration 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} @@ -30,8 +29,6 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { private val nonPartitionedDataPath = testDir + "nonpartitioned" private val partitionedDataPath = testDir + "partitioned" private val indexConfig = IndexConfig("index1", Seq("Query"), Seq("imprs")) - private var nonPartitionedDataDF: DataFrame = _ - private var partitionedDataDF: DataFrame = _ private var hyperspace: Hyperspace = _ override def beforeAll(): Unit = { @@ -40,14 +37,25 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { val sparkSession = spark hyperspace = new Hyperspace(sparkSession) FileUtils.delete(new Path(testDir)) + } + + override def afterAll(): Unit = { + FileUtils.delete(new Path(testDir)) + super.afterAll() + } + after { + FileUtils.delete(new Path(testDir)) + FileUtils.delete(systemPath) + } + + test("Validate refresh index when some file gets deleted.") { // save test data non-partitioned. SampleData.save( spark, nonPartitionedDataPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) - - nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) + val nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) // save test data partitioned. SampleData.save( @@ -55,20 +63,8 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { partitionedDataPath, Seq("Date", "RGUID", "Query", "imprs", "clicks"), Some(Seq("Date", "Query"))) + val partitionedDataDF = spark.read.parquet(partitionedDataPath) - partitionedDataDF = spark.read.parquet(partitionedDataPath) - } - - override def afterAll(): Unit = { - FileUtils.delete(new Path(testDir)) - super.afterAll() - } - - after { - FileUtils.delete(systemPath) - } - - test("Validate refresh index when some file gets deleted.") { Seq(nonPartitionedDataPath, partitionedDataPath).foreach { loc => withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "true", @@ -120,6 +116,12 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { } test("Validate refresh delete fails as expected on an index without lineage.") { + SampleData.save( + spark, + nonPartitionedDataPath, + Seq("Date", "RGUID", "Query", "imprs", "clicks")) + val nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) + withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "false", IndexConstants.REFRESH_DELETE_ENABLED -> "true") { From 7e516df1c837bce30595ceee2bf0404937fcc57d Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 14 Sep 2020 15:00:17 -0700 Subject: [PATCH 12/86] add merge functionality with test --- .../hyperspace/index/IndexLogEntry.scala | 28 ++++++++++++++++++- .../hyperspace/index/IndexLogEntryTest.scala | 28 +++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index fc4b19758..e68036d98 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.spark.sql.types.{DataType, StructType} +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.util.PathUtils @@ -83,7 +84,32 @@ object Content { def fromLeafFiles(files: Seq[FileStatus]): Content = Content(Directory.fromLeafFiles(files)) } -case class Directory(name: String, files: Seq[FileInfo] = Seq(), subDirs: Seq[Directory] = Seq()) +case class Directory( + name: String, + files: Seq[FileInfo] = Seq(), + subDirs: Seq[Directory] = Seq()) { + def merge(that: Directory): Directory = { + if (name.equals(that.name)) { + val allFiles = files ++ that.files + val subDirMap = subDirs.map(dir => dir.name -> dir).toMap + val thatSubDirMap = that.subDirs.map(dir => dir.name -> dir).toMap + val subDir: Seq[Directory] = (subDirMap.keySet ++ thatSubDirMap.keySet).toSeq.map { + dirName => + if (subDirMap.contains(dirName) && thatSubDirMap.contains(dirName)) { + // If both directories contain a subDir with same name, merge subDirs respectively. + subDirMap(dirName).merge(thatSubDirMap(dirName)) + } else { + // Pick the subDir from whoever contains it. + subDirMap.getOrElse(dirName, thatSubDirMap(dirName)) + } + } + + Directory(name, allFiles, subDirs = subDir) + } else { + throw HyperspaceException("Directory names should match for merging Directories.") + } + } +} object Directory { diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala index a237db925..ce195fc78 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala @@ -439,6 +439,34 @@ class IndexLogEntryTest extends SparkFunSuite with SQLHelper with BeforeAndAfter FileUtils.deleteDirectory(tempDir.toFile) } + test("Directory.merge Test: test if merge works as expected.") { + // File Structure + // testDir/a/f1 + // testDir/a/f2 + // testDir/a/b/f3 + // testDir/a/b/f4 + + val tempDir = Files.createDirectories(Paths.get(testDir + "/temp")) + val a = Files.createDirectories(Paths.get(tempDir + "/a")) + val b = Files.createDirectories(Paths.get(a + "/b")) + val f1 = Files.createFile(Paths.get(a + "/f1")) + val f2 = Files.createFile(Paths.get(a + "/f2")) + val f3 = Files.createFile(Paths.get(b + "/f3")) + val f4 = Files.createFile(Paths.get(b + "/f4")) + + val aDirectory = Directory.fromDirectory(toPath(a)) + val bDirectory = Directory.fromDirectory(toPath(b)) + + val expected = Directory.fromLeafFiles(Seq(f1, f2, f3, f4).map(toFileStatus)) + val actual1 = aDirectory.merge(bDirectory) + val actual2 = bDirectory.merge(aDirectory) + + assert(directoryEquals(actual1, expected)) + assert(directoryEquals(actual2, expected)) + + FileUtils.deleteDirectory(tempDir.toFile) + } + private def contentEquals(content1: Content, content2: Content): Boolean = { directoryEquals(content1.root, content2.root) } From ed6d49c793a9f743c154ef3f67a85d1466f75937 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 14 Sep 2020 15:00:56 -0700 Subject: [PATCH 13/86] initial commit for refresh incremental support (append only) --- .../actions/RefreshActionBase.scala | 2 +- .../actions/RefreshIncremental.scala | 94 +++++++++++++++ .../actions/RefreshIncrementalAction.scala | 110 ++++++++++++++++++ .../index/IndexCollectionManager.scala | 4 + .../hyperspace/index/IndexConstants.scala | 1 + .../hyperspace/index/IndexManagerTests.scala | 61 ++++++++++ 6 files changed, 271 insertions(+), 1 deletion(-) create mode 100644 src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala create mode 100644 src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala index 31fefed46..d51029de7 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala @@ -59,7 +59,7 @@ private[actions] abstract class RefreshActionBase( IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) } - final override def logEntry: LogEntry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) + override def logEntry: LogEntry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) final override val transientState: String = REFRESHING diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala new file mode 100644 index 000000000..2a2d1cd9e --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala @@ -0,0 +1,94 @@ +/* + * 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.actions + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DataType, StructType} + +import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.index._ +import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer +import com.microsoft.hyperspace.util.ResolverUtils + +class RefreshIncremental( + spark: SparkSession, + logManager: IndexLogManager, + dataManager: IndexDataManager) + extends RefreshActionBase(spark, logManager, dataManager) { + final override def op(): Unit = { + // TODO: The current implementation picks the number of buckets from session config. + // This should be user-configurable to allow maintain the existing bucket numbers + // in the index log entry. + write(spark, indexableDf, indexConfig) + } + + private lazy val indexableDf = { + // Currently we only support to create an index on a LogicalRelation. + assert(previousIndexLogEntry.relations.size == 1) + val relation = previousIndexLogEntry.relations.head + + // TODO: improve this to take last modified time of files into account. + val indexedFiles = relation.data.properties.content.files.map(_.toString) + + /* + // Option 1 to create indexable df: + // To remove pre-indexed files, add file name column, add file filter and remove file name + // column. + val predicate = indexedFiles.mkString("('", "','", "')") + val temporaryColumn = "_file_name" + return df.withColumn(temporaryColumn, input_file_name) + .where(s"$temporaryColumn not in $predicate") + .drop("_file_name") + */ + + // Option 2 to create indexableDf + val allFiles = df.queryExecution.optimizedPlan.collect { + case LogicalRelation( + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), + _, + _, + _) => + location.allFiles().map(_.getPath.toString) + }.flatten + + val dataSchema = DataType.fromJson(relation.dataSchemaJson).asInstanceOf[StructType] + + // Create a df with only diff files from original list of files + spark.read + .schema(dataSchema) + .format(relation.fileFormat) + .options(relation.options) + .load(allFiles.diff(indexedFiles): _*) + } + + /** + * Create a logEntry with all data files, and index content merged with previous index content. + * @return Merged index log entry. This contains ALL data files (files which were indexed + * previously, and files which are being indexed in this operation). Also contains ALL + * index files (index files for previously indexed data as well as newly created files). + */ + override def logEntry: LogEntry = { + val entry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) + val mergedContent = Content( + previousIndexLogEntry.content.root.merge(entry.content.root) + ) + entry.copy(content = mergedContent) + } +} diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala new file mode 100644 index 000000000..1b5c1d683 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala @@ -0,0 +1,110 @@ +/* + * 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.actions + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions.input_file_name +import org.apache.spark.sql.types.{DataType, StructType} + +import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, REFRESHING} +import com.microsoft.hyperspace.index._ +import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshActionEvent} + +// TODO: This class depends directly on LogEntry. This should be updated such that +// it works with IndexLogEntry only. (for example, this class can take in +// derivedDataset specific logic for refreshing). +class RefreshIncrementalAction( + spark: SparkSession, + final override protected val logManager: IndexLogManager, + dataManager: IndexDataManager) + extends CreateActionBase(dataManager) + with Action { + private lazy val previousLogEntry: LogEntry = { + logManager.getLog(baseId).getOrElse { + throw HyperspaceException("LogEntry must exist for refresh operation") + } + } + + private lazy val previousIndexLogEntry = previousLogEntry.asInstanceOf[IndexLogEntry] + + // Reconstruct a df from schema. Remove pre-indexed data files. + private lazy val df = { + // Currently we only support to create an index on a LogicalRelation. + assert(previousIndexLogEntry.relations.size == 1) + val relation = previousIndexLogEntry.relations.head + val dataSchema = DataType.fromJson(relation.dataSchemaJson).asInstanceOf[StructType] + + spark.read + .schema(dataSchema) + .format(relation.fileFormat) + .options(relation.options) + .load(relation.rootPaths: _*) + } + + private lazy val indexableDf = { + // Currently we only support to create an index on a LogicalRelation. + assert(previousIndexLogEntry.relations.size == 1) + val relation = previousIndexLogEntry.relations.head + + // TODO: improve this to take last modified time of files into account. + val indexedFiles = relation.data.properties.content.files + val predicate = indexedFiles.mkString("('", "','", "')") + + // To remove pre-indexed files, add file name column, add file filter and remove file name + // column. + val temporaryColumn = "_file_name" + df.withColumn(temporaryColumn, input_file_name) + .where(s"$temporaryColumn not in $predicate") + .drop("_file_name") + } + + private lazy val indexConfig: IndexConfig = { + val ddColumns = previousIndexLogEntry.derivedDataset.properties.columns + IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) + } + + final override def logEntry: LogEntry = + getIndexLogEntry( + spark, + df, + indexConfig, + indexDataPath) + + final override val transientState: String = REFRESHING + + final override val finalState: String = ACTIVE + + final override def validate(): Unit = { + if (!previousIndexLogEntry.state.equalsIgnoreCase(ACTIVE)) { + throw HyperspaceException( + s"Refresh is only supported in $ACTIVE state. " + + s"Current index state is ${previousIndexLogEntry.state}") + } + } + + final override def op(): Unit = { + // TODO: The current implementation picks the number of buckets from session config. + // This should be user-configurable to allow maintain the existing bucket numbers + // in the index log entry. + write(spark, indexableDf, indexConfig) + } + + final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { + RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) + } +} diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index ca03cde8f..332a4d3bf 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -70,6 +70,10 @@ class IndexCollectionManager( .getConfString(IndexConstants.REFRESH_DELETE_ENABLED, "false") .toBoolean) { new RefreshDeleteAction(spark, logManager, dataManager).run() + } else if (spark.sessionState.conf + .getConfString(IndexConstants.REFRESH_APPEND_ENABLED, "false") + .toBoolean) { + new RefreshIncremental(spark, logManager, dataManager).run() } else { new RefreshAction(spark, logManager, dataManager).run() } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala index 4860984fe..15c757838 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala @@ -56,4 +56,5 @@ object IndexConstants { val INDEX_LINEAGE_ENABLED_DEFAULT = "false" val REFRESH_DELETE_ENABLED = "spark.hyperspace.index.refresh.delete.enabled" + val REFRESH_APPEND_ENABLED = "spark.hyperspace.index.refresh.append.enabled" } diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 8ce970cbf..0c9a93a70 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -251,6 +251,67 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } } + test("Verify refresh-incremental rebuild of index.") { + Seq(("csv", Map("header" -> "true")), ("parquet", Map()), ("json", Map())).foreach { + case (format, option: Map[String, String]) => + spark.conf.set(IndexConstants.REFRESH_APPEND_ENABLED, true) + val refreshTestLocation = sampleParquetDataLocation + "refresh_" + format + FileUtils.delete(new Path(refreshTestLocation)) + val indexConfig = IndexConfig(s"index_$format", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .options(option) + .format(format) + .save(refreshTestLocation) + val df = spark.read.format(format).options(option).load(refreshTestLocation) + hyperspace.createIndex(df, indexConfig) + var indexCount = + spark.read + .parquet(s"$systemPath/index_$format" + + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") + .count() + assert(indexCount == 10) + + // Change Original Data + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(3) + .write + .mode("append") + .options(option) + .format(format) + .save(refreshTestLocation) + hyperspace.refreshIndex(indexConfig.indexName) + val newIndexLocation = s"$systemPath/index_$format" + indexCount = spark.read + .parquet(newIndexLocation + + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + .count() + + // Check if index got updated + assert(indexCount == 3) + + // Check if lastest log file is updated with newly created index files + val indexPath = PathUtils.makeAbsolute(newIndexLocation) + val logManager = IndexLogManagerFactoryImpl.create(indexPath) + val latestLog = logManager.getLatestLog() + assert(latestLog.isDefined && latestLog.get.isInstanceOf[IndexLogEntry]) + val indexLog = latestLog.get.asInstanceOf[IndexLogEntry] + val indexFiles = indexLog.content.files + assert(indexFiles.nonEmpty) + assert(indexFiles.forall(_.getName.startsWith("part-0"))) + assert(indexLog.state.equals("ACTIVE")) + // Check there exist some files from v__=0 and some from v__=1 + assert(indexFiles.map(_.getParent.getName).toSet.equals(Set("v__=0", "v__=1"))) + + FileUtils.delete(new Path(refreshTestLocation)) + case _ => fail("invalid test") + } + } + private def expectedIndex( indexConfig: IndexConfig, schema: StructType, From 2308ca9e41068b87162927fadd9aab27847ee8b4 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Mon, 14 Sep 2020 21:23:02 -0700 Subject: [PATCH 14/86] comment fix Co-authored-by: Rahul Potharaju --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 0c9a93a70..b6dfcae1c 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -294,7 +294,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { // Check if index got updated assert(indexCount == 3) - // Check if lastest log file is updated with newly created index files + // Check if latest log file is updated with newly created index files val indexPath = PathUtils.makeAbsolute(newIndexLocation) val logManager = IndexLogManagerFactoryImpl.create(indexPath) val latestLog = logManager.getLatestLog() From db46723a99ccc9d284173ed9ed7faf49ff133ce2 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Tue, 15 Sep 2020 12:01:10 -0700 Subject: [PATCH 15/86] Delete RefreshIncrementalAction.scala RefreshIncremental.scala is for the same purpose. --- .../actions/RefreshIncrementalAction.scala | 110 ------------------ 1 file changed, 110 deletions(-) delete mode 100644 src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala deleted file mode 100644 index 1b5c1d683..000000000 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala +++ /dev/null @@ -1,110 +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.actions - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.functions.input_file_name -import org.apache.spark.sql.types.{DataType, StructType} - -import com.microsoft.hyperspace.HyperspaceException -import com.microsoft.hyperspace.actions.Constants.States.{ACTIVE, REFRESHING} -import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshActionEvent} - -// TODO: This class depends directly on LogEntry. This should be updated such that -// it works with IndexLogEntry only. (for example, this class can take in -// derivedDataset specific logic for refreshing). -class RefreshIncrementalAction( - spark: SparkSession, - final override protected val logManager: IndexLogManager, - dataManager: IndexDataManager) - extends CreateActionBase(dataManager) - with Action { - private lazy val previousLogEntry: LogEntry = { - logManager.getLog(baseId).getOrElse { - throw HyperspaceException("LogEntry must exist for refresh operation") - } - } - - private lazy val previousIndexLogEntry = previousLogEntry.asInstanceOf[IndexLogEntry] - - // Reconstruct a df from schema. Remove pre-indexed data files. - private lazy val df = { - // Currently we only support to create an index on a LogicalRelation. - assert(previousIndexLogEntry.relations.size == 1) - val relation = previousIndexLogEntry.relations.head - val dataSchema = DataType.fromJson(relation.dataSchemaJson).asInstanceOf[StructType] - - spark.read - .schema(dataSchema) - .format(relation.fileFormat) - .options(relation.options) - .load(relation.rootPaths: _*) - } - - private lazy val indexableDf = { - // Currently we only support to create an index on a LogicalRelation. - assert(previousIndexLogEntry.relations.size == 1) - val relation = previousIndexLogEntry.relations.head - - // TODO: improve this to take last modified time of files into account. - val indexedFiles = relation.data.properties.content.files - val predicate = indexedFiles.mkString("('", "','", "')") - - // To remove pre-indexed files, add file name column, add file filter and remove file name - // column. - val temporaryColumn = "_file_name" - df.withColumn(temporaryColumn, input_file_name) - .where(s"$temporaryColumn not in $predicate") - .drop("_file_name") - } - - private lazy val indexConfig: IndexConfig = { - val ddColumns = previousIndexLogEntry.derivedDataset.properties.columns - IndexConfig(previousIndexLogEntry.name, ddColumns.indexed, ddColumns.included) - } - - final override def logEntry: LogEntry = - getIndexLogEntry( - spark, - df, - indexConfig, - indexDataPath) - - final override val transientState: String = REFRESHING - - final override val finalState: String = ACTIVE - - final override def validate(): Unit = { - if (!previousIndexLogEntry.state.equalsIgnoreCase(ACTIVE)) { - throw HyperspaceException( - s"Refresh is only supported in $ACTIVE state. " + - s"Current index state is ${previousIndexLogEntry.state}") - } - } - - final override def op(): Unit = { - // TODO: The current implementation picks the number of buckets from session config. - // This should be user-configurable to allow maintain the existing bucket numbers - // in the index log entry. - write(spark, indexableDf, indexConfig) - } - - final override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { - RefreshActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) - } -} From 3cbfea51b85613298281514781a3d164ae4f649e Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 16 Sep 2020 13:45:59 -0700 Subject: [PATCH 16/86] add unit test to show refresh works fine --- .../index/E2EHyperspaceRulesTests.scala | 85 ++++++++++++++++++- 1 file changed, 81 insertions(+), 4 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 9d8f0410a..3166ff653 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -22,10 +22,11 @@ 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 org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData} import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule} -import com.microsoft.hyperspace.util.PathUtils +import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { private val testDir = "src/test/resources/e2eTests/" @@ -396,6 +397,69 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { sortedRowsWithHyperspaceDisabled.sameElements(getSortedRows(dfAfterHyperspaceDisabled))) } + test("Verify Join Index Rule utilizes indexes correctly after incremental refresh.") { + // Setup. Create Data. + val sampleParquetDataLocation = "src/test/resources/sampleparquet" + spark.conf.set(IndexConstants.REFRESH_APPEND_ENABLED, true) + val refreshTestLocation = sampleParquetDataLocation + "refresh" + FileUtils.delete(new Path(refreshTestLocation)) + val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .parquet(refreshTestLocation) + val df = spark.read.load(refreshTestLocation) + + // Create Index. + hyperspace.createIndex(df, indexConfig) + + // Append to Original Data. + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(3) + .write + .mode("append") + .parquet(refreshTestLocation) + + // Refresh Index. + hyperspace.refreshIndex(indexConfig.indexName) + + // Create a Join Query + val leftDf = spark.read.parquet(refreshTestLocation) + val rightDf = spark.read.parquet(refreshTestLocation) + def query(): DataFrame = { + leftDf + .join(rightDf, leftDf("RGUID") === rightDf("RGUID")) + .select(leftDf("RGUID"), rightDf("imprs")) + } + + // Verify Indexes are used, and all index files are picked. + verifyIndexUsage( + query, + getIndexFilesPath(indexConfig.indexName) ++ + getIndexFilesPath(indexConfig.indexName)) + + // Verify Bucketing works as expected. This is reflected in shuffle nodes being eliminated + // when hyperspace is enabled. + spark.disableHyperspace() + val dfWithHyperspaceDisabled = query() + var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s : ShuffleExchangeExec => s + } + assert(shuffleNodes.size == 2) + + spark.enableHyperspace() + val dfWithHyperspaceEnabled = query() + shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s : ShuffleExchangeExec => s + } + assert(shuffleNodes.isEmpty) + + FileUtils.delete(new Path(refreshTestLocation)) + } + test("Test for isHyperspaceEnabled().") { assert(!spark.isHyperspaceEnabled(), "Hyperspace must be disabled by default.") spark.enableHyperspace() @@ -435,9 +499,22 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { } private def getIndexFilesPath(indexName: String): Seq[Path] = { - Content.fromDirectory(new Path( - systemPath, - s"$indexName/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0")).files + var files = Content + .fromDirectory( + new Path(systemPath, s"$indexName/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0")) + .files + + // Append refreshed files to the list if such a directory exists. + val v1Dir = new Path( + systemPath, s"$indexName/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + if (fileSystem.exists(v1Dir)) { + files ++= Content + .fromDirectory( + new Path(systemPath, s"$indexName/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1")) + .files + } + + files } /** From 0c0c376615109a0827bc7d14c71c71295c630585 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 17 Sep 2020 16:43:05 -0700 Subject: [PATCH 17/86] add sort node verification to the test --- .../hyperspace/index/E2EHyperspaceRulesTests.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 3166ff653..390fb271c 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path 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.SortExec import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec @@ -449,13 +450,22 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { case s : ShuffleExchangeExec => s } assert(shuffleNodes.size == 2) + var sortNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s : SortExec => s + } + assert(sortNodes.size == 2) + // Verify that sort nodes are not removed. spark.enableHyperspace() val dfWithHyperspaceEnabled = query() shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { case s : ShuffleExchangeExec => s } assert(shuffleNodes.isEmpty) + sortNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s : SortExec => s + } + assert(sortNodes.size == 2) FileUtils.delete(new Path(refreshTestLocation)) } From eb0c29a78bcfbd272e3c91cd50e15b1b22a9bc5d Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 1 Oct 2020 13:40:15 -0700 Subject: [PATCH 18/86] Update src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala Co-authored-by: Rahul Potharaju --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index b6dfcae1c..44040ae02 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -251,7 +251,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } } - test("Verify refresh-incremental rebuild of index.") { + test("Verify refresh-incremental (append-only) should index only newly appended data.") { Seq(("csv", Map("header" -> "true")), ("parquet", Map()), ("json", Map())).foreach { case (format, option: Map[String, String]) => spark.conf.set(IndexConstants.REFRESH_APPEND_ENABLED, true) From 5e252906f6027d33297b265bf616273eb5a35a50 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 1 Oct 2020 13:40:26 -0700 Subject: [PATCH 19/86] Update src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala Co-authored-by: Rahul Potharaju --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 44040ae02..24934915d 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -275,7 +275,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { .count() assert(indexCount == 10) - // Change Original Data + // Change original data. SampleData.testData .toDF("Date", "RGUID", "Query", "imprs", "clicks") .limit(3) From 3347c6db331d052f98740432711904629f490668 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 1 Oct 2020 13:40:43 -0700 Subject: [PATCH 20/86] Update src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala Co-authored-by: Rahul Potharaju --- .../scala/com/microsoft/hyperspace/index/IndexManagerTests.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 24934915d..883b1aebf 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -255,6 +255,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { Seq(("csv", Map("header" -> "true")), ("parquet", Map()), ("json", Map())).foreach { case (format, option: Map[String, String]) => spark.conf.set(IndexConstants.REFRESH_APPEND_ENABLED, true) + // Setup. Create sample data and index. val refreshTestLocation = sampleParquetDataLocation + "refresh_" + format FileUtils.delete(new Path(refreshTestLocation)) val indexConfig = IndexConfig(s"index_$format", Seq("RGUID"), Seq("imprs")) From 870b2f70006f3877fcbc79c1d746d18e1a545f10 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 1 Oct 2020 13:40:54 -0700 Subject: [PATCH 21/86] Update src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala Co-authored-by: Rahul Potharaju --- .../com/microsoft/hyperspace/actions/RefreshIncremental.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala index ddbc2bed8..b8aff1d8c 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala @@ -36,7 +36,7 @@ class RefreshIncremental( } private lazy val indexableDf = { - // Currently we only support to create an index on a LogicalRelation. + // Currently we only support creating an index on a LogicalRelation. assert(previousIndexLogEntry.relations.size == 1) val relation = previousIndexLogEntry.relations.head From 7ec82e42c716d0fa112a8c5701a3082dcd8c3c7b Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 1 Oct 2020 13:41:12 -0700 Subject: [PATCH 22/86] Update src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala Co-authored-by: Rahul Potharaju --- .../microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala | 2 +- 1 file changed, 1 insertion(+), 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 609e2040e..7759b5fd9 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -438,7 +438,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { // Refresh Index. hyperspace.refreshIndex(indexConfig.indexName) - // Create a Join Query + // Create a Join Query. val leftDf = spark.read.parquet(refreshTestLocation) val rightDf = spark.read.parquet(refreshTestLocation) def query(): DataFrame = { From 76714a615609847e6e682f02b565cb0c028f41dc Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 1 Oct 2020 13:44:57 -0700 Subject: [PATCH 23/86] test bug fix --- .../microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 609e2040e..9aef74654 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -450,8 +450,8 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { // Verify Indexes are used, and all index files are picked. verifyIndexUsage( query, - getIndexFilesPath(indexConfig.indexName) ++ - getIndexFilesPath(indexConfig.indexName)) + getIndexFilesPath(indexConfig.indexName, Seq(0, 1)) ++ + getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) // Verify Bucketing works as expected. This is reflected in shuffle nodes being eliminated // when hyperspace is enabled. From dbb7cccd32c9bf1e69111f7830640582ae735c48 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 1 Oct 2020 14:41:56 -0700 Subject: [PATCH 24/86] Trigger Build From ffb7b900ebd7b8a67bfbd8c99d3604d85ed47436 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 1 Oct 2020 14:58:17 -0700 Subject: [PATCH 25/86] build fix --- .../microsoft/hyperspace/index/RefreshIndexDeleteTests.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala index daed66ef0..cb70ee32f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala @@ -128,7 +128,9 @@ class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) val ex = intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) - assert(ex.getMessage.contains("Refresh delete is only supported on an index with lineage.")) + assert( + ex.getMessage.contains("Index refresh (to handle deleted source data) is only " + + "supported on an index with lineage.")) } } } From 4d73c9af7902c74cc996e693bec6575b435fb853 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 1 Oct 2020 15:10:33 -0700 Subject: [PATCH 26/86] cleanup unnecessary test file --- .../index/RefreshIndexDeleteTests.scala | 136 ------------------ 1 file changed, 136 deletions(-) delete mode 100644 src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala deleted file mode 100644 index cb70ee32f..000000000 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexDeleteTests.scala +++ /dev/null @@ -1,136 +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.index - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.spark.sql.catalyst.plans.SQLHelper - -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} -import com.microsoft.hyperspace.util.FileUtils - -class RefreshIndexDeleteTests extends HyperspaceSuite with SQLHelper { - override val systemPath = new Path("src/test/resources/indexLocation") - private val testDir = "src/test/resources/RefreshIndexDeleteTests/" - private val nonPartitionedDataPath = testDir + "nonpartitioned" - private val partitionedDataPath = testDir + "partitioned" - private val indexConfig = IndexConfig("index1", Seq("Query"), Seq("imprs")) - private var hyperspace: Hyperspace = _ - - override def beforeAll(): Unit = { - super.beforeAll() - - val sparkSession = spark - hyperspace = new Hyperspace(sparkSession) - FileUtils.delete(new Path(testDir)) - } - - override def afterAll(): Unit = { - FileUtils.delete(new Path(testDir)) - super.afterAll() - } - - after { - FileUtils.delete(new Path(testDir)) - FileUtils.delete(systemPath) - } - - test("Validate refresh index when some file gets deleted.") { - // save test data non-partitioned. - SampleData.save( - spark, - nonPartitionedDataPath, - Seq("Date", "RGUID", "Query", "imprs", "clicks")) - val nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) - - // save test data partitioned. - SampleData.save( - spark, - partitionedDataPath, - Seq("Date", "RGUID", "Query", "imprs", "clicks"), - Some(Seq("Date", "Query"))) - val partitionedDataDF = spark.read.parquet(partitionedDataPath) - - Seq(nonPartitionedDataPath, partitionedDataPath).foreach { loc => - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { - withIndex(indexConfig.indexName) { - val dfToIndex = - if (loc.equals(nonPartitionedDataPath)) nonPartitionedDataDF else partitionedDataDF - hyperspace.createIndex(dfToIndex, indexConfig) - - // delete some source data file. - val dataPath = - if (loc.equals(nonPartitionedDataPath)) new Path(nonPartitionedDataPath, "*parquet") - else new Path(partitionedDataPath + "/Date=2018-09-03/Query=ibraco", "*parquet") - - val dataFileNames = dataPath - .getFileSystem(new Configuration) - .globStatus(dataPath) - .map(_.getPath) - - assert(dataFileNames.length > 0) - val fileToDelete = dataFileNames.head - FileUtils.delete(fileToDelete) - - val originalIndexDF = spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + - s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") - val delCount = - originalIndexDF - .filter(s"""${IndexConstants.DATA_FILE_NAME_COLUMN} == "$fileToDelete"""") - .count() - - hyperspace.refreshIndex(indexConfig.indexName) - - val refreshedIndexDF = spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + - s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") - - // validate only index records whose lineage is the deleted file are removed. - assert(refreshedIndexDF.count() == (originalIndexDF.count() - delCount)) - - val lineageFileNames = refreshedIndexDF - .select(IndexConstants.DATA_FILE_NAME_COLUMN) - .distinct() - .collect() - .map(r => new Path(r.getString(0))) - - assert(!lineageFileNames.contains(fileToDelete)) - } - } - } - } - - test("Validate refresh delete fails as expected on an index without lineage.") { - SampleData.save( - spark, - nonPartitionedDataPath, - Seq("Date", "RGUID", "Query", "imprs", "clicks")) - val nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) - - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "false", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { - hyperspace.createIndex(nonPartitionedDataDF, indexConfig) - - val ex = intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) - assert( - ex.getMessage.contains("Index refresh (to handle deleted source data) is only " + - "supported on an index with lineage.")) - } - } -} From b6e43af23de82da3059ee26cb9e8e0089b44d164 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 5 Oct 2020 11:07:26 -0700 Subject: [PATCH 27/86] merge initial --- src/main/scala/com/microsoft/hyperspace/Hyperspace.scala | 5 +++-- .../hyperspace/index/CachingIndexCollectionManager.scala | 4 ++-- .../microsoft/hyperspace/index/IndexCollectionManager.scala | 6 +++--- .../com/microsoft/hyperspace/index/IndexConstants.scala | 2 ++ .../scala/com/microsoft/hyperspace/index/IndexManager.scala | 2 +- 5 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala index cdc786ed0..e17e59a42 100644 --- a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala +++ b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala @@ -18,6 +18,7 @@ package com.microsoft.hyperspace import org.apache.spark.sql.{DataFrame, SparkSession} +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.plananalysis.PlanAnalyzer @@ -73,8 +74,8 @@ class Hyperspace(spark: SparkSession) { * * @param indexName Name of the index to refresh. */ - def refreshIndex(indexName: String): Unit = { - indexManager.refresh(indexName) + def refreshIndex(indexName: String, mode: String = REFRESH_MODE_INCREMENTAL): Unit = { + indexManager.refresh(indexName, mode) } /** diff --git a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala index dc5a47323..56fbc6c4a 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala @@ -92,9 +92,9 @@ class CachingIndexCollectionManager( super.vacuum(indexName) } - override def refresh(indexName: String): Unit = { + override def refresh(indexName: String, mode: String): Unit = { clearCache() - super.refresh(indexName) + super.refresh(indexName, mode) } } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index d04b531f1..443239f33 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.internal.SQLConf import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions._ +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.util.HyperspaceConf class IndexCollectionManager( @@ -63,13 +64,12 @@ class IndexCollectionManager( } } - override def refresh(indexName: String): Unit = { + override def refresh(indexName: String, mode: String): Unit = { withLogManager(indexName) { logManager => val indexPath = PathResolver(spark.sessionState.conf).getIndexPath(indexName) val dataManager = indexDataManagerFactory.create(indexPath) - if (HyperspaceConf.refreshDeleteEnabled(spark)) { + if (mode.equals(REFRESH_MODE_INCREMENTAL)) { new RefreshDeleteAction(spark, logManager, dataManager).run() - } else if (HyperspaceConf.refreshAppendEnabled(spark)) { new RefreshIncremental(spark, logManager, dataManager).run() } else { new RefreshAction(spark, logManager, dataManager).run() diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala index 5e4513ddb..ba1191e27 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala @@ -60,4 +60,6 @@ object IndexConstants { val REFRESH_APPEND_ENABLED = "spark.hyperspace.index.refresh.append.enabled" val REFRESH_APPEND_ENABLED_DEFAULT = "false" + + val REFRESH_MODE_INCREMENTAL = "incremental" } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala index 87d3c638c..993912513 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexManager.scala @@ -65,7 +65,7 @@ trait IndexManager { * * @param indexName Name of the index to refresh. */ - def refresh(indexName: String): Unit + def refresh(indexName: String, mode: String): Unit /** * Cancel api to bring back index from an inconsistent state to the last known stable state. From 1d287fb137fc26141947de86a6431c109e019540 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 5 Oct 2020 11:39:58 -0700 Subject: [PATCH 28/86] review comments --- .../actions/RefreshIncremental.scala | 15 +- .../index/E2EHyperspaceRulesTests.scala | 136 +++++++++--------- .../hyperspace/index/IndexManagerTests.scala | 107 +++++++------- 3 files changed, 130 insertions(+), 128 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala index b8aff1d8c..1e58f327c 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala @@ -54,7 +54,7 @@ class RefreshIncremental( val dataSchema = DataType.fromJson(relation.dataSchemaJson).asInstanceOf[StructType] - // Create a df with only diff files from original list of files + // Create a df with only diff files from original list of files. spark.read .schema(dataSchema) .format(relation.fileFormat) @@ -64,15 +64,22 @@ class RefreshIncremental( /** * Create a logEntry with all data files, and index content merged with previous index content. - * @return Merged index log entry. This contains ALL data files (files which were indexed - * previously, and files which are being indexed in this operation). Also contains ALL - * index files (index files for previously indexed data as well as newly created files). + * This contains ALL data files (files which were indexed previously, and files which are being + * indexed in this operation). It also contains ALL index files (index files for previously + * indexed data as well as newly created files). + * + * @return Merged index log entry. */ override def logEntry: LogEntry = { + // Log entry with complete data and newly index files. val entry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) + + // Merge new index files with old index files. val mergedContent = Content( previousIndexLogEntry.content.root.merge(entry.content.root) ) + + // New entry. entry.copy(content = mergedContent) } diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index cbd09a0d4..3e5ba5fba 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -410,75 +410,77 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { } test("Verify Join Index Rule utilizes indexes correctly after incremental refresh.") { - // Setup. Create Data. - val sampleParquetDataLocation = "src/test/resources/sampleparquet" - spark.conf.set(IndexConstants.REFRESH_APPEND_ENABLED, true) - val refreshTestLocation = sampleParquetDataLocation + "refresh" - FileUtils.delete(new Path(refreshTestLocation)) - val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) - import spark.implicits._ - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(10) - .write - .parquet(refreshTestLocation) - val df = spark.read.load(refreshTestLocation) - - // Create Index. - hyperspace.createIndex(df, indexConfig) + withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { + // Setup. Create Data. + val sampleParquetDataLocation = "src/test/resources/sampleparquet" + val refreshTestLocation = sampleParquetDataLocation + "refresh" + FileUtils.delete(new Path(refreshTestLocation)) + val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .parquet(refreshTestLocation) + val df = spark.read.load(refreshTestLocation) + + // Create Index. + hyperspace.createIndex(df, indexConfig) - // Append to Original Data. - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(3) - .write - .mode("append") - .parquet(refreshTestLocation) + // Append to Original Data. + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(3) + .write + .mode("append") + .parquet(refreshTestLocation) - // Refresh Index. - hyperspace.refreshIndex(indexConfig.indexName) + // Refresh Index. + hyperspace.refreshIndex(indexConfig.indexName) - // Create a Join Query. - val leftDf = spark.read.parquet(refreshTestLocation) - val rightDf = spark.read.parquet(refreshTestLocation) - def query(): DataFrame = { - leftDf - .join(rightDf, leftDf("RGUID") === rightDf("RGUID")) - .select(leftDf("RGUID"), rightDf("imprs")) - } + // Create a Join Query. + val leftDf = spark.read.parquet(refreshTestLocation) + val rightDf = spark.read.parquet(refreshTestLocation) - // Verify Indexes are used, and all index files are picked. - verifyIndexUsage( - query, - getIndexFilesPath(indexConfig.indexName, Seq(0, 1)) ++ - getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) + def query(): DataFrame = { + leftDf + .join(rightDf, leftDf("RGUID") === rightDf("RGUID")) + .select(leftDf("RGUID"), rightDf("imprs")) + } - // Verify Bucketing works as expected. This is reflected in shuffle nodes being eliminated - // when hyperspace is enabled. - spark.disableHyperspace() - val dfWithHyperspaceDisabled = query() - var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { - case s : ShuffleExchangeExec => s - } - assert(shuffleNodes.size == 2) - var sortNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { - case s : SortExec => s - } - assert(sortNodes.size == 2) + // Verify Indexes are used, and all index files are picked. + verifyIndexUsage( + query, + getIndexFilesPath(indexConfig.indexName, Seq(0, 1)) ++ + getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) - // Verify that sort nodes are not removed. - spark.enableHyperspace() - val dfWithHyperspaceEnabled = query() - shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { - case s : ShuffleExchangeExec => s - } - assert(shuffleNodes.isEmpty) - sortNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { - case s : SortExec => s - } - assert(sortNodes.size == 2) + // Verify Bucketing works as expected. This is reflected in shuffle nodes being eliminated + // when hyperspace is enabled. + spark.disableHyperspace() + val dfWithHyperspaceDisabled = query() + var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffleNodes.size == 2) + var sortNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s: SortExec => s + } + assert(sortNodes.size == 2) + + // Verify that sort nodes are not removed. + spark.enableHyperspace() + val dfWithHyperspaceEnabled = query() + shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffleNodes.isEmpty) + sortNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s: SortExec => s + } + assert(sortNodes.size == 2) - FileUtils.delete(new Path(refreshTestLocation)) + FileUtils.delete(new Path(refreshTestLocation)) + } } test("Test for isHyperspaceEnabled().") { @@ -570,11 +572,11 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { } private def getIndexFilesPath(indexName: String, version: Seq[Int] = Seq(0)): Seq[Path] = { - version.flatMap(v => Content - .fromDirectory( - new Path(systemPath, - s"$indexName/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=$v")) - .files) + version.flatMap(v => + Content + .fromDirectory( + new Path(systemPath, s"$indexName/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=$v")) + .files) } /** diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 883b1aebf..6e6673a4f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -252,64 +252,57 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } test("Verify refresh-incremental (append-only) should index only newly appended data.") { - Seq(("csv", Map("header" -> "true")), ("parquet", Map()), ("json", Map())).foreach { - case (format, option: Map[String, String]) => - spark.conf.set(IndexConstants.REFRESH_APPEND_ENABLED, true) - // Setup. Create sample data and index. - val refreshTestLocation = sampleParquetDataLocation + "refresh_" + format - FileUtils.delete(new Path(refreshTestLocation)) - val indexConfig = IndexConfig(s"index_$format", Seq("RGUID"), Seq("imprs")) - import spark.implicits._ - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(10) - .write - .options(option) - .format(format) - .save(refreshTestLocation) - val df = spark.read.format(format).options(option).load(refreshTestLocation) - hyperspace.createIndex(df, indexConfig) - var indexCount = - spark.read - .parquet(s"$systemPath/index_$format" + - s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") - .count() - assert(indexCount == 10) - - // Change original data. - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(3) - .write - .mode("append") - .options(option) - .format(format) - .save(refreshTestLocation) - hyperspace.refreshIndex(indexConfig.indexName) - val newIndexLocation = s"$systemPath/index_$format" - indexCount = spark.read - .parquet(newIndexLocation + - s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { + // Setup. Create sample data and index. + val refreshTestLocation = sampleParquetDataLocation + "refresh" + FileUtils.delete(new Path(refreshTestLocation)) + val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .parquet(refreshTestLocation) + val df = spark.read.parquet(refreshTestLocation) + hyperspace.createIndex(df, indexConfig) + var indexCount = + spark.read + .parquet(s"$systemPath/index" + + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") .count() - - // Check if index got updated - assert(indexCount == 3) - - // Check if latest log file is updated with newly created index files - val indexPath = PathUtils.makeAbsolute(newIndexLocation) - val logManager = IndexLogManagerFactoryImpl.create(indexPath) - val latestLog = logManager.getLatestLog() - assert(latestLog.isDefined && latestLog.get.isInstanceOf[IndexLogEntry]) - val indexLog = latestLog.get.asInstanceOf[IndexLogEntry] - val indexFiles = indexLog.content.files - assert(indexFiles.nonEmpty) - assert(indexFiles.forall(_.getName.startsWith("part-0"))) - assert(indexLog.state.equals("ACTIVE")) - // Check there exist some files from v__=0 and some from v__=1 - assert(indexFiles.map(_.getParent.getName).toSet.equals(Set("v__=0", "v__=1"))) - - FileUtils.delete(new Path(refreshTestLocation)) - case _ => fail("invalid test") + assert(indexCount == 10) + + // Change original data. + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(3) + .write + .mode("append") + .parquet(refreshTestLocation) + hyperspace.refreshIndex(indexConfig.indexName) + val newIndexLocation = s"$systemPath/index" + indexCount = spark.read + .parquet(newIndexLocation + + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + .count() + + // Check if index got updated. + assert(indexCount == 3) + + // Check if latest log file is updated with newly created index files + val indexPath = PathUtils.makeAbsolute(newIndexLocation) + val logManager = IndexLogManagerFactoryImpl.create(indexPath) + val latestLog = logManager.getLatestLog() + assert(latestLog.isDefined && latestLog.get.isInstanceOf[IndexLogEntry]) + val indexLog = latestLog.get.asInstanceOf[IndexLogEntry] + val indexFiles = indexLog.content.files + assert(indexFiles.nonEmpty) + assert(indexFiles.forall(_.getName.startsWith("part-0"))) + assert(indexLog.state.equals("ACTIVE")) + // Check there exist some files from v__=0 and some from v__=1. + assert(indexFiles.map(_.getParent.getName).toSet.equals(Set("v__=0", "v__=1"))) + + FileUtils.delete(new Path(refreshTestLocation)) } } From 101361526c79bbbb037bbbcf3ad459eaffe7fcc0 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 5 Oct 2020 12:07:15 -0700 Subject: [PATCH 29/86] review comments in tests --- .../index/E2EHyperspaceRulesTests.scala | 132 +++++++++--------- .../hyperspace/index/IndexManagerTests.scala | 100 ++++++------- 2 files changed, 115 insertions(+), 117 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 3e5ba5fba..bfe121486 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -410,76 +410,74 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { } test("Verify Join Index Rule utilizes indexes correctly after incremental refresh.") { - withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { - // Setup. Create Data. - val sampleParquetDataLocation = "src/test/resources/sampleparquet" - val refreshTestLocation = sampleParquetDataLocation + "refresh" - FileUtils.delete(new Path(refreshTestLocation)) - val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) - import spark.implicits._ - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(10) - .write - .parquet(refreshTestLocation) - val df = spark.read.load(refreshTestLocation) - - // Create Index. - hyperspace.createIndex(df, indexConfig) - - // Append to Original Data. - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(3) - .write - .mode("append") - .parquet(refreshTestLocation) - - // Refresh Index. - hyperspace.refreshIndex(indexConfig.indexName) - - // Create a Join Query. - val leftDf = spark.read.parquet(refreshTestLocation) - val rightDf = spark.read.parquet(refreshTestLocation) - - def query(): DataFrame = { - leftDf - .join(rightDf, leftDf("RGUID") === rightDf("RGUID")) - .select(leftDf("RGUID"), rightDf("imprs")) - } - - // Verify Indexes are used, and all index files are picked. - verifyIndexUsage( - query, - getIndexFilesPath(indexConfig.indexName, Seq(0, 1)) ++ - getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) + withTempDir { testDir => + val refreshTestLocation = testDir + "/refresh" + withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { + // Setup. Create Data. + val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .parquet(refreshTestLocation) + val df = spark.read.load(refreshTestLocation) + + // Create Index. + hyperspace.createIndex(df, indexConfig) + + // Append to Original Data. + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(3) + .write + .mode("append") + .parquet(refreshTestLocation) + + // Refresh Index. + hyperspace.refreshIndex(indexConfig.indexName) + + // Create a Join Query. + val leftDf = spark.read.parquet(refreshTestLocation) + val rightDf = spark.read.parquet(refreshTestLocation) + + def query(): DataFrame = { + leftDf + .join(rightDf, leftDf("RGUID") === rightDf("RGUID")) + .select(leftDf("RGUID"), rightDf("imprs")) + } - // Verify Bucketing works as expected. This is reflected in shuffle nodes being eliminated - // when hyperspace is enabled. - spark.disableHyperspace() - val dfWithHyperspaceDisabled = query() - var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { - case s: ShuffleExchangeExec => s - } - assert(shuffleNodes.size == 2) - var sortNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { - case s: SortExec => s - } - assert(sortNodes.size == 2) + // Verify Indexes are used, and all index files are picked. + verifyIndexUsage( + query, + getIndexFilesPath(indexConfig.indexName, Seq(0, 1)) ++ + getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) + + // Verify Bucketing works as expected. This is reflected in shuffle nodes being eliminated + // when hyperspace is enabled. + spark.disableHyperspace() + val dfWithHyperspaceDisabled = query() + var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffleNodes.size == 2) + var sortNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s: SortExec => s + } + assert(sortNodes.size == 2) - // Verify that sort nodes are not removed. - spark.enableHyperspace() - val dfWithHyperspaceEnabled = query() - shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { - case s: ShuffleExchangeExec => s - } - assert(shuffleNodes.isEmpty) - sortNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { - case s: SortExec => s + // Verify that sort nodes are not removed. + spark.enableHyperspace() + val dfWithHyperspaceEnabled = query() + shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffleNodes.isEmpty) + sortNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s: SortExec => s + } + assert(sortNodes.size == 2) } - assert(sortNodes.size == 2) - - FileUtils.delete(new Path(refreshTestLocation)) } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 6e6673a4f..a3924b585 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -252,57 +252,57 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } test("Verify refresh-incremental (append-only) should index only newly appended data.") { - withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { - // Setup. Create sample data and index. - val refreshTestLocation = sampleParquetDataLocation + "refresh" - FileUtils.delete(new Path(refreshTestLocation)) - val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) - import spark.implicits._ - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(10) - .write - .parquet(refreshTestLocation) - val df = spark.read.parquet(refreshTestLocation) - hyperspace.createIndex(df, indexConfig) - var indexCount = - spark.read - .parquet(s"$systemPath/index" + - s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") + withTempDir { testDir => + val refreshTestLocation = testDir + "/refresh" + withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { + // Setup. Create sample data and index. + FileUtils.delete(new Path(refreshTestLocation)) + val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .parquet(refreshTestLocation) + val df = spark.read.parquet(refreshTestLocation) + hyperspace.createIndex(df, indexConfig) + var indexCount = + spark.read + .parquet(s"$systemPath/index" + + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") + .count() + assert(indexCount == 10) + + // Change original data. + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(3) + .write + .mode("append") + .parquet(refreshTestLocation) + hyperspace.refreshIndex(indexConfig.indexName) + val newIndexLocation = s"$systemPath/index" + indexCount = spark.read + .parquet(newIndexLocation + + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") .count() - assert(indexCount == 10) - - // Change original data. - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(3) - .write - .mode("append") - .parquet(refreshTestLocation) - hyperspace.refreshIndex(indexConfig.indexName) - val newIndexLocation = s"$systemPath/index" - indexCount = spark.read - .parquet(newIndexLocation + - s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") - .count() - - // Check if index got updated. - assert(indexCount == 3) - - // Check if latest log file is updated with newly created index files - val indexPath = PathUtils.makeAbsolute(newIndexLocation) - val logManager = IndexLogManagerFactoryImpl.create(indexPath) - val latestLog = logManager.getLatestLog() - assert(latestLog.isDefined && latestLog.get.isInstanceOf[IndexLogEntry]) - val indexLog = latestLog.get.asInstanceOf[IndexLogEntry] - val indexFiles = indexLog.content.files - assert(indexFiles.nonEmpty) - assert(indexFiles.forall(_.getName.startsWith("part-0"))) - assert(indexLog.state.equals("ACTIVE")) - // Check there exist some files from v__=0 and some from v__=1. - assert(indexFiles.map(_.getParent.getName).toSet.equals(Set("v__=0", "v__=1"))) - - FileUtils.delete(new Path(refreshTestLocation)) + + // Check if index got updated. + assert(indexCount == 3) + + // Check if latest log file is updated with newly created index files + val indexPath = PathUtils.makeAbsolute(newIndexLocation) + val logManager = IndexLogManagerFactoryImpl.create(indexPath) + val latestLog = logManager.getLatestLog() + assert(latestLog.isDefined && latestLog.get.isInstanceOf[IndexLogEntry]) + val indexLog = latestLog.get.asInstanceOf[IndexLogEntry] + val indexFiles = indexLog.content.files + assert(indexFiles.nonEmpty) + assert(indexFiles.forall(_.getName.startsWith("part-0"))) + assert(indexLog.state.equals("ACTIVE")) + // Check there exist some files from v__=0 and some from v__=1. + assert(indexFiles.map(_.getParent.getName).toSet.equals(Set("v__=0", "v__=1"))) + } } } From 424c5ec3b3b45167aab8665a4761e65b6c0a21b7 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 5 Oct 2020 12:49:51 -0700 Subject: [PATCH 30/86] nit comments --- ...al.scala => RefreshIncrementalAction.scala} | 18 +++++++++++++++--- .../index/IndexCollectionManager.scala | 2 +- 2 files changed, 16 insertions(+), 4 deletions(-) rename src/main/scala/com/microsoft/hyperspace/actions/{RefreshIncremental.scala => RefreshIncrementalAction.scala} (82%) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala similarity index 82% rename from src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala rename to src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala index 1e58f327c..a178bcc8f 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncremental.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala @@ -23,7 +23,21 @@ import org.apache.spark.sql.types.{DataType, StructType} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppendActionEvent} -class RefreshIncremental( +/** + * Action to create indexes on newly arrived data. If the user appends new data to existing, + * pre-indexed data, they can use refresh api to generate indexes only on the additional data. + * + * Algorithm Outline: + * - Identify newly added data files. + * - Create new index version on these files. + * - Update metadata to reflect the latest snapshot of index. This snapshot includes all the old + * and the newly created index files. The source content points to the latest data files. + * + * @param spark SparkSession + * @param logManager Index LogManager for index being refreshed + * @param dataManager Index DataManager for index being refreshed + */ +class RefreshIncrementalAction( spark: SparkSession, logManager: IndexLogManager, dataManager: IndexDataManager) @@ -36,8 +50,6 @@ class RefreshIncremental( } private lazy val indexableDf = { - // Currently we only support creating an index on a LogicalRelation. - assert(previousIndexLogEntry.relations.size == 1) val relation = previousIndexLogEntry.relations.head // TODO: improve this to take last modified time of files into account. diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index d04b531f1..49a1bd0b5 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -70,7 +70,7 @@ class IndexCollectionManager( if (HyperspaceConf.refreshDeleteEnabled(spark)) { new RefreshDeleteAction(spark, logManager, dataManager).run() } else if (HyperspaceConf.refreshAppendEnabled(spark)) { - new RefreshIncremental(spark, logManager, dataManager).run() + new RefreshIncrementalAction(spark, logManager, dataManager).run() } else { new RefreshAction(spark, logManager, dataManager).run() } From d283116c7c1c203754084de50c421f4c61fcd7b7 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 5 Oct 2020 13:19:25 -0700 Subject: [PATCH 31/86] test cleanup --- .../index/E2EHyperspaceRulesTests.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index bfe121486..d30e6863f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -414,10 +414,10 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { val refreshTestLocation = testDir + "/refresh" withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { // Setup. Create Data. - val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + val indexConfig = IndexConfig(s"index", Seq("c2"), Seq("c4")) import spark.implicits._ SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .toDF("c1", "c2", "c3", "c4", "c5") .limit(10) .write .parquet(refreshTestLocation) @@ -428,7 +428,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { // Append to Original Data. SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .toDF("c1", "c2", "c3", "c4", "c5") .limit(3) .write .mode("append") @@ -443,8 +443,8 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { def query(): DataFrame = { leftDf - .join(rightDf, leftDf("RGUID") === rightDf("RGUID")) - .select(leftDf("RGUID"), rightDf("imprs")) + .join(rightDf, leftDf("c2") === rightDf("c2")) + .select(leftDf("c2"), rightDf("c4")) } // Verify Indexes are used, and all index files are picked. @@ -454,7 +454,8 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) // Verify Bucketing works as expected. This is reflected in shuffle nodes being eliminated - // when hyperspace is enabled. + // when hyperspace is enabled. Also verify that sort nodes are not removed. + // Without Hyperspace. spark.disableHyperspace() val dfWithHyperspaceDisabled = query() var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { @@ -466,7 +467,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { } assert(sortNodes.size == 2) - // Verify that sort nodes are not removed. + // With Hyperspace. spark.enableHyperspace() val dfWithHyperspaceEnabled = query() shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { From c0c6fdaf77f986ffee001f3fdcd609a8e8b66f48 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 6 Oct 2020 10:49:37 -0700 Subject: [PATCH 32/86] initial add validation in refresh that no deleted files remain --- .../actions/RefreshActionBase.scala | 34 +++++++++++++++++++ .../actions/RefreshDeleteAction.scala | 34 ------------------- .../actions/RefreshIncrementalAction.scala | 21 ++++++++++++ 3 files changed, 55 insertions(+), 34 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala index 621e17d6f..bd7582344 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala @@ -16,6 +16,7 @@ package com.microsoft.hyperspace.actions +import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.{DataType, StructType} @@ -76,4 +77,37 @@ private[actions] abstract class RefreshActionBase( s"Current index state is ${previousIndexLogEntry.state}") } } + + /** + * Compare list of source data files from previous IndexLogEntry to list + * of current source data files, validate fileInfo for existing files and + * identify deleted source data files. + */ + protected lazy val deletedFiles: Seq[String] = { + val rels = previousIndexLogEntry.relations + val originalFiles = rels.head.data.properties.content.fileInfos + val currentFiles = rels.head.rootPaths + .flatMap { p => + Content + .fromDirectory(path = new Path(p), throwIfNotExists = true) + .fileInfos + } + .map(f => f.name -> f) + .toMap + + var delFiles = Seq[String]() + originalFiles.foreach { f => + currentFiles.get(f.name) match { + case Some(v) => + if (!f.equals(v)) { + throw HyperspaceException( + "Index refresh (to handle deleted source data) aborted. " + + s"Existing source data file info is changed (file: ${f.name}).") + } + case None => delFiles :+= f.name + } + } + + delFiles + } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index 9668fc8d3..011db6125 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -16,7 +16,6 @@ package com.microsoft.hyperspace.actions -import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ @@ -89,37 +88,4 @@ class RefreshDeleteAction( previousIndexLogEntry.numBuckets, indexConfig.indexedColumns) } - - /** - * Compare list of source data files from previous IndexLogEntry to list - * of current source data files, validate fileInfo for existing files and - * identify deleted source data files. - */ - private lazy val deletedFiles: Seq[String] = { - val rels = previousIndexLogEntry.relations - val originalFiles = rels.head.data.properties.content.fileInfos - val currentFiles = rels.head.rootPaths - .flatMap { p => - Content - .fromDirectory(path = new Path(p), throwIfNotExists = true) - .fileInfos - } - .map(f => f.name -> f) - .toMap - - var delFiles = Seq[String]() - originalFiles.foreach { f => - currentFiles.get(f.name) match { - case Some(v) => - if (!f.equals(v)) { - throw HyperspaceException( - "Index refresh (to handle deleted source data) aborted. " + - s"Existing source data file info is changed (file: ${f.name}).") - } - case None => delFiles :+= f.name - } - } - - delFiles - } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala index a178bcc8f..c3c8d5290 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala @@ -20,6 +20,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} import org.apache.spark.sql.types.{DataType, StructType} +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppendActionEvent} @@ -49,6 +50,26 @@ class RefreshIncrementalAction( write(spark, indexableDf, indexConfig) } + /** + * Validate there are no deleted files. If there are deleted files, Please run + * [[RefreshDeleteAction]] before this. + * + * Verify index has lineage column and there are NO deleted source data files. + */ + final override def validate(): Unit = { + super.validate() + if (!previousIndexLogEntry.hasLineageColumn(spark)) { + throw HyperspaceException( + "Index refresh (to handle deleted source data) is " + + "only supported on an index with lineage.") + } + + if (deletedFiles.nonEmpty) { + throw HyperspaceException("Refresh-Incremental aborted as some source data files" + + "have been deleted.") + } + } + private lazy val indexableDf = { val relation = previousIndexLogEntry.relations.head From 25ee26b0407e0ec987f21bd3c72b06aba7112c8a Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 6 Oct 2020 11:09:37 -0700 Subject: [PATCH 33/86] review comments and refactoring --- .../hyperspace/actions/CreateActionBase.scala | 3 -- .../actions/RefreshIncrementalAction.scala | 6 ++-- .../hyperspace/index/IndexManagerTests.scala | 34 +++++++++++-------- 3 files changed, 23 insertions(+), 20 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala index ca89b3573..b46f0ad86 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/CreateActionBase.scala @@ -217,7 +217,4 @@ private[actions] abstract class CreateActionBase(dataManager: IndexDataManager) assert(partitionSchemas.length == 1) partitionSchemas.head.map(_.name) } - - private val absolutePath: UserDefinedFunction = udf( - (filePath: String) => PathUtils.makeAbsolute(filePath).toString) } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala index a178bcc8f..049c56611 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala @@ -33,9 +33,9 @@ import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppe * - Update metadata to reflect the latest snapshot of index. This snapshot includes all the old * and the newly created index files. The source content points to the latest data files. * - * @param spark SparkSession - * @param logManager Index LogManager for index being refreshed - * @param dataManager Index DataManager for index being refreshed + * @param spark SparkSession. + * @param logManager Index LogManager for index being refreshed. + * @param dataManager Index DataManager for index being refreshed. */ class RefreshIncrementalAction( spark: SparkSession, diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index a3924b585..b76c6eef7 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -272,6 +272,8 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") .count() assert(indexCount == 10) + // Check if latest log file is updated with newly created index files. + validateMetadata("index", Set("v__=0")) // Change original data. SampleData.testData @@ -281,31 +283,35 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { .mode("append") .parquet(refreshTestLocation) hyperspace.refreshIndex(indexConfig.indexName) - val newIndexLocation = s"$systemPath/index" indexCount = spark.read - .parquet(newIndexLocation + + .parquet(s"$systemPath/index" + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") .count() // Check if index got updated. assert(indexCount == 3) - // Check if latest log file is updated with newly created index files - val indexPath = PathUtils.makeAbsolute(newIndexLocation) - val logManager = IndexLogManagerFactoryImpl.create(indexPath) - val latestLog = logManager.getLatestLog() - assert(latestLog.isDefined && latestLog.get.isInstanceOf[IndexLogEntry]) - val indexLog = latestLog.get.asInstanceOf[IndexLogEntry] - val indexFiles = indexLog.content.files - assert(indexFiles.nonEmpty) - assert(indexFiles.forall(_.getName.startsWith("part-0"))) - assert(indexLog.state.equals("ACTIVE")) - // Check there exist some files from v__=0 and some from v__=1. - assert(indexFiles.map(_.getParent.getName).toSet.equals(Set("v__=0", "v__=1"))) + // Check if latest log file is updated with newly created index files. + validateMetadata("index", Set("v__=0", "v__=1")) } } } + private def validateMetadata(indexName: String, indexVersions: Set[String]): Unit = { + val newIndexLocation = s"$systemPath/$indexName" + val indexPath = PathUtils.makeAbsolute(newIndexLocation) + val logManager = IndexLogManagerFactoryImpl.create(indexPath) + val latestLog = logManager.getLatestLog() + assert(latestLog.isDefined && latestLog.get.isInstanceOf[IndexLogEntry]) + val indexLog = latestLog.get.asInstanceOf[IndexLogEntry] + val indexFiles = indexLog.content.files + assert(indexFiles.nonEmpty) + assert(indexFiles.forall(_.getName.startsWith("part-0"))) + assert(indexLog.state.equals("ACTIVE")) + // Check all files belong to v__=0. + assert(indexFiles.map(_.getParent.getName).toSet.equals(indexVersions)) + } + private def expectedIndex( indexConfig: IndexConfig, schema: StructType, From cb1039ceb0ac1f1b6ca7e4f484a401c98096f29a Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 6 Oct 2020 11:18:59 -0700 Subject: [PATCH 34/86] add validation in refresh incremental --- .../actions/RefreshIncrementalAction.scala | 30 +++++++++++++++---- 1 file changed, 24 insertions(+), 6 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala index 049c56611..64b11bab5 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala @@ -20,6 +20,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} import org.apache.spark.sql.types.{DataType, StructType} +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppendActionEvent} @@ -49,7 +50,19 @@ class RefreshIncrementalAction( write(spark, indexableDf, indexConfig) } - private lazy val indexableDf = { + /** + * Validate index is in active state for refreshing and there are some appended + * source data file(s). + */ + final override def validate(): Unit = { + super.validate() + + if (appendedFiles.isEmpty) { + throw HyperspaceException("Refresh aborted as no appended source data files found.") + } + } + + private lazy val appendedFiles = { val relation = previousIndexLogEntry.relations.head // TODO: improve this to take last modified time of files into account. @@ -57,13 +70,18 @@ class RefreshIncrementalAction( val allFiles = df.queryExecution.optimizedPlan.collect { case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), - _, - _, - _) => + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), + _, + _, + _) => location.allFiles().map(_.getPath.toString) }.flatten + allFiles.diff(indexedFiles) + } + + private lazy val indexableDf = { + val relation = previousIndexLogEntry.relations.head val dataSchema = DataType.fromJson(relation.dataSchemaJson).asInstanceOf[StructType] // Create a df with only diff files from original list of files. @@ -71,7 +89,7 @@ class RefreshIncrementalAction( .schema(dataSchema) .format(relation.fileFormat) .options(relation.options) - .load(allFiles.diff(indexedFiles): _*) + .load(appendedFiles: _*) } /** From 0724ee241ba9bd298162a950c2aec528e7acc480 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 6 Oct 2020 11:23:34 -0700 Subject: [PATCH 35/86] add test case for failing for no new appended files --- .../hyperspace/index/IndexManagerTests.scala | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index b76c6eef7..a48a645ec 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -251,6 +251,29 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } } + test("Verify refresh-incremental (append-only) throws exception if no new files found.") { + withTempDir { testDir => + val refreshTestLocation = testDir + "/refresh" + withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { + // Setup. Create sample data and index. + FileUtils.delete(new Path(refreshTestLocation)) + val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .parquet(refreshTestLocation) + val df = spark.read.parquet(refreshTestLocation) + hyperspace.createIndex(df, indexConfig) + val ex = intercept[HyperspaceException] { + hyperspace.refreshIndex(indexConfig.indexName) + } + assert(ex.msg.equals("Refresh aborted as no appended source data files found.")) + } + } + } + test("Verify refresh-incremental (append-only) should index only newly appended data.") { withTempDir { testDir => val refreshTestLocation = testDir + "/refresh" From e35419fe4618097e6860f26d6d32c5889cfd95ed Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Tue, 6 Oct 2020 12:28:25 -0700 Subject: [PATCH 36/86] trigger build From ee2a02feb65823defe8f7ded6ddfc9caf2d33542 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Tue, 6 Oct 2020 14:18:39 -0700 Subject: [PATCH 37/86] Update RefreshIncrementalAction.scala --- .../microsoft/hyperspace/actions/RefreshIncrementalAction.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala index 64b11bab5..cc47a2c0f 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala @@ -66,6 +66,7 @@ class RefreshIncrementalAction( val relation = previousIndexLogEntry.relations.head // TODO: improve this to take last modified time of files into account. + // https://github.com/microsoft/hyperspace/issues/182 val indexedFiles = relation.data.properties.content.files.map(_.toString) val allFiles = df.queryExecution.optimizedPlan.collect { From c15c9cb2f30971cb917e13d9a37bfa726476e516 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 7 Oct 2020 11:41:02 -0700 Subject: [PATCH 38/86] fix test cases with new behavior --- .../com/microsoft/hyperspace/Hyperspace.scala | 2 +- .../microsoft/hyperspace/actions/Action.scala | 2 +- .../actions/RefreshAppendAction.scala | 4 ++-- .../actions/RefreshDeleteAction.scala | 8 +++---- .../hyperspace/index/IndexConstants.scala | 1 + .../hyperspace/index/IndexManagerTests.scala | 24 ++----------------- .../hyperspace/index/RefreshIndexTests.scala | 19 ++++++++------- 7 files changed, 22 insertions(+), 38 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala index e17e59a42..9678e7ef4 100644 --- a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala +++ b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala @@ -18,8 +18,8 @@ package com.microsoft.hyperspace import org.apache.spark.sql.{DataFrame, SparkSession} -import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.index._ +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.index.plananalysis.PlanAnalyzer class Hyperspace(spark: SparkSession) { diff --git a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala index 2fbb67cfe..fb767e2e5 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala @@ -18,9 +18,9 @@ package com.microsoft.hyperspace.actions import org.apache.spark.internal.Logging +import com.microsoft.hyperspace.{ActiveSparkSession, HyperspaceException, NoChangesDetected} import com.microsoft.hyperspace.index.{IndexLogManager, LogEntry} import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, HyperspaceEventLogging} -import com.microsoft.hyperspace.{ActiveSparkSession, HyperspaceException, NoChangesDetected} /** * This is a generic Index-Modifying Action interface. It provides APIs to begin and commit diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index 2dbf6fc8d..4a975bd87 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -19,7 +19,7 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} -import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.NoChangesDetected import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppendActionEvent} @@ -57,7 +57,7 @@ class RefreshAppendAction( super.validate() if (appendedFiles.isEmpty) { - throw HyperspaceException("Refresh aborted as no appended source data files found.") + throw NoChangesDetected("Refresh aborted as no appended source data files found.") } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index cdd5afc15..1210af887 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -56,15 +56,15 @@ class RefreshDeleteAction( */ final override def validate(): Unit = { super.validate() + if (deletedFiles.isEmpty) { + throw NoChangesDetected("Refresh aborted as no deleted source data file found.") + } + if (!previousIndexLogEntry.hasLineageColumn(spark)) { throw HyperspaceException( "Index refresh (to handle deleted source data) is " + "only supported on an index with lineage.") } - - if (deletedFiles.isEmpty) { - throw NoChangesDetected("Refresh aborted as no deleted source data file found.") - } } /** diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala index 55edaa9ee..c41c41144 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala @@ -75,4 +75,5 @@ object IndexConstants { val REFRESH_APPEND_ENABLED_DEFAULT = "false" val REFRESH_MODE_INCREMENTAL = "incremental" + val REFRESH_MODE_FULL = "full" } diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index abc6ed3f5..cac2bca1d 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} import com.microsoft.hyperspace.TestUtils.copyWithState import com.microsoft.hyperspace.actions.Constants +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_FULL import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class IndexManagerTests extends HyperspaceSuite with SQLHelper { @@ -225,7 +226,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { .options(option) .format(format) .save(refreshTestLocation) - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_FULL) val newIndexLocation = s"$systemPath/index_$format" indexCount = spark.read .parquet(newIndexLocation + @@ -251,27 +252,6 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } } - test("Verify refresh-incremental (append-only) throws exception if no new files found.") { - withTempPathAsString { testPath => - withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { - // Setup. Create sample data and index. - val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) - import spark.implicits._ - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(10) - .write - .parquet(testPath) - val df = spark.read.parquet(testPath) - hyperspace.createIndex(df, indexConfig) - val ex = intercept[HyperspaceException] { - hyperspace.refreshIndex(indexConfig.indexName) - } - assert(ex.msg.equals("Refresh aborted as no appended source data files found.")) - } - } - } - test("Verify refresh-incremental (append-only) should index only newly appended data.") { withTempPathAsString { testPath => withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 90bd8453a..435d28e7e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -21,7 +21,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} -import com.microsoft.hyperspace.util.FileUtils +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_FULL +import com.microsoft.hyperspace.util.{FileUtils, PathUtils} /** * Unit E2E test cases for RefreshIndex. @@ -89,7 +90,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val originalIndexWithoutDeletedFile = originalIndexDF .filter(s"""${IndexConstants.DATA_FILE_NAME_COLUMN} != "$deletedFile"""") - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_FULL) val refreshedIndexDF = spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") @@ -125,20 +126,22 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { test( "Validate refresh index (to handle deletes from the source data) " + - "is aborted if no source data file is deleted.") { + "is a no-op if no source data file is deleted or appended.") { SampleData.save( spark, nonPartitionedDataPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) val nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) + val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") + val logManager = IndexLogManagerFactoryImpl.create(indexPath) + val latestId = logManager.getLatestId().get - val ex = intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) - assert(ex.getMessage.contains("Refresh aborted as no deleted source data file found.")) + hyperspace.refreshIndex(indexConfig.indexName) + // Check that no new log files were created in this operation. + assert(latestId == logManager.getLatestId().get) } } From 3d1829868e246d79b92ed4d6c8d044ae101a2706 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 7 Oct 2020 13:33:48 -0700 Subject: [PATCH 39/86] add "appended" and "deleted" files to metadata --- .../hyperspace/index/IndexLogEntry.scala | 2 +- .../hyperspace/index/IndexLogEntryTest.scala | 8 ++- .../hyperspace/index/RefreshIndexTests.scala | 50 +++++++++++++++++-- 3 files changed, 52 insertions(+), 8 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index 23a8e8980..efa4b3deb 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -325,7 +325,7 @@ case class Hdfs(properties: Hdfs.Properties) { val kind = "HDFS" } object Hdfs { - case class Properties(content: Content) + case class Properties(content: Content, deleted: Seq[String] = Nil, appended: Seq[String] = Nil) } // IndexLogEntry-specific Relation that represents the source relation. diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala index 85541c3e4..1deed4c20 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexLogEntryTest.scala @@ -125,7 +125,9 @@ class IndexLogEntryTest extends SparkFunSuite with SQLHelper with BeforeAndAfter | "kind" : "NoOp", | "properties" : { } | } - | } + | }, + | "deleted" : ["file:/rootpath/f1"], + | "appended" : ["file:/rootpath/f3"] | }, | "kind" : "HDFS" | }, @@ -163,7 +165,9 @@ class IndexLogEntryTest extends SparkFunSuite with SQLHelper with BeforeAndAfter Seq(Relation( Seq("rootpath"), Hdfs(Hdfs.Properties(Content( - Directory("", Seq(FileInfo("f1", 100L, 100L), FileInfo("f2", 200L, 200L)), Seq())))), + Directory("", Seq(FileInfo("f1", 100L, 100L), FileInfo("f2", 200L, 200L)), Seq())), + Seq("file:/rootpath/f1"), + Seq("file:/rootpath/f3"))), "schema", "type", Map())), diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 435d28e7e..3963ccad9 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} -import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_FULL import com.microsoft.hyperspace.util.{FileUtils, PathUtils} /** @@ -90,7 +89,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val originalIndexWithoutDeletedFile = originalIndexDF .filter(s"""${IndexConstants.DATA_FILE_NAME_COLUMN} != "$deletedFile"""") - hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_FULL) + hyperspace.refreshIndex(indexConfig.indexName) val refreshedIndexDF = spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") @@ -124,9 +123,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } } - test( - "Validate refresh index (to handle deletes from the source data) " + - "is a no-op if no source data file is deleted or appended.") { + test("Validate refresh index is a no-op if no source data file is deleted or appended.") { SampleData.save( spark, nonPartitionedDataPath, @@ -211,6 +208,49 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } } + test("Validate refresh index when some file gets deleted and some appended to source data.") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { + withIndex(indexConfig.indexName) { + // Save test data non-partitioned. + SampleData.save( + spark, + nonPartitionedDataPath, + Seq("Date", "RGUID", "Query", "imprs", "clicks")) + var df = spark.read.parquet(nonPartitionedDataPath) + hyperspace.createIndex(df, indexConfig) + val countOriginal = df.count() + + // Delete one source data file. + deleteDataFile(nonPartitionedDataPath) + val countAfterDelete = spark.read.parquet(nonPartitionedDataPath).count() + assert(countAfterDelete < countOriginal) + + // Add some new data to source. + import spark.implicits._ + SampleData.testData + .take(3) + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .write + .mode("append") + .parquet(nonPartitionedDataPath) + + val countAfterAppend = spark.read.parquet(nonPartitionedDataPath).count() + assert(countAfterDelete + 3 == countAfterAppend) + + hyperspace.refreshIndex(indexConfig.indexName) + + // Check if refreshed index is updated appropriately. + val indexDf = spark.read + .parquet(s"$systemPath/${indexConfig.indexName}/" + + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + .union(spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=2")) + + assert(indexDf.count() == countAfterAppend) + } + } + } + /** * Delete one file from a given path. * From af3284a108e698d479ecadc856b348d3985b8915 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 7 Oct 2020 17:13:04 -0700 Subject: [PATCH 40/86] bug fixes for refresh append and delete to work together for deleted and appended files --- .../actions/RefreshActionBase.scala | 22 ++- .../actions/RefreshAppendAction.scala | 25 +--- .../actions/RefreshDeleteAction.scala | 5 + .../actions/RefreshIncrementalAction.scala | 127 ------------------ .../hyperspace/index/IndexLogEntry.scala | 56 ++++++++ 5 files changed, 86 insertions(+), 149 deletions(-) delete mode 100644 src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala index bd7582344..36041f821 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala @@ -18,6 +18,7 @@ package com.microsoft.hyperspace.actions import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} import org.apache.spark.sql.types.{DataType, StructType} import com.microsoft.hyperspace.HyperspaceException @@ -108,6 +109,25 @@ private[actions] abstract class RefreshActionBase( } } - delFiles + delFiles ++ previousIndexLogEntry.deletedFiles + } + + protected lazy val appendedFiles = { + val relation = previousIndexLogEntry.relations.head + + // TODO: improve this to take last modified time of files into account. + // https://github.com/microsoft/hyperspace/issues/182 + val originalFiles = relation.data.properties.content.files.map(_.toString) + + val allFiles = df.queryExecution.optimizedPlan.collect { + case LogicalRelation( + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), + _, + _, + _) => + location.allFiles().map(_.getPath.toString) + }.flatten + + allFiles.diff(originalFiles) ++ previousIndexLogEntry.appendedFiles } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index 4a975bd87..2ae1eda1d 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -17,7 +17,6 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} import com.microsoft.hyperspace.NoChangesDetected import com.microsoft.hyperspace.index._ @@ -61,25 +60,6 @@ class RefreshAppendAction( } } - private lazy val appendedFiles = { - val relation = previousIndexLogEntry.relations.head - - // TODO: improve this to take last modified time of files into account. - // https://github.com/microsoft/hyperspace/issues/182 - val indexedFiles = relation.data.properties.content.files.map(_.toString) - - val allFiles = df.queryExecution.optimizedPlan.collect { - case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), - _, - _, - _) => - location.allFiles().map(_.getPath.toString) - }.flatten - - allFiles.diff(indexedFiles) - } - private lazy val dfWithAppendedFiles = { val relation = previousIndexLogEntry.relations.head // Create a df with only diff files from original list of files. @@ -109,7 +89,10 @@ class RefreshAppendAction( // https://github.com/microsoft/hyperspace/issues/183 // New entry. - entry.copy(content = mergedContent) + entry + .copy(content = mergedContent) + .withNewDeletedFiles(deletedFiles) + .withAdditionalAppendedFiles(Seq()) } override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index 1210af887..bda07a465 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -88,4 +88,9 @@ class RefreshDeleteAction( previousIndexLogEntry.numBuckets, indexConfig.indexedColumns) } + + override def logEntry: LogEntry = { + val entry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) + entry.withNewDeletedFiles(Seq()).withAdditionalAppendedFiles(appendedFiles) + } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala deleted file mode 100644 index 049963608..000000000 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshIncrementalAction.scala +++ /dev/null @@ -1,127 +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.actions - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} -import org.apache.spark.sql.types.{DataType, StructType} - -import com.microsoft.hyperspace.{HyperspaceException, NoChangesDetected} -import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppendActionEvent} - -/** - * Action to create indexes on newly arrived data. If the user appends new data to existing, - * pre-indexed data, they can use refresh api to generate indexes only on the additional data. - * - * Algorithm Outline: - * - Identify newly added data files. - * - Create new index version on these files. - * - Update metadata to reflect the latest snapshot of index. This snapshot includes all the old - * and the newly created index files. The source content points to the latest data files. - * - * @param spark SparkSession. - * @param logManager Index LogManager for index being refreshed. - * @param dataManager Index DataManager for index being refreshed. - */ -class RefreshIncrementalAction( - spark: SparkSession, - logManager: IndexLogManager, - dataManager: IndexDataManager) - extends RefreshActionBase(spark, logManager, dataManager) { - final override def op(): Unit = { - // TODO: The current implementation picks the number of buckets from session config. - // This should be user-configurable to allow maintain the existing bucket numbers - // in the index log entry. - write(spark, indexableDf, indexConfig) - } - - /** - * Validate there are no deleted files. If there are deleted files, Please run - * [[RefreshDeleteAction]] before this. - * - * Verify index has lineage column and there are NO deleted source data files. - * - * Validate index is in active state for refreshing and there are some appended - * source data file(s). - */ - final override def validate(): Unit = { - super.validate() - if (deletedFiles.nonEmpty) { - throw HyperspaceException( - "Refresh-Incremental aborted as some source data files have been deleted.") - } - - if (appendedFiles.isEmpty) { - throw NoChangesDetected("Refresh aborted as no appended source data files found.") - } - } - - private lazy val appendedFiles = { - val relation = previousIndexLogEntry.relations.head - - // TODO: improve this to take last modified time of files into account. - // https://github.com/microsoft/hyperspace/issues/182 - val indexedFiles = relation.data.properties.content.files.map(_.toString) - - val allFiles = df.queryExecution.optimizedPlan.collect { - case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), - _, - _, - _) => - location.allFiles().map(_.getPath.toString) - }.flatten - - allFiles.diff(indexedFiles) - } - - private lazy val indexableDf = { - val relation = previousIndexLogEntry.relations.head - val dataSchema = DataType.fromJson(relation.dataSchemaJson).asInstanceOf[StructType] - - // Create a df with only diff files from original list of files. - spark.read - .schema(dataSchema) - .format(relation.fileFormat) - .options(relation.options) - .load(appendedFiles: _*) - } - - /** - * Create a logEntry with all data files, and index content merged with previous index content. - * This contains ALL data files (files which were indexed previously, and files which are being - * indexed in this operation). It also contains ALL index files (index files for previously - * indexed data as well as newly created files). - * - * @return Merged index log entry. - */ - override def logEntry: LogEntry = { - // Log entry with complete data and newly index files. - val entry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) - - // Merge new index files with old index files. - val mergedContent = Content(previousIndexLogEntry.content.root.merge(entry.content.root)) - - // New entry. - entry.copy(content = mergedContent) - } - - override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { - RefreshAppendActionEvent(appInfo, logEntry.asInstanceOf[IndexLogEntry], message) - } -} diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index efa4b3deb..110c51151 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -379,6 +379,62 @@ case class IndexLogEntry( .toSet } + def deletedFiles: Seq[String] = { + relations.head.data.properties.deleted + } + + def appendedFiles: Seq[String] = { + relations.head.data.properties.appended + } + + def withAdditionalAppendedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + appended = relations.head.data.properties.appended ++ files)))))))) + } + + def withAdditionalDeletedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + deleted = relations.head.data.properties.deleted ++ files)))))))) + } + + def withNewAppendedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + appended = files)))))))) + } + + def withNewDeletedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + deleted = files)))))))) + } + def bucketSpec: BucketSpec = BucketSpec( numBuckets = numBuckets, From 79ddb79721f44dc6ee1b381a3ad7346fec9a8d61 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 7 Oct 2020 17:27:37 -0700 Subject: [PATCH 41/86] trigger build From 10723e2a138815e18cb76b1d72b9d28bfb15373b Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 7 Oct 2020 17:53:49 -0700 Subject: [PATCH 42/86] minor fix --- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 3963ccad9..e9a07fb99 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -216,7 +216,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { spark, nonPartitionedDataPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) - var df = spark.read.parquet(nonPartitionedDataPath) + val df = spark.read.parquet(nonPartitionedDataPath) hyperspace.createIndex(df, indexConfig) val countOriginal = df.count() From b0771bf997ee6244a690ebc393cc1a7f79bea1d5 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 7 Oct 2020 20:39:29 -0700 Subject: [PATCH 43/86] initial commit --- .../microsoft/hyperspace/HyperspaceException.scala | 2 ++ .../com/microsoft/hyperspace/actions/Action.scala | 5 ++++- .../hyperspace/actions/RefreshAppendAction.scala | 4 ++-- .../hyperspace/actions/RefreshDeleteAction.scala | 11 ++++++----- .../hyperspace/index/IndexManagerTests.scala | 11 +++++++---- .../hyperspace/index/RefreshIndexTests.scala | 11 ++++++++--- 6 files changed, 29 insertions(+), 15 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala b/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala index e397983e7..09ce0dba6 100644 --- a/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala +++ b/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala @@ -17,3 +17,5 @@ package com.microsoft.hyperspace case class HyperspaceException(msg: String) extends Exception(msg) + +case class NoChangesDetected(msg: String) extends Exception(msg) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala index 5106b7cc9..fb767e2e5 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala @@ -18,7 +18,7 @@ package com.microsoft.hyperspace.actions import org.apache.spark.internal.Logging -import com.microsoft.hyperspace.{ActiveSparkSession, HyperspaceException} +import com.microsoft.hyperspace.{ActiveSparkSession, HyperspaceException, NoChangesDetected} import com.microsoft.hyperspace.index.{IndexLogManager, LogEntry} import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, HyperspaceEventLogging} @@ -94,6 +94,9 @@ trait Action extends HyperspaceEventLogging with Logging with ActiveSparkSession end() logEvent(event(appInfo, message = "Operation Succeeded.")) } catch { + case e: NoChangesDetected => + logEvent(event(appInfo, message = s"No-op Operation Recorded: ${e.getMessage}.")) + logWarning(e.msg) case e: Exception => logEvent(event(appInfo, message = s"Operation Failed: ${e.getMessage}.")) throw e diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index 2dbf6fc8d..4a975bd87 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -19,7 +19,7 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} -import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.NoChangesDetected import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppendActionEvent} @@ -57,7 +57,7 @@ class RefreshAppendAction( super.validate() if (appendedFiles.isEmpty) { - throw HyperspaceException("Refresh aborted as no appended source data files found.") + throw NoChangesDetected("Refresh aborted as no appended source data files found.") } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index 9668fc8d3..e5de40ec4 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -21,7 +21,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ -import com.microsoft.hyperspace.HyperspaceException +import com.microsoft.hyperspace.{HyperspaceException, NoChangesDetected} import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshDeleteActionEvent} @@ -57,15 +57,16 @@ class RefreshDeleteAction( */ final override def validate(): Unit = { super.validate() + + if (deletedFiles.isEmpty) { + throw NoChangesDetected("Refresh aborted as no deleted source data file found.") + } + if (!previousIndexLogEntry.hasLineageColumn(spark)) { throw HyperspaceException( "Index refresh (to handle deleted source data) is " + "only supported on an index with lineage.") } - - if (deletedFiles.isEmpty) { - throw HyperspaceException("Refresh aborted as no deleted source data file found.") - } } /** diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index abc6ed3f5..c58c4b0ed 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -264,10 +264,13 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { .parquet(testPath) val df = spark.read.parquet(testPath) hyperspace.createIndex(df, indexConfig) - val ex = intercept[HyperspaceException] { - hyperspace.refreshIndex(indexConfig.indexName) - } - assert(ex.msg.equals("Refresh aborted as no appended source data files found.")) + val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") + val logManager = IndexLogManagerFactoryImpl.create(indexPath) + val latestId = logManager.getLatestId().get + + hyperspace.refreshIndex(indexConfig.indexName) + // Check that no new log files were created in this operation. + assert(latestId == logManager.getLatestId().get) } } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 90bd8453a..7f30e981e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} -import com.microsoft.hyperspace.util.FileUtils +import com.microsoft.hyperspace.util.{FileUtils, PathUtils} /** * Unit E2E test cases for RefreshIndex. @@ -137,8 +137,13 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { IndexConstants.REFRESH_DELETE_ENABLED -> "true") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) - val ex = intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) - assert(ex.getMessage.contains("Refresh aborted as no deleted source data file found.")) + val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") + val logManager = IndexLogManagerFactoryImpl.create(indexPath) + val latestId = logManager.getLatestId().get + + hyperspace.refreshIndex(indexConfig.indexName) + // Check that no new log files were created in this operation. + assert(latestId == logManager.getLatestId().get) } } From 37c0dc82a83927d32c7420a00ac5e14984b7b045 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Wed, 7 Oct 2020 21:32:57 -0700 Subject: [PATCH 44/86] test name fix --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index c58c4b0ed..6508e3b59 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -251,7 +251,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } } - test("Verify refresh-incremental (append-only) throws exception if no new files found.") { + test("Verify refresh-incremental (append-only) is a no-op if no new files found.") { withTempPathAsString { testPath => withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { // Setup. Create sample data and index. From 739d474a39ca2c51e7606cdef513a6b4db373a11 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Wed, 7 Oct 2020 21:33:55 -0700 Subject: [PATCH 45/86] Update RefreshIndexTests.scala --- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 7f30e981e..51a8eb6d0 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -125,7 +125,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { test( "Validate refresh index (to handle deletes from the source data) " + - "is aborted if no source data file is deleted.") { + "is a no-op if no source data file is deleted.") { SampleData.save( spark, nonPartitionedDataPath, From ad2e9dfafed11426fe64a7d6944067159f367a5a Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Wed, 7 Oct 2020 22:03:13 -0700 Subject: [PATCH 46/86] initial commit part 2 --- .../actions/RefreshActionBase.scala | 54 +++++++++++++++++++ .../actions/RefreshAppendAction.scala | 25 ++------- .../actions/RefreshDeleteAction.scala | 36 ++----------- .../hyperspace/index/IndexLogEntry.scala | 48 +++++++++++++++++ 4 files changed, 109 insertions(+), 54 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala index 621e17d6f..36041f821 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala @@ -16,7 +16,9 @@ package com.microsoft.hyperspace.actions +import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} import org.apache.spark.sql.types.{DataType, StructType} import com.microsoft.hyperspace.HyperspaceException @@ -76,4 +78,56 @@ private[actions] abstract class RefreshActionBase( s"Current index state is ${previousIndexLogEntry.state}") } } + + /** + * Compare list of source data files from previous IndexLogEntry to list + * of current source data files, validate fileInfo for existing files and + * identify deleted source data files. + */ + protected lazy val deletedFiles: Seq[String] = { + val rels = previousIndexLogEntry.relations + val originalFiles = rels.head.data.properties.content.fileInfos + val currentFiles = rels.head.rootPaths + .flatMap { p => + Content + .fromDirectory(path = new Path(p), throwIfNotExists = true) + .fileInfos + } + .map(f => f.name -> f) + .toMap + + var delFiles = Seq[String]() + originalFiles.foreach { f => + currentFiles.get(f.name) match { + case Some(v) => + if (!f.equals(v)) { + throw HyperspaceException( + "Index refresh (to handle deleted source data) aborted. " + + s"Existing source data file info is changed (file: ${f.name}).") + } + case None => delFiles :+= f.name + } + } + + delFiles ++ previousIndexLogEntry.deletedFiles + } + + protected lazy val appendedFiles = { + val relation = previousIndexLogEntry.relations.head + + // TODO: improve this to take last modified time of files into account. + // https://github.com/microsoft/hyperspace/issues/182 + val originalFiles = relation.data.properties.content.files.map(_.toString) + + val allFiles = df.queryExecution.optimizedPlan.collect { + case LogicalRelation( + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), + _, + _, + _) => + location.allFiles().map(_.getPath.toString) + }.flatten + + allFiles.diff(originalFiles) ++ previousIndexLogEntry.appendedFiles + } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index 4a975bd87..2ae1eda1d 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -17,7 +17,6 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} import com.microsoft.hyperspace.NoChangesDetected import com.microsoft.hyperspace.index._ @@ -61,25 +60,6 @@ class RefreshAppendAction( } } - private lazy val appendedFiles = { - val relation = previousIndexLogEntry.relations.head - - // TODO: improve this to take last modified time of files into account. - // https://github.com/microsoft/hyperspace/issues/182 - val indexedFiles = relation.data.properties.content.files.map(_.toString) - - val allFiles = df.queryExecution.optimizedPlan.collect { - case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), - _, - _, - _) => - location.allFiles().map(_.getPath.toString) - }.flatten - - allFiles.diff(indexedFiles) - } - private lazy val dfWithAppendedFiles = { val relation = previousIndexLogEntry.relations.head // Create a df with only diff files from original list of files. @@ -109,7 +89,10 @@ class RefreshAppendAction( // https://github.com/microsoft/hyperspace/issues/183 // New entry. - entry.copy(content = mergedContent) + entry + .copy(content = mergedContent) + .withNewDeletedFiles(deletedFiles) + .withAdditionalAppendedFiles(Seq()) } override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index e5de40ec4..bda07a465 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -16,7 +16,6 @@ package com.microsoft.hyperspace.actions -import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ @@ -57,7 +56,6 @@ class RefreshDeleteAction( */ final override def validate(): Unit = { super.validate() - if (deletedFiles.isEmpty) { throw NoChangesDetected("Refresh aborted as no deleted source data file found.") } @@ -91,36 +89,8 @@ class RefreshDeleteAction( indexConfig.indexedColumns) } - /** - * Compare list of source data files from previous IndexLogEntry to list - * of current source data files, validate fileInfo for existing files and - * identify deleted source data files. - */ - private lazy val deletedFiles: Seq[String] = { - val rels = previousIndexLogEntry.relations - val originalFiles = rels.head.data.properties.content.fileInfos - val currentFiles = rels.head.rootPaths - .flatMap { p => - Content - .fromDirectory(path = new Path(p), throwIfNotExists = true) - .fileInfos - } - .map(f => f.name -> f) - .toMap - - var delFiles = Seq[String]() - originalFiles.foreach { f => - currentFiles.get(f.name) match { - case Some(v) => - if (!f.equals(v)) { - throw HyperspaceException( - "Index refresh (to handle deleted source data) aborted. " + - s"Existing source data file info is changed (file: ${f.name}).") - } - case None => delFiles :+= f.name - } - } - - delFiles + override def logEntry: LogEntry = { + val entry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) + entry.withNewDeletedFiles(Seq()).withAdditionalAppendedFiles(appendedFiles) } } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index 1bbbb0ba4..5dbc2b329 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -397,6 +397,54 @@ case class IndexLogEntry( relations.head.data.properties.appendedFiles } + def withAdditionalAppendedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + appendedFiles = relations.head.data.properties.appendedFiles ++ files)))))))) + } + + def withAdditionalDeletedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + deletedFiles = relations.head.data.properties.deletedFiles ++ files)))))))) + } + + def withNewAppendedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + appendedFiles = files)))))))) + } + + def withNewDeletedFiles(files: Seq[String]): IndexLogEntry = { + copy( + source = source.copy( + plan = source.plan.copy( + properties = source.plan.properties.copy( + relations = Seq( + relations.head.copy( + data = relations.head.data.copy( + properties = relations.head.data.properties.copy( + deletedFiles = files)))))))) + } + def bucketSpec: BucketSpec = BucketSpec( numBuckets = numBuckets, From 4d27fe2a70609ab29edf34a21d5bf6335914f51a Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 12:22:26 -0700 Subject: [PATCH 47/86] review comments --- .../hyperspace/HyperspaceException.scala | 2 -- .../microsoft/hyperspace/actions/Action.scala | 12 ++++++------ .../actions/NoChangesException.scala | 19 +++++++++++++++++++ .../actions/RefreshAppendAction.scala | 3 +-- .../actions/RefreshDeleteAction.scala | 4 ++-- .../com/microsoft/hyperspace/TestUtils.scala | 17 +++++++++++++++++ .../hyperspace/index/IndexManagerTests.scala | 18 ++++++++++++++++-- .../hyperspace/index/RefreshIndexTests.scala | 17 +++++++++++++++-- 8 files changed, 76 insertions(+), 16 deletions(-) create mode 100644 src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala diff --git a/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala b/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala index 09ce0dba6..e397983e7 100644 --- a/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala +++ b/src/main/scala/com/microsoft/hyperspace/HyperspaceException.scala @@ -17,5 +17,3 @@ package com.microsoft.hyperspace case class HyperspaceException(msg: String) extends Exception(msg) - -case class NoChangesDetected(msg: String) extends Exception(msg) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala index fb767e2e5..72c992417 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala @@ -18,7 +18,7 @@ package com.microsoft.hyperspace.actions import org.apache.spark.internal.Logging -import com.microsoft.hyperspace.{ActiveSparkSession, HyperspaceException, NoChangesDetected} +import com.microsoft.hyperspace.{ActiveSparkSession, HyperspaceException} import com.microsoft.hyperspace.index.{IndexLogManager, LogEntry} import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, HyperspaceEventLogging} @@ -84,7 +84,7 @@ trait Action extends HyperspaceEventLogging with Logging with ActiveSparkSession val appInfo = AppInfo(sparkContext.sparkUser, sparkContext.applicationId, sparkContext.appName) try { - logEvent(event(appInfo, "Operation Started.")) + logEvent(event(appInfo, "Operation started.")) validate() begin() @@ -92,13 +92,13 @@ trait Action extends HyperspaceEventLogging with Logging with ActiveSparkSession op() end() - logEvent(event(appInfo, message = "Operation Succeeded.")) + logEvent(event(appInfo, message = "Operation succeeded.")) } catch { - case e: NoChangesDetected => - logEvent(event(appInfo, message = s"No-op Operation Recorded: ${e.getMessage}.")) + case e: NoChangesException => + logEvent(event(appInfo, message = s"No-op operation recorded: ${e.getMessage}.")) logWarning(e.msg) case e: Exception => - logEvent(event(appInfo, message = s"Operation Failed: ${e.getMessage}.")) + logEvent(event(appInfo, message = s"Operation failed: ${e.getMessage}.")) throw e } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala b/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala new file mode 100644 index 000000000..4202bed02 --- /dev/null +++ b/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala @@ -0,0 +1,19 @@ +/* + * 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.actions + +private[actions] case class NoChangesException(msg: String) extends Exception(msg) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index 4a975bd87..e6b613d59 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -19,7 +19,6 @@ package com.microsoft.hyperspace.actions import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningAwareFileIndex} -import com.microsoft.hyperspace.NoChangesDetected import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshAppendActionEvent} @@ -57,7 +56,7 @@ class RefreshAppendAction( super.validate() if (appendedFiles.isEmpty) { - throw NoChangesDetected("Refresh aborted as no appended source data files found.") + throw NoChangesException("Refresh append aborted as no appended source data files found.") } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index e5de40ec4..e39b625ec 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -21,7 +21,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ -import com.microsoft.hyperspace.{HyperspaceException, NoChangesDetected} +import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.index._ import com.microsoft.hyperspace.index.DataFrameWriterExtensions.Bucketizer import com.microsoft.hyperspace.telemetry.{AppInfo, HyperspaceEvent, RefreshDeleteActionEvent} @@ -59,7 +59,7 @@ class RefreshDeleteAction( super.validate() if (deletedFiles.isEmpty) { - throw NoChangesDetected("Refresh aborted as no deleted source data file found.") + throw NoChangesException("Refresh delete aborted as no deleted source data file found.") } if (!previousIndexLogEntry.hasLineageColumn(spark)) { diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index 3f99e4354..a0ae38992 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -19,6 +19,7 @@ package com.microsoft.hyperspace import org.apache.hadoop.fs.Path import com.microsoft.hyperspace.index.IndexLogEntry +import com.microsoft.hyperspace.telemetry.{EventLogger, HyperspaceEvent} object TestUtils { def copyWithState(index: IndexLogEntry, state: String): IndexLogEntry = { @@ -45,3 +46,19 @@ object TestUtils { } } } + +/** + * This class can be used to test emitted events from Hyperspace actions. + */ +private[index] class MockEventLogger extends EventLogger { + import com.microsoft.hyperspace.MockEventLogger.emittedEvents + // Reset events for `this` action. + emittedEvents = Seq() + + override def logEvent(event: HyperspaceEvent): Unit = { + emittedEvents :+= event + } +} +private[index] object MockEventLogger { + var emittedEvents: Seq[HyperspaceEvent] = Seq() +} diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index c58c4b0ed..f68a475fa 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -23,9 +23,11 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} import com.microsoft.hyperspace.TestUtils.copyWithState import com.microsoft.hyperspace.actions.Constants +import com.microsoft.hyperspace.telemetry.{CreateActionEvent, RefreshAppendActionEvent} +import com.microsoft.hyperspace.telemetry.Constants.HYPERSPACE_EVENT_LOGGER_CLASS_KEY import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class IndexManagerTests extends HyperspaceSuite with SQLHelper { @@ -253,7 +255,9 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { test("Verify refresh-incremental (append-only) throws exception if no new files found.") { withTempPathAsString { testPath => - withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { + withSQLConf( + IndexConstants.REFRESH_APPEND_ENABLED -> "true", + HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.index.MockEventLogger") { // Setup. Create sample data and index. val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) import spark.implicits._ @@ -271,6 +275,16 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { hyperspace.refreshIndex(indexConfig.indexName) // Check that no new log files were created in this operation. assert(latestId == logManager.getLatestId().get) + + // Check emitted events. + MockEventLogger.emittedEvents match { + case Seq( + _: CreateActionEvent, + _: CreateActionEvent, + _: RefreshAppendActionEvent, + _: RefreshAppendActionEvent) => // pass + case _ => fail() + } } } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 7f30e981e..c26147a25 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -20,7 +20,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} +import com.microsoft.hyperspace.telemetry.{CreateActionEvent, RefreshDeleteActionEvent} +import com.microsoft.hyperspace.telemetry.Constants.HYPERSPACE_EVENT_LOGGER_CLASS_KEY import com.microsoft.hyperspace.util.{FileUtils, PathUtils} /** @@ -134,7 +136,8 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + IndexConstants.REFRESH_DELETE_ENABLED -> "true", + HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.index.MockEventLogger") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") @@ -144,6 +147,16 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { hyperspace.refreshIndex(indexConfig.indexName) // Check that no new log files were created in this operation. assert(latestId == logManager.getLatestId().get) + + // Check emitted events. + MockEventLogger.emittedEvents match { + case Seq( + _: CreateActionEvent, + _: CreateActionEvent, + _: RefreshDeleteActionEvent, + _: RefreshDeleteActionEvent) => // pass + case _ => fail() + } } } From 87ff7b72552f9af0d184eb974f32db4ab76b420b Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 8 Oct 2020 12:42:12 -0700 Subject: [PATCH 48/86] doc --- .../hyperspace/actions/NoChangesException.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala b/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala index 4202bed02..e7bafedb9 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala @@ -16,4 +16,14 @@ package com.microsoft.hyperspace.actions +/** + * This exception represents a No-op required from Hyperspace. Use this exception when a + * hyperspace action is not necessary for index maintenance. + * For example, if the data source has not changed since the last time an index was created on it, + * we don't need to do anything when user calls a `refreshIndex()`. + * + * Hyperspace actions silently catch this exception and do not fail the application. + * + * @param msg Error message. + */ private[actions] case class NoChangesException(msg: String) extends Exception(msg) From e31f17138689c4ff54a9e4cfe05a042e1587267f Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 8 Oct 2020 13:16:45 -0700 Subject: [PATCH 49/86] build failure fix --- src/test/scala/com/microsoft/hyperspace/TestUtils.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index a0ae38992..5cb3d43e0 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -50,7 +50,7 @@ object TestUtils { /** * This class can be used to test emitted events from Hyperspace actions. */ -private[index] class MockEventLogger extends EventLogger { +class MockEventLogger extends EventLogger { import com.microsoft.hyperspace.MockEventLogger.emittedEvents // Reset events for `this` action. emittedEvents = Seq() @@ -59,6 +59,7 @@ private[index] class MockEventLogger extends EventLogger { emittedEvents :+= event } } -private[index] object MockEventLogger { + +object MockEventLogger { var emittedEvents: Seq[HyperspaceEvent] = Seq() } From 47c6730bb2acc1bb5ac1da1585080fee583c7982 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 13:32:41 -0700 Subject: [PATCH 50/86] add unit test for "appended"files update --- .../com/microsoft/hyperspace/TestUtils.scala | 5 +- .../hyperspace/index/RefreshIndexTests.scala | 60 +++++++++++++++++-- 2 files changed, 59 insertions(+), 6 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index a0ae38992..5cb3d43e0 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -50,7 +50,7 @@ object TestUtils { /** * This class can be used to test emitted events from Hyperspace actions. */ -private[index] class MockEventLogger extends EventLogger { +class MockEventLogger extends EventLogger { import com.microsoft.hyperspace.MockEventLogger.emittedEvents // Reset events for `this` action. emittedEvents = Seq() @@ -59,6 +59,7 @@ private[index] class MockEventLogger extends EventLogger { emittedEvents :+= event } } -private[index] object MockEventLogger { + +object MockEventLogger { var emittedEvents: Seq[HyperspaceEvent] = Seq() } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 869027000..b4d234c39 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -140,13 +140,11 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.index.MockEventLogger") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) - val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") - val logManager = IndexLogManagerFactoryImpl.create(indexPath) - val latestId = logManager.getLatestId().get + val latestId = logManager(indexConfig.indexName).getLatestId().get hyperspace.refreshIndex(indexConfig.indexName) // Check that no new log files were created in this operation. - assert(latestId == logManager.getLatestId().get) + assert(latestId == logManager(indexConfig.indexName).getLatestId().get) // Check emitted events. MockEventLogger.emittedEvents match { @@ -226,6 +224,49 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } } + test( + "Validate refresh delete action updates appended files as expected," + + "when some file gets deleted and some appended to source data.") { + withSQLConf( + IndexConstants.INDEX_LINEAGE_ENABLED -> "true", + IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withIndex(indexConfig.indexName) { + SampleData.save( + spark, + nonPartitionedDataPath, + Seq("Date", "RGUID", "Query", "imprs", "clicks")) + val df = spark.read.parquet(nonPartitionedDataPath) + hyperspace.createIndex(df, indexConfig) + + // Delete one source data file. + deleteDataFile(nonPartitionedDataPath) + + val oldFileList = fileList(nonPartitionedDataPath).toSet + + // Add some new data to source. + import spark.implicits._ + SampleData.testData + .take(3) + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .write + .mode("append") + .parquet(nonPartitionedDataPath) + + hyperspace.refreshIndex(indexConfig.indexName) + + // Check if refreshed index metadata has "appendedFiles" and "deletedFiles" updated. + val entry = logManager(indexConfig.indexName).getLatestStableLog() + assert(entry.isDefined) + assert(entry.get.isInstanceOf[IndexLogEntry]) + val indexLogEntry = entry.get.asInstanceOf[IndexLogEntry] + assert(indexLogEntry.deletedFiles.isEmpty) + + val newFileList = fileList(nonPartitionedDataPath).toSet + assert(indexLogEntry.appendedFiles.toSet.equals(newFileList -- oldFileList)) + } + } + } + /** * Delete one file from a given path. * @@ -251,4 +292,15 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { fileToDelete } + + private def logManager(indexName: String): IndexLogManager = { + val indexPath = PathUtils.makeAbsolute(s"$systemPath/$indexName") + IndexLogManagerFactoryImpl.create(indexPath) + } + + private def fileList(path: String): Seq[String] = { + val absolutePath = PathUtils.makeAbsolute(path) + val fs = absolutePath.getFileSystem(new Configuration) + fs.listStatus(absolutePath).toSeq.map(_.getPath.toString) + } } From 7b4a6a8aadaabbd350b2b10ec760a2921f416754 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 13:36:43 -0700 Subject: [PATCH 51/86] test fix --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 2 +- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 0deea9bad..48a451a1d 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -257,7 +257,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { withTempPathAsString { testPath => withSQLConf( IndexConstants.REFRESH_APPEND_ENABLED -> "true", - HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.index.MockEventLogger") { + HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.MockEventLogger") { // Setup. Create sample data and index. val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) import spark.implicits._ diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 869027000..8d93d9c8b 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -137,7 +137,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "true", IndexConstants.REFRESH_DELETE_ENABLED -> "true", - HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.index.MockEventLogger") { + HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.MockEventLogger") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") From f8766c0ff3f46e520e1bc6546fd70870435002ea Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 14:01:39 -0700 Subject: [PATCH 52/86] review comments --- .../com/microsoft/hyperspace/actions/Action.scala | 4 ++-- .../hyperspace/actions/NoChangesException.scala | 2 +- .../scala/com/microsoft/hyperspace/TestUtils.scala | 6 +++++- .../hyperspace/index/IndexManagerTests.scala | 11 ++++++----- .../hyperspace/index/RefreshIndexTests.scala | 11 ++++++----- 5 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala index 72c992417..eb1143cde 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/Action.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/Action.scala @@ -95,10 +95,10 @@ trait Action extends HyperspaceEventLogging with Logging with ActiveSparkSession logEvent(event(appInfo, message = "Operation succeeded.")) } catch { case e: NoChangesException => - logEvent(event(appInfo, message = s"No-op operation recorded: ${e.getMessage}.")) + logEvent(event(appInfo, message = s"No-op operation recorded: ${e.getMessage}")) logWarning(e.msg) case e: Exception => - logEvent(event(appInfo, message = s"Operation failed: ${e.getMessage}.")) + logEvent(event(appInfo, message = s"Operation failed: ${e.getMessage}")) throw e } } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala b/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala index e7bafedb9..7f16ece31 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/NoChangesException.scala @@ -22,7 +22,7 @@ package com.microsoft.hyperspace.actions * For example, if the data source has not changed since the last time an index was created on it, * we don't need to do anything when user calls a `refreshIndex()`. * - * Hyperspace actions silently catch this exception and do not fail the application. + * [[Action.run]] will silently catch this exception and will not fail the application. * * @param msg Error message. */ diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index 5cb3d43e0..5324eb8c3 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -18,6 +18,7 @@ package com.microsoft.hyperspace import org.apache.hadoop.fs.Path +import com.microsoft.hyperspace.MockEventLogger.reset import com.microsoft.hyperspace.index.IndexLogEntry import com.microsoft.hyperspace.telemetry.{EventLogger, HyperspaceEvent} @@ -53,7 +54,7 @@ object TestUtils { class MockEventLogger extends EventLogger { import com.microsoft.hyperspace.MockEventLogger.emittedEvents // Reset events for `this` action. - emittedEvents = Seq() + reset() override def logEvent(event: HyperspaceEvent): Unit = { emittedEvents :+= event @@ -61,5 +62,8 @@ class MockEventLogger extends EventLogger { } object MockEventLogger { + def reset(): Unit = { + emittedEvents = Seq() + } var emittedEvents: Seq[HyperspaceEvent] = Seq() } diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 48a451a1d..822cb3a2c 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} import com.microsoft.hyperspace.TestUtils.copyWithState import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.telemetry.{CreateActionEvent, RefreshAppendActionEvent} import com.microsoft.hyperspace.telemetry.Constants.HYPERSPACE_EVENT_LOGGER_CLASS_KEY +import com.microsoft.hyperspace.telemetry.RefreshAppendActionEvent import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class IndexManagerTests extends HyperspaceSuite with SQLHelper { @@ -272,6 +272,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { val logManager = IndexLogManagerFactoryImpl.create(indexPath) val latestId = logManager.getLatestId().get + MockEventLogger.reset() hyperspace.refreshIndex(indexConfig.indexName) // Check that no new log files were created in this operation. assert(latestId == logManager.getLatestId().get) @@ -279,10 +280,10 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { // Check emitted events. MockEventLogger.emittedEvents match { case Seq( - _: CreateActionEvent, - _: CreateActionEvent, - _: RefreshAppendActionEvent, - _: RefreshAppendActionEvent) => // pass + RefreshAppendActionEvent(_, _, "Operation started."), + RefreshAppendActionEvent(_, _, msg)) + if msg.equals("No-op operation recorded: Refresh append aborted as" + + " no appended source data files found.") => // pass case _ => fail() } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 8d93d9c8b..7319328d3 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -21,8 +21,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} -import com.microsoft.hyperspace.telemetry.{CreateActionEvent, RefreshDeleteActionEvent} import com.microsoft.hyperspace.telemetry.Constants.HYPERSPACE_EVENT_LOGGER_CLASS_KEY +import com.microsoft.hyperspace.telemetry.RefreshDeleteActionEvent import com.microsoft.hyperspace.util.{FileUtils, PathUtils} /** @@ -144,6 +144,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val logManager = IndexLogManagerFactoryImpl.create(indexPath) val latestId = logManager.getLatestId().get + MockEventLogger.reset() hyperspace.refreshIndex(indexConfig.indexName) // Check that no new log files were created in this operation. assert(latestId == logManager.getLatestId().get) @@ -151,10 +152,10 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { // Check emitted events. MockEventLogger.emittedEvents match { case Seq( - _: CreateActionEvent, - _: CreateActionEvent, - _: RefreshDeleteActionEvent, - _: RefreshDeleteActionEvent) => // pass + RefreshDeleteActionEvent(_, _, "Operation started."), + RefreshDeleteActionEvent(_, _, msg)) + if msg.equals("No-op operation recorded: Refresh delete aborted as " + + "no deleted source data file found.") => // pass case _ => fail() } } From 58f13392273e62e9697c076be42e3654c12a2559 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 15:58:06 -0700 Subject: [PATCH 53/86] set logger at session initialization in test cases --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 5 ++--- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 5 ++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 822cb3a2c..2fae0575f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -40,6 +40,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { override def beforeAll(): Unit = { super.beforeAll() + spark.conf.set(HYPERSPACE_EVENT_LOGGER_CLASS_KEY, "com.microsoft.hyperspace.MockEventLogger") FileUtils.delete(new Path(sampleParquetDataLocation)) SampleData.save( @@ -255,9 +256,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { test("Verify refresh-incremental (append-only) is a no-op if no new files found.") { withTempPathAsString { testPath => - withSQLConf( - IndexConstants.REFRESH_APPEND_ENABLED -> "true", - HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.MockEventLogger") { + withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { // Setup. Create sample data and index. val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) import spark.implicits._ diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 7319328d3..68db70ce6 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -38,7 +38,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { override def beforeAll(): Unit = { super.beforeAll() - + spark.conf.set(HYPERSPACE_EVENT_LOGGER_CLASS_KEY, "com.microsoft.hyperspace.MockEventLogger") hyperspace = new Hyperspace(spark) FileUtils.delete(new Path(testDir)) } @@ -136,8 +136,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true", - HYPERSPACE_EVENT_LOGGER_CLASS_KEY -> "com.microsoft.hyperspace.MockEventLogger") { + IndexConstants.REFRESH_DELETE_ENABLED -> "true") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") From 2c55d8632e7ee71555abd7b2ab29f51a58ed0b96 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 17:31:05 -0700 Subject: [PATCH 54/86] move mock logger setting to beginning of all SparkInvolvedSuite tests --- .../scala/com/microsoft/hyperspace/SparkInvolvedSuite.scala | 3 +++ .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 1 - .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 1 - 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/SparkInvolvedSuite.scala b/src/test/scala/com/microsoft/hyperspace/SparkInvolvedSuite.scala index c22391e90..a480316bc 100644 --- a/src/test/scala/com/microsoft/hyperspace/SparkInvolvedSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/SparkInvolvedSuite.scala @@ -21,6 +21,8 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.SparkSession import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import com.microsoft.hyperspace.telemetry.Constants.HYPERSPACE_EVENT_LOGGER_CLASS_KEY + trait SparkInvolvedSuite extends BeforeAndAfterAll with BeforeAndAfter { self: SparkFunSuite => @@ -31,6 +33,7 @@ trait SparkInvolvedSuite extends BeforeAndAfterAll with BeforeAndAfter { protected lazy val spark: SparkSession = SparkSession .builder() .master(s"local[$numParallelism]") + .config(HYPERSPACE_EVENT_LOGGER_CLASS_KEY, "com.microsoft.hyperspace.MockEventLogger") .appName(suiteName) .getOrCreate() diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 2fae0575f..76e6f27d5 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -40,7 +40,6 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { override def beforeAll(): Unit = { super.beforeAll() - spark.conf.set(HYPERSPACE_EVENT_LOGGER_CLASS_KEY, "com.microsoft.hyperspace.MockEventLogger") FileUtils.delete(new Path(sampleParquetDataLocation)) SampleData.save( diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 68db70ce6..1f0889c96 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -38,7 +38,6 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { override def beforeAll(): Unit = { super.beforeAll() - spark.conf.set(HYPERSPACE_EVENT_LOGGER_CLASS_KEY, "com.microsoft.hyperspace.MockEventLogger") hyperspace = new Hyperspace(spark) FileUtils.delete(new Path(testDir)) } From 138e96acb7561073ad7ce48f15ce1cc714a2e842 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 8 Oct 2020 17:33:18 -0700 Subject: [PATCH 55/86] refactoring Co-authored-by: Terry Kim --- src/test/scala/com/microsoft/hyperspace/TestUtils.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index 5324eb8c3..94f90644c 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -62,8 +62,9 @@ class MockEventLogger extends EventLogger { } object MockEventLogger { + var emittedEvents: Seq[HyperspaceEvent] = Seq() + def reset(): Unit = { emittedEvents = Seq() } - var emittedEvents: Seq[HyperspaceEvent] = Seq() } From 767513f35b24b375f2ea346e037694adba657594 Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Thu, 8 Oct 2020 17:34:02 -0700 Subject: [PATCH 56/86] explicit assert in test Co-authored-by: Terry Kim --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 76e6f27d5..43c6987c5 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -278,10 +278,9 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { // Check emitted events. MockEventLogger.emittedEvents match { case Seq( - RefreshAppendActionEvent(_, _, "Operation started."), - RefreshAppendActionEvent(_, _, msg)) - if msg.equals("No-op operation recorded: Refresh append aborted as" + - " no appended source data files found.") => // pass + RefreshDeleteActionEvent(_, _, "Operation started."), + RefreshDeleteActionEvent(_, _, msg)) => + assert(msg.contains("Refresh delete aborted as no deleted source data file found")) case _ => fail() } } From 7338de5e501de2d2b5f17627ac4f4e499302d53c Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 17:39:48 -0700 Subject: [PATCH 57/86] review comets --- src/test/scala/com/microsoft/hyperspace/TestUtils.scala | 2 +- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index 5324eb8c3..9e853cf97 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -53,7 +53,7 @@ object TestUtils { */ class MockEventLogger extends EventLogger { import com.microsoft.hyperspace.MockEventLogger.emittedEvents - // Reset events for `this` action. + // Reset events. reset() override def logEvent(event: HyperspaceEvent): Unit = { diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 1f0889c96..aca55617c 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -151,9 +151,8 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { MockEventLogger.emittedEvents match { case Seq( RefreshDeleteActionEvent(_, _, "Operation started."), - RefreshDeleteActionEvent(_, _, msg)) - if msg.equals("No-op operation recorded: Refresh delete aborted as " + - "no deleted source data file found.") => // pass + RefreshDeleteActionEvent(_, _, msg)) => + assert(msg.contains("Refresh delete aborted as no deleted source data file found.")) case _ => fail() } } From 29ca9b3b289148b125e14848646ae97001fcfc38 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 17:53:06 -0700 Subject: [PATCH 58/86] test bug fixes --- .../com/microsoft/hyperspace/index/IndexManagerTests.scala | 7 +++---- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 1 - 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index 43c6987c5..ec1d9298b 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} import com.microsoft.hyperspace.TestUtils.copyWithState import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.telemetry.Constants.HYPERSPACE_EVENT_LOGGER_CLASS_KEY import com.microsoft.hyperspace.telemetry.RefreshAppendActionEvent import com.microsoft.hyperspace.util.{FileUtils, PathUtils} @@ -278,9 +277,9 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { // Check emitted events. MockEventLogger.emittedEvents match { case Seq( - RefreshDeleteActionEvent(_, _, "Operation started."), - RefreshDeleteActionEvent(_, _, msg)) => - assert(msg.contains("Refresh delete aborted as no deleted source data file found")) + RefreshAppendActionEvent(_, _, "Operation started."), + RefreshAppendActionEvent(_, _, msg)) => + assert(msg.contains("Refresh append aborted as no appended source data files found.")) case _ => fail() } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index aca55617c..bca72bfc2 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} -import com.microsoft.hyperspace.telemetry.Constants.HYPERSPACE_EVENT_LOGGER_CLASS_KEY import com.microsoft.hyperspace.telemetry.RefreshDeleteActionEvent import com.microsoft.hyperspace.util.{FileUtils, PathUtils} From 9d26ddcf417d6c06f5e2e12bfb209ea70b13ebe2 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 18:08:42 -0700 Subject: [PATCH 59/86] whitespace fix --- src/test/scala/com/microsoft/hyperspace/TestUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index d345c98c8..78b8f6ede 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -63,7 +63,7 @@ class MockEventLogger extends EventLogger { object MockEventLogger { var emittedEvents: Seq[HyperspaceEvent] = Seq() - + def reset(): Unit = { emittedEvents = Seq() } From b43b29e1a554b2ff802f7b2bbc1d2bb2e31eb3e7 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Thu, 8 Oct 2020 19:07:45 -0700 Subject: [PATCH 60/86] add refresh-append test for updating delted files --- .../hyperspace/index/RefreshIndexTests.scala | 44 ++++++++++++++++++- 1 file changed, 42 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 2456da1c7..df170ce25 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -222,8 +222,8 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } test( - "Validate refresh delete action updates appended files as expected," + - "when some file gets deleted and some appended to source data.") { + "Validate refresh delete action updates appended and deleted files in metadata as" + + "expected, when some file gets deleted and some appended to source data.") { withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "true", IndexConstants.REFRESH_DELETE_ENABLED -> "true") { @@ -264,6 +264,46 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } } + test( + "Validate refresh append action updates appended and deleted files in metadata as" + + "expected, when some file gets deleted and some appended to source data.") { + withTempPathAsString { testPath => + withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { + withIndex(indexConfig.indexName) { + SampleData.save(spark, testPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) + val df = spark.read.parquet(testPath) + hyperspace.createIndex(df, indexConfig) + + val oldFileList = fileList(testPath).toSet + + // Delete one source data file. + deleteDataFile(testPath) + + // Add some new data to source. + import spark.implicits._ + SampleData.testData + .take(3) + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .write + .mode("append") + .parquet(testPath) + + hyperspace.refreshIndex(indexConfig.indexName) + + // Check if refreshed index metadata has "appendedFiles" and "deletedFiles" updated. + val entry = logManager(indexConfig.indexName).getLatestStableLog() + assert(entry.isDefined) + assert(entry.get.isInstanceOf[IndexLogEntry]) + val indexLogEntry = entry.get.asInstanceOf[IndexLogEntry] + assert(indexLogEntry.appendedFiles.isEmpty) + + val newFileList = fileList(testPath).toSet + assert(indexLogEntry.deletedFiles.toSet.equals(oldFileList -- newFileList)) + } + } + } + } + /** * Delete one file from a given path. * From 8f55e66d28a4d59ecd2e909f35fe79035d69ded3 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 10:02:40 -0700 Subject: [PATCH 61/86] bug fix in updating index log entry after refresh append and deleted --- .../actions/RefreshAppendAction.scala | 5 +-- .../actions/RefreshDeleteAction.scala | 2 +- .../hyperspace/index/IndexLogEntry.scala | 40 +------------------ 3 files changed, 4 insertions(+), 43 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index 8e846ff0b..a5471a0e2 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -88,10 +88,7 @@ class RefreshAppendAction( // https://github.com/microsoft/hyperspace/issues/183 // New entry. - entry - .copy(content = mergedContent) - .withNewDeletedFiles(deletedFiles) - .withAdditionalAppendedFiles(Seq()) + entry.copy(content = mergedContent).withAppendedAndDeletedFiles(Seq(), deletedFiles) } override protected def event(appInfo: AppInfo, message: String): HyperspaceEvent = { diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala index 3dd49a036..015bed62d 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshDeleteAction.scala @@ -91,6 +91,6 @@ class RefreshDeleteAction( override def logEntry: LogEntry = { val entry = getIndexLogEntry(spark, df, indexConfig, indexDataPath) - entry.withNewDeletedFiles(Seq()).withAdditionalAppendedFiles(appendedFiles) + entry.withAppendedAndDeletedFiles(appendedFiles, Seq()) } } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala index 5dbc2b329..073aa7473 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexLogEntry.scala @@ -397,7 +397,7 @@ case class IndexLogEntry( relations.head.data.properties.appendedFiles } - def withAdditionalAppendedFiles(files: Seq[String]): IndexLogEntry = { + def withAppendedAndDeletedFiles(appended: Seq[String], deleted: Seq[String]): IndexLogEntry = { copy( source = source.copy( plan = source.plan.copy( @@ -406,43 +406,7 @@ case class IndexLogEntry( relations.head.copy( data = relations.head.data.copy( properties = relations.head.data.properties.copy( - appendedFiles = relations.head.data.properties.appendedFiles ++ files)))))))) - } - - def withAdditionalDeletedFiles(files: Seq[String]): IndexLogEntry = { - copy( - source = source.copy( - plan = source.plan.copy( - properties = source.plan.properties.copy( - relations = Seq( - relations.head.copy( - data = relations.head.data.copy( - properties = relations.head.data.properties.copy( - deletedFiles = relations.head.data.properties.deletedFiles ++ files)))))))) - } - - def withNewAppendedFiles(files: Seq[String]): IndexLogEntry = { - copy( - source = source.copy( - plan = source.plan.copy( - properties = source.plan.properties.copy( - relations = Seq( - relations.head.copy( - data = relations.head.data.copy( - properties = relations.head.data.properties.copy( - appendedFiles = files)))))))) - } - - def withNewDeletedFiles(files: Seq[String]): IndexLogEntry = { - copy( - source = source.copy( - plan = source.plan.copy( - properties = source.plan.properties.copy( - relations = Seq( - relations.head.copy( - data = relations.head.data.copy( - properties = relations.head.data.properties.copy( - deletedFiles = files)))))))) + appendedFiles = appended, deletedFiles = deleted)))))))) } def bucketSpec: BucketSpec = From 49f34de162e39e3a138e4e4925cb94222745fb0c Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 11:52:28 -0700 Subject: [PATCH 62/86] remove comment --- .../com/microsoft/hyperspace/actions/RefreshAppendAction.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index a5471a0e2..def6d9f08 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -84,9 +84,6 @@ class RefreshAppendAction( // Merge new index files with old index files. val mergedContent = Content(previousIndexLogEntry.content.root.merge(entry.content.root)) - // TODO: update "appended" and "deleted" list in log entry. - // https://github.com/microsoft/hyperspace/issues/183 - // New entry. entry.copy(content = mergedContent).withAppendedAndDeletedFiles(Seq(), deletedFiles) } From e7da7de4b1e26bf974079cc5fad8b68c837d58be Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 11:56:19 -0700 Subject: [PATCH 63/86] fix scalastyle --- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index b3bbb7821..b7fd944b9 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -20,9 +20,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} -import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteAction} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} -import com.microsoft.hyperspace.telemetry.{CreateActionEvent, RefreshAppendActionEvent, RefreshDeleteActionEvent} +import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteAction} +import com.microsoft.hyperspace.telemetry.{RefreshAppendActionEvent, RefreshDeleteActionEvent} import com.microsoft.hyperspace.util.{FileUtils, PathUtils} /** From 3872fd9ca717028b2875c98ca9a1407c72e4c2ac Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 16:51:02 -0700 Subject: [PATCH 64/86] initial commit --- .../com/microsoft/hyperspace/index/rules/RuleUtils.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala b/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala index aeb7eb4ed..f47bce98c 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/rules/RuleUtils.scala @@ -107,7 +107,10 @@ object RuleUtils { indexes.filter(index => index.created && isHybridScanCandidate(index, filesByRelations.flatten)) } else { - indexes.filter(index => index.created && signatureValid(index)) + indexes.filter( + index => + index.created && signatureValid(index) && + index.deletedFiles.isEmpty && index.appendedFiles.isEmpty) } } From f5927ac21856d02a0300439090522567f2b1ad1f Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 19:24:26 -0700 Subject: [PATCH 65/86] review comments --- .../actions/RefreshActionBase.scala | 13 +++ .../hyperspace/index/RefreshIndexTests.scala | 96 ++++++++++--------- 2 files changed, 63 insertions(+), 46 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala index 36041f821..e91ed79ad 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshActionBase.scala @@ -83,6 +83,8 @@ private[actions] abstract class RefreshActionBase( * Compare list of source data files from previous IndexLogEntry to list * of current source data files, validate fileInfo for existing files and * identify deleted source data files. + * Finally, append the previously known deleted files to the result. These + * are the files for which the index was never updated in the past. */ protected lazy val deletedFiles: Seq[String] = { val rels = previousIndexLogEntry.relations @@ -109,9 +111,18 @@ private[actions] abstract class RefreshActionBase( } } + // TODO: Add test for the scenario where existing deletedFiles and newly deleted + // files are updated. https://github.com/microsoft/hyperspace/issues/195. delFiles ++ previousIndexLogEntry.deletedFiles } + /** + * Compare list of source data files from previous IndexLogEntry to list + * of current source data files, validate fileInfo for existing files and + * identify newly appended source data files. + * Finally, append the previously known appended files to the result. These + * are the files for which index was never updated in the past. + */ protected lazy val appendedFiles = { val relation = previousIndexLogEntry.relations.head @@ -128,6 +139,8 @@ private[actions] abstract class RefreshActionBase( location.allFiles().map(_.getPath.toString) }.flatten + // TODO: Add test for the scenario where existing appendedFiles and newly appended + // files are updated. https://github.com/microsoft/hyperspace/issues/195. allFiles.diff(originalFiles) ++ previousIndexLogEntry.appendedFiles } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index df170ce25..df3b92813 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -222,44 +222,44 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } test( - "Validate refresh delete action updates appended and deleted files in metadata as" + + "Validate refresh delete action updates appended and deleted files in metadata as " + "expected, when some file gets deleted and some appended to source data.") { - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { - withIndex(indexConfig.indexName) { - SampleData.save( - spark, - nonPartitionedDataPath, - Seq("Date", "RGUID", "Query", "imprs", "clicks")) - val df = spark.read.parquet(nonPartitionedDataPath) - hyperspace.createIndex(df, indexConfig) - - // Delete one source data file. - deleteDataFile(nonPartitionedDataPath) - - val oldFileList = fileList(nonPartitionedDataPath).toSet - - // Add some new data to source. - import spark.implicits._ - SampleData.testData - .take(3) - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .write - .mode("append") - .parquet(nonPartitionedDataPath) - - hyperspace.refreshIndex(indexConfig.indexName) - - // Check if refreshed index metadata has "appendedFiles" and "deletedFiles" updated. - val entry = logManager(indexConfig.indexName).getLatestStableLog() - assert(entry.isDefined) - assert(entry.get.isInstanceOf[IndexLogEntry]) - val indexLogEntry = entry.get.asInstanceOf[IndexLogEntry] - assert(indexLogEntry.deletedFiles.isEmpty) - - val newFileList = fileList(nonPartitionedDataPath).toSet - assert(indexLogEntry.appendedFiles.toSet.equals(newFileList -- oldFileList)) + withTempPathAsString { nonPartitionedDataPath => + withSQLConf( + IndexConstants.INDEX_LINEAGE_ENABLED -> "true", + IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withIndex(indexConfig.indexName) { + SampleData.save( + spark, + nonPartitionedDataPath, + Seq("Date", "RGUID", "Query", "imprs", "clicks")) + val df = spark.read.parquet(nonPartitionedDataPath) + hyperspace.createIndex(df, indexConfig) + + // Delete one source data file. + deleteDataFile(nonPartitionedDataPath) + + val oldFiles = listFiles(nonPartitionedDataPath).toSet + + // Add some new data to source. + import spark.implicits._ + SampleData.testData + .take(3) + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .write + .mode("append") + .parquet(nonPartitionedDataPath) + + hyperspace.refreshIndex(indexConfig.indexName) + + // Verify "deletedFiles" is cleared and "appendedFiles" is updated after refresh. + val indexLogEntry = getLatestStableLog(indexConfig.indexName) + val latestFiles = listFiles(nonPartitionedDataPath).toSet + + assert(indexLogEntry.deletedFiles.isEmpty) + assert((oldFiles -- latestFiles).isEmpty) + assert(indexLogEntry.appendedFiles.toSet.equals(latestFiles -- oldFiles)) + } } } } @@ -274,7 +274,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val df = spark.read.parquet(testPath) hyperspace.createIndex(df, indexConfig) - val oldFileList = fileList(testPath).toSet + val oldFiles = listFiles(testPath).toSet // Delete one source data file. deleteDataFile(testPath) @@ -290,15 +290,12 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { hyperspace.refreshIndex(indexConfig.indexName) - // Check if refreshed index metadata has "appendedFiles" and "deletedFiles" updated. - val entry = logManager(indexConfig.indexName).getLatestStableLog() - assert(entry.isDefined) - assert(entry.get.isInstanceOf[IndexLogEntry]) - val indexLogEntry = entry.get.asInstanceOf[IndexLogEntry] + // Verify "appendedFiles" is cleared and "deletedFiles" is updated after refresh. + val indexLogEntry = getLatestStableLog(indexConfig.indexName) assert(indexLogEntry.appendedFiles.isEmpty) - val newFileList = fileList(testPath).toSet - assert(indexLogEntry.deletedFiles.toSet.equals(oldFileList -- newFileList)) + val latestFiles = listFiles(testPath).toSet + assert(indexLogEntry.deletedFiles.toSet.equals(oldFiles -- latestFiles)) } } } @@ -335,9 +332,16 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { IndexLogManagerFactoryImpl.create(indexPath) } - private def fileList(path: String): Seq[String] = { + private def listFiles(path: String): Seq[String] = { val absolutePath = PathUtils.makeAbsolute(path) val fs = absolutePath.getFileSystem(new Configuration) fs.listStatus(absolutePath).toSeq.map(_.getPath.toString) } + + private def getLatestStableLog(indexName: String): IndexLogEntry = { + val entry = logManager(indexName).getLatestStableLog() + assert(entry.isDefined) + assert(entry.get.isInstanceOf[IndexLogEntry]) + entry.get.asInstanceOf[IndexLogEntry] + } } From 521edcb589178ece4dca475d871eda5c0be42779 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 19:30:20 -0700 Subject: [PATCH 66/86] rename test path --- .../hyperspace/index/RefreshIndexTests.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index df3b92813..074bb5ec4 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -224,22 +224,22 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { test( "Validate refresh delete action updates appended and deleted files in metadata as " + "expected, when some file gets deleted and some appended to source data.") { - withTempPathAsString { nonPartitionedDataPath => + withTempPathAsString { testPath => withSQLConf( IndexConstants.INDEX_LINEAGE_ENABLED -> "true", IndexConstants.REFRESH_DELETE_ENABLED -> "true") { withIndex(indexConfig.indexName) { SampleData.save( spark, - nonPartitionedDataPath, + testPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) - val df = spark.read.parquet(nonPartitionedDataPath) + val df = spark.read.parquet(testPath) hyperspace.createIndex(df, indexConfig) // Delete one source data file. - deleteDataFile(nonPartitionedDataPath) + deleteDataFile(testPath) - val oldFiles = listFiles(nonPartitionedDataPath).toSet + val oldFiles = listFiles(testPath).toSet // Add some new data to source. import spark.implicits._ @@ -248,13 +248,13 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { .toDF("Date", "RGUID", "Query", "imprs", "clicks") .write .mode("append") - .parquet(nonPartitionedDataPath) + .parquet(testPath) hyperspace.refreshIndex(indexConfig.indexName) // Verify "deletedFiles" is cleared and "appendedFiles" is updated after refresh. val indexLogEntry = getLatestStableLog(indexConfig.indexName) - val latestFiles = listFiles(nonPartitionedDataPath).toSet + val latestFiles = listFiles(testPath).toSet assert(indexLogEntry.deletedFiles.isEmpty) assert((oldFiles -- latestFiles).isEmpty) From 2080edb865e5e49a4475b160410a4dcd16b8a920 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 21:00:55 -0700 Subject: [PATCH 67/86] add test for "deletedFiles" in entry --- .../index/rules/HyperspaceRuleTestSuite.scala | 6 ++-- .../index/rules/RuleUtilsTest.scala | 32 +++++++++++++++++-- 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala index 4b5c29b98..5a0133e7b 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala @@ -34,7 +34,9 @@ trait HyperspaceRuleTestSuite extends HyperspaceSuite { name: String, indexCols: Seq[AttributeReference], includedCols: Seq[AttributeReference], - plan: LogicalPlan): IndexLogEntry = { + plan: LogicalPlan, + appendedFiles: Seq[String] = Seq(), + deletedFiles: Seq[String] = Seq()): IndexLogEntry = { val signClass = new RuleTestHelper.TestSignatureProvider().getClass.getName LogicalPlanSignatureProvider.create(signClass).signature(plan) match { @@ -43,7 +45,7 @@ trait HyperspaceRuleTestSuite extends HyperspaceSuite { Seq( Relation( Seq("dummy"), - Hdfs(Properties(Content(Directory("/")))), + Hdfs(Properties(Content(Directory("/")), appendedFiles, deletedFiles)), "schema", "format", Map())), diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index 757592ca2..1e4ffe992 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFil import org.apache.spark.sql.types.{IntegerType, StringType} import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants} -import com.microsoft.hyperspace.util.{FileUtils, PathUtils} +import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants, LogicalPlanSignatureProvider} +import com.microsoft.hyperspace.util.{FileUtils, HyperspaceConf, PathUtils} class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { override val systemPath = PathUtils.makeAbsolute("src/test/resources/ruleUtilsTest") @@ -280,6 +280,34 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { } } + test( + "Verify no indexes are matched if signature matches but hybrid scan is disabled and" + + " 'deletedFiles' is non-empty.") { + assert(!HyperspaceConf.hybridScanEnabled(spark)) + // Here's an index where the singature is in sync with the latest data, but "deletedFiles" + // list is non-empty. + val entry = + createIndex("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode, deletedFiles = Seq("f1")) + + // Assert that signature of this new index matches with the latest data. + // Below is the logic for signature calculation is picked from RuleUtils class. + assert( + entry.signature.value.equals( + LogicalPlanSignatureProvider + .create(entry.signature.provider) + .signature(t1ProjectNode) + .get)) + + val indexManager = IndexCollectionManager(spark) + val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) + + val usableIndexes = RuleUtils.getCandidateIndexes(spark, allIndexes, t1ProjectNode) + assert(usableIndexes.length === 3) + // Verify that even if signature matched, this index is not picked because of non-empty + // "deleted" files. + assert(!usableIndexes.exists(_.name.equals("t1iTest"))) + } + private def validateLogicalRelation(plan: LogicalPlan, expected: LogicalRelation): Unit = { val r = RuleUtils.getLogicalRelation(plan) assert(r.isDefined) From 3af06eba0cc2858de0b847143b616ad7ffa81e52 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 21:17:02 -0700 Subject: [PATCH 68/86] unit tests added --- .../index/rules/RuleUtilsTest.scala | 34 +++++++++++++++++-- 1 file changed, 31 insertions(+), 3 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index 1e4ffe992..4e5c01dd8 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -281,8 +281,8 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { } test( - "Verify no indexes are matched if signature matches but hybrid scan is disabled and" + - " 'deletedFiles' is non-empty.") { + "RuleUtils.getCandidateIndexes: Verify no indexes are matched even if signature matches but " + + "hybrid scan is disabled and 'deletedFiles' is non-empty.") { assert(!HyperspaceConf.hybridScanEnabled(spark)) // Here's an index where the singature is in sync with the latest data, but "deletedFiles" // list is non-empty. @@ -300,9 +300,37 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { val indexManager = IndexCollectionManager(spark) val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) + assert(allIndexes.exists(_.name.equals("t1iTest"))) + + val usableIndexes = RuleUtils.getCandidateIndexes(spark, allIndexes, t1ProjectNode) + // Verify that even if signature matched, this index is not picked because of non-empty + // "deleted" files. + assert(!usableIndexes.exists(_.name.equals("t1iTest"))) + } + + test( + "RuleUtils.getCandidateIndexes: Verify no indexes are matched even if signature matches but " + + "hybrid scan is disabled and 'appendedFiles' is non-empty.") { + assert(!HyperspaceConf.hybridScanEnabled(spark)) + // Here's an index where the singature is in sync with the latest data, but "deletedFiles" + // list is non-empty. + val entry = + createIndex("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode, appendedFiles = Seq("f1")) + + // Assert that signature of this new index matches with the latest data. + // Below is the logic for signature calculation is picked from RuleUtils class. + assert( + entry.signature.value.equals( + LogicalPlanSignatureProvider + .create(entry.signature.provider) + .signature(t1ProjectNode) + .get)) + + val indexManager = IndexCollectionManager(spark) + val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) + assert(allIndexes.exists(_.name.equals("t1iTest"))) val usableIndexes = RuleUtils.getCandidateIndexes(spark, allIndexes, t1ProjectNode) - assert(usableIndexes.length === 3) // Verify that even if signature matched, this index is not picked because of non-empty // "deleted" files. assert(!usableIndexes.exists(_.name.equals("t1iTest"))) From 315b36dd4dc45224db25a5c1c5455de6e28e5494 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 23:35:50 -0700 Subject: [PATCH 69/86] review comments, test fixes --- .../index/rules/FilterIndexRuleTest.scala | 4 +- .../index/rules/HyperspaceRuleTestSuite.scala | 2 +- .../index/rules/JoinIndexRuleTest.scala | 20 +++--- .../index/rules/RuleUtilsTest.scala | 69 ++++--------------- 4 files changed, 26 insertions(+), 69 deletions(-) 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 5e66afd4c..44d628666 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala @@ -51,10 +51,10 @@ class FilterIndexRuleTest extends HyperspaceRuleTestSuite { scanNode = LogicalRelation(relation, Seq(c1, c2, c3, c4), None, false) val indexPlan = Project(Seq(c1, c2, c3), scanNode) - createIndex(indexName1, Seq(c3, c2), Seq(c1), indexPlan) + createIndexLogEntry(indexName1, Seq(c3, c2), Seq(c1), indexPlan) val index2Plan = Project(Seq(c1, c2, c3, c4), scanNode) - createIndex(indexName2, Seq(c4, c2), Seq(c1, c3), index2Plan) + createIndexLogEntry(indexName2, Seq(c4, c2), Seq(c1, c3), index2Plan) } before { diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala index 5a0133e7b..56a144142 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala @@ -30,7 +30,7 @@ import com.microsoft.hyperspace.index.Hdfs.Properties trait HyperspaceRuleTestSuite extends HyperspaceSuite { private val filenames = Seq("f1.parquet", "f2.parquet") - def createIndex( + def createIndexLogEntry( name: String, indexCols: Seq[AttributeReference], includedCols: Seq[AttributeReference], 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 c5a51eea1..fb31302b2 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/JoinIndexRuleTest.scala @@ -90,11 +90,11 @@ class JoinIndexRuleTest extends HyperspaceRuleTestSuite with SQLHelper { // +- Filter isnotnull(t2c1#4) // +- Relation[t2c1#4,t2c2#5,t2c3#6,t2c4#7] parquet - createIndex("t1i1", Seq(t1c1), Seq(t1c3), t1ProjectNode) - createIndex("t1i2", Seq(t1c1, t1c2), Seq(t1c3), t1ProjectNode) - createIndex("t1i3", Seq(t1c2), Seq(t1c3), t1ProjectNode) - createIndex("t2i1", Seq(t2c1), Seq(t2c3), t2ProjectNode) - createIndex("t2i2", Seq(t2c1, t2c2), Seq(t2c3), t2ProjectNode) + createIndexLogEntry("t1i1", Seq(t1c1), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t1i2", Seq(t1c1, t1c2), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t1i3", Seq(t1c2), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t2i1", Seq(t2c1), Seq(t2c3), t2ProjectNode) + createIndexLogEntry("t2i2", Seq(t2c1, t2c2), Seq(t2c3), t2ProjectNode) } before { @@ -208,8 +208,10 @@ class JoinIndexRuleTest extends HyperspaceRuleTestSuite with SQLHelper { { // Test: should update plan if index exists to cover all implicit columns - val t1TestIndex = createIndex("t1Idx", Seq(t1c1), Seq(t1c2, t1c3, t1c4), t1FilterNode) - val t2TestIndex = createIndex("t2Idx", Seq(t2c1), Seq(t2c2, t2c3, t2c4), t2FilterNode) + val t1TestIndex = + createIndexLogEntry("t1Idx", Seq(t1c1), Seq(t1c2, t1c3, t1c4), t1FilterNode) + val t2TestIndex = + createIndexLogEntry("t2Idx", Seq(t2c1), Seq(t2c2, t2c3, t2c4), t2FilterNode) // clear cache so the new indexes gets added to it clearCache() @@ -406,9 +408,7 @@ class JoinIndexRuleTest extends HyperspaceRuleTestSuite with SQLHelper { // Mark the relation that the rule is applied and verify the plan does not change. val newPlan = plan transform { case r @ LogicalRelation(h: HadoopFsRelation, _, _, _) => - r.copy( - relation = - h.copy(options = Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark)) + r.copy(relation = h.copy(options = Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark)) } assert(JoinIndexRule(newPlan).equals(newPlan)) } diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index 4e5c01dd8..e4d186c8c 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -20,12 +20,12 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, IsNotNull} import org.apache.spark.sql.catalyst.plans.{JoinType, SQLHelper} -import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project, RepartitionByExpression} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation, NoopCache} import org.apache.spark.sql.types.{IntegerType, StringType} import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants, LogicalPlanSignatureProvider} +import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants} import com.microsoft.hyperspace.util.{FileUtils, HyperspaceConf, PathUtils} class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { @@ -79,11 +79,11 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { // +- Filter isnotnull(t2c1#4) // +- Relation[t2c1#4,t2c2#5,t2c3#6,t2c4#7] parquet - createIndex("t1i1", Seq(t1c1), Seq(t1c3), t1ProjectNode) - createIndex("t1i2", Seq(t1c1, t1c2), Seq(t1c3), t1ProjectNode) - createIndex("t1i3", Seq(t1c2), Seq(t1c3), t1ProjectNode) - createIndex("t2i1", Seq(t2c1), Seq(t2c3), t2ProjectNode) - createIndex("t2i2", Seq(t2c1, t2c2), Seq(t2c3), t2ProjectNode) + createIndexLogEntry("t1i1", Seq(t1c1), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t1i2", Seq(t1c1, t1c2), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t1i3", Seq(t1c2), Seq(t1c3), t1ProjectNode) + createIndexLogEntry("t2i1", Seq(t2c1), Seq(t2c3), t2ProjectNode) + createIndexLogEntry("t2i2", Seq(t2c1, t2c2), Seq(t2c3), t2ProjectNode) } test("Verify indexes are matched by signature correctly.") { @@ -284,56 +284,13 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { "RuleUtils.getCandidateIndexes: Verify no indexes are matched even if signature matches but " + "hybrid scan is disabled and 'deletedFiles' is non-empty.") { assert(!HyperspaceConf.hybridScanEnabled(spark)) - // Here's an index where the singature is in sync with the latest data, but "deletedFiles" - // list is non-empty. - val entry = - createIndex("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode, deletedFiles = Seq("f1")) - - // Assert that signature of this new index matches with the latest data. - // Below is the logic for signature calculation is picked from RuleUtils class. - assert( - entry.signature.value.equals( - LogicalPlanSignatureProvider - .create(entry.signature.provider) - .signature(t1ProjectNode) - .get)) - val indexManager = IndexCollectionManager(spark) - val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) - assert(allIndexes.exists(_.name.equals("t1iTest"))) - - val usableIndexes = RuleUtils.getCandidateIndexes(spark, allIndexes, t1ProjectNode) - // Verify that even if signature matched, this index is not picked because of non-empty - // "deleted" files. - assert(!usableIndexes.exists(_.name.equals("t1iTest"))) - } - - test( - "RuleUtils.getCandidateIndexes: Verify no indexes are matched even if signature matches but " + - "hybrid scan is disabled and 'appendedFiles' is non-empty.") { - assert(!HyperspaceConf.hybridScanEnabled(spark)) - // Here's an index where the singature is in sync with the latest data, but "deletedFiles" - // list is non-empty. - val entry = - createIndex("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode, appendedFiles = Seq("f1")) - - // Assert that signature of this new index matches with the latest data. - // Below is the logic for signature calculation is picked from RuleUtils class. - assert( - entry.signature.value.equals( - LogicalPlanSignatureProvider - .create(entry.signature.provider) - .signature(t1ProjectNode) - .get)) - - val indexManager = IndexCollectionManager(spark) - val allIndexes = indexManager.getIndexes(Seq(Constants.States.ACTIVE)) - assert(allIndexes.exists(_.name.equals("t1iTest"))) - - val usableIndexes = RuleUtils.getCandidateIndexes(spark, allIndexes, t1ProjectNode) - // Verify that even if signature matched, this index is not picked because of non-empty - // "deleted" files. - assert(!usableIndexes.exists(_.name.equals("t1iTest"))) + val entry1 = createIndexLogEntry("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode) + val entry2 = entry1.withAppendedAndDeletedFiles(Seq(), Seq("f1")) + val entry3 = entry1.withAppendedAndDeletedFiles(Seq("f2"), Seq()) + val usableIndexes = + RuleUtils.getCandidateIndexes(spark, Seq(entry1, entry2, entry3), t1ProjectNode) + assert(usableIndexes.equals(Seq(entry1))) } private def validateLogicalRelation(plan: LogicalPlan, expected: LogicalRelation): Unit = { From 42de97a4b1ae4643ba12196cac6ba16b84488b3e Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 23:39:33 -0700 Subject: [PATCH 70/86] rephrase test name --- .../com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index e4d186c8c..d9c24c191 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -281,8 +281,8 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { } test( - "RuleUtils.getCandidateIndexes: Verify no indexes are matched even if signature matches but " + - "hybrid scan is disabled and 'deletedFiles' is non-empty.") { + "RuleUtils.getCandidateIndexes: Verify indexes with non-empty 'deletedFiles' or " + + "'appendedFiles' are not usable indexes if hybrid scan is disabled.") { assert(!HyperspaceConf.hybridScanEnabled(spark)) val entry1 = createIndexLogEntry("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode) From 6d1f0fcc89747d62ce806afbe7913a793b5fcf89 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Fri, 9 Oct 2020 23:40:43 -0700 Subject: [PATCH 71/86] autoformat result --- .../hyperspace/index/rules/FilterIndexRuleTest.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 44d628666..dadee7dab 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/FilterIndexRuleTest.scala @@ -134,9 +134,7 @@ class FilterIndexRuleTest extends HyperspaceRuleTestSuite { // Mark the relation that the rule is applied and verify the plan does not change. val newPlan = plan transform { case r @ LogicalRelation(h: HadoopFsRelation, _, _, _) => - r.copy( - relation = - h.copy(options = Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark)) + r.copy(relation = h.copy(options = Map(IndexConstants.INDEX_RELATION_IDENTIFIER))(spark)) } assert(FilterIndexRule(newPlan).equals(newPlan)) } From 5b232d4bca6847cf439a6883a78335d941bfca8b Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 00:08:52 -0700 Subject: [PATCH 72/86] final review commments --- .../index/rules/HyperspaceRuleTestSuite.scala | 2 +- .../index/rules/RuleUtilsTest.scala | 19 ++++++++++--------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala index 56a144142..d4f4f4471 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala @@ -71,7 +71,7 @@ trait HyperspaceRuleTestSuite extends HyperspaceSuite { val logManager = new IndexLogManagerImpl(getIndexRootPath(name)) indexLogEntry.state = Constants.States.ACTIVE - logManager.writeLog(0, indexLogEntry) + assert(logManager.writeLog(0, indexLogEntry)) indexLogEntry case None => throw HyperspaceException("Invalid plan for index dataFrame.") diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index d9c24c191..1932bd081 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -26,7 +26,8 @@ import org.apache.spark.sql.types.{IntegerType, StringType} import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants} -import com.microsoft.hyperspace.util.{FileUtils, HyperspaceConf, PathUtils} +import com.microsoft.hyperspace.index.IndexConstants.INDEX_HYBRID_SCAN_ENABLED +import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { override val systemPath = PathUtils.makeAbsolute("src/test/resources/ruleUtilsTest") @@ -283,14 +284,14 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { test( "RuleUtils.getCandidateIndexes: Verify indexes with non-empty 'deletedFiles' or " + "'appendedFiles' are not usable indexes if hybrid scan is disabled.") { - assert(!HyperspaceConf.hybridScanEnabled(spark)) - - val entry1 = createIndexLogEntry("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode) - val entry2 = entry1.withAppendedAndDeletedFiles(Seq(), Seq("f1")) - val entry3 = entry1.withAppendedAndDeletedFiles(Seq("f2"), Seq()) - val usableIndexes = - RuleUtils.getCandidateIndexes(spark, Seq(entry1, entry2, entry3), t1ProjectNode) - assert(usableIndexes.equals(Seq(entry1))) + withSQLConf(INDEX_HYBRID_SCAN_ENABLED -> "false") { + val entry1 = createIndexLogEntry("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode) + val entry2 = entry1.withAppendedAndDeletedFiles(Seq(), Seq("f1")) + val entry3 = entry1.withAppendedAndDeletedFiles(Seq("f2"), Seq()) + val usableIndexes = + RuleUtils.getCandidateIndexes(spark, Seq(entry1, entry2, entry3), t1ProjectNode) + assert(usableIndexes.equals(Seq(entry1))) + } } private def validateLogicalRelation(plan: LogicalPlan, expected: LogicalRelation): Unit = { From 03f23da184c5bf71f384972853556543a0f82c3b Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 00:16:25 -0700 Subject: [PATCH 73/86] review comments --- .../index/rules/HyperspaceRuleTestSuite.scala | 2 +- .../index/rules/RuleUtilsTest.scala | 19 ++++++++++--------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala index 56a144142..d4f4f4471 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/HyperspaceRuleTestSuite.scala @@ -71,7 +71,7 @@ trait HyperspaceRuleTestSuite extends HyperspaceSuite { val logManager = new IndexLogManagerImpl(getIndexRootPath(name)) indexLogEntry.state = Constants.States.ACTIVE - logManager.writeLog(0, indexLogEntry) + assert(logManager.writeLog(0, indexLogEntry)) indexLogEntry case None => throw HyperspaceException("Invalid plan for index dataFrame.") diff --git a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala index d9c24c191..1932bd081 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/rules/RuleUtilsTest.scala @@ -26,7 +26,8 @@ import org.apache.spark.sql.types.{IntegerType, StringType} import com.microsoft.hyperspace.actions.Constants import com.microsoft.hyperspace.index.{IndexCollectionManager, IndexConfig, IndexConstants} -import com.microsoft.hyperspace.util.{FileUtils, HyperspaceConf, PathUtils} +import com.microsoft.hyperspace.index.IndexConstants.INDEX_HYBRID_SCAN_ENABLED +import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { override val systemPath = PathUtils.makeAbsolute("src/test/resources/ruleUtilsTest") @@ -283,14 +284,14 @@ class RuleUtilsTest extends HyperspaceRuleTestSuite with SQLHelper { test( "RuleUtils.getCandidateIndexes: Verify indexes with non-empty 'deletedFiles' or " + "'appendedFiles' are not usable indexes if hybrid scan is disabled.") { - assert(!HyperspaceConf.hybridScanEnabled(spark)) - - val entry1 = createIndexLogEntry("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode) - val entry2 = entry1.withAppendedAndDeletedFiles(Seq(), Seq("f1")) - val entry3 = entry1.withAppendedAndDeletedFiles(Seq("f2"), Seq()) - val usableIndexes = - RuleUtils.getCandidateIndexes(spark, Seq(entry1, entry2, entry3), t1ProjectNode) - assert(usableIndexes.equals(Seq(entry1))) + withSQLConf(INDEX_HYBRID_SCAN_ENABLED -> "false") { + val entry1 = createIndexLogEntry("t1iTest", Seq(t1c1), Seq(t1c3), t1ProjectNode) + val entry2 = entry1.withAppendedAndDeletedFiles(Seq(), Seq("f1")) + val entry3 = entry1.withAppendedAndDeletedFiles(Seq("f2"), Seq()) + val usableIndexes = + RuleUtils.getCandidateIndexes(spark, Seq(entry1, entry2, entry3), t1ProjectNode) + assert(usableIndexes.equals(Seq(entry1))) + } } private def validateLogicalRelation(plan: LogicalPlan, expected: LogicalRelation): Unit = { From 16f0e2929dece3c0a536b76741bc37426c0bd105 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 00:52:06 -0700 Subject: [PATCH 74/86] review comments, refactoring, reset default mode --- .../com/microsoft/hyperspace/Hyperspace.scala | 15 +++++++++++++-- .../index/CachingIndexCollectionManager.scala | 2 +- .../hyperspace/index/IndexCollectionManager.scala | 8 +++++--- .../index/IndexCollectionManagerTest.scala | 3 ++- 4 files changed, 21 insertions(+), 7 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala index 9678e7ef4..782a5e2e8 100644 --- a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala +++ b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala @@ -19,7 +19,7 @@ package com.microsoft.hyperspace import org.apache.spark.sql.{DataFrame, SparkSession} import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL +import com.microsoft.hyperspace.index.IndexConstants.{REFRESH_MODE_FULL, REFRESH_MODE_INCREMENTAL} import com.microsoft.hyperspace.index.plananalysis.PlanAnalyzer class Hyperspace(spark: SparkSession) { @@ -74,7 +74,18 @@ class Hyperspace(spark: SparkSession) { * * @param indexName Name of the index to refresh. */ - def refreshIndex(indexName: String, mode: String = REFRESH_MODE_INCREMENTAL): Unit = { + def refreshIndex(indexName: String): Unit = { + refreshIndex(indexName, REFRESH_MODE_FULL) + } + + /** + * Update indexes for the latest version of the data. This api provides a few + * + * @param indexName Name of the index to refresh. + * @param mode Refresh mode. Currently supported modes are [[REFRESH_MODE_INCREMENTAL]] and + * [[REFRESH_MODE_FULL]] + */ + def refreshIndex(indexName: String, mode: String): Unit = { indexManager.refresh(indexName, mode) } diff --git a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala index ec283b7a5..7c6d230a6 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala @@ -94,7 +94,7 @@ class CachingIndexCollectionManager( super.vacuum(indexName) } - override def refresh(indexName: String, mode: String = REFRESH_MODE_INCREMENTAL): Unit = { + override def refresh(indexName: String, mode: String): Unit = { clearCache() super.refresh(indexName, mode) } diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index 74f2e4c76..983ffdfc7 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.internal.SQLConf import com.microsoft.hyperspace.HyperspaceException import com.microsoft.hyperspace.actions._ -import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL +import com.microsoft.hyperspace.index.IndexConstants.{REFRESH_MODE_FULL, REFRESH_MODE_INCREMENTAL} class IndexCollectionManager( spark: SparkSession, @@ -63,15 +63,17 @@ class IndexCollectionManager( } } - override def refresh(indexName: String, mode: String = REFRESH_MODE_INCREMENTAL): Unit = { + override def refresh(indexName: String, mode: String): Unit = { withLogManager(indexName) { logManager => val indexPath = PathResolver(spark.sessionState.conf).getIndexPath(indexName) val dataManager = indexDataManagerFactory.create(indexPath) if (mode.equals(REFRESH_MODE_INCREMENTAL)) { new RefreshDeleteAction(spark, logManager, dataManager).run() new RefreshAppendAction(spark, logManager, dataManager).run() - } else { + } else if (mode.equals(REFRESH_MODE_FULL)) { new RefreshAction(spark, logManager, dataManager).run() + } else { + throw HyperspaceException(s"Unsupported refresh mode $mode found.") } } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala index 8b86e658e..8e29885ec 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala @@ -23,6 +23,7 @@ import org.mockito.Mockito.{mock, when} import com.microsoft.hyperspace.{HyperspaceException, SparkInvolvedSuite} import com.microsoft.hyperspace.actions.Constants +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_FULL class IndexCollectionManagerTest extends SparkFunSuite with SparkInvolvedSuite { private val indexSystemPath = "src/test/resources/indexLocation" @@ -134,6 +135,6 @@ class IndexCollectionManagerTest extends SparkFunSuite with SparkInvolvedSuite { test("refresh() throws exception if index is not found") { when(mockFileSystem.exists(new Path(indexSystemPath, "idx4"))).thenReturn(false) - intercept[HyperspaceException](indexCollectionManager.refresh("idx4")) + intercept[HyperspaceException](indexCollectionManager.refresh("idx4", REFRESH_MODE_FULL)) } } From dbfa5332182cb4c862ea4735bc2eed2385809185 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 01:07:07 -0700 Subject: [PATCH 75/86] test fixes with default modes --- .../index/E2EHyperspaceRulesTests.scala | 14 +++-- .../hyperspace/index/IndexManagerTests.scala | 6 +-- .../hyperspace/index/RefreshIndexTests.scala | 52 +++++++++++-------- 3 files changed, 38 insertions(+), 34 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index d804b8099..7e2147783 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFil import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData} +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.index.execution.BucketUnionStrategy import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule} import com.microsoft.hyperspace.util.{FileUtils, PathUtils} @@ -409,7 +410,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { sortedRowsWithHyperspaceDisabled.sameElements(getSortedRows(dfAfterHyperspaceDisabled))) } - test("Verify Join Index Rule utilizes indexes correctly after incremental refresh.") { + test("Verify JoinIndexRule utilizes indexes correctly after incremental refresh.") { withTempPathAsString { testPath => withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { // Setup. Create data. @@ -434,7 +435,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { .parquet(testPath) // Refresh index. - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) // Create a join query. val leftDf = spark.read.parquet(testPath) @@ -490,11 +491,8 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { assert(!spark.isHyperspaceEnabled()) } - test("Validate index usage after refresh with some source data file deleted.") { - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { - + test("Validate index usage after incremental refresh with some source data file deleted.") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { // Save a copy of source data files. val location = testDir + "ixRefreshTest" val dataPath = new Path(location, "*parquet") @@ -532,7 +530,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { assert(planRootPaths.equals(Seq(PathUtils.makeAbsolute(location)))) // Refresh the index to remove deleted source data file records from index. - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) // Verify index usage on latest version of index (v=1) after refresh. verifyIndexUsage(query2, getIndexFilesPath(indexConfig.indexName, Seq(1))) diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index cac2bca1d..e8bcfd820 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, SampleData} import com.microsoft.hyperspace.TestUtils.copyWithState import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_FULL +import com.microsoft.hyperspace.index.IndexConstants.{REFRESH_MODE_FULL, REFRESH_MODE_INCREMENTAL} import com.microsoft.hyperspace.util.{FileUtils, PathUtils} class IndexManagerTests extends HyperspaceSuite with SQLHelper { @@ -252,7 +252,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { } } - test("Verify refresh-incremental (append-only) should index only newly appended data.") { + test("Verify incremental refresh (append-only) should index only newly appended data.") { withTempPathAsString { testPath => withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { // Setup. Create sample data and index. @@ -281,7 +281,7 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { .write .mode("append") .parquet(testPath) - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) indexCount = spark.read .parquet(s"$systemPath/index" + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index af5f8f0ec..ccc4843d6 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteAction} +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.telemetry.{RefreshAppendActionEvent, RefreshDeleteActionEvent} import com.microsoft.hyperspace.util.{FileUtils, PathUtils} @@ -52,7 +53,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { FileUtils.delete(systemPath) } - test("Validate refresh index when some file gets deleted from the source data.") { + test("Validate incremental refresh index when some file gets deleted from the source data.") { // Save test data non-partitioned. SampleData.save( spark, @@ -90,7 +91,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val originalIndexWithoutDeletedFile = originalIndexDF .filter(s"""${IndexConstants.DATA_FILE_NAME_COLUMN} != "$deletedFile"""") - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) val refreshedIndexDF = spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") @@ -102,7 +103,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } test( - "Validate refresh index (to handle deletes from the source data) " + + "Validate incremental refresh index (to handle deletes from the source data) " + "fails as expected on an index without lineage.") { SampleData.save( spark, @@ -117,14 +118,17 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { deleteDataFile(nonPartitionedDataPath) - val ex = intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) + val ex = intercept[HyperspaceException]( + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL)) assert( ex.getMessage.contains(s"Index refresh (to handle deleted source data) is " + "only supported on an index with lineage.")) } } - test("Validate refresh index is a no-op if no source data file is deleted or appended.") { + test( + "Validate incremental refresh index is a no-op if no source data file is deleted or " + + "appended.") { SampleData.save( spark, nonPartitionedDataPath, @@ -137,7 +141,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val latestId = logManager(indexConfig.indexName).getLatestId().get MockEventLogger.reset() - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) // Check that no new log files were created in this operation. assert(latestId == logManager(indexConfig.indexName).getLatestId().get) @@ -156,7 +160,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } test( - "Validate refresh index (to handle deletes from the source data) " + + "Validate incremental refresh index (to handle deletes from the source data) " + "fails as expected when all source data files are deleted.") { Seq(true, false).foreach { deleteDataFolder => withSQLConf( @@ -173,7 +177,8 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { if (deleteDataFolder) { FileUtils.delete(new Path(nonPartitionedDataPath)) - val ex = intercept[AnalysisException](hyperspace.refreshIndex(indexConfig.indexName)) + val ex = intercept[AnalysisException]( + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL)) assert(ex.getMessage.contains("Path does not exist")) } else { @@ -184,7 +189,8 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { .foreach(p => FileUtils.delete(p.getPath)) val ex = - intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) + intercept[HyperspaceException]( + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL)) assert(ex.getMessage.contains("Invalid plan for creating an index.")) } FileUtils.delete(new Path(nonPartitionedDataPath)) @@ -194,7 +200,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } test( - "Validate refresh index (to handle deletes from the source data) " + + "Validate incremental refresh index (to handle deletes from the source data) " + "fails as expected when file info for an existing source data file changes.") { SampleData.save( spark, @@ -214,7 +220,8 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { deletedFile, "I am some random content :).") - val ex = intercept[HyperspaceException](hyperspace.refreshIndex(indexConfig.indexName)) + val ex = intercept[HyperspaceException]( + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL)) assert( ex.getMessage.contains("Index refresh (to handle deleted source data) aborted. " + "Existing source data file info is changed")) @@ -229,10 +236,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { IndexConstants.INDEX_LINEAGE_ENABLED -> "true", IndexConstants.REFRESH_DELETE_ENABLED -> "true") { withIndex(indexConfig.indexName) { - SampleData.save( - spark, - testPath, - Seq("Date", "RGUID", "Query", "imprs", "clicks")) + SampleData.save(spark, testPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) val df = spark.read.parquet(testPath) hyperspace.createIndex(df, indexConfig) @@ -250,12 +254,12 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { .mode("append") .parquet(testPath) - val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") - new RefreshDeleteAction( - spark, - IndexLogManagerFactoryImpl.create(indexPath), - IndexDataManagerFactoryImpl.create(indexPath)) - .run() + val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") + new RefreshDeleteAction( + spark, + IndexLogManagerFactoryImpl.create(indexPath), + IndexDataManagerFactoryImpl.create(indexPath)) + .run() // Verify "deletedFiles" is cleared and "appendedFiles" is updated after refresh. val indexLogEntry = getLatestStableLog(indexConfig.indexName) @@ -311,7 +315,9 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } } - test("Validate refresh index when some file gets deleted and some appended to source data.") { + test( + "Validate incremental refresh index when some file gets deleted and some appended to " + + "source data.") { withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { withIndex(indexConfig.indexName) { // Save test data non-partitioned. @@ -340,7 +346,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val countAfterAppend = spark.read.parquet(nonPartitionedDataPath).count() assert(countAfterDelete + 3 == countAfterAppend) - hyperspace.refreshIndex(indexConfig.indexName) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) // Check if refreshed index is updated appropriately. val indexDf = spark.read From 54660e23f7344fbecc467621c7c8354cb1ac0199 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 01:13:45 -0700 Subject: [PATCH 76/86] remove delete and append flags for RefreshDeleteAction and RefreshApendAction --- .../hyperspace/index/IndexConstants.scala | 13 -- .../hyperspace/util/HyperspaceConf.scala | 14 -- .../index/E2EHyperspaceRulesTests.scala | 128 +++++++++--------- .../hyperspace/index/IndexManagerTests.scala | 72 +++++----- .../hyperspace/index/RefreshIndexTests.scala | 78 +++++------ 5 files changed, 131 insertions(+), 174 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala index 3f54a7adf..363a631f2 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexConstants.scala @@ -73,19 +73,6 @@ object IndexConstants { val INDEX_LINEAGE_ENABLED = "spark.hyperspace.index.lineage.enabled" val INDEX_LINEAGE_ENABLED_DEFAULT = "false" - val REFRESH_DELETE_ENABLED = "spark.hyperspace.index.refresh.delete.enabled" - val REFRESH_DELETE_ENABLED_DEFAULT = "false" - - /** - * This flag enables refreshing index if additional data files are appended to the source. When - * set to false, the refresh call will not run RefreshAppendAction. It will instead go for full - * refresh. - * This flag is temporary, and will be removed when both Append and Delete actions are merged - * for refreshing indexes. - */ - val REFRESH_APPEND_ENABLED = "spark.hyperspace.index.refresh.append.enabled" - val REFRESH_APPEND_ENABLED_DEFAULT = "false" - val REFRESH_MODE_INCREMENTAL = "incremental" val REFRESH_MODE_FULL = "full" } diff --git a/src/main/scala/com/microsoft/hyperspace/util/HyperspaceConf.scala b/src/main/scala/com/microsoft/hyperspace/util/HyperspaceConf.scala index 655391b91..e84d51bac 100644 --- a/src/main/scala/com/microsoft/hyperspace/util/HyperspaceConf.scala +++ b/src/main/scala/com/microsoft/hyperspace/util/HyperspaceConf.scala @@ -39,18 +39,4 @@ object HyperspaceConf { IndexConstants.INDEX_HYBRID_SCAN_DELETE_ENABLED_DEFAULT) .toBoolean } - - def refreshDeleteEnabled(spark: SparkSession): Boolean = { - spark.conf - .get(IndexConstants.REFRESH_DELETE_ENABLED, - IndexConstants.REFRESH_DELETE_ENABLED_DEFAULT) - .toBoolean - } - - def refreshAppendEnabled(spark: SparkSession): Boolean = { - spark.conf - .get(IndexConstants.REFRESH_APPEND_ENABLED, - IndexConstants.REFRESH_APPEND_ENABLED_DEFAULT) - .toBoolean - } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 7e2147783..3c3a1c8b0 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -412,74 +412,72 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { test("Verify JoinIndexRule utilizes indexes correctly after incremental refresh.") { withTempPathAsString { testPath => - withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { - // Setup. Create data. - val indexConfig = IndexConfig("index", Seq("c2"), Seq("c4")) - import spark.implicits._ - SampleData.testData - .toDF("c1", "c2", "c3", "c4", "c5") - .limit(10) - .write - .parquet(testPath) - val df = spark.read.load(testPath) - - // Create index. - hyperspace.createIndex(df, indexConfig) - - // Append to original data. - SampleData.testData - .toDF("c1", "c2", "c3", "c4", "c5") - .limit(3) - .write - .mode("append") - .parquet(testPath) - - // Refresh index. - hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) - - // Create a join query. - val leftDf = spark.read.parquet(testPath) - val rightDf = spark.read.parquet(testPath) - - def query(): DataFrame = { - leftDf - .join(rightDf, leftDf("c2") === rightDf("c2")) - .select(leftDf("c2"), rightDf("c4")) - } + // Setup. Create data. + val indexConfig = IndexConfig("index", Seq("c2"), Seq("c4")) + import spark.implicits._ + SampleData.testData + .toDF("c1", "c2", "c3", "c4", "c5") + .limit(10) + .write + .parquet(testPath) + val df = spark.read.load(testPath) + + // Create index. + hyperspace.createIndex(df, indexConfig) - // Verify indexes are used, and all index files are picked. - verifyIndexUsage( - query, - getIndexFilesPath(indexConfig.indexName, Seq(0, 1)) ++ - getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) - - // With Hyperspace disabled, verify there are shuffle and sort nodes as expected. - spark.disableHyperspace() - val dfWithHyperspaceDisabled = query() - var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { - case s: ShuffleExchangeExec => s - } - assert(shuffleNodes.size == 2) - var sortNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { - case s: SortExec => s - } - assert(sortNodes.size == 2) - - // With Hyperspace enabled, verify bucketing works as expected. This is reflected in - // shuffle nodes being eliminated. - spark.enableHyperspace() - val dfWithHyperspaceEnabled = query() - shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { - case s: ShuffleExchangeExec => s - } - assert(shuffleNodes.isEmpty) + // Append to original data. + SampleData.testData + .toDF("c1", "c2", "c3", "c4", "c5") + .limit(3) + .write + .mode("append") + .parquet(testPath) - // SortExec is expected to be present because there are multiple files per bucket. - sortNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { - case s: SortExec => s - } - assert(sortNodes.size == 2) + // Refresh index. + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) + + // Create a join query. + val leftDf = spark.read.parquet(testPath) + val rightDf = spark.read.parquet(testPath) + + def query(): DataFrame = { + leftDf + .join(rightDf, leftDf("c2") === rightDf("c2")) + .select(leftDf("c2"), rightDf("c4")) + } + + // Verify indexes are used, and all index files are picked. + verifyIndexUsage( + query, + getIndexFilesPath(indexConfig.indexName, Seq(0, 1)) ++ + getIndexFilesPath(indexConfig.indexName, Seq(0, 1))) + + // With Hyperspace disabled, verify there are shuffle and sort nodes as expected. + spark.disableHyperspace() + val dfWithHyperspaceDisabled = query() + var shuffleNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffleNodes.size == 2) + var sortNodes = dfWithHyperspaceDisabled.queryExecution.executedPlan.collect { + case s: SortExec => s + } + assert(sortNodes.size == 2) + + // With Hyperspace enabled, verify bucketing works as expected. This is reflected in + // shuffle nodes being eliminated. + spark.enableHyperspace() + val dfWithHyperspaceEnabled = query() + shuffleNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + assert(shuffleNodes.isEmpty) + + // SortExec is expected to be present because there are multiple files per bucket. + sortNodes = dfWithHyperspaceEnabled.queryExecution.executedPlan.collect { + case s: SortExec => s } + assert(sortNodes.size == 2) } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala index e8bcfd820..380fd6c35 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexManagerTests.scala @@ -254,45 +254,43 @@ class IndexManagerTests extends HyperspaceSuite with SQLHelper { test("Verify incremental refresh (append-only) should index only newly appended data.") { withTempPathAsString { testPath => - withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { - // Setup. Create sample data and index. - val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) - import spark.implicits._ - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(10) - .write - .parquet(testPath) - val df = spark.read.parquet(testPath) - hyperspace.createIndex(df, indexConfig) - var indexCount = - spark.read - .parquet(s"$systemPath/index" + - s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") - .count() - assert(indexCount == 10) - // Check if latest log file is updated with newly created index files. - validateMetadata("index", Set("v__=0")) - - // Change original data. - SampleData.testData - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .limit(3) - .write - .mode("append") - .parquet(testPath) - hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) - indexCount = spark.read + // Setup. Create sample data and index. + val indexConfig = IndexConfig(s"index", Seq("RGUID"), Seq("imprs")) + import spark.implicits._ + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(10) + .write + .parquet(testPath) + val df = spark.read.parquet(testPath) + hyperspace.createIndex(df, indexConfig) + var indexCount = + spark.read .parquet(s"$systemPath/index" + - s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0") .count() - - // Check if index got updated. - assert(indexCount == 3) - - // Check if latest log file is updated with newly created index files. - validateMetadata("index", Set("v__=0", "v__=1")) - } + assert(indexCount == 10) + // Check if latest log file is updated with newly created index files. + validateMetadata("index", Set("v__=0")) + + // Change original data. + SampleData.testData + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .limit(3) + .write + .mode("append") + .parquet(testPath) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) + indexCount = spark.read + .parquet(s"$systemPath/index" + + s"/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + .count() + + // Check if index got updated. + assert(indexCount == 3) + + // Check if latest log file is updated with newly created index files. + validateMetadata("index", Set("v__=0", "v__=1")) } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index ccc4843d6..e70665996 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -70,9 +70,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val partitionedDataDF = spark.read.parquet(partitionedDataPath) Seq(nonPartitionedDataPath, partitionedDataPath).foreach { loc => - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { withIndex(indexConfig.indexName) { val dfToIndex = if (loc.equals(nonPartitionedDataPath)) nonPartitionedDataDF else partitionedDataDF @@ -111,9 +109,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { Seq("Date", "RGUID", "Query", "imprs", "clicks")) val nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "false", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "false") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) deleteDataFile(nonPartitionedDataPath) @@ -163,9 +159,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { "Validate incremental refresh index (to handle deletes from the source data) " + "fails as expected when all source data files are deleted.") { Seq(true, false).foreach { deleteDataFolder => - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { SampleData.save( spark, nonPartitionedDataPath, @@ -208,9 +202,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { Seq("Date", "RGUID", "Query", "imprs", "clicks")) val nonPartitionedDataDF = spark.read.parquet(nonPartitionedDataPath) - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) // Replace a source data file with a new file with same name but different properties. @@ -229,12 +221,10 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } test( - "Validate refresh delete action updates appended and deleted files in metadata as " + + "Validate RefreshDeleteAction updates appended and deleted files in metadata as " + "expected, when some file gets deleted and some appended to source data.") { withTempPathAsString { testPath => - withSQLConf( - IndexConstants.INDEX_LINEAGE_ENABLED -> "true", - IndexConstants.REFRESH_DELETE_ENABLED -> "true") { + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { withIndex(indexConfig.indexName) { SampleData.save(spark, testPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) val df = spark.read.parquet(testPath) @@ -274,43 +264,41 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } test( - "Validate refresh append action updates appended and deleted files in metadata as" + + "Validate RefreshAppendAction updates appended and deleted files in metadata as" + "expected, when some file gets deleted and some appended to source data.") { withTempPathAsString { testPath => - withSQLConf(IndexConstants.REFRESH_APPEND_ENABLED -> "true") { - withIndex(indexConfig.indexName) { - SampleData.save(spark, testPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) - val df = spark.read.parquet(testPath) - hyperspace.createIndex(df, indexConfig) + withIndex(indexConfig.indexName) { + SampleData.save(spark, testPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) + val df = spark.read.parquet(testPath) + hyperspace.createIndex(df, indexConfig) - val oldFiles = listFiles(testPath).toSet + val oldFiles = listFiles(testPath).toSet - // Delete one source data file. - deleteDataFile(testPath) + // Delete one source data file. + deleteDataFile(testPath) - // Add some new data to source. - import spark.implicits._ - SampleData.testData - .take(3) - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .write - .mode("append") - .parquet(testPath) + // Add some new data to source. + import spark.implicits._ + SampleData.testData + .take(3) + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .write + .mode("append") + .parquet(testPath) - val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") - new RefreshAppendAction( - spark, - IndexLogManagerFactoryImpl.create(indexPath), - IndexDataManagerFactoryImpl.create(indexPath)) - .run() + val indexPath = PathUtils.makeAbsolute(s"$systemPath/${indexConfig.indexName}") + new RefreshAppendAction( + spark, + IndexLogManagerFactoryImpl.create(indexPath), + IndexDataManagerFactoryImpl.create(indexPath)) + .run() - // Verify "appendedFiles" is cleared and "deletedFiles" is updated after refresh. - val indexLogEntry = getLatestStableLog(indexConfig.indexName) - assert(indexLogEntry.appendedFiles.isEmpty) + // Verify "appendedFiles" is cleared and "deletedFiles" is updated after refresh. + val indexLogEntry = getLatestStableLog(indexConfig.indexName) + assert(indexLogEntry.appendedFiles.isEmpty) - val latestFiles = listFiles(testPath).toSet - assert(indexLogEntry.deletedFiles.toSet.equals(oldFiles -- latestFiles)) - } + val latestFiles = listFiles(testPath).toSet + assert(indexLogEntry.deletedFiles.toSet.equals(oldFiles -- latestFiles)) } } } From 91a27ea0b0d852cb560d2ce215f92c687c7d55f0 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 12:51:04 -0700 Subject: [PATCH 77/86] added tests as per review comments --- .../com/microsoft/hyperspace/Hyperspace.scala | 3 +- .../index/E2EHyperspaceRulesTests.scala | 72 ++++++++++++++++++- .../hyperspace/index/RefreshIndexTests.scala | 67 +++++++++-------- 3 files changed, 104 insertions(+), 38 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala index 782a5e2e8..0774854c8 100644 --- a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala +++ b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala @@ -79,7 +79,8 @@ class Hyperspace(spark: SparkSession) { } /** - * Update indexes for the latest version of the data. This api provides a few + * Update indexes for the latest version of the data. This api provides a few supported + * refresh modes as listed below. * * @param indexName Name of the index to refresh. * @param mode Refresh mode. Currently supported modes are [[REFRESH_MODE_INCREMENTAL]] and diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 3c3a1c8b0..de01b5509 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -18,7 +18,7 @@ package com.microsoft.hyperspace.index import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisException, DataFrame, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, 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.SortExec @@ -31,7 +31,7 @@ import com.microsoft.hyperspace.index.execution.BucketUnionStrategy import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule} import com.microsoft.hyperspace.util.{FileUtils, PathUtils} -class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { +class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { private val testDir = "src/test/resources/e2eTests/" private val nonPartitionedDataPath = testDir + "sampleparquet" private val partitionedDataPath = testDir + "samplepartitionedparquet" @@ -410,7 +410,7 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { sortedRowsWithHyperspaceDisabled.sameElements(getSortedRows(dfAfterHyperspaceDisabled))) } - test("Verify JoinIndexRule utilizes indexes correctly after incremental refresh.") { + test("Verify JoinIndexRule utilizes indexes correctly after incremental refresh (append-only).") { withTempPathAsString { testPath => // Setup. Create data. val indexConfig = IndexConfig("index", Seq("c2"), Seq("c4")) @@ -536,6 +536,72 @@ class E2EHyperspaceRulesTests extends HyperspaceSuite with SQLHelper { } } + test( + "Verify JoinIndexRule utilizes indexes correctly after incremental refresh when some file" + + "gets deleted and some appended to source data.") { + withTempPathAsString { testPath => + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { + // Setup. Create data. + val indexConfig = IndexConfig("index", Seq("c2"), Seq("c4")) + import spark.implicits._ + SampleData.testData + .toDF("c1", "c2", "c3", "c4", "c5") + .limit(10) + .write + .parquet(testPath) + val df = spark.read.load(testPath) + + // Create index. + hyperspace.createIndex(df, indexConfig) + + // Delete some source data file. + val dataPath = new Path(testPath, "*parquet") + val dataFileNames = dataPath + .getFileSystem(new Configuration) + .globStatus(dataPath) + .map(_.getPath) + + assert(dataFileNames.nonEmpty) + val fileToDelete = dataFileNames.head + FileUtils.delete(fileToDelete) + + // Append to original data. + SampleData.testData + .toDF("c1", "c2", "c3", "c4", "c5") + .limit(3) + .write + .mode("append") + .parquet(testPath) + + // Refresh index. + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) + + // Create a join query. + val leftDf = spark.read.parquet(testPath) + val rightDf = spark.read.parquet(testPath) + + def query(): DataFrame = { + leftDf + .join(rightDf, leftDf("c2") === rightDf("c2")) + .select(leftDf("c2"), rightDf("c4")) + } + + // Verify indexes are used, and all index files are picked. + verifyIndexUsage( + query, + getIndexFilesPath(indexConfig.indexName, Seq(1, 2)) ++ + getIndexFilesPath(indexConfig.indexName, Seq(1, 2))) + + // Verify correctness of results. + spark.disableHyperspace() + val dfWithHyperspaceDisabled = query() + spark.enableHyperspace() + val dfWithHyperspaceEnabled = query() + checkAnswer(dfWithHyperspaceDisabled, dfWithHyperspaceEnabled) + } + } + } + /** * Check that if the query plan has the expected rootPaths. * diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index e70665996..0fde16b82 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -18,9 +18,9 @@ package com.microsoft.hyperspace.index import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, Implicits, MockEventLogger, SampleData} import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteAction} import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.telemetry.{RefreshAppendActionEvent, RefreshDeleteActionEvent} @@ -306,44 +306,43 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { test( "Validate incremental refresh index when some file gets deleted and some appended to " + "source data.") { - withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { - withIndex(indexConfig.indexName) { - // Save test data non-partitioned. - SampleData.save( - spark, - nonPartitionedDataPath, - Seq("Date", "RGUID", "Query", "imprs", "clicks")) - val df = spark.read.parquet(nonPartitionedDataPath) - hyperspace.createIndex(df, indexConfig) - val countOriginal = df.count() + withTempPathAsString { testPath => + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { + withIndex(indexConfig.indexName) { + // Save test data non-partitioned. + SampleData.save(spark, testPath, Seq("Date", "RGUID", "Query", "imprs", "clicks")) + val df = spark.read.parquet(testPath) + hyperspace.createIndex(df, indexConfig) + val countOriginal = df.count() - // Delete one source data file. - deleteDataFile(nonPartitionedDataPath) - val countAfterDelete = spark.read.parquet(nonPartitionedDataPath).count() - assert(countAfterDelete < countOriginal) + // Delete one source data file. + deleteDataFile(testPath) + val countAfterDelete = spark.read.parquet(testPath).count() + assert(countAfterDelete < countOriginal) - // Add some new data to source. - import spark.implicits._ - SampleData.testData - .take(3) - .toDF("Date", "RGUID", "Query", "imprs", "clicks") - .write - .mode("append") - .parquet(nonPartitionedDataPath) + // Add some new data to source. + import spark.implicits._ + SampleData.testData + .take(3) + .toDF("Date", "RGUID", "Query", "imprs", "clicks") + .write + .mode("append") + .parquet(testPath) - val countAfterAppend = spark.read.parquet(nonPartitionedDataPath).count() - assert(countAfterDelete + 3 == countAfterAppend) + val countAfterAppend = spark.read.parquet(testPath).count() + assert(countAfterDelete + 3 == countAfterAppend) - hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) - // Check if refreshed index is updated appropriately. - val indexDf = spark.read - .parquet(s"$systemPath/${indexConfig.indexName}/" + - s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") - .union(spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + - s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=2")) + // Check if refreshed index is updated appropriately. + val indexDf = spark.read + .parquet(s"$systemPath/${indexConfig.indexName}/" + + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=1") + .union(spark.read.parquet(s"$systemPath/${indexConfig.indexName}/" + + s"${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=2")) - assert(indexDf.count() == countAfterAppend) + assert(indexDf.count() == countAfterAppend) + } } } } From 458083e9857bfdc2dd776817786fe94a832688e1 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 12:53:04 -0700 Subject: [PATCH 78/86] review comments --- .../microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala | 1 - .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index de01b5509..57ddaf36e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -19,7 +19,6 @@ package com.microsoft.hyperspace.index import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, 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.SortExec import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 0fde16b82..5f0ed8c28 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -20,7 +20,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, Implicits, MockEventLogger, SampleData} +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteAction} import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.telemetry.{RefreshAppendActionEvent, RefreshDeleteActionEvent} From 012c89ba622703bf2c9f08a8106f811f306df245 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 12:57:01 -0700 Subject: [PATCH 79/86] nit --- .../microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala | 2 +- 1 file changed, 1 insertion(+), 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 57ddaf36e..c9db946f7 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -536,7 +536,7 @@ class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { } test( - "Verify JoinIndexRule utilizes indexes correctly after incremental refresh when some file" + + "Verify JoinIndexRule utilizes indexes correctly after incremental refresh when some file " + "gets deleted and some appended to source data.") { withTempPathAsString { testPath => withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { From 60585d78b2e84c8d46a4597ddcd16ca347f4c4d2 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Sat, 10 Oct 2020 13:20:10 -0700 Subject: [PATCH 80/86] add issue details for a known issue --- .../com/microsoft/hyperspace/actions/RefreshAction.scala | 6 +++--- .../microsoft/hyperspace/actions/RefreshAppendAction.scala | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala index 547fa0dfa..67cbe282f 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala @@ -37,9 +37,9 @@ class RefreshAction( extends RefreshActionBase(spark, logManager, dataManager) { final override def op(): Unit = { - // TODO: The current implementation picks the number of buckets from session config. - // This should be user-configurable to allow maintain the existing bucket numbers - // in the index log entry. + // TODO: The current implementation picks the number of buckets INDEX_LINEAGE_ENABLED from + // session config. This should be user-configurable to allow maintain the existing bucket + // numbers in the index log entry. https://github.com/microsoft/hyperspace/issues/196. write(spark, df, indexConfig) } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index def6d9f08..174d04477 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -41,9 +41,9 @@ class RefreshAppendAction( dataManager: IndexDataManager) extends RefreshActionBase(spark, logManager, dataManager) { final override def op(): Unit = { - // TODO: The current implementation picks the number of buckets from session config. - // This should be user-configurable to allow maintain the existing bucket numbers - // in the index log entry. + // TODO: The current implementation picks the number of buckets INDEX_LINEAGE_ENABLED from + // session config. This should be user-configurable to allow maintain the existing bucket + // numbers in the index log entry. https://github.com/microsoft/hyperspace/issues/196. write(spark, dfWithAppendedFiles, indexConfig) } From 3b09a59020b4cbe474d55d179f3e8fb22d4a4b6d Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 12 Oct 2020 10:56:53 -0700 Subject: [PATCH 81/86] review comments --- src/main/scala/com/microsoft/hyperspace/Hyperspace.scala | 5 ++--- .../microsoft/hyperspace/index/IndexCollectionManager.scala | 4 ++-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala index 0774854c8..f1203ebf9 100644 --- a/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala +++ b/src/main/scala/com/microsoft/hyperspace/Hyperspace.scala @@ -19,7 +19,7 @@ package com.microsoft.hyperspace import org.apache.spark.sql.{DataFrame, SparkSession} import com.microsoft.hyperspace.index._ -import com.microsoft.hyperspace.index.IndexConstants.{REFRESH_MODE_FULL, REFRESH_MODE_INCREMENTAL} +import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_FULL import com.microsoft.hyperspace.index.plananalysis.PlanAnalyzer class Hyperspace(spark: SparkSession) { @@ -83,8 +83,7 @@ class Hyperspace(spark: SparkSession) { * refresh modes as listed below. * * @param indexName Name of the index to refresh. - * @param mode Refresh mode. Currently supported modes are [[REFRESH_MODE_INCREMENTAL]] and - * [[REFRESH_MODE_FULL]] + * @param mode Refresh mode. Currently supported modes are `incremental` and `full`. */ def refreshIndex(indexName: String, mode: String): Unit = { indexManager.refresh(indexName, mode) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index 983ffdfc7..dec7d81a6 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -67,10 +67,10 @@ class IndexCollectionManager( withLogManager(indexName) { logManager => val indexPath = PathResolver(spark.sessionState.conf).getIndexPath(indexName) val dataManager = indexDataManagerFactory.create(indexPath) - if (mode.equals(REFRESH_MODE_INCREMENTAL)) { + if (mode.equalsIgnoreCase(REFRESH_MODE_INCREMENTAL)) { new RefreshDeleteAction(spark, logManager, dataManager).run() new RefreshAppendAction(spark, logManager, dataManager).run() - } else if (mode.equals(REFRESH_MODE_FULL)) { + } else if (mode.equalsIgnoreCase(REFRESH_MODE_FULL)) { new RefreshAction(spark, logManager, dataManager).run() } else { throw HyperspaceException(s"Unsupported refresh mode $mode found.") From f93831a97b0890076b46521c63c350b08c19813c Mon Sep 17 00:00:00 2001 From: Apoorve Dave <66283785+apoorvedave1@users.noreply.github.com> Date: Mon, 12 Oct 2020 11:51:24 -0700 Subject: [PATCH 82/86] Update src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala Co-authored-by: Terry Kim --- .../com/microsoft/hyperspace/index/IndexCollectionManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala index dec7d81a6..b8fccdc52 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/IndexCollectionManager.scala @@ -73,7 +73,7 @@ class IndexCollectionManager( } else if (mode.equalsIgnoreCase(REFRESH_MODE_FULL)) { new RefreshAction(spark, logManager, dataManager).run() } else { - throw HyperspaceException(s"Unsupported refresh mode $mode found.") + throw HyperspaceException(s"Unsupported refresh mode '$mode' found.") } } } From 87f4a8a6387120b42e2d482d67f06651df9917dc Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 12 Oct 2020 12:25:59 -0700 Subject: [PATCH 83/86] review comments and test refactoring --- .../hyperspace/actions/RefreshAction.scala | 7 +-- .../actions/RefreshAppendAction.scala | 7 +-- .../index/CachingIndexCollectionManager.scala | 2 - .../com/microsoft/hyperspace/TestUtils.scala | 34 ++++++++++++++ .../index/E2EHyperspaceRulesTests.scala | 20 ++------- .../index/IndexCollectionManagerTest.scala | 13 ++++-- .../hyperspace/index/RefreshIndexTests.scala | 44 ++++--------------- 7 files changed, 63 insertions(+), 64 deletions(-) diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala index 67cbe282f..8b9af26fd 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAction.scala @@ -37,9 +37,10 @@ class RefreshAction( extends RefreshActionBase(spark, logManager, dataManager) { final override def op(): Unit = { - // TODO: The current implementation picks the number of buckets INDEX_LINEAGE_ENABLED from - // session config. This should be user-configurable to allow maintain the existing bucket - // numbers in the index log entry. https://github.com/microsoft/hyperspace/issues/196. + // TODO: The current implementation picks the number of buckets and + // "spark.hyperspace.index.lineage.enabled" from session config. This should be + // user-configurable to allow maintain the existing bucket numbers in the index log entry. + // https://github.com/microsoft/hyperspace/issues/196. write(spark, df, indexConfig) } diff --git a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala index 174d04477..133b48720 100644 --- a/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala +++ b/src/main/scala/com/microsoft/hyperspace/actions/RefreshAppendAction.scala @@ -41,9 +41,10 @@ class RefreshAppendAction( dataManager: IndexDataManager) extends RefreshActionBase(spark, logManager, dataManager) { final override def op(): Unit = { - // TODO: The current implementation picks the number of buckets INDEX_LINEAGE_ENABLED from - // session config. This should be user-configurable to allow maintain the existing bucket - // numbers in the index log entry. https://github.com/microsoft/hyperspace/issues/196. + // TODO: The current implementation picks the number of buckets and + // "spark.hyperspace.index.lineage.enabled" from session config. This should be + // user-configurable to allow maintain the existing bucket numbers in the index log entry. + // https://github.com/microsoft/hyperspace/issues/196. write(spark, dfWithAppendedFiles, indexConfig) } diff --git a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala index 7c6d230a6..56fbc6c4a 100644 --- a/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala +++ b/src/main/scala/com/microsoft/hyperspace/index/CachingIndexCollectionManager.scala @@ -18,8 +18,6 @@ package com.microsoft.hyperspace.index import org.apache.spark.sql.{DataFrame, SparkSession} -import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL - /** * An IndexCollectionManager which leverages a cache to * accelerate fetching indexes, with below properties: diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index 78b8f6ede..8c3687d10 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -16,11 +16,13 @@ package com.microsoft.hyperspace +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import com.microsoft.hyperspace.MockEventLogger.reset import com.microsoft.hyperspace.index.IndexLogEntry import com.microsoft.hyperspace.telemetry.{EventLogger, HyperspaceEvent} +import com.microsoft.hyperspace.util.FileUtils object TestUtils { def copyWithState(index: IndexLogEntry, state: String): IndexLogEntry = { @@ -46,6 +48,38 @@ object TestUtils { path.getName +: splitPath(path.getParent) } } + + /** + * Delete some files from a given path. + * + * @param path Path to the parent folder containing data files. + * @param isPartitioned Is data folder partitioned or not. + * @param pattern File name pattern to delete. + * @param numFilesToDelete Num files to delete. + * @return Paths to the deleted file. + */ + def deleteDataFiles( + path: String, + isPartitioned: Boolean = false, + pattern: String = "*parquet", + numFilesToDelete: Int = 1): Seq[Path] = { + val dataPath = if (isPartitioned) { + new Path(s"$path/*/*", pattern) + } else { + new Path(path, pattern) + } + + val dataFileNames = dataPath + .getFileSystem(new Configuration) + .globStatus(dataPath) + .map(_.getPath) + + assert(dataFileNames.length >= numFilesToDelete) + val filesToDelete = dataFileNames.take(numFilesToDelete) + filesToDelete.foreach(FileUtils.delete(_)) + + filesToDelete + } } /** diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index c9db946f7..3f8f4164e 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import com.microsoft.hyperspace.TestUtils.deleteDataFiles import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData} import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.index.execution.BucketUnionStrategy @@ -508,14 +509,7 @@ class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { verifyIndexUsage(query1, getIndexFilesPath(indexConfig.indexName)) // Delete some source data file. - val dataFileNames = dataPath - .getFileSystem(new Configuration) - .globStatus(dataPath) - .map(_.getPath) - - assert(dataFileNames.nonEmpty) - val fileToDelete = dataFileNames.head - FileUtils.delete(fileToDelete) + deleteDataFiles(location) def query2(): DataFrame = spark.read.parquet(location).filter("c3 == 'facebook'").select("c3", "c1") @@ -554,15 +548,7 @@ class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { hyperspace.createIndex(df, indexConfig) // Delete some source data file. - val dataPath = new Path(testPath, "*parquet") - val dataFileNames = dataPath - .getFileSystem(new Configuration) - .globStatus(dataPath) - .map(_.getPath) - - assert(dataFileNames.nonEmpty) - val fileToDelete = dataFileNames.head - FileUtils.delete(fileToDelete) + deleteDataFiles(testPath) // Append to original data. SampleData.testData diff --git a/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala b/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala index 8e29885ec..895f6ae16 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/IndexCollectionManagerTest.scala @@ -23,7 +23,7 @@ import org.mockito.Mockito.{mock, when} import com.microsoft.hyperspace.{HyperspaceException, SparkInvolvedSuite} import com.microsoft.hyperspace.actions.Constants -import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_FULL +import com.microsoft.hyperspace.index.IndexConstants.{REFRESH_MODE_FULL, REFRESH_MODE_INCREMENTAL} class IndexCollectionManagerTest extends SparkFunSuite with SparkInvolvedSuite { private val indexSystemPath = "src/test/resources/indexLocation" @@ -53,8 +53,7 @@ class IndexCollectionManagerTest extends SparkFunSuite with SparkInvolvedSuite { .Columns(Seq("RGUID"), Seq("Date")), "", 10)), - Content( - Directory(s"$indexPath/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0")), + Content(Directory(s"$indexPath/${IndexConstants.INDEX_VERSION_DIRECTORY_PREFIX}=0")), Source(SparkPlan(sourcePlanProperties)), Map()) entry.state = Constants.States.ACTIVE @@ -133,8 +132,14 @@ class IndexCollectionManagerTest extends SparkFunSuite with SparkInvolvedSuite { intercept[HyperspaceException](indexCollectionManager.restore("idx4")) } - test("refresh() throws exception if index is not found") { + test("refresh() with mode = 'full' throws exception if index is not found") { when(mockFileSystem.exists(new Path(indexSystemPath, "idx4"))).thenReturn(false) intercept[HyperspaceException](indexCollectionManager.refresh("idx4", REFRESH_MODE_FULL)) } + + test("refresh() with mode = 'incremental' throws exception if index is not found") { + when(mockFileSystem.exists(new Path(indexSystemPath, "idx4"))).thenReturn(false) + intercept[HyperspaceException]( + indexCollectionManager.refresh("idx4", REFRESH_MODE_INCREMENTAL)) + } } diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 5f0ed8c28..37fe1f74f 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -18,14 +18,14 @@ package com.microsoft.hyperspace.index import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} +import org.apache.spark.sql.{AnalysisException, QueryTest} import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} +import com.microsoft.hyperspace.TestUtils.deleteDataFiles import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteAction} import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.telemetry.{RefreshAppendActionEvent, RefreshDeleteActionEvent} import com.microsoft.hyperspace.util.{FileUtils, PathUtils} - /** * Unit E2E test cases for RefreshIndex. */ @@ -78,9 +78,9 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { // Delete one source data file. val deletedFile = if (loc.equals(nonPartitionedDataPath)) { - deleteDataFile(nonPartitionedDataPath) + deleteDataFiles(nonPartitionedDataPath).head } else { - deleteDataFile(partitionedDataPath, true) + deleteDataFiles(partitionedDataPath, true).head } // Validate only index records whose lineage is the deleted file are removed. @@ -112,7 +112,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "false") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) - deleteDataFile(nonPartitionedDataPath) + deleteDataFiles(nonPartitionedDataPath) val ex = intercept[HyperspaceException]( hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL)) @@ -206,7 +206,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) // Replace a source data file with a new file with same name but different properties. - val deletedFile = deleteDataFile(nonPartitionedDataPath) + val deletedFile = deleteDataFiles(nonPartitionedDataPath).head FileUtils.createFile( deletedFile.getFileSystem(new Configuration), deletedFile, @@ -231,7 +231,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { hyperspace.createIndex(df, indexConfig) // Delete one source data file. - deleteDataFile(testPath) + deleteDataFiles(testPath) val oldFiles = listFiles(testPath).toSet @@ -275,7 +275,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val oldFiles = listFiles(testPath).toSet // Delete one source data file. - deleteDataFile(testPath) + deleteDataFiles(testPath) // Add some new data to source. import spark.implicits._ @@ -316,7 +316,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val countOriginal = df.count() // Delete one source data file. - deleteDataFile(testPath) + deleteDataFiles(testPath) val countAfterDelete = spark.read.parquet(testPath).count() assert(countAfterDelete < countOriginal) @@ -347,32 +347,6 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } } - /** - * Delete one file from a given path. - * - * @param path Path to the parent folder containing data files. - * @param isPartitioned Is data folder partitioned or not. - * @return Path to the deleted file. - */ - private def deleteDataFile(path: String, isPartitioned: Boolean = false): Path = { - val dataPath = if (isPartitioned) { - new Path(s"$path/*/*", "*parquet") - } else { - new Path(path, "*parquet") - } - - val dataFileNames = dataPath - .getFileSystem(new Configuration) - .globStatus(dataPath) - .map(_.getPath) - - assert(dataFileNames.nonEmpty) - val fileToDelete = dataFileNames.head - FileUtils.delete(fileToDelete) - - fileToDelete - } - private def logManager(indexName: String): IndexLogManager = { val indexPath = PathUtils.makeAbsolute(s"$systemPath/$indexName") IndexLogManagerFactoryImpl.create(indexPath) From 0ba4241b1b0a0405f677097f0554d23d780ea0b9 Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 12 Oct 2020 12:35:36 -0700 Subject: [PATCH 84/86] scalastyle --- .../microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala | 2 +- .../com/microsoft/hyperspace/index/RefreshIndexTests.scala | 1 + 2 files changed, 2 insertions(+), 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 3f8f4164e..4b1a486f6 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import com.microsoft.hyperspace.TestUtils.deleteDataFiles import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData} +import com.microsoft.hyperspace.TestUtils.deleteDataFiles import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.index.execution.BucketUnionStrategy import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule} diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 37fe1f74f..63ca3ccd0 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -26,6 +26,7 @@ import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteActio import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.telemetry.{RefreshAppendActionEvent, RefreshDeleteActionEvent} import com.microsoft.hyperspace.util.{FileUtils, PathUtils} + /** * Unit E2E test cases for RefreshIndex. */ From bdc66207355dd6349a0f7bf7d7ae54bc48aa5e5a Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 12 Oct 2020 14:02:55 -0700 Subject: [PATCH 85/86] cleanup --- .../com/microsoft/hyperspace/TestUtils.scala | 28 +++----- .../index/E2EHyperspaceRulesTests.scala | 64 +++++++++---------- .../hyperspace/index/RefreshIndexTests.scala | 29 ++++++--- 3 files changed, 60 insertions(+), 61 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala index 8c3687d10..96bce037a 100644 --- a/src/test/scala/com/microsoft/hyperspace/TestUtils.scala +++ b/src/test/scala/com/microsoft/hyperspace/TestUtils.scala @@ -50,32 +50,22 @@ object TestUtils { } /** - * Delete some files from a given path. + * Delete files from a given path. * - * @param path Path to the parent folder containing data files. - * @param isPartitioned Is data folder partitioned or not. + * @param path Path to the folder containing files. * @param pattern File name pattern to delete. - * @param numFilesToDelete Num files to delete. + * @param numFilesToDelete Number of files to delete. * @return Paths to the deleted file. */ - def deleteDataFiles( - path: String, - isPartitioned: Boolean = false, - pattern: String = "*parquet", - numFilesToDelete: Int = 1): Seq[Path] = { - val dataPath = if (isPartitioned) { - new Path(s"$path/*/*", pattern) - } else { - new Path(path, pattern) - } - - val dataFileNames = dataPath + def deleteFiles(path: String, pattern: String, numFilesToDelete: Int): Seq[Path] = { + val pathToDelete = new Path(path, pattern) + val fileNames = pathToDelete .getFileSystem(new Configuration) - .globStatus(dataPath) + .globStatus(pathToDelete) .map(_.getPath) - assert(dataFileNames.length >= numFilesToDelete) - val filesToDelete = dataFileNames.take(numFilesToDelete) + assert(fileNames.length >= numFilesToDelete) + val filesToDelete = fileNames.take(numFilesToDelete) filesToDelete.foreach(FileUtils.delete(_)) filesToDelete diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 4b1a486f6..7996a8b23 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -24,12 +24,11 @@ import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData} -import com.microsoft.hyperspace.TestUtils.deleteDataFiles +import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData, TestUtils} import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.index.execution.BucketUnionStrategy import com.microsoft.hyperspace.index.rules.{FilterIndexRule, JoinIndexRule} -import com.microsoft.hyperspace.util.{FileUtils, PathUtils} +import com.microsoft.hyperspace.util.PathUtils class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { private val testDir = "src/test/resources/e2eTests/" @@ -490,42 +489,41 @@ class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { } test("Validate index usage after incremental refresh with some source data file deleted.") { - withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { - // Save a copy of source data files. - val location = testDir + "ixRefreshTest" - val dataPath = new Path(location, "*parquet") - val dataColumns = Seq("c1", "c2", "c3", "c4", "c5") - SampleData.save(spark, location, dataColumns) - - // Create index on original source data files. - val df = spark.read.parquet(location) - val indexConfig = IndexConfig("filterIndex", Seq("c3"), Seq("c1")) - hyperspace.createIndex(df, indexConfig) + withTempPathAsString { testPath => + withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "true") { + // Save a copy of source data files. + val dataColumns = Seq("c1", "c2", "c3", "c4", "c5") + SampleData.save(spark, testPath, dataColumns) - // Verify index usage for index version (v=0). - def query1(): DataFrame = - spark.read.parquet(location).filter("c3 == 'facebook'").select("c3", "c1") + // Create index on original source data files. + val df = spark.read.parquet(testPath) + val indexConfig = IndexConfig("filterIndex", Seq("c3"), Seq("c1")) + hyperspace.createIndex(df, indexConfig) - verifyIndexUsage(query1, getIndexFilesPath(indexConfig.indexName)) + // Verify index usage for index version (v=0). + def query1(): DataFrame = + spark.read.parquet(testPath).filter("c3 == 'facebook'").select("c3", "c1") - // Delete some source data file. - deleteDataFiles(location) + verifyIndexUsage(query1, getIndexFilesPath(indexConfig.indexName)) - def query2(): DataFrame = - spark.read.parquet(location).filter("c3 == 'facebook'").select("c3", "c1") + // Delete some source data file. + TestUtils.deleteFiles(testPath, "*parquet", 1) - // Verify index is not used. - spark.enableHyperspace() - val planRootPaths = getAllRootPaths(query2().queryExecution.optimizedPlan) - spark.disableHyperspace() - assert(planRootPaths.equals(Seq(PathUtils.makeAbsolute(location)))) + def query2(): DataFrame = + spark.read.parquet(testPath).filter("c3 == 'facebook'").select("c3", "c1") - // Refresh the index to remove deleted source data file records from index. - hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) + // Verify index is not used. + spark.enableHyperspace() + val planRootPaths = getAllRootPaths(query2().queryExecution.optimizedPlan) + spark.disableHyperspace() + assert(planRootPaths.equals(Seq(PathUtils.makeAbsolute(testPath)))) - // Verify index usage on latest version of index (v=1) after refresh. - verifyIndexUsage(query2, getIndexFilesPath(indexConfig.indexName, Seq(1))) - FileUtils.delete(dataPath) + // Refresh the index to remove deleted source data file records from index. + hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) + + // Verify index usage on latest version of index (v=1) after refresh. + verifyIndexUsage(query2, getIndexFilesPath(indexConfig.indexName, Seq(1))) + } } } @@ -548,7 +546,7 @@ class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { hyperspace.createIndex(df, indexConfig) // Delete some source data file. - deleteDataFiles(testPath) + TestUtils.deleteFiles(testPath, "*parquet", 1) // Append to original data. SampleData.testData diff --git a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala index 63ca3ccd0..5e7b8d88d 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/RefreshIndexTests.scala @@ -20,8 +20,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, QueryTest} -import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData} -import com.microsoft.hyperspace.TestUtils.deleteDataFiles +import com.microsoft.hyperspace.{Hyperspace, HyperspaceException, MockEventLogger, SampleData, TestUtils} import com.microsoft.hyperspace.actions.{RefreshAppendAction, RefreshDeleteAction} import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_INCREMENTAL import com.microsoft.hyperspace.telemetry.{RefreshAppendActionEvent, RefreshDeleteActionEvent} @@ -79,9 +78,9 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { // Delete one source data file. val deletedFile = if (loc.equals(nonPartitionedDataPath)) { - deleteDataFiles(nonPartitionedDataPath).head + deleteOneDataFile(nonPartitionedDataPath) } else { - deleteDataFiles(partitionedDataPath, true).head + deleteOneDataFile(partitionedDataPath, true) } // Validate only index records whose lineage is the deleted file are removed. @@ -113,7 +112,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { withSQLConf(IndexConstants.INDEX_LINEAGE_ENABLED -> "false") { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) - deleteDataFiles(nonPartitionedDataPath) + deleteOneDataFile(nonPartitionedDataPath) val ex = intercept[HyperspaceException]( hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL)) @@ -207,7 +206,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { hyperspace.createIndex(nonPartitionedDataDF, indexConfig) // Replace a source data file with a new file with same name but different properties. - val deletedFile = deleteDataFiles(nonPartitionedDataPath).head + val deletedFile = deleteOneDataFile(nonPartitionedDataPath) FileUtils.createFile( deletedFile.getFileSystem(new Configuration), deletedFile, @@ -232,7 +231,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { hyperspace.createIndex(df, indexConfig) // Delete one source data file. - deleteDataFiles(testPath) + deleteOneDataFile(testPath) val oldFiles = listFiles(testPath).toSet @@ -276,7 +275,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val oldFiles = listFiles(testPath).toSet // Delete one source data file. - deleteDataFiles(testPath) + deleteOneDataFile(testPath) // Add some new data to source. import spark.implicits._ @@ -317,7 +316,7 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { val countOriginal = df.count() // Delete one source data file. - deleteDataFiles(testPath) + deleteOneDataFile(testPath) val countAfterDelete = spark.read.parquet(testPath).count() assert(countAfterDelete < countOriginal) @@ -348,6 +347,18 @@ class RefreshIndexTests extends QueryTest with HyperspaceSuite { } } + /** + * Delete one file from a given path. + * + * @param path Path to the parent folder containing data files. + * @param isPartitioned Is data folder partitioned or not. + * @return Path to the deleted file. + */ + private def deleteOneDataFile(path: String, isPartitioned: Boolean = false): Path = { + val dataPath = if (isPartitioned) s"$path/*/*" else path + TestUtils.deleteFiles(dataPath, "*parquet", 1).head + } + private def logManager(indexName: String): IndexLogManager = { val indexPath = PathUtils.makeAbsolute(s"$systemPath/$indexName") IndexLogManagerFactoryImpl.create(indexPath) From 03be354c0f092fae9afb7d83c379fb1da3f9d1aa Mon Sep 17 00:00:00 2001 From: Apoorve Dave Date: Mon, 12 Oct 2020 14:20:13 -0700 Subject: [PATCH 86/86] minor test fix --- .../hyperspace/index/E2EHyperspaceRulesTests.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala index 7996a8b23..0af0dbd33 100644 --- a/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala +++ b/src/test/scala/com/microsoft/hyperspace/index/E2EHyperspaceRulesTests.scala @@ -501,20 +501,17 @@ class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { hyperspace.createIndex(df, indexConfig) // Verify index usage for index version (v=0). - def query1(): DataFrame = + def query(): DataFrame = spark.read.parquet(testPath).filter("c3 == 'facebook'").select("c3", "c1") - verifyIndexUsage(query1, getIndexFilesPath(indexConfig.indexName)) + verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName)) // Delete some source data file. TestUtils.deleteFiles(testPath, "*parquet", 1) - def query2(): DataFrame = - spark.read.parquet(testPath).filter("c3 == 'facebook'").select("c3", "c1") - // Verify index is not used. spark.enableHyperspace() - val planRootPaths = getAllRootPaths(query2().queryExecution.optimizedPlan) + val planRootPaths = getAllRootPaths(query().queryExecution.optimizedPlan) spark.disableHyperspace() assert(planRootPaths.equals(Seq(PathUtils.makeAbsolute(testPath)))) @@ -522,7 +519,7 @@ class E2EHyperspaceRulesTests extends QueryTest with HyperspaceSuite { hyperspace.refreshIndex(indexConfig.indexName, REFRESH_MODE_INCREMENTAL) // Verify index usage on latest version of index (v=1) after refresh. - verifyIndexUsage(query2, getIndexFilesPath(indexConfig.indexName, Seq(1))) + verifyIndexUsage(query, getIndexFilesPath(indexConfig.indexName, Seq(1))) } } }