Skip to content

Commit

Permalink
[SPARK-32268][SQL] Row-level Runtime Filtering
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR proposes row-level runtime filters in Spark to reduce intermediate data volume for operators like shuffle, join and aggregate, and hence improve performance. We propose two mechanisms to do this: semi-join filters or bloom filters, and both mechanisms are proposed to co-exist side-by-side behind feature configs.
[Design Doc](https://docs.google.com/document/d/16IEuyLeQlubQkH8YuVuXWKo2-grVIoDJqQpHZrE7q04/edit?usp=sharing) with more details.

### Why are the changes needed?

With Semi-Join, we see 9 queries improve for the TPC DS 3TB benchmark, and no regressions.
With Bloom Filter, we see 10 queries improve for the TPC DS 3TB benchmark, and no regressions.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added tests

Closes apache#35789 from somani/rf.

Lead-authored-by: Abhishek Somani <[email protected]>
Co-authored-by: Abhishek Somani <[email protected]>
Co-authored-by: Yuming Wang <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
  • Loading branch information
3 people authored and cloud-fan committed Mar 23, 2022
1 parent 4e60638 commit 1f4e4c8
Show file tree
Hide file tree
Showing 14 changed files with 1,432 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,13 @@ int getVersionNumber() {
*/
public abstract void writeTo(OutputStream out) throws IOException;

/**
* @return the number of set bits in this {@link BloomFilter}.
*/
public long cardinality() {
throw new UnsupportedOperationException("Not implemented");
}

/**
* Reads in a {@link BloomFilter} from an input stream. It is the caller's responsibility to close
* the stream.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,6 +207,11 @@ public BloomFilter intersectInPlace(BloomFilter other) throws IncompatibleMergeE
return this;
}

@Override
public long cardinality() {
return this.bits.cardinality();
}

private BloomFilterImpl checkCompatibilityForMerge(BloomFilter other)
throws IncompatibleMergeException {
// Duplicates the logic of `isCompatible` here to provide better error message.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.expressions

import java.io.ByteArrayInputStream

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, JavaCode, TrueLiteral}
import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
import org.apache.spark.sql.types._
import org.apache.spark.util.sketch.BloomFilter

/**
* An internal scalar function that returns the membership check result (either true or false)
* for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
* Not that since the function is "might contain", always returning true regardless is not
* wrong.
* Note that this expression requires that `bloomFilterExpression` is either a constant value or
* an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
*
* @param bloomFilterExpression the Binary data of Bloom filter.
* @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
*/
case class BloomFilterMightContain(
bloomFilterExpression: Expression,
valueExpression: Expression) extends BinaryExpression {

override def nullable: Boolean = true
override def left: Expression = bloomFilterExpression
override def right: Expression = valueExpression
override def prettyName: String = "might_contain"
override def dataType: DataType = BooleanType

override def checkInputDataTypes(): TypeCheckResult = {
(left.dataType, right.dataType) match {
case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
(BinaryType, LongType) =>
bloomFilterExpression match {
case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
TypeCheckResult.TypeCheckSuccess
case _ =>
TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
"should be either a constant value or a scalar subquery expression")
}
case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
}
}

override protected def withNewChildrenInternal(
newBloomFilterExpression: Expression,
newValueExpression: Expression): BloomFilterMightContain =
copy(bloomFilterExpression = newBloomFilterExpression,
valueExpression = newValueExpression)

// The bloom filter created from `bloomFilterExpression`.
@transient private lazy val bloomFilter = {
val bytes = bloomFilterExpression.eval().asInstanceOf[Array[Byte]]
if (bytes == null) null else deserialize(bytes)
}

override def eval(input: InternalRow): Any = {
if (bloomFilter == null) {
null
} else {
val value = valueExpression.eval(input)
if (value == null) null else bloomFilter.mightContainLong(value.asInstanceOf[Long])
}
}

override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
if (bloomFilter == null) {
ev.copy(isNull = TrueLiteral, value = JavaCode.defaultLiteral(dataType))
} else {
val bf = ctx.addReferenceObj("bloomFilter", bloomFilter, classOf[BloomFilter].getName)
val valueEval = valueExpression.genCode(ctx)
ev.copy(code = code"""
${valueEval.code}
boolean ${ev.isNull} = ${valueEval.isNull};
${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
if (!${ev.isNull}) {
${ev.value} = $bf.mightContainLong((Long)${valueEval.value});
}""")
}
}

final def deserialize(bytes: Array[Byte]): BloomFilter = {
val in = new ByteArrayInputStream(bytes)
val bloomFilter = BloomFilter.readFrom(in)
in.close()
bloomFilter
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,179 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.expressions.aggregate

import java.io.ByteArrayInputStream
import java.io.ByteArrayOutputStream

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.trees.TernaryLike
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.util.sketch.BloomFilter

/**
* An internal aggregate function that creates a Bloom filter from input values.
*
* @param child Child expression of Long values for creating a Bloom filter.
* @param estimatedNumItemsExpression The number of estimated distinct items (optional).
* @param numBitsExpression The number of bits to use (optional).
*/
case class BloomFilterAggregate(
child: Expression,
estimatedNumItemsExpression: Expression,
numBitsExpression: Expression,
override val mutableAggBufferOffset: Int,
override val inputAggBufferOffset: Int)
extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {

def this(child: Expression, estimatedNumItemsExpression: Expression,
numBitsExpression: Expression) = {
this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
}

def this(child: Expression, estimatedNumItemsExpression: Expression) = {
this(child, estimatedNumItemsExpression,
// 1 byte per item.
Multiply(estimatedNumItemsExpression, Literal(8L)))
}

def this(child: Expression) = {
this(child, Literal(SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_EXPECTED_NUM_ITEMS)),
Literal(SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_NUM_BITS)))
}

override def checkInputDataTypes(): TypeCheckResult = {
(first.dataType, second.dataType, third.dataType) match {
case (_, NullType, _) | (_, _, NullType) =>
TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
case (LongType, LongType, LongType) =>
if (!estimatedNumItemsExpression.foldable) {
TypeCheckFailure("The estimated number of items provided must be a constant literal")
} else if (estimatedNumItems <= 0L) {
TypeCheckFailure("The estimated number of items must be a positive value " +
s" (current value = $estimatedNumItems)")
} else if (!numBitsExpression.foldable) {
TypeCheckFailure("The number of bits provided must be a constant literal")
} else if (numBits <= 0L) {
TypeCheckFailure("The number of bits must be a positive value " +
s" (current value = $numBits)")
} else {
require(estimatedNumItems <=
SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))
require(numBits <= SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))
TypeCheckSuccess
}
case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
s"arguments, but it's [${first.dataType.catalogString}, " +
s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
}
}
override def nullable: Boolean = true

