Data Skipping Index Part 3-2: Rule (#482)
This commit is contained in:
Родитель
b06cbddb18
Коммит
9735b57be5
|
@ -127,6 +127,9 @@ ThisBuild / Test / fork := true
|
|||
|
||||
ThisBuild / Test / javaOptions += "-Xmx1024m"
|
||||
|
||||
// Needed to test both non-codegen and codegen parts of expressions
|
||||
ThisBuild / Test / envVars += "SPARK_TESTING" -> "1"
|
||||
|
||||
ThisBuild / coverageExcludedPackages := "com\\.fasterxml.*;com\\.microsoft\\.hyperspace\\.shim"
|
||||
|
||||
/**
|
||||
|
|
|
@ -14,15 +14,11 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.shim
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, ExprId}
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, Literal}
|
||||
import org.apache.spark.sql.catalyst.expressions.aggregate.First
|
||||
|
||||
trait ExprMatcher {
|
||||
|
||||
/**
|
||||
* Returns true if the given expression matches the expression this matcher
|
||||
* is associated with.
|
||||
*/
|
||||
def apply(e: Expression): Boolean
|
||||
object FirstNullSafe {
|
||||
def apply(child: Expression): First = First(child, Literal(false))
|
||||
}
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.shim
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.expressions.aggregate.First
|
||||
|
||||
object FirstNullSafe {
|
||||
def apply(child: Expression): First = First(child, false)
|
||||
}
|
|
@ -21,7 +21,7 @@ import org.apache.spark.sql.{DataFrame, SparkSession}
|
|||
import com.microsoft.hyperspace.index._
|
||||
import com.microsoft.hyperspace.index.IndexConstants.{OPTIMIZE_MODE_QUICK, REFRESH_MODE_FULL}
|
||||
import com.microsoft.hyperspace.index.plananalysis.{CandidateIndexAnalyzer, PlanAnalyzer}
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace.withHyperspaceRuleDisabled
|
||||
import com.microsoft.hyperspace.index.sources.FileBasedSourceProviderManager
|
||||
|
||||
class Hyperspace(spark: SparkSession) {
|
||||
|
@ -189,15 +189,6 @@ class Hyperspace(spark: SparkSession) {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def withHyperspaceRuleDisabled(f: => Unit): Unit = {
|
||||
try {
|
||||
ApplyHyperspace.disableForIndexMaintenance.set(true)
|
||||
f
|
||||
} finally {
|
||||
ApplyHyperspace.disableForIndexMaintenance.set(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object Hyperspace extends ActiveSparkSession {
|
||||
|
|
|
@ -128,4 +128,22 @@ object IndexConstants {
|
|||
val DATASKIPPING_TARGET_INDEX_DATA_FILE_SIZE =
|
||||
"spark.hyperspace.index.dataskipping.targetIndexDataFileSize"
|
||||
val DATASKIPPING_TARGET_INDEX_DATA_FILE_SIZE_DEFAULT = "268435456" // 256 MiB
|
||||
|
||||
/**
|
||||
* Maximum number of index data files.
|
||||
*
|
||||
* The number of index data files determined by targetIndexFileSize is
|
||||
* capped by this value.
|
||||
*/
|
||||
val DATASKIPPING_MAX_INDEX_DATA_FILE_COUNT =
|
||||
"spark.hyperspace.index.dataskipping.maxIndexDataFileCount"
|
||||
val DATASKIPPING_MAX_INDEX_DATA_FILE_COUNT_DEFAULT = "10000"
|
||||
|
||||
/**
|
||||
* If set to true, partition sketches for partition columns are included when
|
||||
* creating data skipping indexes. This does not affect existing indexes.
|
||||
*/
|
||||
val DATASKIPPING_AUTO_PARTITION_SKETCH =
|
||||
"spark.hyperspace.index.dataskipping.autoPartitionSketch"
|
||||
val DATASKIPPING_AUTO_PARTITION_SKETCH_DEFAULT = "true"
|
||||
}
|
||||
|
|
|
@ -16,7 +16,8 @@
|
|||
|
||||
package com.microsoft.hyperspace.index
|
||||
|
||||
import org.apache.spark.sql.execution.datasources.InMemoryFileIndex
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.execution.datasources.{FileIndex, InMemoryFileIndex}
|
||||
|
||||
import com.microsoft.hyperspace.index.plananalysis.FilterReason
|
||||
|
||||
|
@ -68,4 +69,17 @@ object IndexLogEntryTags {
|
|||
// If it's enabled, FILTER_REASONS and APPLIED_INDEX_RULES info will be tagged.
|
||||
val INDEX_PLAN_ANALYSIS_ENABLED: IndexLogEntryTag[Boolean] =
|
||||
IndexLogEntryTag[Boolean]("indexPlanAnalysisEnabled")
|
||||
|
||||
// DATASKIPPING_INDEX_DATA_PREDICATE stores the index predicate translated
|
||||
// from the plan's filter or join condition.
|
||||
val DATASKIPPING_INDEX_PREDICATE: IndexLogEntryTag[Option[Expression]] =
|
||||
IndexLogEntryTag[Option[Expression]]("dataskippingIndexPredicate")
|
||||
|
||||
// DATASKIPPING_INDEX_FILEINDEX stores InMemoryFileIndex for the index data.
|
||||
val DATASKIPPING_INDEX_FILEINDEX: IndexLogEntryTag[InMemoryFileIndex] =
|
||||
IndexLogEntryTag[InMemoryFileIndex]("dataskippingIndexRelation")
|
||||
|
||||
// DATASKIPPING_INDEX_FILEINDEX stores InMemoryFileIndex for the source data.
|
||||
val DATASKIPPING_SOURCE_FILEINDEX: IndexLogEntryTag[FileIndex] =
|
||||
IndexLogEntryTag[FileIndex]("dataskippingSourceRelation")
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ package com.microsoft.hyperspace.index
|
|||
|
||||
import java.net.URLDecoder
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.DataFrame
|
||||
import org.apache.spark.sql.functions.udf
|
||||
|
||||
|
@ -64,4 +65,12 @@ object IndexUtils {
|
|||
*/
|
||||
lazy val decodeInputFileName = udf(
|
||||
(p: String) => URLDecoder.decode(p.replace("+", "%2B"), "UTF-8"))
|
||||
|
||||
/**
|
||||
* Returns the path part of the URI-like string.
|
||||
*
|
||||
* This can be used to compare the results of input_file_name() and the paths
|
||||
* stored in FileIdTracker.
|
||||
*/
|
||||
lazy val getPath = udf((p: String) => new Path(p).toUri.getPath)
|
||||
}
|
||||
|
|
|
@ -16,13 +16,20 @@
|
|||
|
||||
package com.microsoft.hyperspace.index.dataskipping
|
||||
|
||||
import org.apache.spark.sql.{Column, DataFrame, SaveMode}
|
||||
import org.apache.spark.sql.functions.{input_file_name, min, spark_partition_id}
|
||||
import scala.collection.mutable
|
||||
|
||||
import org.apache.spark.sql.{Column, DataFrame, SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.functions.input_file_name
|
||||
import org.apache.spark.sql.hyperspace.utils.StructTypeUtils
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import com.microsoft.hyperspace.HyperspaceException
|
||||
import com.microsoft.hyperspace.index._
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketch.Sketch
|
||||
import com.microsoft.hyperspace.index.dataskipping.util.{DataFrameUtils, ExpressionUtils}
|
||||
import com.microsoft.hyperspace.index.dataskipping.expressions._
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches.{PartitionSketch, Sketch}
|
||||
import com.microsoft.hyperspace.index.dataskipping.util.DataFrameUtils
|
||||
import com.microsoft.hyperspace.util.HyperspaceConf
|
||||
|
||||
/**
|
||||
|
@ -30,10 +37,12 @@ import com.microsoft.hyperspace.util.HyperspaceConf
|
|||
* files in relations using sketches.
|
||||
*
|
||||
* @param sketches List of sketches for this index
|
||||
* @param schema Index data schema
|
||||
* @param properties Properties for this index; see [[Index.properties]] for details.
|
||||
*/
|
||||
case class DataSkippingIndex(
|
||||
sketches: Seq[Sketch],
|
||||
schema: StructType,
|
||||
override val properties: Map[String, String] = Map.empty)
|
||||
extends Index {
|
||||
assert(sketches.nonEmpty, "At least one sketch is required.")
|
||||
|
@ -61,7 +70,8 @@ case class DataSkippingIndex(
|
|||
}
|
||||
|
||||
override def optimize(ctx: IndexerContext, indexDataFilesToOptimize: Seq[FileInfo]): Unit = {
|
||||
val indexData = ctx.spark.read.parquet(indexDataFilesToOptimize.map(_.name): _*)
|
||||
val indexData =
|
||||
ctx.spark.read.schema(schema).parquet(indexDataFilesToOptimize.map(_.name): _*)
|
||||
writeImpl(ctx, indexData, SaveMode.Overwrite)
|
||||
}
|
||||
|
||||
|
@ -71,7 +81,10 @@ case class DataSkippingIndex(
|
|||
deletedSourceDataFiles: Seq[FileInfo],
|
||||
indexContent: Content): (Index, Index.UpdateMode) = {
|
||||
if (appendedSourceData.nonEmpty) {
|
||||
writeImpl(ctx, index(ctx, appendedSourceData.get), SaveMode.Overwrite)
|
||||
writeImpl(
|
||||
ctx,
|
||||
DataSkippingIndex.createIndexData(ctx, sketches, appendedSourceData.get),
|
||||
SaveMode.Overwrite)
|
||||
}
|
||||
if (deletedSourceDataFiles.nonEmpty) {
|
||||
val spark = ctx.spark
|
||||
|
@ -98,23 +111,188 @@ case class DataSkippingIndex(
|
|||
override def refreshFull(
|
||||
ctx: IndexerContext,
|
||||
sourceData: DataFrame): (DataSkippingIndex, DataFrame) = {
|
||||
val updatedIndex = copy(sketches = ExpressionUtils.resolve(ctx.spark, sketches, sourceData))
|
||||
(updatedIndex, updatedIndex.index(ctx, sourceData))
|
||||
val resolvedSketches = ExpressionUtils.resolve(ctx.spark, sketches, sourceData)
|
||||
val indexData = DataSkippingIndex.createIndexData(ctx, resolvedSketches, sourceData)
|
||||
val updatedIndex = copy(sketches = resolvedSketches, schema = indexData.schema)
|
||||
(updatedIndex, indexData)
|
||||
}
|
||||
|
||||
override def equals(that: Any): Boolean =
|
||||
that match {
|
||||
case DataSkippingIndex(thatSketches, _) => sketches.toSet == thatSketches.toSet
|
||||
case DataSkippingIndex(thatSketches, thatSchema, _) =>
|
||||
sketches.toSet == thatSketches.toSet && schema == thatSchema
|
||||
case _ => false
|
||||
}
|
||||
|
||||
override def hashCode: Int = sketches.map(_.hashCode).sum
|
||||
|
||||
/**
|
||||
* Translate the given filter/join condition for the source data to a
|
||||
* predicate that can be used to filter out unnecessary source data files
|
||||
* when applied to index data.
|
||||
*
|
||||
* For example, a filter condition "A = 1" can be translated into an index
|
||||
* predicate "Min_A <= 1 && Max_A >= 1" to filter out files which cannot
|
||||
* satisfy the condition for any rows in the file.
|
||||
*
|
||||
* It is assumed that the condition is in negation normal form. If it is not,
|
||||
* then it may fail to translate the condition which would have been possible
|
||||
* otherwise. This is a valid assumption for Spark 2.4 and later.
|
||||
*/
|
||||
def translateFilterCondition(
|
||||
spark: SparkSession,
|
||||
condition: Expression,
|
||||
source: LogicalPlan): Option[Expression] = {
|
||||
val resolvedExprs =
|
||||
ExpressionUtils.getResolvedExprs(spark, sketches, source).getOrElse { return None }
|
||||
val predMap = buildPredicateMap(condition, source, resolvedExprs)
|
||||
|
||||
// Create a single index predicate for a single source predicate node,
|
||||
// by combining individual index predicates with And.
|
||||
// True is returned if there are no index predicates for the source predicate node.
|
||||
def toIndexPred(sourcePred: Expression): Expression = {
|
||||
predMap.get(sourcePred).map(_.reduceLeft(And)).getOrElse(Literal.TrueLiteral)
|
||||
}
|
||||
|
||||
// Compose an index predicate visiting the source predicate tree recursively.
|
||||
def composeIndexPred(sourcePred: Expression): Expression =
|
||||
sourcePred match {
|
||||
case and: And => And(toIndexPred(and), and.mapChildren(composeIndexPred))
|
||||
case or: Or => And(toIndexPred(or), or.mapChildren(composeIndexPred))
|
||||
case leaf => toIndexPred(leaf)
|
||||
}
|
||||
|
||||
val indexPredicate = composeIndexPred(condition)
|
||||
|
||||
// Apply constant folding to get the final predicate.
|
||||
// This is a trimmed down version of the BooleanSimplification rule.
|
||||
// It's just enough to determine whether the index is applicable or not.
|
||||
val optimizePredicate: PartialFunction[Expression, Expression] = {
|
||||
case And(Literal.TrueLiteral, right) => right
|
||||
case And(left, Literal.TrueLiteral) => left
|
||||
case Or(Literal.TrueLiteral, _) => Literal.TrueLiteral
|
||||
case Or(_, Literal.TrueLiteral) => Literal.TrueLiteral
|
||||
}
|
||||
val optimizedIndexPredicate = indexPredicate.transformUp(optimizePredicate)
|
||||
|
||||
// Return None if the index predicate is True - meaning no conversion can be done.
|
||||
if (optimizedIndexPredicate == Literal.TrueLiteral) {
|
||||
None
|
||||
} else {
|
||||
Some(optimizedIndexPredicate)
|
||||
}
|
||||
}
|
||||
|
||||
private def writeImpl(ctx: IndexerContext, indexData: DataFrame, writeMode: SaveMode): Unit = {
|
||||
// require instead of assert, as the condition can potentially be broken by
|
||||
// code which is external to dataskipping.
|
||||
require(
|
||||
indexData.schema.sameType(schema),
|
||||
"Schema of the index data doesn't match the index schema: " +
|
||||
s"index data schema = ${indexData.schema.toDDL}, index schema = ${schema.toDDL}")
|
||||
indexData.cache()
|
||||
indexData.count() // force cache
|
||||
val indexDataSize = DataFrameUtils.getSizeInBytes(indexData)
|
||||
val targetIndexDataFileSize = HyperspaceConf.DataSkipping.targetIndexDataFileSize(ctx.spark)
|
||||
val maxIndexDataFileCount = HyperspaceConf.DataSkipping.maxIndexDataFileCount(ctx.spark)
|
||||
val numFiles = {
|
||||
val n = indexDataSize / targetIndexDataFileSize
|
||||
math.min(math.max(1, n), maxIndexDataFileCount).toInt
|
||||
}
|
||||
val repartitionedIndexData = indexData.repartition(numFiles)
|
||||
repartitionedIndexData.write.mode(writeMode).parquet(ctx.indexDataPath.toString)
|
||||
indexData.unpersist()
|
||||
}
|
||||
|
||||
/**
|
||||
* Collects index predicates for each node in the source predicate.
|
||||
*/
|
||||
private def buildPredicateMap(
|
||||
predicate: Expression,
|
||||
source: LogicalPlan,
|
||||
resolvedExprs: Map[Sketch, Seq[Expression]])
|
||||
: scala.collection.Map[Expression, Seq[Expression]] = {
|
||||
val predMap = mutable.Map[Expression, mutable.Buffer[Expression]]()
|
||||
val sketchesWithIndex = sketches.zipWithIndex
|
||||
val nameMap = source.output.map(attr => attr.exprId -> attr.name).toMap
|
||||
val attrMap = buildAttrMap(predicate, resolvedExprs, nameMap)
|
||||
val valueExtractor = AttrValueExtractor(attrMap)
|
||||
def updatePredMap(sourcePred: Expression): Unit = {
|
||||
val indexPreds = sketchesWithIndex.flatMap {
|
||||
case (sketch, idx) =>
|
||||
sketch.convertPredicate(
|
||||
sourcePred,
|
||||
resolvedExprs(sketch),
|
||||
aggrNames(idx).map(UnresolvedAttribute.quoted),
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
}
|
||||
if (indexPreds.nonEmpty) {
|
||||
predMap.getOrElseUpdate(sourcePred, mutable.Buffer.empty) ++= indexPreds
|
||||
}
|
||||
}
|
||||
def forEachTerm(p: Expression, f: Expression => Unit): Unit = {
|
||||
f(p)
|
||||
p match {
|
||||
case And(_, _) | Or(_, _) => p.children.foreach(forEachTerm(_, f))
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
forEachTerm(predicate, updatePredMap)
|
||||
predMap
|
||||
}
|
||||
|
||||
private def buildAttrMap(
|
||||
predicate: Expression,
|
||||
resolvedExprs: Map[Sketch, Seq[Expression]],
|
||||
nameMap: Map[ExprId, String]): Map[Attribute, Expression] = {
|
||||
val partitionSketchIdx = sketches.indexWhere(_.isInstanceOf[PartitionSketch])
|
||||
if (partitionSketchIdx != -1) {
|
||||
val partitionSketch = sketches(partitionSketchIdx)
|
||||
val sketchValues = aggrNames(partitionSketchIdx).map(UnresolvedAttribute.quoted)
|
||||
val exprExtractors = resolvedExprs(partitionSketch).map(NormalizedExprExtractor(_, nameMap))
|
||||
val exprsAndValues = exprExtractors.zip(sketchValues)
|
||||
predicate.references
|
||||
.flatMap(a => exprsAndValues.find(_._1.unapply(a).isDefined).map(a -> _._2))
|
||||
.toMap
|
||||
} else {
|
||||
Map.empty
|
||||
}
|
||||
}
|
||||
|
||||
private def aggrNames(i: Int): Seq[String] = {
|
||||
aggregateFunctions
|
||||
.slice(sketchOffsets(i), sketchOffsets(i + 1))
|
||||
.map(_.expr.asInstanceOf[NamedExpression].name)
|
||||
}
|
||||
|
||||
/**
|
||||
* Sketch offsets are used to map each sketch to its corresponding columns
|
||||
* in the dataframe.
|
||||
*/
|
||||
@transient
|
||||
private lazy val sketchOffsets: Seq[Int] =
|
||||
sketches.map(_.aggregateFunctions.length).scanLeft(0)(_ + _)
|
||||
|
||||
@transient
|
||||
private lazy val aggregateFunctions = DataSkippingIndex.getNamedAggregateFunctions(sketches)
|
||||
}
|
||||
|
||||
object DataSkippingIndex {
|
||||
// $COVERAGE-OFF$ https://github.com/scoverage/scalac-scoverage-plugin/issues/125
|
||||
final val kind = "DataSkippingIndex"
|
||||
final val kindAbbr = "DS"
|
||||
// $COVERAGE-ON$
|
||||
|
||||
/**
|
||||
* Creates index data for the given source data.
|
||||
*/
|
||||
def index(ctx: IndexerContext, sourceData: DataFrame): DataFrame = {
|
||||
def createIndexData(
|
||||
ctx: IndexerContext,
|
||||
sketches: Seq[Sketch],
|
||||
sourceData: DataFrame): DataFrame = {
|
||||
val fileNameCol = "input_file_name"
|
||||
val aggregateFunctions = getNamedAggregateFunctions(sketches)
|
||||
val indexDataWithFileName = sourceData
|
||||
.groupBy(input_file_name().as(fileNameCol))
|
||||
.agg(aggregateFunctions.head, aggregateFunctions.tail: _*)
|
||||
|
@ -124,44 +302,21 @@ case class DataSkippingIndex(
|
|||
val relation = RelationUtils.getRelation(spark, sourceData.queryExecution.optimizedPlan)
|
||||
import spark.implicits._
|
||||
val fileIdDf = ctx.fileIdTracker
|
||||
.getIdToFileMapping(relation.pathNormalizer)
|
||||
.getIdToFileMapping()
|
||||
.toDF(IndexConstants.DATA_FILE_NAME_ID, fileNameCol)
|
||||
|
||||
indexDataWithFileName
|
||||
.join(
|
||||
fileIdDf.hint("broadcast"),
|
||||
IndexUtils.decodeInputFileName(indexDataWithFileName(fileNameCol)) ===
|
||||
fileIdDf(fileNameCol))
|
||||
IndexUtils.getPath(IndexUtils.decodeInputFileName(indexDataWithFileName(fileNameCol))) ===
|
||||
IndexUtils.getPath(fileIdDf(fileNameCol)))
|
||||
.select(
|
||||
IndexConstants.DATA_FILE_NAME_ID,
|
||||
indexDataWithFileName.columns.filterNot(_ == fileNameCol).map(c => s"`$c`"): _*)
|
||||
}
|
||||
|
||||
private def writeImpl(ctx: IndexerContext, indexData: DataFrame, writeMode: SaveMode): Unit = {
|
||||
indexData.cache()
|
||||
indexData.count() // force cache
|
||||
val indexDataSize = DataFrameUtils.getSizeInBytes(indexData)
|
||||
val targetIndexDataFileSize = HyperspaceConf.DataSkipping.targetIndexDataFileSize(ctx.spark)
|
||||
val numFiles = indexDataSize / targetIndexDataFileSize
|
||||
if (!numFiles.isValidInt) {
|
||||
throw HyperspaceException(
|
||||
"Could not create index data files due to too many files: " +
|
||||
s"indexDataSize=$indexDataSize, targetIndexDataFileSize=$targetIndexDataFileSize")
|
||||
}
|
||||
val repartitionedIndexData = indexData.repartition(math.max(1, numFiles.toInt))
|
||||
repartitionedIndexData.write.mode(writeMode).parquet(ctx.indexDataPath.toString)
|
||||
indexData.unpersist()
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a normalized column name valid for a Parquet format.
|
||||
*/
|
||||
private def getNormalizeColumnName(name: String): String = {
|
||||
name.replaceAll("[ ,;{}()\n\t=]", "_")
|
||||
}
|
||||
|
||||
@transient
|
||||
private lazy val aggregateFunctions = sketches.flatMap { s =>
|
||||
def getNamedAggregateFunctions(sketches: Seq[Sketch]): Seq[Column] = {
|
||||
sketches.flatMap { s =>
|
||||
val aggrs = s.aggregateFunctions
|
||||
assert(aggrs.nonEmpty)
|
||||
aggrs.zipWithIndex.map {
|
||||
|
@ -171,9 +326,10 @@ case class DataSkippingIndex(
|
|||
}
|
||||
}
|
||||
|
||||
object DataSkippingIndex {
|
||||
// $COVERAGE-OFF$ https://github.com/scoverage/scalac-scoverage-plugin/issues/125
|
||||
final val kind = "DataSkippingIndex"
|
||||
final val kindAbbr = "DS"
|
||||
// $COVERAGE-ON$
|
||||
/**
|
||||
* Returns a normalized column name valid for a Parquet format.
|
||||
*/
|
||||
private def getNormalizeColumnName(name: String): String = {
|
||||
name.replaceAll("[ ,;{}()\n\t=]", "_")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,14 +16,13 @@
|
|||
|
||||
package com.microsoft.hyperspace.index.dataskipping
|
||||
|
||||
import scala.collection.mutable
|
||||
import org.apache.spark.sql.{DataFrame, SparkSession}
|
||||
|
||||
import org.apache.spark.sql.DataFrame
|
||||
|
||||
import com.microsoft.hyperspace.HyperspaceException
|
||||
import com.microsoft.hyperspace.{Hyperspace, HyperspaceException}
|
||||
import com.microsoft.hyperspace.index.{IndexConfigTrait, IndexerContext}
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketch.Sketch
|
||||
import com.microsoft.hyperspace.index.dataskipping.util.ExpressionUtils
|
||||
import com.microsoft.hyperspace.index.dataskipping.expressions.ExpressionUtils
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches.{PartitionSketch, Sketch}
|
||||
import com.microsoft.hyperspace.util.HyperspaceConf
|
||||
|
||||
/**
|
||||
* DataSkippingIndexConfig is used to create a [[DataSkippingIndex]] via
|
||||
|
@ -59,9 +58,29 @@ case class DataSkippingIndexConfig(
|
|||
sourceData: DataFrame,
|
||||
properties: Map[String, String]): (DataSkippingIndex, DataFrame) = {
|
||||
val resolvedSketches = ExpressionUtils.resolve(ctx.spark, sketches, sourceData)
|
||||
checkDuplicateSketches(resolvedSketches)
|
||||
val index = DataSkippingIndex(resolvedSketches, properties)
|
||||
(index, index.index(ctx, sourceData))
|
||||
val autoPartitionSketch = HyperspaceConf.DataSkipping.autoPartitionSketch(ctx.spark)
|
||||
val partitionSketchOpt =
|
||||
if (autoPartitionSketch) getPartitionSketch(ctx.spark, sourceData)
|
||||
else None
|
||||
val finalSketches = partitionSketchOpt.toSeq ++ resolvedSketches
|
||||
checkDuplicateSketches(finalSketches)
|
||||
val indexData = DataSkippingIndex.createIndexData(ctx, finalSketches, sourceData)
|
||||
val index = DataSkippingIndex(finalSketches, indexData.schema, properties)
|
||||
(index, indexData)
|
||||
}
|
||||
|
||||
private def getPartitionSketch(
|
||||
spark: SparkSession,
|
||||
sourceData: DataFrame): Option[PartitionSketch] = {
|
||||
val relation = Hyperspace
|
||||
.getContext(spark)
|
||||
.sourceProviderManager
|
||||
.getRelation(sourceData.queryExecution.optimizedPlan)
|
||||
if (relation.partitionSchema.nonEmpty) {
|
||||
Some(PartitionSketch(relation.partitionSchema.map(f => (f.name, Some(f.dataType)))))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
private def checkDuplicateSketches(sketches: Seq[Sketch]): Unit = {
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.execution
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.{Column, DataFrame, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.functions.isnull
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import com.microsoft.hyperspace.index.{FileIdTracker, IndexConstants}
|
||||
|
||||
/**
|
||||
* FileIndex implementation that filters out files using the index data and
|
||||
* index predicates generated by applying a data skipping index to the source.
|
||||
*/
|
||||
class DataSkippingFileIndex(
|
||||
sparkSession: SparkSession,
|
||||
fileIdTracker: FileIdTracker,
|
||||
indexData: DataFrame,
|
||||
private[dataskipping] val indexDataPred: Expression, // exposed for test
|
||||
private[dataskipping] val baseFileIndex: FileIndex)
|
||||
extends FileIndex {
|
||||
|
||||
override def listFiles(
|
||||
partitionFilters: Seq[Expression],
|
||||
dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
|
||||
val partitions = baseFileIndex.listFiles(partitionFilters, dataFilters)
|
||||
import sparkSession.implicits._
|
||||
val pathCol = "__path"
|
||||
val filesWithId = partitions
|
||||
.flatMap(_.files.map(f => (f.getPath.toString, fileIdTracker.addFile(f))))
|
||||
.toDF(pathCol, IndexConstants.DATA_FILE_NAME_ID)
|
||||
val selectedFiles = filesWithId
|
||||
.hint("broadcast")
|
||||
.join(indexData, Seq(IndexConstants.DATA_FILE_NAME_ID), "left")
|
||||
.filter(isnull(indexData(IndexConstants.DATA_FILE_NAME_ID)) || new Column(indexDataPred))
|
||||
.select(pathCol)
|
||||
.collect
|
||||
.map(_.getString(0))
|
||||
.toSet
|
||||
val selectedPartitions = partitions
|
||||
.map(p => p.copy(files = p.files.filter(f => selectedFiles.contains(f.getPath.toString))))
|
||||
.filter(_.files.nonEmpty)
|
||||
selectedPartitions
|
||||
}
|
||||
|
||||
override def rootPaths: Seq[Path] = baseFileIndex.rootPaths
|
||||
|
||||
override def inputFiles: Array[String] = baseFileIndex.inputFiles
|
||||
|
||||
override def refresh(): Unit = baseFileIndex.refresh()
|
||||
|
||||
override def sizeInBytes: Long = baseFileIndex.sizeInBytes
|
||||
|
||||
override def partitionSchema: StructType = baseFileIndex.partitionSchema
|
||||
|
||||
override def metadataOpsTimeNs: Option[Long] = baseFileIndex.metadataOpsTimeNs
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SubqueryExpression}
|
||||
|
||||
case class AttrValueExtractor(attrMap: Map[Attribute, Expression]) extends ExpressionExtractor {
|
||||
override def unapply(e: Expression): Option[Expression] = {
|
||||
if (canTransform(e)) Some(transform(e)) else None
|
||||
}
|
||||
|
||||
private def canTransform(e: Expression): Boolean = {
|
||||
e.deterministic &&
|
||||
e.references.forall(attrMap.contains) &&
|
||||
!SubqueryExpression.hasSubquery(e)
|
||||
}
|
||||
|
||||
private def transform(e: Expression): Expression = {
|
||||
e.transform { case a: Attribute => attrMap(a) }
|
||||
}
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
|
||||
trait ExpressionExtractor {
|
||||
def unapply(e: Expression): Option[Expression]
|
||||
}
|
|
@ -14,26 +14,23 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import java.util.UUID
|
||||
|
||||
import org.apache.spark.sql.{DataFrame, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Project, Window}
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.types.{BooleanType, DataType}
|
||||
|
||||
import com.microsoft.hyperspace.HyperspaceException
|
||||
import com.microsoft.hyperspace.index.IndexUtils
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketch.Sketch
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches.Sketch
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace.withHyperspaceRuleDisabled
|
||||
|
||||
object ExpressionUtils {
|
||||
|
||||
val nullExprId = ExprId(0, new UUID(0, 0))
|
||||
|
||||
/**
|
||||
* Returns copies of the given sketches with the indexed columns replaced by
|
||||
* resolved column names and data types.
|
||||
|
@ -60,9 +57,8 @@ object ExpressionUtils {
|
|||
private def checkExprs(
|
||||
exprWithExpectedDataTypes: Seq[(String, Option[DataType])],
|
||||
sourceData: DataFrame): Seq[DataType] = {
|
||||
val (exprs, expectedDataTypes) =
|
||||
(exprWithExpectedDataTypes.map(_._1), exprWithExpectedDataTypes.map(_._2))
|
||||
def throwNotSupportedIf(cond: Boolean, msg: => String) = {
|
||||
val exprs = exprWithExpectedDataTypes.map(_._1)
|
||||
def throwNotSupportedIf(cond: Boolean, msg: => String): Unit = {
|
||||
if (cond) {
|
||||
throw HyperspaceException(s"DataSkippingIndex does not support indexing $msg")
|
||||
}
|
||||
|
@ -87,7 +83,7 @@ object ExpressionUtils {
|
|||
e.find(_.isInstanceOf[SubqueryExpression]).nonEmpty,
|
||||
s"an expression which has a subquery: $expr")
|
||||
throwNotSupportedIf(
|
||||
e.find(_.isInstanceOf[AttributeReference]).isEmpty,
|
||||
e.find(_.isInstanceOf[Attribute]).isEmpty,
|
||||
s"an expression which does not reference source columns: $expr")
|
||||
if (expectedDataType.nonEmpty && expectedDataType.get != analyzedExpr.dataType) {
|
||||
throw HyperspaceException(
|
||||
|
@ -101,14 +97,11 @@ object ExpressionUtils {
|
|||
/**
|
||||
* Used to workaround the issue where UnresolvedAttribute.sql() doesn't work as expected.
|
||||
*/
|
||||
private case class QuotedAttribute(name: String) extends LeafExpression {
|
||||
private case class QuotedAttribute(name: String) extends LeafExpression with Unevaluable {
|
||||
override def sql: String = name
|
||||
|
||||
// $COVERAGE-OFF$ code never used
|
||||
override def nullable: Boolean = throw new NotImplementedError
|
||||
override def eval(input: InternalRow): Any = throw new NotImplementedError
|
||||
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
|
||||
throw new NotImplementedError
|
||||
override def dataType: DataType = throw new NotImplementedError
|
||||
// $COVERAGE-ON$
|
||||
}
|
||||
|
@ -121,12 +114,33 @@ object ExpressionUtils {
|
|||
*/
|
||||
def normalize(expr: Expression): Expression = {
|
||||
expr.transformUp {
|
||||
case a: AttributeReference => a.withExprId(nullExprId).withQualifier(Nil)
|
||||
case a: Attribute => a.withExprId(nullExprId).withQualifier(Nil)
|
||||
case g @ GetStructField(child, ordinal, _) => g.copy(child, ordinal, None)
|
||||
// Undo HandleNullInputsForUDF and ReplaceNullWithFalseInPredicate so
|
||||
// that we can match scala UDF invocations. HandleNullInputsForUDF rule
|
||||
// transforms ScalaUDF(args...) into If(Or(IsNull(args)...), null,
|
||||
// ScalaUDF(KnownNotNull(args)...)), and ReplaceNullWithFalseInPredicate
|
||||
// rule transforms null into false. ReplaceNullWithFalseInPredicate is
|
||||
// sensitive to the tree shape.
|
||||
// This is a safe operation because we ignore null values when computing
|
||||
// sketch values. For example, MinMaxSketch("f(a)") will only collect
|
||||
// min/max values of non-null results of "f(a)". Then we can use those
|
||||
// sketch values to translate filter conditions like "f(a) = 1". Here,
|
||||
// we are only interested in whether those "f(a)" refers to the same
|
||||
// thing, not how they should be evaluated. Normalized expressions are
|
||||
// only meant to be compared, not evaluated.
|
||||
case If(
|
||||
ExtractIsNullDisjunction(args1),
|
||||
Literal(null | false, dataType1),
|
||||
udf @ ExtractScalaUDF(dataType2, ExtractKnownNotNullArgs(args2)))
|
||||
if args1 == args2 && dataType1 == dataType2 =>
|
||||
udf.copy(children = args2)
|
||||
}
|
||||
}
|
||||
|
||||
// Needed because ScalaUDF has a different number of arguments depending on Spark versions.
|
||||
// Exposed for test
|
||||
private[dataskipping] val nullExprId: ExprId = ExprId(0, new UUID(0, 0))
|
||||
|
||||
private[dataskipping] object ExtractScalaUDF {
|
||||
def unapply(e: ScalaUDF): Option[(DataType, Seq[Expression])] = {
|
||||
Some((e.dataType, e.children))
|
||||
|
@ -151,4 +165,53 @@ object ExpressionUtils {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns sketch expressions that can be used to match indexed expressions
|
||||
* and expressions in the filter condition. For example, when a user creates
|
||||
* an index with MinMaxSketch("A"), we create an expression corresponding to
|
||||
* "A" here, and later we try to match expression nodes in a filter condition,
|
||||
* say, EqualTo(AttributeReference("A"), Literal(1)), to the expression for
|
||||
* "A".
|
||||
*
|
||||
* We need this step as the filter/join conditions are given to us as a tree
|
||||
* of expressions in the Spark's optimizer, whereas the indexed expressions
|
||||
* are provided and stored as strings.
|
||||
*/
|
||||
def getResolvedExprs(
|
||||
spark: SparkSession,
|
||||
sketches: Seq[Sketch],
|
||||
source: LogicalPlan): Option[Map[Sketch, Seq[Expression]]] = {
|
||||
val resolvedExprs = sketches.map { s =>
|
||||
s -> s.expressions.map {
|
||||
case (expr, dataTypeOpt) =>
|
||||
val parsedExpr = spark.sessionState.sqlParser.parseExpression(expr)
|
||||
val dataType = dataTypeOpt.get
|
||||
val filter = dataType match {
|
||||
case BooleanType => Filter(parsedExpr, source)
|
||||
case _ => Filter(PredicateWrapper(parsedExpr), source)
|
||||
}
|
||||
val optimizedFilter = withHyperspaceRuleDisabled {
|
||||
spark.sessionState.optimizer
|
||||
.execute(spark.sessionState.analyzer.execute(filter))
|
||||
.asInstanceOf[Filter]
|
||||
}
|
||||
val resolvedExpr = dataType match {
|
||||
case BooleanType => optimizedFilter.condition
|
||||
case _ => optimizedFilter.condition.asInstanceOf[PredicateWrapper].child
|
||||
}
|
||||
if (resolvedExpr.dataType != dataType) {
|
||||
return None
|
||||
}
|
||||
normalize(resolvedExpr)
|
||||
}
|
||||
}.toMap
|
||||
Some(resolvedExprs)
|
||||
}
|
||||
|
||||
// Used to preserve sketch expressions during optimization
|
||||
private case class PredicateWrapper(override val child: Expression)
|
||||
extends UnaryExpression
|
||||
with Unevaluable
|
||||
with Predicate
|
||||
}
|
|
@ -14,17 +14,15 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExprId}
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, ExprId}
|
||||
|
||||
case class NormalizedExprMatcher(expr: Expression, nameMap: Map[ExprId, String])
|
||||
extends ExprMatcher {
|
||||
def apply(e: Expression): Boolean = {
|
||||
val renamed = e.transformUp {
|
||||
case a: AttributeReference => a.withName(nameMap(a.exprId))
|
||||
}
|
||||
case class NormalizedExprExtractor(expr: Expression, nameMap: Map[ExprId, String])
|
||||
extends ExpressionExtractor {
|
||||
def unapply(e: Expression): Option[Expression] = {
|
||||
val renamed = e.transformUp { case a: Attribute => a.withName(nameMap(a.exprId)) }
|
||||
val normalized = ExpressionUtils.normalize(renamed)
|
||||
expr == normalized
|
||||
if (expr == normalized) Some(expr) else None
|
||||
}
|
||||
}
|
|
@ -14,11 +14,11 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Expression}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
|
||||
import org.apache.spark.sql.catalyst.util.{ArrayData, TypeUtils}
|
||||
import org.apache.spark.sql.types.IntegerType
|
||||
|
@ -28,12 +28,13 @@ import org.apache.spark.sql.types.IntegerType
|
|||
* than (greater than or equal to) the value (right), or null if there is no such
|
||||
* element.
|
||||
*
|
||||
* If the value (right) is null, null is returned.
|
||||
*
|
||||
* Preconditions (unchecked):
|
||||
* - The array must not be null.
|
||||
* - Elements in the array must be in ascending order.
|
||||
* - The array must not contain null elements.
|
||||
* - The array must not contain duplicate elements.
|
||||
* - The value must not be null.
|
||||
*/
|
||||
private[dataskipping] case class SortedArrayLowerBound(left: Expression, right: Expression)
|
||||
extends BinaryExpression {
|
||||
|
@ -47,6 +48,7 @@ private[dataskipping] case class SortedArrayLowerBound(left: Expression, right:
|
|||
override def eval(input: InternalRow): Any = {
|
||||
val arr = left.eval(input).asInstanceOf[ArrayData]
|
||||
val value = right.eval(input)
|
||||
if (value != null) {
|
||||
val dt = right.dataType
|
||||
val n = arr.numElements()
|
||||
if (n > 0) {
|
||||
|
@ -58,6 +60,7 @@ private[dataskipping] case class SortedArrayLowerBound(left: Expression, right:
|
|||
return index + 1
|
||||
}
|
||||
}
|
||||
}
|
||||
null
|
||||
}
|
||||
|
||||
|
@ -73,6 +76,7 @@ private[dataskipping] case class SortedArrayLowerBound(left: Expression, right:
|
|||
val binarySearch = SortedArrayUtils.binarySearchCodeGen(ctx, dt)
|
||||
val resultCode =
|
||||
s"""
|
||||
|if (!(${rightGen.isNull})) {
|
||||
| int $n = $arr.numElements();
|
||||
| if ($n > 0) {
|
||||
| if (!(${ctx.genGreater(dt, value, firstValueInArr)})) {
|
||||
|
@ -83,6 +87,7 @@ private[dataskipping] case class SortedArrayLowerBound(left: Expression, right:
|
|||
| ${ev.value} = $binarySearch($arr, 0, $n, $value).index() + 1;
|
||||
| }
|
||||
| }
|
||||
|}
|
||||
""".stripMargin
|
||||
ev.copy(code = code"""
|
||||
${leftGen.code}
|
|
@ -14,7 +14,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator}
|
||||
import org.apache.spark.sql.catalyst.util.ArrayData
|
|
@ -0,0 +1,123 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types.BooleanType
|
||||
|
||||
case class EqualToExtractor(left: ExpressionExtractor, right: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[(Expression, Expression)] =
|
||||
p match {
|
||||
case EqualTo(left(l), right(r)) => Some((l, r))
|
||||
case EqualTo(right(r), left(l)) => Some((l, r))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class EqualNullSafeExtractor(left: ExpressionExtractor, right: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[(Expression, Expression)] =
|
||||
p match {
|
||||
case EqualNullSafe(left(l), right(r)) => Some((l, r))
|
||||
case EqualNullSafe(right(r), left(l)) => Some((l, r))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class LessThanExtractor(left: ExpressionExtractor, right: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[(Expression, Expression)] =
|
||||
p match {
|
||||
case LessThan(left(l), right(r)) => Some((l, r))
|
||||
case GreaterThan(right(r), left(l)) => Some((l, r))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class LessThanOrEqualExtractor(left: ExpressionExtractor, right: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[(Expression, Expression)] =
|
||||
p match {
|
||||
case LessThanOrEqual(left(l), right(r)) => Some((l, r))
|
||||
case GreaterThanOrEqual(right(r), left(l)) => Some((l, r))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class IsNullExtractor(expr: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[Expression] =
|
||||
p match {
|
||||
case IsNull(expr(e)) => Some(e)
|
||||
case EqualNullSafe(expr(e), v: Literal) if v.value == null => Some(e)
|
||||
case EqualNullSafe(v: Literal, expr(e)) if v.value == null => Some(e)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class IsNotNullExtractor(expr: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[Expression] =
|
||||
p match {
|
||||
case IsNotNull(expr(e)) => Some(e)
|
||||
// Spark 2.4 lacks a rule transforming Not(IsNull) to IsNotNull
|
||||
case Not(IsNull(expr(e))) => Some(e)
|
||||
case Not(EqualNullSafe(expr(e), v: Literal)) if v.value == null => Some(e)
|
||||
case Not(EqualNullSafe(v: Literal, expr(e))) if v.value == null => Some(e)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class IsTrueExtractor(expr: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[Expression] =
|
||||
p.dataType match {
|
||||
case BooleanType =>
|
||||
p match {
|
||||
case expr(e) => Some(e)
|
||||
case EqualTo(expr(e), Literal(true, BooleanType)) => Some(e)
|
||||
case EqualTo(Literal(true, BooleanType), expr(e)) => Some(e)
|
||||
case _ => None
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class IsFalseExtractor(expr: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[Expression] =
|
||||
p.dataType match {
|
||||
case BooleanType =>
|
||||
p match {
|
||||
case Not(expr(e)) => Some(e)
|
||||
case EqualTo(expr(e), Literal(false, BooleanType)) => Some(e)
|
||||
case EqualTo(Literal(false, BooleanType), expr(e)) => Some(e)
|
||||
case _ => None
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class InExtractor(arg: ExpressionExtractor, element: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[(Expression, Seq[Expression])] =
|
||||
p match {
|
||||
case In(arg(a), vs) =>
|
||||
Some((a, vs.map(element.unapply(_).getOrElse { return None })))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
case class InSetExtractor(arg: ExpressionExtractor) {
|
||||
def unapply(p: Expression): Option[(Expression, Set[Any])] =
|
||||
p match {
|
||||
case InSet(arg(a), vs) => Some((a, vs))
|
||||
case _ => None
|
||||
}
|
||||
}
|
|
@ -0,0 +1,105 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.rules
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.AttributeReference
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.hyperspace.utils.logicalPlanToDataFrame
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import com.microsoft.hyperspace.index.{IndexLogEntry, IndexLogEntryTags}
|
||||
import com.microsoft.hyperspace.index.dataskipping.DataSkippingIndex
|
||||
import com.microsoft.hyperspace.index.dataskipping.execution.DataSkippingFileIndex
|
||||
import com.microsoft.hyperspace.index.plans.logical.IndexHadoopFsRelation
|
||||
import com.microsoft.hyperspace.index.rules._
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace.PlanToSelectedIndexMap
|
||||
|
||||
object ApplyDataSkippingIndex extends HyperspaceRule {
|
||||
protected override val filtersOnQueryPlan: Seq[QueryPlanIndexFilter] =
|
||||
IndexTypeFilter[DataSkippingIndex]() :: FilterPlanNodeFilter :: FilterConditionFilter :: Nil
|
||||
|
||||
protected override val indexRanker: IndexRankFilter = DataSkippingIndexRanker
|
||||
|
||||
override def applyIndex(plan: LogicalPlan, indexes: PlanToSelectedIndexMap): LogicalPlan = {
|
||||
if (indexes.isEmpty) {
|
||||
return plan
|
||||
}
|
||||
plan match {
|
||||
case filter @ Filter(_, ExtractRelation(relation)) =>
|
||||
val indexLogEntry = indexes(relation.plan)
|
||||
val indexDataPred = indexLogEntry
|
||||
.getTagValue(plan, IndexLogEntryTags.DATASKIPPING_INDEX_PREDICATE)
|
||||
.get
|
||||
.getOrElse { return plan }
|
||||
val indexData = logicalPlanToDataFrame(spark, getIndexDataRelation(indexLogEntry))
|
||||
val originalFileIndex = indexLogEntry.withCachedTag(
|
||||
relation.plan,
|
||||
IndexLogEntryTags.DATASKIPPING_SOURCE_FILEINDEX) {
|
||||
relation.getOrCreateFileIndex(spark)
|
||||
}
|
||||
val dataSkippingFileIndex = new DataSkippingFileIndex(
|
||||
spark,
|
||||
indexLogEntry.fileIdTracker,
|
||||
indexData,
|
||||
indexDataPred,
|
||||
originalFileIndex)
|
||||
val newRelation = relation.createLogicalRelation(
|
||||
IndexHadoopFsRelation(
|
||||
relation.createHadoopFsRelation(
|
||||
dataSkippingFileIndex,
|
||||
relation.schema,
|
||||
relation.options),
|
||||
spark,
|
||||
indexLogEntry),
|
||||
relation.output.map(_.asInstanceOf[AttributeReference]))
|
||||
filter.copy(child = newRelation)
|
||||
case _ => plan
|
||||
}
|
||||
}
|
||||
|
||||
override def score(plan: LogicalPlan, indexes: PlanToSelectedIndexMap): Int = {
|
||||
if (indexes.isEmpty) {
|
||||
return 0
|
||||
}
|
||||
// Return the lowest score so that covering indexes take precedence over
|
||||
// data skipping indexes.
|
||||
1
|
||||
}
|
||||
|
||||
private def getIndexDataRelation(indexLogEntry: IndexLogEntry): LogicalRelation = {
|
||||
val indexDataSchema = indexLogEntry.derivedDataset.asInstanceOf[DataSkippingIndex].schema
|
||||
val indexDataLoc =
|
||||
indexLogEntry.withCachedTag(IndexLogEntryTags.DATASKIPPING_INDEX_FILEINDEX) {
|
||||
new InMemoryFileIndex(
|
||||
spark,
|
||||
indexLogEntry.content.files,
|
||||
Map.empty,
|
||||
Some(indexDataSchema),
|
||||
FileStatusCache.getOrCreate(spark))
|
||||
}
|
||||
LogicalRelation(
|
||||
HadoopFsRelation(
|
||||
indexDataLoc,
|
||||
StructType(Nil),
|
||||
indexDataSchema,
|
||||
None,
|
||||
new ParquetFileFormat,
|
||||
Map.empty)(spark))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.rules
|
||||
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
|
||||
import com.microsoft.hyperspace.index.dataskipping.DataSkippingIndex
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace.{PlanToIndexesMap, PlanToSelectedIndexMap}
|
||||
import com.microsoft.hyperspace.index.rules.IndexRankFilter
|
||||
|
||||
object DataSkippingIndexRanker extends IndexRankFilter {
|
||||
override def apply(
|
||||
plan: LogicalPlan,
|
||||
applicableIndexes: PlanToIndexesMap): PlanToSelectedIndexMap = {
|
||||
// TODO: Multiple data skipping index can be applied to the same plan node,
|
||||
// although the effectiveness decreases as more indexes are applied.
|
||||
// The framework should be updated to allow multiple indexes.
|
||||
// For now, simply choose the index with most sketches.
|
||||
applicableIndexes.collect {
|
||||
case (plan, indexes) if indexes.nonEmpty =>
|
||||
plan -> indexes.maxBy(_.derivedDataset.asInstanceOf[DataSkippingIndex].sketches.length)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.rules
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
|
||||
|
||||
import com.microsoft.hyperspace.index.IndexLogEntryTags
|
||||
import com.microsoft.hyperspace.index.dataskipping.DataSkippingIndex
|
||||
import com.microsoft.hyperspace.index.plananalysis.FilterReasons
|
||||
import com.microsoft.hyperspace.index.rules.{ExtractRelation, QueryPlanIndexFilter}
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace.PlanToIndexesMap
|
||||
|
||||
/**
|
||||
* FilterConditionFilter filters indexes out if
|
||||
* 1) an index cannot be applied to the filter condition.
|
||||
*/
|
||||
object FilterConditionFilter extends QueryPlanIndexFilter {
|
||||
override def apply(plan: LogicalPlan, candidateIndexes: PlanToIndexesMap): PlanToIndexesMap = {
|
||||
if (candidateIndexes.isEmpty) {
|
||||
return Map.empty
|
||||
}
|
||||
plan match {
|
||||
case Filter(condition: Expression, ExtractRelation(relation)) =>
|
||||
val applicableIndexes = candidateIndexes(relation.plan).flatMap { indexLogEntry =>
|
||||
val indexDataPredOpt =
|
||||
indexLogEntry.withCachedTag(plan, IndexLogEntryTags.DATASKIPPING_INDEX_PREDICATE) {
|
||||
val index = indexLogEntry.derivedDataset.asInstanceOf[DataSkippingIndex]
|
||||
index.translateFilterCondition(spark, condition, relation.plan)
|
||||
}
|
||||
if (withFilterReasonTag(
|
||||
plan,
|
||||
indexLogEntry,
|
||||
FilterReasons.IneligibleFilterCondition(condition.sql))(
|
||||
indexDataPredOpt.nonEmpty)) {
|
||||
Some(indexLogEntry)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
if (applicableIndexes.nonEmpty) {
|
||||
Map(relation.plan -> applicableIndexes)
|
||||
} else {
|
||||
Map.empty
|
||||
}
|
||||
case _ => Map.empty
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.rules
|
||||
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
|
||||
|
||||
import com.microsoft.hyperspace.index.rules.{ExtractRelation, QueryPlanIndexFilter, RuleUtils}
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace.PlanToIndexesMap
|
||||
|
||||
/**
|
||||
* FilterPlanNodeFilter filters indexes out if
|
||||
* 1) the given plan is not eligible filter plan node.
|
||||
* 2) the source plan of index is not part of the filter plan.
|
||||
*/
|
||||
object FilterPlanNodeFilter extends QueryPlanIndexFilter {
|
||||
override def apply(plan: LogicalPlan, candidateIndexes: PlanToIndexesMap): PlanToIndexesMap = {
|
||||
if (candidateIndexes.isEmpty) {
|
||||
return Map.empty
|
||||
}
|
||||
plan match {
|
||||
case Filter(_, ExtractRelation(relation)) if !RuleUtils.isIndexApplied(relation) =>
|
||||
candidateIndexes.filterKeys(relation.plan.equals)
|
||||
case _ => Map.empty
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,43 +0,0 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketch
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.expressions.aggregate.{Max, Min}
|
||||
import org.apache.spark.sql.types.DataType
|
||||
|
||||
/**
|
||||
* Sketch based on minimum and maximum values for a given expression.
|
||||
*
|
||||
* @param expr Expression from which min/max values are calculated
|
||||
* @param dataType Optional data type to specify the expected data type of the
|
||||
* expression. If not specified, it is deduced automatically.
|
||||
* If the actual data type of the expression is different from this,
|
||||
* an error is thrown. Users are recommended to leave this parameter to
|
||||
* None.
|
||||
*/
|
||||
case class MinMaxSketch(override val expr: String, override val dataType: Option[DataType] = None)
|
||||
extends SingleExprSketch[MinMaxSketch](expr, dataType) {
|
||||
override def name: String = "MinMax"
|
||||
|
||||
override def withNewExpression(newExpr: (String, Option[DataType])): MinMaxSketch = {
|
||||
copy(expr = newExpr._1, dataType = newExpr._2)
|
||||
}
|
||||
|
||||
override def aggregateFunctions: Seq[Expression] =
|
||||
Min(parsedExpr).toAggregateExpression() :: Max(parsedExpr).toAggregateExpression() :: Nil
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketches
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.expressions.aggregate.{Max, Min}
|
||||
import org.apache.spark.sql.catalyst.util.{ArrayData, TypeUtils}
|
||||
import org.apache.spark.sql.types.{ArrayType, DataType}
|
||||
|
||||
import com.microsoft.hyperspace.index.dataskipping.expressions._
|
||||
import com.microsoft.hyperspace.index.dataskipping.util.ArrayUtils
|
||||
|
||||
/**
|
||||
* Sketch based on minimum and maximum values for a given expression.
|
||||
*
|
||||
* @param expr Expression from which min/max values are calculated
|
||||
* @param dataType Optional data type to specify the expected data type of the
|
||||
* expression. If not specified, it is deduced automatically.
|
||||
* If the actual data type of the expression is different from this,
|
||||
* an error is thrown. Users are recommended to leave this parameter to
|
||||
* None.
|
||||
*/
|
||||
case class MinMaxSketch(override val expr: String, override val dataType: Option[DataType] = None)
|
||||
extends SingleExprSketch[MinMaxSketch](expr, dataType) {
|
||||
override def name: String = "MinMax"
|
||||
|
||||
override def withNewExpression(newExpr: (String, Option[DataType])): MinMaxSketch = {
|
||||
copy(expr = newExpr._1, dataType = newExpr._2)
|
||||
}
|
||||
|
||||
override def aggregateFunctions: Seq[Expression] =
|
||||
Min(parsedExpr).toAggregateExpression() :: Max(parsedExpr).toAggregateExpression() :: Nil
|
||||
|
||||
override def convertPredicate(
|
||||
predicate: Expression,
|
||||
resolvedExprs: Seq[Expression],
|
||||
sketchValues: Seq[Expression],
|
||||
nameMap: Map[ExprId, String],
|
||||
valueExtractor: ExpressionExtractor): Option[Expression] = {
|
||||
val min = sketchValues(0)
|
||||
val max = sketchValues(1)
|
||||
// TODO: Add third sketch value "hasNull" of type bool
|
||||
// true if the expr can be null in the file, false if otherwise
|
||||
// to optimize IsNull (can skip files with hasNull = false)
|
||||
// This can be also done as a separate sketch, e.g. HasNullSketch
|
||||
// Should evaluate which way is better
|
||||
val resolvedExpr = resolvedExprs.head
|
||||
val dataType = resolvedExpr.dataType
|
||||
val exprExtractor = NormalizedExprExtractor(resolvedExpr, nameMap)
|
||||
val ExprIsTrue = IsTrueExtractor(exprExtractor)
|
||||
val ExprIsFalse = IsFalseExtractor(exprExtractor)
|
||||
val ExprIsNotNull = IsNotNullExtractor(exprExtractor)
|
||||
val ExprEqualTo = EqualToExtractor(exprExtractor, valueExtractor)
|
||||
val ExprEqualNullSafe = EqualNullSafeExtractor(exprExtractor, valueExtractor)
|
||||
val ExprLessThan = LessThanExtractor(exprExtractor, valueExtractor)
|
||||
val ExprLessThanOrEqualTo = LessThanOrEqualExtractor(exprExtractor, valueExtractor)
|
||||
val ExprGreaterThan = LessThanExtractor(valueExtractor, exprExtractor)
|
||||
val ExprGreaterThanOrEqualTo = LessThanOrEqualExtractor(valueExtractor, exprExtractor)
|
||||
val ExprIn = InExtractor(exprExtractor, valueExtractor)
|
||||
val ExprInSet = InSetExtractor(exprExtractor)
|
||||
Option(predicate)
|
||||
.collect {
|
||||
case ExprIsTrue(_) => max
|
||||
case ExprIsFalse(_) => Not(min)
|
||||
case ExprIsNotNull(_) => IsNotNull(min)
|
||||
case ExprEqualTo(_, v) => And(LessThanOrEqual(min, v), GreaterThanOrEqual(max, v))
|
||||
case ExprEqualNullSafe(_, v) =>
|
||||
Or(IsNull(v), And(LessThanOrEqual(min, v), GreaterThanOrEqual(max, v)))
|
||||
case ExprLessThan(_, v) => LessThan(min, v)
|
||||
case ExprLessThanOrEqualTo(_, v) => LessThanOrEqual(min, v)
|
||||
case ExprGreaterThan(v, _) => GreaterThan(max, v)
|
||||
case ExprGreaterThanOrEqualTo(v, _) => GreaterThanOrEqual(max, v)
|
||||
case ExprIn(_, vs) =>
|
||||
vs.map(v => And(LessThanOrEqual(min, v), GreaterThanOrEqual(max, v))).reduceLeft(Or)
|
||||
case ExprInSet(_, vs) =>
|
||||
val sortedValues = Literal(
|
||||
ArrayData.toArrayData(
|
||||
ArrayUtils.toArray(
|
||||
vs.filter(_ != null).toArray.sorted(TypeUtils.getInterpretedOrdering(dataType)),
|
||||
dataType)),
|
||||
ArrayType(dataType, containsNull = false))
|
||||
LessThanOrEqual(ElementAt(sortedValues, SortedArrayLowerBound(sortedValues, min)), max)
|
||||
// TODO: StartsWith, Like with constant prefix
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketches
|
||||
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types.DataType
|
||||
|
||||
import com.microsoft.hyperspace.index.dataskipping.expressions._
|
||||
import com.microsoft.hyperspace.shim.FirstNullSafe
|
||||
|
||||
/**
|
||||
* Internal implementation for partitioning column.
|
||||
*
|
||||
* This sketch is automatically created for each partitioning column if the
|
||||
* source data has partitioning columns. This enables data skipping indexes to
|
||||
* be usable with filter conditions having a disjunction involving partitioning
|
||||
* columns and indexed columns. For example, a filter condition like "A = 1 or
|
||||
* B = 1" will be translated into an index predicate "(Min_A <= 1 and Max_A >=
|
||||
* 1) or B = 1" where A is an indexed column of MinMaxSketch and B is a
|
||||
* partitioning column.
|
||||
*/
|
||||
private[dataskipping] case class PartitionSketch(
|
||||
override val expressions: Seq[(String, Option[DataType])])
|
||||
extends Sketch {
|
||||
|
||||
override def indexedColumns: Seq[String] = exprStrings
|
||||
|
||||
override def referencedColumns: Seq[String] = exprStrings
|
||||
|
||||
override def withNewExpressions(
|
||||
newExpressions: Seq[(String, Option[DataType])]): PartitionSketch = {
|
||||
copy(expressions = newExpressions)
|
||||
}
|
||||
|
||||
override def aggregateFunctions: Seq[Expression] = {
|
||||
val parser = SparkSession.getActiveSession.get.sessionState.sqlParser
|
||||
exprStrings.map { e =>
|
||||
FirstNullSafe(parser.parseExpression(e)).toAggregateExpression()
|
||||
}
|
||||
}
|
||||
|
||||
override def toString: String = s"Partition(${exprStrings.mkString(", ")})"
|
||||
|
||||
override def convertPredicate(
|
||||
predicate: Expression,
|
||||
resolvedExprs: Seq[Expression],
|
||||
sketchValues: Seq[Expression],
|
||||
nameMap: Map[ExprId, String],
|
||||
valueExtractor: ExpressionExtractor): Option[Expression] = {
|
||||
predicate match {
|
||||
case And(_, _) | Or(_, _) => None
|
||||
case valueExtractor(v) if (predicate.references.nonEmpty) => Some(v)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
private def exprStrings: Seq[String] = expressions.map(_._1)
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketch
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketches
|
||||
|
||||
import scala.reflect.ClassTag
|
||||
|
|
@ -14,12 +14,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketch
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketches
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, ExprId}
|
||||
import org.apache.spark.sql.types.DataType
|
||||
|
||||
import com.microsoft.hyperspace.index.dataskipping.expressions.ExpressionExtractor
|
||||
|
||||
/**
|
||||
* Represents a sketch specification for data skipping indexes.
|
||||
*
|
||||
|
@ -75,4 +77,42 @@ trait Sketch {
|
|||
* Returns the hash code for this sketch.
|
||||
*/
|
||||
def hashCode: Int
|
||||
|
||||
/**
|
||||
* Converts the given predicate node for source data to an index predicate
|
||||
* that can be used to filter out unnecessary source files when applied to
|
||||
* index data.
|
||||
*
|
||||
* The returned predicate should evaluate to true for an index data row
|
||||
* if the corresponding source data file cannot be excluded. The returned
|
||||
* predicate should be used only to find files that cannot be skipped. In
|
||||
* other words, it must not be negated to find files that can be skipped,
|
||||
* because it can evaluate to null.
|
||||
*
|
||||
* The implementation should consider the given predicate as a single node,
|
||||
* not a tree that must be traversed recursively, because that part is
|
||||
* handled by the framework.
|
||||
*
|
||||
* @param predicate Source predicate node
|
||||
* @param resolvedExprs Sketch expressions that can be used to match
|
||||
* expressions in the source predicate; for example,
|
||||
* MinMaxSketch("A") will be given an expression corresponding to
|
||||
* "A".
|
||||
* @param sketchValues Sketch value references in index data; for example,
|
||||
* MinMaxSketch("A") will be given two expressions corresponding to
|
||||
* Min(A) and Max(A) in the index data. If the predicate is
|
||||
* convertible, the implementation should return a predicate
|
||||
* composed of these sketch values.
|
||||
* @param nameMap Map used to normalize attributes in the source predicate by
|
||||
* looking up the attribute name with ExprId; this is needed because
|
||||
* the attribute name in the predicate may have different cases
|
||||
* (lower/upper cases).
|
||||
* @return Converted predicate for index data
|
||||
*/
|
||||
def convertPredicate(
|
||||
predicate: Expression,
|
||||
resolvedExprs: Seq[Expression],
|
||||
sketchValues: Seq[Expression],
|
||||
nameMap: Map[ExprId, String],
|
||||
valueExtractor: ExpressionExtractor): Option[Expression]
|
||||
}
|
|
@ -1,199 +0,0 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types.{BooleanType, DataType}
|
||||
|
||||
// Value extractors returning Option[Literal] check if the value is not null
|
||||
// because we're only interested in non-null values.
|
||||
//
|
||||
// Also note that we don't go overboard to match every pattern because
|
||||
// we assume that Catalyst optimizer will give us an optimized predicate in NNF.
|
||||
// It means in general we don't have to deal with Not, or worry about
|
||||
// foldable expressions because they will be optimized to literals.
|
||||
//
|
||||
// There are some differences between Spark versions and that's why we include
|
||||
// some patterns that are no longer needed in newer Spark versions.
|
||||
|
||||
/**
|
||||
* Extracts the non-null literal value in the predicate if it's equivalent to
|
||||
* <expr> = <literal>.
|
||||
*
|
||||
* For the purpose of data skipping, we don't extract the value if it's null.
|
||||
* If the literal is not null, then the only way to make the predicate
|
||||
* <expr> = <literal> or <expr> <=> <literal> is when the expression is not null
|
||||
* and its value is equal to the literal value.
|
||||
*/
|
||||
case class EqualToExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Option[Literal] =
|
||||
p match {
|
||||
case EqualTo(e, v: Literal) if v.value != null && matcher(e) => Some(v)
|
||||
case EqualTo(v: Literal, e) if v.value != null && matcher(e) => Some(v)
|
||||
case EqualNullSafe(e, v: Literal) if v.value != null && matcher(e) => Some(v)
|
||||
case EqualNullSafe(v: Literal, e) if v.value != null && matcher(e) => Some(v)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the non-null literal value in the predicate if it's equivalent to
|
||||
* <expr> < <literal>.
|
||||
*/
|
||||
case class LessThanExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Option[Literal] =
|
||||
p match {
|
||||
case LessThan(e, v: Literal) if v.value != null && matcher(e) => Some(v)
|
||||
case GreaterThan(v: Literal, e) if v.value != null && matcher(e) => Some(v)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the non-null literal value in the predicate if it's equivalent to
|
||||
* <expr> <= <literal>.
|
||||
*/
|
||||
case class LessThanOrEqualToExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Option[Literal] =
|
||||
p match {
|
||||
case LessThanOrEqual(e, v: Literal) if v.value != null && matcher(e) => Some(v)
|
||||
case GreaterThanOrEqual(v: Literal, e) if v.value != null && matcher(e) => Some(v)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the non-null literal value in the predicate if it's equivalent to
|
||||
* <expr> > <literal>.
|
||||
*/
|
||||
case class GreaterThanExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Option[Literal] =
|
||||
p match {
|
||||
case GreaterThan(e, v: Literal) if v.value != null && matcher(e) => Some(v)
|
||||
case LessThan(v: Literal, e) if v.value != null && matcher(e) => Some(v)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the non-null literal value in the predicate if it's equivalent to
|
||||
* <expr> >= <literal>.
|
||||
*/
|
||||
case class GreaterThanOrEqualToExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Option[Literal] =
|
||||
p match {
|
||||
case GreaterThanOrEqual(e, v: Literal) if v.value != null && matcher(e) => Some(v)
|
||||
case LessThanOrEqual(v: Literal, e) if v.value != null && matcher(e) => Some(v)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Matches the predicate if it's equivalent to <expr> IS NULL.
|
||||
*/
|
||||
case class IsNullExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Boolean =
|
||||
p match {
|
||||
case IsNull(e) if matcher(e) => true
|
||||
case EqualNullSafe(e, v: Literal) if v.value == null && matcher(e) => true
|
||||
case EqualNullSafe(v: Literal, e) if v.value == null && matcher(e) => true
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Matches the predicate if it's equivalent to <expr> IS NOT NULL.
|
||||
*
|
||||
* Not(IsNull(<expr>)) is also matched because it can be in the predicate in
|
||||
* Spark 2.4. Since Spark 3.0, this is optimized to IsNotNull(<expr>).
|
||||
*/
|
||||
case class IsNotNullExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Boolean =
|
||||
p match {
|
||||
case IsNotNull(e) if matcher(e) => true
|
||||
case Not(IsNull(e)) if matcher(e) => true // for Spark 2.4
|
||||
case Not(EqualNullSafe(e, v: Literal)) if v.value == null && matcher(e) => true
|
||||
case Not(EqualNullSafe(v: Literal, e)) if v.value == null && matcher(e) => true
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Matches the predicate if it's equivalent to <expr> = true.
|
||||
*
|
||||
* Note that boolean expressions can be a predicate on their own, not needing
|
||||
* EqualTo with true. To avoid false matches, we check that the type of the
|
||||
* expression is BooleanType, although it's not strictly necessary because our
|
||||
* predicate conversion does not go down the predicate tree unless it's And/Or
|
||||
* and Spark has already checked the expression is Boolean if it's a direct
|
||||
* child of And/Or.
|
||||
*/
|
||||
case class IsTrueExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Boolean = {
|
||||
val EqualTo = EqualToExtractor(matcher)
|
||||
p.dataType == BooleanType && (p match {
|
||||
case EqualTo(Literal(true, BooleanType)) => true
|
||||
case e if matcher(e) => true
|
||||
case _ => false
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Matches the predicate if it's equivalent to <expr> = false.
|
||||
*/
|
||||
case class IsFalseExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Boolean = {
|
||||
val EqualTo = EqualToExtractor(matcher)
|
||||
p.dataType == BooleanType && (p match {
|
||||
case EqualTo(Literal(false, BooleanType)) => true
|
||||
case Not(e) if matcher(e) => true
|
||||
case _ => false
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts non-null values in the predicate if it's equivalent to
|
||||
* <expr> IN (<lit>*).
|
||||
*
|
||||
* In Spark, In is created if the number of values in the list does not exceed
|
||||
* spark.sql.optimizer.inSetConversionThreshold.
|
||||
*/
|
||||
case class InExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Option[(Seq[Literal])] =
|
||||
p match {
|
||||
case In(e, vs) if vs.nonEmpty && vs.forall(v => v.isInstanceOf[Literal]) && matcher(e) =>
|
||||
Some(vs.map(_.asInstanceOf[Literal]).filter(_.value != null))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts non-null values in the predicate if it's equivalent to
|
||||
* <expr> IN (<lit>*).
|
||||
*
|
||||
* InSet is created instead of In if the list size is larger than
|
||||
* spark.sql.optimizer.inSetConversionThreshold.
|
||||
*/
|
||||
case class InSetExtractor(matcher: ExprMatcher) {
|
||||
def unapply(p: Expression): Option[(Set[Any])] =
|
||||
p match {
|
||||
case InSet(e, vs) if matcher(e) => Some(vs)
|
||||
case _ => None
|
||||
}
|
||||
}
|
|
@ -148,4 +148,11 @@ object FilterReasons {
|
|||
override def verboseStr: String =
|
||||
s"Another candidate index is applied: $appliedIndex"
|
||||
}
|
||||
|
||||
case class IneligibleFilterCondition(condition: String) extends FilterReason {
|
||||
override final def codeStr: String = "INELIGIBLE_FILTER_CONDITION"
|
||||
override val args = Seq("condition" -> condition)
|
||||
override def verboseStr: String =
|
||||
s"Ineligible filter condition: $condition"
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.spark.sql.hyperspace.utils.logicalPlanToDataFrame
|
|||
|
||||
import com.microsoft.hyperspace.{HyperspaceException, Implicits}
|
||||
import com.microsoft.hyperspace.index.IndexConstants
|
||||
import com.microsoft.hyperspace.index.plans.logical.IndexHadoopFsRelation
|
||||
import com.microsoft.hyperspace.shim.ExtractFileSourceScanExecRelation
|
||||
|
||||
/**
|
||||
|
@ -212,7 +213,13 @@ object PlanAnalyzer {
|
|||
plan: SparkPlan,
|
||||
indexes: DataFrame,
|
||||
bufferStream: BufferStream): Unit = {
|
||||
val usedIndexes = indexes.filter(indexes("indexLocation").isin(getPaths(plan): _*))
|
||||
val usedIndexNames = plan.collect {
|
||||
case ExtractFileSourceScanExecRelation(rel: IndexHadoopFsRelation) =>
|
||||
rel.indexName
|
||||
}
|
||||
val usedIndexes = indexes.filter(
|
||||
indexes("indexLocation").isin(getPaths(plan): _*) ||
|
||||
indexes("name").isin(usedIndexNames: _*))
|
||||
usedIndexes.collect().foreach { row =>
|
||||
bufferStream
|
||||
.write(row.getAs("name").toString)
|
||||
|
|
|
@ -45,6 +45,24 @@ class IndexHadoopFsRelation(
|
|||
s"Hyperspace(Type: ${index.derivedDataset.kindAbbr}, " +
|
||||
s"Name: ${index.name}, LogVersion: ${index.id})"
|
||||
}
|
||||
|
||||
def indexName: String = index.name
|
||||
|
||||
override def toString(): String = indexPlanStr
|
||||
|
||||
}
|
||||
|
||||
object IndexHadoopFsRelation {
|
||||
def apply(
|
||||
rel: HadoopFsRelation,
|
||||
spark: SparkSession,
|
||||
index: IndexLogEntry): IndexHadoopFsRelation = {
|
||||
new IndexHadoopFsRelation(
|
||||
rel.location,
|
||||
rel.partitionSchema,
|
||||
rel.dataSchema,
|
||||
rel.bucketSpec,
|
||||
rel.fileFormat,
|
||||
rel.options)(spark, index)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,4 +63,13 @@ object ApplyHyperspace
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
def withHyperspaceRuleDisabled[T](f: => T): T = {
|
||||
try {
|
||||
disableForIndexMaintenance.set(true)
|
||||
f
|
||||
} finally {
|
||||
disableForIndexMaintenance.set(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,13 +21,15 @@ import scala.collection.mutable
|
|||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
|
||||
import com.microsoft.hyperspace.index.covering.{FilterIndexRule, JoinIndexRule}
|
||||
import com.microsoft.hyperspace.index.dataskipping.rules.ApplyDataSkippingIndex
|
||||
import com.microsoft.hyperspace.index.rules.ApplyHyperspace.PlanToIndexesMap
|
||||
|
||||
/**
|
||||
* Apply Hyperspace indexes based on the score of each index application.
|
||||
*/
|
||||
class ScoreBasedIndexPlanOptimizer {
|
||||
private val rules: Seq[HyperspaceRule] = FilterIndexRule :: JoinIndexRule :: NoOpRule :: Nil
|
||||
private val rules: Seq[HyperspaceRule] =
|
||||
Seq(FilterIndexRule, JoinIndexRule, ApplyDataSkippingIndex, NoOpRule)
|
||||
|
||||
// Map for memoization. The key is the logical plan before applying [[HyperspaceRule]]s
|
||||
// and its value is a pair of best transformed plan and its score.
|
||||
|
|
|
@ -88,6 +88,13 @@ class DefaultFileBasedRelation(spark: SparkSession, override val plan: LogicalRe
|
|||
case _ => None
|
||||
}
|
||||
|
||||
override def getOrCreateFileIndex(spark: SparkSession): FileIndex = {
|
||||
plan.relation match {
|
||||
case HadoopFsRelation(location: FileIndex, _, _, _, _, _) =>
|
||||
location
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates [[HadoopFsRelation]] based on the current relation.
|
||||
*
|
||||
|
|
|
@ -18,12 +18,12 @@ package com.microsoft.hyperspace.index.sources.iceberg
|
|||
|
||||
import collection.JavaConverters._
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.iceberg.{FileScanTask, Schema, Table}
|
||||
import org.apache.iceberg.{FileScanTask, Schema, Table, TableScan}
|
||||
import org.apache.iceberg.spark.SparkSchemaUtil
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation}
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
|
@ -70,7 +70,7 @@ class IcebergRelation(
|
|||
* All the files that the current Iceberg table uses for read.
|
||||
*/
|
||||
override lazy val allFiles: Seq[FileStatus] = {
|
||||
table.newScan().planFiles().iterator().asScala.toSeq.map(toFileStatus)
|
||||
fileScanTasks.map(toFileStatus)
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -160,6 +160,33 @@ class IcebergRelation(
|
|||
SparkSchemaUtil.convert(schema)
|
||||
}
|
||||
|
||||
override def getOrCreateFileIndex(spark: SparkSession): InMemoryFileIndex = {
|
||||
val rootPaths = if (snapshotId.isEmpty) {
|
||||
if (table.spec().isUnpartitioned) {
|
||||
Seq(PathUtils.makeAbsolute(table.location(), spark.sessionState.newHadoopConf()))
|
||||
} else {
|
||||
// We'd like to use a single root path here, but
|
||||
// then the partition discovery fails
|
||||
// because there is a directory called "metadata"
|
||||
// under the root path.
|
||||
fileScanTasks.map(t => toPath(t).getParent).distinct
|
||||
}
|
||||
} else {
|
||||
// Listing all files with InMemoryFileIndex should be avoided,
|
||||
// but there is no other way as there is no FileIndex implementation
|
||||
// for Iceberg that supports snapshots.
|
||||
fileScanTasks.map(toPath)
|
||||
}
|
||||
new InMemoryFileIndex(
|
||||
spark,
|
||||
rootPaths,
|
||||
partitionBasePath
|
||||
.map(PartitioningAwareFileIndex.BASE_PATH_PARAM -> _)
|
||||
.toMap,
|
||||
Some(schema),
|
||||
FileStatusCache.getOrCreate(spark))
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates [[HadoopFsRelation]] based on the current relation.
|
||||
*
|
||||
|
@ -191,10 +218,26 @@ class IcebergRelation(
|
|||
new LogicalRelation(hadoopFsRelation, updatedOutput, None, false)
|
||||
}
|
||||
|
||||
private def toFileStatus(fileScanTask: FileScanTask): FileStatus = {
|
||||
val path = PathUtils.makeAbsolute(
|
||||
private def newScan(): TableScan = {
|
||||
if (snapshotId.isDefined) {
|
||||
table.newScan().useSnapshot(snapshotId.get)
|
||||
} else {
|
||||
table.newScan()
|
||||
}
|
||||
}
|
||||
|
||||
private def fileScanTasks: Seq[FileScanTask] = {
|
||||
newScan().planFiles().iterator().asScala.toSeq
|
||||
}
|
||||
|
||||
private def toPath(fileScanTask: FileScanTask): Path = {
|
||||
PathUtils.makeAbsolute(
|
||||
new Path(fileScanTask.file().path().toString),
|
||||
spark.sessionState.newHadoopConf())
|
||||
}
|
||||
|
||||
private def toFileStatus(fileScanTask: FileScanTask): FileStatus = {
|
||||
val path = toPath(fileScanTask)
|
||||
val fs = path.getFileSystem(spark.sessionState.newHadoopConf())
|
||||
val fullPath = if (!path.isAbsolute) {
|
||||
new Path(s"${fs.getWorkingDirectory.toString}/${path.toString}")
|
||||
|
|
|
@ -105,6 +105,11 @@ trait FileBasedRelation extends SourceRelation {
|
|||
*/
|
||||
def partitionBasePath: Option[String]
|
||||
|
||||
/**
|
||||
* Returns [[FileIndex]] for the current relation.
|
||||
*/
|
||||
def getOrCreateFileIndex(spark: SparkSession): FileIndex
|
||||
|
||||
/**
|
||||
* Creates [[HadoopFsRelation]] based on the current relation.
|
||||
*
|
||||
|
|
|
@ -130,6 +130,49 @@ object HyperspaceConf {
|
|||
}
|
||||
longValue
|
||||
}
|
||||
|
||||
def maxIndexDataFileCount(spark: SparkSession): Int = {
|
||||
// TODO: Consider using a systematic way to validate the config value
|
||||
// like Spark's ConfigBuilder
|
||||
val value = spark.conf
|
||||
.get(
|
||||
IndexConstants.DATASKIPPING_MAX_INDEX_DATA_FILE_COUNT,
|
||||
IndexConstants.DATASKIPPING_MAX_INDEX_DATA_FILE_COUNT_DEFAULT)
|
||||
val intValue =
|
||||
try {
|
||||
value.toInt
|
||||
} catch {
|
||||
case e: NumberFormatException =>
|
||||
throw HyperspaceException(
|
||||
s"${IndexConstants.DATASKIPPING_MAX_INDEX_DATA_FILE_COUNT} " +
|
||||
s"should be int, but was $value")
|
||||
}
|
||||
if (intValue <= 0) {
|
||||
throw HyperspaceException(
|
||||
s"${IndexConstants.DATASKIPPING_MAX_INDEX_DATA_FILE_COUNT} " +
|
||||
s"should be a positive number.")
|
||||
}
|
||||
intValue
|
||||
}
|
||||
|
||||
def autoPartitionSketch(spark: SparkSession): Boolean = {
|
||||
// TODO: Consider using a systematic way to validate the config value
|
||||
// like Spark's ConfigBuilder
|
||||
val value = spark.conf
|
||||
.get(
|
||||
IndexConstants.DATASKIPPING_AUTO_PARTITION_SKETCH,
|
||||
IndexConstants.DATASKIPPING_AUTO_PARTITION_SKETCH_DEFAULT)
|
||||
val booleanValue =
|
||||
try {
|
||||
value.toBoolean
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
throw HyperspaceException(
|
||||
s"${IndexConstants.DATASKIPPING_AUTO_PARTITION_SKETCH} " +
|
||||
s"should be boolean, but was $value")
|
||||
}
|
||||
booleanValue
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -34,5 +34,6 @@ package object utils {
|
|||
implicit class StructTypeUtils(st: StructType) {
|
||||
// Expose package-private method
|
||||
def merge(that: StructType): StructType = st.merge(that)
|
||||
def sameType(that: StructType): Boolean = st.sameType(that)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
import org.apache.iceberg.{PartitionSpec => IcebergPartitionSpec, Table, TableProperties}
|
||||
import org.apache.iceberg.hadoop.HadoopTables
|
||||
import org.apache.iceberg.spark.SparkSchemaUtil
|
||||
import org.apache.spark.sql.DataFrame
|
||||
|
||||
trait IcebergTestUtils {
|
||||
def createIcebergTable(dataPath: String, sourceDf: DataFrame): Table = {
|
||||
val props = Map(TableProperties.WRITE_NEW_DATA_LOCATION -> dataPath).asJava
|
||||
val schema = SparkSchemaUtil.convert(sourceDf.schema)
|
||||
val part = IcebergPartitionSpec.builderFor(schema).build()
|
||||
new HadoopTables().create(schema, part, props, dataPath)
|
||||
}
|
||||
|
||||
def createIcebergTableWithPartitions(
|
||||
dataPath: String,
|
||||
sourceDf: DataFrame,
|
||||
partCol: String): Table = {
|
||||
val props = Map(TableProperties.WRITE_NEW_DATA_LOCATION -> dataPath).asJava
|
||||
val schema = SparkSchemaUtil.convert(sourceDf.schema)
|
||||
val part = IcebergPartitionSpec.builderFor(schema).identity(partCol).build()
|
||||
new HadoopTables().create(schema, part, props, dataPath)
|
||||
}
|
||||
}
|
|
@ -30,7 +30,7 @@ import com.microsoft.hyperspace.index.IndexConstants.{GLOBBING_PATTERN_KEY, REFR
|
|||
import com.microsoft.hyperspace.index.IndexLogEntryTags._
|
||||
import com.microsoft.hyperspace.index.covering.JoinIndexRule
|
||||
import com.microsoft.hyperspace.index.dataskipping.DataSkippingIndexConfig
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketch.MinMaxSketch
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches.MinMaxSketch
|
||||
import com.microsoft.hyperspace.index.execution.BucketUnionStrategy
|
||||
import com.microsoft.hyperspace.index.rules.{ApplyHyperspace, CandidateIndexCollector}
|
||||
import com.microsoft.hyperspace.util.PathUtils
|
||||
|
|
|
@ -19,22 +19,19 @@ package com.microsoft.hyperspace.index
|
|||
import org.apache.commons.lang.StringUtils
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.iceberg.{PartitionSpec => IcebergPartitionSpec, Table, TableProperties}
|
||||
import org.apache.iceberg.hadoop.HadoopTables
|
||||
import org.apache.iceberg.spark.SparkSchemaUtil
|
||||
import org.apache.spark.sql.{DataFrame, QueryTest}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
import com.microsoft.hyperspace.{Hyperspace, Implicits, SampleData, TestConfig}
|
||||
import com.microsoft.hyperspace.{Hyperspace, IcebergTestUtils, Implicits, SampleData, TestConfig}
|
||||
import com.microsoft.hyperspace.TestUtils.latestIndexLogEntry
|
||||
import com.microsoft.hyperspace.index.IndexConstants.REFRESH_MODE_QUICK
|
||||
import com.microsoft.hyperspace.index.plananalysis.{PlainTextMode, PlanAnalyzer}
|
||||
import com.microsoft.hyperspace.util.PathUtils
|
||||
import com.microsoft.hyperspace.util.PathUtils.DataPathFilter
|
||||
|
||||
class IcebergIntegrationTest extends QueryTest with HyperspaceSuite {
|
||||
class IcebergIntegrationTest extends QueryTest with HyperspaceSuite with IcebergTestUtils {
|
||||
override val indexLocationDirName = "icebergIntegrationTest"
|
||||
|
||||
private val sampleData = SampleData.testData
|
||||
|
@ -131,7 +128,7 @@ class IcebergIntegrationTest extends QueryTest with HyperspaceSuite {
|
|||
val dfFromSample = testData
|
||||
.toDF("Date", "RGUID", "Query", "imprs", "clicks")
|
||||
|
||||
createIcebergTableWithDayPartition(testPath, dfFromSample)
|
||||
createIcebergTableWithPartitions(testPath, dfFromSample, "Date")
|
||||
|
||||
dfFromSample.write
|
||||
.format("iceberg")
|
||||
|
@ -387,20 +384,6 @@ class IcebergIntegrationTest extends QueryTest with HyperspaceSuite {
|
|||
rootPaths.exists(_.toString.contains(p)))
|
||||
}
|
||||
|
||||
def createIcebergTable(dataPath: String, sourceDf: DataFrame): Table = {
|
||||
val props = Map(TableProperties.WRITE_NEW_DATA_LOCATION -> dataPath).asJava
|
||||
val schema = SparkSchemaUtil.convert(sourceDf.schema)
|
||||
val part = IcebergPartitionSpec.builderFor(schema).build()
|
||||
new HadoopTables().create(schema, part, props, dataPath)
|
||||
}
|
||||
|
||||
def createIcebergTableWithDayPartition(dataPath: String, sourceDf: DataFrame): Table = {
|
||||
val props = Map(TableProperties.WRITE_NEW_DATA_LOCATION -> dataPath).asJava
|
||||
val schema = SparkSchemaUtil.convert(sourceDf.schema)
|
||||
val part = IcebergPartitionSpec.builderFor(schema).identity("Date").build()
|
||||
new HadoopTables().create(schema, part, props, dataPath)
|
||||
}
|
||||
|
||||
private def truncate(s: String): String = {
|
||||
StringUtils.abbreviate(s, 100)
|
||||
}
|
||||
|
|
|
@ -18,11 +18,11 @@ package com.microsoft.hyperspace.index.dataskipping
|
|||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.functions.{input_file_name, max, min}
|
||||
import org.apache.spark.sql.types.{LongType, StringType}
|
||||
import org.apache.spark.sql.types.{IntegerType, LongType, StringType}
|
||||
|
||||
import com.microsoft.hyperspace.HyperspaceException
|
||||
import com.microsoft.hyperspace.index.IndexConstants
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketch._
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches._
|
||||
|
||||
class DataSkippingIndexConfigTest extends DataSkippingSuite {
|
||||
test("indexName returns the index name.") {
|
||||
|
@ -92,6 +92,46 @@ class DataSkippingIndexConfigTest extends DataSkippingSuite {
|
|||
assert(index.sketches === Seq(MinMaxSketch("Foo", Some(LongType))))
|
||||
}
|
||||
|
||||
test("createIndex creates partition sketches for partitioned source data.") {
|
||||
val sourceData =
|
||||
createPartitionedSourceData(spark.range(10).selectExpr("id as A", "id * 2 as B"), Seq("A"))
|
||||
val indexConfig = DataSkippingIndexConfig("MyIndex", MinMaxSketch("B"))
|
||||
val (index, indexData) = indexConfig.createIndex(ctx, sourceData, Map())
|
||||
assert(
|
||||
index.sketches === Seq(
|
||||
PartitionSketch(Seq(("A", Some(IntegerType)))),
|
||||
MinMaxSketch("B", Some(LongType))))
|
||||
}
|
||||
|
||||
test(
|
||||
"createIndex creates partition sketches for partitioned source data " +
|
||||
"with multiple partition columns.") {
|
||||
val sourceData =
|
||||
createPartitionedSourceData(
|
||||
spark.range(10).selectExpr("id as A", "id as B", "id * 2 as C"),
|
||||
Seq("A", "B"))
|
||||
val indexConfig = DataSkippingIndexConfig("MyIndex", MinMaxSketch("C"))
|
||||
val (index, indexData) = indexConfig.createIndex(ctx, sourceData, Map())
|
||||
assert(
|
||||
index.sketches === Seq(
|
||||
PartitionSketch(Seq(("A", Some(IntegerType)), ("B", Some(IntegerType)))),
|
||||
MinMaxSketch("C", Some(LongType))))
|
||||
}
|
||||
|
||||
test(
|
||||
"createIndex does not create partition sketches for partitioned source data " +
|
||||
"if the config is turned off.") {
|
||||
withSQLConf(IndexConstants.DATASKIPPING_AUTO_PARTITION_SKETCH -> "false") {
|
||||
val sourceData =
|
||||
createPartitionedSourceData(
|
||||
spark.range(10).selectExpr("id as A", "id * 2 as B"),
|
||||
Seq("A"))
|
||||
val indexConfig = DataSkippingIndexConfig("MyIndex", MinMaxSketch("B"))
|
||||
val (index, indexData) = indexConfig.createIndex(ctx, sourceData, Map())
|
||||
assert(index.sketches === Seq(MinMaxSketch("B", Some(LongType))))
|
||||
}
|
||||
}
|
||||
|
||||
test("createIndex throws an error if the data type is wrong.") {
|
||||
val sourceData = createSourceData(spark.range(10).toDF("Foo"))
|
||||
val indexConfig = DataSkippingIndexConfig("MyIndex", MinMaxSketch("foO", Some(StringType)))
|
||||
|
|
|
@ -16,14 +16,45 @@
|
|||
|
||||
package com.microsoft.hyperspace.index.dataskipping
|
||||
|
||||
import com.microsoft.hyperspace.HyperspaceException
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketch.MinMaxSketch
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.SparkException
|
||||
import org.apache.spark.sql.{DataFrame, SaveMode}
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class DataSkippingIndexIntegrationTest extends DataSkippingSuite {
|
||||
import com.microsoft.hyperspace._
|
||||
import com.microsoft.hyperspace.index.IndexConstants
|
||||
import com.microsoft.hyperspace.index.covering.CoveringIndexConfig
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches._
|
||||
import com.microsoft.hyperspace.index.plans.logical.IndexHadoopFsRelation
|
||||
import com.microsoft.hyperspace.shim.ExtractFileSourceScanExecRelation
|
||||
|
||||
class DataSkippingIndexIntegrationTest extends DataSkippingSuite with IcebergTestUtils {
|
||||
import spark.implicits._
|
||||
|
||||
override val numParallelism: Int = 10
|
||||
|
||||
test("MinMax index is applied for a filter query (EqualTo).") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
|
||||
test("Empty relation is returned if no files match the index predicate.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = -1")
|
||||
checkIndexApplied(query, 0)
|
||||
}
|
||||
|
||||
test("MinMax index is applied for a filter query (EqualTo) with expression.") {
|
||||
val df = createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A + B")))
|
||||
def query: DataFrame = df.filter("A+B < 40")
|
||||
checkIndexApplied(query, 2)
|
||||
}
|
||||
|
||||
test("Non-deterministic expression is blocked.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
val ex = intercept[HyperspaceException](
|
||||
|
@ -88,4 +119,554 @@ class DataSkippingIndexIntegrationTest extends DataSkippingSuite {
|
|||
"DataSkippingIndex does not support indexing an expression which does not " +
|
||||
"reference source columns: myfunc()"))
|
||||
}
|
||||
|
||||
test("MinMax index is applied for a filter query (EqualTo) with UDF.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
spark.udf.register("myfunc", (a: Int) => a * 2)
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("myfunc(A)")))
|
||||
def query: DataFrame = df.filter("myfunc(A) = 10")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
|
||||
test("UDF matching is based on the name, not the actual lambda object.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
spark.udf.register("myfunc", (a: Int) => a * 2)
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("myfunc(A)")))
|
||||
// Register a new function with the same semantics.
|
||||
spark.udf.register("myfunc", (a: Int) => 2 * a)
|
||||
def query: DataFrame = df.filter("myfunc(A) = 10")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
|
||||
test("MinMax index is not applied for a filter query if it is not applicable.") {
|
||||
val df = createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("B")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, numParallelism)
|
||||
}
|
||||
|
||||
test("MinMax index is not applied for a filter query if the filter condition is unsuitable.") {
|
||||
val df = createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1 or B = 2")
|
||||
checkIndexApplied(query, numParallelism)
|
||||
}
|
||||
|
||||
test("MinMax index is not applied for a filter query if the filter condition is IsNull.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A is null")
|
||||
checkIndexApplied(query, numParallelism)
|
||||
}
|
||||
|
||||
test("Multiple indexes are applied to multiple filters.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"), path = "TA")
|
||||
val df2 = createSourceData(spark.range(100, 200).toDF("B"), path = "TB")
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
hs.createIndex(df2, DataSkippingIndexConfig("myind2", MinMaxSketch("B")))
|
||||
def query: DataFrame = df.filter("A = 10").union(df2.filter("B = 110"))
|
||||
checkIndexApplied(query, 2)
|
||||
}
|
||||
|
||||
test("Single index is applied to multiple filters.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 10").union(df.filter("A = 20"))
|
||||
checkIndexApplied(query, 2)
|
||||
}
|
||||
|
||||
test("Single index is applied to a single filter.") {
|
||||
val df = createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 10").union(df.filter("B = 120"))
|
||||
checkIndexApplied(query, numParallelism + 1)
|
||||
}
|
||||
|
||||
test(
|
||||
"DataSkippingIndex works correctly for CSV where the same source data files can be " +
|
||||
"interpreted differently.") {
|
||||
// String order: 1 < 10 < 2
|
||||
// Int order: 1 < 2 < 10
|
||||
createFile(dataPath("1.csv"), Seq("a", "1", "2", "10").mkString("\n").getBytes())
|
||||
createFile(dataPath("2.csv"), Seq("a", "3", "4", "5").mkString("\n").getBytes())
|
||||
val paths = Seq(dataPath("1.csv").toString, dataPath("2.csv").toString)
|
||||
val dfString = spark.read.option("header", "true").csv(paths: _*)
|
||||
assert(dfString.schema.head.dataType === StringType)
|
||||
val dfInt = spark.read.option("header", "true").option("inferSchema", "true").csv(paths: _*)
|
||||
assert(dfInt.schema.head.dataType === IntegerType)
|
||||
|
||||
withIndex("myind") {
|
||||
hs.createIndex(dfString, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
checkIndexApplied(dfString.filter("A = 3"), 2)
|
||||
checkIndexApplied(dfString.filter("A = 10"), 2)
|
||||
checkIndexApplied(dfString.filter("A = '3'"), 1)
|
||||
checkIndexApplied(dfString.filter("A = '10'"), 1)
|
||||
checkIndexApplied(dfInt.filter("A = 3"), 2)
|
||||
checkIndexApplied(dfInt.filter("A = 10"), 2)
|
||||
checkIndexApplied(dfInt.filter("A = '3'"), 2)
|
||||
checkIndexApplied(dfInt.filter("A = '10'"), 2)
|
||||
}
|
||||
withIndex("myind") {
|
||||
hs.createIndex(dfInt, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
checkIndexApplied(dfString.filter("A = 3"), 2)
|
||||
checkIndexApplied(dfString.filter("A = 10"), 2)
|
||||
checkIndexApplied(dfString.filter("A = '3'"), 2)
|
||||
checkIndexApplied(dfString.filter("A = '10'"), 2)
|
||||
checkIndexApplied(dfInt.filter("A = 3"), 2)
|
||||
checkIndexApplied(dfInt.filter("A = 10"), 1)
|
||||
checkIndexApplied(dfInt.filter("A = '3'"), 2)
|
||||
checkIndexApplied(dfInt.filter("A = '10'"), 1)
|
||||
}
|
||||
}
|
||||
|
||||
test("MinMax index is applied for a filter query (EqualTo) with selection.") {
|
||||
val df = createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1").select("B")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
|
||||
test("MinMax index can be refreshed (mode = incremental).") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
createSourceData(spark.range(100, 200).toDF("A"), saveMode = SaveMode.Append)
|
||||
hs.refreshIndex("myind", "incremental")
|
||||
def query: DataFrame = spark.read.parquet(dataPath().toString).filter("A = 1 OR A = 123")
|
||||
checkIndexApplied(query, 2)
|
||||
assert(numIndexDataFiles("myind") === 2)
|
||||
}
|
||||
|
||||
test("MinMax index can be refreshed (mode = full).") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
createSourceData(spark.range(100, 200).toDF("A"), saveMode = SaveMode.Append)
|
||||
hs.refreshIndex("myind", "full")
|
||||
def query: DataFrame = spark.read.parquet(dataPath().toString).filter("A = 1 OR A = 123")
|
||||
checkIndexApplied(query, 2)
|
||||
assert(numIndexDataFiles("myind") === 1)
|
||||
}
|
||||
|
||||
test("MinMax index can be refreshed (mode = full) for partitioned data.") {
|
||||
val df = createPartitionedSourceData(
|
||||
spark.range(100).selectExpr("id as A", "cast(id / 10 as int) as B"),
|
||||
Seq("B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
createPartitionedSourceData(
|
||||
spark.range(100, 200).selectExpr("id as A", "cast(id / 15 as int) as B"),
|
||||
Seq("B"),
|
||||
saveMode = SaveMode.Append)
|
||||
hs.refreshIndex("myind", "full")
|
||||
def query: DataFrame = spark.read.parquet(dataPath().toString).filter("A = 1 OR A = 123")
|
||||
checkIndexApplied(query, 2)
|
||||
assert(numIndexDataFiles("myind") === 1)
|
||||
}
|
||||
|
||||
test(
|
||||
"MinMax index can be applied without refresh when source files are added " +
|
||||
"if hybrid scan is enabled.") {
|
||||
withSQLConf(
|
||||
IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "1") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
createSourceData(spark.range(100, 200).toDF("A"), saveMode = SaveMode.Append)
|
||||
def query: DataFrame = spark.read.parquet(dataPath().toString).filter("A = 1 OR A = 123")
|
||||
checkIndexApplied(query, 11)
|
||||
}
|
||||
}
|
||||
|
||||
test("Empty source data does not cause an error.") {
|
||||
val df = createSourceData(spark.range(0).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
|
||||
test("Empty source data followed by refresh incremental works as expected.") {
|
||||
val df = createSourceData(spark.range(0).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
createSourceData(spark.range(100).toDF("A"), saveMode = SaveMode.Append)
|
||||
hs.refreshIndex("myind", "incremental")
|
||||
def query: DataFrame = spark.read.parquet(dataPath().toString).filter("A = 1")
|
||||
checkIndexApplied(query, 2)
|
||||
}
|
||||
|
||||
test("MinMax index can be optimized.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
createSourceData(spark.range(100, 200).toDF("A"), saveMode = SaveMode.Append)
|
||||
hs.refreshIndex("myind", "incremental")
|
||||
assert(numIndexDataFiles("myind") === 2)
|
||||
hs.optimizeIndex("myind")
|
||||
assert(numIndexDataFiles("myind") === 1)
|
||||
def query: DataFrame = spark.read.parquet(dataPath().toString).filter("A = 1 OR A = 123")
|
||||
checkIndexApplied(query, 2)
|
||||
}
|
||||
|
||||
test("CoveringIndex is applied if both CoveringIndex and DataSkippingIndex are applicable.") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("ds", MinMaxSketch("A")))
|
||||
hs.createIndex(df, CoveringIndexConfig("ci", Seq("A"), Nil))
|
||||
spark.enableHyperspace
|
||||
def query: DataFrame = df.filter("A = 1 or A = 50")
|
||||
val rel = query.queryExecution.optimizedPlan.collect {
|
||||
case LogicalRelation(rel: IndexHadoopFsRelation, _, _, _) => rel
|
||||
}
|
||||
assert(rel.map(_.indexName) === Seq("ci"))
|
||||
checkAnswer(query, Seq(1, 50).toDF("A"))
|
||||
}
|
||||
|
||||
test("DataSkippingIndex is applied if CoveringIndex is not applicable.") {
|
||||
val df = createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("ds", MinMaxSketch("A")))
|
||||
hs.createIndex(df, CoveringIndexConfig("ci", Seq("A"), Nil))
|
||||
spark.enableHyperspace
|
||||
def query: DataFrame = df.filter("A = 1 or A = 50")
|
||||
val rel = query.queryExecution.optimizedPlan.collect {
|
||||
case LogicalRelation(rel: IndexHadoopFsRelation, _, _, _) => rel
|
||||
}
|
||||
assert(rel.map(_.indexName) === Seq("ds"))
|
||||
checkAnswer(query, Seq((1, 2), (50, 100)).toDF("A", "B"))
|
||||
}
|
||||
|
||||
test("Both CoveringIndex and DataSkippnigIndex can be applied.") {
|
||||
val df = createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B"))
|
||||
hs.createIndex(df, CoveringIndexConfig("ci", Seq("A"), Nil))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("ds", MinMaxSketch("B")))
|
||||
spark.enableHyperspace
|
||||
def query: DataFrame = df.filter("A = 1").select("A").union(df.filter("B = 100").select("A"))
|
||||
val rel = query.queryExecution.optimizedPlan.collect {
|
||||
case LogicalRelation(rel: IndexHadoopFsRelation, _, _, _) => rel
|
||||
}
|
||||
assert(rel.map(_.indexName).sorted === Seq("ci", "ds"))
|
||||
checkAnswer(query, Seq(1, 50).toDF("A"))
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with files having special characters in their name.") {
|
||||
assume(!Path.WINDOWS)
|
||||
val df = createSourceData(spark.range(100).toDF("A"), "table ,.;'`~!@#$%^&()_+|\"<>")
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with catalog tables") {
|
||||
withTable("T") {
|
||||
spark.range(100).toDF("A").write.saveAsTable("T")
|
||||
val df = spark.read.table("T")
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with partitioned data.") {
|
||||
val df = createPartitionedSourceData(
|
||||
spark.range(1000).selectExpr("cast(id/10 as int) as A", "id as B"),
|
||||
Seq("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("B")))
|
||||
def query: DataFrame = df.filter("A = 1 or B = 100")
|
||||
checkIndexApplied(query, 2)
|
||||
}
|
||||
|
||||
test(
|
||||
"DataSkippingIndex works correctly with partitioned data " +
|
||||
"with multiple partition columns.") {
|
||||
val df = createPartitionedSourceData(
|
||||
spark
|
||||
.range(1000)
|
||||
.selectExpr("cast(id/100 as int) as A", "cast(id/10 as int) as B", "id as C"),
|
||||
Seq("A", "B"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("C")))
|
||||
def query: DataFrame = df.filter("A = 1 or B = 1 or C = 1")
|
||||
checkIndexApplied(query, 12)
|
||||
}
|
||||
|
||||
test(
|
||||
"DataSkippingIndex works correctly with partitioned data " +
|
||||
"with a different filter condition.") {
|
||||
val df = createPartitionedSourceData(
|
||||
spark.range(1000).selectExpr("cast(id/200 as int)*200 as A", "id as B"),
|
||||
Seq("A"))
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("B")))
|
||||
def query: DataFrame = df.filter("A = B")
|
||||
checkIndexApplied(query, 5)
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with Delta Lake tables.") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.delta.DeltaLakeFileBasedSourceBuilder") {
|
||||
val df = createSourceData(spark.range(100).toDF("A"), format = "delta")
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with partitioned Delta Lake tables.") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.delta.DeltaLakeFileBasedSourceBuilder") {
|
||||
val df = createPartitionedSourceData(
|
||||
spark.range(100).selectExpr("id as A", "cast(id / 10 as int) as B"),
|
||||
Seq("B"),
|
||||
format = "delta")
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1 or B = 5")
|
||||
checkIndexApplied(query, 2)
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with Delta time travel.") {
|
||||
withTable("T") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.delta.DeltaLakeFileBasedSourceBuilder",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "10",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_DELETED_RATIO_THRESHOLD -> "10") {
|
||||
|
||||
// version 0
|
||||
spark.range(100).toDF("A").write.format("delta").save(dataPath("T").toString)
|
||||
|
||||
// version 1
|
||||
spark
|
||||
.range(100, 200)
|
||||
.toDF("A")
|
||||
.write
|
||||
.format("delta")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
|
||||
// version 2
|
||||
spark
|
||||
.range(200, 300)
|
||||
.toDF("A")
|
||||
.write
|
||||
.format("delta")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
|
||||
val df = (v: Int) =>
|
||||
spark.read.format("delta").option("versionAsOf", v).load(dataPath("T").toString)
|
||||
|
||||
// Create an index with version 1 data
|
||||
hs.createIndex(df(1), DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
|
||||
def query0: DataFrame = df(0).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query0, 1)
|
||||
|
||||
def query1: DataFrame = df(1).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query1, 2)
|
||||
|
||||
def query2: DataFrame = df(2).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query2, 12)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with Delta time travel with partitions.") {
|
||||
withTable("T") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.delta.DeltaLakeFileBasedSourceBuilder",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "10",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_DELETED_RATIO_THRESHOLD -> "10") {
|
||||
|
||||
// version 0
|
||||
spark
|
||||
.range(100)
|
||||
.selectExpr("id as A", "cast(id / 10 as int) as B")
|
||||
.write
|
||||
.format("delta")
|
||||
.save(dataPath("T").toString)
|
||||
|
||||
// version 1
|
||||
spark
|
||||
.range(100, 200)
|
||||
.selectExpr("id as A", "cast(id / 15 as int) as B")
|
||||
.write
|
||||
.format("delta")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
|
||||
// version 2
|
||||
spark
|
||||
.range(200, 300)
|
||||
.selectExpr("id as A", "cast(id / 20 as int) as B")
|
||||
.write
|
||||
.format("delta")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
|
||||
val df = (v: Int) =>
|
||||
spark.read.format("delta").option("versionAsOf", v).load(dataPath("T").toString)
|
||||
|
||||
// Create an index with version 1 data
|
||||
hs.createIndex(df(1), DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
|
||||
def query0: DataFrame = df(0).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query0, 1)
|
||||
|
||||
def query1: DataFrame = df(1).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query1, 2)
|
||||
|
||||
def query2: DataFrame = df(2).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query2, 12)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with Iceberg tables.") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.iceberg.IcebergFileBasedSourceBuilder") {
|
||||
val data = spark.range(100).toDF("A")
|
||||
createIcebergTable(dataPath("T").toString, data)
|
||||
val df = createSourceData(data, format = "iceberg")
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with partitioned Iceberg tables.") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.iceberg.IcebergFileBasedSourceBuilder") {
|
||||
val data = spark.range(100).selectExpr("id as A", "cast(id / 10 as int) as B")
|
||||
createIcebergTableWithPartitions(dataPath("T").toString, data, "B")
|
||||
val df = createPartitionedSourceData(data, Seq("B"), format = "iceberg")
|
||||
hs.createIndex(df, DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
def query: DataFrame = df.filter("A = 1")
|
||||
checkIndexApplied(query, 1)
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with Iceberg time travel.") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.iceberg.IcebergFileBasedSourceBuilder",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "10",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_DELETED_RATIO_THRESHOLD -> "10") {
|
||||
|
||||
// version 0
|
||||
val data = spark.range(100).toDF("A")
|
||||
val table = createIcebergTable(dataPath("T").toString, data)
|
||||
data.write.format("iceberg").mode("overwrite").save(dataPath("T").toString)
|
||||
val v0 = table.currentSnapshot.snapshotId
|
||||
|
||||
// version 1
|
||||
spark
|
||||
.range(100, 200)
|
||||
.toDF("A")
|
||||
.write
|
||||
.format("iceberg")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
table.newTransaction().commitTransaction()
|
||||
val v1 = table.currentSnapshot.snapshotId
|
||||
|
||||
// version 2
|
||||
spark
|
||||
.range(200, 300)
|
||||
.toDF("A")
|
||||
.write
|
||||
.format("iceberg")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
table.newTransaction().commitTransaction()
|
||||
val v2 = table.currentSnapshot.snapshotId
|
||||
|
||||
val df = (v: Long) =>
|
||||
spark.read.format("iceberg").option("snapshot-id", v).load(dataPath("T").toString)
|
||||
|
||||
// Create an index with version 1 data
|
||||
hs.createIndex(df(v1), DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
|
||||
def query0: DataFrame = df(v0).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query0, 1)
|
||||
|
||||
def query1: DataFrame = df(v1).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query1, 2)
|
||||
|
||||
def query2: DataFrame = df(v2).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query2, 12)
|
||||
}
|
||||
}
|
||||
|
||||
test("DataSkippingIndex works correctly with Iceberg time travel with partitions.") {
|
||||
withSQLConf(
|
||||
"spark.hyperspace.index.sources.fileBasedBuilders" ->
|
||||
"com.microsoft.hyperspace.index.sources.iceberg.IcebergFileBasedSourceBuilder",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_ENABLED -> "true",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_APPENDED_RATIO_THRESHOLD -> "10",
|
||||
IndexConstants.INDEX_HYBRID_SCAN_DELETED_RATIO_THRESHOLD -> "10") {
|
||||
|
||||
// version 0
|
||||
val data = spark.range(100).selectExpr("id as A", "cast(id / 10 as int) as B")
|
||||
val table = createIcebergTable(dataPath("T").toString, data)
|
||||
data.write.format("iceberg").mode("overwrite").save(dataPath("T").toString)
|
||||
val v0 = table.currentSnapshot.snapshotId
|
||||
|
||||
// version 1
|
||||
spark
|
||||
.range(100, 200)
|
||||
.selectExpr("id as A", "cast(id / 15 as int) as B")
|
||||
.write
|
||||
.format("iceberg")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
table.newTransaction().commitTransaction()
|
||||
val v1 = table.currentSnapshot.snapshotId
|
||||
|
||||
// version 2
|
||||
spark
|
||||
.range(200, 300)
|
||||
.selectExpr("id as A", "cast(id / 20 as int) as B")
|
||||
.write
|
||||
.format("iceberg")
|
||||
.mode("append")
|
||||
.save(dataPath("T").toString)
|
||||
table.newTransaction().commitTransaction()
|
||||
val v2 = table.currentSnapshot.snapshotId
|
||||
|
||||
val df = (v: Long) =>
|
||||
spark.read.format("iceberg").option("snapshot-id", v).load(dataPath("T").toString)
|
||||
|
||||
// Create an index with version 1 data
|
||||
hs.createIndex(df(v1), DataSkippingIndexConfig("myind", MinMaxSketch("A")))
|
||||
|
||||
def query0: DataFrame = df(v0).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query0, 1)
|
||||
|
||||
def query1: DataFrame = df(v1).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query1, 2)
|
||||
|
||||
def query2: DataFrame = df(v2).filter("A = 1 or A = 101 or A = 201")
|
||||
checkIndexApplied(query2, 12)
|
||||
}
|
||||
}
|
||||
|
||||
def checkIndexApplied(query: => DataFrame, numExpectedFiles: Int): Unit = {
|
||||
withClue(s"query = ${query.queryExecution.logical}numExpectedFiles = $numExpectedFiles\n") {
|
||||
spark.disableHyperspace
|
||||
val queryWithoutIndex = query
|
||||
queryWithoutIndex.collect()
|
||||
spark.enableHyperspace
|
||||
val queryWithIndex = query
|
||||
queryWithIndex.collect()
|
||||
checkAnswer(queryWithIndex, queryWithoutIndex)
|
||||
assert(numAccessedFiles(queryWithIndex) === numExpectedFiles)
|
||||
}
|
||||
}
|
||||
|
||||
def numIndexDataFiles(name: String): Int = {
|
||||
val manager = Hyperspace.getContext(spark).indexCollectionManager
|
||||
val latestVersion = manager.getIndexVersions(name, Seq("ACTIVE")).max
|
||||
manager.getIndex(name, latestVersion).get.content.files.length
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,56 +16,60 @@
|
|||
|
||||
package com.microsoft.hyperspace.index.dataskipping
|
||||
|
||||
import scala.util.parsing.json.JSON
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.SaveMode
|
||||
import org.apache.spark.sql.functions.{input_file_name, max, min}
|
||||
import org.apache.spark.sql.types.IntegerType
|
||||
import org.apache.spark.sql.types.{IntegerType, StructType}
|
||||
|
||||
import com.microsoft.hyperspace.HyperspaceException
|
||||
import com.microsoft.hyperspace.index.{Content, FileInfo, Index, IndexConstants}
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketch.MinMaxSketch
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches.MinMaxSketch
|
||||
import com.microsoft.hyperspace.util.JsonUtils
|
||||
|
||||
class DataSkippingIndexTest extends DataSkippingSuite {
|
||||
override val numParallelism: Int = 3
|
||||
override val numParallelism: Int = 10
|
||||
|
||||
val emptyStructType = new StructType()
|
||||
|
||||
test("""kind returns "DataSkippingIndex".""") {
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")))
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")), emptyStructType)
|
||||
assert(index.kind === "DataSkippingIndex")
|
||||
}
|
||||
|
||||
test("""kindAbbr returns "DS".""") {
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")))
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")), emptyStructType)
|
||||
assert(index.kindAbbr === "DS")
|
||||
}
|
||||
|
||||
test("indexedColumns returns indexed columns of sketches.") {
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")))
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")), emptyStructType)
|
||||
assert(index.indexedColumns === Seq("A", "B"))
|
||||
}
|
||||
|
||||
test("referencedColumns returns indexed columns of sketches.") {
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")))
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")), emptyStructType)
|
||||
assert(index.referencedColumns === Seq("A", "B"))
|
||||
}
|
||||
|
||||
test(
|
||||
"withNewProperties returns a new index which copies the original index except the " +
|
||||
"properties.") {
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")))
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")), emptyStructType)
|
||||
val newIndex = index.withNewProperties(Map("foo" -> "bar"))
|
||||
assert(newIndex.properties === Map("foo" -> "bar"))
|
||||
assert(newIndex.sketches === index.sketches)
|
||||
}
|
||||
|
||||
test("statistics returns a string-formatted list of sketches.") {
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")))
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")), emptyStructType)
|
||||
assert(index.statistics() === Map("sketches" -> "MinMax(A), MinMax(B)"))
|
||||
}
|
||||
|
||||
test("canHandleDeletedFiles returns true.") {
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")))
|
||||
val index = DataSkippingIndex(Seq(MinMaxSketch("A")), emptyStructType)
|
||||
assert(index.canHandleDeletedFiles === true)
|
||||
}
|
||||
|
||||
|
@ -78,6 +82,14 @@ class DataSkippingIndexTest extends DataSkippingSuite {
|
|||
checkAnswer(writtenIndexData, indexData)
|
||||
}
|
||||
|
||||
test("write throws an exception if the index data schema doesn't match.") {
|
||||
val sourceData = createSourceData(spark.range(100).toDF("A"))
|
||||
val indexConfig = DataSkippingIndexConfig("myIndex", MinMaxSketch("A"))
|
||||
val (index, _) = indexConfig.createIndex(ctx, sourceData, Map())
|
||||
val ex = intercept[IllegalArgumentException](index.write(ctx, spark.range(10).toDF("B")))
|
||||
assert(ex.getMessage.contains("Schema of the index data doesn't match the index schema"))
|
||||
}
|
||||
|
||||
test("optimize reduces the number of index data files.") {
|
||||
val targetIndexDataFileSize = 100000
|
||||
val expectedNumIndexDataFiles = 1
|
||||
|
@ -112,16 +124,15 @@ class DataSkippingIndexTest extends DataSkippingSuite {
|
|||
}
|
||||
}
|
||||
|
||||
test("write throws an exception if target index data file size is too small.") {
|
||||
withSQLConf(IndexConstants.DATASKIPPING_TARGET_INDEX_DATA_FILE_SIZE -> "1") {
|
||||
test("write does not create more files than maxIndexDataFileCount.") {
|
||||
withSQLConf(
|
||||
IndexConstants.DATASKIPPING_TARGET_INDEX_DATA_FILE_SIZE -> "1",
|
||||
IndexConstants.DATASKIPPING_MAX_INDEX_DATA_FILE_COUNT -> "3") {
|
||||
val indexConfig = DataSkippingIndexConfig("myIndex", MinMaxSketch("A"))
|
||||
val sourceData = createSourceData(spark.range(100).toDF("A"))
|
||||
val sourceData = createSourceData(spark.range(10000).toDF("A"))
|
||||
val (index, indexData) = indexConfig.createIndex(ctx, sourceData, Map())
|
||||
val mockIndexData = RDDTestUtils.getMockDataFrameWithFakeSize(spark, 4000000000L)
|
||||
val ex = intercept[HyperspaceException](index.write(ctx, mockIndexData))
|
||||
assert(
|
||||
ex.getMessage.contains("Could not create index data files due to too many files: " +
|
||||
"indexDataSize=4000000000, targetIndexDataFileSize=1"))
|
||||
index.write(ctx, indexData)
|
||||
assert(listFiles(indexDataPath).filter(isParquet).length === 3)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -231,38 +242,45 @@ class DataSkippingIndexTest extends DataSkippingSuite {
|
|||
}
|
||||
|
||||
test("At least one sketch must be specified.") {
|
||||
val ex = intercept[AssertionError](DataSkippingIndex(Nil))
|
||||
val ex = intercept[AssertionError](DataSkippingIndex(Nil, emptyStructType))
|
||||
assert(ex.getMessage().contains("At least one sketch is required"))
|
||||
}
|
||||
|
||||
test("Indexes are equal if they have the same sketches and data types.") {
|
||||
val ds1 = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")))
|
||||
val ds2 = DataSkippingIndex(Seq(MinMaxSketch("B"), MinMaxSketch("A")))
|
||||
val ds1 = DataSkippingIndex(Seq(MinMaxSketch("A"), MinMaxSketch("B")), emptyStructType)
|
||||
val ds2 = DataSkippingIndex(Seq(MinMaxSketch("B"), MinMaxSketch("A")), emptyStructType)
|
||||
assert(ds1 === ds2)
|
||||
assert(ds1.hashCode === ds2.hashCode)
|
||||
}
|
||||
|
||||
test("Indexes are not equal to objects which are not indexes.") {
|
||||
val ds = DataSkippingIndex(Seq(MinMaxSketch("A")))
|
||||
val ds = DataSkippingIndex(Seq(MinMaxSketch("A")), emptyStructType)
|
||||
assert(ds !== "ds")
|
||||
}
|
||||
|
||||
test("Index can be serialized.") {
|
||||
val ds = DataSkippingIndex(Seq(MinMaxSketch("A", Some(IntegerType))), Map("a" -> "b"))
|
||||
val ds = DataSkippingIndex(
|
||||
Seq(MinMaxSketch("A", Some(IntegerType))),
|
||||
emptyStructType,
|
||||
Map("a" -> "b"))
|
||||
val json = JsonUtils.toJson(ds)
|
||||
assert(
|
||||
json ===
|
||||
val expectedJson =
|
||||
"""|{
|
||||
| "type" : "com.microsoft.hyperspace.index.dataskipping.DataSkippingIndex",
|
||||
| "sketches" : [ {
|
||||
| "type" : "com.microsoft.hyperspace.index.dataskipping.sketch.MinMaxSketch",
|
||||
| "type" : "com.microsoft.hyperspace.index.dataskipping.sketches.MinMaxSketch",
|
||||
| "expr" : "A",
|
||||
| "dataType" : "integer"
|
||||
| } ],
|
||||
| "schema" : {
|
||||
| "type" : "struct",
|
||||
| "fields" : [ ]
|
||||
| },
|
||||
| "properties" : {
|
||||
| "a" : "b"
|
||||
| }
|
||||
|}""".stripMargin)
|
||||
|}""".stripMargin
|
||||
assert(JSON.parseFull(json) === JSON.parseFull(expectedJson))
|
||||
}
|
||||
|
||||
test("Index can be deserialized.") {
|
||||
|
@ -270,16 +288,20 @@ class DataSkippingIndexTest extends DataSkippingSuite {
|
|||
"""|{
|
||||
| "type" : "com.microsoft.hyperspace.index.dataskipping.DataSkippingIndex",
|
||||
| "sketches" : [ {
|
||||
| "type" : "com.microsoft.hyperspace.index.dataskipping.sketch.MinMaxSketch",
|
||||
| "type" : "com.microsoft.hyperspace.index.dataskipping.sketches.MinMaxSketch",
|
||||
| "expr" : "A",
|
||||
| "dataType" : "integer"
|
||||
| } ],
|
||||
| "schema" : {
|
||||
| "type" : "struct",
|
||||
| "fields" : [ ]
|
||||
| },
|
||||
| "properties" : {
|
||||
| "a" : "b"
|
||||
| }
|
||||
|}""".stripMargin
|
||||
val ds = JsonUtils.fromJson[DataSkippingIndex](json)
|
||||
assert(ds === DataSkippingIndex(Seq(MinMaxSketch("A", Some(IntegerType)))))
|
||||
assert(ds === DataSkippingIndex(Seq(MinMaxSketch("A", Some(IntegerType))), emptyStructType))
|
||||
assert(ds.properties === Map("a" -> "b"))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,9 @@ import scala.collection.AbstractIterator
|
|||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileStatus, Path, RemoteIterator}
|
||||
import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.execution.DataSourceScanExec
|
||||
import org.apache.spark.sql.execution.datasources.FilePartition
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
|
||||
import com.microsoft.hyperspace.Hyperspace
|
||||
import com.microsoft.hyperspace.index._
|
||||
|
@ -52,6 +55,7 @@ trait DataSkippingSuite extends QueryTest with HyperspaceSuite {
|
|||
|
||||
after {
|
||||
FileUtils.delete(tempDir)
|
||||
spark.catalog.clearCache()
|
||||
}
|
||||
|
||||
def dataPath(path: String = "T"): Path = new Path(dataPathRoot, path)
|
||||
|
@ -60,16 +64,43 @@ trait DataSkippingSuite extends QueryTest with HyperspaceSuite {
|
|||
originalData: DataFrame,
|
||||
path: String = "T",
|
||||
saveMode: SaveMode = SaveMode.Overwrite,
|
||||
appendedDataOnly: Boolean = false): DataFrame = {
|
||||
appendedDataOnly: Boolean = false,
|
||||
format: String = "parquet"): DataFrame = {
|
||||
val p = dataPath(path)
|
||||
val oldFiles = listFiles(p).toSet
|
||||
originalData.write.mode(saveMode).parquet(p.toString)
|
||||
originalData.write.mode(saveMode).format(format).save(p.toString)
|
||||
updateFileIdTracker(p)
|
||||
if (appendedDataOnly) {
|
||||
val newFiles = listFiles(p).filterNot(oldFiles.contains)
|
||||
spark.read.parquet(newFiles.map(_.getPath.toString): _*)
|
||||
spark.read.format(format).load(newFiles.map(_.getPath.toString): _*)
|
||||
} else {
|
||||
spark.read.parquet(p.toString)
|
||||
spark.read.format(format).load(p.toString)
|
||||
}
|
||||
}
|
||||
|
||||
def createPartitionedSourceData(
|
||||
originalData: DataFrame,
|
||||
partitioningColumns: Seq[String],
|
||||
path: String = "T",
|
||||
saveMode: SaveMode = SaveMode.Overwrite,
|
||||
appendedDataOnly: Boolean = false,
|
||||
format: String = "parquet"): DataFrame = {
|
||||
val p = dataPath(path)
|
||||
val oldFiles = listFiles(p).toSet
|
||||
originalData.write
|
||||
.partitionBy(partitioningColumns: _*)
|
||||
.mode(saveMode)
|
||||
.format(format)
|
||||
.save(p.toString)
|
||||
updateFileIdTracker(p)
|
||||
if (appendedDataOnly) {
|
||||
val newFiles = listFiles(p).filterNot(oldFiles.contains)
|
||||
spark.read
|
||||
.option("basePath", p.toString)
|
||||
.format(format)
|
||||
.load(newFiles.map(_.getPath.toString): _*)
|
||||
} else {
|
||||
spark.read.format(format).load(p.toString)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -110,5 +141,55 @@ trait DataSkippingSuite extends QueryTest with HyperspaceSuite {
|
|||
fs.delete(path, true)
|
||||
}
|
||||
|
||||
def isParquet: FileStatus => Boolean = _.getPath.getName.endsWith(".parquet")
|
||||
def createFile(path: Path, data: Array[Byte]): Unit = {
|
||||
val fs = path.getFileSystem(new Configuration)
|
||||
val out = fs.create(path)
|
||||
out.write(data)
|
||||
out.close()
|
||||
}
|
||||
|
||||
def isParquet: FileStatus => Boolean = _.getPath.getName.endsWith(".parquet")
|
||||
|
||||
def withAndWithoutCodegen(testFun: => Unit): Unit = {
|
||||
import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._
|
||||
Seq(false, true).foreach { codegenEnabled =>
|
||||
withClue(s"codegenEnabled = $codegenEnabled") {
|
||||
val mode = if (codegenEnabled) CODEGEN_ONLY else NO_CODEGEN
|
||||
withSQLConf(
|
||||
SQLConf.CODEGEN_FACTORY_MODE.key -> mode.toString,
|
||||
SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegenEnabled.toString) {
|
||||
testFun
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def createIndexLogEntry(indexConfig: IndexConfigTrait, sourceData: DataFrame): IndexLogEntry = {
|
||||
val (index, indexData) = indexConfig.createIndex(ctx, sourceData, Map())
|
||||
index.write(ctx, indexData)
|
||||
IndexLogEntry(
|
||||
indexConfig.indexName,
|
||||
index,
|
||||
Content.fromDirectory(indexDataPath, fileIdTracker, new Configuration),
|
||||
Source(
|
||||
SparkPlan(SparkPlan.Properties(
|
||||
Seq(RelationUtils
|
||||
.getRelation(spark, sourceData.queryExecution.optimizedPlan)
|
||||
.createRelationMetadata(fileIdTracker)),
|
||||
null,
|
||||
null,
|
||||
LogicalPlanFingerprint(
|
||||
LogicalPlanFingerprint.Properties(Seq(Signature("sp", "sig"))))))),
|
||||
Map.empty)
|
||||
}
|
||||
|
||||
def numAccessedFiles(df: DataFrame): Int = {
|
||||
df.queryExecution.executedPlan.collect {
|
||||
case scan: DataSourceScanExec =>
|
||||
val files = scan.inputRDDs
|
||||
.flatMap(
|
||||
_.partitions.flatMap(_.asInstanceOf[FilePartition].files.map(_.filePath).toSet))
|
||||
files.length
|
||||
}.sum
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,38 +0,0 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping
|
||||
|
||||
import org.apache.spark.SparkContext
|
||||
import org.apache.spark.sql.{DataFrame, SparkSession}
|
||||
import org.apache.spark.storage.RDDInfo
|
||||
import org.mockito.Mockito.{mock, spy, when}
|
||||
|
||||
object RDDTestUtils {
|
||||
def getMockDataFrameWithFakeSize(spark: SparkSession, size: Long): DataFrame = {
|
||||
val df = spy(spark.emptyDataFrame)
|
||||
val rdd = spy(df.rdd)
|
||||
val mockSparkContext = mock(classOf[SparkContext])
|
||||
val mockRddStorageInfo = mock(classOf[RDDInfo])
|
||||
when(df.rdd).thenReturn(rdd)
|
||||
when(rdd.id).thenReturn(42)
|
||||
when(rdd.context).thenReturn(mockSparkContext)
|
||||
when(mockSparkContext.getRDDStorageInfo).thenReturn(Array[RDDInfo](mockRddStorageInfo))
|
||||
when(mockRddStorageInfo.id).thenReturn(42)
|
||||
when(mockRddStorageInfo.memSize).thenReturn(size)
|
||||
df
|
||||
}
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.execution
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.DataFrame
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.types._
|
||||
import org.mockito.Mockito.{mock, verify, when}
|
||||
|
||||
import com.microsoft.hyperspace.index.dataskipping._
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches._
|
||||
|
||||
class DataSkippingFileIndexTest extends DataSkippingSuite {
|
||||
test("DataSkippingFileIndex delegates methods to the FileIndex it is based on.") {
|
||||
val baseFileIndex = mock(classOf[FileIndex])
|
||||
val dsFileIndex = new DataSkippingFileIndex(
|
||||
spark,
|
||||
fileIdTracker,
|
||||
spark.emptyDataFrame,
|
||||
Literal.TrueLiteral,
|
||||
baseFileIndex)
|
||||
when(baseFileIndex.rootPaths).thenReturn(Seq(new Path("x")))
|
||||
assert(dsFileIndex.rootPaths === Seq(new Path("x")))
|
||||
when(baseFileIndex.inputFiles).thenReturn(Array("x/a", "x/b"))
|
||||
assert(dsFileIndex.inputFiles === Array("x/a", "x/b"))
|
||||
dsFileIndex.refresh()
|
||||
verify(baseFileIndex).refresh()
|
||||
when(baseFileIndex.sizeInBytes).thenReturn(12345)
|
||||
assert(dsFileIndex.sizeInBytes === 12345)
|
||||
val structType = StructType(StructField("A", IntegerType) :: Nil)
|
||||
when(baseFileIndex.partitionSchema).thenReturn(structType)
|
||||
assert(dsFileIndex.partitionSchema === structType)
|
||||
when(baseFileIndex.metadataOpsTimeNs).thenReturn(Some(100L))
|
||||
assert(dsFileIndex.metadataOpsTimeNs === Some(100L))
|
||||
}
|
||||
|
||||
test("listFiles returns partition directories with filtered files.") {
|
||||
val dsFileIndex = createDataSkippingFileIndex(
|
||||
spark.range(100).toDF("A"),
|
||||
MinMaxSketch("A"),
|
||||
LessThanOrEqual(UnresolvedAttribute("MinMax_A__0"), Literal(1)))
|
||||
val selectedPartitions = dsFileIndex.listFiles(Nil, Nil)
|
||||
val allPartitions = dsFileIndex.baseFileIndex.listFiles(Nil, Nil)
|
||||
assert(partitionsSize(selectedPartitions) < partitionsSize(allPartitions))
|
||||
assert(partitionsContain(allPartitions, selectedPartitions))
|
||||
}
|
||||
|
||||
def createDataSkippingFileIndex(
|
||||
df: DataFrame,
|
||||
sketch: Sketch,
|
||||
indexDataPred: Expression): DataSkippingFileIndex = {
|
||||
val sourceData = createSourceData(df)
|
||||
val baseFileIndex = sourceData.queryExecution.optimizedPlan.collectFirst {
|
||||
case LogicalRelation(HadoopFsRelation(location, _, _, _, _, _), _, _, _) => location
|
||||
}.get
|
||||
val (index, indexData) =
|
||||
DataSkippingIndexConfig("myind", sketch).createIndex(ctx, sourceData, Map.empty)
|
||||
new DataSkippingFileIndex(spark, fileIdTracker, indexData, indexDataPred, baseFileIndex)
|
||||
}
|
||||
|
||||
def partitionsSize(partitions: Seq[PartitionDirectory]): Long = {
|
||||
partitions.flatMap(_.files.map(_.getLen)).sum
|
||||
}
|
||||
|
||||
def partitionsContain(
|
||||
partitions: Seq[PartitionDirectory],
|
||||
others: Seq[PartitionDirectory]): Boolean = {
|
||||
val partitionsMap = partitions.map(pd => (pd.values, pd.files)).toMap
|
||||
others.forall { pd =>
|
||||
val files = partitionsMap.getOrElse(pd.values, Nil).toSet
|
||||
pd.files.forall(f => files.contains(f))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types._
|
||||
|
@ -37,6 +37,22 @@ class ExpressionUtilsTest extends HyperspaceSuite {
|
|||
assert(ExpressionUtils.normalize(expr) === expected)
|
||||
}
|
||||
|
||||
test("normalize removes expressions inserted for UDF.") {
|
||||
val arg = AttributeReference("A", IntegerType)(ExprId(42), Seq("t"))
|
||||
val func = (x: Int) => x + 1
|
||||
val expr = If(
|
||||
IsNull(arg),
|
||||
Literal(null, IntegerType),
|
||||
ScalaUDF(func, IntegerType, Seq(KnownNotNull(arg)), Nil))
|
||||
val expected =
|
||||
ScalaUDF(
|
||||
func,
|
||||
IntegerType,
|
||||
Seq(arg.withExprId(ExpressionUtils.nullExprId).withQualifier(Nil)),
|
||||
Nil)
|
||||
assert(ExpressionUtils.normalize(expr) === expected)
|
||||
}
|
||||
|
||||
test("ExtractIsNullDisjunction matches IsNull.") {
|
||||
val expr = IsNull(Literal(null))
|
||||
val args = expr match {
|
|
@ -0,0 +1,205 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types.{BooleanType, IntegerType}
|
||||
import org.mockito.ArgumentMatchers.any
|
||||
import org.mockito.Mockito.{mock, when}
|
||||
|
||||
import com.microsoft.hyperspace.index.HyperspaceSuite
|
||||
|
||||
class ExtractorsTest extends HyperspaceSuite {
|
||||
val a = AttributeReference("a", IntegerType)()
|
||||
val b = AttributeReference("b", IntegerType)()
|
||||
val c = AttributeReference("c", IntegerType)()
|
||||
val d = AttributeReference("d", BooleanType)()
|
||||
val e = AttributeReference("e", BooleanType)()
|
||||
|
||||
val aa = Literal(0)
|
||||
val bb = Literal(1)
|
||||
val dd = Literal(true)
|
||||
|
||||
def getExtractor(x: Expression, y: Expression): ExpressionExtractor = {
|
||||
val extractor = mock(classOf[ExpressionExtractor])
|
||||
when(extractor.unapply(any())).thenReturn(None)
|
||||
when(extractor.unapply(x)).thenReturn(Some(y))
|
||||
extractor
|
||||
}
|
||||
|
||||
val aExtractor = getExtractor(a, aa)
|
||||
val bExtractor = getExtractor(b, bb)
|
||||
val dExtractor = getExtractor(d, dd)
|
||||
|
||||
test("EqualToExtractor matches EqualTo(a, b).") {
|
||||
assert(EqualToExtractor(aExtractor, bExtractor).unapply(EqualTo(a, b)) === Some((aa, bb)))
|
||||
}
|
||||
|
||||
test("EqualToExtractor matches EqualTo(b, a).") {
|
||||
assert(EqualToExtractor(aExtractor, bExtractor).unapply(EqualTo(b, a)) === Some((aa, bb)))
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualTo(a, c).") {
|
||||
assert(EqualToExtractor(aExtractor, bExtractor).unapply(EqualTo(a, c)) === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor matches LessThan(a, b).") {
|
||||
assert(LessThanExtractor(aExtractor, bExtractor).unapply(LessThan(a, b)) === Some((aa, bb)))
|
||||
}
|
||||
|
||||
test("LessThanExtractor matches GreaterThan(b, a).") {
|
||||
assert(
|
||||
LessThanExtractor(aExtractor, bExtractor).unapply(GreaterThan(b, a)) ===
|
||||
Some((aa, bb)))
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match LessThan(b, a).") {
|
||||
assert(LessThanExtractor(aExtractor, bExtractor).unapply(LessThan(b, a)) === None)
|
||||
}
|
||||
|
||||
test("LessThanOrEqualExtractor matches LessThanOrEqual(a, b).") {
|
||||
assert(
|
||||
LessThanOrEqualExtractor(aExtractor, bExtractor).unapply(LessThanOrEqual(a, b)) ===
|
||||
Some((aa, bb)))
|
||||
}
|
||||
|
||||
test("LessThanOrEqualExtractor matches GreaterThanOrEqual(b, a).") {
|
||||
assert(
|
||||
LessThanOrEqualExtractor(aExtractor, bExtractor).unapply(GreaterThanOrEqual(b, a)) ===
|
||||
Some((aa, bb)))
|
||||
}
|
||||
|
||||
test("LessThanOrEqualExtractor does not match LessThanOrEqual(b, a).") {
|
||||
assert(
|
||||
LessThanOrEqualExtractor(aExtractor, bExtractor).unapply(LessThanOrEqual(b, a)) ===
|
||||
None)
|
||||
}
|
||||
|
||||
test("IsNullExtractor matches IsNull(a).") {
|
||||
assert(IsNullExtractor(aExtractor).unapply(IsNull(a)) === Some(aa))
|
||||
}
|
||||
|
||||
test("IsNullExtractor matches EqualNullSafe(a, null).") {
|
||||
assert(IsNullExtractor(aExtractor).unapply(EqualNullSafe(a, Literal(null))) === Some(aa))
|
||||
}
|
||||
|
||||
test("IsNullExtractor matches EqualNullSafe(null, a).") {
|
||||
assert(IsNullExtractor(aExtractor).unapply(EqualNullSafe(Literal(null), a)) === Some(aa))
|
||||
}
|
||||
|
||||
test("IsNullExtractor does not match IsNull(c).") {
|
||||
assert(IsNullExtractor(aExtractor).unapply(IsNull(c)) === None)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches IsNotNull(a).") {
|
||||
assert(IsNotNullExtractor(aExtractor).unapply(IsNotNull(a)) === Some(aa))
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches Not(IsNull(a)).") {
|
||||
assert(IsNotNullExtractor(aExtractor).unapply(Not(IsNull(a))) === Some(aa))
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches Not(EqualNullSafe(a, null)).") {
|
||||
assert(
|
||||
IsNotNullExtractor(aExtractor).unapply(Not(EqualNullSafe(a, Literal(null)))) ===
|
||||
Some(aa))
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches Not(EqualNullSafe(null, a)).") {
|
||||
assert(
|
||||
IsNotNullExtractor(aExtractor).unapply(Not(EqualNullSafe(Literal(null), a))) ===
|
||||
Some(aa))
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor does not match IsNotNull(c).") {
|
||||
assert(IsNotNullExtractor(aExtractor).unapply(IsNotNull(c)) === None)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor matches d.") {
|
||||
assert(IsTrueExtractor(dExtractor).unapply(d) === Some(dd))
|
||||
}
|
||||
|
||||
test("IsTrueExtractor matches EqualTo(d, true).") {
|
||||
assert(IsTrueExtractor(dExtractor).unapply(EqualTo(d, Literal(true))) === Some(dd))
|
||||
}
|
||||
|
||||
test("IsTrueExtractor matches EqualTo(true, d).") {
|
||||
assert(IsTrueExtractor(dExtractor).unapply(EqualTo(Literal(true), d)) === Some(dd))
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match e.") {
|
||||
assert(IsTrueExtractor(dExtractor).unapply(e) === None)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match a.") {
|
||||
assert(IsTrueExtractor(dExtractor).unapply(a) === None)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor matches Not(d).") {
|
||||
assert(IsFalseExtractor(dExtractor).unapply(Not(d)) === Some(dd))
|
||||
}
|
||||
|
||||
test("IsFalseExtractor matches EqualTo(d, false).") {
|
||||
assert(IsFalseExtractor(dExtractor).unapply(EqualTo(d, Literal(false))) === Some(dd))
|
||||
}
|
||||
|
||||
test("IsFalseExtractor matches EqualTo(false, d).") {
|
||||
assert(IsFalseExtractor(dExtractor).unapply(EqualTo(Literal(false), d)) === Some(dd))
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match Not(e).") {
|
||||
assert(IsFalseExtractor(dExtractor).unapply(Not(e)) === None)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match a.") {
|
||||
assert(IsFalseExtractor(dExtractor).unapply(a) === None)
|
||||
}
|
||||
|
||||
test("InExtractor matches In(a, Seq()).") {
|
||||
assert(InExtractor(aExtractor, bExtractor).unapply(In(a, Seq())) === Some((aa, Seq())))
|
||||
}
|
||||
|
||||
test("InExtractor matches In(a, Seq(b)).") {
|
||||
assert(InExtractor(aExtractor, bExtractor).unapply(In(a, Seq(b))) === Some((aa, Seq(bb))))
|
||||
}
|
||||
|
||||
test("InExtractor matches In(a, Seq(b, b)).") {
|
||||
assert(
|
||||
InExtractor(aExtractor, bExtractor).unapply(In(a, Seq(b, b))) ===
|
||||
Some((aa, Seq(bb, bb))))
|
||||
}
|
||||
|
||||
test("InExtractor does not match In(a, Seq(b, c)).") {
|
||||
assert(InExtractor(aExtractor, bExtractor).unapply(In(a, Seq(b, c))) === None)
|
||||
}
|
||||
|
||||
test("InExtractor does not match In(c, Seq(b, b)).") {
|
||||
assert(InExtractor(aExtractor, bExtractor).unapply(In(c, Seq(b, b))) === None)
|
||||
}
|
||||
|
||||
test("InSetExtractor matches InSet(a, Set()).") {
|
||||
assert(InSetExtractor(aExtractor).unapply(InSet(a, Set[Any]())) === Some((aa, Set[Any]())))
|
||||
}
|
||||
|
||||
test("InSetExtractor matches InSet(a, Set(1)).") {
|
||||
assert(InSetExtractor(aExtractor).unapply(InSet(a, Set[Any](1))) === Some((aa, Set[Any](1))))
|
||||
}
|
||||
|
||||
test("InSetExtractor does not match InSet(c, Set(1)).") {
|
||||
assert(InSetExtractor(aExtractor).unapply(InSet(c, Set[Any](1))) === None)
|
||||
}
|
||||
}
|
|
@ -14,23 +14,25 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
import com.microsoft.hyperspace.index.HyperspaceSuite
|
||||
|
||||
class NormalizedExprMatcherTest extends HyperspaceSuite {
|
||||
val matcher = NormalizedExprMatcher(
|
||||
class NormalizedExprExtractorTest extends HyperspaceSuite {
|
||||
val extractor = NormalizedExprExtractor(
|
||||
AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId, Nil),
|
||||
Map(ExprId(42) -> "A"))
|
||||
|
||||
test("apply returns true if the expression matches.") {
|
||||
assert(matcher(AttributeReference("a", IntegerType)(ExprId(42), Nil)) === true)
|
||||
assert(
|
||||
extractor.unapply(AttributeReference("a", IntegerType)(ExprId(42), Nil)) ===
|
||||
Some(extractor.expr))
|
||||
}
|
||||
|
||||
test("apply returns false if the expression does not match") {
|
||||
assert(matcher(Literal(42)) === false)
|
||||
assert(extractor.unapply(Literal(42)) === None)
|
||||
}
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
package com.microsoft.hyperspace.index.dataskipping.expressions
|
||||
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen._
|
||||
|
@ -84,4 +84,10 @@ class SortedArrayLowerBoundTest
|
|||
checkEvaluation(expr(IntegerType, Seq(1, 3), 4), null)
|
||||
checkEvaluation(expr(IntegerType, Seq(1, 3, 5), 6), null)
|
||||
}
|
||||
|
||||
test("SortedArrayLowerBound returns null if the value is null.") {
|
||||
checkEvaluation(expr(IntegerType, Seq(1), null), null)
|
||||
checkEvaluation(expr(IntegerType, Seq(1, 3), null), null)
|
||||
checkEvaluation(expr(IntegerType, Seq(1, 3, 5), null), null)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,372 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.rules
|
||||
|
||||
import org.apache.spark.sql.DataFrame
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.hyperspace.utils.logicalPlanToDataFrame
|
||||
|
||||
import com.microsoft.hyperspace.index._
|
||||
import com.microsoft.hyperspace.index.dataskipping._
|
||||
import com.microsoft.hyperspace.index.dataskipping.execution.DataSkippingFileIndex
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches._
|
||||
|
||||
class ApplyDataSkippingIndexTest extends DataSkippingSuite {
|
||||
import spark.implicits._
|
||||
|
||||
override val numParallelism: Int = 10
|
||||
|
||||
test("applyIndex returns the unmodified plan if no index is given.") {
|
||||
val sourceData = createSourceData(spark.range(100).toDF("A"))
|
||||
val query = sourceData.filter("A = 1")
|
||||
val plan = query.queryExecution.optimizedPlan
|
||||
assert(ApplyDataSkippingIndex.applyIndex(plan, Map.empty) === plan)
|
||||
}
|
||||
|
||||
test("score returns 0 if no index is given.") {
|
||||
val sourceData = createSourceData(spark.range(100).toDF("A"))
|
||||
val query = sourceData.filter("A = 1")
|
||||
val plan = query.queryExecution.optimizedPlan
|
||||
assert(ApplyDataSkippingIndex.score(plan, Map.empty) === 0)
|
||||
}
|
||||
|
||||
case class SourceData(df: () => DataFrame, description: String)
|
||||
|
||||
case class Param(
|
||||
sourceData: SourceData,
|
||||
filter: String,
|
||||
sketches: Seq[Sketch],
|
||||
numExpectedFiles: Int,
|
||||
setup: Option[() => _])
|
||||
|
||||
object Param {
|
||||
def apply(
|
||||
sourceData: SourceData,
|
||||
filter: String,
|
||||
sketch: Sketch,
|
||||
numExpectedFiles: Int): Param = {
|
||||
Param(sourceData, filter, Seq(sketch), numExpectedFiles, None)
|
||||
}
|
||||
|
||||
def apply(
|
||||
sourceData: SourceData,
|
||||
filter: String,
|
||||
sketches: Seq[Sketch],
|
||||
numExpectedFiles: Int): Param = {
|
||||
Param(sourceData, filter, sketches, numExpectedFiles, None)
|
||||
}
|
||||
|
||||
def apply(
|
||||
sourceData: SourceData,
|
||||
filter: String,
|
||||
sketch: Sketch,
|
||||
numExpectedFiles: Int,
|
||||
setup: () => _): Param = {
|
||||
Param(sourceData, filter, Seq(sketch), numExpectedFiles, Some(setup))
|
||||
}
|
||||
|
||||
def apply(
|
||||
sourceData: SourceData,
|
||||
filter: String,
|
||||
sketches: Seq[Sketch],
|
||||
numExpectedFiles: Int,
|
||||
setup: () => _): Param = {
|
||||
Param(sourceData, filter, sketches, numExpectedFiles, Some(setup))
|
||||
}
|
||||
}
|
||||
|
||||
def dataI: SourceData =
|
||||
SourceData(() => createSourceData(spark.range(100).toDF("A")), "source [A:Int]")
|
||||
|
||||
def dataII: SourceData =
|
||||
SourceData(
|
||||
() => createSourceData(spark.range(100).selectExpr("id as A", "id * 2 as B")),
|
||||
"source [A:Int, B:Int]")
|
||||
|
||||
def dataIN: SourceData =
|
||||
SourceData(
|
||||
() =>
|
||||
createSourceData(
|
||||
Seq[Integer](1, 2, null, null, null, null, 7, 8, 9, null, 11, 12, null, 14, null, null,
|
||||
17, null, 19, 20).toDF("A")),
|
||||
"source [A:Int] with nulls")
|
||||
|
||||
def dataIIP: SourceData =
|
||||
SourceData(
|
||||
() =>
|
||||
createPartitionedSourceData(
|
||||
spark.range(100).selectExpr("cast(id / 10 as int) as A", "id as B"),
|
||||
Seq("A")),
|
||||
"source [A:Int, B:Int] partitioned")
|
||||
|
||||
def dataD: SourceData =
|
||||
SourceData(
|
||||
() => createSourceData(spark.range(100).map(_.toDouble).toDF("A")),
|
||||
"source [A:Double]")
|
||||
|
||||
def dataDS: SourceData =
|
||||
SourceData(
|
||||
() =>
|
||||
createSourceData(
|
||||
Seq(
|
||||
0.0,
|
||||
1.0,
|
||||
1.5,
|
||||
Double.NegativeInfinity,
|
||||
Double.PositiveInfinity,
|
||||
Double.NaN,
|
||||
3.14,
|
||||
2.718,
|
||||
-1.1,
|
||||
-0.0).toDF("A")),
|
||||
"source [A:Double] small")
|
||||
|
||||
def dataN2: SourceData =
|
||||
SourceData(
|
||||
() =>
|
||||
createSourceData(
|
||||
spark.read.json(Seq(
|
||||
"""{"a": 1, "b": {"a": 0, "c": 2, "d": "x"}}""",
|
||||
"""{"a": 2, "b": {"a": 0, "c": 3, "d": "y"}}""",
|
||||
"""{"a": 3, "b": {"a": 1, "c": 4, "d": "x"}}""",
|
||||
"""{"a": 4, "b": {"a": 2, "c": null, "d": "x"}}""",
|
||||
"""{"a": 2, "b": {"a": 2, "c": 6, "d": "x"}}""",
|
||||
"""{"a": 2, "b": {"a": 1, "c": 7, "d": "x"}}""",
|
||||
"""{"b": {"c": 8, "d": "x"}}""",
|
||||
"""{"b": {"d": "y"}}""",
|
||||
"""{"a": 3}""",
|
||||
"""{"b": {"c": 11}}""").toDS)),
|
||||
"source [A:Int, B:[A:Int, C:Int, D: String]]")
|
||||
|
||||
def dataN3: SourceData =
|
||||
SourceData(
|
||||
() =>
|
||||
createSourceData(
|
||||
spark.read.json(Seq(
|
||||
"""{"a": {"b": {"c": 1}}}""",
|
||||
"""{"a": {"b": {"c": 2}}}""",
|
||||
"""{"a": {"b": {"c": 3}}}""",
|
||||
"""{"a": {"b": {"c": null}}}""",
|
||||
"""{"a": {"b": {"c": 5}}}""",
|
||||
"""{"a": {"b": {"c": 6}}}""",
|
||||
"""{"a": {"b": {"c": 7}}}""",
|
||||
"""{"a": {"b": {"c": 8}}}""",
|
||||
"""{"a": null}""",
|
||||
"""{"a": {"b": {"c": 0}}}""").toDS)),
|
||||
"source [A:[B:[C:Int]]]")
|
||||
|
||||
def dataB: SourceData =
|
||||
SourceData(
|
||||
() =>
|
||||
createSourceData(
|
||||
Seq(
|
||||
Array[Byte](0, 0, 0, 0),
|
||||
Array[Byte](0, 1, 0, 1),
|
||||
Array[Byte](1, 2, 3, 4),
|
||||
Array[Byte](5, 6, 7, 8),
|
||||
Array[Byte](32, 32, 32, 32),
|
||||
Array[Byte](64, 64, 64, 64),
|
||||
Array[Byte](1, 1, 1, 1),
|
||||
Array[Byte](-128, -128, -128, -128),
|
||||
Array[Byte](127, 127, 127, 127),
|
||||
Array[Byte](-1, 1, 0, 0)).toDF("A")),
|
||||
"source [A:Binary]")
|
||||
|
||||
def dataS: SourceData =
|
||||
SourceData(
|
||||
() =>
|
||||
createSourceData(
|
||||
Seq(
|
||||
"foo1",
|
||||
"foo2000",
|
||||
"foo3",
|
||||
"foo4",
|
||||
"foo5",
|
||||
null,
|
||||
"foo7",
|
||||
"foo8",
|
||||
"foo9",
|
||||
"baar",
|
||||
null)
|
||||
.toDF("A")),
|
||||
"source [A:String]")
|
||||
|
||||
Seq(
|
||||
Param(dataI, "A = 10", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "50 = a", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "A = -10", MinMaxSketch("a"), 0),
|
||||
Param(dataI, "A = 5 + 5", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "A = 10 or A = 30", MinMaxSketch("A"), 2),
|
||||
Param(dataI, "A is null", MinMaxSketch("A"), 10),
|
||||
Param(dataI, "!(A is null)", MinMaxSketch("A"), 10),
|
||||
Param(dataI, "A is not null", MinMaxSketch("A"), 10),
|
||||
Param(dataI, "!(A is not null)", MinMaxSketch("A"), 10),
|
||||
Param(dataI, "A <=> 10", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "10 <=> A", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "A <=> null", MinMaxSketch("A"), 10),
|
||||
Param(dataI, "A <25", MinMaxSketch("A"), 3),
|
||||
Param(dataI, "30>A", MinMaxSketch("A"), 3),
|
||||
Param(dataI, "31 > A", MinMaxSketch("a"), 4),
|
||||
Param(dataI, "A > 25", MinMaxSketch("a"), 8),
|
||||
Param(dataI, "28 < A", MinMaxSketch("a"), 8),
|
||||
Param(dataI, "29< A", MinMaxSketch("A"), 7),
|
||||
Param(dataI, "A <= 25", MinMaxSketch("A"), 3),
|
||||
Param(dataI, "29 >= A", MinMaxSketch("A"), 3),
|
||||
Param(dataI, "30>=A", MinMaxSketch("A"), 4),
|
||||
Param(dataI, "A >= 25", MinMaxSketch("A"), 8),
|
||||
Param(dataI, "29 <= A", MinMaxSketch("A"), 8),
|
||||
Param(dataI, "30 <= A", MinMaxSketch("A"), 7),
|
||||
Param(dataI, "A != 1", MinMaxSketch("A"), 10),
|
||||
Param(dataI, "not (A != 1 and A != 10)", MinMaxSketch("A"), 2),
|
||||
Param(dataI, "!(!(A = 1))", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "!(A < 20)", MinMaxSketch("A"), 8),
|
||||
Param(dataI, "not (A not in (1, 2, 3))", MinMaxSketch("A"), 1),
|
||||
Param(dataS, "A < 'foo'", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "a = 10", MinMaxSketch("A"), 1),
|
||||
Param(dataI, "A = 10", MinMaxSketch("a"), 1),
|
||||
Param(dataI, "A in (1, 2, 3, null, 10)", MinMaxSketch("A"), 2),
|
||||
Param(dataI, "A in (10,9,8,7,6,5,4,3,2,1,50,49,48,47,46,45)", MinMaxSketch("A"), 4),
|
||||
Param(dataS, "A in ('foo1', 'foo5', 'foo9')", MinMaxSketch("A"), 3),
|
||||
Param(
|
||||
dataS,
|
||||
"A in ('foo1','a','b','c','d','e','f','g','h','i','j','k')",
|
||||
MinMaxSketch("A"),
|
||||
1),
|
||||
Param(dataD, "A in (1,2,3,15,16,17)", MinMaxSketch("A"), 2),
|
||||
Param(dataD, "A in (1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16)", MinMaxSketch("A"), 2),
|
||||
Param(dataB, "A in (x'00000000', x'0001', x'0002', x'05060708')", MinMaxSketch("A"), 2),
|
||||
Param(
|
||||
dataB,
|
||||
"A in (x'00',x'01',x'02',x'03',x'04',x'05',x'06',x'07',x'08',x'09',x'0a',x'20202020')",
|
||||
MinMaxSketch("A"),
|
||||
1),
|
||||
Param(dataI, "A BETWEEN 27 AND 51", MinMaxSketch("A"), 4),
|
||||
Param(dataI, "IF(A=1,2,3)=2", MinMaxSketch("A"), 10),
|
||||
Param(dataII, "A = 10 OR B = 50", Seq(MinMaxSketch("A"), MinMaxSketch("B")), 2),
|
||||
Param(dataII, "A = 10 or B = 50", Seq(MinMaxSketch("A")), 10),
|
||||
Param(dataII, "B = 50 or A = 10", Seq(MinMaxSketch("A")), 10),
|
||||
Param(dataII, "A = 10 and B = 20", MinMaxSketch("A"), 1),
|
||||
Param(dataII, "a = 10 AND b = 20", Seq(MinMaxSketch("A"), MinMaxSketch("B")), 1),
|
||||
Param(dataII, "A < 30 and B > 20", MinMaxSketch("A"), 3),
|
||||
Param(dataII, "A < 30 and b > 40", Seq(MinMaxSketch("a"), MinMaxSketch("B")), 1),
|
||||
Param(dataII, "A = 10 and B = 90", Seq(MinMaxSketch("A"), MinMaxSketch("B")), 0),
|
||||
Param(dataIN, "A is not null", MinMaxSketch("A"), 7),
|
||||
Param(dataIN, "!(A <=> null)", MinMaxSketch("A"), 7),
|
||||
Param(dataIN, "A = 2", MinMaxSketch("A"), 1),
|
||||
Param(dataIN, "A is null", MinMaxSketch("A"), 10),
|
||||
Param(dataIIP, "B = 10", MinMaxSketch("B"), 1),
|
||||
Param(dataIIP, "A = 5 and B = 20", MinMaxSketch("B"), 0),
|
||||
Param(dataIIP, "A < 5 and B = 20", MinMaxSketch("B"), 1),
|
||||
Param(dataN2, "B.C = 2", MinMaxSketch("B.C"), 1),
|
||||
Param(dataN2, "B.c = 2", MinMaxSketch("b.C"), 1),
|
||||
Param(dataN2, "b.c < 5", MinMaxSketch("b.c"), 3),
|
||||
Param(dataN3, "A.B.C = 2", MinMaxSketch("a.B.C"), 1),
|
||||
Param(dataDS, "A = 1.0", MinMaxSketch("A"), 1),
|
||||
Param(dataDS, "A <= 1.5", MinMaxSketch("A"), 6),
|
||||
Param(dataDS, "A >= 1.5", MinMaxSketch("A"), 5),
|
||||
Param(dataD, "A in (1, 2, 3, 10)", MinMaxSketch("A"), 2),
|
||||
Param(dataII, "A + B < 100", MinMaxSketch("a+b"), 4),
|
||||
Param(
|
||||
dataI,
|
||||
"F(A) = 10",
|
||||
MinMaxSketch("F(A)"),
|
||||
1,
|
||||
() => spark.udf.register("F", (a: Int) => a * 2)),
|
||||
Param(
|
||||
dataI,
|
||||
"is_less_than_23(A)",
|
||||
MinMaxSketch("is_less_than_23(A)"),
|
||||
3,
|
||||
() => spark.udf.register("is_less_than_23", (a: Int) => a < 23)),
|
||||
Param(
|
||||
dataI,
|
||||
"!is_less_than_23(A)",
|
||||
MinMaxSketch("is_less_than_23(A)"),
|
||||
8,
|
||||
() => spark.udf.register("is_less_than_23", (a: Int) => a < 23)),
|
||||
Param(
|
||||
dataII,
|
||||
"A < 50 and F(A,B) < 20",
|
||||
Seq(MinMaxSketch("A"), MinMaxSketch("F(A,B)")),
|
||||
2,
|
||||
() => spark.udf.register("F", (a: Int, b: Int) => b - a)),
|
||||
Param(
|
||||
dataI,
|
||||
"f(a) < 30",
|
||||
MinMaxSketch("F(a)"),
|
||||
2,
|
||||
() => spark.udf.register("F", (a: Int) => a * 2)),
|
||||
Param(
|
||||
dataI,
|
||||
"IF(A IS NULL,NULL,F(A))=2",
|
||||
MinMaxSketch("A"),
|
||||
10,
|
||||
() => spark.udf.register("F", (a: Int) => a * 2))).foreach {
|
||||
case Param(sourceData, filter, sketches, numExpectedFiles, setup) =>
|
||||
test(
|
||||
s"applyIndex works as expected for ${sourceData.description}: " +
|
||||
s"filter=[$filter], sketches=[${sketches.mkString(", ")}], " +
|
||||
s"numExpectedFiles=[$numExpectedFiles]") {
|
||||
val indexConfig = DataSkippingIndexConfig("ind1", sketches.head, sketches.tail: _*)
|
||||
if (setup.nonEmpty) {
|
||||
setup.get.apply()
|
||||
}
|
||||
testApplyIndex(sourceData.df(), filter, indexConfig, numExpectedFiles)
|
||||
}
|
||||
}
|
||||
|
||||
def testApplyIndex(
|
||||
sourceData: DataFrame,
|
||||
filter: String,
|
||||
indexConfig: DataSkippingIndexConfig,
|
||||
numExpectedFiles: Int): Unit = {
|
||||
val originalNumFiles = listFiles(dataPath()).filter(isParquet).length
|
||||
val query = sourceData.filter(filter)
|
||||
val plan = query.queryExecution.optimizedPlan
|
||||
val indexLogEntry = createIndexLogEntry(indexConfig, sourceData)
|
||||
val indexDataPred = indexLogEntry.derivedDataset
|
||||
.asInstanceOf[DataSkippingIndex]
|
||||
.translateFilterCondition(
|
||||
spark,
|
||||
plan.asInstanceOf[Filter].condition,
|
||||
sourceData.queryExecution.optimizedPlan)
|
||||
indexLogEntry.setTagValue(plan, IndexLogEntryTags.DATASKIPPING_INDEX_PREDICATE, indexDataPred)
|
||||
val optimizedPlan = ApplyDataSkippingIndex.applyIndex(
|
||||
plan,
|
||||
Map(sourceData.queryExecution.optimizedPlan -> indexLogEntry))
|
||||
if (indexDataPred.isEmpty) {
|
||||
assert(optimizedPlan === plan)
|
||||
} else {
|
||||
assert(optimizedPlan !== plan)
|
||||
optimizedPlan match {
|
||||
case Filter(
|
||||
_,
|
||||
LogicalRelation(
|
||||
HadoopFsRelation(location: DataSkippingFileIndex, _, _, _, _, _),
|
||||
_,
|
||||
_,
|
||||
_)) =>
|
||||
assert(location.indexDataPred === indexDataPred.get)
|
||||
case _ => fail(s"unexpected optimizedPlan: $optimizedPlan")
|
||||
}
|
||||
}
|
||||
val optimizedDf = logicalPlanToDataFrame(spark, optimizedPlan)
|
||||
checkAnswer(optimizedDf, query)
|
||||
assert(numAccessedFiles(optimizedDf) === numExpectedFiles)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.rules
|
||||
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
|
||||
import com.microsoft.hyperspace.index.IndexLogEntryTags
|
||||
import com.microsoft.hyperspace.index.dataskipping._
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches._
|
||||
import com.microsoft.hyperspace.index.plananalysis.FilterReasons.IneligibleFilterCondition
|
||||
|
||||
class FilterConditionFilterTest extends DataSkippingSuite {
|
||||
test("apply returns an empty map if there are no candidate indexes.") {
|
||||
val df = spark.range(10).toDF("A")
|
||||
assert(FilterConditionFilter(df.queryExecution.optimizedPlan, Map.empty) === Map.empty)
|
||||
}
|
||||
|
||||
test("apply returns an empty map if the plan is not a filter.") {
|
||||
val df = createSourceData(spark.range(10).toDF("A"))
|
||||
val indexConfig = DataSkippingIndexConfig("myind", MinMaxSketch("A"))
|
||||
val indexLogEntry = createIndexLogEntry(indexConfig, df)
|
||||
val candidateIndexes = Map(df.queryExecution.optimizedPlan -> Seq(indexLogEntry))
|
||||
assert(FilterConditionFilter(df.queryExecution.optimizedPlan, candidateIndexes) === Map.empty)
|
||||
}
|
||||
|
||||
test("apply creates an index data predicate if the index can be applied to the plan.") {
|
||||
val df = createSourceData(spark.range(10).toDF("A"))
|
||||
val indexConfig = DataSkippingIndexConfig("myind", MinMaxSketch("A"))
|
||||
val indexLogEntry = createIndexLogEntry(indexConfig, df)
|
||||
val candidateIndexes = Map(df.queryExecution.optimizedPlan -> Seq(indexLogEntry))
|
||||
val plan = df.filter("A = 1").queryExecution.optimizedPlan
|
||||
assert(FilterConditionFilter(plan, candidateIndexes) === candidateIndexes)
|
||||
val indexDataPredOpt =
|
||||
indexLogEntry.getTagValue(plan, IndexLogEntryTags.DATASKIPPING_INDEX_PREDICATE)
|
||||
assert(
|
||||
indexDataPredOpt === Some(Some(And(
|
||||
IsNotNull(UnresolvedAttribute("MinMax_A__0")),
|
||||
And(
|
||||
LessThanOrEqual(UnresolvedAttribute("MinMax_A__0"), Literal(1L)),
|
||||
GreaterThanOrEqual(UnresolvedAttribute("MinMax_A__1"), Literal(1L)))))))
|
||||
}
|
||||
|
||||
test("apply returns an empty map if the filter condition is not suitable.") {
|
||||
val df = createSourceData(spark.range(10).selectExpr("id as A", "id * 2 as B"))
|
||||
val indexConfig = DataSkippingIndexConfig("myind", MinMaxSketch("A"))
|
||||
val indexLogEntry = createIndexLogEntry(indexConfig, df)
|
||||
indexLogEntry.setTagValue(IndexLogEntryTags.INDEX_PLAN_ANALYSIS_ENABLED, true)
|
||||
val candidateIndexes = Map(df.queryExecution.optimizedPlan -> Seq(indexLogEntry))
|
||||
val plan = df.filter("B = 1").queryExecution.optimizedPlan
|
||||
assert(FilterConditionFilter(plan, candidateIndexes) === Map.empty)
|
||||
val reason = indexLogEntry.getTagValue(plan, IndexLogEntryTags.FILTER_REASONS)
|
||||
assert(reason === Some(List(IneligibleFilterCondition("((`B` IS NOT NULL) AND (`B` = 1L))"))))
|
||||
}
|
||||
|
||||
test("apply returns only the applicable indexes when there are multiple candidate indexes.") {
|
||||
val df = createSourceData(spark.range(10).selectExpr("id as A", "id * 2 as B"))
|
||||
val indexConfig1 = DataSkippingIndexConfig("myind", MinMaxSketch("A"))
|
||||
val indexConfig2 = DataSkippingIndexConfig("myind", MinMaxSketch("B"))
|
||||
val indexLogEntry1 = createIndexLogEntry(indexConfig1, df)
|
||||
val indexLogEntry2 = createIndexLogEntry(indexConfig2, df)
|
||||
val candidateIndexes =
|
||||
Map(df.queryExecution.optimizedPlan -> Seq(indexLogEntry1, indexLogEntry2))
|
||||
val plan = df.filter("A = 1").queryExecution.optimizedPlan
|
||||
assert(
|
||||
FilterConditionFilter(plan, candidateIndexes) === Map(
|
||||
df.queryExecution.optimizedPlan -> Seq(indexLogEntry1)))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.rules
|
||||
|
||||
import com.microsoft.hyperspace.index.dataskipping._
|
||||
import com.microsoft.hyperspace.index.dataskipping.sketches._
|
||||
|
||||
class FilterPlanNodeFilterTest extends DataSkippingSuite {
|
||||
test("apply returns an empty map if there are no candidate indexes.") {
|
||||
val df = spark.range(10).toDF("A")
|
||||
assert(FilterPlanNodeFilter(df.queryExecution.optimizedPlan, Map.empty) === Map.empty)
|
||||
}
|
||||
|
||||
test("apply returns an empty map if the plan is not a filter.") {
|
||||
val df = createSourceData(spark.range(10).toDF("A"))
|
||||
val indexConfig = DataSkippingIndexConfig("myind", MinMaxSketch("A"))
|
||||
val indexLogEntry = createIndexLogEntry(indexConfig, df)
|
||||
val candidateIndexes = Map(df.queryExecution.optimizedPlan -> Seq(indexLogEntry))
|
||||
val plan = df.groupBy("A").count().queryExecution.optimizedPlan
|
||||
assert(FilterPlanNodeFilter(plan, candidateIndexes) === Map.empty)
|
||||
}
|
||||
|
||||
test("apply returns applicable indexes only.") {
|
||||
val df1 = createSourceData(spark.range(10).toDF("A"), "T1")
|
||||
val df2 = createSourceData(spark.range(10).toDF("A"), "T2")
|
||||
val indexConfig = DataSkippingIndexConfig("myind", MinMaxSketch("A"))
|
||||
val indexLogEntry1 = createIndexLogEntry(indexConfig, df1)
|
||||
val indexLogEntry2 = createIndexLogEntry(indexConfig, df2)
|
||||
val candidateIndexes = Map(
|
||||
df1.queryExecution.optimizedPlan -> Seq(indexLogEntry1),
|
||||
df2.queryExecution.optimizedPlan -> Seq(indexLogEntry2))
|
||||
val plan = df1.filter("A = 1").queryExecution.optimizedPlan
|
||||
assert(
|
||||
FilterPlanNodeFilter(plan, candidateIndexes) === Map(
|
||||
df1.queryExecution.optimizedPlan -> Seq(indexLogEntry1)))
|
||||
}
|
||||
}
|
|
@ -1,66 +0,0 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketch
|
||||
|
||||
import org.apache.spark.sql.{Column, QueryTest}
|
||||
import org.mockito.Mockito.mock
|
||||
|
||||
import com.microsoft.hyperspace.index.HyperspaceSuite
|
||||
|
||||
class MinMaxSketchTest extends QueryTest with HyperspaceSuite {
|
||||
import spark.implicits._
|
||||
|
||||
test("indexedColumns returns the indexed column.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
assert(sketch.indexedColumns === Seq("A"))
|
||||
}
|
||||
|
||||
test("referencedColumns returns the indexed column.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
assert(sketch.referencedColumns === Seq("A"))
|
||||
}
|
||||
|
||||
test("aggregateFunctions returns min and max aggregation functions.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val aggrs = sketch.aggregateFunctions.map(new Column(_))
|
||||
val data = Seq(1, -1, 10, 2, 4).toDF("A")
|
||||
checkAnswer(data.select(aggrs: _*), Seq((-1, 10)).toDF)
|
||||
}
|
||||
|
||||
test("toString returns a reasonable string.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
assert(sketch.toString === "MinMax(A)")
|
||||
}
|
||||
|
||||
test("Two sketches are equal if their columns are equal.") {
|
||||
assert(MinMaxSketch("A") === MinMaxSketch("A"))
|
||||
assert(MinMaxSketch("A") !== MinMaxSketch("a"))
|
||||
assert(MinMaxSketch("b") !== MinMaxSketch("B"))
|
||||
assert(MinMaxSketch("B") === MinMaxSketch("B"))
|
||||
}
|
||||
|
||||
test("MinMaxSketch is different from other sketches.") {
|
||||
val s1 = MinMaxSketch("A")
|
||||
val s2 = mock(classOf[Sketch])
|
||||
assert(s1 !== s2)
|
||||
}
|
||||
|
||||
test("hashCode is reasonably implemented.") {
|
||||
assert(MinMaxSketch("A").hashCode === MinMaxSketch("A").hashCode)
|
||||
assert(MinMaxSketch("A").hashCode !== MinMaxSketch("a").hashCode)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,338 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketches
|
||||
|
||||
import org.apache.spark.sql.{Column, QueryTest}
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types._
|
||||
import org.mockito.Mockito.mock
|
||||
|
||||
import com.microsoft.hyperspace.index.HyperspaceSuite
|
||||
import com.microsoft.hyperspace.index.dataskipping.expressions._
|
||||
|
||||
class MinMaxSketchTest extends QueryTest with HyperspaceSuite {
|
||||
import spark.implicits._
|
||||
|
||||
val valueExtractor = AttrValueExtractor(Map.empty)
|
||||
|
||||
test("indexedColumns returns the indexed column.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
assert(sketch.indexedColumns === Seq("A"))
|
||||
}
|
||||
|
||||
test("referencedColumns returns the indexed column.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
assert(sketch.referencedColumns === Seq("A"))
|
||||
}
|
||||
|
||||
test("aggregateFunctions returns min and max aggregation functions.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val aggrs = sketch.aggregateFunctions.map(new Column(_))
|
||||
val data = Seq(1, -1, 10, 2, 4).toDF("A")
|
||||
checkAnswer(data.select(aggrs: _*), Seq((-1, 10)).toDF)
|
||||
}
|
||||
|
||||
test("toString returns a reasonable string.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
assert(sketch.toString === "MinMax(A)")
|
||||
}
|
||||
|
||||
test("Two sketches are equal if their columns are equal.") {
|
||||
assert(MinMaxSketch("A") === MinMaxSketch("A"))
|
||||
assert(MinMaxSketch("A") !== MinMaxSketch("a"))
|
||||
assert(MinMaxSketch("b") !== MinMaxSketch("B"))
|
||||
assert(MinMaxSketch("B") === MinMaxSketch("B"))
|
||||
}
|
||||
|
||||
test("MinMaxSketch is different from other sketches.") {
|
||||
val s1 = MinMaxSketch("A")
|
||||
val s2 = mock(classOf[Sketch])
|
||||
assert(s1 !== s2)
|
||||
}
|
||||
|
||||
test("hashCode is reasonably implemented.") {
|
||||
assert(MinMaxSketch("A").hashCode === MinMaxSketch("A").hashCode)
|
||||
assert(MinMaxSketch("A").hashCode !== MinMaxSketch("a").hashCode)
|
||||
}
|
||||
|
||||
test("convertPredicate converts EqualTo(<col>, <lit>).") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate = EqualTo(AttributeReference("A", IntegerType)(ExprId(0)), Literal(42))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal(42)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal(42))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts EqualTo(<lit>, <col>).") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate = EqualTo(Literal(42), AttributeReference("A", IntegerType)(ExprId(0)))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal(42)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal(42))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts EqualTo(<struct field access>, <lit>).") {
|
||||
val sketch = MinMaxSketch("A.C")
|
||||
val structAccess = GetStructField(
|
||||
AttributeReference("A", StructType(Seq(StructField("C", IntegerType))))(ExprId(0)),
|
||||
0)
|
||||
val predicate = EqualTo(structAccess, Literal(42))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(structAccess.transformUp {
|
||||
case attr: AttributeReference => attr.withExprId(ExpressionUtils.nullExprId)
|
||||
}),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal(42)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal(42))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts EqualTo(<nested struct field access>, <lit>).") {
|
||||
val sketch = MinMaxSketch("A.B.C")
|
||||
val structAccess = GetStructField(
|
||||
GetStructField(
|
||||
AttributeReference(
|
||||
"A",
|
||||
StructType(Seq(StructField("B", StructType(Seq(StructField("C", IntegerType)))))))(
|
||||
ExprId(0)),
|
||||
0),
|
||||
0)
|
||||
val predicate = EqualTo(structAccess, Literal(42))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(structAccess.transformUp {
|
||||
case attr: AttributeReference => attr.withExprId(ExpressionUtils.nullExprId)
|
||||
}),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal(42)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal(42))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts EqualTo(<col>, <lit>) - string type.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate =
|
||||
EqualTo(AttributeReference("A", StringType)(ExprId(0)), Literal.create("hello", StringType))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", StringType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal.create("hello", StringType)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal.create("hello", StringType))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts EqualTo(<col>, <lit>) - double type.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate =
|
||||
EqualTo(AttributeReference("A", StringType)(ExprId(0)), Literal(3.14, DoubleType))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", StringType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal(3.14, DoubleType)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal(3.14, DoubleType))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts LessThan.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate = LessThan(AttributeReference("A", IntegerType)(ExprId(0)), Literal(42))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(LessThan(sketchValues(0), Literal(42)))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts LessThan - string type.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate = LessThan(
|
||||
AttributeReference("A", StringType)(ExprId(0)),
|
||||
Literal.create("hello", StringType))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", StringType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(LessThan(sketchValues(0), Literal.create("hello", StringType)))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts LessThanOrEqual.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate = LessThanOrEqual(AttributeReference("A", IntegerType)(ExprId(0)), Literal(42))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(LessThanOrEqual(sketchValues(0), Literal(42)))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts GreaterThan.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate = GreaterThan(AttributeReference("A", IntegerType)(ExprId(0)), Literal(42))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(GreaterThan(sketchValues(1), Literal(42)))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts GreaterThanOrEqual.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate =
|
||||
GreaterThanOrEqual(AttributeReference("A", IntegerType)(ExprId(0)), Literal(42))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(GreaterThanOrEqual(sketchValues(1), Literal(42)))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts In.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate =
|
||||
In(AttributeReference("A", IntegerType)(ExprId(0)), Seq(Literal(42), Literal(23)))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
Or(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal(42)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal(42))),
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal(23)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal(23)))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate converts In - string type.") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate =
|
||||
In(
|
||||
AttributeReference("A", StringType)(ExprId(0)),
|
||||
Seq(Literal.create("hello", StringType), Literal.create("world", StringType)))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", StringType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = Some(
|
||||
Or(
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal.create("hello", StringType)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal.create("hello", StringType))),
|
||||
And(
|
||||
LessThanOrEqual(sketchValues(0), Literal.create("world", StringType)),
|
||||
GreaterThanOrEqual(sketchValues(1), Literal.create("world", StringType)))))
|
||||
assert(result === expected)
|
||||
}
|
||||
|
||||
test("convertPredicate does not convert Not(EqualTo(<col>, <lit>)).") {
|
||||
val sketch = MinMaxSketch("A")
|
||||
val predicate = Not(EqualTo(AttributeReference("A", IntegerType)(ExprId(0)), Literal(42)))
|
||||
val sketchValues = Seq(UnresolvedAttribute("min"), UnresolvedAttribute("max"))
|
||||
val nameMap = Map(ExprId(0) -> "A")
|
||||
val result = sketch.convertPredicate(
|
||||
predicate,
|
||||
Seq(AttributeReference("A", IntegerType)(ExpressionUtils.nullExprId)),
|
||||
sketchValues,
|
||||
nameMap,
|
||||
valueExtractor)
|
||||
val expected = None
|
||||
assert(result === expected)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,166 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.sketches
|
||||
|
||||
import org.apache.spark.sql.{Column, QueryTest}
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
import com.microsoft.hyperspace.index.HyperspaceSuite
|
||||
import com.microsoft.hyperspace.index.dataskipping.expressions._
|
||||
|
||||
class PartitionSketchTest extends QueryTest with HyperspaceSuite {
|
||||
import spark.implicits._
|
||||
|
||||
test("indexedColumns returns the indexed column.") {
|
||||
val sketch = PartitionSketch(Seq(("A", None)))
|
||||
assert(sketch.indexedColumns === Seq("A"))
|
||||
}
|
||||
|
||||
test("referencedColumns returns the indexed column.") {
|
||||
val sketch = PartitionSketch(Seq(("a", None), ("b", None)))
|
||||
assert(sketch.referencedColumns === Seq("a", "b"))
|
||||
}
|
||||
|
||||
test("aggregateFunctions returns first aggregation function.") {
|
||||
val sketch = PartitionSketch(Seq(("A", None)))
|
||||
val aggrs = sketch.aggregateFunctions.map(new Column(_))
|
||||
val data = Seq(1, 1, 1, 1, 1).toDF("A")
|
||||
checkAnswer(data.select(aggrs: _*), Seq(1).toDF)
|
||||
}
|
||||
|
||||
test("toString returns a human-readable string representation.") {
|
||||
val sketch = PartitionSketch(Seq(("A", None), ("B", None)))
|
||||
assert(sketch.toString === "Partition(A, B)")
|
||||
}
|
||||
|
||||
test("Two sketches are equal if their columns are equal.") {
|
||||
assert(PartitionSketch(Seq(("A", None))) === PartitionSketch(Seq(("A", None))))
|
||||
assert(
|
||||
PartitionSketch(Seq(("A", Some(LongType)))) !==
|
||||
PartitionSketch(Seq(("A", Some(IntegerType)))))
|
||||
}
|
||||
|
||||
test("hashCode is reasonably implemented.") {
|
||||
assert(
|
||||
PartitionSketch(Seq(("A", None))).hashCode === PartitionSketch(Seq(("A", None))).hashCode)
|
||||
assert(
|
||||
PartitionSketch(Seq(("A", Some(LongType)))).hashCode !==
|
||||
PartitionSketch(Seq(("A", Some(IntegerType)))).hashCode)
|
||||
}
|
||||
|
||||
val a = AttributeReference("A", IntegerType)(ExprId(0))
|
||||
val b = AttributeReference("B", IntegerType)(ExprId(1))
|
||||
val sketchA = UnresolvedAttribute("Partition_A")
|
||||
val sketchB = UnresolvedAttribute("Partition_B")
|
||||
|
||||
{
|
||||
val sketch = PartitionSketch(Seq(("A", Some(IntegerType))))
|
||||
val resolvedExprs = Seq(ExpressionUtils.normalize(a))
|
||||
val sketchValues = Seq(sketchA)
|
||||
val valueExtractor = AttrValueExtractor(Map(a -> sketchA))
|
||||
|
||||
test("convertPredicate converts EqualTo(<col>, <lit>).") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
EqualTo(a, Literal(42)),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === Some(EqualTo(sketchA, Literal(42))))
|
||||
}
|
||||
|
||||
test("convertPredicate does not convert non-deterministic expression.") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
EqualTo(a, Rand(42)),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === None)
|
||||
}
|
||||
|
||||
test("convertPredicate does not convert literal.") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
Literal(42),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === None)
|
||||
}
|
||||
|
||||
test("convertPredicate does not convert conjunction.") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
And(LessThan(a, Literal(42)), GreaterThan(a, Literal(23))),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === None)
|
||||
}
|
||||
|
||||
test("convertPredicate does not convert disjunction.") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
Or(LessThan(a, Literal(42)), GreaterThan(a, Literal(23))),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === None)
|
||||
}
|
||||
|
||||
test("convertPredicate does not convert predicate having subquery.") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
InSubquery(Seq(a), ListQuery(LocalRelation(a))),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === None)
|
||||
}
|
||||
|
||||
test("convertPredicate does not convert predicate having unknown attr.") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
EqualTo(a, b),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === None)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
val sketch = PartitionSketch(Seq(("A", Some(IntegerType)), ("B", Some(IntegerType))))
|
||||
val resolvedExprs = Seq(a, b).map(ExpressionUtils.normalize)
|
||||
val sketchValues = Seq(sketchA, sketchB)
|
||||
val valueExtractor = AttrValueExtractor(Map(a -> sketchA, b -> sketchB))
|
||||
|
||||
test("convertPredicate converts EqualTo(<col>, <col>).") {
|
||||
assert(
|
||||
sketch.convertPredicate(
|
||||
EqualTo(a, b),
|
||||
resolvedExprs,
|
||||
sketchValues,
|
||||
Map.empty,
|
||||
valueExtractor) === Some(EqualTo(sketchA, sketchB)))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,946 +0,0 @@
|
|||
/*
|
||||
* Copyright (2021) The Hyperspace Project Authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.microsoft.hyperspace.index.dataskipping.util
|
||||
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.types.{BooleanType, IntegerType}
|
||||
import org.mockito.Mockito.{mock, when}
|
||||
|
||||
import com.microsoft.hyperspace.index.HyperspaceSuite
|
||||
|
||||
class ExtractorsTest extends HyperspaceSuite {
|
||||
val a = AttributeReference("A", IntegerType)()
|
||||
val b = AttributeReference("B", BooleanType)()
|
||||
val zero = Literal(0, IntegerType)
|
||||
val one = Literal(1, IntegerType)
|
||||
val two = Literal(2, IntegerType)
|
||||
val trueLit = Literal(true, BooleanType)
|
||||
val falseLit = Literal(false, BooleanType)
|
||||
val nullInt = Literal(null, IntegerType)
|
||||
val nullBool = Literal(null, BooleanType)
|
||||
|
||||
val aMatcher = {
|
||||
val matcher = mock(classOf[ExprMatcher])
|
||||
when(matcher.apply(a)).thenReturn(true)
|
||||
matcher
|
||||
}
|
||||
val bMatcher = {
|
||||
val matcher = mock(classOf[ExprMatcher])
|
||||
when(matcher.apply(b)).thenReturn(true)
|
||||
matcher
|
||||
}
|
||||
val nonMatcher = mock(classOf[ExprMatcher])
|
||||
|
||||
// EqualToExtractor
|
||||
{
|
||||
val AEqualTo = EqualToExtractor(aMatcher)
|
||||
val NoneEqualTo = EqualToExtractor(nonMatcher)
|
||||
|
||||
test("EqualToExtractor matches EqualTo(<expr>, <lit>) if matcher(<expr>).") {
|
||||
val value = EqualTo(a, zero) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualTo(<expr>, null).") {
|
||||
val value = EqualTo(a, nullInt) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualTo(<expr>, <expr>).") {
|
||||
val value = EqualTo(a, a) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualTo(<expr>, <lit>) if !matcher(<expr>).") {
|
||||
val value = EqualTo(a, zero) match {
|
||||
case NoneEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor matches EqualTo(<lit>, <expr>) if matcher(<expr>).") {
|
||||
val value = EqualTo(zero, a) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualTo(null, <expr>).") {
|
||||
val value = EqualTo(nullInt, a) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualTo(<lit>, <expr>) if !matcher(<expr>).") {
|
||||
val value = EqualTo(zero, a) match {
|
||||
case NoneEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor matches EqualNullSafe(<expr>, <lit>) if matcher(<expr>).") {
|
||||
val value = EqualNullSafe(a, zero) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualNullSafe(<expr>, null).") {
|
||||
val value = EqualNullSafe(a, nullInt) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualNullSafe(<expr>, <lit>) if !matcher(<expr>).") {
|
||||
val value = EqualNullSafe(a, zero) match {
|
||||
case NoneEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor matches EqualNullSafe(<lit>, <expr>) if matcher(<expr>).") {
|
||||
val value = EqualNullSafe(zero, a) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualNullSafe(null, <expr>).") {
|
||||
val value = EqualNullSafe(nullInt, a) match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match EqualNullSafe(<lit>, <expr>) if !matcher(<expr>).") {
|
||||
val value = EqualNullSafe(zero, a) match {
|
||||
case NoneEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("EqualToExtractor does not match expressions other than EqualTo/EqualNullSafe.") {
|
||||
val value = zero match {
|
||||
case AEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
}
|
||||
|
||||
// LessThanExtractor
|
||||
{
|
||||
val ALessThan = LessThanExtractor(aMatcher)
|
||||
val NoneLessThan = LessThanExtractor(nonMatcher)
|
||||
|
||||
test("LessThanExtractor matches LessThan(<expr>, <lit>) if matcher(<expr>).") {
|
||||
val value = LessThan(a, zero) match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match LessThan(<expr>, null).") {
|
||||
val value = LessThan(a, nullInt) match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match LessThan(<expr>, <expr>).") {
|
||||
val value = LessThan(a, a) match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match LessThan(<expr>, <lit>) if !matcher(<expr>).") {
|
||||
val value = LessThan(a, zero) match {
|
||||
case NoneLessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match LessThan(<lit>, <expr>).") {
|
||||
val value = LessThan(zero, a) match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor matches GreaterThan(<lit>, <expr>) if matcher(<expr>).") {
|
||||
val value = GreaterThan(zero, a) match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match GreaterThan(null, <expr>).") {
|
||||
val value = GreaterThan(nullInt, a) match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match GreaterThan(<lit>, <expr>) if !matcher(<expr>).") {
|
||||
val value = GreaterThan(zero, a) match {
|
||||
case NoneLessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match GreaterThan(<expr>, <lit>).") {
|
||||
val value = GreaterThan(a, zero) match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanExtractor does not match expressions other than LessThan.") {
|
||||
val value = zero match {
|
||||
case ALessThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
}
|
||||
|
||||
// LessThanOrEqualToExtractor
|
||||
{
|
||||
val ALessThanOrEqualTo = LessThanOrEqualToExtractor(aMatcher)
|
||||
val NoneLessThanOrEqualTo = LessThanOrEqualToExtractor(nonMatcher)
|
||||
|
||||
test(
|
||||
"LessThanOrEqualToExtractor matches LessThanOrEqual(<expr>, <lit>) if matcher(<expr>).") {
|
||||
val value = LessThanOrEqual(a, zero) match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("LessThanOrEqualToExtractor does not match LessThanOrEqual(<expr>, null).") {
|
||||
val value = LessThanOrEqual(a, nullInt) match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanOrEqualToExtractor does not match LessThanOrEqual(<expr>, <expr>).") {
|
||||
val value = LessThanOrEqual(a, a) match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test(
|
||||
"LessThanOrEqualToExtractor does not match LessThanOrEqual(<expr>, <lit>) " +
|
||||
"if !matcher(<expr>).") {
|
||||
val value = LessThanOrEqual(a, zero) match {
|
||||
case NoneLessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanOrEqualToExtractor does not match LessThanOrEqual(<lit>, <expr>).") {
|
||||
val value = LessThanOrEqual(zero, a) match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test(
|
||||
"LessThanOrEqualToExtractor matches GreaterThanOrEqual(<lit>, <expr>) if matcher(<expr>).") {
|
||||
val value = GreaterThanOrEqual(zero, a) match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("LessThanOrEqualToExtractor does not match GreaterThanOrEqual(null, <expr>).") {
|
||||
val value = GreaterThanOrEqual(nullInt, a) match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test(
|
||||
"LessThanOrEqualToExtractor does not match GreaterThanOrEqual(<lit>, <expr>) " +
|
||||
"if !matcher(<expr>).") {
|
||||
val value = GreaterThanOrEqual(zero, a) match {
|
||||
case NoneLessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanOrEqualToExtractor does not match GreaterThanOrEqual(<expr>, <lit>).") {
|
||||
val value = GreaterThanOrEqual(a, zero) match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("LessThanOrEqualToExtractor does not match expressions other than LessThanOrEqual.") {
|
||||
val value = zero match {
|
||||
case ALessThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
}
|
||||
|
||||
// GreaterThanExtractor
|
||||
{
|
||||
val AGreaterThan = GreaterThanExtractor(aMatcher)
|
||||
val NoneGreaterThan = GreaterThanExtractor(nonMatcher)
|
||||
|
||||
test("GreaterThanExtractor matches GreaterThan(<expr>, <lit>) if matcher(<expr>).") {
|
||||
val value = GreaterThan(a, zero) match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match GreaterThan(<expr>, null).") {
|
||||
val value = GreaterThan(a, nullInt) match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match GreaterThan(<expr>, <expr>).") {
|
||||
val value = GreaterThan(a, a) match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match GreaterThan(<expr>, <lit>) if !matcher(<expr>).") {
|
||||
val value = GreaterThan(a, zero) match {
|
||||
case NoneGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match GreaterThan(<lit>, <expr>).") {
|
||||
val value = GreaterThan(zero, a) match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor matches LessThan(<lit>, <expr>) if matcher(<expr>).") {
|
||||
val value = LessThan(zero, a) match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match LessThan(null, <expr>).") {
|
||||
val value = LessThan(nullInt, a) match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match LessThan(<lit>, <expr>) if !matcher(<expr>).") {
|
||||
val value = LessThan(zero, a) match {
|
||||
case NoneGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match LessThan(<expr>, <lit>).") {
|
||||
val value = LessThan(a, zero) match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanExtractor does not match expressions other than GreaterThan.") {
|
||||
val value = zero match {
|
||||
case AGreaterThan(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
}
|
||||
|
||||
// GreaterThanOrEqualToExtractor
|
||||
{
|
||||
val AGreaterThanOrEqualTo = GreaterThanOrEqualToExtractor(aMatcher)
|
||||
val NoneGreaterThanOrEqualTo = GreaterThanOrEqualToExtractor(nonMatcher)
|
||||
|
||||
test(
|
||||
"GreaterThanOrEqualToExtractor matches GreaterThanOrEqual(<expr>, <lit>) " +
|
||||
"if matcher(<expr>).") {
|
||||
val value = GreaterThanOrEqual(a, zero) match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("GreaterThanOrEqualToExtractor does not match GreaterThanOrEqual(<expr>, null).") {
|
||||
val value = GreaterThanOrEqual(a, nullInt) match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanOrEqualToExtractor does not match GreaterThanOrEqual(<expr>, <expr>).") {
|
||||
val value = GreaterThanOrEqual(a, a) match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test(
|
||||
"GreaterThanOrEqualToExtractor does not match GreaterThanOrEqual(<expr>, <lit>) " +
|
||||
"if !matcher(<expr>).") {
|
||||
val value = GreaterThanOrEqual(a, zero) match {
|
||||
case NoneGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanOrEqualToExtractor does not match GreaterThanOrEqual(<lit>, <expr>).") {
|
||||
val value = GreaterThanOrEqual(zero, a) match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test(
|
||||
"GreaterThanOrEqualToExtractor matches LessThanOrEqual(<lit>, <expr>) if matcher(<expr>).") {
|
||||
val value = LessThanOrEqual(zero, a) match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(zero))
|
||||
}
|
||||
|
||||
test("GreaterThanOrEqualToExtractor does not match LessThanOrEqual(null, <expr>).") {
|
||||
val value = LessThanOrEqual(nullInt, a) match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test(
|
||||
"GreaterThanOrEqualToExtractor does not match LessThanOrEqual(<lit>, <expr>) " +
|
||||
"if !matcher(<expr>).") {
|
||||
val value = LessThanOrEqual(zero, a) match {
|
||||
case NoneGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("GreaterThanOrEqualToExtractor does not match LessThanOrEqual(<expr>, <lit>).") {
|
||||
val value = LessThanOrEqual(a, zero) match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test(
|
||||
"GreaterThanOrEqualToExtractor does not match expressions other than GreaterThanOrEqual.") {
|
||||
val value = zero match {
|
||||
case AGreaterThanOrEqualTo(value) => Some(value)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
}
|
||||
|
||||
// IsNullExtractor
|
||||
{
|
||||
val AIsNull = IsNullExtractor(aMatcher)
|
||||
val NoneIsNull = IsNullExtractor(nonMatcher)
|
||||
|
||||
test("IsNullExtractor matches IsNull(<expr>) if matcher(<expr>).") {
|
||||
val value = IsNull(a) match {
|
||||
case AIsNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsNullExtractor does not match IsNull(<expr>) if !matcher(<expr>).") {
|
||||
val value = IsNull(a) match {
|
||||
case NoneIsNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsNullExtractor matches EqualNullSafe(<expr>, null).") {
|
||||
val value = EqualNullSafe(a, nullInt) match {
|
||||
case AIsNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsNullExtractor does not match EqualNullSafe(<expr>, <lit>).") {
|
||||
val value = EqualNullSafe(a, zero) match {
|
||||
case AIsNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsNullExtractor matches EqualNullSafe(null, <expr>).") {
|
||||
val value = EqualNullSafe(nullInt, a) match {
|
||||
case AIsNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsNullExtractor does not match EqualNullSafe(<lit>, <expr>).") {
|
||||
val value = EqualNullSafe(zero, a) match {
|
||||
case AIsNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsNullExtractor does not match expressions other than IsNull/EqualNullSafe.") {
|
||||
val value = zero match {
|
||||
case AIsNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
}
|
||||
|
||||
// IsNotNullExtractor
|
||||
{
|
||||
val AIsNotNull = IsNotNullExtractor(aMatcher)
|
||||
val NoneIsNotNull = IsNotNullExtractor(nonMatcher)
|
||||
|
||||
test("IsNotNullExtractor matches IsNotNull(<expr>) if matcher(<expr>).") {
|
||||
val value = IsNotNull(a) match {
|
||||
case AIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor does not match IsNotNull(<expr>) if !matcher(<expr>).") {
|
||||
val value = IsNotNull(a) match {
|
||||
case NoneIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches Not(IsNull(<expr>)) if matcher(<expr>).") {
|
||||
val value = Not(IsNull(a)) match {
|
||||
case AIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches Not(IsNull(<expr>)) if !matcher(<expr>).") {
|
||||
val value = Not(IsNull(a)) match {
|
||||
case NoneIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches Not(EqualNullSafe(<expr>, null)).") {
|
||||
val value = Not(EqualNullSafe(a, nullInt)) match {
|
||||
case AIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor does not match Not(EqualNullSafe(<expr>, <lit>)).") {
|
||||
val value = Not(EqualNullSafe(a, zero)) match {
|
||||
case AIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor matches Not(EqualNullSafe(null, <expr>)).") {
|
||||
val value = Not(EqualNullSafe(nullInt, a)) match {
|
||||
case AIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsNotNullExtractor does not match Not(EqualNullSafe(<lit>, <expr>)).") {
|
||||
val value = Not(EqualNullSafe(zero, a)) match {
|
||||
case AIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test(
|
||||
"IsNotNullExtractor does not match expressions other than IsNotNull/Not(EqualNullSafe).") {
|
||||
val value = zero match {
|
||||
case AIsNotNull() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
}
|
||||
|
||||
// IsTrueExtractor
|
||||
{
|
||||
val AIsTrue = IsTrueExtractor(aMatcher)
|
||||
val BIsTrue = IsTrueExtractor(bMatcher)
|
||||
val NoneIsTrue = IsTrueExtractor(nonMatcher)
|
||||
|
||||
test("IsTrueExtractor matches <expr> if matcher(<expr>).") {
|
||||
val value = b match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match <expr> if !matcher(<expr>).") {
|
||||
val value = b match {
|
||||
case NoneIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match <expr> if type is not boolean.") {
|
||||
val value = a match {
|
||||
case AIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor matches EqualTo(<expr>, true) if matcher(<expr>).") {
|
||||
val value = EqualTo(b, trueLit) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match EqualTo(<expr>, false).") {
|
||||
val value = EqualTo(b, falseLit) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor matches EqualTo(true, <expr>) if matcher(<expr>).") {
|
||||
val value = EqualTo(trueLit, b) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match EqualTo(false, <expr>).") {
|
||||
val value = EqualTo(falseLit, b) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor matches EqualNullSafe(<expr>, true) if matcher(<expr>).") {
|
||||
val value = EqualNullSafe(b, trueLit) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match EqualNullSafe(<expr>, false).") {
|
||||
val value = EqualNullSafe(b, falseLit) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor matches EqualNullSafe(true, <expr>) if matcher(<expr>).") {
|
||||
val value = EqualNullSafe(trueLit, b) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match EqualNullSafe(false, <expr>).") {
|
||||
val value = EqualNullSafe(falseLit, b) match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsTrueExtractor does not match <lit>.") {
|
||||
val value = zero match {
|
||||
case BIsTrue() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
}
|
||||
|
||||
// IsFalseExtractor
|
||||
{
|
||||
val BIsFalse = IsFalseExtractor(bMatcher)
|
||||
val NoneIsFalse = IsFalseExtractor(nonMatcher)
|
||||
|
||||
test("IsFalseExtractor matches Not(<expr>) if matcher(<expr>).") {
|
||||
val value = Not(b) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match Not(<expr>) if !matcher(<expr>).") {
|
||||
val value = Not(b) match {
|
||||
case NoneIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor matches EqualTo(<expr>, false) if matcher(<expr>).") {
|
||||
val value = EqualTo(b, falseLit) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match EqualTo(<expr>, true).") {
|
||||
val value = EqualTo(b, trueLit) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor matches EqualTo(false, <expr>) if matcher(<expr>).") {
|
||||
val value = EqualTo(falseLit, b) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match EqualTo(true, <expr>).") {
|
||||
val value = EqualTo(trueLit, b) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor matches EqualNullSafe(<expr>, false) if matcher(<expr>).") {
|
||||
val value = EqualNullSafe(b, falseLit) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match EqualNullSafe(<expr>, true).") {
|
||||
val value = EqualNullSafe(b, trueLit) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor matches EqualNullSafe(false, <expr>) if matcher(<expr>).") {
|
||||
val value = EqualNullSafe(falseLit, b) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match EqualNullSafe(true, <expr>).") {
|
||||
val value = EqualNullSafe(trueLit, b) match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
|
||||
test("IsFalseExtractor does not match <lit>.") {
|
||||
val value = zero match {
|
||||
case BIsFalse() => true
|
||||
case _ => false
|
||||
}
|
||||
assert(!value)
|
||||
}
|
||||
}
|
||||
|
||||
// InExtractor
|
||||
{
|
||||
val AIn = InExtractor(aMatcher)
|
||||
val NoneIn = InExtractor(nonMatcher)
|
||||
|
||||
test("InExtractor matches In(<expr>, <lit>*) if matcher(<expr>).") {
|
||||
val value = In(a, Seq(zero, one, two)) match {
|
||||
case AIn(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(Seq(zero, one, two)))
|
||||
}
|
||||
|
||||
test("InExtractor matches In(<expr>, <lit>*) if !matcher(<expr>).") {
|
||||
val value = In(a, Seq(zero, one, two)) match {
|
||||
case NoneIn(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("InExtractor does not match In(<expr>, <lit>*) if <lit>* is empty.") {
|
||||
val value = In(a, Nil) match {
|
||||
case AIn(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("InExtractor matches In(<expr>, <lit>*) if some <lit>s are null.") {
|
||||
val value = In(a, Seq(zero, nullInt, nullInt, two)) match {
|
||||
case AIn(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(Seq(zero, two)))
|
||||
}
|
||||
|
||||
test("InExtractor does not match In(<expr>, <expr>*).") {
|
||||
val value = In(a, Seq(zero, two, a)) match {
|
||||
case AIn(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("InExtractor does not match other than In.") {
|
||||
val value = a match {
|
||||
case AIn(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
}
|
||||
|
||||
// InSetExtractor
|
||||
{
|
||||
val AInSet = InSetExtractor(aMatcher)
|
||||
val NoneInSet = InSetExtractor(nonMatcher)
|
||||
|
||||
test("InSetExtractor matches InSet(<expr>, <set>) if matcher(<expr>).") {
|
||||
val value = InSet(a, Set(0, 1, 2)) match {
|
||||
case AInSet(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === Some(Set(0, 1, 2)))
|
||||
}
|
||||
|
||||
test("InSetExtractor matches InSet(<expr>, <lit>*) if !matcher(<expr>).") {
|
||||
val value = InSet(a, Set(0, 1, 2)) match {
|
||||
case NoneInSet(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
|
||||
test("InSetExtractor does not match other than InSet.") {
|
||||
val value = a match {
|
||||
case AInSet(values) => Some(values)
|
||||
case _ => None
|
||||
}
|
||||
assert(value === None)
|
||||
}
|
||||
}
|
||||
}
|
Загрузка…
Ссылка в новой задаче