Skip to content

Commit

Permalink
feat(prometheus): relax length limit for pipe-only EqualsRegex filters (
Browse files Browse the repository at this point in the history
  • Loading branch information
alextheimer authored Oct 23, 2023
1 parent 52852fb commit 24b0ccb
Show file tree
Hide file tree
Showing 8 changed files with 210 additions and 23 deletions.
3 changes: 3 additions & 0 deletions core/src/main/resources/filodb-defaults.conf
Original file line number Diff line number Diff line change
Expand Up @@ -321,6 +321,9 @@ filodb {
# If true, an exception is thrown if any ExecPlan::execute result size is greater than result-byte-limit.
enforce-result-byte-limit = false

# Remove this config to always enforce the usual limit.
relaxed-pipe-only-equals-regex-limit = 10000

# Minimum step required for a query
min-step = 5 seconds

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ import filodb.core.binaryrecord2.MapItemConsumer
import filodb.core.memstore.ratelimit.CardinalityTracker
import filodb.core.metadata.Column.ColumnType.{MapColumn, StringColumn}
import filodb.core.metadata.PartitionSchema
import filodb.core.query.{ColumnFilter, Filter}
import filodb.core.query.{ColumnFilter, Filter, QueryUtils}
import filodb.core.query.Filter._
import filodb.memory.{BinaryRegionLarge, UTF8StringMedium, UTF8StringShort}
import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String => UTF8Str}
Expand Down Expand Up @@ -828,9 +828,6 @@ class PartKeyLuceneIndex(ref: DatasetRef,
logger.info(s"Refreshed index searchers to make reads consistent for dataset=$ref shard=$shardNum")
}

val regexChars = Array('.', '?', '+', '*', '|', '{', '}', '[', ']', '(', ')', '"', '\\')
val regexCharsMinusPipe = (regexChars.toSet - '|').toArray

// scalastyle:off method.length
private def leafFilter(column: String, filter: Filter): Query = {

Expand All @@ -848,14 +845,14 @@ class PartKeyLuceneIndex(ref: DatasetRef,
} else if (regex.replaceAll("\\.\\*", "") == "") {
// if label=~".*" then match all docs since promQL matches .* with absent label too
new MatchAllDocsQuery
} else if (regex.forall(c => !regexChars.contains(c))) {
} else if (!QueryUtils.containsRegexChars(regex)) {
// if all regex special chars absent, then treat like Equals
equalsQuery(regex)
} else if (regex.forall(c => !regexCharsMinusPipe.contains(c))) {
} else if (QueryUtils.containsPipeOnlyRegex(regex)) {
// if pipe is only regex special char present, then convert to IN query
new TermInSetQuery(column, regex.split('|').map(t => new BytesRef(t)): _*)
} else if (regex.endsWith(".*") && regex.length > 2 &&
regex.dropRight(2).forall(c => !regexChars.contains(c))) {
!QueryUtils.containsRegexChars(regex.dropRight(2))) {
// if suffix is .* and no regex special chars present in non-empty prefix, then use prefix query
new PrefixQuery(new Term(column, regex.dropRight(2)))
} else {
Expand Down
26 changes: 26 additions & 0 deletions core/src/main/scala/filodb.core/query/QueryUtils.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
package filodb.core.query

/**
* Storage for utility functions.
*/
object QueryUtils {
val REGEX_CHARS = Array('.', '?', '+', '*', '|', '{', '}', '[', ']', '(', ')', '"', '\\')

private val regexCharsMinusPipe = (REGEX_CHARS.toSet - '|').toArray

/**
* Returns true iff the argument string contains any special regex chars.
*/
def containsRegexChars(str: String): Boolean = {
str.exists(REGEX_CHARS.contains(_))
}

/**
* Returns true iff the argument string contains no special regex
* characters except the pipe character ('|').
* True is also returned when the string contains no pipes.
*/
def containsPipeOnlyRegex(str: String): Boolean = {
str.forall(!regexCharsMinusPipe.contains(_))
}
}
59 changes: 59 additions & 0 deletions core/src/test/scala/filodb.core/query/QueryUtilsSpec.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
package filodb.core.query

import org.scalatest.funspec.AnyFunSpec
import org.scalatest.matchers.must.Matchers
import org.scalatest.matchers.should.Matchers.convertToAnyShouldWrapper

class QueryUtilsSpec extends AnyFunSpec with Matchers{
it("should correctly identify regex chars") {
val testsNoRegex = Seq(
"",
",",
"no regex-1234!@#%&_,/`~=<>';:"
)
// includes one test for each single regex char
val testsRegex = QueryUtils.REGEX_CHARS.map(c => c.toString) ++ Seq(
"\\",
"\\\\",
"foo\\.bar", // escape chars don't affect the result (at least for now).
"foo\\\\.bar",
"foo|bar",
"foo\\|bar",
".foo\\|bar"
)
for (test <- testsNoRegex) {
QueryUtils.containsRegexChars(test) shouldEqual false
}
for (test <- testsRegex) {
QueryUtils.containsRegexChars(test) shouldEqual true
}
}

it ("should correctly identify non-pipe regex chars") {
val testsPipeOnly = Seq(
"",
"|",
"||||",
"a|b|c|d|e",
"foobar-1|2|34!@|#%&_,|/`~=<>|';:",
// NOTE: some regex chars are missing from QueryUtils.REGEX_CHARS.
// This is intentional to preserve existing behavior.
"^foo|bar$"
)
// includes one test for each single non-pipe regex char
val testsNonPipeOnly = QueryUtils.REGEX_CHARS.filter(c => c != '|').map(c => c.toString) ++ Seq(
"\\",
"\\\\",
"foo\\|bar", // escape chars don't affect the result (at least for now).
"foo\\\\|bar",
"foo.bar.baz|",
"!@#$%^&*()_+{}[];':\""
)
for (test <- testsPipeOnly) {
QueryUtils.containsPipeOnlyRegex(test) shouldEqual true
}
for (test <- testsNonPipeOnly) {
QueryUtils.containsPipeOnlyRegex(test) shouldEqual false
}
}
}
24 changes: 20 additions & 4 deletions prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3,14 +3,16 @@ package filodb.prometheus.ast
import scala.util.Try

import filodb.core.{query, GlobalConfig}
import filodb.core.query.{ColumnFilter, RangeParams}
import filodb.core.query.{ColumnFilter, QueryUtils, RangeParams}
import filodb.prometheus.parse.Parser
import filodb.query._

object Vectors {
val PromMetricLabel = "__name__"
val TypeLabel = "_type_"
val BucketFilterLabel = "_bucket_"
val conf = GlobalConfig.systemConfig
val queryConfig = conf.getConfig("filodb.query")
}

object WindowConstants {
Expand Down Expand Up @@ -265,9 +267,23 @@ sealed trait Vector extends Expression {
case NotRegexMatch => require(labelValue.length <= Parser.REGEX_MAX_LEN,
s"Regular expression filters should be <= ${Parser.REGEX_MAX_LEN} characters")
ColumnFilter(labelMatch.label, query.Filter.NotEqualsRegex(labelValue))
case RegexMatch => require(labelValue.length <= Parser.REGEX_MAX_LEN,
s"Regular expression filters should be <= ${Parser.REGEX_MAX_LEN} characters")
ColumnFilter(labelMatch.label, query.Filter.EqualsRegex(labelValue))
case RegexMatch =>
// Relax the length limit only for matchers that contain at most the "|" special character.
val shouldRelax = queryConfig.hasPath("relaxed-pipe-only-equals-regex-limit") &&
QueryUtils.containsPipeOnlyRegex(labelValue)
if (shouldRelax) {
val limit = queryConfig.getInt("relaxed-pipe-only-equals-regex-limit");
require(labelValue.length <= limit,
s"Regular expression filters should be <= $limit characters " +
s"when no special characters except '|' are used. " +
s"Violating filter is: ${labelMatch.label}=$labelValue")
} else {
require(labelValue.length <= Parser.REGEX_MAX_LEN,
s"Regular expression filters should be <= ${Parser.REGEX_MAX_LEN} characters " +
s"when non-`|` special characters are used. " +
s"Violating filter is: ${labelMatch.label}=$labelValue")
}
ColumnFilter(labelMatch.label, query.Filter.EqualsRegex(labelValue))
case NotEqual(false) => ColumnFilter(labelMatch.label, query.Filter.NotEquals(labelValue))
case other: Any => throw new IllegalArgumentException(s"Unknown match operator $other")
}
Expand Down
28 changes: 22 additions & 6 deletions prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ package filodb.prometheus.parse
import com.typesafe.scalalogging.StrictLogging

import filodb.core.GlobalConfig
import filodb.core.query.{ColumnFilter, Filter, QueryConfig}
import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryUtils}
import filodb.prometheus.ast._
import filodb.query.{LabelValues, LogicalPlan}

Expand All @@ -19,9 +19,11 @@ object Parser extends StrictLogging {
case object Antlr extends Mode
case object Shadow extends Mode

private val conf = GlobalConfig.systemConfig
private val queryConf = conf.getConfig("filodb.query")

val mode: Mode = {
val conf = GlobalConfig.systemConfig
val queryConfig = QueryConfig(conf.getConfig("filodb.query"))
val queryConfig = QueryConfig(queryConf)
val parser = queryConfig.parser
logger.info(s"Query parser mode: $parser")
parser match {
Expand Down Expand Up @@ -138,9 +140,23 @@ object Parser extends StrictLogging {
case NotRegexMatch => require(l.value.length <= REGEX_MAX_LEN,
s"Regular expression filters should be <= $REGEX_MAX_LEN characters")
ColumnFilter(l.label, Filter.NotEqualsRegex(l.value))
case RegexMatch => require(l.value.length <= REGEX_MAX_LEN,
s"Regular expression filters should be <= $REGEX_MAX_LEN characters")
ColumnFilter(l.label, Filter.EqualsRegex(l.value))
case RegexMatch =>
// Relax the length limit only for matchers that contain at most the "|" special character.
val shouldRelax = queryConf.hasPath("relaxed-pipe-only-equals-regex-limit") &&
QueryUtils.containsPipeOnlyRegex(l.value)
if (shouldRelax) {
val limit = queryConf.getInt("relaxed-pipe-only-equals-regex-limit");
require(l.value.length <= limit,
s"Regular expression filters should be <= $limit characters " +
s"when no special characters except '|' are used. " +
s"Violating filter is: ${l.label}=${l.value}")
} else {
require(l.value.length <= Parser.REGEX_MAX_LEN,
s"Regular expression filters should be <= ${Parser.REGEX_MAX_LEN} characters " +
s"when non-`|` special characters are used. " +
s"Violating filter is: ${l.label}=${l.value}")
}
ColumnFilter(l.label, Filter.EqualsRegex(l.value))
case NotEqual(false) => ColumnFilter(l.label, Filter.NotEquals(l.value))
case other: Any => throw new IllegalArgumentException(s"Unknown match operator $other")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,16 +6,16 @@ import filodb.core.GlobalConfig
import filodb.core.binaryrecord2.{BinaryRecordRowReader, StringifyMapItemConsumer}
import filodb.core.metadata.Column.ColumnType
import filodb.core.metadata.PartitionSchema
import filodb.core.query.{Result => _, _}
import filodb.core.query.{QueryUtils, Result => _, _}
import filodb.prometheus.parse.Parser.REGEX_MAX_LEN
import filodb.query.{QueryResult => FiloQueryResult, _}
import filodb.query.AggregationOperator.Avg
import filodb.query.exec.{ExecPlan, HistToPromSeriesMapper}

object PrometheusModel {
import com.softwaremill.quicklens._
val conf = GlobalConfig.defaultsFromUrl
val queryConfig = conf.getConfig("filodb.query")
private val conf = GlobalConfig.systemConfig
private val queryConfig = conf.getConfig("filodb.query")

/**
* If the result contains Histograms, automatically convert them to Prometheus vector-per-bucket output
Expand All @@ -42,9 +42,22 @@ object PrometheusModel {
case MatchType.EQUAL => Filter.Equals(m.getValue)
case MatchType.NOT_EQUAL => Filter.NotEquals(m.getValue)
case MatchType.REGEX_MATCH =>
require(m.getValue.length <= REGEX_MAX_LEN, s"Regular expression filters should " +
s"be <= ${REGEX_MAX_LEN} characters")
Filter.EqualsRegex(m.getValue)
// Relax the length limit only for matchers that contain at most the "|" special character.
val shouldRelax = queryConfig.getBoolean("relaxed-pipe-only-equals-regex-limit") &&
QueryUtils.containsPipeOnlyRegex(m.getValue)
if (shouldRelax) {
val limit = queryConfig.getInt("relaxed-pipe-only-equals-regex-limit");
require(m.getValue.length <= limit,
s"Regular expression filters should be <= $limit characters " +
s"when no special characters except '|' are used. " +
s"Violating filter is: ${m.getName}=${m.getValue}")
} else {
require(m.getValue.length <= REGEX_MAX_LEN,
s"Regular expression filters should be <= $REGEX_MAX_LEN characters " +
s"when non-`|` special characters are used. " +
s"Violating filter is: ${m.getName}=${m.getValue}")
}
Filter.EqualsRegex(m.getValue)
case MatchType.REGEX_NO_MATCH =>
require(m.getValue.length <= REGEX_MAX_LEN, s"Regular expression filters should " +
s"be <= ${REGEX_MAX_LEN} characters")
Expand Down
57 changes: 57 additions & 0 deletions prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ import filodb.query.{BinaryJoin, LogicalPlan}
import org.scalatest.funspec.AnyFunSpec
import org.scalatest.matchers.should.Matchers


import java.util.concurrent.TimeUnit.{DAYS, HOURS, MINUTES, SECONDS}
import scala.concurrent.duration.Duration

Expand Down Expand Up @@ -933,6 +934,62 @@ class ParserSpec extends AnyFunSpec with Matchers {
}
}

it("should correctly enforce filter length limit for EqualsRegex") {
val lim = Parser.REGEX_MAX_LEN

// true -> should not throw an exception; else false
val succeedQueryPairs = Seq(
// Begin successful...
(true, s"""job=~"${"a".repeat(lim)}""""),
(true, s"""job=~"${".".repeat(lim)}""""),
// May want to enforce this (no regex chars, but length limit exceeded).
(true, s"""job=~"${"a".repeat(lim + 1)}""""),
(true, s"""job=~"${"a".repeat(lim + 1).split("").mkString("|")}""""),
(true, s"""job=~"${"a|".repeat(10) + "a".repeat(lim)}""""),
// May want to enforce this (single "|"-joined value exceeds limit).
(true, s"""job=~"${"a|".repeat(10) + "a".repeat(lim + 1)}""""),

// Begin unsuccessful...
(false, s"""job=~"${".".repeat(lim + 1)}""""),
(false, s"""job=~"${".".repeat(lim + 1).split("").mkString("|")}""""),
(false, s"""job=~"${"a".repeat(lim + 1)}.*"}"""),
(false, s"""job=~"${"a".repeat(lim + 1).split("").mkString("|")}.*""""),
)
val timeParams = TimeStepParams(1000, 1, 1000)
val testFunc = (shouldSucceed: Boolean, lambda: () => Unit) => {
if (shouldSucceed) {
lambda.apply()
} else {
intercept[IllegalArgumentException] { lambda.apply() }
}
}
for ((shouldSucceed, filterString) <- succeedQueryPairs) {
val queryString = s"""foo{$filterString}"""
val idLambdaPairs: Seq[(String, () => Unit)] = Seq(
("q", () => Parser.queryToLogicalPlan(queryString, timeParams.start, timeParams.step)),
("qr", () => Parser.queryRangeToLogicalPlan(queryString, timeParams)),
("ln", () => Parser.labelNamesQueryToLogicalPlan(queryString, timeParams)),
("lv", () => Parser.labelValuesQueryToLogicalPlan(Seq("job"), Some(filterString), timeParams)),
("m", () => Parser.metadataQueryToLogicalPlan(queryString, timeParams)),
("lc", () => Parser.labelCardinalityToLogicalPlan(queryString, timeParams))
)
idLambdaPairs.foreach{case (id, lambda) =>
try {
testFunc.apply(shouldSucceed, lambda)
} catch {
case t: Throwable =>
val failMsg =
s"""======= TEST FAILED =======
|filterString: $filterString
|id: $id
|""".stripMargin
println(failMsg)
throw t
}
}
}
}

private def printBinaryJoin( lp: LogicalPlan, level: Int = 0) : scala.Unit = {
if (!lp.isInstanceOf[BinaryJoin]) {
info(s"${" "*level}" + lp.toString)
Expand Down

0 comments on commit 24b0ccb

Please sign in to comment.