override def dataType: DataType = BinaryType

override def prettyName: String = "bloom_filter_agg"

// Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
private lazy val estimatedNumItems: Long =
Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))

// Mark as lazy so that `numBits` is not evaluated during tree transformation.
private lazy val numBits: Long =
Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))

override def first: Expression = child

override def second: Expression = estimatedNumItemsExpression

override def third: Expression = numBitsExpression

override protected def withNewChildrenInternal(
newChild: Expression,
newEstimatedNumItemsExpression: Expression,
newNumBitsExpression: Expression): BloomFilterAggregate = {
copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
numBitsExpression = newNumBitsExpression)
}

override def createAggregationBuffer(): BloomFilter = {
BloomFilter.create(estimatedNumItems, numBits)
}

override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
val value = child.eval(inputRow)
// Ignore null values.
if (value == null) {
return buffer
}
buffer.putLong(value.asInstanceOf[Long])
buffer
}

override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
buffer.mergeInPlace(other)
}

override def eval(buffer: BloomFilter): Any = {
if (buffer.cardinality() == 0) {
// There's no set bit in the Bloom filter and hence no not-null value is processed.
return null
}
serialize(buffer)
}

override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
copy(mutableAggBufferOffset = newOffset)

override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
copy(inputAggBufferOffset = newOffset)

override def serialize(obj: BloomFilter): Array[Byte] = {
BloomFilterAggregate.serialize(obj)
}

override def deserialize(bytes: Array[Byte]): BloomFilter = {
BloomFilterAggregate.deserialize(bytes)
}
}

object BloomFilterAggregate {
final def serialize(obj: BloomFilter): Array[Byte] = {
// BloomFilterImpl.writeTo() writes 2 integers (version number and num hash functions), hence
// the +8
val size = (obj.bitSize() / 8) + 8
require(size <= Integer.MAX_VALUE, s"actual number of bits is too large $size")
val out = new ByteArrayOutputStream(size.intValue())
obj.writeTo(out)
out.close()
out.toByteArray
}

final def deserialize(bytes: Array[Byte]): BloomFilter = {
val in = new ByteArrayInputStream(bytes)
val bloomFilter = BloomFilter.readFrom(in)
in.close()
bloomFilter
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -360,6 +360,8 @@ case class Invoke(

lazy val argClasses = ScalaReflection.expressionJavaClasses(arguments)

final override val nodePatterns: Seq[TreePattern] = Seq(INVOKE)

override def nullable: Boolean = targetObject.nullable || needNullCheck || returnNullable
override def children: Seq[Expression] = targetObject +: arguments
override lazy val deterministic: Boolean = isDeterministic && arguments.forall(_.deterministic)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -287,6 +287,22 @@ trait PredicateHelper extends AliasHelper with Logging {
}
}
}

/**
* Returns whether an expression is likely to be selective
*/
def isLikelySelective(e: Expression): Boolean = e match {
case Not(expr) => isLikelySelective(expr)
case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
case _: StringRegexExpression => true
case _: BinaryComparison => true
case _: In | _: InSet => true
case _: StringPredicate => true
case BinaryPredicate(_) => true
case _: MultiLikeBase => true
case _ => false
}
}

@ExpressionDescription(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure,
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.trees.BinaryLike
import org.apache.spark.sql.catalyst.trees.TreePattern.{LIKE_FAMLIY, TreePattern}
import org.apache.spark.sql.catalyst.trees.TreePattern.{LIKE_FAMLIY, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, TreePattern}
import org.apache.spark.sql.catalyst.util.{GenericArrayData, StringUtils}
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -627,6 +627,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio
@transient private var lastReplacementInUTF8: UTF8String = _
// result buffer write by Matcher
@transient private lazy val result: StringBuffer = new StringBuffer
final override val nodePatterns: Seq[TreePattern] = Seq(REGEXP_REPLACE)

override def nullSafeEval(s: Any, p: Any, r: Any, i: Any): Any = {
if (!p.equals(lastRegex)) {
Expand Down Expand Up @@ -751,6 +752,8 @@ abstract class RegExpExtractBase
// last regex pattern, we cache it for performance concern
@transient private var pattern: Pattern = _

final override val nodePatterns: Seq[TreePattern] = Seq(REGEXP_EXTRACT_FAMILY)

override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType)
override def first: Expression = subject
override def second: Expression = regexp
Expand Down
Loading

0 comments on commit 1f4e4c8

Please sign in to comment.