diff --git a/.github/workflows/scala.yml b/.github/workflows/scala.yml index ea37425d92..3e01dddbe9 100644 --- a/.github/workflows/scala.yml +++ b/.github/workflows/scala.yml @@ -4,7 +4,7 @@ on: push: branches: [ develop ] pull_request: - branches: [ develop, integration, main ] + branches: [ develop, integration, main, feat-index-rust ] jobs: test: @@ -19,6 +19,14 @@ jobs: with: java-version: '11' distribution: 'adopt' + - name: Install Rust Toolchain + uses: actions-rust-lang/setup-rust-toolchain@v1 + with: + components: rustfmt, clippy + target: x86_64-apple-darwin, aarch64-apple-darwin, aarch64-unknown-linux-gnu + cache-workspaces: "core/src/rust -> target" + - name: Install cargo-zigbuild + run: pip install cargo-zigbuild - name: Run tests run: .github/workflows/runtests.sh - name: Coverage Reports diff --git a/.gitignore b/.gitignore index fef830a629..a2dcb6e69c 100644 --- a/.gitignore +++ b/.gitignore @@ -32,4 +32,7 @@ metastore_db/ **/kafka/src/test/scala/filodb/kafka/shard* *lib* +# Allow Rust's lib.rs since we're otherwise blocking *lib* above +!lib.rs + coordinator/src/test/resources/ diff --git a/README.md b/README.md index bfa3864f77..6153006a32 100644 --- a/README.md +++ b/README.md @@ -95,6 +95,8 @@ To compile the .mermaid source files to .png's, install the [Mermaid CLI](http:/ 3. [Apache Cassandra](http://cassandra.apache.org/) 2.x or 3.x (We prefer using [CCM](https://github.com/pcmanus/ccm) for local testing) - For testing, install a single node C* cluster, like this: `ccm create v39_single -v 3.9 -n 1 -s` 4. [Apache Kafka](http://kafka.apache.org/) 0.10.x or above +5. [Rust](https://www.rust-lang.org/tools/install) to build native components +6. A working C compiler for your system (GCC or Clang) Optional: diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala index 17aff74735..57ea013ddf 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala @@ -4,7 +4,9 @@ import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{IntervalSelector, RawSeries, SeriesKeysByFilters} +import filodb.query.LogicalPlan.getColumnFilterGroup +import filodb.query.util.{ExcludeAggRule, HierarchicalQueryExperienceParams, IncludeAggRule} +import filodb.query.{Aggregate, BinaryJoin, IntervalSelector, RawSeries, SeriesKeysByFilters} class LogicalPlanParserSpec extends AnyFunSpec with Matchers { @@ -294,4 +296,648 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val res = LogicalPlanParser.convertToQuery(lp) res shouldEqual "(http_requests_total{job=\"app\"} + 2.1)" } + + it("LogicalPlan update for hierarchical aggregation queries with Aggregate and BinaryJoin") { + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + val nextLevelAggregationTags = Set("aggTag", "aggTag2") + // CASE 1 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - Both lhs and rhs should be updated + val binaryJoinAggregationBothOptimization = "sum(metric1:::agg{aggTag=\"app\"}) + sum(metric2:::agg{aggTag=\"app\"})" + var lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationBothOptimization, t) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + lpUpdated.isInstanceOf[BinaryJoin] shouldEqual true + lpUpdated.asInstanceOf[BinaryJoin].lhs.isInstanceOf[Aggregate] shouldEqual true + lpUpdated.asInstanceOf[BinaryJoin].rhs.isInstanceOf[Aggregate] shouldEqual true + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter( x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith(nextLevelAggregatedMetricSuffix).shouldEqual(true) + ) + // CASE 2 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - rhs should be updated + val binaryJoinAggregationRHSOptimization = "sum(metric1:::agg{nonAggTag=\"abc\"}) + sum(metric2:::agg{aggTag=\"app\"})" + lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationRHSOptimization, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("metric2:::agg_2") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("metric1:::agg") + ) + // CASE 3 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - lhs should be updated and rhs should not since it is + // not an aggregated metric, even if both the metrics qualify for aggregation + val binaryJoinAggregationLHSOptimization = "sum(metric1:::agg{aggTag=\"abc\"}) + sum(metric2{aggTag=\"app\"})" + lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationLHSOptimization, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("metric2") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("metric1:::agg_2") + ) + } + + it("LogicalPlan update for hierarchical aggregation queries with by clause and include tags") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + val nextLevelAggregationTags = Set("aggTag", "aggTag2", "aggTag3", "aggTag4") + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) + // CASE 1 - Aggregate with by clause - should update the metric name as `by` clause labels are part of include tags + var query = "sum(rate(my_counter:::agg{aggTag=\"spark\", aggTag2=\"app\"}[5m])) by (aggTag4, aggTag3)" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 2 - should NOT update since bottomk aggregation operator is not allowed as of now + query = "sum(bottomk(2, my_counter:::agg{aggTag=\"spark\", aggTag2=\"filodb\"}) by (aggTag3, aggTag4)) by (aggTag4)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter( x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 3 - should NOT update since the by clause labels are not part of include tags + query = "sum(rate(my_counter:::agg{aggTag=\"spark\", aggTag2=\"app\"}[5m])) by (aggTag4, aggTag3, nonAggTag)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 4 - should update since the by clause labels are part of include tags - binary join case + query = "sum(my_gauge:::agg{aggTag=\"spark\", aggTag2=\"filodb\"}) by (aggTag, aggTag2) and on(aggTag, aggTag2) sum(my_counter:::agg{aggTag=\"spark\", aggTag2=\"filodb\"}) by (aggTag, aggTag2)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith(nextLevelAggregatedMetricSuffix).shouldEqual(true) + ) + // CASE 5 - lhs should not be updated since it does not match regex pattern - binary join case + query = "sum(my_gauge{aggTag=\"spark\", aggTag2=\"filodb\"}) by (aggTag, aggTag2) and on(aggTag, aggTag2) sum(my_counter:::agg{aggTag=\"spark\", aggTag2=\"filodb\"}) by (aggTag, aggTag2)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 6 - rhs should not be updated since it has column filters which is not present in include tags + query = "sum(my_gauge:::agg{aggTag=\"spark\", aggTag2=\"filodb\"}) by (aggTag, aggTag2) and on(aggTag, aggTag2) sum(my_counter:::agg{aggTag=\"spark\", aggTag2=\"filodb\", nonAggTag=\"1\"}) by (aggTag, aggTag2)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg_2") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + } + + it("LogicalPlan update for hierarchical aggregation queries with by clause and exclude tags") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + val nextLevelAggregationExcludeTags = Set("excludeAggTag", "excludeAggTag2") + val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationExcludeTags) + val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeAggRule)) + // CASE 1 - should update the metric name as `by` clause labels are not part of exclude tags + var query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) by (host)" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 2 - should NOT update the metric name as column filters are not part of exclude tags + query = "sum(rate(my_counter:::agg{nonAggTag=\"spark\", application=\"app\", excludeAggTag2=\"2.0\"}[5m]))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 2 - should NOT update since bottomk aggregation operator is not allowed as of now + query = "sum(bottomk(2, my_counter:::agg{job=\"spark\", application=\"filodb\"})) by (host)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 3 - should NOT update since the by clause labels intersect with exclude tags + query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) by (excludeAggTag2, excludeAggTag, id)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 4 - should update since the by clause labels are not part of exclude tags - binary join case + query = "sum(my_gauge:::agg{job=\"spark\", application=\"filodb\"}) by (id, host) + sum(your_gauge:::agg{job=\"spark\", application=\"filodb\"}) by (id, host)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith(nextLevelAggregatedMetricSuffix).shouldEqual(true) + ) + // CASE 5 - lhs should not be updated since it does not match regex pattern - binary join case + query = "sum(my_gauge{job=\"spark\", application=\"filodb\"}) by (id, host) - sum(your_gauge:::agg{job=\"spark\", application=\"filodb\"}) by (id, host)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("your_gauge:::agg_2") + ) + // CASE 6 - rhs should not be updated since it has column filters which are part of exclude tags + query = "sum(my_gauge:::agg{job=\"spark\", application=\"filodb\"}) by (id, host) / sum(your_gauge:::agg{job=\"spark\", application=\"filodb\", excludeAggTag2=\"1\"}) by (id, host)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg_2") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("your_gauge:::agg") + ) + } + + it ("LogicalPlan update for hierarchical aggregation queries with without clause and exclude tags") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + val nextLevelAggregationExcludeTags = Set("excludeAggTag", "excludeAggTag2") + val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationExcludeTags) + val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeAggRule)) + // CASE 1 - should update since the exclude tags are subset of the without clause labels + var query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) without (excludeAggTag2, excludeAggTag)" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 2 - should NOT update since bottomk aggregation operator is not allowed as of now + query = "sum(bottomk(2, my_counter:::agg{job=\"spark\", application=\"filodb\"}) without (excludeAggTag, excludeAggTag2))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter( x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 3 - should NOT update since the column filter label is part of exclude tags + query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\", excludeAggTag2=\"2\"}[5m]))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 4 - should update since the exclude tags are subset of the without clause labels + query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) without (excludeAggTag2, excludeAggTag, id)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 5 - should not update since the exclude tags are not subset of the without clause labels + query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) without (excludeAggTag2)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 6 - should update since the exclude tags are subset of without clause labels - binary join case + query = "sum(my_gauge:::agg{job=\"spark\", application=\"filodb\"}) without (excludeAggTag2, excludeAggTag) and ignoring(excludeAggTag2, excludeAggTag) sum(my_counter:::agg{job=\"spark\", application=\"filodb\"}) without (excludeAggTag2, excludeAggTag)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith(nextLevelAggregatedMetricSuffix).shouldEqual(true) + ) + // CASE 7 - lhs should not be updated since it does not match regex pattern - binary join case + query = "sum(my_gauge{job=\"spark\", application=\"filodb\"}) without (excludeAggTag2, excludeAggTag) and ignoring(excludeAggTag2, excludeAggTag) sum(my_counter:::agg{job=\"spark\", application=\"filodb\"}) without (excludeAggTag2, excludeAggTag)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 8 - rhs should not be updated since it has column filters which is part of exclude tags + query = "sum(my_gauge:::agg{job=\"spark\", application=\"filodb\"}) without (excludeAggTag2, excludeAggTag) and ignoring(excludeAggTag2, excludeAggTag) sum(my_counter:::agg{job=\"spark\", application=\"filodb\", excludeAggTag2=\"1\"}) without (excludeAggTag2, excludeAggTag)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg_2") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + } + + it ("LogicalPlan update for hierarchical aggregation queries with without clause and include tags") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + val nextLevelAggregationTags = Set("job", "application", "instance", "version") + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) + // All the cases should not be updated since without clause with include tags is not supported as of now + var query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) without (version, instance)" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + query = "sum(bottomk(2, my_counter:::agg{job=\"spark\", application=\"filodb\"}) without (instance, version))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + query = "sum(my_gauge:::agg{job=\"spark\", application=\"filodb\"}) without (version, instance) + sum(your_gauge:::agg{job=\"spark\", application=\"filodb\"}) without (version, instance)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith(":::agg").shouldEqual(true) + ) + query = "sum(my_gauge{job=\"spark\", application=\"filodb\"}) without (version, instance) - sum(your_gauge:::agg{job=\"spark\", application=\"filodb\"}) without (version, instance)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("your_gauge:::agg") + ) + query = "sum(my_gauge:::agg{job=\"spark\", application=\"filodb\"}) without (version, instance) / sum(your_gauge:::agg{job=\"spark\", application=\"filodb\", version=\"1\"}) without (version, instance)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg") + ) + filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("your_gauge:::agg") + ) + } + + it ("LogicalPlan update for hierarchical aggregation queries should not update simple raw and range queries") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + val nextLevelAggregationTags = Set("job", "application", "instance", "version") + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) + // CASE 1: Raw queries lp should not be updated directly + var query = "my_counter:::agg{job=\"spark\", application=\"app\"}[5m]" + var lp = Parser.queryToLogicalPlan(query, t.start, t.step) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + // CASE 2: Simple range query without aggregates lp should not be updated directly + query = "rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter( x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + } + + it("LogicalPlan update for hierarchical aggregation queries should update for only allowed aggregate operators") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + val nextLevelAggregationTags = Set("job", "application", "instance", "version") + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) + // CASE 1: count aggregate should not be allowed + var query = "count(my_gauge:::agg{job=\"spark\", application=\"app\"})" + var lp = Parser.queryToLogicalPlan(query, t.start, t.step) + lp.isInstanceOf[Aggregate] shouldEqual true + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg") + ) + // CASE 2: sum aggregate should be allowed + query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m]))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lp.isInstanceOf[Aggregate] shouldEqual true + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 3: avg aggregate should not be allowed + query = "avg(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m]))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lp.isInstanceOf[Aggregate] shouldEqual true + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg") + ) + } + + it("LogicalPlan update for hierarchical nested aggregation queries") { + // common parameters using include tags + val t = TimeStepParams(700, 1000, 10000) + val nextLevelAggregatedMetricSuffix = "agg_2" + var nextLevelAggregationTags = Set("aggTag1", "aggTag2", "aggTag3") + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) + // CASE 1: should update the metric name as `by` clause labels are part of include tags + var query = "sum(sum(my_counter:::agg{aggTag1=\"spark\", aggTag2=\"app\"}) by (aggTag1, aggTag2, aggTag3))" + var lp = Parser.queryToLogicalPlan(query, t.start, t.step) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_counter:::agg_2") + ) + // CASE 2: should not update since count aggregate operator is not allowed + query = "sum by (aggTag1, aggTag2) (count by (aggTag1, aggTag2) (my_gauge:::agg{aggTag1=\"a\",aggTag2=\"b\"}))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg") + ) + // CASE 3: should update since min aggregate operator is allowed + query = "sum by (aggTag1, aggTag2) (sum by (aggTag1, aggTag2) (my_gauge:::agg{aggTag1=\"a\",aggTag2=\"b\"}))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg_2") + ) + // using excludeTags + nextLevelAggregationTags = Set("excludeAggTag1", "excludeAggTag2") + val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeAggRule)) + // CASE 4: should update since excludeTags are not used + query = "sum by (aggTag1, aggTag2) (sum by (aggTag1, aggTag2) (my_gauge:::agg{aggTag1=\"a\", aggTag2=\"b\"}))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg_2") + ) + // CASE 5: should not update since excludeTags are used + query = "sum by (excludeAggTag1,aggTag2) (sum by (excludeAggTag1,aggTag1, aggTag2) (my_gauge:::agg{aggTag1=\"a\", aggTag2=\"b\"}))" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(excludeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("my_gauge:::agg") + ) + } + + it("LogicalPlan update for BinaryJoin with multiple agg rules and suffixes") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) + // Query with multiple agg rules and suffixes + val includeParams = HierarchicalQueryExperienceParams(":::", + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) + // CASE 1 - should update - simple binary join with two different aggregated metrics and suffixes, both of which are satisfying the next level aggregation metric constraints + var query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"})" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith("_2").shouldEqual(true) + ) + // CASE 2 - should NOT update as rhs is using an exclude tag + query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) + sum(your_gauge:::suffix2{excludeTag1=\"spark\", notExcludeTag2=\"filodb\"})" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + var updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1_2").shouldEqual(true) + updatedMetricNamesSet.contains("your_gauge:::suffix2").shouldEqual(true) // not updated + // CASE 3 - should NOT update as lhs is not using an include tag + query = "sum(my_gauge:::suffix1{notIncludeTag1=\"spark\", includeTag2=\"filodb\"}) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"})" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1").shouldEqual(true) // not updated + updatedMetricNamesSet.contains("your_gauge:::suffix2_2").shouldEqual(true) + // CASE 4 - should NOT update as both lhs and rhs are not using appropriate tags for next level aggregation metric + query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", notIncludeTag2=\"filodb\"}) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", excludeTag2=\"filodb\"})" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1").shouldEqual(true) // not updated + updatedMetricNamesSet.contains("your_gauge:::suffix2").shouldEqual(true) // not updated + } + + it("LogicalPlan update for BinaryJoin with multiple agg rules and suffixes with by clauses") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) + // Query with multiple agg rules and suffixes + val includeParams = HierarchicalQueryExperienceParams(":::", + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) + // CASE 1 - should update - both lhs and rhs are satisfying the next level aggregation metric constraints + var query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (notExcludeTag1)" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith("_2").shouldEqual(true) + ) + // CASE 2 - should NOT update as rhs is using an exclude tag + query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (excludeTag1)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + var updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1_2").shouldEqual(true) + updatedMetricNamesSet.contains("your_gauge:::suffix2").shouldEqual(true) // not updated + // CASE 3 - should NOT update as lhs is not using an include tag + query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (notIncludeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (notExludeTag1)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1").shouldEqual(true) // not updated + updatedMetricNamesSet.contains("your_gauge:::suffix2_2").shouldEqual(true) + // CASE 4 - should NOT update as both lhs and rhs are not using appropriate tags for next level aggregation metric + query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (notIncludeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (excludeTag1)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1").shouldEqual(true) // not updated + updatedMetricNamesSet.contains("your_gauge:::suffix2").shouldEqual(true) // not updated + } + + it("LogicalPlan update for BinaryJoin with multiple agg rules and suffixes with by and without clauses") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) + // Query with multiple agg rules and suffixes + val includeParams = HierarchicalQueryExperienceParams(":::", + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) + // CASE 1 - should update - both lhs and rhs are satisfying the next level aggregation metric constraints + var query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) without (excludeTag1, excludeTag2)" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + filterGroups.foreach( + filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .endsWith("_2").shouldEqual(true) + ) + // CASE 2 - should NOT update as excludeRule tags is not subset of rhs without clause labels + query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) without (excludeTag1)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + var updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1_2").shouldEqual(true) + updatedMetricNamesSet.contains("your_gauge:::suffix2").shouldEqual(true) // not updated + } + + it("LogicalPlan should not update when next level aggregation metric suffix is not matching agg rules") { + // common parameters + val t = TimeStepParams(700, 1000, 10000) + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) + // Query with multiple agg rules and suffixes + val includeParams = HierarchicalQueryExperienceParams(":::", + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) + // CASE 1 - should not update - both lhs and rhs metric are not using suffix passed for lp update + var query = "sum(my_gauge:::no_rule{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1)" + var lp = Parser.queryRangeToLogicalPlan(query, t) + var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + var filterGroups = getColumnFilterGroup(lpUpdated) + var updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::no_rule").shouldEqual(true)// not updated + // CASE 2 - should NOT update rhs as it is not using the given suffix + query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::no_rule2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (notExcludeTag1)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::suffix1_2").shouldEqual(true) + updatedMetricNamesSet.contains("your_gauge:::no_rule2").shouldEqual(true) // not updated + // CASE 3 - should NOT update lhs as it is not using the given suffix + query = "sum(my_gauge:::no_rule{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (notExcludeTag1)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::no_rule").shouldEqual(true)// not updated + updatedMetricNamesSet.contains("your_gauge:::suffix2_2").shouldEqual(true) + // CASE 3 - should NOT update lhs and rhs as it is not using the given suffix + query = "sum(my_gauge:::no_rule{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::no_rule2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (notExcludeTag1)" + lp = Parser.queryRangeToLogicalPlan(query, t) + lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) + filterGroups = getColumnFilterGroup(lpUpdated) + updatedMetricNamesSet = filterGroups.flatten.filter(x => x.column == "__name__") + .map(_.filter.valuesStrings.head.asInstanceOf[String]).toSet + updatedMetricNamesSet.contains("my_gauge:::no_rule").shouldEqual(true) // not updated + updatedMetricNamesSet.contains("your_gauge:::no_rule2").shouldEqual(true) // not updated + } } diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 493e777127..90a24f4562 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -331,6 +331,30 @@ filodb { deployment-partition-name = "local" query { + # hierarchical query experience configs + hierarchical { + # Parent logical plans for which the hierarchical query experience is enabled + # This follows the "OR" convention if there are multiple parent logical plans which are enabled in lp tree + allowed-parent-logical-plans = [ + "Aggregate" + ] + + # There are few periodic series plans which stores the RawSeries plan. This config allows us to select the + # logical plans which are allowed to manipulate/optimize the underlying RawSeries + allowed-periodic-series-plans-with-raw-series = [ + "PeriodicSeriesWithWindowing", + "PeriodicSeries" + ] + + # Range functions allowed for hierarchical query experience + allowed-range-functions = [ + "rate", + "increase", + ] + + # Aggregations allowed for hierarchical query experience + allowed-aggregation-operators = ["sum"] + } # feature flag to enable start time inclusive in window lookback inclusive-range = true @@ -400,6 +424,9 @@ filodb { # If unspecified, all workspaces are disabled by default. workspaces-disabled-max-min = ["disabled_ws"] + # config to figure out the tenant column filter to enable max min column selection + max-min-tenant-column-filter = "_ws_" + routing { enable-remote-raw-exports = false max-time-range-remote-raw-export = 180 minutes @@ -786,6 +813,27 @@ filodb { block-memory-manager-percent = 71 } + # Settings for Tantivy backed indexes + tantivy { + # Max number of items to keep in the column cache. This speeds up retrieval of values, + # especially on fast queries. Each cached item is very small, < 1KB + column-cache-count = 1000 + + # Max size of the query results cache, in bytes. This can have a major performance win + # for alert type queries that are periodically running the same query over and over. + query-cache-max-bytes = 50MB + + # Estimated size of an item in the query cache, in bytes. This is the size in bits + # of the number of documents each segment searches over, estimated to 250k docs. + # This is a hint to the cache only and does not bound the max number of items. + query-cache-estimated-item-size = 31250 + + # Percentage of deleted docs in a segment that will flag this to be considered + # for a merge. Setting this too high will leave too many documents around + # and increase query time. + deleted-doc-merge-threshold = 0.1 + } + # At the cost of some extra heap memory, we can track queries holding shared lock for a long time # and starving the exclusive access of lock for eviction track-queries-holding-eviction-lock = true @@ -806,6 +854,11 @@ filodb { # Whether to add the _type_ label to all time series for the purpose of filtering type-field-indexing-enabled = false + + # The Part Key index implementation to use. Supported values: + # lucene - Lucene based index (default) + # tantivy - Tantivy based index + part-key-index-type = lucene } # for standalone worker cluster configuration, see akka-bootstrapper diff --git a/core/src/main/scala/filodb.core/GlobalConfig.scala b/core/src/main/scala/filodb.core/GlobalConfig.scala index b556aa9693..87ca5eab40 100644 --- a/core/src/main/scala/filodb.core/GlobalConfig.scala +++ b/core/src/main/scala/filodb.core/GlobalConfig.scala @@ -4,6 +4,8 @@ import com.typesafe.config.{Config, ConfigFactory} import com.typesafe.scalalogging.StrictLogging import scala.jdk.CollectionConverters._ +import filodb.core.metadata.DatasetOptions + /** * Loads the overall configuration in a specific order: * - System properties @@ -49,4 +51,25 @@ object GlobalConfig extends StrictLogging { case true => Some(systemConfig.getStringList("filodb.query.workspaces-disabled-max-min").asScala.toSet) } + // Column filter used to check the enabling/disabling the use of max-min columns + val maxMinTenantColumnFilter = systemConfig.getString("filodb.query.max-min-tenant-column-filter") + + // default dataset-options config + val datasetOptions: Option[DatasetOptions] = + systemConfig.hasPath("filodb.partition-schema.options") match { + case false => None + case true => { + val datasetOptionsConfig = systemConfig.getConfig("filodb.partition-schema.options") + Some(DatasetOptions.fromConfig(datasetOptionsConfig)) + } + } + + val PromMetricLabel = "__name__" + + val hierarchicalConfig: Option[Config] = { + systemConfig.hasPath("filodb.query.hierarchical") match { + case false => None + case true => Some(systemConfig.getConfig("filodb.query.hierarchical")) + } + } } diff --git a/core/src/main/scala/filodb.core/Utils.scala b/core/src/main/scala/filodb.core/Utils.scala index 79191234fb..3384097928 100644 --- a/core/src/main/scala/filodb.core/Utils.scala +++ b/core/src/main/scala/filodb.core/Utils.scala @@ -1,9 +1,11 @@ package filodb.core +import java.io.{File, IOException} import java.lang.management.ManagementFactory import com.typesafe.config.{Config, ConfigRenderOptions} import com.typesafe.scalalogging.StrictLogging +import scala.util.{Failure, Try} object Utils extends StrictLogging { private val threadMbean = ManagementFactory.getThreadMXBean @@ -37,4 +39,34 @@ object Utils extends StrictLogging { logger.info(s"Available memory calculated or configured as $availableMem") availableMem } + + // Recursively delete a folder + def deleteRecursively(f: File, deleteRoot: Boolean = false): Try[Boolean] = { + val subDirDeletion: Try[Boolean] = + if (f.isDirectory) + f.listFiles match { + case xs: Array[File] if xs != null && !xs.isEmpty => + val subDirDeletions: Array[Try[Boolean]] = xs map (f => deleteRecursively(f, true)) + subDirDeletions reduce ((reduced, thisOne) => { + thisOne match { + // Ensures even if one Right(_) is found, thr response will be Right(Throwable) + case scala.util.Success(_) if reduced == scala.util.Success(true) => thisOne + case Failure(_) => thisOne + case _ => reduced + } + }) + case _ => scala.util.Success(true) + } + else + scala.util.Success(true) + + subDirDeletion match { + case scala.util.Success(_) => + if (deleteRoot) { + if (f.delete()) scala.util.Success(true) else Failure(new IOException(s"Unable to delete $f")) + } else scala.util.Success(true) + case right@Failure(_) => right + } + + } } diff --git a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala index 2d3f335a5c..0ad6b3f038 100644 --- a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala +++ b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala @@ -52,7 +52,14 @@ class RawIndexBootstrapper(colStore: ColumnStore) { colStore.scanPartKeys(ref, shardNum) .map { pk => val partId = assignPartId(pk) - index.addPartKey(pk.partKey, partId, pk.startTime, pk.endTime)() + // -1 is returned if we skiped the part key for any reason, such as + // unknown schema or memory issues + // + // assignPartId will log these cases, so no extra logging is needed + // here + if (partId != -1) { + index.addPartKey(pk.partKey, partId, pk.startTime, pk.endTime)() + } } .countL .map { count => diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 760fd9e823..b200af78f4 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -263,7 +263,7 @@ TimeSeriesShard(ref, schemas, storeConfig, numShards, quotaSource, shardNum, buf logger.debug(s"Creating TSPartition for ODP from part ID $id in dataset=$ref shard=$shardNum") // If not there, then look up in Lucene and get the details for { partKeyBytesRef <- partKeyIndex.partKeyFromPartId(id) - unsafeKeyOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(partKeyBytesRef.offset) + unsafeKeyOffset = PartKeyIndexRaw.bytesRefToUnsafeOffset(partKeyBytesRef.offset) group = partKeyGroup(schemas.part.binSchema, partKeyBytesRef.bytes, unsafeKeyOffset, numGroups) sch <- Option(schemas(RecordSchema.schemaID(partKeyBytesRef.bytes, unsafeKeyOffset))) } yield { diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyIndex.scala index 05dab31dc3..ce5409f3c8 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyIndex.scala @@ -1,12 +1,264 @@ package filodb.core.memstore +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Path +import java.util.regex.Pattern + +import scala.collection.immutable.HashSet +import scala.util.{Failure, Success} + +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon +import kamon.metric.MeasurementUnit import org.apache.lucene.util.BytesRef +import filodb.core.{concurrentCache, DatasetRef, Utils} +import filodb.core.Types.PartitionKey +import filodb.core.binaryrecord2.MapItemConsumer +import filodb.core.memstore.PartKeyIndexRaw.{bytesRefToUnsafeOffset, createTempDir, END_TIME, START_TIME} +import filodb.core.memstore.PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset +import filodb.core.memstore.PartKeyQueryBuilder.removeRegexAnchors import filodb.core.memstore.ratelimit.CardinalityTracker -import filodb.core.metadata.PartitionSchema -import filodb.core.query.ColumnFilter +import filodb.core.metadata.{PartitionSchema, Schemas} +import filodb.core.metadata.Column.ColumnType.{MapColumn, StringColumn} +import filodb.core.query.{ColumnFilter, Filter, QueryUtils} +import filodb.core.query.Filter.{And, Equals, EqualsRegex, In, NotEquals, NotEqualsRegex} +import filodb.memory.{UTF8StringMedium, UTF8StringShort} +import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String => UTF8Str} + +object PartKeyIndexRaw { + // NOTE: these partId fields need to be separate because Lucene 9.7.0 enforces consistent types for document + // field values (i.e. a field cannot have both numeric and string values). Additional details can be found + // here: https://github.com/apache/lucene/pull/11 + final val PART_ID_DV = "__partIdDv__" + final val PART_ID_FIELD = "__partIdField__" + final val START_TIME = "__startTime__" + final val END_TIME = "__endTime__" + final val PART_KEY = "__partKey__" + final val LABEL_LIST = s"__labelList__" + final val FACET_FIELD_PREFIX = "$facet_" + final val LABEL_LIST_FACET = FACET_FIELD_PREFIX + LABEL_LIST + + final val ignoreIndexNames = HashSet(START_TIME, PART_KEY, END_TIME, PART_ID_FIELD, PART_ID_DV) + + def bytesRefToUnsafeOffset(bytesRefOffset: Int): Int = bytesRefOffset + UnsafeUtils.arayOffset + + private def defaultTempDir(ref: DatasetRef, shardNum: Int): File = { + val baseDir = new File(System.getProperty("java.io.tmpdir")) + val baseName = s"partKeyIndex-$ref-$shardNum-${System.currentTimeMillis()}-" + val tempDir = new File(baseDir, baseName) + tempDir + } + + private def createTempDir(ref: DatasetRef, shardNum: Int): File = { + val tempDir = defaultTempDir(ref, shardNum) + tempDir.mkdir() + tempDir + } +} + +abstract class PartKeyIndexRaw(ref: DatasetRef, + shardNum: Int, + schema: PartitionSchema, + diskLocation: Option[File] = None, + protected val lifecycleManager: Option[IndexMetadataStore] = None, + protected val addMetricTypeField: Boolean = true) + extends StrictLogging { + + protected val startTimeLookupLatency = Kamon.histogram("index-startTimes-for-odp-lookup-latency", + MeasurementUnit.time.nanoseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + protected val labelValuesQueryLatency = Kamon.histogram("index-label-values-query-latency", + MeasurementUnit.time.nanoseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + protected val queryIndexLookupLatency = Kamon.histogram("index-partition-lookup-latency", + MeasurementUnit.time.nanoseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + protected val partIdFromPartKeyLookupLatency = Kamon.histogram("index-ingestion-partId-lookup-latency", + MeasurementUnit.time.nanoseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + private val _indexDiskLocation = diskLocation.map(new File(_, ref.dataset + File.separator + shardNum)) + .getOrElse(createTempDir(ref, shardNum)).toPath + + def indexDiskLocation: Path = _indexDiskLocation + + // If index rebuild is triggered or the state is Building, simply clean up the index directory and start + // index rebuild + if ( + lifecycleManager.forall(_.shouldTriggerRebuild(ref, shardNum)) + ) { + logger.info(s"Cleaning up indexDirectory=$indexDiskLocation for dataset=$ref, shard=$shardNum") + Utils.deleteRecursively(indexDiskLocation.toFile) match { + case Success(_) => // Notify the handler that the directory is now empty + logger.info(s"Cleaned directory for dataset=$ref, shard=$shardNum and index directory=$indexDiskLocation") + notifyLifecycleListener(IndexState.Empty, System.currentTimeMillis) + + case Failure(t) => // Update index state as TriggerRebuild again and rethrow the exception + logger.warn(s"Exception while deleting directory for dataset=$ref, shard=$shardNum " + + s"and index directory=$indexDiskLocation with stack trace", t) + notifyLifecycleListener(IndexState.TriggerRebuild, System.currentTimeMillis) + throw new IllegalStateException("Unable to clean up index directory", t) + } + } + //else { + // TODO here we assume there is non-empty index which we need to validate + //} + + protected def loadIndexData[T](ctor: () => T): T = try { + ctor() + } catch { + case e: Exception => + // If an exception is thrown here there is something wrong with the index or the directory + // We will attempt once by cleaning the directory and try instantiating the index again + logger.warn(s"Index for dataset:${ref.dataset} and shard: $shardNum possibly corrupt," + + s"index directory will be cleaned up and index rebuilt", e) + Utils.deleteRecursively(indexDiskLocation.toFile) match { + case Success(_) => // Notify the handler that the directory is now empty + logger.info(s"Cleaned directory for dataset=$ref," + + s"shard=$shardNum and index directory=$indexDiskLocation") + notifyLifecycleListener(IndexState.Empty, System.currentTimeMillis) + case Failure(t) => logger.warn(s"Exception while deleting directory for dataset=$ref," + + s"shard=$shardNum and index directory=$indexDiskLocation with stack trace", t) + // If we still see failure, set the TriggerRebuild and rethrow the exception + notifyLifecycleListener(IndexState.TriggerRebuild, System.currentTimeMillis) + throw new IllegalStateException("Unable to clean up index directory", t) + } + // Retry again after cleaning up the index directory, if it fails again, something needs to be looked into. + ctor() + } + + + def notifyLifecycleListener(state: IndexState.Value, time: Long): Unit = + lifecycleManager.foreach(_.updateState(ref, shardNum, state, time)) + + + + protected def partKeyString(docId: String, + partKeyOnHeapBytes: Array[Byte], + partKeyBytesRefOffset: Int = 0): String = { + val partHash = schema.binSchema.partitionHash(partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset)) + //scalastyle:off + s"shard=$shardNum partId=$docId partHash=$partHash [${ + TimeSeriesPartition + .partKeyString(schema, partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset)) + }]" + //scalastyle:on + } + + private val utf8ToStrCache = concurrentCache[UTF8Str, String](PartKeyLuceneIndex.MAX_STR_INTERN_ENTRIES) + + /** + * Map of partKey column to the logic for indexing the column (aka Indexer). + * Optimization to avoid match logic while iterating through each column of the partKey + */ + protected final val indexers = schema.columns.zipWithIndex.map { case (c, pos) => + c.columnType match { + case StringColumn => new Indexer { + val colName = UTF8Str(c.name) + def fromPartKey(base: Any, offset: Long, partIndex: Int): Unit = { + val strOffset = schema.binSchema.blobOffset(base, offset, pos) + val numBytes = schema.binSchema.blobNumBytes(base, offset, pos) + val value = new String(base.asInstanceOf[Array[Byte]], strOffset.toInt - UnsafeUtils.arayOffset, + numBytes, StandardCharsets.UTF_8) + addIndexedField(colName.toString, value) + } + def getNamesValues(key: PartitionKey): Seq[(UTF8Str, UTF8Str)] = ??? // not used + } + case MapColumn => new Indexer { + private val colName = c.name + + private val mapConsumer = new MapItemConsumer { + def consume(keyBase: Any, keyOffset: Long, valueBase: Any, valueOffset: Long, index: Int): Unit = { + import filodb.core._ + val key = utf8ToStrCache.getOrElseUpdate(new UTF8Str(keyBase, keyOffset + 1, + UTF8StringShort.numBytes(keyBase, keyOffset)), + _.toString) + val value = new String(valueBase.asInstanceOf[Array[Byte]], + unsafeOffsetToBytesRefOffset(valueOffset + 2), // add 2 to move past numBytes + UTF8StringMedium.numBytes(valueBase, valueOffset), StandardCharsets.UTF_8) + addIndexedMapFieldFromClientData(colName, key, value) + } + } + + def fromPartKey(base: Any, offset: Long, partIndex: Int): Unit = { + schema.binSchema.consumeMapItems(base, offset, pos, mapConsumer) + } + def getNamesValues(key: PartitionKey): Seq[(UTF8Str, UTF8Str)] = ??? // not used + } + case other: Any => + logger.warn(s"Column $c has type that cannot be indexed and will be ignored right now") + NoOpIndexer + } + }.toArray + + protected val numPartColumns = schema.columns.length + + private final val emptyStr = "" + // Multi-column facets to be created on "partition-schema" columns + protected def createMultiColumnFacets(partKeyOnHeapBytes: Array[Byte], partKeyBytesRefOffset: Int): Unit = { + schema.options.multiColumnFacets.foreach(facetCols => { + facetCols match { + case (name, cols) => { + val concatFacetValue = cols.map { col => + val colInfoOpt = schema.columnIdxLookup.get(col) + colInfoOpt match { + case Some((columnInfo, pos)) if columnInfo.columnType == StringColumn => + val base = partKeyOnHeapBytes + val offset = bytesRefToUnsafeOffset(partKeyBytesRefOffset) + val strOffset = schema.binSchema.blobOffset(base, offset, pos) + val numBytes = schema.binSchema.blobNumBytes(base, offset, pos) + new String(base, strOffset.toInt - UnsafeUtils.arayOffset, + numBytes, StandardCharsets.UTF_8) + case _ => emptyStr + } + }.mkString("\u03C0") + addMultiColumnFacet(name, concatFacetValue) + } + } + }) + } -trait PartKeyIndexRaw { + /** + * Add an indexed field + value defined in a map field to the document being prepared + */ + protected def addIndexedMapFieldFromClientData(mapColumn: String, labelName: String, value: String): Unit = { + if (addMetricTypeField) { + // do not index any existing _type_ tag since this is reserved and should not be passed in by clients + if (labelName != Schemas.TypeLabel) + addIndexedMapField(mapColumn, labelName, value) + else + logger.warn("Map column with name '_type_' is a reserved label. Not indexing it.") + // I would have liked to log the entire PK to debug, but it is not accessible from here. + // Ignoring for now, since the plan of record is to drop reserved labels at ingestion gateway. + } else { + addIndexedMapField(mapColumn, labelName, value) + } + } + + /** + * Add an indexed field + value to the document being prepared + */ + protected def addIndexedField(key: String, value: String): Unit + + /** + * Add an indexed field + value defined in a map field to the document being prepared - internal + * logic specific to index + */ + protected def addIndexedMapField(mapColumn: String, key: String, value: String): Unit + + /** + * Add a facet for a computed multi-column facet + */ + protected def addMultiColumnFacet(key: String, value: String): Unit /** * Clear the index by deleting all documents and commit @@ -50,9 +302,9 @@ trait PartKeyIndexRaw { def indexNumEntries: Long /** - * Number of documents in flushed index, includes tombstones for deletes + * Memory used by index in mmap segments, mostly for index data */ - def indexNumEntriesWithTombstones: Long + def indexMmapBytes: Long /** * Closes the index for read by other clients. Check for implementation if commit would be done @@ -79,11 +331,13 @@ trait PartKeyIndexRaw { */ def labelNamesEfficient(colFilters: Seq[ColumnFilter], startTime: Long, endTime: Long): Seq[String] + protected final val LABEL_NAMES_AND_VALUES_DEFAULT_LIMIT = 100 + /** * Use faceting to get field/index values given a column filter and time range */ def labelValuesEfficient(colFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, - colName: String, limit: Int = 100): Seq[String] + colName: String, limit: Int = LABEL_NAMES_AND_VALUES_DEFAULT_LIMIT): Seq[String] /** * Add new part key to index @@ -132,7 +386,7 @@ trait PartKeyIndexRaw { /** * Commit index contents to disk */ - def commit(): Long + def commit(): Unit /** * Update existing part key document with new endTime. @@ -181,12 +435,17 @@ trait PartKeyIndexRaw { } -trait PartKeyIndexDownsampled extends PartKeyIndexRaw { +abstract class PartKeyIndexDownsampled(ref: DatasetRef, + shardNum: Int, + schema: PartitionSchema, + diskLocation: Option[File] = None, + lifecycleManager: Option[IndexMetadataStore] = None, + addMetricTypeField: Boolean = true) + extends PartKeyIndexRaw(ref, shardNum, schema, diskLocation, lifecycleManager, + addMetricTypeField = addMetricTypeField) { def getCurrentIndexState(): (IndexState.Value, Option[Long]) - def notifyLifecycleListener(state: IndexState.Value, time: Long): Unit - /** * Iterate through the LuceneIndex and calculate cardinality count */ @@ -205,3 +464,227 @@ trait PartKeyIndexDownsampled extends PartKeyIndexRaw { endTime: Long, func: (BytesRef) => Unit): Int } + +/** + * Base class to convert incoming FiloDB ColumnFilters into the index native + * query format. Uses a visitor pattern to allow the implementation to build + * different query object patterns. + * + * The query language supports the following: + * * Boolean queries (many subqueries with MUST / MUST_NOT). These may be nested. + * * Equals queries (term match) + * * Regex queries (regex term match) + * * TermIn queries (term presence in list) + * * Prefix queries (term starts with) + * * MatchAll queries (match all values) + * * Range queries (start to end on a long value) + * + * This list may expand over time. + * + * The top level is always a Boolean query, even if it only has one child query. A + * Boolean query will never have zero child queries. Leaf nodes are always non-Boolean + * queries. + * + * For example: + * + represents MUST, - represents MUST_NOT + * Input query -> ((+Equals(Col, A) -Equals(Col, B)) +Regex(Col2, C.*)) + * + * The visitor call pattern would be: + * * visitStartBooleanQuery + * * visitStartBooleanQuery + * * visitEqualsQuery(Col, A, MUST) + * * visitEqualsQuery(Col, B, MUST_NOT) + * * visitEndBooleanQuery + * * visitRegexQuery(Col2, C.*, MUST) + * * visitEndBooleanQuery + * + * This class should be used instead of putting query parsing code in each + * implementation so that common optimizations, such as query rewriting, + * can occur in one place. + */ +abstract class PartKeyQueryBuilder { + /** + * Start a new boolean query + */ + protected def visitStartBooleanQuery(): Unit + + /** + * Indicate the current boolean query has ended + */ + protected def visitEndBooleanQuery(): Unit + + /** + * Add a new equals query to the current boolean query + */ + protected def visitEqualsQuery(column: String, term: String, occur: PartKeyQueryOccur): Unit + + /** + * Add a new Regex query to the current boolean query + */ + protected def visitRegexQuery(column: String, pattern: String, occur: PartKeyQueryOccur): Unit + + /** + * Add a TermsIn query to the current boolean query + */ + protected def visitTermInQuery(column: String, terms: Seq[String], occur: PartKeyQueryOccur): Unit + + /** + * Add a prefix query to the current boolean query + */ + protected def visitPrefixQuery(column: String, prefix: String, occur: PartKeyQueryOccur): Unit + + /** + * Add a match all query to the current boolean query + */ + protected def visitMatchAllQuery(): Unit + + /** + * Add a range match to the current boolean query + */ + protected def visitRangeQuery(column: String, start: Long, end: Long, occur: PartKeyQueryOccur): Unit + + protected def visitQuery(columnFilters: Seq[ColumnFilter]): Unit = { + visitStartBooleanQuery() + if(columnFilters.isEmpty) { + // No filters should match all documents + visitMatchAllQuery() + } else { + columnFilters.foreach { filter => + visitFilter(filter.column, filter.filter) + } + } + visitEndBooleanQuery() + } + + protected def visitQueryWithStartAndEnd(columnFilters: Seq[ColumnFilter], startTime: Long, + endTime: Long): Unit = { + visitStartBooleanQuery() + columnFilters.foreach { filter => + visitFilter(filter.column, filter.filter) + } + // Query optimization - don't time range filter if we're specifying max bounds and would match + // everything anyway + if (endTime < Long.MaxValue ) { + visitRangeQuery(START_TIME, 0, endTime, OccurMust) + } + if (startTime > 0) { + visitRangeQuery(END_TIME, startTime, Long.MaxValue, OccurMust) + } + + // Query optimization - if we produced no filters after optimization emit a match all as the time ranges + // we skipped would have covered that + if (endTime == Long.MaxValue && startTime <= 0 && columnFilters.isEmpty) { + visitMatchAllQuery() + } + + visitEndBooleanQuery() + } + + // scalastyle:off method.length + private def visitFilter(column: String, filter: Filter): Unit = { + def equalsQuery(value: String): Unit = { + if (value.nonEmpty) visitEqualsQuery(column, value, OccurMust) + else visitFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. + } + + filter match { + case EqualsRegex(value) => + val regex = removeRegexAnchors(value.toString) + if (regex == "") { + // if label=~"" then match empty string or label not present condition too + visitFilter(column, NotEqualsRegex(".+")) + } else if (regex.replaceAll("\\.\\*", "") == "") { + // if label=~".*" then match all docs since promQL matches .* with absent label too + visitMatchAllQuery() + } else if (!QueryUtils.containsRegexChars(regex)) { + // if all regex special chars absent, then treat like Equals + equalsQuery(regex) + } else if (QueryUtils.containsPipeOnlyRegex(regex)) { + // if pipe is only regex special char present, then convert to IN query + visitTermInQuery(column, regex.split('|'), OccurMust) + } else if (regex.endsWith(".*") && regex.length > 2 && + !QueryUtils.containsRegexChars(regex.dropRight(2))) { + // if suffix is .* and no regex special chars present in non-empty prefix, then use prefix query + visitPrefixQuery(column, regex.dropRight(2), OccurMust) + } else { + // regular non-empty regex query + visitRegexQuery(column, regex, OccurMust) + } + + case NotEqualsRegex(value) => + val term = removeRegexAnchors(value.toString) + visitStartBooleanQuery() + visitMatchAllQuery() + visitRegexQuery(column, term, OccurMustNot) + visitEndBooleanQuery() + + case Equals(value) => + equalsQuery(value.toString) + + case NotEquals(value) => + val str = value.toString + visitStartBooleanQuery() + str.isEmpty match { + case true => + visitRegexQuery(column, ".*", OccurMust) + case false => + visitMatchAllQuery() + } + visitEqualsQuery(column, str, OccurMustNot) + visitEndBooleanQuery() + + case In(values) => + visitTermInQuery(column, values.toArray.map(t => t.toString), OccurMust) + + case And(lhs, rhs) => + visitStartBooleanQuery() + visitFilter(column, lhs) + visitFilter(column, rhs) + visitEndBooleanQuery() + + case _ => throw new UnsupportedOperationException + } + } + //scalastyle:on method.length +} + +object PartKeyQueryBuilder { + + /** + * Remove leading anchor ^ and ending anchor $. + * + * @param regex the orignal regex string. + * @return the regex string without anchors. + */ + def removeRegexAnchors(regex: String): String = { + removeRegexTailDollarSign(regex.stripPrefix("^")) + } + + private def removeRegexTailDollarSign(regex: String): String = { + // avoid unnecessary calculation when $ is not present at the end of the regex. + if (regex.nonEmpty && regex.last == '$' && Pattern.matches("""^(|.*[^\\])(\\\\)*\$$""", regex)) { + // (|.*[^\\]) means either empty or a sequence end with a character not \. + // (\\\\)* means any number of \\. + // remove the last $ if it is not \$. + // $ at locations other than the end will not be removed. + regex.substring(0, regex.length - 1) + } else { + regex + } + } +} + +/** + * Enumeration of occur values for a query - MUST vs MUST_NOT + */ +sealed trait PartKeyQueryOccur + +/** + * Query must match + */ +case object OccurMust extends PartKeyQueryOccur + +/** + * Query must not match + */ +case object OccurMustNot extends PartKeyQueryOccur \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 1cacce1d34..ac35863739 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -1,25 +1,19 @@ package filodb.core.memstore import java.io.File -import java.io.IOException +import java.lang.management.{BufferPoolMXBean, ManagementFactory} import java.nio.charset.StandardCharsets import java.util import java.util.{Base64, PriorityQueue} -import java.util.regex.Pattern import scala.collection.JavaConverters._ -import scala.collection.immutable.HashSet import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.util.Failure -import scala.util.Success -import scala.util.Try import com.github.benmanes.caffeine.cache.{Caffeine, LoadingCache} import com.googlecode.javaewah.{EWAHCompressedBitmap, IntIterator} import com.typesafe.scalalogging.StrictLogging import kamon.Kamon -import kamon.metric.MeasurementUnit import org.apache.lucene.analysis.standard.StandardAnalyzer import org.apache.lucene.document._ import org.apache.lucene.document.Field.Store @@ -34,40 +28,22 @@ import org.apache.lucene.util.{BytesRef, InfoStream} import org.apache.lucene.util.automaton.RegExp import spire.syntax.cfor._ -import filodb.core.{concurrentCache, DatasetRef} +import filodb.core.DatasetRef import filodb.core.Types.PartitionKey -import filodb.core.binaryrecord2.{MapItemConsumer, RecordSchema} +import filodb.core.binaryrecord2.RecordSchema import filodb.core.memstore.ratelimit.CardinalityTracker import filodb.core.metadata.{PartitionSchema, Schemas} -import filodb.core.metadata.Column.ColumnType.{MapColumn, StringColumn} -import filodb.core.query.{ColumnFilter, Filter, QueryUtils} -import filodb.core.query.Filter._ -import filodb.memory.{BinaryRegionLarge, UTF8StringMedium, UTF8StringShort} +import filodb.core.query.{ColumnFilter, Filter} +import filodb.memory.BinaryRegionLarge import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String => UTF8Str} object PartKeyLuceneIndex { - // NOTE: these partId fields need to be separate because Lucene 9.7.0 enforces consistent types for document - // field values (i.e. a field cannot have both numeric and string values). Additional details can be found - // here: https://github.com/apache/lucene/pull/11 - final val PART_ID_DV = "__partIdDv__" - final val PART_ID_FIELD = "__partIdField__" - final val START_TIME = "__startTime__" - final val END_TIME = "__endTime__" - final val PART_KEY = "__partKey__" - final val LABEL_LIST = s"__labelList__" - final val FACET_FIELD_PREFIX = "$facet_" - final val LABEL_LIST_FACET = FACET_FIELD_PREFIX + LABEL_LIST - - final val ignoreIndexNames = HashSet(START_TIME, PART_KEY, END_TIME, PART_ID_FIELD, PART_ID_DV) - val MAX_STR_INTERN_ENTRIES = 10000 val MAX_TERMS_TO_ITERATE = 10000 val FACET_FIELD_MAX_LEN = 1000 val NOT_FOUND = -1 - def bytesRefToUnsafeOffset(bytesRefOffset: Int): Int = bytesRefOffset + UnsafeUtils.arayOffset - def unsafeOffsetToBytesRefOffset(offset: Long): Int = offset.toInt - UnsafeUtils.arayOffset def partKeyBytesRef(partKeyBase: Array[Byte], partKeyOffset: Long): BytesRef = { @@ -75,19 +51,6 @@ object PartKeyLuceneIndex { BinaryRegionLarge.numBytes(partKeyBase, partKeyOffset)) } - def defaultTempDir(ref: DatasetRef, shardNum: Int): File = { - val baseDir = new File(System.getProperty("java.io.tmpdir")) - val baseName = s"partKeyIndex-$ref-$shardNum-${System.currentTimeMillis()}-" - val tempDir = new File(baseDir, baseName) - tempDir - } - - private def createTempDir(ref: DatasetRef, shardNum: Int): File = { - val tempDir = defaultTempDir(ref, shardNum) - tempDir.mkdir() - tempDir - } - def partKeyByteRefToSHA256Digest(bytes: Array[Byte], offset: Int, length: Int): String = { import java.security.MessageDigest val md: MessageDigest = MessageDigest.getInstance("SHA-256") @@ -95,29 +58,6 @@ object PartKeyLuceneIndex { val strDigest = Base64.getEncoder.encodeToString(md.digest()) strDigest } - - private def removeRegexTailDollarSign(regex: String): String = { - // avoid unnecessary calculation when $ is not present at the end of the regex. - if (regex.nonEmpty && regex.last == '$' && Pattern.matches("""^(|.*[^\\])(\\\\)*\$$""", regex)) { - // (|.*[^\\]) means either empty or a sequence end with a character not \. - // (\\\\)* means any number of \\. - // remove the last $ if it is not \$. - // $ at locations other than the end will not be removed. - regex.substring(0, regex.length - 1) - } else { - regex - } - } - - /** - * Remove leading anchor ^ and ending anchor $. - * - * @param regex the orignal regex string. - * @return the regex string without anchors. - */ - def removeRegexAnchors(regex: String): String = { - removeRegexTailDollarSign(regex.stripPrefix("^")) - } } final case class TermInfo(term: UTF8Str, freq: Int) @@ -131,70 +71,20 @@ class PartKeyLuceneIndex(ref: DatasetRef, shardNum: Int, retentionMillis: Long, // only used to calculate fallback startTime diskLocation: Option[File] = None, - val lifecycleManager: Option[IndexMetadataStore] = None, + lifecycleManager: Option[IndexMetadataStore] = None, useMemoryMappedImpl: Boolean = true, disableIndexCaching: Boolean = false, addMetricTypeField: Boolean = true - ) extends StrictLogging with PartKeyIndexDownsampled { + ) extends PartKeyIndexDownsampled(ref, shardNum, schema, diskLocation, lifecycleManager, + addMetricTypeField = addMetricTypeField) { import PartKeyLuceneIndex._ - - val startTimeLookupLatency = Kamon.histogram("index-startTimes-for-odp-lookup-latency", - MeasurementUnit.time.nanoseconds) - .withTag("dataset", ref.dataset) - .withTag("shard", shardNum) - - val queryIndexLookupLatency = Kamon.histogram("index-partition-lookup-latency", - MeasurementUnit.time.nanoseconds) - .withTag("dataset", ref.dataset) - .withTag("shard", shardNum) - - val partIdFromPartKeyLookupLatency = Kamon.histogram("index-ingestion-partId-lookup-latency", - MeasurementUnit.time.nanoseconds) - .withTag("dataset", ref.dataset) - .withTag("shard", shardNum) - - val labelValuesQueryLatency = Kamon.histogram("index-label-values-query-latency", - MeasurementUnit.time.nanoseconds) - .withTag("dataset", ref.dataset) - .withTag("shard", shardNum) + import PartKeyIndexRaw._ val readerStateCacheHitRate = Kamon.gauge("index-reader-state-cache-hit-rate") .withTag("dataset", ref.dataset) .withTag("shard", shardNum) - private val numPartColumns = schema.columns.length - - - val indexDiskLocation = diskLocation.map(new File(_, ref.dataset + File.separator + shardNum)) - .getOrElse(createTempDir(ref, shardNum)).toPath - - // If index rebuild is triggered or the state is Building, simply clean up the index directory and start - // index rebuild - if ( - lifecycleManager.forall(_.shouldTriggerRebuild(ref, shardNum)) - ) { - logger.info(s"Cleaning up indexDirectory=$indexDiskLocation for dataset=$ref, shard=$shardNum") - deleteRecursively(indexDiskLocation.toFile) match { - case Success(_) => // Notify the handler that the directory is now empty - logger.info(s"Cleaned directory for dataset=$ref, shard=$shardNum and index directory=$indexDiskLocation") - notifyLifecycleListener(IndexState.Empty, System.currentTimeMillis) - - case Failure(t) => // Update index state as TriggerRebuild again and rethrow the exception - logger.warn(s"Exception while deleting directory for dataset=$ref, shard=$shardNum " + - s"and index directory=$indexDiskLocation with stack trace", t) - notifyLifecycleListener(IndexState.TriggerRebuild, System.currentTimeMillis) - throw new IllegalStateException("Unable to clean up index directory", t) - } - } - //else { - // TODO here we assume there is non-empty index which we need to validate - //} - - def notifyLifecycleListener(state: IndexState.Value, time: Long): Unit = - lifecycleManager.foreach(_.updateState(ref, shardNum, state, time)) - - val fsDirectory = if (useMemoryMappedImpl) new MMapDirectory(indexDiskLocation) else @@ -215,30 +105,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, } private val indexWriter = - try { - new IndexWriterPlus(fsDirectory, createIndexWriterConfig(), ref, shardNum) - } catch { - case e: Exception => - // If an exception is thrown here there is something wrong with the index or the directory - // We will attempt once by cleaning the directory and try instantiating the index again - logger.warn(s"Index for dataset:${ref.dataset} and shard: $shardNum possibly corrupt," + - s"index directory will be cleaned up and index rebuilt", e) - deleteRecursively(indexDiskLocation.toFile) match { - case Success(_) => // Notify the handler that the directory is now empty - logger.info(s"Cleaned directory for dataset=$ref," + - s"shard=$shardNum and index directory=$indexDiskLocation") - notifyLifecycleListener(IndexState.Empty, System.currentTimeMillis) - case Failure(t) => logger.warn(s"Exception while deleting directory for dataset=$ref," + - s"shard=$shardNum and index directory=$indexDiskLocation with stack trace", t) - // If we still see failure, set the TriggerRebuild and rethrow the exception - notifyLifecycleListener(IndexState.TriggerRebuild, System.currentTimeMillis) - throw new IllegalStateException("Unable to clean up index directory", t) - } - // Retry again after cleaning up the index directory, if it fails again, something needs to be looked into. - new IndexWriterPlus(fsDirectory, createIndexWriterConfig(), ref, shardNum) - } - - private val utf8ToStrCache = concurrentCache[UTF8Str, String](PartKeyLuceneIndex.MAX_STR_INTERN_ENTRIES) + loadIndexData(() => new IndexWriterPlus(fsDirectory, createIndexWriterConfig(), ref, shardNum)) //scalastyle:off private val searcherManager = @@ -270,35 +137,6 @@ class PartKeyLuceneIndex(ref: DatasetRef, facetEnabledAllLabels || (facetEnabledShardKeyLabels && schema.options.shardKeyColumns.contains(label)) } - private def deleteRecursively(f: File, deleteRoot: Boolean = false): Try[Boolean] = { - val subDirDeletion: Try[Boolean] = - if (f.isDirectory) - f.listFiles match { - case xs: Array[File] if xs != null && !xs.isEmpty => - val subDirDeletions: Array[Try[Boolean]] = xs map (f => deleteRecursively(f, true)) - subDirDeletions reduce((reduced, thisOne) => { - thisOne match { - // Ensures even if one Right(_) is found, thr response will be Right(Throwable) - case Success(_) if reduced == Success(true) => thisOne - case Failure(_) => thisOne - case _ => reduced - } - }) - case _ => Success(true) - } - else - Success(true) - - subDirDeletion match { - case Success(_) => - if (deleteRoot) { - if (f.delete()) Success(true) else Failure(new IOException(s"Unable to delete $f")) - } else Success(true) - case right @ Failure(_) => right - } - - } - case class ReusableLuceneDocument() { var facetsConfig: FacetsConfig = _ @@ -384,48 +222,6 @@ class PartKeyLuceneIndex(ref: DatasetRef, override def initialValue(): ReusableLuceneDocument = new ReusableLuceneDocument } - private val mapConsumer = new MapItemConsumer { - def consume(keyBase: Any, keyOffset: Long, valueBase: Any, valueOffset: Long, index: Int): Unit = { - import filodb.core._ - val key = utf8ToStrCache.getOrElseUpdate(new UTF8Str(keyBase, keyOffset + 1, - UTF8StringShort.numBytes(keyBase, keyOffset)), - _.toString) - val value = new String(valueBase.asInstanceOf[Array[Byte]], - unsafeOffsetToBytesRefOffset(valueOffset + 2), // add 2 to move past numBytes - UTF8StringMedium.numBytes(valueBase, valueOffset), StandardCharsets.UTF_8) - addIndexedField(key, value, clientData = true) - } - } - - /** - * Map of partKey column to the logic for indexing the column (aka Indexer). - * Optimization to avoid match logic while iterating through each column of the partKey - */ - private final val indexers = schema.columns.zipWithIndex.map { case (c, pos) => - c.columnType match { - case StringColumn => new Indexer { - val colName = UTF8Str(c.name) - def fromPartKey(base: Any, offset: Long, partIndex: Int): Unit = { - val strOffset = schema.binSchema.blobOffset(base, offset, pos) - val numBytes = schema.binSchema.blobNumBytes(base, offset, pos) - val value = new String(base.asInstanceOf[Array[Byte]], strOffset.toInt - UnsafeUtils.arayOffset, - numBytes, StandardCharsets.UTF_8) - addIndexedField(colName.toString, value, clientData = true) - } - def getNamesValues(key: PartitionKey): Seq[(UTF8Str, UTF8Str)] = ??? // not used - } - case MapColumn => new Indexer { - def fromPartKey(base: Any, offset: Long, partIndex: Int): Unit = { - schema.binSchema.consumeMapItems(base, offset, pos, mapConsumer) - } - def getNamesValues(key: PartitionKey): Seq[(UTF8Str, UTF8Str)] = ??? // not used - } - case other: Any => - logger.warn(s"Column $c has type that cannot be indexed and will be ignored right now") - NoOpIndexer - } - }.toArray - def getCurrentIndexState(): (IndexState.Value, Option[Long]) = lifecycleManager.map(_.currentState(this.ref, this.shardNum)).getOrElse((IndexState.Empty, None)) @@ -446,14 +242,14 @@ class PartKeyLuceneIndex(ref: DatasetRef, def partIdsEndedBefore(endedBefore: Long): debox.Buffer[Int] = { val collector = new PartIdCollector(Int.MaxValue) - val deleteQuery = LongPoint.newRangeQuery(PartKeyLuceneIndex.END_TIME, 0, endedBefore) + val deleteQuery = LongPoint.newRangeQuery(END_TIME, 0, endedBefore) withNewSearcher(s => s.search(deleteQuery, collector)) collector.result } def removePartitionsEndedBefore(endedBefore: Long, returnApproxDeletedCount: Boolean = true): Int = { - val deleteQuery = LongPoint.newRangeQuery(PartKeyLuceneIndex.END_TIME, 0, endedBefore) + val deleteQuery = LongPoint.newRangeQuery(END_TIME, 0, endedBefore) // SInce delete does not return the deleted document count, we query to get the count that match the filter criteria // and then delete the documents val approxDeletedCount = if (returnApproxDeletedCount) { @@ -493,7 +289,10 @@ class PartKeyLuceneIndex(ref: DatasetRef, def indexNumEntries: Long = indexWriter.getDocStats().numDocs - def indexNumEntriesWithTombstones: Long = indexWriter.getDocStats().maxDoc + def indexMmapBytes: Long = { + ManagementFactory.getPlatformMXBeans(classOf[BufferPoolMXBean]).asScala + .find(_.getName == "mapped").get.getMemoryUsed + } def closeIndex(): Unit = { logger.info(s"Closing index on dataset=$ref shard=$shardNum") @@ -529,7 +328,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, } def labelValuesEfficient(colFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, - colName: String, limit: Int = 100): Seq[String] = { + colName: String, limit: Int = LABEL_NAMES_AND_VALUES_DEFAULT_LIMIT): Seq[String] = { require(facetEnabledForLabel(colName), s"Faceting not enabled for label $colName; labelValuesEfficient should not have been called") @@ -619,23 +418,12 @@ class PartKeyLuceneIndex(ref: DatasetRef, ret } - /** - * - * @param clientData pass true if the field data has come from metric source, and false if internally setting the - * field data. This is used to determine if the type field data should be indexed or not. - */ - private def addIndexedField(labelName: String, value: String, clientData: Boolean): Unit = { - if (clientData && addMetricTypeField) { - // do not index any existing _type_ tag since this is reserved and should not be passed in by clients - if (labelName != Schemas.TypeLabel) - luceneDocument.get().addField(labelName, value) - else - logger.warn("Map column with name '_type_' is a reserved label. Not indexing it.") - // I would have liked to log the entire PK to debug, but it is not accessible from here. - // Ignoring for now, since the plan of record is to drop reserved labels at ingestion gateway. - } else { - luceneDocument.get().addField(labelName, value) - } + protected def addIndexedField(labelName: String, value: String): Unit = { + luceneDocument.get().addField(labelName, value) + } + + protected def addIndexedMapField(mapColumn: String, key: String, value: String): Unit = { + luceneDocument.get().addField(key, value) } def addPartKey(partKeyOnHeapBytes: Array[Byte], @@ -669,19 +457,6 @@ class PartKeyLuceneIndex(ref: DatasetRef, indexWriter.updateDocument(term, docToAdd) } - - private def partKeyString(docId: String, - partKeyOnHeapBytes: Array[Byte], - partKeyBytesRefOffset: Int = 0): String = { - val partHash = schema.binSchema.partitionHash(partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset)) - //scalastyle:off - s"shard=$shardNum partId=$docId partHash=$partHash [${ - TimeSeriesPartition - .partKeyString(schema, partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset)) - }]" - //scalastyle:on - } - private def makeDocument(partKeyOnHeapBytes: Array[Byte], partKeyBytesRefOffset: Int, partKeyNumBytes: Int, @@ -694,7 +469,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, val schemaName = Schemas.global.schemaName(RecordSchema.schemaID(partKeyOnHeapBytes, UnsafeUtils.arayOffset)) if (addMetricTypeField) - addIndexedField(Schemas.TypeLabel, schemaName, clientData = false) + addIndexedField(Schemas.TypeLabel, schemaName) cforRange { 0 until numPartColumns } { i => indexers(i).fromPartKey(partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset), partId) @@ -703,29 +478,8 @@ class PartKeyLuceneIndex(ref: DatasetRef, luceneDocument.get().document } - private final val emptyStr = "" - // Multi-column facets to be created on "partition-schema" columns - private def createMultiColumnFacets(partKeyOnHeapBytes: Array[Byte], partKeyBytesRefOffset: Int): Unit = { - schema.options.multiColumnFacets.foreach(facetCols => { - facetCols match { - case (name, cols) => { - val concatFacetValue = cols.map { col => - val colInfoOpt = schema.columnIdxLookup.get(col) - colInfoOpt match { - case Some((columnInfo, pos)) if columnInfo.columnType == StringColumn => - val base = partKeyOnHeapBytes - val offset = bytesRefToUnsafeOffset(partKeyBytesRefOffset) - val strOffset = schema.binSchema.blobOffset(base, offset, pos) - val numBytes = schema.binSchema.blobNumBytes(base, offset, pos) - new String(base, strOffset.toInt - UnsafeUtils.arayOffset, - numBytes, StandardCharsets.UTF_8) - case _ => emptyStr - } - }.mkString("\u03C0") - luceneDocument.get().addFacet(name, concatFacetValue, false) - } - } - }) + protected override def addMultiColumnFacet(key: String, value: String): Unit = { + luceneDocument.get().addFacet(key, value, false) } def partKeyFromPartId(partId: Int): Option[BytesRef] = { @@ -735,7 +489,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, } def startTimeFromPartId(partId: Int): Long = { - val collector = new NumericDocValueCollector(PartKeyLuceneIndex.START_TIME) + val collector = new NumericDocValueCollector(START_TIME) withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector)) collector.singleResult } @@ -759,10 +513,10 @@ class PartKeyLuceneIndex(ref: DatasetRef, collector.startTimes } - def commit(): Long = indexWriter.commit() + def commit(): Unit = indexWriter.commit() def endTimeFromPartId(partId: Int): Long = { - val collector = new NumericDocValueCollector(PartKeyLuceneIndex.END_TIME) + val collector = new NumericDocValueCollector(END_TIME) withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector)) collector.singleResult } @@ -822,78 +576,6 @@ class PartKeyLuceneIndex(ref: DatasetRef, logger.info(s"Refreshed index searchers to make reads consistent for dataset=$ref shard=$shardNum") } - // scalastyle:off method.length - private def leafFilter(column: String, filter: Filter): Query = { - - def equalsQuery(value: String): Query = { - if (value.nonEmpty) new TermQuery(new Term(column, value)) - else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. - } - - filter match { - case EqualsRegex(value) => - val regex = removeRegexAnchors(value.toString) - if (regex == "") { - // if label=~"" then match empty string or label not present condition too - leafFilter(column, NotEqualsRegex(".+")) - } else if (regex.replaceAll("\\.\\*", "") == "") { - // if label=~".*" then match all docs since promQL matches .* with absent label too - new MatchAllDocsQuery - } else if (!QueryUtils.containsRegexChars(regex)) { - // if all regex special chars absent, then treat like Equals - equalsQuery(regex) - } 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 && - !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 { - // regular non-empty regex query - new RegexpQuery(new Term(column, regex), RegExp.NONE) - } - - case NotEqualsRegex(value) => - val term = new Term(column, removeRegexAnchors(value.toString)) - val allDocs = new MatchAllDocsQuery - val booleanQuery = new BooleanQuery.Builder - booleanQuery.add(allDocs, Occur.FILTER) - booleanQuery.add(new RegexpQuery(term, RegExp.NONE), Occur.MUST_NOT) - booleanQuery.build() - - case Equals(value) => - equalsQuery(value.toString) - - case NotEquals(value) => - val str = value.toString - val term = new Term(column, str) - val booleanQuery = new BooleanQuery.Builder - str.isEmpty match { - case true => - val termAll = new Term(column, ".*") - booleanQuery.add(new RegexpQuery(termAll, RegExp.NONE), Occur.FILTER) - case false => - val allDocs = new MatchAllDocsQuery - booleanQuery.add(allDocs, Occur.FILTER) - } - booleanQuery.add(new TermQuery(term), Occur.MUST_NOT) - booleanQuery.build() - - case In(values) => - new TermInSetQuery(column, values.toArray.map(t => new BytesRef(t.toString)): _*) - - case And(lhs, rhs) => - val andQuery = new BooleanQuery.Builder - andQuery.add(leafFilter(column, lhs), Occur.FILTER) - andQuery.add(leafFilter(column, rhs), Occur.FILTER) - andQuery.build() - - case _ => throw new UnsupportedOperationException - } - } - //scalastyle:on method.length - def partIdsFromFilters(columnFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, @@ -950,15 +632,8 @@ class PartKeyLuceneIndex(ref: DatasetRef, } private def colFiltersToQuery(columnFilters: Seq[ColumnFilter], startTime: PartitionKey, endTime: PartitionKey) = { - val booleanQuery = new BooleanQuery.Builder - columnFilters.foreach { filter => - val q = leafFilter(filter.column, filter.filter) - booleanQuery.add(q, Occur.FILTER) - } - booleanQuery.add(LongPoint.newRangeQuery(START_TIME, 0, endTime), Occur.FILTER) - booleanQuery.add(LongPoint.newRangeQuery(END_TIME, startTime, Long.MaxValue), Occur.FILTER) - val query = booleanQuery.build() - new ConstantScoreQuery(query) // disable scoring + val queryBuilder = new LuceneQueryBuilder() + queryBuilder.buildQueryWithStartAndEnd(columnFilters, startTime, endTime) } def partIdFromPartKeySlow(partKeyBase: Any, @@ -968,18 +643,15 @@ class PartKeyLuceneIndex(ref: DatasetRef, .map { pair => ColumnFilter(pair._1, Filter.Equals(pair._2)) } val startExecute = System.nanoTime() - val booleanQuery = new BooleanQuery.Builder - columnFilters.foreach { filter => - val q = leafFilter(filter.column, filter.filter) - booleanQuery.add(q, Occur.FILTER) - } - val query = booleanQuery.build() + val queryBuilder = new LuceneQueryBuilder() + val query = queryBuilder.buildQuery(columnFilters) + logger.debug(s"Querying dataset=$ref shard=$shardNum partKeyIndex with: $query") var chosenPartId: Option[Int] = None def handleMatch(partId: Int, candidate: BytesRef): Unit = { // we need an equals check because there can potentially be another partKey with additional tags if (schema.binSchema.equals(partKeyBase, partKeyOffset, - candidate.bytes, PartKeyLuceneIndex.bytesRefToUnsafeOffset(candidate.offset))) { + candidate.bytes, PartKeyIndexRaw.bytesRefToUnsafeOffset(candidate.offset))) { logger.debug(s"There is already a partId=$partId assigned for " + s"${schema.binSchema.stringify(partKeyBase, partKeyOffset)} in" + s" dataset=$ref shard=$shardNum") @@ -1000,6 +672,100 @@ class PartKeyLuceneIndex(ref: DatasetRef, } } +protected class LuceneQueryBuilder extends PartKeyQueryBuilder { + + private val stack: mutable.ArrayStack[BooleanQuery.Builder] = mutable.ArrayStack() + + private def toLuceneOccur(occur: PartKeyQueryOccur): Occur = { + occur match { + case OccurMust => Occur.MUST + case OccurMustNot => Occur.MUST_NOT + } + } + + override protected def visitStartBooleanQuery(): Unit = { + stack.push(new BooleanQuery.Builder) + } + + override protected def visitEndBooleanQuery(): Unit = { + val builder = stack.pop() + val query = builder.build() + + val parent = stack.top + parent.add(query, Occur.FILTER) + } + + override protected def visitEqualsQuery(column: String, term: String, occur: PartKeyQueryOccur): Unit = { + val query = new TermQuery(new Term(column, term)) + + val parent = stack.top + parent.add(query, toLuceneOccur(occur)) + } + + override protected def visitRegexQuery(column: String, pattern: String, occur: PartKeyQueryOccur): Unit = { + val query = new RegexpQuery(new Term(column, pattern), RegExp.NONE) + + val parent = stack.top + parent.add(query, toLuceneOccur(occur)) + } + + override protected def visitTermInQuery(column: String, terms: Seq[String], occur: PartKeyQueryOccur): Unit = { + val query = new TermInSetQuery(column, terms.toArray.map(t => new BytesRef(t)): _*) + + val parent = stack.top + parent.add(query, toLuceneOccur(occur)) + } + + override protected def visitPrefixQuery(column: String, prefix: String, occur: PartKeyQueryOccur): Unit = { + val query = new PrefixQuery(new Term(column, prefix)) + + val parent = stack.top + parent.add(query, toLuceneOccur(occur)) + } + + override protected def visitMatchAllQuery(): Unit = { + val query = new MatchAllDocsQuery + + val parent = stack.top + parent.add(query, Occur.MUST) + } + + override protected def visitRangeQuery(column: String, start: Long, end: Long, + occur: PartKeyQueryOccur): Unit = { + val query = LongPoint.newRangeQuery(column, start, end) + + val parent = stack.top + parent.add(query, toLuceneOccur(occur)) + } + + def buildQuery(columnFilters: Seq[ColumnFilter]): Query = { + visitStartBooleanQuery() + visitQuery(columnFilters) + + val builder = stack.pop() + if (stack.nonEmpty) { + // Should never happen given inputs, sanity check on invalid queries + throw new RuntimeException("Query stack not empty after processing") + } + val query = builder.build() + new ConstantScoreQuery(query) // disable scoring + } + + def buildQueryWithStartAndEnd(columnFilters: Seq[ColumnFilter], startTime: PartitionKey, + endTime: PartitionKey): Query = { + visitStartBooleanQuery() + visitQueryWithStartAndEnd(columnFilters, startTime, endTime) + + val builder = stack.pop() + if (stack.nonEmpty) { + // Should never happen given inputs, sanity check on invalid queries + throw new RuntimeException("Query stack not empty after processing") + } + val query = builder.build() + new ConstantScoreQuery(query) // disable scoring + } +} + /** * In this lucene index collector, we read through the entire lucene index periodically and re-calculate * the cardinality count from scratch. This class iterates through each document in lucene, extracts a shard-key @@ -1012,14 +778,14 @@ class CardinalityCountBuilder(partSchema: PartitionSchema, cardTracker: Cardinal // gets called for each segment override def doSetNextReader(context: LeafReaderContext): Unit = { - partKeyDv = context.reader().getBinaryDocValues(PartKeyLuceneIndex.PART_KEY) + partKeyDv = context.reader().getBinaryDocValues(PartKeyIndexRaw.PART_KEY) } // gets called for each matching document in current segment override def collect(doc: Int): Unit = { if (partKeyDv.advanceExact(doc)) { val binaryValue = partKeyDv.binaryValue() - val unsafePkOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(binaryValue.offset) + val unsafePkOffset = PartKeyIndexRaw.bytesRefToUnsafeOffset(binaryValue.offset) val shardKey = partSchema.binSchema.colValues( binaryValue.bytes, unsafePkOffset, partSchema.options.shardKeyColumns) @@ -1067,7 +833,7 @@ class SinglePartKeyCollector extends SimpleCollector { // gets called for each segment override def doSetNextReader(context: LeafReaderContext): Unit = { - partKeyDv = context.reader().getBinaryDocValues(PartKeyLuceneIndex.PART_KEY) + partKeyDv = context.reader().getBinaryDocValues(PartKeyIndexRaw.PART_KEY) } // gets called for each matching document in current segment @@ -1091,7 +857,7 @@ class SinglePartIdCollector extends SimpleCollector { // gets called for each segment override def doSetNextReader(context: LeafReaderContext): Unit = { - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) + partIdDv = context.reader().getNumericDocValues(PartKeyIndexRaw.PART_ID_DV) } // gets called for each matching document in current segment @@ -1118,7 +884,7 @@ class SinglePartIdCollector extends SimpleCollector { */ class TopKPartIdsCollector(limit: Int) extends Collector with StrictLogging { - import PartKeyLuceneIndex._ + import PartKeyIndexRaw._ var endTimeDv: NumericDocValues = _ var partIdDv: NumericDocValues = _ @@ -1178,7 +944,7 @@ class PartIdCollector(limit: Int) extends SimpleCollector { override def doSetNextReader(context: LeafReaderContext): Unit = { //set the subarray of the numeric values for all documents in the context - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) + partIdDv = context.reader().getNumericDocValues(PartKeyIndexRaw.PART_ID_DV) } override def collect(doc: Int): Unit = { @@ -1201,8 +967,8 @@ class PartIdStartTimeCollector extends SimpleCollector { override def doSetNextReader(context: LeafReaderContext): Unit = { //set the subarray of the numeric values for all documents in the context - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) - startTimeDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.START_TIME) + partIdDv = context.reader().getNumericDocValues(PartKeyIndexRaw.PART_ID_DV) + startTimeDv = context.reader().getNumericDocValues(PartKeyIndexRaw.START_TIME) } override def collect(doc: Int): Unit = { @@ -1225,9 +991,9 @@ class PartKeyRecordCollector(limit: Int) extends SimpleCollector { override def scoreMode(): ScoreMode = ScoreMode.COMPLETE_NO_SCORES override def doSetNextReader(context: LeafReaderContext): Unit = { - partKeyDv = context.reader().getBinaryDocValues(PartKeyLuceneIndex.PART_KEY) - startTimeDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.START_TIME) - endTimeDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.END_TIME) + partKeyDv = context.reader().getBinaryDocValues(PartKeyIndexRaw.PART_KEY) + startTimeDv = context.reader().getNumericDocValues(PartKeyIndexRaw.START_TIME) + endTimeDv = context.reader().getNumericDocValues(PartKeyIndexRaw.END_TIME) } override def collect(doc: Int): Unit = { @@ -1252,8 +1018,8 @@ class ActionCollector(action: (Int, BytesRef) => Unit) extends SimpleCollector { override def scoreMode(): ScoreMode = ScoreMode.COMPLETE_NO_SCORES override def doSetNextReader(context: LeafReaderContext): Unit = { - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) - partKeyDv = context.reader().getBinaryDocValues(PartKeyLuceneIndex.PART_KEY) + partIdDv = context.reader().getNumericDocValues(PartKeyIndexRaw.PART_ID_DV) + partKeyDv = context.reader().getBinaryDocValues(PartKeyIndexRaw.PART_KEY) } override def collect(doc: Int): Unit = { @@ -1277,7 +1043,7 @@ class PartKeyActionCollector(action: (BytesRef) => Unit) extends SimpleCollector override def scoreMode(): ScoreMode = ScoreMode.COMPLETE_NO_SCORES override def doSetNextReader(context: LeafReaderContext): Unit = { - partKeyDv = context.reader().getBinaryDocValues(PartKeyLuceneIndex.PART_KEY) + partKeyDv = context.reader().getBinaryDocValues(PartKeyIndexRaw.PART_KEY) } override def collect(doc: Int): Unit = { diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyTantivyIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyTantivyIndex.scala new file mode 100644 index 0000000000..8b24c0caa7 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/PartKeyTantivyIndex.scala @@ -0,0 +1,739 @@ +package filodb.core.memstore + +import java.io.File +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets +import java.nio.file.Files +import java.util.concurrent.{ScheduledThreadPoolExecutor, TimeUnit} + +import scala.collection.mutable.ArrayBuffer + +import com.typesafe.scalalogging.StrictLogging +import debox.Buffer +import kamon.Kamon +import kamon.metric.MeasurementUnit +import org.apache.commons.lang3.SystemUtils +import org.apache.lucene.util.BytesRef +import spire.implicits.cforRange + +import filodb.core.DatasetRef +import filodb.core.binaryrecord2.RecordSchema +import filodb.core.memstore.PartKeyIndexRaw.{bytesRefToUnsafeOffset, ignoreIndexNames, FACET_FIELD_PREFIX, + PART_ID_FIELD} +import filodb.core.metadata.{PartitionSchema, Schemas} +import filodb.core.metadata.Column.ColumnType.{MapColumn, StringColumn} +import filodb.core.query.{ColumnFilter, Filter} +import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} + +object PartKeyTantivyIndex { + def startMemoryProfiling(): Unit = { + TantivyNativeMethods.startMemoryProfiling() + } + + def stopMemoryProfiling(): Unit = { + TantivyNativeMethods.stopMemoryProfiling() + } +} + +class PartKeyTantivyIndex(ref: DatasetRef, + schema: PartitionSchema, + shardNum: Int, + retentionMillis: Long, // only used to calculate fallback startTime + diskLocation: Option[File] = None, + lifecycleManager: Option[IndexMetadataStore] = None, + columnCacheCount: Long = 1000, + queryCacheMaxSize: Long = 50 * 1000 * 1000, + queryCacheEstimatedItemSize: Long = 31250, + deletedDocMergeThreshold: Float = 0.1f, + addMetricTypeField: Boolean = true + ) extends PartKeyIndexRaw(ref, shardNum, schema, diskLocation, lifecycleManager, + addMetricTypeField = addMetricTypeField) { + + private val cacheHitRate = Kamon.gauge("index-tantivy-cache-hit-rate") + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + private val refreshLatency = Kamon.histogram("index-tantivy-commit-refresh-latency", + MeasurementUnit.time.nanoseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + // Compute field names for native schema code + private val schemaFields = schema.columns.filter { c => + c.columnType == StringColumn + }.map { c => + c.name + }.toArray + + private val schemaMapFields = schema.columns.filter { c => + c.columnType == MapColumn + }.map { c => + c.name + }.toArray + + private val schemaMultiColumnFacets = schema.options.multiColumnFacets.keys.toArray + + // Native handle for cross JNI operations + private var indexHandle: Long = loadIndexData(() => TantivyNativeMethods.newIndexHandle(indexDiskLocation.toString, + schemaFields, schemaMapFields, schemaMultiColumnFacets, columnCacheCount, queryCacheMaxSize, + queryCacheEstimatedItemSize, deletedDocMergeThreshold)) + + logger.info(s"Created tantivy index for dataset=$ref shard=$shardNum at $indexDiskLocation") + + override def reset(): Unit = { + TantivyNativeMethods.reset(indexHandle) + } + + private var flushThreadPool: ScheduledThreadPoolExecutor = _ + + override def startFlushThread(flushDelayMinSeconds: Int, flushDelayMaxSeconds: Int): Unit = { + if (flushThreadPool != UnsafeUtils.ZeroPointer) { + // Already running + logger.warn("startFlushThread called when already running, ignoring") + return + } + + flushThreadPool = new ScheduledThreadPoolExecutor(1) + + flushThreadPool.scheduleAtFixedRate(() => { + // Commit / refresh + val start = System.nanoTime() + refreshReadersBlocking() + val elapsed = System.nanoTime() - start + refreshLatency.record(elapsed) + + // Emit cache stats + val cache_stats = TantivyNativeMethods.getCacheHitRates(indexHandle) + + cacheHitRate.withTag("label", "query").update(cache_stats(0)) + cacheHitRate.withTag("label", "column").update(cache_stats(1)) + }, flushDelayMinSeconds, + flushDelayMinSeconds, TimeUnit.SECONDS) + } + + override def partIdsEndedBefore(endedBefore: Long): Buffer[Int] = { + val result: debox.Buffer[Int] = debox.Buffer.empty[Int] + val partIds = TantivyNativeMethods.partIdsEndedBefore(indexHandle, endedBefore) + + result.extend(partIds) + + result + } + + override def removePartitionsEndedBefore(endedBefore: Long, returnApproxDeletedCount: Boolean): Int = { + TantivyNativeMethods.removePartitionsEndedBefore(indexHandle, endedBefore, returnApproxDeletedCount) + } + + override def removePartKeys(partIds: Buffer[Int]): Unit = { + if (!partIds.isEmpty) { + TantivyNativeMethods.removePartKeys(indexHandle, partIds.toArray) + } + } + + override def indexRamBytes: Long = { + TantivyNativeMethods.indexRamBytes(indexHandle) + } + + override def indexNumEntries: Long = { + TantivyNativeMethods.indexNumEntries(indexHandle) + } + + override def indexMmapBytes: Long = { + TantivyNativeMethods.indexMmapBytes(indexHandle) + } + + override def closeIndex(): Unit = { + logger.info(s"Closing index on dataset=$ref shard=$shardNum") + + if (flushThreadPool != UnsafeUtils.ZeroPointer) { + flushThreadPool.shutdown() + flushThreadPool.awaitTermination(60, TimeUnit.SECONDS) + } + + commit() + TantivyNativeMethods.freeIndexHandle(indexHandle) + indexHandle = 0 + } + + override def indexNames(limit: Int): Seq[String] = { + decodeStringArray(TantivyNativeMethods.indexNames(indexHandle)).filterNot { + n => ignoreIndexNames.contains(n) || n.startsWith(FACET_FIELD_PREFIX) + } + } + + override def indexValues(fieldName: String, topK: Int): Seq[TermInfo] = { + val results = TantivyNativeMethods.indexValues(indexHandle, fieldName, topK) + + val buffer = ByteBuffer.wrap(results) + buffer.order(ByteOrder.LITTLE_ENDIAN) + + val parsedResults = new ArrayBuffer[TermInfo]() + + while (buffer.hasRemaining) { + val count = buffer.getLong + val strLen = buffer.getInt + val strBytes = new Array[Byte](strLen) + buffer.get(strBytes) + + parsedResults += TermInfo(ZeroCopyUTF8String.apply(strBytes), count.toInt) + } + + parsedResults + } + + private def decodeStringArray(arr: Array[Byte]): Seq[String] = { + val buffer = ByteBuffer.wrap(arr) + buffer.order(ByteOrder.LITTLE_ENDIAN) + + val parsedResults = new ArrayBuffer[String]() + + while (buffer.hasRemaining) { + val strLen = buffer.getInt + val strBytes = new Array[Byte](strLen) + buffer.get(strBytes) + + parsedResults += new String(strBytes, StandardCharsets.UTF_8) + } + + parsedResults + } + + override def labelNamesEfficient(colFilters: Seq[ColumnFilter], startTime: Long, endTime: Long): Seq[String] = { + val start = System.nanoTime() + val queryBuilder = new TantivyQueryBuilder() + val query = queryBuilder.buildQuery(colFilters) + + val results = TantivyNativeMethods.labelNames(indexHandle, query, LABEL_NAMES_AND_VALUES_DEFAULT_LIMIT, + startTime, endTime) + + labelValuesQueryLatency.record(System.nanoTime() - start) + + decodeStringArray(results) + } + + override def labelValuesEfficient(colFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, + colName: String, limit: Int): Seq[String] = { + val start = System.nanoTime() + val queryBuilder = new TantivyQueryBuilder() + val query = queryBuilder.buildQuery(colFilters) + + val results = TantivyNativeMethods.labelValues(indexHandle, query, colName, limit, startTime, endTime) + + labelValuesQueryLatency.record(System.nanoTime() - start) + + decodeStringArray(results) + } + + override def addPartKey(partKeyOnHeapBytes: Array[Byte], partId: Int, startTime: Long, endTime: Long, + partKeyBytesRefOffset: Int)(partKeyNumBytes: Int, documentId: String): Unit = { + logger.debug(s"Adding document ${partKeyString(documentId, partKeyOnHeapBytes, partKeyBytesRefOffset)} " + + s"with startTime=$startTime endTime=$endTime into dataset=$ref shard=$shardNum") + makeDocument(partKeyOnHeapBytes, partKeyBytesRefOffset, partKeyNumBytes, partId, documentId, startTime, endTime, + upsert = false) + } + + override def upsertPartKey(partKeyOnHeapBytes: Array[Byte], partId: Int, startTime: Long, endTime: Long, + partKeyBytesRefOffset: Int)(partKeyNumBytes: Int, documentId: String): Unit = { + logger.debug(s"Upserting document ${partKeyString(documentId, partKeyOnHeapBytes, partKeyBytesRefOffset)} " + + s"with startTime=$startTime endTime=$endTime into dataset=$ref shard=$shardNum") + makeDocument(partKeyOnHeapBytes, partKeyBytesRefOffset, partKeyNumBytes, partId, documentId, startTime, endTime, + upsert = true) + } + + override def partKeyFromPartId(partId: Int): Option[BytesRef] = { + val results = searchFromFilters(Seq(ColumnFilter(PART_ID_FIELD, Filter.Equals(partId.toString))), + 0, Long.MaxValue, 1, TantivyNativeMethods.queryPartKey) + + if (results == null) { + None + } else { + Some(new BytesRef(results, 0, results.length)) + } + } + + private val NOT_FOUND = -1 + + override def startTimeFromPartId(partId: Int): Long = { + val rawResult = TantivyNativeMethods.startTimeFromPartIds(indexHandle, Seq(partId).toArray) + + if (rawResult.length == 0) { + NOT_FOUND + } else { + rawResult(1) + } + } + + override def endTimeFromPartId(partId: Int): Long = { + TantivyNativeMethods.endTimeFromPartId(indexHandle, partId) + } + + override def startTimeFromPartIds(partIds: Iterator[Int]): debox.Map[Int, Long] = { + val startExecute = System.nanoTime() + val span = Kamon.currentSpan() + val partIdsArray = partIds.toArray + + val result = debox.Map.empty[Int, Long] + val rawResult = TantivyNativeMethods.startTimeFromPartIds(indexHandle, partIdsArray) + var idx = 0 + while (idx < rawResult.length) { + result.update(rawResult(idx).toInt, rawResult(idx + 1)) + idx += 2 + } + + span.tag(s"num-partitions-to-page", partIdsArray.length) + val latency = System.nanoTime - startExecute + span.mark(s"index-startTimes-for-odp-lookup-latency=${latency}ns") + startTimeLookupLatency.record(latency) + + result + } + + override def commit(): Unit = { + TantivyNativeMethods.commit(indexHandle) + } + + override def updatePartKeyWithEndTime(partKeyOnHeapBytes: Array[Byte], partId: Int, endTime: Long, + partKeyBytesRefOffset: Int)(partKeyNumBytes: Int, documentId: String): Unit = { + var startTime = startTimeFromPartId(partId) // look up index for old start time + if (startTime == NOT_FOUND) { + startTime = System.currentTimeMillis() - retentionMillis + logger.warn(s"Could not find in Lucene startTime for partId=$partId in dataset=$ref. Using " + + s"$startTime instead.", new IllegalStateException()) // assume this time series started retention period ago + } + logger.debug(s"Updating document ${partKeyString(documentId, partKeyOnHeapBytes, partKeyBytesRefOffset)} " + + s"with startTime=$startTime endTime=$endTime into dataset=$ref shard=$shardNum") + + makeDocument(partKeyOnHeapBytes, partKeyBytesRefOffset, partKeyNumBytes, + partId, documentId, startTime, endTime, upsert = true) + } + + override def refreshReadersBlocking(): Unit = { + TantivyNativeMethods.refreshReaders(indexHandle) + } + + private def searchFromFilters[T](columnFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, + limit: Int, + searchFunc: (Long, Array[Byte], Long, Long, Long) => Array[T]): Array[T] = { + val startExecute = System.nanoTime() + val span = Kamon.currentSpan() + val queryBuilder = new TantivyQueryBuilder() + val query = queryBuilder.buildQuery(columnFilters) + val results = searchFunc(indexHandle, query, limit, startTime, endTime) + val latency = System.nanoTime - startExecute + span.mark(s"index-partition-lookup-latency=${latency}ns") + queryIndexLookupLatency.record(latency) + + results + } + + override def partIdsFromFilters(columnFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, + limit: Int): Buffer[Int] = { + val results = searchFromFilters(columnFilters, startTime, endTime, limit, TantivyNativeMethods.queryPartIds) + + // "unsafe" means you must not modify the array you're passing in after creating the buffer + // We don't, so this is more performant + debox.Buffer.unsafe(results) + } + + override def partKeyRecordsFromFilters(columnFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, + limit: Int): Seq[PartKeyLuceneIndexRecord] = { + val results = searchFromFilters(columnFilters, startTime, endTime, limit, TantivyNativeMethods.queryPartKeyRecords) + + val buffer = ByteBuffer.wrap(results) + buffer.order(ByteOrder.LITTLE_ENDIAN) + + val parsedResults = new ArrayBuffer[PartKeyLuceneIndexRecord]() + + while (buffer.hasRemaining) { + val start = buffer.getLong + val end = buffer.getLong + val pkLen = buffer.getInt + val pk = new Array[Byte](pkLen) + buffer.get(pk) + + parsedResults += PartKeyLuceneIndexRecord(pk, start, end) + } + + parsedResults + } + + override def partIdFromPartKeySlow(partKeyBase: Any, partKeyOffset: Long): Option[Int] = { + val partKey = schema.binSchema.asByteArray(partKeyBase, partKeyOffset) + val startExecute = System.nanoTime() + + val id = TantivyNativeMethods.partIdFromPartKey(indexHandle, partKey) + + partIdFromPartKeyLookupLatency.record(System.nanoTime - startExecute) + + if (id == -1) { + None + } else { + Some(id) + } + } + + override def singlePartKeyFromFilters(columnFilters: Seq[ColumnFilter], startTime: Long, + endTime: Long): Option[Array[Byte]] = { + val results = searchFromFilters(columnFilters, 0, Long.MaxValue, 1, TantivyNativeMethods.queryPartKey) + + Option(results) + } + + override protected def addIndexedField(key: String, value: String): Unit = { + val buffer = docBufferLocal.get() + + // 1 - indexed field + buffer += 1 + ByteBufferEncodingUtils.writeStringToBuffer(key, buffer) + ByteBufferEncodingUtils.writeStringToBuffer(value, buffer) + } + + protected def addIndexedMapField(mapColumn: String, key: String, value: String): Unit = { + val buffer = docBufferLocal.get() + + // 2 - map field + buffer += 2 + ByteBufferEncodingUtils.writeStringToBuffer(mapColumn, buffer) + ByteBufferEncodingUtils.writeStringToBuffer(key, buffer) + ByteBufferEncodingUtils.writeStringToBuffer(value, buffer) + } + + protected override def addMultiColumnFacet(key: String, value: String): Unit = { + val buffer = docBufferLocal.get() + + // 3 - mc field + buffer += 3 + ByteBufferEncodingUtils.writeStringToBuffer(key, buffer) + ByteBufferEncodingUtils.writeStringToBuffer(value, buffer) + } + + // Ideally this would be a map of field -> value or something similar. + // However, passing a Map to the Rust code generates a much more expensive + // back and forth between JVM code and Rust code to get data. + // + // To solve this efficiency problem we pack into a byte buffer with a simple + // serialization format that the Rust side can decode quickly without JVM + // callbacks. + private val docBufferLocal = new ThreadLocal[ArrayBuffer[Byte]]() { + override def initialValue(): ArrayBuffer[Byte] = new ArrayBuffer[Byte](4096) + } + + private def makeDocument(partKeyOnHeapBytes: Array[Byte], + partKeyBytesRefOffset: Int, + partKeyNumBytes: Int, + partId: Int, + documentId: String, + startTime: Long, + endTime: Long, + upsert: Boolean): Unit = { + docBufferLocal.get().clear() + + // If configured and enabled, Multi-column facets will be created on "partition-schema" columns + createMultiColumnFacets(partKeyOnHeapBytes, partKeyBytesRefOffset) + + val schemaName = Schemas.global.schemaName(RecordSchema.schemaID(partKeyOnHeapBytes, UnsafeUtils.arayOffset)) + if (addMetricTypeField) + addIndexedField(Schemas.TypeLabel, schemaName) + + cforRange { + 0 until numPartColumns + } { i => + indexers(i).fromPartKey(partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset), partId) + } + + TantivyNativeMethods.ingestDocument(indexHandle, partKeyOnHeapBytes, partKeyBytesRefOffset, partKeyNumBytes, + partId, documentId, startTime, endTime, docBufferLocal.get().toArray, upsert) + } + + def dumpCacheStats(): String = { + TantivyNativeMethods.dumpCacheStats(indexHandle) + } +} + +object ByteBufferEncodingUtils { + def writeStringToBuffer(s: String, buffer: ArrayBuffer[Byte]): Unit = { + val bytes = s.getBytes + writeLengthToBuffer(bytes.length, buffer) + buffer ++= bytes + } + + def writeLengthToBuffer(len: Int, buffer: ArrayBuffer[Byte]): Unit = { + buffer += len.toByte + buffer += (len >> 8).toByte + } +} + +object TantivyQueryBuilder { + private val bufferLocal = new ThreadLocal[ArrayBuffer[Byte]]() { + override def initialValue(): ArrayBuffer[Byte] = new ArrayBuffer[Byte](4096) + } +} + +class TantivyQueryBuilder extends PartKeyQueryBuilder with StrictLogging { + private final val TERMINATOR_BYTE: Byte = 0 + private final val BOOLEAN_TYPE_BYTE: Byte = 1 + private final val EQUALS_TYPE_BYTE: Byte = 2 + private final val REGEX_TYPE_BYTE: Byte = 3 + private final val TERM_IN_TYPE_BYTE: Byte = 4 + private final val PREFIX_TYPE_BYTE: Byte = 5 + private final val MATCH_ALL_TYPE_BYTE: Byte = 6 + private final val LONG_RANGE_TYPE_BYTE: Byte = 7 + + private final val OCCUR_MUST: Byte = 1 + private final val OCCUR_MUST_NOT: Byte = 2 + + private val buffer = { + val buffer = TantivyQueryBuilder.bufferLocal.get() + buffer.clear() + + buffer + } + + private def writeString(s: String): Unit = { + ByteBufferEncodingUtils.writeStringToBuffer(s, buffer) + } + + private def writeLong(v: Long): Unit = { + buffer += v.toByte + buffer += (v >> 8).toByte + buffer += (v >> 16).toByte + buffer += (v >> 24).toByte + buffer += (v >> 32).toByte + buffer += (v >> 40).toByte + buffer += (v >> 48).toByte + buffer += (v >> 56).toByte + } + + private def writeOccur(occur: PartKeyQueryOccur): Unit = { + occur match { + case OccurMust => buffer += OCCUR_MUST + case OccurMustNot => buffer += OCCUR_MUST_NOT + } + } + + override protected def visitStartBooleanQuery(): Unit = { + if (buffer.nonEmpty) { + // Nested, add occur byte + buffer += OCCUR_MUST + } + buffer += BOOLEAN_TYPE_BYTE + } + + override protected def visitEndBooleanQuery(): Unit = { + buffer += TERMINATOR_BYTE + } + + override protected def visitEqualsQuery(column: String, term: String, occur: PartKeyQueryOccur): Unit = { + writeOccur(occur) + + // Type byte, col len, col, term len, term + buffer += EQUALS_TYPE_BYTE + writeString(column) + writeString(term) + } + + override protected def visitRegexQuery(column: String, pattern: String, occur: PartKeyQueryOccur): Unit = { + writeOccur(occur) + + // Type byte, col len, col, pattern len, pattern + buffer += REGEX_TYPE_BYTE + writeString(column) + writeString(pattern) + } + + override protected def visitTermInQuery(column: String, terms: Seq[String], occur: PartKeyQueryOccur): Unit = { + writeOccur(occur) + + // Type byte, column len, column bytes, (for each term -> term len, term), 0 length + buffer += TERM_IN_TYPE_BYTE + writeString(column) + + ByteBufferEncodingUtils.writeLengthToBuffer(terms.length, buffer) + for (term <- terms) { + writeString(term) + } + } + + override protected def visitPrefixQuery(column: String, prefix: String, occur: PartKeyQueryOccur): Unit = { + writeOccur(occur) + + // Type byte, col len, col, prefix len, prefix + buffer += PREFIX_TYPE_BYTE + writeString(column) + writeString(prefix) + } + + override protected def visitMatchAllQuery(): Unit = { + buffer += OCCUR_MUST + buffer += MATCH_ALL_TYPE_BYTE + } + + override protected def visitRangeQuery(column: String, start: Long, end: Long, occur: PartKeyQueryOccur): Unit = { + writeOccur(occur) + + // Type byte, col len, col, start, end + buffer += LONG_RANGE_TYPE_BYTE + writeString(column) + writeLong(start) + writeLong(end) + } + + def buildQuery(columnFilters: Seq[ColumnFilter]): Array[Byte] = { + visitQuery(columnFilters) + + buffer.toArray + } + + def buildQueryWithStartAndEnd(columnFilters: Seq[ColumnFilter], start: Long, end: Long): Array[Byte] = { + visitQueryWithStartAndEnd(columnFilters, start, end) + + buffer.toArray + } +} + +// JNI methods +// Thread safety - +// * Index handle creation / cleanup is not thread safe. +// * Other operations are thread safe and may involve an internal mutex +protected object TantivyNativeMethods { + // Load native library from jar + private def loadLibrary(): Unit = { + val tempDir = Files.createTempDirectory("filodb-native-") + + val lib = System.mapLibraryName("filodb_core") + + val arch = SystemUtils.OS_ARCH + val kernel = if (SystemUtils.IS_OS_LINUX) { + "linux" + } else if (SystemUtils.IS_OS_MAC) { + "darwin" + } else if (SystemUtils.IS_OS_WINDOWS) { + "windows" + } else { + sys.error(s"Unhandled platform ${SystemUtils.OS_NAME}") + } + + val resourcePath: String = "/native/" + kernel + "/" + arch + "/" + lib + val resourceStream = Option(TantivyNativeMethods.getClass.getResourceAsStream(resourcePath)).get + + val finalPath = tempDir.resolve(lib) + Files.copy(resourceStream, finalPath) + + System.load(finalPath.toAbsolutePath.toString) + } + + loadLibrary() + + @native + def newIndexHandle(diskLocation: String, schemaFields: Array[String], + schemaMapFields: Array[String], schemaMultiColumnFacets: Array[String], + columnCacheSize: Long, queryCacheMaxSize: Long, queryCacheItemSize: Long, + deletedDocMergeThreshold: Float): Long + + // Free memory used by an index handle + @native + def freeIndexHandle(handle: Long): Unit + + // Force refresh any readers to be up to date (primarily used by tests) + @native + def refreshReaders(handle: Long): Unit + + // Reset index data (delete all docs) + @native + def reset(handle: Long): Unit + + // Commit changes to the index + @native + def commit(handle: Long): Unit + + // Ingest a new document + // scalastyle:off parameter.number + @native + def ingestDocument(handle: Long, partKeyData: Array[Byte], partKeyOffset: Int, + partKeyNumBytes: Int, partId: Int, documentId: String, + startTime: Long, endTime: Long, fields: Array[Byte], + upsert: Boolean): Unit + // scalastyle:on parameter.number + + // Get the estimated amount of RAM being used by this index + @native + def indexRamBytes(handle: Long): Long + + // Get the estimated amount of Mmap space being used by this index + @native + def indexMmapBytes(handle: Long): Long + + // Get the number of entries (docs) in the index + @native + def indexNumEntries(handle: Long): Long + + // Get part IDs that ended before a given time + @native + def partIdsEndedBefore(handle: Long, endedBefore: Long): Array[Int] + + // Remove docs with given part keys + @native + def removePartKeys(handle: Long, keys: Array[Int]): Unit + + // Get the list of unique indexed field names + @native + def indexNames(handle: Long): Array[Byte] + + // Get the list of unique values for a field + @native + def indexValues(handle: Long, fieldName: String, topK: Int): Array[Byte] + + // Get the list of unique indexed field names + @native + def labelNames(handle: Long, query: Array[Byte], limit: Int, start: Long, end: Long): Array[Byte] + + // Get the list of unique values for a field + @native + def labelValues(handle: Long, query: Array[Byte], colName: String, limit: Int, start: Long, end: Long): Array[Byte] + + // Get the list of part IDs given a query + @native + def queryPartIds(handle: Long, query: Array[Byte], limit: Long, start: Long, end: Long): Array[Int] + + // Get the list of part IDs given a query + @native + def queryPartKeyRecords(handle: Long, query: Array[Byte], limit: Long, start: Long, + end: Long): Array[Byte] + + // Get a part key by query + @native + def queryPartKey(handle: Long, query: Array[Byte], limit: Long, start: Long, end: Long): Array[Byte] + + /// Get a part ID from a part key + @native + def partIdFromPartKey(handle: Long, partKey : Array[Byte]): Int + + // Get map of start times from partition ID list + @native + def startTimeFromPartIds(handle: Long, partIds: Array[Int]): Array[Long] + + // Get end time from part ID + @native + def endTimeFromPartId(handle: Long, partId: Int): Long + + // Remove partition IDs and return approximate deleted count + @native + def removePartitionsEndedBefore(handle: Long, endedBefore: Long, returnApproxDeletedCount: Boolean): Int + + // Get cache hit rates for stats + // Array of (query cache, column cache) + @native + def getCacheHitRates(handle: Long): Array[Double] + + // Dump stats - mainly meant for testing + @native + def dumpCacheStats(handle: Long): String + + // Start memory profiling if enabled for this build, or no-op + @native + def startMemoryProfiling(): Unit + + // Start memory profiling if enabled for this build, or no-op + @native + def stopMemoryProfiling(): Unit +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 49b465cfae..0ed968b741 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -285,8 +285,13 @@ class TimeSeriesShard(val ref: DatasetRef, private val indexFacetingEnabledAllLabels = filodbConfig.getBoolean("memstore.index-faceting-enabled-for-all-labels") private val numParallelFlushes = filodbConfig.getInt("memstore.flush-task-parallelism") private val disableIndexCaching = filodbConfig.getBoolean("memstore.disable-index-caching") + private val partKeyIndexType = filodbConfig.getString("memstore.part-key-index-type") private val typeFieldIndexingEnabled = filodbConfig.getBoolean("memstore.type-field-indexing-enabled") - + private val tantivyColumnCacheCount = filodbConfig.getLong("memstore.tantivy.column-cache-count") + private val tantivyQueryCacheSize = filodbConfig.getMemorySize("memstore.tantivy.query-cache-max-bytes") + private val tantivyQueryCacheEstimatedItemSize = + filodbConfig.getMemorySize("memstore.tantivy.query-cache-estimated-item-size") + private val tantivyDeletedDocMergeThreshold = filodbConfig.getDouble("memstore.tantivy.deleted-doc-merge-threshold") /////// END CONFIGURATION FIELDS /////////////////// @@ -312,10 +317,19 @@ class TimeSeriesShard(val ref: DatasetRef, * Used to answer queries not involving the full partition key. * Maintained using a high-performance bitmap index. */ - private[memstore] final val partKeyIndex: PartKeyIndexRaw = new PartKeyLuceneIndex(ref, schemas.part, - indexFacetingEnabledAllLabels, indexFacetingEnabledShardKeyLabels, shardNum, - storeConfig.diskTTLSeconds * 1000, disableIndexCaching = disableIndexCaching, - addMetricTypeField = typeFieldIndexingEnabled) + private[memstore] final val partKeyIndex: PartKeyIndexRaw = partKeyIndexType match { + case "lucene" => new PartKeyLuceneIndex(ref, schemas.part, + indexFacetingEnabledAllLabels, indexFacetingEnabledShardKeyLabels, shardNum, + storeConfig.diskTTLSeconds * 1000, disableIndexCaching = disableIndexCaching, + addMetricTypeField = typeFieldIndexingEnabled) + case "tantivy" => new PartKeyTantivyIndex(ref, schemas.part, + shardNum, storeConfig.diskTTLSeconds * 1000, columnCacheCount = tantivyColumnCacheCount, + queryCacheMaxSize = tantivyQueryCacheSize.toBytes, + queryCacheEstimatedItemSize = tantivyQueryCacheEstimatedItemSize.toBytes, + deletedDocMergeThreshold = tantivyDeletedDocMergeThreshold.toFloat, + addMetricTypeField = typeFieldIndexingEnabled) + case x => sys.error(s"Unsupported part key index type: '$x'") + } private val cardTracker: CardinalityTracker = initCardTracker() @@ -1222,7 +1236,7 @@ class TimeSeriesShard(val ref: DatasetRef, } partIter.skippedPartIDs.foreach { pId => partKeyIndex.partKeyFromPartId(pId).foreach { pk => - val unsafePkOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(pk.offset) + val unsafePkOffset = PartKeyIndexRaw.bytesRefToUnsafeOffset(pk.offset) val schema = schemas(RecordSchema.schemaID(pk.bytes, unsafePkOffset)) val shardKey = schema.partKeySchema.colValues(pk.bytes, unsafePkOffset, schemas.part.options.shardKeyColumns) @@ -1907,7 +1921,7 @@ class TimeSeriesShard(val ref: DatasetRef, partitions.get(partID) match { case TimeSeriesShard.OutOfMemPartition => partKeyIndex.partKeyFromPartId(partID).map { pkBytesRef => - val unsafeKeyOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(pkBytesRef.offset) + val unsafeKeyOffset = PartKeyIndexRaw.bytesRefToUnsafeOffset(pkBytesRef.offset) RecordSchema.schemaID(pkBytesRef.bytes, unsafeKeyOffset) }.getOrElse(-1) case p: TimeSeriesPartition => p.schema.schemaHash diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala index f486f620e1..f15b701ce5 100644 --- a/core/src/main/scala/filodb.core/metadata/Schemas.scala +++ b/core/src/main/scala/filodb.core/metadata/Schemas.scala @@ -308,7 +308,8 @@ final case class Schemas(part: PartitionSchema, val numSamplesPerChunk = chunkDurationMillis / resolutionMs // find number of chunks to be scanned. Ceil division needed here val numChunksPerTs = (queryDurationMs + chunkDurationMillis - 1) / chunkDurationMillis - val bytesPerSample = colIds.map(c => bytesPerSampleSwag((schemaId, c))).sum + // The schema provided does not match existing, give the the sample a weight of histogram. + val bytesPerSample = colIds.map(c => bytesPerSampleSwag.getOrElse((schemaId, c), 20.0)).sum val estDataSize = bytesPerSample * numTsPartitions * numSamplesPerChunk * numChunksPerTs estDataSize } @@ -327,7 +328,8 @@ final case class Schemas(part: PartitionSchema, chunkMethod: ChunkScanMethod ): Double = { val numSamplesPerChunk = chunkDurationMillis / resolutionMs - val bytesPerSample = colIds.map(c => bytesPerSampleSwag((schemaId, c))).sum + // The schema provided does not match existing, give the the sample a weight of histogram. + val bytesPerSample = colIds.map(c => bytesPerSampleSwag.getOrElse((schemaId, c), 20.0)).sum var estDataSize = 0d pkRecs.foreach { pkRec => val intersection = Math.min(chunkMethod.endTime, pkRec.endTime) - Math.max(chunkMethod.startTime, pkRec.startTime) diff --git a/core/src/rust/Cargo.lock b/core/src/rust/Cargo.lock new file mode 100644 index 0000000000..19349fe0d3 --- /dev/null +++ b/core/src/rust/Cargo.lock @@ -0,0 +1,1484 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e4503c46a5c0c7844e948c9a4d6acd9f50cccb4de1c48eb9e291ea17470c678" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.8.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +dependencies = [ + "cfg-if", + "getrandom", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +dependencies = [ + "memchr", +] + +[[package]] +name = "allocator-api2" +version = "0.2.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" + +[[package]] +name = "arc-swap" +version = "1.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" + +[[package]] +name = "async-trait" +version = "0.1.81" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "autocfg" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" + +[[package]] +name = "backtrace" +version = "0.3.73" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cc23269a4f8976d0a4d2e7109211a419fe30e8d88d677cd60b6bc79c5732e0a" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "base64" +version = "0.22.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" + +[[package]] +name = "bitflags" +version = "2.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de" + +[[package]] +name = "bitpacking" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c1d3e2bfd8d06048a179f7b17afc3188effa10385e7b00dc65af6aae732ea92" +dependencies = [ + "crunchy", +] + +[[package]] +name = "bumpalo" +version = "3.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8318a53db07bb3f8dca91a600466bdb3f2eaadeedfdbcf02e1accbad9271ba50" + +[[package]] +name = "cc" +version = "1.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fb8dd288a69fc53a1996d7ecfbf4a20d59065bff137ce7e56bbd620de191189" +dependencies = [ + "jobserver", + "libc", + "shlex", +] + +[[package]] +name = "census" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f4c707c6a209cbe82d10abd08e1ea8995e9ea937d2550646e02798948992be0" + +[[package]] +name = "cesu8" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "combine" +version = "4.6.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" +dependencies = [ + "bytes", + "memchr", +] + +[[package]] +name = "crc32fast" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "crossbeam-channel" +version = "0.5.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33480d6946193aa8033910124896ca395333cae7e2d1113d1fef6c3272217df2" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" + +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + +[[package]] +name = "deranged" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b42b6fa04a440b495c8b04d0e71b707c585f83cb9cb28cf8cd0d976c315e31b4" +dependencies = [ + "powerfmt", + "serde", +] + +[[package]] +name = "dhat" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "98cd11d84628e233de0ce467de10b8633f4ddaecafadefc86e13b84b8739b827" +dependencies = [ + "backtrace", + "lazy_static", + "mintex", + "parking_lot", + "rustc-hash", + "serde", + "serde_json", + "thousands", +] + +[[package]] +name = "downcast-rs" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75b325c5dbd37f80359721ad39aca5a29fb04c89279657cffdda8736d0c0b9d2" + +[[package]] +name = "either" +version = "1.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + +[[package]] +name = "errno" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" +dependencies = [ + "libc", + "windows-sys 0.52.0", +] + +[[package]] +name = "fastdivide" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59668941c55e5c186b8b58c391629af56774ec768f73c08bbcd56f09348eb00b" + +[[package]] +name = "fastrand" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" + +[[package]] +name = "filesize" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12d741e2415d4e2e5bd1c1d00409d1a8865a57892c2d689b504365655d237d43" +dependencies = [ + "winapi", +] + +[[package]] +name = "filodb_core" +version = "0.1.0" +dependencies = [ + "bytes", + "dhat", + "filesize", + "hashbrown", + "jni", + "nohash-hasher", + "nom", + "num-derive", + "num-traits", + "quick_cache", + "regex", + "tantivy", + "tantivy-common", + "tantivy-fst", + "tantivy_utils", + "thiserror", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "fs4" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7e180ac76c23b45e767bd7ae9579bc0bb458618c4bc71835926e098e61d15f8" +dependencies = [ + "rustix", + "windows-sys 0.52.0", +] + +[[package]] +name = "getrandom" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.29.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" + +[[package]] +name = "hashbrown" +version = "0.14.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" +dependencies = [ + "ahash", + "allocator-api2", +] + +[[package]] +name = "hermit-abi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" + +[[package]] +name = "htmlescape" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e9025058dae765dee5070ec375f591e2ba14638c63feff74f13805a72e523163" + +[[package]] +name = "instant" +version = "0.1.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" +dependencies = [ + "cfg-if", + "js-sys", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "itertools" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" + +[[package]] +name = "jni" +version = "0.21.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a87aa2bb7d2af34197c04845522473242e1aa17c12f4935d5856491a7fb8c97" +dependencies = [ + "cesu8", + "cfg-if", + "combine", + "jni-sys", + "log", + "thiserror", + "walkdir", + "windows-sys 0.45.0", +] + +[[package]] +name = "jni-sys" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" + +[[package]] +name = "jobserver" +version = "0.1.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" +dependencies = [ + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.70" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1868808506b929d7b0cfa8f75951347aa71bb21144b7791bae35d9bccfcfe37a" +dependencies = [ + "wasm-bindgen", +] + +[[package]] +name = "lazy_static" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" + +[[package]] +name = "levenshtein_automata" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c2cdeb66e45e9f36bfad5bbdb4d2384e70936afbee843c6f6543f0c551ebb25" + +[[package]] +name = "libc" +version = "0.2.155" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97b3888a4aecf77e811145cadf6eef5901f4782c53886191b2f693f24761847c" + +[[package]] +name = "libm" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" + +[[package]] +name = "linux-raw-sys" +version = "0.4.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" + +[[package]] +name = "lock_api" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" + +[[package]] +name = "lru" +version = "0.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37ee39891760e7d94734f6f63fedc29a2e4a152f836120753a72503f09fcf904" +dependencies = [ + "hashbrown", +] + +[[package]] +name = "lz4_flex" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" + +[[package]] +name = "measure_time" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbefd235b0aadd181626f281e1d684e116972988c14c264e42069d5e8a5775cc" +dependencies = [ + "instant", + "log", +] + +[[package]] +name = "memchr" +version = "2.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" + +[[package]] +name = "memmap2" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe751422e4a8caa417e13c3ea66452215d7d63e19e604f4980461212f3ae1322" +dependencies = [ + "libc", +] + +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8a240ddb74feaf34a79a7add65a741f3167852fba007066dcac1ca548d89c08" +dependencies = [ + "adler", +] + +[[package]] +name = "mintex" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9bec4598fddb13cc7b528819e697852653252b760f1228b7642679bf2ff2cd07" + +[[package]] +name = "murmurhash32" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2195bf6aa996a481483b29d62a7663eed3fe39600c460e323f8ff41e90bdd89b" + +[[package]] +name = "nohash-hasher" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2bf50223579dc7cdcfb3bfcacf7069ff68243f8c363f62ffa99cf000a6b9c451" + +[[package]] +name = "nom" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "num-conv" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51d515d32fb182ee37cda2ccdcb92950d6a3c2893aa280e540671c2cd0f3b1d9" + +[[package]] +name = "num-derive" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed3955f1a9c7c0c15e092f9c887db08b1fc683305fdf6eb6684f22555355e202" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "num-traits" +version = "0.2.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "num_cpus" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" +dependencies = [ + "hermit-abi", + "libc", +] + +[[package]] +name = "object" +version = "0.36.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "27b64972346851a39438c60b341ebc01bba47464ae329e55cf343eb93964efd9" +dependencies = [ + "memchr", +] + +[[package]] +name = "once_cell" +version = "1.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" + +[[package]] +name = "oneshot" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e296cf87e61c9cfc1a61c3c63a0f7f286ed4554e0e22be84e8a38e1d264a2a29" + +[[package]] +name = "ownedbytes" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3a059efb063b8f425b948e042e6b9bd85edfe60e913630ed727b23e2dfcc558" +dependencies = [ + "stable_deref_trait", +] + +[[package]] +name = "parking_lot" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-targets 0.52.6", +] + +[[package]] +name = "pkg-config" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" + +[[package]] +name = "powerfmt" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" + +[[package]] +name = "ppv-lite86" +version = "0.2.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +dependencies = [ + "zerocopy", +] + +[[package]] +name = "proc-macro2" +version = "1.0.86" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "quick_cache" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec932c60e6faf77dc6601ea149a23d821598b019b450bb1d98fe89c0301c0b61" +dependencies = [ + "ahash", + "equivalent", + "hashbrown", + "parking_lot", +] + +[[package]] +name = "quote" +version = "1.0.36" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "rand_distr" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32cb0b9bc82b0a0876c2dd994a7e7a2683d3e7390ca40e6886785ef0c7e3ee31" +dependencies = [ + "num-traits", + "rand", +] + +[[package]] +name = "rayon" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b418a60154510ca1a002a752ca9714984e21e4241e804d32555251faf8b78ffa" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + +[[package]] +name = "redox_syscall" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2a908a6e00f1fdd0dfd9c0eb08ce85126f6d8bbda50017e74bc4a4b7d4a926a4" +dependencies = [ + "bitflags", +] + +[[package]] +name = "regex" +version = "1.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4219d74c6b67a3654a9fbebc4b419e22126d13d2f3c4a07ee0cb61ff79a79619" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" + +[[package]] +name = "rust-stemmers" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e46a2036019fdb888131db7a4c847a1063a7493f971ed94ea82c67eada63ca54" +dependencies = [ + "serde", + "serde_derive", +] + +[[package]] +name = "rustc-demangle" +version = "0.1.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" + +[[package]] +name = "rustc-hash" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" + +[[package]] +name = "rustix" +version = "0.38.34" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" +dependencies = [ + "bitflags", + "errno", + "libc", + "linux-raw-sys", + "windows-sys 0.52.0", +] + +[[package]] +name = "ryu" +version = "1.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "serde" +version = "1.0.207" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5665e14a49a4ea1b91029ba7d3bca9f299e1f7cfa194388ccc20f14743e784f2" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.207" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6aea2634c86b0e8ef2cfdc0c340baede54ec27b1e46febd7f80dffb2aa44a00e" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "serde_json" +version = "1.0.124" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66ad62847a56b3dba58cc891acd13884b9c61138d330c0d7b6181713d4fce38d" +dependencies = [ + "itoa", + "memchr", + "ryu", + "serde", +] + +[[package]] +name = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + +[[package]] +name = "sketches-ddsketch" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85636c14b73d81f541e525f585c0a2109e6744e1565b5c1668e31c70c10ed65c" +dependencies = [ + "serde", +] + +[[package]] +name = "smallvec" +version = "1.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" + +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + +[[package]] +name = "syn" +version = "2.0.74" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fceb41e3d546d0bd83421d3409b1460cc7444cd389341a4c880fe7a042cb3d7" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "tantivy" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8d0582f186c0a6d55655d24543f15e43607299425c5ad8352c242b914b31856" +dependencies = [ + "aho-corasick", + "arc-swap", + "base64", + "bitpacking", + "byteorder", + "census", + "crc32fast", + "crossbeam-channel", + "downcast-rs", + "fastdivide", + "fnv", + "fs4", + "htmlescape", + "itertools", + "levenshtein_automata", + "log", + "lru", + "lz4_flex", + "measure_time", + "memmap2", + "num_cpus", + "once_cell", + "oneshot", + "rayon", + "regex", + "rust-stemmers", + "rustc-hash", + "serde", + "serde_json", + "sketches-ddsketch", + "smallvec", + "tantivy-bitpacker", + "tantivy-columnar", + "tantivy-common", + "tantivy-fst", + "tantivy-query-grammar", + "tantivy-stacker", + "tantivy-tokenizer-api", + "tempfile", + "thiserror", + "time", + "uuid", + "winapi", +] + +[[package]] +name = "tantivy-bitpacker" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "284899c2325d6832203ac6ff5891b297fc5239c3dc754c5bc1977855b23c10df" +dependencies = [ + "bitpacking", +] + +[[package]] +name = "tantivy-columnar" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12722224ffbe346c7fec3275c699e508fd0d4710e629e933d5736ec524a1f44e" +dependencies = [ + "downcast-rs", + "fastdivide", + "itertools", + "serde", + "tantivy-bitpacker", + "tantivy-common", + "tantivy-sstable", + "tantivy-stacker", +] + +[[package]] +name = "tantivy-common" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8019e3cabcfd20a1380b491e13ff42f57bb38bf97c3d5fa5c07e50816e0621f4" +dependencies = [ + "async-trait", + "byteorder", + "ownedbytes", + "serde", + "time", +] + +[[package]] +name = "tantivy-fst" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d60769b80ad7953d8a7b2c70cdfe722bbcdcac6bccc8ac934c40c034d866fc18" +dependencies = [ + "byteorder", + "regex-syntax", + "utf8-ranges", +] + +[[package]] +name = "tantivy-query-grammar" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "847434d4af57b32e309f4ab1b4f1707a6c566656264caa427ff4285c4d9d0b82" +dependencies = [ + "nom", +] + +[[package]] +name = "tantivy-sstable" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c69578242e8e9fc989119f522ba5b49a38ac20f576fc778035b96cc94f41f98e" +dependencies = [ + "tantivy-bitpacker", + "tantivy-common", + "tantivy-fst", + "zstd", +] + +[[package]] +name = "tantivy-stacker" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c56d6ff5591fc332739b3ce7035b57995a3ce29a93ffd6012660e0949c956ea8" +dependencies = [ + "murmurhash32", + "rand_distr", + "tantivy-common", +] + +[[package]] +name = "tantivy-tokenizer-api" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2a0dcade25819a89cfe6f17d932c9cedff11989936bf6dd4f336d50392053b04" +dependencies = [ + "serde", +] + +[[package]] +name = "tantivy_utils" +version = "0.1.0" +dependencies = [ + "hashbrown", + "nohash-hasher", + "quick_cache", + "tantivy", + "tantivy-common", + "tantivy-fst", +] + +[[package]] +name = "tempfile" +version = "3.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04cbcdd0c794ebb0d4cf35e88edd2f7d2c4c3e9a5a6dab322839b321c6a87a64" +dependencies = [ + "cfg-if", + "fastrand", + "once_cell", + "rustix", + "windows-sys 0.59.0", +] + +[[package]] +name = "thiserror" +version = "1.0.63" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0342370b38b6a11b6cc11d6a805569958d54cfa061a29969c3b5ce2ea405724" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.63" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "thousands" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3bf63baf9f5039dadc247375c29eb13706706cfde997d0330d05aa63a77d8820" + +[[package]] +name = "time" +version = "0.3.36" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dfd88e563464686c916c7e46e623e520ddc6d79fa6641390f2e3fa86e83e885" +dependencies = [ + "deranged", + "itoa", + "num-conv", + "powerfmt", + "serde", + "time-core", + "time-macros", +] + +[[package]] +name = "time-core" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" + +[[package]] +name = "time-macros" +version = "0.2.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f252a68540fde3a3877aeea552b832b40ab9a69e318efd078774a01ddee1ccf" +dependencies = [ + "num-conv", + "time-core", +] + +[[package]] +name = "unicode-ident" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" + +[[package]] +name = "utf8-ranges" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fcfc827f90e53a02eaef5e535ee14266c1d569214c6aa70133a624d8a3164ba" + +[[package]] +name = "uuid" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81dfa00651efa65069b0b6b651f4aaa31ba9e3c3ce0137aaad053604ee7e0314" +dependencies = [ + "getrandom", + "serde", +] + +[[package]] +name = "version_check" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" + +[[package]] +name = "walkdir" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "wasm-bindgen" +version = "0.2.93" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a82edfc16a6c469f5f44dc7b571814045d60404b55a0ee849f9bcfa2e63dd9b5" +dependencies = [ + "cfg-if", + "once_cell", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.93" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9de396da306523044d3302746f1208fa71d7532227f15e347e2d93e4145dd77b" +dependencies = [ + "bumpalo", + "log", + "once_cell", + "proc-macro2", + "quote", + "syn", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.93" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "585c4c91a46b072c92e908d99cb1dcdf95c5218eeb6f3bf1efa991ee7a68cccf" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.93" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" +dependencies = [ + "proc-macro2", + "quote", + "syn", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.93" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c62a0a307cb4a311d3a07867860911ca130c3494e8c2719593806c08bc5d0484" + +[[package]] +name = "web-sys" +version = "0.3.70" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26fdeaafd9bd129f65e7c031593c24d62186301e0c72c8978fa1678be7d532c0" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-util" +version = "0.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-sys" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" +dependencies = [ + "windows-targets 0.42.2", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-targets" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +dependencies = [ + "windows_aarch64_gnullvm 0.42.2", + "windows_aarch64_msvc 0.42.2", + "windows_i686_gnu 0.42.2", + "windows_i686_msvc 0.42.2", + "windows_x86_64_gnu 0.42.2", + "windows_x86_64_gnullvm 0.42.2", + "windows_x86_64_msvc 0.42.2", +] + +[[package]] +name = "windows-targets" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" +dependencies = [ + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", + "windows_i686_gnullvm", + "windows_i686_msvc 0.52.6", + "windows_x86_64_gnu 0.52.6", + "windows_x86_64_gnullvm 0.52.6", + "windows_x86_64_msvc 0.52.6", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" + +[[package]] +name = "windows_i686_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" + +[[package]] +name = "windows_i686_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" + +[[package]] +name = "windows_i686_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" + +[[package]] +name = "windows_i686_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" + +[[package]] +name = "zerocopy" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" +dependencies = [ + "byteorder", + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "zstd" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" +dependencies = [ + "zstd-safe", +] + +[[package]] +name = "zstd-safe" +version = "7.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" +dependencies = [ + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.13+zstd.1.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" +dependencies = [ + "cc", + "pkg-config", +] diff --git a/core/src/rust/Cargo.toml b/core/src/rust/Cargo.toml new file mode 100644 index 0000000000..b5de6f2808 --- /dev/null +++ b/core/src/rust/Cargo.toml @@ -0,0 +1,9 @@ +[workspace] +resolver = "2" +members = ["filodb_core", "tantivy_utils"] + +# Keep debug symbols in the final binary +# This makes the binary slightly larger (~20MB), but makes profiling much more useful +# and has no runtime impact +[profile.release] +debug = true diff --git a/core/src/rust/clippy.toml b/core/src/rust/clippy.toml new file mode 100644 index 0000000000..0358cdb508 --- /dev/null +++ b/core/src/rust/clippy.toml @@ -0,0 +1,2 @@ +allow-unwrap-in-tests = true +allow-expect-in-tests = true diff --git a/core/src/rust/filodb_core/Cargo.toml b/core/src/rust/filodb_core/Cargo.toml new file mode 100644 index 0000000000..10f86f6c61 --- /dev/null +++ b/core/src/rust/filodb_core/Cargo.toml @@ -0,0 +1,31 @@ +[package] +name = "filodb_core" +version = "0.1.0" +edition = "2021" +rust-version = "1.78" + +[lib] +crate-type = ["cdylib"] + +[dependencies] +dhat = "0.3.3" +filesize = "0.2.0" +hashbrown = "0.14.5" +jni = "0.21.1" +nohash-hasher = "0.2.0" +nom = "7.1.3" +num-derive = "0.4.2" +num-traits = "0.2.19" +quick_cache = { version = "0.6.2", features = ["stats"] } +regex = "1.10.5" +tantivy = "0.22.0" +tantivy-common = "0.7.0" +tantivy-fst = "0.5.0" +tantivy_utils = { path = "../tantivy_utils" } +thiserror = "1.0.62" + +[dev-dependencies] +bytes = "1.6.1" + +[features] +dhat-heap = [] diff --git a/core/src/rust/filodb_core/src/errors.rs b/core/src/rust/filodb_core/src/errors.rs new file mode 100644 index 0000000000..87b2de147b --- /dev/null +++ b/core/src/rust/filodb_core/src/errors.rs @@ -0,0 +1,44 @@ +//! Error types to translate to exceptions for the JVM + +use std::borrow::Cow; + +use jni::JNIEnv; + +const RUNTIME_EXCEPTION_CLASS: &str = "java/lang/RuntimeException"; + +/// Result type for java exception methods +pub type JavaResult = Result; + +/// Error type that can be thrown as an exception +#[derive(Debug)] +pub struct JavaException { + class: &'static str, + message: Cow<'static, str>, +} + +impl JavaException { + /// Create a new java.lang.RuntimeException + pub fn new_runtime_exception(message: impl Into>) -> Self { + Self::new(RUNTIME_EXCEPTION_CLASS, message) + } + + /// Create a new exception with a specified class and message + pub fn new(class: &'static str, message: impl Into>) -> Self { + Self { + class, + message: message.into(), + } + } + + /// Throw the generated exception on a JNIEnv + pub fn set_exception_details(&self, env: &mut JNIEnv) { + let _ = env.throw_new(self.class, &self.message); + } +} + +// Default conversion for Rust std errors - throw RuntimeException +impl From for JavaException { + fn from(value: T) -> Self { + Self::new_runtime_exception(format!("{value}")) + } +} diff --git a/core/src/rust/filodb_core/src/exec.rs b/core/src/rust/filodb_core/src/exec.rs new file mode 100644 index 0000000000..b4f18a5ff0 --- /dev/null +++ b/core/src/rust/filodb_core/src/exec.rs @@ -0,0 +1,56 @@ +//! Helpers for executing code in a JNI method + +use jni::{sys::jobject, JNIEnv}; + +use crate::errors::JavaResult; + +/// Execs a function in a JNI context, supplying an environment +/// and translating any errors to exceptions +/// +/// All JNI functions should use this to ensure error handling +/// is properly done +/// +/// Do *not* panic in any calls - avoid unwrap, expect, etc. +pub fn jni_exec(env: &mut JNIEnv, func: F) -> T +where + F: FnOnce(&mut JNIEnv) -> JavaResult, + T: EarlyReturn, +{ + let ret = func(env); + match ret { + Ok(r) => r, + Err(e) => { + // An error occurred, throw an exception + e.set_exception_details(env); + + T::abort_value() + } + } +} + +/// Trait for early return values when an exception is being thrown +pub trait EarlyReturn { + fn abort_value() -> Self; +} + +impl EarlyReturn for jobject { + fn abort_value() -> Self { + std::ptr::null_mut() + } +} + +impl EarlyReturn for i32 { + fn abort_value() -> Self { + 0 + } +} + +impl EarlyReturn for i64 { + fn abort_value() -> Self { + 0 + } +} + +impl EarlyReturn for () { + fn abort_value() -> Self {} +} diff --git a/core/src/rust/filodb_core/src/index.rs b/core/src/rust/filodb_core/src/index.rs new file mode 100644 index 0000000000..e151e7127c --- /dev/null +++ b/core/src/rust/filodb_core/src/index.rs @@ -0,0 +1,183 @@ +//! Methods to create / destroy the index + +use jni::{ + objects::{JClass, JObjectArray, JString}, + sys::{jfloat, jlong}, + JNIEnv, +}; +use tantivy::{ + directory::MmapDirectory, + indexer::LogMergePolicy, + schema::{ + BytesOptions, FacetOptions, Field, JsonObjectOptions, NumericOptions, Schema, + SchemaBuilder, TextFieldIndexing, TextOptions, + }, + IndexBuilder, IndexSettings, ReloadPolicy, TantivyDocument, +}; +use tantivy_utils::field_constants::{self, facet_field_name, LABEL_LIST}; + +use crate::{ + errors::{JavaException, JavaResult}, + exec::jni_exec, + jnienv::JNIEnvExt, + state::IndexHandle, +}; + +pub const WRITER_MEM_BUDGET: usize = 50 * 1024 * 1024; + +/// Create a new index state object by loading and configuring schema +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_newIndexHandle( + mut env: JNIEnv, + _class: JClass, + disk_location: JString, + schema_fields: JObjectArray, + map_fields: JObjectArray, + multi_column_facet_fields: JObjectArray, + column_cache_size: jlong, + query_cache_max_size: jlong, + query_cache_estimated_item_size: jlong, + deleted_doc_merge_threshold: jfloat, +) -> jlong { + jni_exec(&mut env, |env| { + let disk_location: String = env.get_string(&disk_location)?.into(); + std::fs::create_dir_all(&disk_location)?; + + let directory = MmapDirectory::open(disk_location)?; + + // Build the schema for documents + let (schema, default_field) = + build_schema(env, &schema_fields, &map_fields, &multi_column_facet_fields)?; + + // Open index + let settings = IndexSettings { + ..Default::default() + }; + + let index = IndexBuilder::new() + .schema(schema.clone()) + .settings(settings) + .open_or_create(directory.clone())?; + + let writer = index.writer::(WRITER_MEM_BUDGET)?; + + let mut merge_policy = LogMergePolicy::default(); + merge_policy.set_del_docs_ratio_before_merge(deleted_doc_merge_threshold); + + writer.set_merge_policy(Box::new(merge_policy)); + + let reader = index + .reader_builder() + // It's tempting to use Manual here as we call refresh periodically + // from a timer thread. However, refresh just means that you can see + // all uncommitted documents, not that all merges have completed. This + // means that background merges that are happening that could speed up + // queries aren't avaialble when manual is used. Instead we use + // on commit - the cost of this is minor since it's a FS notification + // and reloading the segment list is fairly cheap and infrequent. + .reload_policy(ReloadPolicy::OnCommitWithDelay) + .try_into()?; + + Ok(IndexHandle::new_handle( + schema, + default_field, + writer, + reader, + directory, + column_cache_size as u64, + query_cache_max_size as u64, + query_cache_estimated_item_size as u64, + )?) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_freeIndexHandle( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) { + jni_exec(&mut env, |_| { + unsafe { + drop(Box::from_raw(handle as *mut IndexHandle)); + } + + Ok(()) + }); +} + +fn build_schema( + env: &mut JNIEnv, + schema_fields: &JObjectArray, + map_fields: &JObjectArray, + multi_column_facet_fields: &JObjectArray, +) -> JavaResult<(Schema, Option)> { + let mut builder = SchemaBuilder::new(); + + let text_options = TextOptions::default().set_indexing_options( + TextFieldIndexing::default() + .set_tokenizer("raw") + .set_fieldnorms(false), + ); + + let random_access_text_options = text_options.clone().set_fast(Some("raw")); + + let numeric_options = NumericOptions::default().set_indexed().set_fast(); + + // Bytes values are faster to read via the doc store vs fast fields and we don't need any of the fast + // field only features like iterating by sorted values + let byte_options = BytesOptions::default().set_indexed().set_stored(); + + builder.add_text_field(field_constants::DOCUMENT_ID, text_options.clone()); + builder.add_i64_field(field_constants::PART_ID, numeric_options.clone()); + builder.add_bytes_field(field_constants::PART_KEY, byte_options); + builder.add_i64_field(field_constants::START_TIME, numeric_options.clone()); + builder.add_i64_field(field_constants::END_TIME, numeric_options.clone()); + builder.add_text_field(field_constants::TYPE, text_options.clone()); + + // Fields from input schema + env.foreach_string_in_array(schema_fields, |name| { + builder.add_text_field(&name, random_access_text_options.clone()); + + Ok(()) + })?; + + // Map fields - only one supported + let len = env.get_array_length(map_fields)?; + if len > 1 { + return Err(JavaException::new_runtime_exception( + "More than one map field specified", + )); + } + + let default_field = if len == 1 { + let name = env.get_object_array_element(map_fields, 0)?.into(); + let name = env.get_rust_string(&name)?; + + let field = builder.add_json_field( + &name, + JsonObjectOptions::default() + .set_indexing_options( + TextFieldIndexing::default() + .set_tokenizer("raw") + .set_fieldnorms(false), + ) + .set_fast(Some("raw")), + ); + + Some(field) + } else { + None + }; + + env.foreach_string_in_array(multi_column_facet_fields, |name| { + builder.add_text_field(&name, random_access_text_options.clone()); + + Ok(()) + })?; + + // Default facet for label list, always added + builder.add_facet_field(&facet_field_name(LABEL_LIST), FacetOptions::default()); + + Ok((builder.build(), default_field)) +} diff --git a/core/src/rust/filodb_core/src/ingestion.rs b/core/src/rust/filodb_core/src/ingestion.rs new file mode 100644 index 0000000000..dff436372e --- /dev/null +++ b/core/src/rust/filodb_core/src/ingestion.rs @@ -0,0 +1,266 @@ +//! Methods that modify the index / do data ingestion + +use std::{ops::Bound, sync::atomic::Ordering}; + +use fields::add_fields; +use jni::{ + objects::{JByteArray, JClass, JIntArray, JString}, + sys::{jboolean, jint, jlong, JNI_TRUE}, + JNIEnv, +}; +use tantivy::{ + collector::Count, + indexer::UserOperation, + query::{RangeQuery, TermSetQuery}, + schema::Facet, + TantivyDocument, Term, +}; +use tantivy_utils::field_constants::{self, facet_field_name}; + +use crate::{ + errors::JavaResult, + exec::jni_exec, + jnienv::JNIEnvExt, + state::{IndexHandle, IngestingDocument}, +}; + +mod fields; + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_reset( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + + handle.changes_pending.store(false, Ordering::SeqCst); + + let mut writer = handle.writer.write()?; + writer.delete_all_documents()?; + writer.commit()?; + + handle.changes_pending.store(false, Ordering::SeqCst); + + Ok(()) + }); +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_commit( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + + handle.changes_pending.store(false, Ordering::SeqCst); + + let mut writer = handle.writer.write()?; + writer.commit()?; + + Ok(()) + }); +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_ingestDocument( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + part_key_data: JByteArray, + part_key_offset: jint, + part_key_num_bytes: jint, + part_id: jint, + document_id: JString, + start_time: jlong, + end_time: jlong, + fields: JByteArray, + upsert: jboolean, +) { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let mut ingesting_doc = IngestingDocument::default(); + + if part_id > -1 { + ingesting_doc.doc.add_i64( + handle.schema.get_field(field_constants::PART_ID)?, + part_id.into(), + ); + } + + let document_id = env.get_rust_string(&document_id)?; + ingesting_doc.doc.add_text( + handle.schema.get_field(field_constants::DOCUMENT_ID)?, + document_id.clone(), + ); + + ingesting_doc.doc.add_i64( + handle.schema.get_field(field_constants::START_TIME)?, + start_time, + ); + + ingesting_doc.doc.add_i64( + handle.schema.get_field(field_constants::END_TIME)?, + end_time, + ); + + let bytes = env.get_byte_array_offset_len( + &part_key_data, + part_key_offset as usize, + part_key_num_bytes as usize, + )?; + + ingesting_doc + .doc + .add_bytes(handle.schema.get_field(field_constants::PART_KEY)?, bytes); + + // Add dynamic fields + let fields = env.get_byte_array(&fields)?; + add_fields(&fields, &mut ingesting_doc, &handle.schema)?; + + let doc = prepare_tantivy_doc(handle, &mut ingesting_doc)?; + + // Save it + let writer = handle.writer.read()?; + + if upsert == JNI_TRUE { + let delete_term = Term::from_field_text( + handle.schema.get_field(field_constants::DOCUMENT_ID)?, + &document_id, + ); + + let writer = handle.writer.read()?; + writer.run([UserOperation::Delete(delete_term), UserOperation::Add(doc)])?; + + handle.changes_pending.store(true, Ordering::SeqCst); + } else { + writer.add_document(doc)?; + } + + handle.changes_pending.store(true, Ordering::SeqCst); + + Ok(()) + }); +} + +fn prepare_tantivy_doc( + handle: &IndexHandle, + ingesting_doc: &mut IngestingDocument, +) -> JavaResult { + let mut map_values = std::mem::take(&mut ingesting_doc.map_values); + + // Insert map columns we've built up + for (key, value) in map_values.drain() { + ingesting_doc + .doc + .add_object(handle.schema.get_field(&key)?, value); + } + + // Build final facet for field list + let mut field_names = std::mem::take(&mut ingesting_doc.field_names); + field_names.sort(); + + for field in field_names { + add_facet( + handle, + ingesting_doc, + field_constants::LABEL_LIST, + &[field.as_str()], + )?; + } + + let doc = std::mem::take(&mut ingesting_doc.doc); + + Ok(doc) +} + +fn add_facet( + handle: &IndexHandle, + ingesting_doc: &mut IngestingDocument, + name: &str, + value: &[&str], +) -> JavaResult<()> { + if !name.is_empty() && !value.is_empty() { + ingesting_doc.doc.add_facet( + handle.schema.get_field(&facet_field_name(name))?, + Facet::from_path(value), + ); + } + + Ok(()) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_removePartKeys( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + keys: JIntArray, +) { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + let mut terms = vec![]; + + let field = handle.schema.get_field(field_constants::PART_ID)?; + + let len = env.get_array_length(&keys)?; + let mut part_ids = vec![0i32; len as usize]; + + env.get_int_array_region(&keys, 0, &mut part_ids)?; + + for part_id in part_ids { + terms.push(Term::from_field_i64(field, part_id as i64)); + } + + let query = Box::new(TermSetQuery::new(terms)); + + let writer = handle.writer.read()?; + writer.delete_query(query)?; + + handle.changes_pending.store(true, Ordering::SeqCst); + + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_removePartitionsEndedBefore( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + ended_before: jlong, + return_deleted_count: jboolean, +) -> jint { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let query = RangeQuery::new_i64_bounds( + field_constants::END_TIME.to_string(), + Bound::Included(0), + // To match existing Lucene index behavior, make this inclusive even though it's named + // "ended before" in the API + Bound::Included(ended_before), + ); + + let java_ret = if return_deleted_count == JNI_TRUE { + let searcher = handle.reader.searcher(); + + let collector = Count; + + searcher.search(&query, &collector)? + } else { + 0 + }; + + let writer = handle.writer.read()?; + writer.delete_query(Box::new(query))?; + + handle.changes_pending.store(true, Ordering::SeqCst); + + Ok(java_ret as i32) + }) +} diff --git a/core/src/rust/filodb_core/src/ingestion/fields.rs b/core/src/rust/filodb_core/src/ingestion/fields.rs new file mode 100644 index 0000000000..585b9306df --- /dev/null +++ b/core/src/rust/filodb_core/src/ingestion/fields.rs @@ -0,0 +1,204 @@ +//! Working with field data + +use std::collections::BTreeMap; + +use nom::{Err, IResult}; +use num_derive::FromPrimitive; +use tantivy::schema::Schema; + +use crate::{ + parser::{parse_string, parse_type_id, AsNomError, ParserError, TypeParseResult}, + state::IngestingDocument, +}; + +#[derive(FromPrimitive)] +#[repr(u8)] +enum FieldTypeId { + Indexed = 1, + Map = 2, + Multicolumn = 3, +} + +pub fn add_fields<'a>( + input: &'a [u8], + doc: &mut IngestingDocument, + schema: &Schema, +) -> IResult<&'a [u8], (), ParserError> { + let mut next_input = input; + + while !next_input.is_empty() { + let (input, type_id) = parse_type_id(next_input)?; + + let (input, _) = match type_id { + TypeParseResult::Success(FieldTypeId::Indexed) => { + parse_indexed_field(input, doc, schema)? + } + TypeParseResult::Success(FieldTypeId::Map) => parse_map_field(input, doc)?, + TypeParseResult::Success(FieldTypeId::Multicolumn) => { + parse_multicolumn_field(input, doc, schema)? + } + TypeParseResult::Failure(type_id) => { + return Err(Err::Failure(ParserError::UnknownType(type_id))) + } + }; + + next_input = input; + } + + Ok((next_input, ())) +} + +fn parse_indexed_field<'a>( + input: &'a [u8], + doc: &mut IngestingDocument, + schema: &Schema, +) -> IResult<&'a [u8], (), ParserError> { + let (input, field_name) = parse_string(input)?; + let (input, value) = parse_string(input)?; + + let field = schema.get_field(&field_name).to_nom_err()?; + + doc.doc.add_text(field, value); + doc.field_names.push(field_name.to_string()); + + Ok((input, ())) +} + +fn parse_map_field<'a>( + input: &'a [u8], + doc: &mut IngestingDocument, +) -> IResult<&'a [u8], (), ParserError> { + let (input, map_name) = parse_string(input)?; + let (input, field_name) = parse_string(input)?; + let (input, value) = parse_string(input)?; + + // Create new map for this map column if needed + if !doc.map_values.contains_key(map_name.as_ref()) { + doc.map_values.insert(map_name.to_string(), BTreeMap::new()); + } + + // Capture value + doc.map_values + .get_mut(map_name.as_ref()) + .ok_or_else(|| Err::Failure(ParserError::InternalMapError))? + .insert(field_name.to_string(), value.to_string().into()); + doc.field_names.push(field_name.to_string()); + + Ok((input, ())) +} + +fn parse_multicolumn_field<'a>( + input: &'a [u8], + doc: &mut IngestingDocument, + schema: &Schema, +) -> IResult<&'a [u8], (), ParserError> { + let (input, field_name) = parse_string(input)?; + let (input, value) = parse_string(input)?; + + let field = schema.get_field(&field_name).to_nom_err()?; + + doc.doc.add_text(field, value); + doc.field_names.push(field_name.to_string()); + + Ok((input, ())) +} + +#[cfg(test)] +mod tests { + use bytes::BufMut; + use tantivy::{schema::OwnedValue, Document}; + + use tantivy_utils::test_utils::{ + build_test_schema, COL1_NAME, JSON_ATTRIBUTE1_NAME, JSON_COL_NAME, + }; + + use super::*; + + #[test] + fn test_parse_indexed_field() { + let mut doc = IngestingDocument::default(); + let index = build_test_schema(); + + let mut buf = vec![]; + + let expected = "abcd"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(expected.len() as u16); + buf.put_slice(expected.as_bytes()); + + let _ = parse_indexed_field(&buf, &mut doc, &index.schema).expect("Should succeed"); + + assert!(doc.field_names.contains(&COL1_NAME.to_string())); + assert_eq!( + **doc + .doc + .get_sorted_field_values() + .first() + .unwrap() + .1 + .first() + .unwrap(), + OwnedValue::Str(expected.into()) + ); + } + + #[test] + fn test_parse_map_field() { + let mut doc = IngestingDocument::default(); + + let mut buf = vec![]; + + let expected = "abcd"; + + buf.put_u16_le(JSON_COL_NAME.len() as u16); + buf.put_slice(JSON_COL_NAME.as_bytes()); + buf.put_u16_le(JSON_ATTRIBUTE1_NAME.len() as u16); + buf.put_slice(JSON_ATTRIBUTE1_NAME.as_bytes()); + buf.put_u16_le(expected.len() as u16); + buf.put_slice(expected.as_bytes()); + + let _ = parse_map_field(&buf, &mut doc).expect("Should succeed"); + + assert!(doc.field_names.contains(&JSON_ATTRIBUTE1_NAME.to_string())); + assert_eq!( + *doc.map_values + .get(JSON_COL_NAME) + .unwrap() + .get(JSON_ATTRIBUTE1_NAME) + .unwrap(), + OwnedValue::Str(expected.into()) + ); + } + + #[test] + fn test_parse_multicolumn_field() { + let mut doc = IngestingDocument::default(); + let index = build_test_schema(); + + let mut buf = vec![]; + + let expected = "abcd"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(expected.len() as u16); + buf.put_slice(expected.as_bytes()); + + let _ = parse_multicolumn_field(&buf, &mut doc, &index.schema).expect("Should succeed"); + + assert!(doc.field_names.contains(&COL1_NAME.to_string())); + assert_eq!( + **doc + .doc + .get_sorted_field_values() + .first() + .unwrap() + .1 + .first() + .unwrap(), + OwnedValue::Str(expected.into()) + ); + } +} diff --git a/core/src/rust/filodb_core/src/jnienv.rs b/core/src/rust/filodb_core/src/jnienv.rs new file mode 100644 index 0000000000..3adff82c8d --- /dev/null +++ b/core/src/rust/filodb_core/src/jnienv.rs @@ -0,0 +1,88 @@ +//! Extensions to JNIEnv + +use jni::{ + objects::{JByteArray, JObject, JObjectArray, JString}, + JNIEnv, +}; + +use crate::errors::JavaResult; + +/// Helper extensions for working with JVM types +#[allow(dead_code)] +pub trait JNIEnvExt<'a> { + /// Get a rust string from Java String + fn get_rust_string(&mut self, obj: &JString) -> JavaResult; + + /// Get a class name and return it as a string + /// This is equivilant to Java code `obj.class.name` + fn get_object_class_name(&mut self, obj: &JObject) -> JavaResult; + + /// Run a closure over every String in a String[] + fn foreach_string_in_array(&mut self, array: &JObjectArray, func: F) -> JavaResult<()> + where + F: FnMut(String) -> JavaResult<()>; + + /// Get a byte array from the JVM + fn get_byte_array_offset_len( + &mut self, + array: &JByteArray, + offset: usize, + len: usize, + ) -> JavaResult>; + + /// Get a byte array from the JVM + fn get_byte_array(&mut self, array: &JByteArray) -> JavaResult>; +} + +impl<'a> JNIEnvExt<'a> for JNIEnv<'a> { + fn get_rust_string(&mut self, obj: &JString) -> JavaResult { + let ret = self.get_string(obj)?.into(); + Ok(ret) + } + + fn get_object_class_name(&mut self, obj: &JObject) -> JavaResult { + let class = self.get_object_class(obj)?; + let name = self + .get_field(&class, "name", "Ljava/lang/String;")? + .l()? + .into(); + + let ret = self.get_string(&name)?.into(); + Ok(ret) + } + + fn foreach_string_in_array(&mut self, array: &JObjectArray, mut func: F) -> JavaResult<()> + where + F: FnMut(String) -> JavaResult<()>, + { + let len = self.get_array_length(array)?; + for idx in 0..len { + let s = self.get_object_array_element(array, idx)?.into(); + let s = self.get_rust_string(&s)?; + func(s)?; + } + + Ok(()) + } + + fn get_byte_array_offset_len( + &mut self, + array: &JByteArray, + offset: usize, + len: usize, + ) -> JavaResult> { + let mut bytes = vec![0u8; len]; + let bytes_ptr = bytes.as_mut_ptr() as *mut i8; + let bytes_ptr = unsafe { std::slice::from_raw_parts_mut(bytes_ptr, len) }; + + self.get_byte_array_region(array, offset as i32, bytes_ptr)?; + + Ok(bytes) + } + + fn get_byte_array(&mut self, array: &JByteArray) -> JavaResult> { + let len = self.get_array_length(array)?; + + self.get_byte_array_offset_len(array, 0, len as usize) + } +} diff --git a/core/src/rust/filodb_core/src/lib.rs b/core/src/rust/filodb_core/src/lib.rs new file mode 100644 index 0000000000..433082ee97 --- /dev/null +++ b/core/src/rust/filodb_core/src/lib.rs @@ -0,0 +1,30 @@ +//! Native methods for FiloDB core +//! +//! This library extensively uses JNI to interop with JVM code. +//! +//! Any new code should do the following to ensure consistency: +//! +//! * All JNI methods should be wrapped in jni_exec. This turns any +//! Rust errors into RuntimeExceptions and allows for cleaner Rust +//! error handling. +//! * No panic/unwrap/expect calls should be used. Panicing will destroy +//! the JVM process. +//! * Try to use primitive types when possible. Getting fields on JVM +//! objects requires reflection like overhead that can't be optimized +//! as well +//! * Minimize the calls back into the JVM. Perfer to get passed in +//! needed information as arguments vs calling object methods. +//! + +#![deny(clippy::expect_used, clippy::unwrap_used, clippy::panic)] + +mod errors; +mod exec; +mod index; +mod ingestion; +mod jnienv; +mod parser; +mod profile; +mod query_parser; +mod reader; +mod state; diff --git a/core/src/rust/filodb_core/src/parser.rs b/core/src/rust/filodb_core/src/parser.rs new file mode 100644 index 0000000000..240193b6f8 --- /dev/null +++ b/core/src/rust/filodb_core/src/parser.rs @@ -0,0 +1,146 @@ +//! Binary parser helpers + +use std::borrow::Cow; + +use nom::{ + bytes::streaming::take, + error::{ErrorKind, ParseError}, + number::streaming::{le_u16, u8}, + IResult, +}; +use num_traits::FromPrimitive; +use tantivy::TantivyError; +use thiserror::Error; + +/// Error type for query parsing issues +#[derive(Error, Debug)] +pub enum ParserError { + #[error("Core parsing error: {0:?}")] + Nom(ErrorKind), + #[error("Index error: {0}")] + IndexError(#[from] TantivyError), + #[error("Unknown type byte: {0}")] + UnknownType(u8), + #[error("Unknown occur byte: {0}")] + UnknownOccur(u8), + #[error("Internal map error")] + InternalMapError, +} + +pub trait AsNomError { + fn to_nom_err(self) -> Result>; +} + +impl AsNomError for Result { + fn to_nom_err(self) -> Result> { + match self { + Err(e) => Err(nom::Err::Failure(e.into())), + Ok(x) => Ok(x), + } + } +} + +impl<'a> ParseError<&'a [u8]> for ParserError { + fn from_error_kind(_input: &'a [u8], kind: ErrorKind) -> Self { + ParserError::Nom(kind) + } + + fn append(_input: &'a [u8], _kind: ErrorKind, other: Self) -> Self { + other + } +} + +pub fn parse_string(input: &[u8]) -> IResult<&[u8], Cow<'_, str>, ParserError> { + let (input, length) = le_u16(input)?; + let (input, string_data) = take(length)(input)?; + + Ok((input, String::from_utf8_lossy(string_data))) +} + +#[derive(PartialEq, Debug)] +pub enum TypeParseResult { + Success(T), + Failure(u8), +} + +impl From for TypeParseResult +where + T: FromPrimitive, +{ + fn from(value: u8) -> Self { + match T::from_u8(value) { + Some(val) => Self::Success(val), + None => Self::Failure(value), + } + } +} + +pub fn parse_type_id(input: &[u8]) -> IResult<&[u8], TypeParseResult, ParserError> +where + T: FromPrimitive, +{ + let (input, type_id) = u8(input)?; + Ok((input, type_id.into())) +} + +#[cfg(test)] +mod tests { + use bytes::BufMut; + use num_derive::FromPrimitive; + + use super::*; + + #[test] + fn test_parse_string() { + let mut buf = vec![]; + + let expected = "abcd"; + + buf.put_u16_le(expected.len() as u16); + buf.put_slice(expected.as_bytes()); + + let (_, result) = parse_string(&buf).expect("Should succeed"); + + assert_eq!(result, expected); + } + + #[test] + fn test_parse_empty_string() { + let mut buf = vec![]; + + buf.put_u16_le(0); + + let (_, result) = parse_string(&buf).expect("Should succeed"); + + assert_eq!(result, ""); + } + + #[derive(FromPrimitive, Debug, PartialEq)] + #[repr(u8)] + pub enum TestTypeId { + Val1 = 1, + Val2 = 2, + } + + #[test] + fn test_parse_type_id() { + let mut buf = vec![]; + + buf.put_u8(1); + + let (_, result) = parse_type_id(&buf).expect("Should succeed"); + + assert_eq!(result, TypeParseResult::Success(TestTypeId::Val1)); + } + + #[test] + fn test_parse_type_id_invalid() { + let mut buf = vec![]; + + buf.put_u8(3); + + let (_, result) = parse_type_id::(&buf).expect("Should succeed"); + + assert_eq!(result, TypeParseResult::Failure(3)); + } +} diff --git a/core/src/rust/filodb_core/src/profile.rs b/core/src/rust/filodb_core/src/profile.rs new file mode 100644 index 0000000000..d3718586bd --- /dev/null +++ b/core/src/rust/filodb_core/src/profile.rs @@ -0,0 +1,149 @@ +//! Helpers for profiling / testing + +#[cfg(feature = "dhat-heap")] +use std::sync::Mutex; + +use jni::{ + objects::JClass, + sys::{jdoubleArray, jlong, jstring}, + JNIEnv, +}; + +use crate::{exec::jni_exec, state::IndexHandle}; + +/// Get cache stats info +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_dumpCacheStats( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) -> jstring { + jni_exec(&mut env, |env| { + let index = IndexHandle::get_ref_from_handle(handle); + + let (column_hits, column_misses) = index.column_cache.stats(); + let (query_hits, query_misses) = index.query_cache_stats(); + + let output = format!( + "Column cache: {} hits {} misses {}% hit\nQuery cache: {} hits {} misses {}% hit", + column_hits, + column_misses, + cache_hit_rate(column_hits, column_misses), + query_hits, + query_misses, + cache_hit_rate(query_hits, query_misses), + ); + + let java_str = env.new_string(output)?; + + Ok(java_str.into_raw()) + }) +} + +/// Get cache hit rates +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_getCacheHitRates( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) -> jdoubleArray { + jni_exec(&mut env, |env| { + let index = IndexHandle::get_ref_from_handle(handle); + + let (column_hits, column_misses) = index.column_cache.stats(); + let (query_hits, query_misses) = index.query_cache_stats(); + + let column_total = column_hits + column_misses; + let query_total = query_hits + query_misses; + let column_hit_rate = if column_total == 0 { + 1.0f64 + } else { + (column_hits as f64) / (column_total) as f64 + }; + + let query_hit_rate = if query_total == 0 { + 1.0f64 + } else { + (query_hits as f64) / (query_total) as f64 + }; + + // Contract with JVM code is (query hit rate, column hit rate) + let hit_rates = [query_hit_rate, column_hit_rate]; + + let result = env.new_double_array(hit_rates.len() as i32)?; + env.set_double_array_region(&result, 0, &hit_rates)?; + + Ok(result.into_raw()) + }) +} + +/// Start memory profiling +#[no_mangle] +#[allow(unused_mut, unused_variables)] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_startMemoryProfiling( + mut env: JNIEnv, + _class: JClass, +) { + #[cfg(feature = "dhat-heap")] + jni_exec(&mut env, |_| { + PROFILER.lock()?.replace(dhat::Profiler::new_heap()); + + Ok(()) + }); +} + +/// Stop memory profiling +#[no_mangle] +#[allow(unused_mut, unused_variables)] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_stopMemoryProfiling( + mut env: JNIEnv, + _class: JClass, +) { + #[cfg(feature = "dhat-heap")] + jni_exec(&mut env, |_| { + PROFILER.lock()?.take(); + + Ok(()) + }); +} + +#[cfg(feature = "dhat-heap")] +#[global_allocator] +static ALLOC: dhat::Alloc = dhat::Alloc; + +#[cfg(feature = "dhat-heap")] +static PROFILER: Mutex> = Mutex::new(None); + +fn cache_hit_rate(hits: u64, misses: u64) -> String { + format!("{:0.2}", (hits as f64 / (hits + misses) as f64) * 100.0) +} + +#[cfg(test)] +mod tests { + use quick_cache::sync::Cache; + + use super::*; + + #[test] + fn test_cache_hit_percent() { + let cache: Cache = Cache::new(100); + + for i in 0..20 { + cache.insert(i, ()); + } + + for i in 0..100 { + cache.get(&i); + } + + let hits = cache.hits(); + let misses = cache.misses(); + + assert_eq!(20, hits); + assert_eq!(80, misses); + + let hit_rate = cache_hit_rate(hits, misses); + + assert_eq!("20.00", hit_rate); + } +} diff --git a/core/src/rust/filodb_core/src/query_parser.rs b/core/src/rust/filodb_core/src/query_parser.rs new file mode 100644 index 0000000000..82a4d572cb --- /dev/null +++ b/core/src/rust/filodb_core/src/query_parser.rs @@ -0,0 +1,997 @@ +//! Query parsers and builders + +use std::ops::Bound; + +use nom::{ + number::{complete::le_i64, streaming::le_u16}, + Err, IResult, +}; +use num_derive::FromPrimitive; +use tantivy::{ + query::{ + AllQuery, BooleanQuery, EmptyQuery, Occur, Query, RangeQuery, TermQuery, TermSetQuery, + }, + schema::{Field, IndexRecordOption, Schema}, + TantivyError, Term, +}; +use tantivy_utils::query::{ + prefix_query::PrefixQuery, range_aware_regex::RangeAwareRegexQuery, JSON_PREFIX_SEPARATOR, +}; + +use crate::parser::{parse_string, parse_type_id, AsNomError, ParserError, TypeParseResult}; + +pub mod filodb_query; + +/// Query format +/// +/// Queries are complex trees of predicates that must be supported. This prevents us from easily +/// encoding them in primitive types across the JNI boundary. +/// +/// To avoid making the Rust code a series of complex reflection operations and to make this +/// as efficient as possible a new binary format is defined for the JVM code to pass to the Rust +/// code inside a byte array. +/// +/// Each query entry is encoded starting with a single byte type ID. See the `QueryTypeId` enum +/// for possible values. For each child query in a boolean query it is encoded via an 8 bit occur value, +/// a 8 bit type id, and a 16 bit length followed by a UTF-8 string with the specified length. +/// +/// As a simple example, consider a boolean query like: +/// +/// f1:ABC AND f2:DEF +/// +/// The encoded sequence would roughly look like: +/// +/// 01 - start of boolean query +/// 01 - query must match +/// 02 - equals query +/// 03 00 - string of length 3 +/// 41 42 43 - UTF8 encoding of 'ABC' +/// 01 - query must match +/// 02 - equals query +/// 03 00 - string of length 3 +/// 44 45 46 - UTF8 encoding of 'DEF' +/// 00 - end of boolean query + +/// Query type encoding +#[derive(FromPrimitive)] +#[repr(u8)] +pub enum QueryTypeId { + /// A boolean query composed of other queries + Boolean = 1, + /// A term match / not match query + Equals = 2, + /// A regular express term match / not match query + Regex = 3, + /// A term must be in / not in a specified list + TermIn = 4, + /// A term must start with / not start with input + Prefix = 5, + /// Match all documents + MatchAll = 6, + /// Start->End range query on a long field + LongRange = 7, +} + +/// Occurs encoding +#[derive(FromPrimitive)] +#[repr(u8)] +pub enum Occurs { + /// Query must match + Must = 1, + /// Query must not match + MustNot = 2, +} + +/// Parse a query from binary format +/// +/// `default_field` is used to search JSON (map) columns +/// If an input column name isn't in the list of indexed columns +/// default_field is used and the input name is used as a JSON path +pub fn parse_query<'a>( + input: &'a [u8], + schema: &Schema, + default_field: Option, +) -> IResult<&'a [u8], Box, ParserError> { + let (input, type_id) = parse_type_id(input)?; + + match type_id { + TypeParseResult::Failure(type_id) => Err(Err::Failure(ParserError::UnknownType(type_id))), + TypeParseResult::Success(QueryTypeId::Boolean) => { + parse_boolean_query(input, schema, default_field) + } + TypeParseResult::Success(QueryTypeId::Equals) => { + parse_equals_query(input, schema, default_field) + } + TypeParseResult::Success(QueryTypeId::Regex) => { + parse_regex_query(input, schema, default_field) + } + TypeParseResult::Success(QueryTypeId::TermIn) => { + parse_term_in_query(input, schema, default_field) + } + TypeParseResult::Success(QueryTypeId::Prefix) => { + parse_prefix_query(input, schema, default_field) + } + TypeParseResult::Success(QueryTypeId::MatchAll) => { + let query = AllQuery; + Ok((input, Box::new(query))) + } + TypeParseResult::Success(QueryTypeId::LongRange) => { + parse_long_range_query(input, schema, default_field) + } + } +} + +fn parse_boolean_query<'a>( + input: &'a [u8], + schema: &Schema, + default_field: Option, +) -> IResult<&'a [u8], Box, ParserError> { + let mut subqueries = vec![]; + let mut next_input = input; + loop { + let (input, occur) = parse_type_id(next_input)?; + + let occur = match occur { + TypeParseResult::Success(Occurs::Must) => tantivy::query::Occur::Must, + TypeParseResult::Success(Occurs::MustNot) => tantivy::query::Occur::MustNot, + TypeParseResult::Failure(0) => { + // End of boolean marker + next_input = input; + break; + } + TypeParseResult::Failure(occur) => { + return Err(Err::Failure(ParserError::UnknownOccur(occur))) + } + }; + + let (input, query) = parse_query(input, schema, default_field)?; + + next_input = input; + + subqueries.push((occur, query)); + } + + // Query optimization - 0 elements or 1 element can be special cased + let query = if subqueries.is_empty() { + Box::new(EmptyQuery) + } else if subqueries.len() == 1 && subqueries[0].0 == Occur::Must { + subqueries.remove(0).1 + } else { + Box::new(BooleanQuery::new(subqueries)) + }; + + Ok((next_input, query)) +} + +fn value_with_prefix(prefix: &str, value: &str) -> String { + format!( + "{}{}{}", + prefix, + if prefix.is_empty() { + "" + } else { + JSON_PREFIX_SEPARATOR + }, + value + ) +} + +/// Build a query using a specified field and value, handling JSON +/// fields +/// +/// If `field` points to a valid indexed field +/// then it is used as is and the prefix is set to an empty string +/// +/// If `field` does not point to an indexed field then `default_field` +/// is used and prefix is a formed JSON prefix based on `field` that +/// should be included in any terms +fn query_with_field_and_value( + schema: &Schema, + default_field: Option, + field: &str, + func: T, +) -> Result, Err> +where + T: FnOnce(Field, &str) -> Result, TantivyError>, +{ + let Some((field, prefix)) = schema.find_field_with_default(field, default_field) else { + // If it's an invalid field then map to an empty query, which will emulate Lucene's behavior + return Ok(Box::new(EmptyQuery)); + }; + + func(field, prefix).to_nom_err() +} + +fn parse_equals_query<'a>( + input: &'a [u8], + schema: &Schema, + default_field: Option, +) -> IResult<&'a [u8], Box, ParserError> { + let (input, column) = parse_string(input)?; + let (input, text) = parse_string(input)?; + + let query = query_with_field_and_value(schema, default_field, &column, |field, prefix| { + Ok(Box::new(TermQuery::new( + Term::from_field_text(field, &value_with_prefix(prefix, &text)), + IndexRecordOption::Basic, + ))) + })?; + + Ok((input, query)) +} + +fn parse_regex_query<'a>( + input: &'a [u8], + schema: &Schema, + default_field: Option, +) -> IResult<&'a [u8], Box, ParserError> { + let (input, column) = parse_string(input)?; + let (input, text) = parse_string(input)?; + + let query = query_with_field_and_value(schema, default_field, &column, |field, prefix| { + let query = RangeAwareRegexQuery::from_pattern(&text, prefix, field)?; + + Ok(Box::new(query)) + })?; + + Ok((input, query)) +} + +fn parse_term_in_query<'a>( + input: &'a [u8], + schema: &Schema, + default_field: Option, +) -> IResult<&'a [u8], Box, ParserError> { + let (input, column) = parse_string(input)?; + let (input, term_count) = le_u16(input)?; + + // Repeated for each term + let mut terms = vec![]; + let mut next_input = input; + for _ in 0..term_count { + let (input, text) = parse_string(next_input)?; + + if let Some((field, prefix)) = schema.find_field_with_default(&column, default_field) { + terms.push(Term::from_field_text( + field, + &value_with_prefix(prefix, &text), + )); + }; + + next_input = input; + } + let query = TermSetQuery::new(terms); + + Ok((next_input, Box::new(query))) +} + +fn parse_prefix_query<'a>( + input: &'a [u8], + schema: &Schema, + default_field: Option, +) -> IResult<&'a [u8], Box, ParserError> { + let (input, column) = parse_string(input)?; + let (input, text) = parse_string(input)?; + + let query = query_with_field_and_value(schema, default_field, &column, |field, prefix| { + let query = PrefixQuery::new(&text, prefix, field); + + Ok(Box::new(query)) + })?; + + Ok((input, query)) +} + +fn parse_long_range_query<'a>( + input: &'a [u8], + schema: &Schema, + _default_field: Option, +) -> IResult<&'a [u8], Box, ParserError> { + let (input, column) = parse_string(input)?; + + let field = schema.get_field(&column).to_nom_err()?; + let field_name = schema.get_field_entry(field).name(); + + // 8 byte start + let (input, start) = le_i64(input)?; + + // 8 byte end + let (input, end) = le_i64(input)?; + + let query = RangeQuery::new_i64_bounds( + field_name.to_string(), + Bound::Included(start), + Bound::Included(end), + ); + + Ok((input, Box::new(query))) +} + +#[cfg(test)] +mod tests { + use bytes::BufMut; + use tantivy::{collector::DocSetCollector, query::Occur}; + use tantivy_utils::field_constants::PART_ID; + + use tantivy_utils::test_utils::{ + build_test_schema, COL1_NAME, COL2_NAME, JSON_ATTRIBUTE1_NAME, JSON_ATTRIBUTE2_NAME, + JSON_COL_NAME, + }; + + use super::*; + + #[test] + fn test_parse_equals() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "ABC"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter.len() as u16); + buf.put_slice(filter.as_bytes()); + + let (_, query) = parse_equals_query(&buf, &index.schema, None).expect("Should succeed"); + + let unboxed = query.downcast_ref::().unwrap(); + + assert_eq!( + unboxed.term().field(), + index.schema.get_field(COL1_NAME).unwrap() + ); + assert_eq!(unboxed.term().value().as_str().unwrap(), filter); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_equals_json_field() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "value"; + + buf.put_u16_le(JSON_ATTRIBUTE1_NAME.len() as u16); + buf.put_slice(JSON_ATTRIBUTE1_NAME.as_bytes()); + buf.put_u16_le(filter.len() as u16); + buf.put_slice(filter.as_bytes()); + + let (_, query) = parse_equals_query(&buf, &index.schema, Some(index.json_field)) + .expect("Should succeed"); + + let unboxed = query.downcast_ref::().unwrap(); + + assert_eq!( + unboxed.term().field(), + index.schema.get_field(JSON_COL_NAME).unwrap() + ); + assert_eq!( + unboxed.term().value().as_str().unwrap(), + format!("{}\0s{}", JSON_ATTRIBUTE1_NAME, filter) + ); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_equals_missing_value() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "ABCD"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter.len() as u16); + + let err = parse_equals_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing requires 4 bytes/chars"); + } + + #[test] + fn test_parse_equals_invalid_col_name() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "ABCD"; + + buf.put_u16_le(4); + buf.put_slice("invl".as_bytes()); + buf.put_u16_le(filter.len() as u16); + buf.put_slice(filter.as_bytes()); + + let (_, query) = parse_equals_query(&buf, &index.schema, Some(index.json_field)) + .expect("Should succeed"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 0); + } + + #[test] + fn test_parse_equals_missing_col_name() { + let index = build_test_schema(); + + let mut buf = vec![]; + + buf.put_u16_le(COL1_NAME.len() as u16); + + let err = parse_equals_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing requires 4 bytes/chars"); + } + + #[test] + fn test_parse_boolean() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "ABC"; + let filter2 = "abc"; + + buf.put_u8(Occurs::Must as u8); + buf.put_u8(QueryTypeId::Equals as u8); + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + + buf.put_u8(Occurs::MustNot as u8); + buf.put_u8(QueryTypeId::Equals as u8); + buf.put_u16_le(COL2_NAME.len() as u16); + buf.put_slice(COL2_NAME.as_bytes()); + buf.put_u16_le(filter2.len() as u16); + buf.put_slice(filter2.as_bytes()); + + buf.put_u8(0); // End of boolean marker + + let (_, query) = parse_boolean_query(&buf, &index.schema, None).expect("Should succeed"); + + let unboxed = query.downcast_ref::().unwrap(); + + assert_eq!(unboxed.clauses().len(), 2); + + let (occur, clause) = unboxed.clauses().first().unwrap(); + let clause = clause.downcast_ref::().unwrap(); + + assert_eq!(Occur::Must, *occur); + + assert_eq!( + clause.term().field(), + index.schema.get_field(COL1_NAME).unwrap() + ); + assert_eq!(clause.term().value().as_str().unwrap(), filter1); + + let (occur, clause) = unboxed.clauses().get(1).unwrap(); + let clause = clause.downcast_ref::().unwrap(); + + assert_eq!(Occur::MustNot, *occur); + + assert_eq!( + clause.term().field(), + index.schema.get_field(COL2_NAME).unwrap() + ); + assert_eq!(clause.term().value().as_str().unwrap(), filter2); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_boolean_nested() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "ABC"; + let filter2 = "abc"; + + // Nest boolean queries - (All) And (Equals And Not Equals) + buf.put_u8(Occurs::Must as u8); + buf.put_u8(QueryTypeId::Boolean as u8); + + buf.put_u8(Occurs::Must as u8); + buf.put_u8(QueryTypeId::MatchAll as u8); + buf.put_u8(0); // End of boolean marker + + buf.put_u8(Occurs::Must as u8); + buf.put_u8(QueryTypeId::Boolean as u8); + + buf.put_u8(Occurs::Must as u8); + buf.put_u8(QueryTypeId::Equals as u8); + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + + buf.put_u8(Occurs::MustNot as u8); + buf.put_u8(QueryTypeId::Equals as u8); + buf.put_u16_le(COL2_NAME.len() as u16); + buf.put_slice(COL2_NAME.as_bytes()); + buf.put_u16_le(filter2.len() as u16); + buf.put_slice(filter2.as_bytes()); + + buf.put_u8(0); // End of boolean marker + buf.put_u8(0); // End of boolean marker + + let (_, query) = parse_boolean_query(&buf, &index.schema, None).expect("Should succeed"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_boolean_json_fields() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "value"; + let filter2 = "value"; + + buf.put_u8(Occurs::Must as u8); + buf.put_u8(QueryTypeId::Equals as u8); + buf.put_u16_le(JSON_ATTRIBUTE1_NAME.len() as u16); + buf.put_slice(JSON_ATTRIBUTE1_NAME.as_bytes()); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + + buf.put_u8(Occurs::MustNot as u8); + buf.put_u8(QueryTypeId::Equals as u8); + buf.put_u16_le(JSON_ATTRIBUTE2_NAME.len() as u16); + buf.put_slice(JSON_ATTRIBUTE2_NAME.as_bytes()); + buf.put_u16_le(filter2.len() as u16); + buf.put_slice(filter2.as_bytes()); + + buf.put_u8(0); // End of boolean marker + + let (_, query) = parse_boolean_query(&buf, &index.schema, Some(index.json_field)) + .expect("Should succeed"); + + let unboxed = query.downcast_ref::().unwrap(); + + assert_eq!(unboxed.clauses().len(), 2); + + let (occur, clause) = unboxed.clauses().first().unwrap(); + let clause = clause.downcast_ref::().unwrap(); + + assert_eq!(Occur::Must, *occur); + + assert_eq!( + clause.term().field(), + index.schema.get_field(JSON_COL_NAME).unwrap() + ); + assert_eq!( + clause.term().value().as_str().unwrap(), + format!("{}\0s{}", JSON_ATTRIBUTE1_NAME, filter1) + ); + + let (occur, clause) = unboxed.clauses().get(1).unwrap(); + let clause = clause.downcast_ref::().unwrap(); + + assert_eq!(Occur::MustNot, *occur); + + assert_eq!( + clause.term().field(), + index.schema.get_field(JSON_COL_NAME).unwrap() + ); + assert_eq!( + clause.term().value().as_str().unwrap(), + format!("{}\0s{}", JSON_ATTRIBUTE2_NAME, filter2) + ); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_boolean_missing_end() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "ABCD"; + + buf.put_u8(Occurs::Must as u8); + buf.put_u8(QueryTypeId::Equals as u8); + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + + let err = parse_boolean_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing requires 1 bytes/chars"); + } + + #[test] + fn test_parse_boolean_invalid_type() { + let index = build_test_schema(); + + let mut buf = vec![]; + + buf.put_u8(Occurs::Must as u8); + buf.put_u8(255); + + let err = parse_boolean_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing Failure: UnknownType(255)"); + } + + #[test] + fn test_parse_boolean_invalid_occur() { + let index = build_test_schema(); + + let mut buf = vec![]; + + buf.put_u8(255); + + let err = parse_boolean_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing Failure: UnknownOccur(255)"); + } + + #[test] + fn test_parse_regex() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "AB.*"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter.len() as u16); + buf.put_slice(filter.as_bytes()); + + let (_, query) = parse_regex_query(&buf, &index.schema, None).expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create regex"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_partial() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "AB"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter.len() as u16); + buf.put_slice(filter.as_bytes()); + + let (_, query) = parse_regex_query(&buf, &index.schema, None).expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create regex"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Should not match since it only covers the start of string + assert_eq!(results.len(), 0); + } + + #[test] + fn test_parse_regex_json_field() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "val.*"; + + buf.put_u16_le(JSON_ATTRIBUTE1_NAME.len() as u16); + buf.put_slice(JSON_ATTRIBUTE1_NAME.as_bytes()); + buf.put_u16_le(filter.len() as u16); + buf.put_slice(filter.as_bytes()); + + let (_, query) = + parse_regex_query(&buf, &index.schema, Some(index.json_field)).expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create regex"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_regex_missing_value() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "a.*"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter.len() as u16); + + let err = parse_regex_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing requires 3 bytes/chars"); + } + + #[test] + fn test_parse_regex_invalid_col_name() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter = "a.*"; + + buf.put_u16_le(4); + buf.put_slice("invl".as_bytes()); + buf.put_u16_le(filter.len() as u16); + buf.put_slice(filter.as_bytes()); + + let (_, query) = + parse_regex_query(&buf, &index.schema, Some(index.json_field)).expect("Should succeed"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 0); + } + + #[test] + fn test_parse_regex_missing_col_name() { + let index = build_test_schema(); + + let mut buf = vec![]; + + buf.put_u16_le(COL1_NAME.len() as u16); + + let err = parse_regex_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing requires 4 bytes/chars"); + } + + #[test] + fn test_parse_term_in() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "ABC"; + let filter2 = "DEF"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(2); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + buf.put_u16_le(filter2.len() as u16); + buf.put_slice(filter2.as_bytes()); + + let (_, query) = parse_term_in_query(&buf, &index.schema, None).expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create term set"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 2); + } + + #[test] + fn test_parse_term_in_json_field() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "value"; + let filter2 = "othervalue"; + + buf.put_u16_le(JSON_ATTRIBUTE1_NAME.len() as u16); + buf.put_slice(JSON_ATTRIBUTE1_NAME.as_bytes()); + buf.put_u16_le(2); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + buf.put_u16_le(filter2.len() as u16); + buf.put_slice(filter2.as_bytes()); + + let (_, query) = parse_term_in_query(&buf, &index.schema, Some(index.json_field)) + .expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create term set"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 2); + } + + #[test] + fn test_parse_term_in_incomplete_term_list() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "abcd"; + let filter2 = "def"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(3); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + buf.put_u16_le(filter2.len() as u16); + buf.put_slice(filter2.as_bytes()); + + let err = parse_term_in_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing requires 2 bytes/chars"); + } + + #[test] + fn test_parse_prefix() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "AB"; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + + let (_, query) = parse_prefix_query(&buf, &index.schema, None).expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create prefix"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_prefix_json_field() { + let index = build_test_schema(); + + let mut buf = vec![]; + + let filter1 = "val"; + + buf.put_u16_le(JSON_ATTRIBUTE1_NAME.len() as u16); + buf.put_slice(JSON_ATTRIBUTE1_NAME.as_bytes()); + buf.put_u16_le(filter1.len() as u16); + buf.put_slice(filter1.as_bytes()); + + let (_, query) = parse_prefix_query(&buf, &index.schema, Some(index.json_field)) + .expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create prefix"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_long_range() { + let index = build_test_schema(); + + let mut buf = vec![]; + + buf.put_u16_le(PART_ID.len() as u16); + buf.put_slice(PART_ID.as_bytes()); + buf.put_i64_le(1); + buf.put_i64_le(2); + + let (_, query) = parse_long_range_query(&buf, &index.schema, None).expect("Should succeed"); + + let _ = query + .downcast_ref::() + .expect("Should create range"); + + let collector = DocSetCollector; + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!(results.len(), 1); + } + + #[test] + fn test_parse_long_range_missing_end() { + let index = build_test_schema(); + + let mut buf = vec![]; + + buf.put_u16_le(COL1_NAME.len() as u16); + buf.put_slice(COL1_NAME.as_bytes()); + buf.put_i64_le(1); + + let err = parse_long_range_query(&buf, &index.schema, None).expect_err("Should fail"); + + assert_eq!(format!("{err}"), "Parsing Error: Nom(Eof)"); + } +} diff --git a/core/src/rust/filodb_core/src/query_parser/filodb_query.rs b/core/src/rust/filodb_core/src/query_parser/filodb_query.rs new file mode 100644 index 0000000000..53b577a7f5 --- /dev/null +++ b/core/src/rust/filodb_core/src/query_parser/filodb_query.rs @@ -0,0 +1,274 @@ +//! Cachable query implementation + +use std::{ops::Bound, sync::Arc}; + +use quick_cache::Weighter; +use tantivy::{ + query::{AllQuery, Query, RangeQuery, TermQuery, TermSetQuery}, + schema::{Field, IndexRecordOption, Schema}, + SegmentId, TantivyError, Term, +}; +use tantivy_common::BitSet; +use tantivy_utils::field_constants; + +use super::parse_query; + +/// A query that can potentially be cached +/// +/// We can't just hold a reference to Tantivy's Query object because +/// they don't implement Hash/Equals so they can't be a key +#[derive(Debug, Hash, PartialEq, Eq, PartialOrd, Ord, Clone)] +pub enum FiloDBQuery { + /// A complex query that is serialized in byte form + Complex(Arc>), + /// Search by part key + ByPartKey(Arc>), + /// Search by list of part IDs + ByPartIds(Arc>), + /// Search by end time + ByEndTime(i64), + /// Search for single part ID (not cached) + ByPartId(i32), + /// All docs query (not cached) + All, +} + +impl tantivy_utils::query::cache::CachableQuery for FiloDBQuery { + fn should_cache(&self) -> bool { + match self { + FiloDBQuery::Complex(_) => true, + FiloDBQuery::ByPartIds(_) => true, + FiloDBQuery::ByEndTime(_) => true, + // No point caching all docs - the "query" is constant time anyway + &FiloDBQuery::All => false, + // A single term lookup is very efficient - no benefit in caching the doc ID + FiloDBQuery::ByPartId(_) => false, + // Also single term lookup + FiloDBQuery::ByPartKey(_) => false, + } + } + + fn to_query( + &self, + schema: &Schema, + default_field: Option, + ) -> Result, TantivyError> { + match self { + FiloDBQuery::Complex(query_bytes) => { + let (_, query) = parse_query(query_bytes, schema, default_field) + .map_err(|e| TantivyError::InternalError(format!("{:#}", e)))?; + + Ok(query) + } + FiloDBQuery::ByPartKey(part_key) => { + let field = schema.get_field(field_constants::PART_KEY)?; + let term = Term::from_field_bytes(field, part_key); + let query = TermQuery::new(term, IndexRecordOption::Basic); + + Ok(Box::new(query)) + } + FiloDBQuery::ByPartIds(part_ids) => { + let part_id_field = schema.get_field(field_constants::PART_ID)?; + + let mut terms = Vec::with_capacity(part_ids.len()); + for id in part_ids.iter() { + let term = Term::from_field_i64(part_id_field, *id as i64); + terms.push(term); + } + + let query = TermSetQuery::new(terms); + + Ok(Box::new(query)) + } + FiloDBQuery::All => Ok(Box::new(AllQuery)), + FiloDBQuery::ByPartId(part_id) => { + let part_id_field = schema.get_field(field_constants::PART_ID)?; + let term = Term::from_field_i64(part_id_field, *part_id as i64); + + let query = TermQuery::new(term, IndexRecordOption::Basic); + + Ok(Box::new(query)) + } + FiloDBQuery::ByEndTime(ended_at) => { + let query = RangeQuery::new_i64_bounds( + field_constants::END_TIME.to_string(), + Bound::Included(0), + Bound::Included(*ended_at), + ); + + Ok(Box::new(query)) + } + } + } +} + +#[derive(Clone, Default)] +pub struct CachableQueryWeighter; + +// We want our cache to hold a maximum number of items based on their total size in RAM vs item count +// This is because not all segments are the same size / not all queries to cache are equal +// +// To do this we compute the weight of a given cache item as the size of the query key + the size +// of the cached bitfield of results. This enables quick_cache to ensure we never go too much above +// a fixed amount of RAM usage. +// +// The weight does not impact which items get evicted first, just how many need to get evicted to +// make space for a new incoming item. +impl Weighter<(SegmentId, FiloDBQuery), Arc> for CachableQueryWeighter { + fn weight(&self, key: &(SegmentId, FiloDBQuery), val: &Arc) -> u64 { + let bitset_size = ((val.max_value() as usize + 63) / 64) * 8; + let key_size = std::mem::size_of::<(SegmentId, FiloDBQuery)>(); + + let type_size = match &key.1 { + FiloDBQuery::Complex(bytes) => bytes.len() + std::mem::size_of::>(), + FiloDBQuery::ByPartKey(part_key) => part_key.len() + std::mem::size_of::>(), + FiloDBQuery::ByPartIds(part_ids) => { + (part_ids.len() * std::mem::size_of::()) + std::mem::size_of::>() + } + FiloDBQuery::All => 0, + FiloDBQuery::ByPartId(_) => 0, + FiloDBQuery::ByEndTime(_) => 0, + }; + + (type_size + key_size + bitset_size) as u64 + } +} + +#[cfg(test)] +mod tests { + use tantivy::query::EmptyQuery; + + use tantivy_utils::{query::cache::CachableQuery as _, test_utils::build_test_schema}; + + use super::*; + + #[test] + fn test_should_cache() { + assert!(FiloDBQuery::Complex(Arc::new([0u8; 0].into())).should_cache()); + assert!(FiloDBQuery::ByPartIds(Arc::new([0i32; 0].into())).should_cache()); + assert!(FiloDBQuery::ByEndTime(0).should_cache()); + assert!(!FiloDBQuery::All.should_cache()); + assert!(!FiloDBQuery::ByPartId(0).should_cache()); + assert!(!FiloDBQuery::ByPartKey(Arc::new([0u8; 0].into())).should_cache()); + } + + #[test] + fn test_complex_query() { + let index = build_test_schema(); + let weighter = CachableQueryWeighter; + let reader = index.searcher.segment_readers().first().unwrap(); + let query = FiloDBQuery::Complex(Arc::new([1u8, 0u8].into())); + + let parsed = query.to_query(&index.schema, None).expect("Should succeed"); + + assert!(parsed.is::()); + + assert_eq!( + weighter.weight( + &(reader.segment_id(), query), + &Arc::new(BitSet::with_max_value(1)) + ), + 58 + ); + } + + #[test] + fn test_partkey_query() { + let index = build_test_schema(); + let weighter = CachableQueryWeighter; + let reader = index.searcher.segment_readers().first().unwrap(); + let query = FiloDBQuery::ByPartKey(Arc::new([1u8, 0u8].into())); + + let parsed = query.to_query(&index.schema, None).expect("Should succeed"); + + assert!(parsed.is::()); + + assert_eq!( + weighter.weight( + &(reader.segment_id(), query), + &Arc::new(BitSet::with_max_value(1)) + ), + 58 + ); + } + + #[test] + fn test_endtime_query() { + let index = build_test_schema(); + let weighter = CachableQueryWeighter; + let reader = index.searcher.segment_readers().first().unwrap(); + let query = FiloDBQuery::ByEndTime(0); + + let parsed = query.to_query(&index.schema, None).expect("Should succeed"); + + assert!(parsed.is::()); + + assert_eq!( + weighter.weight( + &(reader.segment_id(), query), + &Arc::new(BitSet::with_max_value(1)) + ), + 40 + ); + } + + #[test] + fn test_all_query() { + let index = build_test_schema(); + let weighter = CachableQueryWeighter; + let reader = index.searcher.segment_readers().first().unwrap(); + let query = FiloDBQuery::All; + + let parsed = query.to_query(&index.schema, None).expect("Should succeed"); + + assert!(parsed.is::()); + + assert_eq!( + weighter.weight( + &(reader.segment_id(), query), + &Arc::new(BitSet::with_max_value(1)) + ), + 40 + ); + } + + #[test] + fn test_partid_query() { + let index = build_test_schema(); + let weighter = CachableQueryWeighter; + let reader = index.searcher.segment_readers().first().unwrap(); + let query = FiloDBQuery::ByPartId(0); + + let parsed = query.to_query(&index.schema, None).expect("Should succeed"); + + assert!(parsed.is::()); + + assert_eq!( + weighter.weight( + &(reader.segment_id(), query), + &Arc::new(BitSet::with_max_value(1)) + ), + 40 + ); + } + + #[test] + fn test_partids_query() { + let index = build_test_schema(); + let weighter = CachableQueryWeighter; + let reader = index.searcher.segment_readers().first().unwrap(); + let query = FiloDBQuery::ByPartIds(Arc::new([1, 2].into())); + + let parsed = query.to_query(&index.schema, None).expect("Should succeed"); + + assert!(parsed.is::()); + + assert_eq!( + weighter.weight( + &(reader.segment_id(), query), + &Arc::new(BitSet::with_max_value(1)) + ), + 64 + ); + } +} diff --git a/core/src/rust/filodb_core/src/reader.rs b/core/src/rust/filodb_core/src/reader.rs new file mode 100644 index 0000000000..72f814a00b --- /dev/null +++ b/core/src/rust/filodb_core/src/reader.rs @@ -0,0 +1,586 @@ +//! Methods related to reading / querying the index + +use std::sync::atomic::Ordering; + +use hashbrown::HashSet; +use jni::{ + objects::{JByteArray, JClass, JIntArray, JObject, JString}, + sys::{jbyteArray, jint, jintArray, jlong, jlongArray}, + JNIEnv, +}; +use tantivy::schema::FieldType; +use tantivy_utils::collectors::part_id_collector::PartIdCollector; +use tantivy_utils::collectors::string_field_collector::StringFieldCollector; +use tantivy_utils::collectors::time_collector::TimeCollector; +use tantivy_utils::collectors::time_range_filter::TimeRangeFilter; +use tantivy_utils::collectors::{ + index_collector::collect_from_index, part_key_record_collector::PartKeyRecordCollector, +}; +use tantivy_utils::collectors::{ + part_key_collector::PartKeyCollector, part_key_record_collector::PartKeyRecord, +}; +use tantivy_utils::field_constants::{self, facet_field_name}; + +use crate::{ + errors::{JavaException, JavaResult}, + exec::jni_exec, + jnienv::JNIEnvExt, + query_parser::filodb_query::FiloDBQuery, + state::IndexHandle, +}; + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_indexRamBytes( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) -> jlong { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + + Ok(handle.query_cache_size() as i64) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_indexMmapBytes( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) -> jlong { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + + Ok(handle.mmap_size() as i64) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_refreshReaders( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + { + let changes_pending = handle.changes_pending.swap(false, Ordering::SeqCst); + + if changes_pending { + let mut writer = handle.writer.write()?; + writer.commit()?; + } + + handle.reader.reload()?; + }; + + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_indexNumEntries( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) -> jlong { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + let searcher = handle.reader.searcher(); + + Ok(searcher.num_docs() as jlong) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_partIdsEndedBefore( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + ended_before: jlong, +) -> jintArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let query = FiloDBQuery::ByEndTime(ended_before); + let collector = PartIdCollector::new(usize::MAX, handle.column_cache.clone()); + + let results = handle.execute_cachable_query(query, collector)?; + + let java_ret = env.new_int_array(results.len() as i32)?; + env.set_int_array_region(&java_ret, 0, &results)?; + + Ok(java_ret.into_raw()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_partIdFromPartKey( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + part_id: JByteArray, +) -> jint { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let bytes = env.get_byte_array(&part_id)?; + + let query = FiloDBQuery::ByPartKey(bytes.into_boxed_slice().into()); + + let collector = PartIdCollector::new(1, handle.column_cache.clone()); + let results = handle + .execute_cachable_query(query, collector)? + .into_iter() + .next(); + + let result = results.unwrap_or(-1); + + Ok(result) + }) +} + +fn fetch_label_names( + query: FiloDBQuery, + handle: &IndexHandle, + results: &mut HashSet, + limit: i32, + start: i64, + end: i64, +) -> JavaResult<()> { + let field = facet_field_name(field_constants::LABEL_LIST); + let collector = StringFieldCollector::new( + &field, + limit as usize, + usize::MAX, + handle.column_cache.clone(), + ); + + let query_results = if matches!(query, FiloDBQuery::All) { + collect_from_index(&handle.searcher(), collector)? + } else { + let filter_collector = + TimeRangeFilter::new(&collector, start, end, handle.column_cache.clone()); + handle.execute_cachable_query(query, filter_collector)? + }; + + for (facet, _count) in query_results { + results.insert(facet.to_string()); + } + + Ok(()) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_labelNames( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + query: JByteArray, + limit: jint, + start: jlong, + end: jlong, +) -> jbyteArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let mut results = HashSet::new(); + + let query_bytes = env.get_byte_array(&query)?; + + let query = FiloDBQuery::Complex(query_bytes.into_boxed_slice().into()); + fetch_label_names(query, handle, &mut results, limit, start, end)?; + + encode_string_array(env, results) + }) +} + +fn encode_string_array(env: &mut JNIEnv, arr: HashSet) -> JavaResult { + let len: usize = arr + .iter() + .map(|s| std::mem::size_of::() + s.len()) + .sum(); + + let mut serialzied_bytes = Vec::with_capacity(len); + for s in arr.iter() { + serialzied_bytes.extend((s.len() as i32).to_le_bytes()); + serialzied_bytes.extend(s.as_bytes()); + } + + let java_ret = env.new_byte_array(len as i32)?; + let bytes_ptr = serialzied_bytes.as_ptr() as *const i8; + let bytes_ptr = unsafe { std::slice::from_raw_parts(bytes_ptr, len) }; + + env.set_byte_array_region(&java_ret, 0, bytes_ptr)?; + + Ok(java_ret.into_raw()) +} + +const LABEL_NAMES_AND_VALUES_DEFAULT_LIMIT: i32 = 100; + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_indexNames( + mut env: JNIEnv, + _class: JClass, + handle: jlong, +) -> jbyteArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let mut results = HashSet::new(); + + // For each indexed field, include it + // For map fields, include encoded sub fields + for (_field, field_entry) in handle.schema.fields() { + match field_entry.field_type() { + FieldType::JsonObject(..) => { + // Skip this, we're going to get subfields via the facet below + } + _ => { + results.insert(field_entry.name().to_string()); + } + }; + } + + let query = FiloDBQuery::All; + fetch_label_names( + query, + handle, + &mut results, + LABEL_NAMES_AND_VALUES_DEFAULT_LIMIT, + 0, + i64::MAX, + )?; + + encode_string_array(env, results) + }) +} + +// This matches the constant in PartKeyLuceneIndex.scala to keep results +// consistent between the two index types +const MAX_TERMS_TO_ITERATE: usize = 10_000; + +fn query_label_values( + query: FiloDBQuery, + handle: &IndexHandle, + mut field: String, + limit: usize, + term_limit: usize, + start: i64, + end: i64, +) -> JavaResult> { + let field_and_prefix = handle + .schema + .find_field_with_default(&field, handle.default_field); + + if let Some((f, prefix)) = field_and_prefix { + if !prefix.is_empty() { + let field_name = handle.schema.get_field_entry(f).name(); + field = format!("{}.{}", field_name, prefix); + } + + let collector = + StringFieldCollector::new(&field, limit, term_limit, handle.column_cache.clone()); + + if matches!(query, FiloDBQuery::All) { + Ok(collect_from_index(&handle.searcher(), collector)?) + } else { + let filter_collector = + TimeRangeFilter::new(&collector, start, end, handle.column_cache.clone()); + Ok(handle.execute_cachable_query(query, filter_collector)?) + } + } else { + // Invalid field, no values + Ok(vec![]) + } +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_labelValues( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + query: JByteArray, + field: JString, + top_k: jint, + start: jlong, + end: jlong, +) -> jbyteArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let top_k = top_k as usize; + + let field = env.get_rust_string(&field)?; + + let query_bytes = env.get_byte_array(&query)?; + + let query = FiloDBQuery::Complex(query_bytes.into_boxed_slice().into()); + + let results = query_label_values(query, handle, field, top_k, usize::MAX, start, end)?; + + let len: usize = results + .iter() + .map(|(s, _)| std::mem::size_of::() + s.len()) + .sum(); + + let mut serialzied_bytes = Vec::with_capacity(len); + for (s, _) in results.iter() { + serialzied_bytes.extend((s.len() as i32).to_le_bytes()); + serialzied_bytes.extend(s.as_bytes()); + } + + let java_ret = env.new_byte_array(len as i32)?; + let bytes_ptr = serialzied_bytes.as_ptr() as *const i8; + let bytes_ptr = unsafe { std::slice::from_raw_parts(bytes_ptr, len) }; + + env.set_byte_array_region(&java_ret, 0, bytes_ptr)?; + + Ok(java_ret.into_raw()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_indexValues( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + field: JString, + top_k: jint, +) -> jbyteArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let top_k = top_k as usize; + + let field = env.get_rust_string(&field)?; + + let query = FiloDBQuery::All; + let results = query_label_values( + query, + handle, + field, + MAX_TERMS_TO_ITERATE, + MAX_TERMS_TO_ITERATE, + 0, + i64::MAX, + )?; + + // String length, plus count, plus string data + let results_len: usize = results + .iter() + .take(top_k) + .map(|(value, _)| value.len() + std::mem::size_of::() + std::mem::size_of::()) + .sum(); + let mut serialzied_bytes = Vec::with_capacity(results_len); + for (value, count) in results.into_iter().take(top_k) { + serialzied_bytes.extend(count.to_le_bytes()); + serialzied_bytes.extend((value.len() as i32).to_le_bytes()); + serialzied_bytes.extend(value.as_bytes()); + } + + let java_ret = env.new_byte_array(results_len as i32)?; + let bytes_ptr = serialzied_bytes.as_ptr() as *const i8; + let bytes_ptr = unsafe { std::slice::from_raw_parts(bytes_ptr, results_len) }; + + env.set_byte_array_region(&java_ret, 0, bytes_ptr)?; + + Ok(java_ret.into_raw()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_queryPartIds( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + query: JByteArray, + limit: jint, + start: jlong, + end: jlong, +) -> jintArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let query_bytes = env.get_byte_array(&query)?; + + let query = FiloDBQuery::Complex(query_bytes.into_boxed_slice().into()); + + let collector = PartIdCollector::new(limit as usize, handle.column_cache.clone()); + let filter_collector = + TimeRangeFilter::new(&collector, start, end, handle.column_cache.clone()); + + let results = handle.execute_cachable_query(query, filter_collector)?; + + let java_ret = env.new_int_array(results.len() as i32)?; + env.set_int_array_region(&java_ret, 0, &results)?; + + Ok(java_ret.into_raw()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_queryPartKeyRecords( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + query: JByteArray, + limit: jint, + start: jlong, + end: jlong, +) -> jbyteArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let query_bytes = env.get_byte_array(&query)?; + + let searcher = handle.searcher(); + let query = FiloDBQuery::Complex(query_bytes.into_boxed_slice().into()); + + let collector = PartKeyRecordCollector::new(limit as usize, handle.column_cache.clone()); + let filter_collector = + TimeRangeFilter::new(&collector, start, end, handle.column_cache.clone()); + let results = + handle.execute_cachable_query_with_searcher(query, filter_collector, &searcher)?; + + let mut results: Vec = results + .into_iter() + .map(|x| x.resolve(&searcher)) + .collect::, _>>()?; + + let results_len: usize = results.iter().map(|x| x.serialized_len()).sum(); + let mut results_vec: Vec = Vec::with_capacity(results_len); + + for r in results.drain(..) { + r.serialize(&mut results_vec); + } + + let java_ret = env.new_byte_array(results_len as i32)?; + let bytes_ptr = results_vec.as_ptr() as *const i8; + let bytes_ptr = unsafe { std::slice::from_raw_parts(bytes_ptr, results_len) }; + + env.set_byte_array_region(&java_ret, 0, bytes_ptr)?; + + Ok(java_ret.into_raw()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_queryPartKey( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + query: JByteArray, + limit: jint, + start: jlong, + end: jlong, +) -> jbyteArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + if limit != 1 { + return Err(JavaException::new_runtime_exception( + "Only limit of 1 is supported for queryPartKey", + )); + } + + let query_bytes = env.get_byte_array(&query)?; + let query = FiloDBQuery::Complex(query_bytes.into_boxed_slice().into()); + let searcher = handle.searcher(); + + let collector = PartKeyCollector::new(); + let filter_collector = + TimeRangeFilter::new(&collector, start, end, handle.column_cache.clone()); + + let results = + handle.execute_cachable_query_with_searcher(query, filter_collector, &searcher)?; + + let java_ret = match results { + Some(part_key) => { + let part_key = part_key.resolve(&searcher)?; + + let bytes_obj = env.new_byte_array(part_key.len() as i32)?; + let bytes_ptr = part_key.as_ptr() as *const i8; + let bytes_ptr = unsafe { std::slice::from_raw_parts(bytes_ptr, part_key.len()) }; + + env.set_byte_array_region(&bytes_obj, 0, bytes_ptr)?; + + bytes_obj.into_raw() + } + None => JObject::null().into_raw(), + }; + + Ok(java_ret) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_startTimeFromPartIds( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + part_ids: JIntArray, +) -> jlongArray { + jni_exec(&mut env, |env| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let len = env.get_array_length(&part_ids)?; + + let mut part_id_values = vec![0i32; len as usize]; + env.get_int_array_region(&part_ids, 0, &mut part_id_values[..])?; + + let query = FiloDBQuery::ByPartIds(part_id_values.into_boxed_slice().into()); + + let collector = TimeCollector::new( + field_constants::START_TIME, + usize::MAX, + handle.column_cache.clone(), + ); + + let results = handle.execute_cachable_query(query, collector)?; + + // Return is encoded as a single long array of tuples of part id, start time repeated. For example + // the first part ID is at offset 0, then its start time is at offset 1, the next part id is at offset 2 + // and its start time is at offset 3, etc. + // + // This lets us avoid non primitive types in the return, which greatly improves performance + let java_ret = env.new_long_array(results.len() as i32 * 2)?; + let mut local_array = Vec::with_capacity(results.len() * 2); + + for (p, t) in results { + local_array.push(p as i64); + local_array.push(t); + } + + env.set_long_array_region(&java_ret, 0, &local_array)?; + + Ok(java_ret.into_raw()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_filodb_core_memstore_TantivyNativeMethods_00024_endTimeFromPartId( + mut env: JNIEnv, + _class: JClass, + handle: jlong, + part_id: jint, +) -> jlong { + jni_exec(&mut env, |_| { + let handle = IndexHandle::get_ref_from_handle(handle); + + let query = FiloDBQuery::ByPartId(part_id); + + let collector = + TimeCollector::new(field_constants::END_TIME, 1, handle.column_cache.clone()); + + let results = handle.execute_cachable_query(query, collector)?; + + let result = results + .into_iter() + .next() + .map(|(_id, time)| time) + .unwrap_or(-1); + + Ok(result) + }) +} diff --git a/core/src/rust/filodb_core/src/state.rs b/core/src/rust/filodb_core/src/state.rs new file mode 100644 index 0000000000..284a49c5ff --- /dev/null +++ b/core/src/rust/filodb_core/src/state.rs @@ -0,0 +1,160 @@ +//! State objects shared with Java + +use std::{ + collections::{BTreeMap, HashMap}, + sync::{atomic::AtomicBool, RwLock}, +}; + +use filesize::PathExt; +use jni::sys::jlong; +use tantivy::{ + directory::{MmapDirectory, WatchCallback, WatchHandle}, + schema::{Field, OwnedValue, Schema}, + Directory, IndexReader, IndexWriter, Searcher, TantivyDocument, TantivyError, +}; +use tantivy_utils::{ + collectors::{ + column_cache::ColumnCache, + limited_collector::{LimitedCollector, LimitedSegmentCollector}, + }, + query::cache::QueryCache, +}; + +use crate::query_parser::filodb_query::{CachableQueryWeighter, FiloDBQuery}; + +pub struct IndexHandle { + // Fields that don't need explicit synchronization + // + // + // Schema for this nidex + pub schema: Schema, + // Default field for JSON searches + pub default_field: Option, + // Active reader + pub reader: IndexReader, + // Cache of query -> docs + query_cache: QueryCache, + // Are there changes pending to commit + pub changes_pending: AtomicBool, + // Column lookup cache + pub column_cache: ColumnCache, + // Mmap dir - used for stats only + pub mmap_directory: MmapDirectory, + // Watch handle - notifies when to clear the column cache + _watch_handle: WatchHandle, + + // Fields that need synchronization + // + // + // Active writer + pub writer: RwLock, +} + +impl IndexHandle { + #[allow(clippy::too_many_arguments)] + pub fn new_handle( + schema: Schema, + default_field: Option, + writer: IndexWriter, + reader: IndexReader, + mmap_directory: MmapDirectory, + column_cache_size: u64, + query_cache_max_size: u64, + query_cache_estimated_item_size: u64, + ) -> tantivy::Result { + let estimated_item_count: u64 = query_cache_max_size / query_cache_estimated_item_size; + let column_cache = ColumnCache::new(column_cache_size as usize); + + let cache = column_cache.clone(); + // When the index segment list changes, clear the column cache to release those mmaped files + let watch_handle = mmap_directory.watch(WatchCallback::new(move || { + cache.clear(); + }))?; + + let obj = Box::new(Self { + schema, + default_field, + writer: RwLock::new(writer), + reader, + changes_pending: AtomicBool::new(false), + query_cache: QueryCache::new(estimated_item_count, query_cache_max_size), + column_cache, + mmap_directory, + _watch_handle: watch_handle, + }); + + Ok(Box::into_raw(obj) as jlong) + } + + /// Decode handle back into a reference + pub fn get_ref_from_handle<'a>(handle: jlong) -> &'a Self { + let ptr = handle as *const IndexHandle; + + unsafe { &*ptr } + } + + pub fn query_cache_stats(&self) -> (u64, u64) { + self.query_cache.query_cache_stats() + } + + pub fn query_cache_size(&self) -> u64 { + self.query_cache.size() + } + + pub fn mmap_size(&self) -> u64 { + self.mmap_directory + .get_cache_info() + .mmapped + .into_iter() + .map(|p| p.as_path().size_on_disk().unwrap_or(0)) + .sum() + } + + pub fn searcher(&self) -> Searcher { + self.reader.searcher() + } + + pub fn execute_cachable_query( + &self, + cachable_query: FiloDBQuery, + collector: C, + ) -> Result + where + C: LimitedCollector, + C::Child: LimitedSegmentCollector, + { + let searcher = self.reader.searcher(); + + self.execute_cachable_query_with_searcher(cachable_query, collector, &searcher) + } + + pub fn execute_cachable_query_with_searcher( + &self, + cachable_query: FiloDBQuery, + collector: C, + searcher: &Searcher, + ) -> Result + where + C: LimitedCollector, + C::Child: LimitedSegmentCollector, + { + self.query_cache.search( + searcher, + &self.schema, + self.default_field, + cachable_query, + collector, + ) + } +} + +/// A document that is actively being built up for ingesting +#[derive(Default)] +pub struct IngestingDocument { + // List of map entries we're building up to store in the document + pub map_values: HashMap>, + // List of field names in the document being ingested + pub field_names: Vec, + // Document state for ingestion + pub doc: TantivyDocument, +} diff --git a/core/src/rust/tantivy_utils/Cargo.toml b/core/src/rust/tantivy_utils/Cargo.toml new file mode 100644 index 0000000000..6491e78e98 --- /dev/null +++ b/core/src/rust/tantivy_utils/Cargo.toml @@ -0,0 +1,13 @@ +[package] +name = "tantivy_utils" +version = "0.1.0" +edition = "2021" +rust-version = "1.78" + +[dependencies] +hashbrown = "0.14.5" +nohash-hasher = "0.2.0" +quick_cache = { version = "0.6.2", features = ["stats"] } +tantivy = "0.22.0" +tantivy-common = "0.7.0" +tantivy-fst = "0.5.0" diff --git a/core/src/rust/tantivy_utils/src/collectors.rs b/core/src/rust/tantivy_utils/src/collectors.rs new file mode 100644 index 0000000000..2fa1ab07a3 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors.rs @@ -0,0 +1,11 @@ +//! Common collectors + +pub mod column_cache; +pub mod index_collector; +pub mod limited_collector; +pub mod part_id_collector; +pub mod part_key_collector; +pub mod part_key_record_collector; +pub mod string_field_collector; +pub mod time_collector; +pub mod time_range_filter; diff --git a/core/src/rust/tantivy_utils/src/collectors/column_cache.rs b/core/src/rust/tantivy_utils/src/collectors/column_cache.rs new file mode 100644 index 0000000000..a0f868f3ad --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/column_cache.rs @@ -0,0 +1,270 @@ +//! Cache for fast field columns + +use std::sync::Arc; + +use quick_cache::{sync::Cache, Equivalent}; +use tantivy::{ + columnar::{BytesColumn, Column, DynamicColumn, HasAssociatedColumnType, StrColumn}, + SegmentId, SegmentReader, +}; + +// Max column items to cache. These are relatively cheap (< 1KB) +// 1 item per column, per segment +const DEFAULT_COLUMN_CACHE_ITEM_COUNT: usize = 1000; + +// Helper to avoid having to clone strings just to do a cache lookup +#[derive(Hash, PartialEq, Eq, Debug, Clone)] +struct CacheKey<'a>(SegmentId, &'a str); + +impl<'a> From> for (SegmentId, String) { + fn from(value: CacheKey<'a>) -> Self { + (value.0, value.1.to_string()) + } +} + +impl<'a> Equivalent<(SegmentId, String)> for CacheKey<'a> { + fn equivalent(&self, key: &(SegmentId, String)) -> bool { + self.0 == key.0 && self.1 == key.1 + } +} + +/// Opening DynamicColumn instances requires parsing some headers +/// and other items that while fast, can add up if you're doing this +/// thousands of times a second. Since columns for a given segment +/// are immutable once created caching this parsed data is safe +/// and cheap and can result in major speedups on things like +/// point queries. +#[derive(Clone)] +pub struct ColumnCache { + cache: Arc>, +} + +impl Default for ColumnCache { + fn default() -> Self { + Self::new(DEFAULT_COLUMN_CACHE_ITEM_COUNT) + } +} + +impl ColumnCache { + pub fn new(size: usize) -> Self { + Self { + cache: Arc::new(Cache::new(size)), + } + } + + pub fn clear(&self) { + self.cache.clear(); + } + + pub fn stats(&self) -> (u64, u64) { + (self.cache.hits(), self.cache.misses()) + } + + pub fn get_column( + &self, + reader: &SegmentReader, + field: &str, + ) -> tantivy::Result>> + where + T: HasAssociatedColumnType, + DynamicColumn: From>, + DynamicColumn: Into>>, + { + let key = CacheKey(reader.segment_id(), field); + + if let Some(col) = self.cache.get(&key) { + Ok(col.into()) + } else { + let column: Option> = reader.fast_fields().column_opt(field)?; + + if let Some(col) = column { + self.cache.insert(key.into(), col.clone().into()); + + Ok(Some(col)) + } else { + Ok(None) + } + } + } + + pub fn get_bytes_column( + &self, + reader: &SegmentReader, + field: &str, + ) -> tantivy::Result> { + let key = CacheKey(reader.segment_id(), field); + + if let Some(col) = self.cache.get(&key) { + Ok(col.into()) + } else { + let column: Option = reader.fast_fields().bytes(field)?; + + if let Some(col) = column { + self.cache.insert(key.into(), col.clone().into()); + + Ok(Some(col)) + } else { + Ok(None) + } + } + } + + pub fn get_str_column( + &self, + reader: &SegmentReader, + field: &str, + ) -> tantivy::Result> { + let key = CacheKey(reader.segment_id(), field); + + if let Some(col) = self.cache.get(&key) { + Ok(col.into()) + } else { + let column: Option = reader.fast_fields().str(field)?; + + if let Some(col) = column { + self.cache.insert(key.into(), col.clone().into()); + + Ok(Some(col)) + } else { + Ok(None) + } + } + } +} + +#[cfg(test)] +mod tests { + use crate::field_constants::{PART_ID, PART_KEY}; + use std::hash::{DefaultHasher, Hash, Hasher}; + + use crate::test_utils::{build_test_schema, COL1_NAME}; + + use super::*; + + #[test] + fn test_cache_key_equivilance() { + let index = build_test_schema(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let key = CacheKey(reader.segment_id(), "foo"); + let owned_key: (SegmentId, String) = key.clone().into(); + + assert_eq!(key.0, owned_key.0); + assert_eq!(key.1, owned_key.1); + + let mut hasher = DefaultHasher::new(); + key.hash(&mut hasher); + let key_hash = hasher.finish(); + + let mut hasher = DefaultHasher::new(); + owned_key.hash(&mut hasher); + let owned_key_hash = hasher.finish(); + + assert_eq!(key_hash, owned_key_hash); + } + + #[test] + fn test_cache_miss() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let _: Column = cache + .get_column(reader, PART_ID) + .expect("Should succeed") + .expect("Should return one item"); + + assert_eq!(cache.cache.misses(), 1); + assert_eq!(cache.cache.hits(), 0); + } + + #[test] + fn test_cache_hit() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let _: Column = cache + .get_column(reader, PART_ID) + .expect("Should succeed") + .expect("Should return one item"); + + let _: Column = cache + .get_column(reader, PART_ID) + .expect("Should succeed") + .expect("Should return one item"); + + assert_eq!(cache.cache.misses(), 1); + assert_eq!(cache.cache.hits(), 1); + } + + #[test] + fn test_str_cache_miss() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let _ = cache + .get_str_column(reader, COL1_NAME) + .expect("Should succeed") + .expect("Should return one item"); + + assert_eq!(cache.cache.misses(), 1); + assert_eq!(cache.cache.hits(), 0); + } + + #[test] + fn test_str_cache_hit() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let _ = cache + .get_str_column(reader, COL1_NAME) + .expect("Should succeed") + .expect("Should return one item"); + + let _ = cache + .get_str_column(reader, COL1_NAME) + .expect("Should succeed") + .expect("Should return one item"); + + assert_eq!(cache.cache.misses(), 1); + assert_eq!(cache.cache.hits(), 1); + } + + #[test] + fn test_bytes_cache_miss() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let _ = cache + .get_bytes_column(reader, PART_KEY) + .expect("Should succeed") + .expect("Should return one item"); + + assert_eq!(cache.cache.misses(), 1); + assert_eq!(cache.cache.hits(), 0); + } + + #[test] + fn test_bytes_cache_hit() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let _ = cache + .get_bytes_column(reader, PART_KEY) + .expect("Should succeed") + .expect("Should return one item"); + + let _ = cache + .get_bytes_column(reader, PART_KEY) + .expect("Should succeed") + .expect("Should return one item"); + + assert_eq!(cache.cache.misses(), 1); + assert_eq!(cache.cache.hits(), 1); + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/index_collector.rs b/core/src/rust/tantivy_utils/src/collectors/index_collector.rs new file mode 100644 index 0000000000..33ae4a73ff --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/index_collector.rs @@ -0,0 +1,43 @@ +//! Collector that can run over an entire index without a query +//! + +use tantivy::{collector::SegmentCollector, Searcher, SegmentReader, TantivyError}; + +use super::limited_collector::{LimitCounter, LimitedCollector, LimitedSegmentCollector}; + +/// Index Segment collector +pub trait IndexCollector: LimitedCollector +where + Self::Child: LimitedSegmentCollector, +{ + /// Colllect data across an entire index segment + fn collect_over_index( + &self, + reader: &SegmentReader, + limiter: &mut LimitCounter, + ) -> Result<::Fruit, TantivyError>; +} + +pub fn collect_from_index(searcher: &Searcher, collector: C) -> Result +where + C: IndexCollector, + C::Child: LimitedSegmentCollector, +{ + let segment_readers = searcher.segment_readers(); + let mut fruits: Vec<::Fruit> = + Vec::with_capacity(segment_readers.len()); + + let mut limiter = LimitCounter::new(collector.limit()); + + for segment_reader in segment_readers.iter() { + let results = collector.collect_over_index(segment_reader, &mut limiter)?; + + fruits.push(results); + + if limiter.at_limit() { + break; + } + } + + collector.merge_fruits(fruits) +} diff --git a/core/src/rust/tantivy_utils/src/collectors/limited_collector.rs b/core/src/rust/tantivy_utils/src/collectors/limited_collector.rs new file mode 100644 index 0000000000..5c04b5bd5a --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/limited_collector.rs @@ -0,0 +1,97 @@ +//! Collector that can abort early for return limits, for example + +use tantivy::{ + collector::{Collector, SegmentCollector}, + query::Weight, + DocId, Score, SegmentReader, TantivyError, TERMINATED, +}; + +mod limit_counter; +mod unlimited_collector; + +pub use limit_counter::{LimitCounter, LimitCounterOptionExt}; +pub use unlimited_collector::{UnlimitedCollector, UnlimitedSegmentCollector}; + +/// Marker struct for exceeding limits as an error +pub struct LimitExceeded; + +pub type LimitResult = Result<(), LimitExceeded>; + +/// Segment collector that can use a limiter +pub trait LimitedSegmentCollector: SegmentCollector { + fn collect_with_limiter( + &mut self, + doc: DocId, + score: Score, + limiter: Option<&mut LimitCounter>, + ) -> LimitResult; +} + +/// A collector that can use a limiter to abort early +/// This is modelled off the Lucene behavior where you can +/// throw an exception to stop the collection. +/// +/// Since Rust has no exceptions, we need an error path which +/// requires an extension trait. +pub trait LimitedCollector +where + Self: Collector, + Self::Child: LimitedSegmentCollector, +{ + /// Returns configured limit + fn limit(&self) -> usize; + + fn collect_segment_with_limiter( + &self, + weight: &dyn Weight, + segment_ord: u32, + reader: &SegmentReader, + limiter: &mut LimitCounter, + ) -> Result<::Fruit, TantivyError> { + let mut segment_collector = self.for_segment(segment_ord, reader)?; + let mut scorer = weight.scorer(reader, 1.0)?; + + // This is an extension of the logic that the base Collector trait provides: + // For each document the scorer produces: + // * Check if it is alive if we have an alive_bitset + // * Collect it with the limiter method + // * If the collect method returns an error that signals we're at the limit, abort + // + // This code does not handle scoring, in part because there's no usage of scoring in FiloDB. + match (reader.alive_bitset(), self.requires_scoring()) { + (Some(alive_bitset), false) => { + let mut doc = scorer.doc(); + while doc != TERMINATED { + if alive_bitset.is_alive(doc) + && segment_collector + .collect_with_limiter(doc, scorer.score(), Some(limiter)) + .is_err() + { + // Hit limit + break; + } + doc = scorer.advance(); + } + } + (None, false) => { + let mut doc = scorer.doc(); + while doc != TERMINATED { + if segment_collector + .collect_with_limiter(doc, scorer.score(), Some(limiter)) + .is_err() + { + break; + } + doc = scorer.advance(); + } + } + (_, true) => { + return Err(TantivyError::InvalidArgument( + "Scoring not supported".into(), + )); + } + } + + Ok(segment_collector.harvest()) + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/limited_collector/limit_counter.rs b/core/src/rust/tantivy_utils/src/collectors/limited_collector/limit_counter.rs new file mode 100644 index 0000000000..e08747414f --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/limited_collector/limit_counter.rs @@ -0,0 +1,66 @@ +//! Counter for limiting + +use super::{LimitExceeded, LimitResult}; + +/// Counter to keep track of and enforce a limit +pub struct LimitCounter { + limit: usize, + count: usize, +} + +impl LimitCounter { + pub fn new(limit: usize) -> Self { + Self { limit, count: 0 } + } + + /// Increment the seen items, fail if hit the limit + #[inline] + pub fn increment(&mut self) -> LimitResult { + self.count += 1; + + if self.count >= self.limit { + Err(LimitExceeded) + } else { + Ok(()) + } + } + + pub fn at_limit(&self) -> bool { + self.count >= self.limit + } +} + +pub trait LimitCounterOptionExt { + fn increment(&mut self) -> LimitResult; +} + +impl LimitCounterOptionExt for Option<&mut LimitCounter> { + #[inline] + fn increment(&mut self) -> LimitResult { + match self { + Some(limiter) => limiter.increment(), + None => Ok(()), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_limit_counter() { + let mut counter = LimitCounter::new(2); + + // Count 0 + assert!(!counter.at_limit()); + + // Count 1 + assert!(counter.increment().is_ok()); + assert!(!counter.at_limit()); + + // Count 2 + assert!(counter.increment().is_err()); + assert!(counter.at_limit()); + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/limited_collector/unlimited_collector.rs b/core/src/rust/tantivy_utils/src/collectors/limited_collector/unlimited_collector.rs new file mode 100644 index 0000000000..702a9e030b --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/limited_collector/unlimited_collector.rs @@ -0,0 +1,113 @@ +//! Helper to wrap collectors as unlimited +//! This is needed for collectors outside this crate, such as the +//! built-in Tantivy ones + +use tantivy::{ + collector::{Collector, SegmentCollector}, + DocId, Score, SegmentReader, +}; + +use super::{LimitCounter, LimitResult, LimitedCollector, LimitedSegmentCollector}; + +/// Wrapper to adapt not limited collectors into the limiting framework +pub struct UnlimitedCollector +where + T: Collector, +{ + inner: T, +} + +impl UnlimitedCollector +where + T: Collector, +{ + pub fn new(inner: T) -> Self { + Self { inner } + } +} + +impl LimitedCollector for UnlimitedCollector +where + T: Collector, +{ + fn limit(&self) -> usize { + usize::MAX + } +} + +impl Collector for UnlimitedCollector +where + T: Collector, +{ + type Fruit = T::Fruit; + + type Child = UnlimitedSegmentCollector; + + fn for_segment( + &self, + segment_local_id: tantivy::SegmentOrdinal, + segment: &SegmentReader, + ) -> tantivy::Result { + let segment_collector = self.inner.for_segment(segment_local_id, segment)?; + + Ok(UnlimitedSegmentCollector::new(segment_collector)) + } + + fn requires_scoring(&self) -> bool { + self.inner.requires_scoring() + } + + fn merge_fruits( + &self, + segment_fruits: Vec<::Fruit>, + ) -> tantivy::Result { + self.inner.merge_fruits(segment_fruits) + } +} + +pub struct UnlimitedSegmentCollector +where + T: SegmentCollector, +{ + inner: T, +} + +impl UnlimitedSegmentCollector +where + T: SegmentCollector, +{ + pub fn new(inner: T) -> Self { + Self { inner } + } +} + +impl LimitedSegmentCollector for UnlimitedSegmentCollector +where + T: SegmentCollector, +{ + fn collect_with_limiter( + &mut self, + doc: DocId, + score: Score, + _limiter: Option<&mut LimitCounter>, + ) -> LimitResult { + self.inner.collect(doc, score); + + Ok(()) + } +} + +impl SegmentCollector for UnlimitedSegmentCollector +where + T: SegmentCollector, +{ + type Fruit = T::Fruit; + + fn collect(&mut self, doc: DocId, score: Score) { + self.inner.collect(doc, score) + } + + fn harvest(self) -> Self::Fruit { + self.inner.harvest() + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/part_id_collector.rs b/core/src/rust/tantivy_utils/src/collectors/part_id_collector.rs new file mode 100644 index 0000000000..51549fb9e4 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/part_id_collector.rs @@ -0,0 +1,153 @@ +//! Collector to pull part IDs from a document + +use std::cmp::min; + +use crate::field_constants; +use tantivy::{ + collector::{Collector, SegmentCollector}, + columnar::Column, + TantivyError, +}; + +use crate::collectors::column_cache::ColumnCache; + +use super::limited_collector::{ + LimitCounterOptionExt, LimitResult, LimitedCollector, LimitedSegmentCollector, +}; + +pub struct PartIdCollector { + limit: usize, + column_cache: ColumnCache, +} + +impl PartIdCollector { + pub fn new(limit: usize, column_cache: ColumnCache) -> Self { + Self { + limit, + column_cache, + } + } +} + +impl LimitedCollector for PartIdCollector { + fn limit(&self) -> usize { + self.limit + } +} + +impl Collector for PartIdCollector { + type Fruit = Vec; + + type Child = PartIdSegmentCollector; + + fn for_segment( + &self, + _segment_local_id: tantivy::SegmentOrdinal, + segment: &tantivy::SegmentReader, + ) -> tantivy::Result { + let column: Column = self + .column_cache + .get_column(segment, field_constants::PART_ID)? + .ok_or_else(|| TantivyError::FieldNotFound(field_constants::PART_ID.to_string()))?; + + Ok(PartIdSegmentCollector { + column, + docs: Vec::new(), + }) + } + + fn requires_scoring(&self) -> bool { + false + } + + fn merge_fruits(&self, segment_fruits: Vec>) -> tantivy::Result> { + let len: usize = min(segment_fruits.iter().map(|x| x.len()).sum(), self.limit); + + let mut result = Vec::with_capacity(len); + for part_ids in segment_fruits { + result.extend(part_ids.iter().take(self.limit - result.len())); + } + + Ok(result) + } +} + +pub struct PartIdSegmentCollector { + column: Column, + docs: Vec, +} + +impl LimitedSegmentCollector for PartIdSegmentCollector { + fn collect_with_limiter( + &mut self, + doc: tantivy::DocId, + _score: tantivy::Score, + mut limiter: Option<&mut super::limited_collector::LimitCounter>, + ) -> LimitResult { + if let Some(val) = self.column.first(doc) { + self.docs.push(val as i32); + limiter.increment()?; + } + + Ok(()) + } +} + +impl SegmentCollector for PartIdSegmentCollector { + type Fruit = Vec; + + fn collect(&mut self, doc: tantivy::DocId, score: tantivy::Score) { + let _ = self.collect_with_limiter(doc, score, None); + } + + fn harvest(self) -> Self::Fruit { + self.docs + } +} + +#[cfg(test)] +mod tests { + use hashbrown::HashSet; + use tantivy::query::AllQuery; + + use crate::test_utils::build_test_schema; + + use super::*; + + #[test] + fn test_part_id_collector() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + + let collector = PartIdCollector::new(usize::MAX, cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Two docs, IDs 1 and 10 + assert_eq!( + results.into_iter().collect::>(), + [1, 10].into_iter().collect::>() + ); + } + + #[test] + fn test_part_id_collector_with_limit() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + + let collector = PartIdCollector::new(1, cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Which doc matches first is non deterministic, just check length + assert_eq!(results.len(), 1); + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/part_key_collector.rs b/core/src/rust/tantivy_utils/src/collectors/part_key_collector.rs new file mode 100644 index 0000000000..f289df3582 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/part_key_collector.rs @@ -0,0 +1,162 @@ +//! Collector for part key binary data + +use tantivy::{ + collector::{Collector, SegmentCollector}, + schema::OwnedValue, + DocAddress, Searcher, TantivyDocument, TantivyError, +}; + +use crate::field_constants::PART_KEY; + +use super::limited_collector::{LimitResult, LimitedCollector, LimitedSegmentCollector}; + +#[derive(Default)] +pub struct PartKeyCollector; + +pub struct UnresolvedPartKey(DocAddress); + +impl UnresolvedPartKey { + pub fn resolve(self, searcher: &Searcher) -> Result, TantivyError> { + let doc_data = searcher.doc::(self.0)?; + let part_key_field = searcher.schema().get_field(PART_KEY)?; + + let Some(OwnedValue::Bytes(part_key)) = doc_data + .into_iter() + .filter(|x| x.field == part_key_field) + .map(|x| x.value) + .next() + else { + return Err(TantivyError::FieldNotFound(PART_KEY.to_string())); + }; + + Ok(part_key) + } +} + +impl PartKeyCollector { + pub fn new() -> Self { + Self {} + } +} + +impl LimitedCollector for PartKeyCollector { + fn limit(&self) -> usize { + usize::MAX + } +} + +impl Collector for PartKeyCollector { + type Fruit = Option; + + type Child = PartKeySegmentCollector; + + fn for_segment( + &self, + segment_local_id: tantivy::SegmentOrdinal, + _segment: &tantivy::SegmentReader, + ) -> tantivy::Result { + Ok(PartKeySegmentCollector { + segment_local_id, + doc: None, + }) + } + + fn requires_scoring(&self) -> bool { + false + } + + fn merge_fruits( + &self, + segment_fruits: Vec>, + ) -> tantivy::Result> { + Ok(segment_fruits.into_iter().flatten().next()) + } +} + +pub struct PartKeySegmentCollector { + segment_local_id: u32, + doc: Option, +} + +impl LimitedSegmentCollector for PartKeySegmentCollector { + fn collect_with_limiter( + &mut self, + doc: tantivy::DocId, + score: tantivy::Score, + _limiter: Option<&mut super::limited_collector::LimitCounter>, + ) -> LimitResult { + self.collect(doc, score); + + Ok(()) + } +} + +impl SegmentCollector for PartKeySegmentCollector { + type Fruit = Option; + + fn collect(&mut self, doc: tantivy::DocId, _score: tantivy::Score) { + if self.doc.is_some() { + return; + } + + self.doc = Some(UnresolvedPartKey(DocAddress::new( + self.segment_local_id, + doc, + ))); + } + + fn harvest(self) -> Self::Fruit { + self.doc + } +} + +#[cfg(test)] +mod tests { + use tantivy::{ + query::{EmptyQuery, TermQuery}, + schema::IndexRecordOption, + Term, + }; + + use crate::test_utils::{build_test_schema, COL1_NAME}; + + use super::*; + + #[test] + fn test_part_key_collector() { + let index = build_test_schema(); + + let collector = PartKeyCollector::new(); + let query = TermQuery::new( + Term::from_field_text(index.schema.get_field(COL1_NAME).unwrap(), "ABC"), + IndexRecordOption::Basic, + ); + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + assert_eq!( + results.map(|r| r.resolve(&index.searcher).unwrap()), + Some(vec![0x41, 0x41]) + ); + } + + #[test] + fn test_part_key_collector_no_match() { + let index = build_test_schema(); + + let collector = PartKeyCollector::new(); + + let query = EmptyQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Which doc matches first is non deterministic, just check length + assert_eq!(results.map(|r| r.resolve(&index.searcher).unwrap()), None); + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/part_key_record_collector.rs b/core/src/rust/tantivy_utils/src/collectors/part_key_record_collector.rs new file mode 100644 index 0000000000..65b1033190 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/part_key_record_collector.rs @@ -0,0 +1,272 @@ +//! Collector for part key binary data + +use std::cmp::min; +use std::mem::size_of; + +use crate::collectors::column_cache::ColumnCache; +use crate::field_constants::{END_TIME, PART_KEY, START_TIME}; +use tantivy::schema::OwnedValue; +use tantivy::{ + collector::{Collector, SegmentCollector}, + columnar::Column, + TantivyError, +}; +use tantivy::{DocAddress, Searcher, TantivyDocument}; + +use super::limited_collector::{ + LimitCounterOptionExt, LimitResult, LimitedCollector, LimitedSegmentCollector, +}; + +/// Records returned from queries +#[derive(Debug, PartialEq, Hash, PartialOrd, Eq)] +pub struct UnresolvedPartKeyRecord { + pub doc_id: DocAddress, + pub start_time: i64, + pub end_time: i64, +} + +impl UnresolvedPartKeyRecord { + pub fn resolve(self, searcher: &Searcher) -> Result { + let doc_data = searcher.doc::(self.doc_id)?; + let part_key_field = searcher.schema().get_field(PART_KEY)?; + + let Some(OwnedValue::Bytes(part_key)) = doc_data + .into_iter() + .filter(|x| x.field == part_key_field) + .map(|x| x.value) + .next() + else { + return Err(TantivyError::FieldNotFound(PART_KEY.to_string())); + }; + + Ok(PartKeyRecord { + part_key, + start_time: self.start_time, + end_time: self.end_time, + }) + } +} + +/// Records returned from queries +#[derive(Debug, PartialEq, Hash, PartialOrd, Eq)] +pub struct PartKeyRecord { + pub part_key: Vec, + pub start_time: i64, + pub end_time: i64, +} + +impl PartKeyRecord { + pub fn serialized_len(&self) -> usize { + // Two i64 ints, 1 u32 int, byte array + self.part_key.len() + size_of::() + (2 * size_of::()) + } + + pub fn serialize(self, vec: &mut impl Extend) { + // Serialize as start time, end time, part_key len, part_key + vec.extend(self.start_time.to_le_bytes()); + vec.extend(self.end_time.to_le_bytes()); + + let len = self.part_key.len() as u32; + vec.extend(len.to_le_bytes()); + vec.extend(self.part_key); + } +} + +pub struct PartKeyRecordCollector { + limit: usize, + column_cache: ColumnCache, +} + +impl PartKeyRecordCollector { + pub fn new(limit: usize, column_cache: ColumnCache) -> Self { + Self { + limit, + column_cache, + } + } +} + +impl LimitedCollector for PartKeyRecordCollector { + fn limit(&self) -> usize { + self.limit + } +} + +impl Collector for PartKeyRecordCollector { + type Fruit = Vec; + + type Child = PartKeyRecordSegmentCollector; + + fn for_segment( + &self, + segment_local_id: tantivy::SegmentOrdinal, + segment: &tantivy::SegmentReader, + ) -> tantivy::Result { + let start_time_column: Column = self + .column_cache + .get_column(segment, START_TIME)? + .ok_or_else(|| TantivyError::FieldNotFound(START_TIME.to_string()))?; + + let end_time_column: Column = self + .column_cache + .get_column(segment, END_TIME)? + .ok_or_else(|| TantivyError::FieldNotFound(END_TIME.to_string()))?; + + Ok(PartKeyRecordSegmentCollector { + segment_ord: segment_local_id, + start_time_column, + end_time_column, + docs: Vec::new(), + }) + } + + fn requires_scoring(&self) -> bool { + false + } + + fn merge_fruits( + &self, + segment_fruits: Vec>, + ) -> tantivy::Result> { + let len: usize = min(segment_fruits.iter().map(|x| x.len()).sum(), self.limit); + + let mut result = Vec::with_capacity(len); + for part_ids in segment_fruits { + result.extend(part_ids.into_iter().take(self.limit - result.len())); + } + + Ok(result) + } +} + +pub struct PartKeyRecordSegmentCollector { + segment_ord: u32, + start_time_column: Column, + end_time_column: Column, + docs: Vec, +} + +impl LimitedSegmentCollector for PartKeyRecordSegmentCollector { + fn collect_with_limiter( + &mut self, + doc: tantivy::DocId, + _score: tantivy::Score, + mut limiter: Option<&mut super::limited_collector::LimitCounter>, + ) -> LimitResult { + let doc_id = DocAddress::new(self.segment_ord, doc); + + let Some(start_time) = self.start_time_column.first(doc) else { + return Ok(()); + }; + + let Some(end_time) = self.end_time_column.first(doc) else { + return Ok(()); + }; + + self.docs.push(UnresolvedPartKeyRecord { + doc_id, + start_time, + end_time, + }); + + limiter.increment()?; + + Ok(()) + } +} + +impl SegmentCollector for PartKeyRecordSegmentCollector { + type Fruit = Vec; + + fn collect(&mut self, doc: tantivy::DocId, score: tantivy::Score) { + let _ = self.collect_with_limiter(doc, score, None); + } + + fn harvest(self) -> Self::Fruit { + self.docs + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use tantivy::query::AllQuery; + + use crate::test_utils::build_test_schema; + + use super::*; + + #[test] + fn test_part_key_collector() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = PartKeyRecordCollector::new(usize::MAX, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Two docs, IDs 1 and 10 + assert_eq!( + results + .into_iter() + .map(|x| x.resolve(&index.searcher).unwrap()) + .collect::>(), + [ + PartKeyRecord { + part_key: vec![0x41, 0x41], + start_time: 1234, + end_time: 1235 + }, + PartKeyRecord { + part_key: vec![0x42, 0x42], + start_time: 4321, + end_time: 10000 + } + ] + .into_iter() + .collect::>() + ); + } + + #[test] + fn test_part_key_collector_with_limit() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = PartKeyRecordCollector::new(1, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Which doc matches first is non deterministic, just check length + assert_eq!(results.len(), 1); + } + + #[test] + fn test_part_key_record_serialize() { + let record = PartKeyRecord { + part_key: vec![0xAAu8; 2], + start_time: 1, + end_time: 2, + }; + + // 8 bytes, 8 bytes, 4 bytes, 2 bytes + assert_eq!(22, record.serialized_len()); + + let mut vec = vec![]; + record.serialize(&mut vec); + + assert_eq!( + vec![1, 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 170, 170], + vec + ); + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/string_field_collector.rs b/core/src/rust/tantivy_utils/src/collectors/string_field_collector.rs new file mode 100644 index 0000000000..f0ee817a8e --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/string_field_collector.rs @@ -0,0 +1,370 @@ +//! Collector to string values from a document + +use core::str; +use std::collections::hash_map::Entry; + +use hashbrown::HashMap; +use nohash_hasher::IntMap; +use tantivy::{ + collector::{Collector, SegmentCollector}, + columnar::StrColumn, + TantivyError, +}; + +use crate::collectors::column_cache::ColumnCache; + +use super::{ + index_collector::IndexCollector, + limited_collector::{ + LimitCounter, LimitCounterOptionExt, LimitResult, LimitedCollector, LimitedSegmentCollector, + }, +}; + +pub struct StringFieldCollector<'a> { + field: &'a str, + limit: usize, + term_limit: usize, + column_cache: ColumnCache, +} + +impl<'a> StringFieldCollector<'a> { + pub fn new(field: &'a str, limit: usize, term_limit: usize, column_cache: ColumnCache) -> Self { + Self { + field, + limit, + term_limit, + column_cache, + } + } +} + +impl<'a> LimitedCollector for StringFieldCollector<'a> { + fn limit(&self) -> usize { + self.limit + } +} + +impl<'a> Collector for StringFieldCollector<'a> { + type Fruit = Vec<(String, u64)>; + + type Child = StringFieldSegmentCollector; + + fn for_segment( + &self, + _segment_local_id: tantivy::SegmentOrdinal, + segment: &tantivy::SegmentReader, + ) -> tantivy::Result { + let column = self.column_cache.get_str_column(segment, self.field)?; + + Ok(StringFieldSegmentCollector { + column, + docs: IntMap::default(), + term_limit: self.term_limit, + }) + } + + fn requires_scoring(&self) -> bool { + false + } + + fn merge_fruits( + &self, + segment_fruits: Vec>, + ) -> tantivy::Result> { + let mut results = if self.limit < usize::MAX { + HashMap::with_capacity(self.limit) + } else { + HashMap::new() + }; + + for mut map in segment_fruits.into_iter() { + for (value, count) in map.drain() { + *results.entry(value).or_insert(0) += count; + } + } + + let mut results: Vec<_> = results.drain().collect(); + results.sort_by(|(_, count_a), (_, count_b)| count_b.cmp(count_a)); + + let results = results.into_iter().take(self.limit).collect(); + + Ok(results) + } +} + +pub struct StringFieldSegmentCollector { + column: Option, + docs: IntMap, + term_limit: usize, +} + +impl LimitedSegmentCollector for StringFieldSegmentCollector { + fn collect_with_limiter( + &mut self, + doc: tantivy::DocId, + _score: tantivy::Score, + mut limiter: Option<&mut super::limited_collector::LimitCounter>, + ) -> LimitResult { + if let Some(column) = &self.column { + for ord in column.term_ords(doc) { + if self.docs.len() >= self.term_limit { + break; + } + + // We wait to translate to strings later to reduce + // the number of times we have to copy the data out + // to one per ord + let entry = self.docs.entry(ord); + let increment = matches!(entry, Entry::Vacant(_)); + *entry.or_insert(0) += 1; + + if increment { + limiter.increment()?; + } + } + } + + Ok(()) + } +} + +impl SegmentCollector for StringFieldSegmentCollector { + type Fruit = HashMap; + + fn collect(&mut self, doc: tantivy::DocId, score: tantivy::Score) { + let _ = self.collect_with_limiter(doc, score, None); + } + + fn harvest(self) -> Self::Fruit { + self.docs + .into_iter() + .map(|(ord, count)| { + if let Some(column) = &self.column { + let mut value = String::new(); + let _ = column.ord_to_str(ord, &mut value); + + (value, count) + } else { + (String::new(), count) + } + }) + .filter(|(k, _v)| !k.is_empty()) + .collect() + } +} + +impl<'a> IndexCollector for StringFieldCollector<'a> { + fn collect_over_index( + &self, + reader: &tantivy::SegmentReader, + limiter: &mut LimitCounter, + ) -> Result<::Fruit, tantivy::TantivyError> { + let Some((field, prefix)) = reader.schema().find_field(self.field) else { + return Err(TantivyError::FieldNotFound(self.field.to_string())); + }; + + let mut ret = if self.limit < usize::MAX { + HashMap::with_capacity(self.limit) + } else { + HashMap::new() + }; + + if limiter.at_limit() { + return Ok(ret); + } + + let index_reader = reader.inverted_index(field)?; + let mut index_reader = index_reader.terms().range(); + if !prefix.is_empty() { + // Only look at prefix range + index_reader = index_reader.ge(format!("{}\0", prefix)); + index_reader = index_reader.lt(format!("{}\u{001}", prefix)); + } + let mut index_reader = index_reader.into_stream()?; + while !limiter.at_limit() && index_reader.advance() { + let mut key_bytes = index_reader.key(); + if !prefix.is_empty() { + // Skip prefix + key_bytes = &key_bytes[prefix.len() + 2..]; + } + if key_bytes.is_empty() { + continue; + } + + let key = str::from_utf8(key_bytes) + .map_err(|e| TantivyError::InternalError(e.to_string()))?; + + // capture it + ret.insert(key.to_string(), index_reader.value().doc_freq as u64); + + // No need to check error, the check at the top of the while will handle it + let _ = limiter.increment(); + } + + Ok(ret) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use tantivy::query::AllQuery; + + use crate::{ + collectors::index_collector::collect_from_index, + test_utils::{build_test_schema, COL1_NAME, JSON_COL_NAME}, + }; + + use super::*; + + #[test] + fn test_string_field_index_collector() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = StringFieldCollector::new(COL1_NAME, usize::MAX, usize::MAX, column_cache); + + let results = collect_from_index(&index.searcher, collector).expect("Should succeed"); + + // Two docs + assert_eq!( + results.into_iter().collect::>(), + [("ABC".to_string(), 1), ("DEF".to_string(), 1)] + .into_iter() + .collect::>() + ); + } + + #[test] + fn test_string_field_collector() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = StringFieldCollector::new(COL1_NAME, usize::MAX, usize::MAX, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Two docs + assert_eq!( + results.into_iter().collect::>(), + [("ABC".to_string(), 1), ("DEF".to_string(), 1)] + .into_iter() + .collect::>() + ); + } + + #[test] + fn test_string_field_collector_json() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let col_name = format!("{}.{}", JSON_COL_NAME, "f1"); + let collector = StringFieldCollector::new(&col_name, usize::MAX, usize::MAX, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Two docs + assert_eq!( + results.into_iter().collect::>(), + [("value".to_string(), 1), ("othervalue".to_string(), 1)] + .into_iter() + .collect::>() + ); + } + + #[test] + fn test_string_field_index_collector_json() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let col_name = format!("{}.{}", JSON_COL_NAME, "f1"); + let collector = StringFieldCollector::new(&col_name, usize::MAX, usize::MAX, column_cache); + + let results = collect_from_index(&index.searcher, collector).expect("Should succeed"); + + // Two docs + assert_eq!( + results.into_iter().collect::>(), + [("value".to_string(), 1), ("othervalue".to_string(), 1)] + .into_iter() + .collect::>() + ); + } + + #[test] + fn test_string_field_collector_json_invalid_field() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let col_name = format!("{}.{}", JSON_COL_NAME, "invalid"); + let collector = StringFieldCollector::new(&col_name, usize::MAX, usize::MAX, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // No results, no failure + assert_eq!( + results.into_iter().collect::>(), + [].into_iter().collect::>() + ); + } + + #[test] + fn test_string_field_collector_index_json_invalid_field() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let col_name = format!("{}.{}", JSON_COL_NAME, "invalid"); + let collector = StringFieldCollector::new(&col_name, usize::MAX, usize::MAX, column_cache); + + let results = collect_from_index(&index.searcher, collector).expect("Should succeed"); + + // No results, no failure + assert_eq!( + results.into_iter().collect::>(), + [].into_iter().collect::>() + ); + } + + #[test] + fn test_string_field_collector_with_limit() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = StringFieldCollector::new(COL1_NAME, 1, usize::MAX, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Which doc matches first is non deterministic, just check length + assert_eq!(results.len(), 1); + } + + #[test] + fn test_string_field_index_collector_with_limit() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = StringFieldCollector::new(COL1_NAME, 1, usize::MAX, column_cache); + + let results = collect_from_index(&index.searcher, collector).expect("Should succeed"); + + // Which doc matches first is non deterministic, just check length + assert_eq!(results.len(), 1); + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/time_collector.rs b/core/src/rust/tantivy_utils/src/collectors/time_collector.rs new file mode 100644 index 0000000000..624cc4580d --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/time_collector.rs @@ -0,0 +1,172 @@ +//! Collector to pull part IDs + time values from a document + +use std::cmp::min; + +use tantivy::{ + collector::{Collector, SegmentCollector}, + columnar::Column, + TantivyError, +}; + +use crate::collectors::column_cache::ColumnCache; +use crate::field_constants; + +use super::limited_collector::{ + LimitCounterOptionExt, LimitResult, LimitedCollector, LimitedSegmentCollector, +}; + +pub struct TimeCollector<'a> { + time_field: &'a str, + limit: usize, + column_cache: ColumnCache, +} + +impl<'a> TimeCollector<'a> { + pub fn new(time_field: &'a str, limit: usize, column_cache: ColumnCache) -> Self { + Self { + time_field, + limit, + column_cache, + } + } +} + +impl<'a> LimitedCollector for TimeCollector<'a> { + fn limit(&self) -> usize { + self.limit + } +} + +impl<'a> Collector for TimeCollector<'a> { + // Tuple of part_id, time + type Fruit = Vec<(i32, i64)>; + + type Child = TimeSegmentCollector; + + fn for_segment( + &self, + _segment_local_id: tantivy::SegmentOrdinal, + segment: &tantivy::SegmentReader, + ) -> tantivy::Result { + let id_column: Column = self + .column_cache + .get_column(segment, field_constants::PART_ID)? + .ok_or_else(|| TantivyError::FieldNotFound(field_constants::PART_ID.to_string()))?; + + let time_column: Column = self + .column_cache + .get_column(segment, self.time_field)? + .ok_or_else(|| TantivyError::FieldNotFound(self.time_field.to_string()))?; + + Ok(TimeSegmentCollector { + id_column, + time_column, + docs: Vec::new(), + }) + } + + fn requires_scoring(&self) -> bool { + false + } + + fn merge_fruits( + &self, + segment_fruits: Vec>, + ) -> tantivy::Result> { + let len: usize = min(segment_fruits.iter().map(|x| x.len()).sum(), self.limit); + + let mut result = Vec::with_capacity(len); + for part_ids in segment_fruits { + result.extend(part_ids.iter().take(self.limit - result.len())); + } + + Ok(result) + } +} + +pub struct TimeSegmentCollector { + id_column: Column, + time_column: Column, + docs: Vec<(i32, i64)>, +} + +impl LimitedSegmentCollector for TimeSegmentCollector { + fn collect_with_limiter( + &mut self, + doc: tantivy::DocId, + _score: tantivy::Score, + mut limiter: Option<&mut super::limited_collector::LimitCounter>, + ) -> LimitResult { + if let Some(id) = self.id_column.first(doc) { + if let Some(time) = self.time_column.first(doc) { + self.docs.push((id as i32, time)); + limiter.increment()?; + } + } + + Ok(()) + } +} + +impl SegmentCollector for TimeSegmentCollector { + type Fruit = Vec<(i32, i64)>; + + fn collect(&mut self, doc: tantivy::DocId, score: tantivy::Score) { + let _ = self.collect_with_limiter(doc, score, None); + } + + fn harvest(self) -> Self::Fruit { + self.docs + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use field_constants::START_TIME; + use tantivy::query::AllQuery; + + use crate::test_utils::build_test_schema; + + use super::*; + + #[test] + fn test_time_collector() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = TimeCollector::new(START_TIME, usize::MAX, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Two docs, IDs 1 and 10 + assert_eq!( + results.into_iter().collect::>(), + [(1, 1234), (10, 4321)] + .into_iter() + .collect::>() + ); + } + + #[test] + fn test_part_id_collector_with_limit() { + let index = build_test_schema(); + let column_cache = ColumnCache::default(); + + let collector = TimeCollector::new(START_TIME, 1, column_cache); + let query = AllQuery; + + let results = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Which doc matches first is non deterministic, just check length + assert_eq!(results.len(), 1); + } +} diff --git a/core/src/rust/tantivy_utils/src/collectors/time_range_filter.rs b/core/src/rust/tantivy_utils/src/collectors/time_range_filter.rs new file mode 100644 index 0000000000..34bc8828b6 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/collectors/time_range_filter.rs @@ -0,0 +1,262 @@ +//! Filter collector that applies a start and end time range + +use tantivy::{ + collector::{Collector, SegmentCollector}, + columnar::Column, + TantivyError, +}; + +use crate::collectors::column_cache::ColumnCache; +use crate::field_constants; + +use super::limited_collector::{LimitResult, LimitedCollector, LimitedSegmentCollector}; + +/// Filters results on a time range +pub struct TimeRangeFilter<'a, T> +where + T: LimitedCollector, + T::Child: LimitedSegmentCollector, +{ + /// Inner collector + collector: &'a T, + /// Start time + start: i64, + /// End time + end: i64, + /// Column cache + column_cache: ColumnCache, +} + +impl<'a, T> TimeRangeFilter<'a, T> +where + T: LimitedCollector, + T::Child: LimitedSegmentCollector, +{ + pub fn new(collector: &'a T, start: i64, end: i64, column_cache: ColumnCache) -> Self { + Self { + collector, + start, + end, + column_cache, + } + } +} + +impl<'a, T> LimitedCollector for TimeRangeFilter<'a, T> +where + T: LimitedCollector, + T::Child: LimitedSegmentCollector, +{ + fn limit(&self) -> usize { + self.collector.limit() + } +} + +impl<'a, T> Collector for TimeRangeFilter<'a, T> +where + T: LimitedCollector, + T::Child: LimitedSegmentCollector, +{ + type Fruit = T::Fruit; + + type Child = TimeRangeFilterSegmentCollector; + + fn for_segment( + &self, + segment_local_id: tantivy::SegmentOrdinal, + segment: &tantivy::SegmentReader, + ) -> tantivy::Result { + let start_column = if self.end < i64::MAX { + Some( + self.column_cache + .get_column(segment, field_constants::START_TIME)? + .ok_or_else(|| { + TantivyError::FieldNotFound(field_constants::START_TIME.to_string()) + })?, + ) + } else { + None + }; + + let end_column = if self.start > 0 { + Some( + self.column_cache + .get_column(segment, field_constants::END_TIME)? + .ok_or_else(|| { + TantivyError::FieldNotFound(field_constants::END_TIME.to_string()) + })?, + ) + } else { + None + }; + + let collector = self.collector.for_segment(segment_local_id, segment)?; + + Ok(TimeRangeFilterSegmentCollector:: { + start_column, + end_column, + start_time: self.start, + end_time: self.end, + collector, + }) + } + + fn requires_scoring(&self) -> bool { + false + } + + fn merge_fruits( + &self, + segment_fruits: Vec<::Fruit>, + ) -> tantivy::Result { + self.collector.merge_fruits(segment_fruits) + } +} + +pub struct TimeRangeFilterSegmentCollector +where + T: LimitedSegmentCollector, +{ + collector: T, + start_column: Option>, + end_column: Option>, + start_time: i64, + end_time: i64, +} + +impl LimitedSegmentCollector for TimeRangeFilterSegmentCollector +where + T: LimitedSegmentCollector, +{ + fn collect_with_limiter( + &mut self, + doc: tantivy::DocId, + score: tantivy::Score, + limiter: Option<&mut super::limited_collector::LimitCounter>, + ) -> LimitResult { + if let Some(start_column) = &self.start_column { + let doc_start = start_column.first(doc).unwrap_or(0); + if doc_start > self.end_time { + return Ok(()); + } + } + + if let Some(end_column) = &self.end_column { + let doc_end = end_column.first(doc).unwrap_or(i64::MAX); + if doc_end < self.start_time { + return Ok(()); + } + } + + self.collector.collect_with_limiter(doc, score, limiter) + } +} + +impl SegmentCollector for TimeRangeFilterSegmentCollector +where + T: LimitedSegmentCollector, +{ + type Fruit = T::Fruit; + + fn collect(&mut self, doc: tantivy::DocId, score: tantivy::Score) { + let _ = self.collect_with_limiter(doc, score, None); + } + + fn harvest(self) -> Self::Fruit { + self.collector.harvest() + } +} + +#[cfg(test)] +mod tests { + use tantivy::{collector::Count, query::AllQuery}; + + use crate::{collectors::limited_collector::UnlimitedCollector, test_utils::build_test_schema}; + + use super::*; + + #[test] + fn test_time_filter_match_all() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + + let query = AllQuery; + let collector = UnlimitedCollector::new(Count); + let collector = TimeRangeFilter::new(&collector, 0, i64::MAX, cache); + let result = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Should match both docs since there's no effective time filter + assert_eq!(result, 2); + } + + #[test] + fn test_time_filter_match_end_filter() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + + let query = AllQuery; + let collector = UnlimitedCollector::new(Count); + let collector = TimeRangeFilter::new(&collector, 0, 2000, cache); + let result = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Should match one doc since the other starts after end + assert_eq!(result, 1); + } + + #[test] + fn test_time_filter_match_start_filter() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + + let query = AllQuery; + let collector = UnlimitedCollector::new(Count); + let collector = TimeRangeFilter::new(&collector, 2000, i64::MAX, cache); + let result = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Should match one doc since the other ends after start + assert_eq!(result, 1); + } + + #[test] + fn test_time_filter_match_overlap() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + + let query = AllQuery; + let collector = UnlimitedCollector::new(Count); + let collector = TimeRangeFilter::new(&collector, 1000, 2000, cache); + let result = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Should match one doc since the other ends after start + assert_eq!(result, 1); + } + + #[test] + fn test_time_filter_match_outside_range() { + let index = build_test_schema(); + let cache = ColumnCache::default(); + + let query = AllQuery; + let collector = UnlimitedCollector::new(Count); + let collector = TimeRangeFilter::new(&collector, 20_000, 20_000, cache); + let result = index + .searcher + .search(&query, &collector) + .expect("Should succeed"); + + // Should match no docs - out of range + assert_eq!(result, 0); + } +} diff --git a/core/src/rust/tantivy_utils/src/field_constants.rs b/core/src/rust/tantivy_utils/src/field_constants.rs new file mode 100644 index 0000000000..33c7fd6d66 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/field_constants.rs @@ -0,0 +1,17 @@ +//! Field names + +pub fn facet_field_name(name: &str) -> String { + format!("{}{}", FACET_FIELD_PREFIX, name) +} + +// These should be kept in sync with the constants in PartKeyIndex.scala +// as they're fields that can be directly queried via incoming filters +// or fields that are filtered out of label lists +pub const DOCUMENT_ID: &str = "__partIdField__"; +pub const PART_ID: &str = "__partIdDv__"; +pub const PART_KEY: &str = "__partKey__"; +pub const LABEL_LIST: &str = "__labelList__"; +pub const FACET_FIELD_PREFIX: &str = "$facet_"; +pub const START_TIME: &str = "__startTime__"; +pub const END_TIME: &str = "__endTime__"; +pub const TYPE: &str = "_type_"; diff --git a/core/src/rust/tantivy_utils/src/lib.rs b/core/src/rust/tantivy_utils/src/lib.rs new file mode 100644 index 0000000000..6251a7925b --- /dev/null +++ b/core/src/rust/tantivy_utils/src/lib.rs @@ -0,0 +1,8 @@ +//! Common utilities for tantivy operations +#![deny(clippy::expect_used, clippy::unwrap_used, clippy::panic)] + +pub mod collectors; +pub mod field_constants; +pub mod query; + +pub mod test_utils; diff --git a/core/src/rust/tantivy_utils/src/query.rs b/core/src/rust/tantivy_utils/src/query.rs new file mode 100644 index 0000000000..78242b259b --- /dev/null +++ b/core/src/rust/tantivy_utils/src/query.rs @@ -0,0 +1,9 @@ +//! Helpers for queries + +pub mod bitset_weight; +pub mod cache; +pub mod prefix_query; +pub mod range_aware_regex; +pub mod shared_doc_set; + +pub const JSON_PREFIX_SEPARATOR: &str = "\0s"; diff --git a/core/src/rust/tantivy_utils/src/query/bitset_weight.rs b/core/src/rust/tantivy_utils/src/query/bitset_weight.rs new file mode 100644 index 0000000000..9325f444cc --- /dev/null +++ b/core/src/rust/tantivy_utils/src/query/bitset_weight.rs @@ -0,0 +1,90 @@ +//! Weight adapter for a cached bitset + +use std::sync::Arc; + +use tantivy::{ + query::{ConstScorer, Explanation, Scorer, Weight}, + DocId, Score, SegmentReader, TantivyError, +}; +use tantivy_common::BitSet; + +use super::shared_doc_set::SharedDocSet; + +/// Weight that can play back a cached doc set +pub struct BitSetWeight { + bitset: Arc, +} + +impl BitSetWeight { + pub fn new(bitset: Arc) -> Self { + BitSetWeight { bitset } + } +} + +impl Weight for BitSetWeight { + fn scorer(&self, _reader: &SegmentReader, _boost: Score) -> tantivy::Result> { + let docs = SharedDocSet::new(self.bitset.clone()); + Ok(Box::new(ConstScorer::new(docs, 1.0))) + } + + fn explain(&self, reader: &SegmentReader, doc: DocId) -> tantivy::Result { + let mut scorer = self.scorer(reader, 1.0)?; + if scorer.seek(doc) == doc { + Ok(Explanation::new("BitSetWeight", 1.0)) + } else { + Err(TantivyError::InvalidArgument( + "Document does not exist".to_string(), + )) + } + } +} + +#[cfg(test)] +mod tests { + use tantivy::TERMINATED; + + use crate::test_utils::build_test_schema; + + use super::*; + + #[test] + fn test_bitset_weight() { + let index = build_test_schema(); + + let mut bitset = BitSet::with_max_value(100); + bitset.insert(1); + bitset.insert(10); + bitset.insert(100); + + let weight = BitSetWeight::new(bitset.into()); + let reader = index.searcher.segment_readers().first().unwrap(); + + let mut scorer = weight.scorer(reader, 1.0).expect("Should succeed"); + + assert_eq!(scorer.doc(), 1); + scorer.advance(); + assert_eq!(scorer.doc(), 10); + scorer.advance(); + assert_eq!(scorer.doc(), 100); + scorer.advance(); + assert_eq!(scorer.doc(), TERMINATED); + } + + #[test] + fn test_bitset_explain() { + let index = build_test_schema(); + + let mut bitset = BitSet::with_max_value(100); + bitset.insert(1); + + let weight = BitSetWeight::new(bitset.into()); + let reader = index.searcher.segment_readers().first().unwrap(); + + let explanation = weight.explain(reader, 1).expect("Should succeed"); + + assert_eq!( + format!("{:?}", explanation), + "Explanation({\n \"value\": 1.0,\n \"description\": \"BitSetWeight\"\n})" + ); + } +} diff --git a/core/src/rust/tantivy_utils/src/query/cache.rs b/core/src/rust/tantivy_utils/src/query/cache.rs new file mode 100644 index 0000000000..3cba309161 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/query/cache.rs @@ -0,0 +1,257 @@ +//! Cached query support + +use std::{hash::Hash, sync::Arc}; + +use quick_cache::{sync::Cache, Equivalent, Weighter}; +use tantivy::{ + collector::SegmentCollector, + query::{EnableScoring, Query, Weight}, + schema::{Field, Schema}, + Searcher, SegmentId, TantivyError, +}; +use tantivy_common::BitSet; + +use crate::collectors::limited_collector::{ + LimitCounter, LimitedCollector, LimitedSegmentCollector, +}; + +use super::bitset_weight::BitSetWeight; + +/// Cache for query results +/// +/// The cache key is a tuple of segment ID and query. The specific format +/// of the query part is left to the caller as it may be a serialized format. +/// +/// The key is a bitfield of documents that match the query for a given segment. +/// The bitfield size in bits will be equal to the number of documents in the +/// segment. We keep the BitSet in an Arc to reduce data copies as once created +/// the field is immutable. +pub struct QueryCache +where + QueryType: CachableQuery, + WeighterType: Weighter<(SegmentId, QueryType), Arc> + Default + Clone, +{ + // Cache of query -> docs + cache: Cache<(SegmentId, QueryType), Arc, WeighterType>, +} + +/// Trait for cachable query keys +pub trait CachableQuery: Eq + PartialEq + Hash + Clone { + /// Should this query be cached? + fn should_cache(&self) -> bool; + + fn to_query( + &self, + schema: &Schema, + default_field: Option, + ) -> Result, TantivyError>; +} + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct CachableQueryKey<'a, QueryType>(pub SegmentId, pub &'a QueryType) +where + QueryType: Clone + PartialEq + Eq; + +impl<'a, QueryType> From> for (SegmentId, QueryType) +where + QueryType: Clone + PartialEq + Eq, +{ + fn from(value: CachableQueryKey<'a, QueryType>) -> Self { + (value.0, value.1.clone()) + } +} + +impl<'a, QueryType> Equivalent<(SegmentId, QueryType)> for CachableQueryKey<'a, QueryType> +where + QueryType: Clone + PartialEq + Eq, +{ + fn equivalent(&self, key: &(SegmentId, QueryType)) -> bool { + self.0 == key.0 && *self.1 == key.1 + } +} + +// Tuning parameters for query cache +const DEFAULT_QUERY_CACHE_MAX_SIZE_BYTES: u64 = 50_000_000; +// Rough estimate of bitset size - 250k docs +const DEFAULT_QUERY_CACHE_AVG_ITEM_SIZE: u64 = 31250; + +impl QueryCache +where + QueryType: CachableQuery, + WeighterType: Weighter<(SegmentId, QueryType), Arc> + Default + Clone, +{ + pub fn new(estimated_items_count: u64, weight_capacity: u64) -> Self { + Self { + cache: Cache::with_weighter( + estimated_items_count as usize, + weight_capacity, + WeighterType::default(), + ), + } + } + + pub fn query_cache_stats(&self) -> (u64, u64) { + (self.cache.hits(), self.cache.misses()) + } + + /// Gets the current cache size, in bytes + pub fn size(&self) -> u64 { + self.cache.weight() + } + + /// Execute a cachable query + pub fn search( + &self, + searcher: &Searcher, + schema: &Schema, + default_field: Option, + cachable_query: QueryType, + collector: C, + ) -> Result + where + C: LimitedCollector, + C::Child: LimitedSegmentCollector, + { + let scoring = EnableScoring::disabled_from_searcher(searcher); + + let mut query_weight: Option> = None; + + let segment_readers = searcher.segment_readers(); + let mut fruits: Vec<::Fruit> = + Vec::with_capacity(segment_readers.len()); + + let mut limiter = LimitCounter::new(collector.limit()); + + // Note - the query optimizations here only work for the single threaded querying. That matches + // the pattern FiloDB uses because it will dispatch multiple queries at a time on different threads, + // so this results in net improvement anyway. If we need to change to the multithreaded executor + // in the future then the lazy query evaluation code will need some work + for (segment_ord, segment_reader) in segment_readers.iter().enumerate() { + // Is it cached + let cache_key = CachableQueryKey(segment_reader.segment_id(), &cachable_query); + + let docs = if let Some(docs) = self.cache.get(&cache_key) { + // Cache hit + docs + } else { + // Build query if needed. We do this lazily as it may be expensive to parse a regex, for example. + // This can give a 2-4x speedup in some cases. + let weight = if let Some(weight) = &query_weight { + weight + } else { + let query = cachable_query.to_query(schema, default_field)?; + let weight = query.weight(scoring)?; + + query_weight = Some(weight); + + // Unwrap is safe here because we just set the value + #[allow(clippy::unwrap_used)] + query_weight.as_ref().unwrap() + }; + + // Load bit set + let mut bitset = BitSet::with_max_value(segment_reader.max_doc()); + + weight.for_each_no_score(segment_reader, &mut |docs| { + for doc in docs.iter().cloned() { + bitset.insert(doc); + } + })?; + + let bitset = Arc::new(bitset); + + if cachable_query.should_cache() { + self.cache.insert(cache_key.into(), bitset.clone()); + } + + bitset + }; + + let weight = BitSetWeight::new(docs); + let results = collector.collect_segment_with_limiter( + &weight, + segment_ord as u32, + segment_reader, + &mut limiter, + )?; + + fruits.push(results); + + if limiter.at_limit() { + break; + } + } + + collector.merge_fruits(fruits) + } +} + +impl Default for QueryCache +where + QueryType: CachableQuery, + WeighterType: Weighter<(SegmentId, QueryType), Arc> + Default + Clone, +{ + fn default() -> Self { + const QUERY_CACHE_ESTIMATED_ITEM_COUNT: u64 = + DEFAULT_QUERY_CACHE_MAX_SIZE_BYTES / DEFAULT_QUERY_CACHE_AVG_ITEM_SIZE; + + Self::new( + QUERY_CACHE_ESTIMATED_ITEM_COUNT, + DEFAULT_QUERY_CACHE_MAX_SIZE_BYTES, + ) + } +} + +#[cfg(test)] +mod tests { + use std::hash::{DefaultHasher, Hasher}; + + use tantivy::query::AllQuery; + + use crate::test_utils::build_test_schema; + + use super::*; + + #[derive(Clone, Eq, PartialEq, Hash, Debug)] + pub enum TestQuery { + Test(u32), + } + + impl CachableQuery for TestQuery { + fn should_cache(&self) -> bool { + true + } + + fn to_query( + &self, + _schema: &Schema, + _default_field: Option, + ) -> Result, TantivyError> { + Ok(Box::new(AllQuery)) + } + } + + #[test] + fn test_cache_key_equivilance() { + let index = build_test_schema(); + let reader = index.searcher.segment_readers().first().unwrap(); + + let query = TestQuery::Test(1234); + + let key = CachableQueryKey(reader.segment_id(), &query); + let owned_key: (SegmentId, TestQuery) = key.clone().into(); + + assert_eq!(key.0, owned_key.0); + assert_eq!(*key.1, owned_key.1); + + let mut hasher = DefaultHasher::new(); + key.hash(&mut hasher); + let key_hash = hasher.finish(); + + let mut hasher = DefaultHasher::new(); + owned_key.hash(&mut hasher); + let owned_key_hash = hasher.finish(); + + assert_eq!(key_hash, owned_key_hash); + } +} diff --git a/core/src/rust/tantivy_utils/src/query/prefix_query.rs b/core/src/rust/tantivy_utils/src/query/prefix_query.rs new file mode 100644 index 0000000000..145e3c6aa9 --- /dev/null +++ b/core/src/rust/tantivy_utils/src/query/prefix_query.rs @@ -0,0 +1,96 @@ +//! Query that does a prefix match + +use std::sync::Arc; + +use tantivy::{ + query::{AutomatonWeight, Query}, + schema::Field, +}; +use tantivy_fst::Automaton; + +use super::{range_aware_regex::SkipAutomaton, JSON_PREFIX_SEPARATOR}; + +#[derive(Debug, Clone)] +pub struct PrefixQuery { + automaton: Arc>, + field: Field, + json_path: String, +} + +impl PrefixQuery { + pub fn new(prefix: &str, json_path: &str, field: Field) -> Self { + let automaton = PrefixAutomaton { + prefix: prefix.as_bytes().into(), + }; + let automaton = SkipAutomaton::new( + automaton, + if json_path.is_empty() { + 0 + } else { + json_path.len() + JSON_PREFIX_SEPARATOR.len() + }, + ); + + Self { + automaton: Arc::new(automaton), + field, + json_path: json_path.into(), + } + } +} + +impl Query for PrefixQuery { + fn weight( + &self, + _enable_scoring: tantivy::query::EnableScoring<'_>, + ) -> tantivy::Result> { + let automaton = self.automaton.clone(); + let weight: AutomatonWeight> = if self.json_path.is_empty() { + AutomatonWeight::new(self.field, automaton) + } else { + AutomatonWeight::new_for_json_path(self.field, automaton, self.json_path.as_bytes()) + }; + + Ok(Box::new(weight)) + } +} + +#[derive(Debug)] +pub struct PrefixAutomaton { + prefix: Box<[u8]>, +} + +impl Automaton for PrefixAutomaton { + // The state here is simple - it's the byte offset we're currently checking + // A value of prefix.len() means we've checked everything and we match + // A value of MAX means we had a mismatch and will never match + type State = usize; + + fn start(&self) -> Self::State { + 0 + } + + fn is_match(&self, state: &Self::State) -> bool { + *state == self.prefix.len() + } + + fn accept(&self, state: &Self::State, byte: u8) -> Self::State { + if *state < self.prefix.len() { + if byte == self.prefix[*state] { + *state + 1 + } else { + usize::MAX + } + } else { + *state + } + } + + fn can_match(&self, state: &Self::State) -> bool { + *state != usize::MAX + } + + fn will_always_match(&self, state: &Self::State) -> bool { + *state == self.prefix.len() + } +} diff --git a/core/src/rust/tantivy_utils/src/query/range_aware_regex.rs b/core/src/rust/tantivy_utils/src/query/range_aware_regex.rs new file mode 100644 index 0000000000..3574dc011d --- /dev/null +++ b/core/src/rust/tantivy_utils/src/query/range_aware_regex.rs @@ -0,0 +1,343 @@ +//! Range aware Regex query + +use std::sync::Arc; + +use tantivy::{ + query::{AutomatonWeight, EnableScoring, Query, Weight}, + schema::Field, + TantivyError, +}; +use tantivy_fst::{Automaton, Regex}; + +use super::JSON_PREFIX_SEPARATOR; + +// Tantivy's in box RegexQuery looks at all possible dictionary values for matches +// For JSON fields this means looking at a lot of values for other fields that can never match +// This class is range aware limiting the number of considered terms + +#[derive(Debug, Clone)] +pub struct RangeAwareRegexQuery { + regex: Arc>, + prefix: String, + field: Field, +} + +impl RangeAwareRegexQuery { + /// Creates a new RegexQuery from a given pattern + pub fn from_pattern( + regex_pattern: &str, + prefix: &str, + field: Field, + ) -> Result { + let regex = create_regex(regex_pattern)?; + + let regex = SkipAutomaton::new( + regex, + if prefix.is_empty() { + 0 + } else { + prefix.len() + JSON_PREFIX_SEPARATOR.len() + }, + ); + + Ok(RangeAwareRegexQuery { + regex: regex.into(), + prefix: if prefix.is_empty() { + String::new() + } else { + format!("{}\0s", prefix) + }, + field, + }) + } + + fn specialized_weight(&self) -> AutomatonWeight> { + if self.prefix.is_empty() { + AutomatonWeight::new(self.field, self.regex.clone()) + } else { + AutomatonWeight::new_for_json_path( + self.field, + self.regex.clone(), + self.prefix.as_bytes(), + ) + } + } +} + +impl Query for RangeAwareRegexQuery { + fn weight(&self, _enabled_scoring: EnableScoring<'_>) -> Result, TantivyError> { + Ok(Box::new(self.specialized_weight())) + } +} + +fn create_regex(pattern: &str) -> Result { + Regex::new(pattern) + .map_err(|err| TantivyError::InvalidArgument(format!("RanageAwareRegexQuery: {err}"))) +} + +#[derive(Debug)] +pub struct SkipAutomaton { + inner: A, + skip_size: usize, +} + +impl SkipAutomaton { + pub fn new(inner: A, skip_size: usize) -> Self { + Self { inner, skip_size } + } +} + +#[derive(Clone)] +pub struct SkipAutomatonState { + count: usize, + inner: A, +} + +impl Automaton for SkipAutomaton +where + A: Automaton, + A::State: Clone, +{ + type State = SkipAutomatonState; + + fn start(&self) -> Self::State { + Self::State { + count: 0, + inner: self.inner.start(), + } + } + + fn is_match(&self, state: &Self::State) -> bool { + if state.count < self.skip_size { + false + } else { + self.inner.is_match(&state.inner) + } + } + + fn accept(&self, state: &Self::State, byte: u8) -> Self::State { + let mut state = state.clone(); + + if state.count < self.skip_size { + state.count += 1 + } else { + state.inner = self.inner.accept(&state.inner, byte); + }; + + state + } + + fn can_match(&self, state: &Self::State) -> bool { + if state.count < self.skip_size { + true + } else { + self.inner.can_match(&state.inner) + } + } + + fn will_always_match(&self, state: &Self::State) -> bool { + if state.count < self.skip_size { + false + } else { + self.inner.will_always_match(&state.inner) + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + // For back compat reasons we must ensure the regex language used covers all non-optional items here: + // https://lucene.apache.org/core/9_0_0/core/org/apache/lucene/util/automaton/RegExp.html + // + // These tests validate this + + fn regex_matches(pattern: &str, input: &str) -> bool { + let regex = create_regex(pattern).expect("Regex should compile"); + + let mut state = regex.start(); + + for b in input.as_bytes() { + if regex.will_always_match(&state) { + return true; + } + + if !regex.can_match(&state) { + return false; + } + + state = regex.accept(&state, *b); + } + + regex.is_match(&state) + } + + #[test] + fn test_regex_empty() { + assert!(regex_matches("", "")) + } + + #[test] + fn test_regex_literal() { + assert!(regex_matches("abcd", "abcd")) + } + + #[test] + fn test_regex_incomplete() { + assert!(!regex_matches("abcd", "ab")) + } + + #[test] + fn test_regex_longer_string() { + assert!(!regex_matches("ab", "abcd")) + } + + #[test] + fn test_regex_substring() { + assert!(!regex_matches("bc", "abcd")) + } + + #[test] + fn test_regex_union() { + assert!(regex_matches("a|b", "a")); + assert!(regex_matches("a|b", "b")); + assert!(!regex_matches("a|b", "c")); + } + + #[test] + fn test_regex_question_mark() { + assert!(regex_matches("a?", "a")); + assert!(regex_matches("a?", "")); + assert!(!regex_matches("a?", "b")); + assert!(!regex_matches("a?", "aa")); + } + + #[test] + fn test_regex_asterisk() { + assert!(regex_matches("a*", "a")); + assert!(regex_matches("a*", "")); + assert!(!regex_matches("a*", "b")); + assert!(regex_matches("a*", "aa")); + } + + #[test] + fn test_regex_plus() { + assert!(regex_matches("a+", "a")); + assert!(!regex_matches("a+", "")); + assert!(!regex_matches("a+", "b")); + assert!(regex_matches("a+", "aa")); + } + + #[test] + fn test_regex_n() { + assert!(regex_matches("a{1}", "a")); + assert!(!regex_matches("a{1}", "")); + assert!(!regex_matches("a{1}", "b")); + assert!(!regex_matches("a{1}", "aa")); + } + + #[test] + fn test_regex_n_or_more() { + assert!(regex_matches("a{1,}", "a")); + assert!(!regex_matches("a{1,}", "")); + assert!(!regex_matches("a{1,}", "b")); + assert!(regex_matches("a{1,}", "aa")); + } + + #[test] + fn test_regex_n_m() { + assert!(regex_matches("a{1,2}", "a")); + assert!(!regex_matches("a{1,2}", "")); + assert!(!regex_matches("a{1,2}", "b")); + assert!(regex_matches("a{1,2}", "aa")); + assert!(!regex_matches("a{1,2}", "aaa")); + } + + #[test] + fn test_regex_char_class() { + assert!(regex_matches("[ab]", "a")); + assert!(regex_matches("[ab]", "b")); + assert!(!regex_matches("[ab]", "c")); + assert!(!regex_matches("[ab]", "aa")); + } + + #[test] + fn test_regex_not_char_class() { + assert!(!regex_matches("[^ab]", "a")); + assert!(!regex_matches("[^ab]", "b")); + assert!(regex_matches("[^ab]", "c")); + assert!(!regex_matches("[^ab]", "aa")); + } + + #[test] + fn test_regex_char_class_range() { + assert!(regex_matches("[a-z]", "a")); + assert!(regex_matches("[a-z]", "b")); + assert!(!regex_matches("[a-z]", "0")); + assert!(!regex_matches("[a-z]", "aa")); + } + + #[test] + fn test_regex_dot() { + assert!(regex_matches(".", "a")); + assert!(regex_matches(".", "b")); + assert!(!regex_matches(".", "aa")); + } + + #[test] + fn test_regex_group() { + assert!(regex_matches("(a)", "a")); + assert!(!regex_matches("(a)", "b")); + assert!(!regex_matches("(a)", "aa")); + } + + #[test] + fn test_regex_digit() { + assert!(regex_matches(r"\d", "0")); + assert!(!regex_matches(r"\d", "b")); + assert!(!regex_matches(r"\d", "01")); + } + + #[test] + fn test_regex_not_digit() { + assert!(regex_matches(r"\D", "b")); + assert!(!regex_matches(r"\D", "0")); + assert!(!regex_matches(r"\D", "ab")); + } + + #[test] + fn test_regex_whitespace() { + assert!(regex_matches(r"\s", " ")); + assert!(!regex_matches(r"\s", "b")); + assert!(!regex_matches(r"\s", " ")); + } + + #[test] + fn test_regex_not_whitespace() { + assert!(regex_matches(r"\S", "a")); + assert!(!regex_matches(r"\S", " ")); + assert!(!regex_matches(r"\S", "aa")); + } + + #[test] + fn test_regex_word() { + assert!(regex_matches(r"\w", "a")); + assert!(!regex_matches(r"\w", "-")); + assert!(!regex_matches(r"\w", "aa")); + } + + #[test] + fn test_regex_not_word() { + assert!(regex_matches(r"\W", "-")); + assert!(!regex_matches(r"\W", "a")); + assert!(!regex_matches(r"\W", "--")); + } + + #[test] + fn test_regex_escape() { + assert!(regex_matches(r"\\", r"\")); + assert!(!regex_matches(r"\\", "-")); + assert!(!regex_matches(r"\\", r"\\")); + } +} diff --git a/core/src/rust/tantivy_utils/src/query/shared_doc_set.rs b/core/src/rust/tantivy_utils/src/query/shared_doc_set.rs new file mode 100644 index 0000000000..7fd7e8ba4c --- /dev/null +++ b/core/src/rust/tantivy_utils/src/query/shared_doc_set.rs @@ -0,0 +1,176 @@ +//! Low memcpy sharable docset + +use std::sync::Arc; + +use tantivy::{DocId, DocSet, TERMINATED}; +use tantivy_common::{BitSet, TinySet}; + +/// Allows for efficient copying of docsets from an immutable bitset +/// This is doing the same job as BitSetDocSet, but without the memcpy +/// each time we want to create a new instance +pub struct SharedDocSet { + bits: Arc, + current_word_num: u32, + current_word: TinySet, + current_doc: DocId, +} + +impl SharedDocSet { + pub fn new(bits: Arc) -> Self { + let current_word = if bits.max_value() == 0 { + TinySet::empty() + } else { + bits.tinyset(0) + }; + + let mut ret = Self { + bits, + current_word_num: 0, + current_word, + current_doc: 0, + }; + + ret.advance(); + ret + } + + #[inline] + fn word_count(&self) -> u32 { + (self.bits.max_value() + 63) / 64 + } +} + +impl DocSet for SharedDocSet { + #[inline] + fn advance(&mut self) -> DocId { + // Case 1 - bits still in the current word + if let Some(bit) = self.current_word.pop_lowest() { + self.current_doc = (self.current_word_num * 64) + bit; + + // Case 2 - no more words + } else if (self.current_word_num + 1) >= self.word_count() { + self.current_doc = TERMINATED; + + // Case 3 - advance to next word + } else if let Some(word_num) = self.bits.first_non_empty_bucket(self.current_word_num + 1) { + self.current_word_num = word_num; + self.current_word = self.bits.tinyset(word_num); + + // This is safe because first_non_empty bucket ensured it is non-empty + #[allow(clippy::unwrap_used)] + let bit = self.current_word.pop_lowest().unwrap(); + self.current_doc = (self.current_word_num * 64) + bit; + + // Case 4 - end of set + } else { + self.current_doc = TERMINATED; + } + + self.current_doc + } + + fn doc(&self) -> DocId { + self.current_doc + } + + fn seek(&mut self, target: DocId) -> DocId { + if target >= self.bits.max_value() { + self.current_doc = TERMINATED + } + + let target_word = target / 64; + if target_word > self.current_word_num { + self.current_word_num = target_word; + self.current_word = self.bits.tinyset(self.current_word_num); + + self.current_word = self + .current_word + .intersect(TinySet::range_greater_or_equal(target_word)); + self.advance(); + } else { + while self.current_doc < target { + self.advance(); + } + } + + self.current_doc + } + + fn size_hint(&self) -> u32 { + self.bits.len() as u32 + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_empty_docset() { + let bits = BitSet::with_max_value(0); + let mut docset = SharedDocSet::new(bits.into()); + + assert_eq!(docset.size_hint(), 0); + assert_eq!(docset.doc(), TERMINATED); + assert_eq!(docset.advance(), TERMINATED); + assert_eq!(docset.seek(0), TERMINATED); + } + + #[test] + fn test_full_docset() { + let bits = BitSet::with_max_value_and_full(1000); + let mut docset = SharedDocSet::new(bits.into()); + + assert_eq!(docset.size_hint(), 1000); + for i in 0..1000 { + assert_eq!(i as DocId, docset.doc()); + docset.advance(); + } + + assert_eq!(docset.doc(), TERMINATED); + } + + #[test] + fn test_full_docset_seek() { + let bits = BitSet::with_max_value_and_full(1000); + let mut docset = SharedDocSet::new(bits.into()); + + assert_eq!(docset.size_hint(), 1000); + docset.seek(50); + for i in 50..1000 { + assert_eq!(i as DocId, docset.doc()); + docset.advance(); + } + + assert_eq!(docset.doc(), TERMINATED); + } + + #[test] + fn test_sparse_docset() { + let mut bits = BitSet::with_max_value(1000); + bits.insert(100); + bits.insert(235); + let mut docset = SharedDocSet::new(bits.into()); + + assert_eq!(docset.size_hint(), 2); + assert_eq!(docset.doc(), 100); + docset.advance(); + assert_eq!(docset.doc(), 235); + docset.advance(); + assert_eq!(docset.doc(), TERMINATED); + } + + #[test] + fn test_sparse_docset_seek() { + let mut bits = BitSet::with_max_value(1000); + bits.insert(100); + bits.insert(235); + let mut docset = SharedDocSet::new(bits.into()); + + assert_eq!(docset.size_hint(), 2); + docset.seek(101); + assert_eq!(docset.doc(), 235); + docset.advance(); + assert_eq!(docset.doc(), TERMINATED); + } +} diff --git a/core/src/rust/tantivy_utils/src/test_utils.rs b/core/src/rust/tantivy_utils/src/test_utils.rs new file mode 100644 index 0000000000..ee85064efe --- /dev/null +++ b/core/src/rust/tantivy_utils/src/test_utils.rs @@ -0,0 +1,100 @@ +//! Utilites for testing + +use crate::field_constants; +use tantivy::{ + schema::{ + Field, JsonObjectOptions, Schema, SchemaBuilder, TextFieldIndexing, FAST, INDEXED, STORED, + STRING, + }, + Index, Searcher, TantivyDocument, +}; + +pub const COL1_NAME: &str = "col1"; +pub const COL2_NAME: &str = "col2"; +pub const JSON_COL_NAME: &str = "json_col"; +pub const JSON_ATTRIBUTE1_NAME: &str = "f1"; +pub const JSON_ATTRIBUTE2_NAME: &str = "f2"; + +pub struct TestIndex { + pub schema: Schema, + pub searcher: Searcher, + pub json_field: Field, +} + +// Allow unwraps since this is test code +#[allow(clippy::unwrap_used)] +pub fn build_test_schema() -> TestIndex { + let mut builder = SchemaBuilder::new(); + + builder.add_text_field(COL1_NAME, STRING | FAST); + builder.add_text_field(COL2_NAME, STRING | FAST); + builder.add_i64_field(field_constants::PART_ID, INDEXED | FAST); + builder.add_i64_field(field_constants::START_TIME, INDEXED | FAST); + builder.add_i64_field(field_constants::END_TIME, INDEXED | FAST); + builder.add_bytes_field(field_constants::PART_KEY, INDEXED | FAST | STORED); + builder.add_json_field( + JSON_COL_NAME, + JsonObjectOptions::default() + .set_indexing_options(TextFieldIndexing::default().set_tokenizer("raw")) + .set_fast(Some("raw")), + ); + + let schema = builder.build(); + + let index = Index::create_in_ram(schema.clone()); + + { + let mut writer = index.writer::(50_000_000).unwrap(); + + let doc = TantivyDocument::parse_json( + &schema, + r#"{ + "col1": "ABC", + "col2": "def", + "__partIdDv__": 1, + "__startTime__": 1234, + "__endTime__": 1235, + "__partKey__": "QUE=", + "json_col": { + "f1": "value", + "f2": "value2" + } + }"#, + ) + .unwrap(); + + writer.add_document(doc).unwrap(); + + let doc = TantivyDocument::parse_json( + &schema, + r#"{ + "col1": "DEF", + "col2": "abc", + "__partIdDv__": 10, + "__startTime__": 4321, + "__endTime__": 10000, + "__partKey__": "QkI=", + "json_col": { + "f1": "othervalue", + "f2": "othervalue2" + } + }"#, + ) + .unwrap(); + + writer.add_document(doc).unwrap(); + + writer.commit().unwrap(); + } + + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); + + let json_field = schema.get_field(JSON_COL_NAME).unwrap(); + + TestIndex { + schema, + searcher, + json_field, + } +} diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index a482034e68..ddccfe9aa0 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -99,6 +99,13 @@ filodb { block-memory-manager-percent = 60 } + tantivy { + column-cache-count = 1000 + query-cache-max-bytes = 50MB + query-cache-estimated-item-size = 31250 + deleted-doc-merge-threshold = 0.1 + } + flush-task-parallelism = 1 ensure-block-memory-headroom-percent = 5 ensure-tsp-count-headroom-percent = 5 @@ -110,6 +117,7 @@ filodb { index-faceting-enabled-for-all-labels = true disable-index-caching = false type-field-indexing-enabled = true + part-key-index-type = lucene } tasks { diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyIndexRawSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyIndexRawSpec.scala new file mode 100644 index 0000000000..6459f09194 --- /dev/null +++ b/core/src/test/scala/filodb.core/memstore/PartKeyIndexRawSpec.scala @@ -0,0 +1,926 @@ +package filodb.core.memstore + +import filodb.core.query.ColumnFilter +import filodb.core.query.Filter.{Equals, EqualsRegex, NotEquals, NotEqualsRegex} +import filodb.core.GdeltTestData.{dataset1, dataset6, dataset7, partKeyFromRecords, readers, records, uniqueReader} +import filodb.core.binaryrecord2.{RecordBuilder, RecordSchema} +import filodb.core.DatasetRef +import filodb.core.metadata.{PartitionSchema, Schemas} +import filodb.memory.format.UnsafeUtils.ZeroPointer +import filodb.memory.format.{UnsafeUtils, UTF8Wrapper} +import filodb.memory.format.ZeroCopyUTF8String.StringToUTF8 +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.must.Matchers.{contain, not} +import org.scalatest.matchers.should.Matchers.{convertToAnyShouldWrapper, equal} + +import java.io.{File, FileFilter} +import java.nio.file.{Files, StandardOpenOption} +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration.DurationInt +import scala.util.Random + +trait PartKeyIndexRawSpec { + this: AnyFunSpec => + + def partKeyOnHeap(partKeySchema: RecordSchema, + base: Any, + offset: Long): Array[Byte] = partKeySchema.asByteArray(base, offset) + + protected def createNewIndex(ref: DatasetRef, + schema: PartitionSchema, + facetEnabledAllLabels: Boolean, + facetEnabledShardKeyLabels: Boolean, + shardNum: Int, + retentionMillis: Long, + diskLocation: Option[File] = None, + lifecycleManager: Option[IndexMetadataStore] = None): PartKeyIndexRaw + + // scalastyle:off method.length + // scalastyle:off cyclomatic.complexity + def commonPartKeyTests(keyIndex: PartKeyIndexRaw, partBuilder: RecordBuilder): Unit = { + it("should add part keys and parse filters correctly") { + val start = System.currentTimeMillis() + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + val end = System.currentTimeMillis() + keyIndex.refreshReadersBlocking() + + // Should get empty iterator when passing no filters + val partNums1 = keyIndex.partIdsFromFilters(Nil, start, end) + partNums1.toIterable() should contain theSameElementsAs List(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) + + // return only 4 partIds - empty filter + val partNumsLimit = keyIndex.partIdsFromFilters(Nil, start, end, 4) + partNumsLimit.length shouldEqual 4 + // It's not deterministic which 4 docs are returned, but we can check that a valid part ID is in the list + List(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) should contain allElementsOf partNumsLimit.toIterable() + + val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val partNums2 = keyIndex.partIdsFromFilters(Seq(filter2), start, end) + partNums2.toIterable() should contain theSameElementsAs List(7, 8, 9) + + // return only 2 partIds - with filter + val partNumsLimitFilter = keyIndex.partIdsFromFilters(Seq(filter2), start, end, 2) + partNumsLimitFilter.length shouldEqual 2 + List(7,8,9) should contain allElementsOf partNumsLimitFilter.toIterable() + + val filter3 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums3 = keyIndex.partIdsFromFilters(Seq(filter3), start, end) + partNums3.toIterable() should contain theSameElementsAs List(8, 9) + + val filter4 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums4 = keyIndex.partIdsFromFilters(Seq(filter4), 10, start-1) + partNums4 shouldEqual debox.Buffer.empty[Int] + + val filter5 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums5 = keyIndex.partIdsFromFilters(Seq(filter5), end + 100, end + 100000) + partNums5 should not equal debox.Buffer.empty[Int] + + val filter6 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums6 = keyIndex.partIdsFromFilters(Seq(filter6), start - 10000, end ) + partNums6 should not equal debox.Buffer.empty[Int] + + val filter7 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums7 = keyIndex.partIdsFromFilters(Seq(filter7), (start + end)/2, end + 1000 ) + partNums7 should not equal debox.Buffer.empty[Int] + + // tests to validate schema ID filter + val filter8 = Seq( ColumnFilter("Actor2Code", Equals("GOV".utf8)), + ColumnFilter("_type_", Equals("schemaID:46894".utf8))) + val partNums8 = keyIndex.partIdsFromFilters(filter8, start, end) + partNums8.toIterable() should contain theSameElementsAs List(7, 8, 9) + + val filter9 = Seq( ColumnFilter("Actor2Code", Equals("GOV".utf8)), + ColumnFilter("_type_", Equals("prom-counter".utf8))) + val partNums9 = keyIndex.partIdsFromFilters(filter9, start, end) + partNums9.length shouldEqual 0 + } + + it("should parse filters with UTF8Wrapper and string correctly") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + + keyIndex.refreshReadersBlocking() + + val filter2 = ColumnFilter("Actor2Name", Equals(UTF8Wrapper("REGIME".utf8))) + val partNums2 = keyIndex.partIdsFromFilters(Seq(filter2), 0, Long.MaxValue) + partNums2.toIterable() should contain theSameElementsAs List(8, 9) + + val filter3 = ColumnFilter("Actor2Name", Equals("REGIME")) + val partNums3 = keyIndex.partIdsFromFilters(Seq(filter3), 0, Long.MaxValue) + partNums3.toIterable() should contain theSameElementsAs List(8, 9) + } + + it("should fetch part key records from filters correctly") { + // Add the first ten keys and row numbers + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, i, i, i + 10)() + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result = keyIndex.partKeyRecordsFromFilters(Seq(filter2), 0, Long.MaxValue) + val expected = Seq(pkrs(7), pkrs(8), pkrs(9)) + + result.map(_.partKey.toSeq) should contain theSameElementsAs expected.map(_.partKey.toSeq) + result.map( p => (p.startTime, p.endTime)) should contain theSameElementsAs expected.map( p => (p.startTime, p.endTime)) + } + + it("should fetch only two part key records from filters") { + // Add the first ten keys and row numbers + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, i, i, i + 10)() + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result = keyIndex.partKeyRecordsFromFilters(Seq(filter2), 0, Long.MaxValue, 2) + val expected = Seq(pkrs(7), pkrs(8), pkrs(9)) + + result.length shouldEqual 2 + // Which two elements are chosen aren't deterministic - any of 7,8,9 may come back + expected.map(_.partKey.toSeq) should contain allElementsOf result.map(_.partKey.toSeq) + expected.map(p => (p.startTime, p.endTime)) should contain allElementsOf result.map(p => (p.startTime, p.endTime)) + } + + it("should fetch records from filters correctly with a missing label != with a non empty value") { + // Weird case in prometheus where if a non existent label is used with != check with an empty value, + // the returned result includes all results where the label is missing and those where the label exists + // and the value is not equal to the provided value. + // Check next test case for the check for empty value + + // Add the first ten keys and row numbers + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, i, i, i + 10)() + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + // Filter != condition on a field that doesn't exist must not affect the result + + val filter1 = ColumnFilter("some", NotEquals("t".utf8)) + val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result = keyIndex.partKeyRecordsFromFilters(Seq(filter1, filter2), 0, Long.MaxValue) + val expected = Seq(pkrs(7), pkrs(8), pkrs(9)) + + result.map(_.partKey.toSeq) should contain theSameElementsAs expected.map(_.partKey.toSeq) + result.map( p => (p.startTime, p.endTime)) should contain theSameElementsAs expected.map( p => (p.startTime, p.endTime)) + } + + it("should not fetch records from filters correctly with a missing label != with an empty value") { + + // Weird case in prometheus where if a non existent label is used with != check with an empty value, + // the returned result is empty + + // Add the first ten keys and row numbers + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, i, i, i + 10)() + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + // Filter != condition on a field that doesn't exist must not affect the result + + val filter1 = ColumnFilter("some", NotEquals("".utf8)) + val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result = keyIndex.partKeyRecordsFromFilters(Seq(filter1, filter2), 0, Long.MaxValue) + result.isEmpty shouldBe true + } + + it("should add part keys and fetch startTimes correctly for more than 1024 keys") { + val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit + val start = System.currentTimeMillis() + // we dont care much about the partKey here, but the startTime against partId. + val partKeys = Stream.continually(readers.head).take(numPartIds).toList + partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i)() + } + keyIndex.refreshReadersBlocking() + + val startTimes = keyIndex.startTimeFromPartIds((0 until numPartIds).iterator) + for { i <- 0 until numPartIds} { + startTimes(i) shouldEqual start + i + } + } + + it("should upsert part keys with endtime by partKey should work and return only active partkeys") { + // Add the first ten keys and row numbers + val expectedSHA256PartIds = ArrayBuffer.empty[String] + partKeyFromRecords(dataset6, records(dataset6, uniqueReader.take(10)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + val time = System.currentTimeMillis() + val pkOnHeap = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pkOnHeap, -1, time)( + pkOnHeap.length, + PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pkOnHeap, 0, pkOnHeap.length)) + if (i % 2 == 0) { + keyIndex.upsertPartKey(pkOnHeap, -1, time, time + 300)(pkOnHeap.length, + PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pkOnHeap, 0, pkOnHeap.length)) + } else { + expectedSHA256PartIds.append(PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pkOnHeap, 0, pkOnHeap.length)) + } + } + keyIndex.refreshReadersBlocking() + + keyIndex.indexNumEntries shouldEqual 10 + + val activelyIngestingParts = + keyIndex.partKeyRecordsFromFilters(Nil, System.currentTimeMillis() + 500, Long.MaxValue) + activelyIngestingParts.size shouldBe 5 + + + val actualSha256PartIds = activelyIngestingParts.map(r => { + PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(r.partKey, 0, r.partKey.length) + }) + + expectedSHA256PartIds.toSet shouldEqual actualSha256PartIds.toSet + } + + it("should add part keys and fetch partIdsEndedBefore and removePartKeys correctly for more than 1024 keys") { + val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit + val start = 1000 + // we dont care much about the partKey here, but the startTime against partId. + val partKeys = Stream.continually(readers.head).take(numPartIds).toList + partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i, start + i + 100)() + } + keyIndex.refreshReadersBlocking() + + val pIds = keyIndex.partIdsEndedBefore(start + 200) + val pIdsList = pIds.toList() + for { i <- 0 until numPartIds} { + pIdsList.contains(i) shouldEqual (if (i <= 100) true else false) + } + + keyIndex.removePartKeys(pIds) + keyIndex.refreshReadersBlocking() + + for { i <- 0 until numPartIds} { + keyIndex.partKeyFromPartId(i).isDefined shouldEqual (if (i <= 100) false else true) + } + + } + + it("should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys with del count") { + // Identical to test + // it("should add part keys and fetch partIdsEndedBefore and removePartKeys correctly for more than 1024 keys") + // However, combines them not requiring us to get partIds and pass them tpo remove + val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit + val start = 1000 + // we dont care much about the partKey here, but the startTime against partId. + val partKeys = Stream.continually(readers.head).take(numPartIds).toList + partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i, start + i + 100)() + } + keyIndex.refreshReadersBlocking() + val pIdsList = keyIndex.partIdsEndedBefore(start + 200).toList() + for { i <- 0 until numPartIds} { + pIdsList.contains(i) shouldEqual (i <= 100) + } + val numDeleted = keyIndex.removePartitionsEndedBefore(start + 200) + numDeleted shouldEqual pIdsList.size + keyIndex.refreshReadersBlocking() + + // Ensure everything expected is present or deleted + for { i <- 0 until numPartIds} { + // Everything with partId > 100 should be present + keyIndex.partKeyFromPartId(i).isDefined shouldEqual (i > 100) + } + + // Important, while the unit test uses partIds to assert the presence or absence before and after deletion + // it is no longer required to have partIds in the index on non unit test setup + } + + it("should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys w/o del count") { + // identical + // to should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys w/o del count + // except that this one returns 0 for numDocuments deleted + + val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit + val start = 1000 + val partKeys = Stream.continually(readers.head).take(numPartIds).toList + partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i, start + i + 100)() + } + keyIndex.refreshReadersBlocking() + val pIdsList = keyIndex.partIdsEndedBefore(start + 200).toList() + for { i <- 0 until numPartIds} { + pIdsList.contains(i) shouldEqual (i <= 100) + } + val numDeleted = keyIndex.removePartitionsEndedBefore(start + 200, false) + numDeleted shouldEqual 0 + keyIndex.refreshReadersBlocking() + + // Ensure everything expected is present or deleted + for { i <- 0 until numPartIds} { + // Everything with partId > 100 should be present + keyIndex.partKeyFromPartId(i).isDefined shouldEqual (i > 100) + } + + // Important, while the unit test uses partIds to assert the presence or absence before and after deletion + // it is no longer required to have partIds in the index on non unit test setup + } + + it("should update part keys with endtime and parse filters correctly") { + val start = System.currentTimeMillis() + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + val time = System.currentTimeMillis() + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, time)() + keyIndex.refreshReadersBlocking() // updates need to be able to read startTime from index, so commit + keyIndex.updatePartKeyWithEndTime(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, time + 10000)() + } + val end = System.currentTimeMillis() + keyIndex.refreshReadersBlocking() + + // Should get empty iterator when passing no filters + val partNums1 = keyIndex.partIdsFromFilters(Nil, start, end) + partNums1.toIterable() should contain theSameElementsAs List(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) + + val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val partNums2 = keyIndex.partIdsFromFilters(Seq(filter2), start, end) + partNums2.toIterable() should contain theSameElementsAs List(7, 8, 9) + + val filter3 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums3 = keyIndex.partIdsFromFilters(Seq(filter3), start, end) + partNums3.toIterable() should contain theSameElementsAs List(8, 9) + + val filter4 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums4 = keyIndex.partIdsFromFilters(Seq(filter4), 10, start-1) + partNums4 shouldEqual debox.Buffer.empty[Int] + + val filter5 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums5 = keyIndex.partIdsFromFilters(Seq(filter5), end + 20000, end + 100000) + partNums5 shouldEqual debox.Buffer.empty[Int] + + val filter6 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums6 = keyIndex.partIdsFromFilters(Seq(filter6), start - 10000, end-1 ) + partNums6 should not equal debox.Buffer.empty[Int] + + val filter7 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) + val partNums7 = keyIndex.partIdsFromFilters(Seq(filter7), (start + end)/2, end + 1000 ) + partNums7 should not equal debox.Buffer.empty[Int] + } + + it("should obtain indexed names and values") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + + keyIndex.refreshReadersBlocking() + + keyIndex.indexNames(10).toList should contain theSameElementsAs Seq("_type_", "Actor2Code", "Actor2Name") + keyIndex.indexValues("not_found").toSeq should equal (Nil) + + val infos = Seq("AFR", "CHN", "COP", "CVL", "EGYEDU").map(_.utf8).map(TermInfo(_, 1)) + val top2infos = Seq(TermInfo("GOV".utf8, 3), TermInfo("AGR".utf8, 2)) + // top 2 items by frequency + keyIndex.indexValues("Actor2Code", 2) shouldEqual top2infos + val allValues = keyIndex.indexValues("Actor2Code") + allValues take 2 shouldEqual top2infos + allValues.drop(2).toSet shouldEqual infos.toSet + } + + it("should be able to AND multiple filters together") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + + keyIndex.refreshReadersBlocking() + + val filters1 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), + ColumnFilter("Actor2Name", Equals("REGIME".utf8))) + val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1.toIterable() should contain theSameElementsAs List(8, 9) + + val filters2 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), + ColumnFilter("Actor2Name", Equals("CHINA".utf8))) + val partNums2 = keyIndex.partIdsFromFilters(filters2, 0, Long.MaxValue) + partNums2 shouldEqual debox.Buffer.empty[Int] + } + + it("should be able to convert pipe regex to TermInSetQuery") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + keyIndex.refreshReadersBlocking() + + val filters1 = Seq(ColumnFilter("Actor2Code", EqualsRegex("GOV|KHM|LAB|MED".utf8))) + val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1.toIterable() should contain theSameElementsAs List(7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 22, 23, 24, 25, 26, 28, 29, 73, 81, 90) + } + + it("should be able to convert prefix regex to PrefixQuery") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + keyIndex.refreshReadersBlocking() + + val filters1 = Seq(ColumnFilter("Actor2Name", EqualsRegex("C.*".utf8))) + val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1.toIterable() should contain theSameElementsAs List(3, 12, 22, 23, 24, 31, 59, 60, 66, 69, 72, 78, 79, 80, 88, 89) + } + + + it("should ignore unsupported columns and return empty filter") { + val index2 = createNewIndex(dataset1.ref, dataset1.schema.partition, true, true, 0, 1.hour.toMillis) + partKeyFromRecords(dataset1, records(dataset1, readers.take(10))).zipWithIndex.foreach { case (addr, i) => + index2.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + keyIndex.refreshReadersBlocking() + + val filters1 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), ColumnFilter("Year", Equals(1979))) + val partNums1 = index2.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1 shouldEqual debox.Buffer.empty[Int] + } + + it("should be able to fetch partKey from partId and partId from partKey") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + val partKeyBytes = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(partKeyBytes, i, System.currentTimeMillis())() + keyIndex.refreshReadersBlocking() + keyIndex.partKeyFromPartId(i).get.bytes shouldEqual partKeyBytes + // keyIndex.partIdFromPartKey(new BytesRef(partKeyBytes)) shouldEqual i + } + } + + it("should be able to fetch label names efficiently using facets") { + + val index3 = createNewIndex(DatasetRef("prometheus"), Schemas.promCounter.partition, + true, true, 0, 1.hour.toMillis) + val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, + "_ns_".utf8 -> "my_ns".utf8) + + // create 1000 time series with 10 metric names + for { i <- 0 until 1000} { + val dynamicLabelNum = i % 5 + val infoLabelNum = i % 10 + val partKey = partBuilder.partKeyFromObjects(Schemas.promCounter, s"counter", + seriesTags + (s"dynamicLabel$dynamicLabelNum".utf8 -> s"dynamicLabelValue".utf8) + + (s"infoLabel$infoLabelNum".utf8 -> s"infoLabelValue".utf8) + ) + index3.addPartKey(partKeyOnHeap(Schemas.promCounter.partition.binSchema, ZeroPointer, partKey), i, 5)() + } + index3.refreshReadersBlocking() + + val filters1 = Seq(ColumnFilter("_ws_", Equals("my_ws")), ColumnFilter("_metric_", Equals("counter"))) + + val partNums1 = index3.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1.length shouldEqual 1000 + + val labelValues1 = index3.labelNamesEfficient(filters1, 0, Long.MaxValue) + labelValues1.toSet shouldEqual (0 until 5).map(c => s"dynamicLabel$c").toSet ++ + (0 until 10).map(c => s"infoLabel$c").toSet ++ + Set("_ns_", "_ws_", "_metric_", "_type_") + } + + it("should be able to fetch label values efficiently using facets") { + val index3 = createNewIndex(DatasetRef("prometheus"), Schemas.promCounter.partition, + true, true, 0, 1.hour.toMillis) + val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, + "_ns_".utf8 -> "my_ns".utf8) + + // create 1000 time series with 10 metric names + for { i <- 0 until 1000} { + val counterNum = i % 10 + val partKey = partBuilder.partKeyFromObjects(Schemas.promCounter, s"counter$counterNum", + seriesTags + ("instance".utf8 -> s"instance$i".utf8)) + index3.addPartKey(partKeyOnHeap(Schemas.promCounter.partition.binSchema, ZeroPointer, partKey), i, 5)() + } + index3.refreshReadersBlocking() + val filters1 = Seq(ColumnFilter("_ws_", Equals("my_ws"))) + + val partNums1 = index3.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1.length shouldEqual 1000 + + val labelValues1 = index3.labelValuesEfficient(filters1, 0, Long.MaxValue, "_metric_") + labelValues1.toSet shouldEqual (0 until 10).map(c => s"counter$c").toSet + + val filters2 = Seq(ColumnFilter("_ws_", Equals("NotExist"))) + val labelValues2 = index3.labelValuesEfficient(filters2, 0, Long.MaxValue, "_metric_") + labelValues2.size shouldEqual 0 + + val filters3 = Seq(ColumnFilter("_metric_", Equals("counter1"))) + val labelValues3 = index3.labelValuesEfficient(filters3, 0, Long.MaxValue, "_metric_") + labelValues3 shouldEqual Seq("counter1") + + val labelValues4 = index3.labelValuesEfficient(filters1, 0, Long.MaxValue, "instance", 1000) + labelValues4.toSet shouldEqual (0 until 1000).map(c => s"instance$c").toSet + } + + // Testcases to test additionalFacet config + + it("should be able to fetch label values efficiently using additonal facets") { + val facetIndex = createNewIndex(dataset7.ref, dataset7.schema.partition, + true, true, 0, 1.hour.toMillis) + val addedKeys = partKeyFromRecords(dataset7, records(dataset7, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val start = Math.abs(Random.nextLong()) + facetIndex.addPartKey(partKeyOnHeap(dataset7.partKeySchema, ZeroPointer, addr), i, start)() + } + facetIndex.refreshReadersBlocking() + val filters1 = Seq.empty + + val partNums1 = facetIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1.length shouldEqual 10 + + val labelValues1 = facetIndex.labelValuesEfficient(filters1, 0, Long.MaxValue, "Actor2Code") + labelValues1.length shouldEqual 7 + + val labelValues2 = facetIndex.labelValuesEfficient(filters1, 0, Long.MaxValue, "Actor2Code-Actor2Name") + labelValues2.length shouldEqual 8 + + val labelValues3 = facetIndex.labelValuesEfficient(filters1, 0, Long.MaxValue, "Actor2Name-Actor2Code") + labelValues3.length shouldEqual 8 + + labelValues1.sorted.toSet shouldEqual labelValues2.map(_.split("\u03C0")(0)).sorted.toSet + labelValues1.sorted.toSet shouldEqual labelValues3.map(_.split("\u03C0")(1)).sorted.toSet + + val filters2 = Seq(ColumnFilter("Actor2Code", Equals("GOV"))) + + val labelValues12 = facetIndex.labelValuesEfficient(filters2, 0, Long.MaxValue, "Actor2Name") + labelValues12.length shouldEqual 2 + + val labelValues22 = facetIndex.labelValuesEfficient(filters2, 0, Long.MaxValue, "Actor2Code-Actor2Name") + labelValues22.length shouldEqual 2 + + val labelValues32 = facetIndex.labelValuesEfficient(filters2, 0, Long.MaxValue, "Actor2Name-Actor2Code") + labelValues32.length shouldEqual 2 + + labelValues12.sorted shouldEqual labelValues22.map(_.split("\u03C0")(1)).sorted + labelValues12.sorted shouldEqual labelValues32.map(_.split("\u03C0")(0)).sorted + + } + + + it("must clean the input directory for Index state apart from Synced and Refreshing") { + val events = ArrayBuffer.empty[(IndexState.Value, Long)] + IndexState.values.foreach { + indexState => + val indexDirectory = new File( + System.getProperty("java.io.tmpdir"), "part-key-lucene-index-event") + val shardDirectory = new File(indexDirectory, dataset6.ref + File.separator + "0") + shardDirectory.mkdirs() + new File(shardDirectory, "empty").createNewFile() + // Validate the file named empty exists + assert(shardDirectory.list().exists(_.equals("empty"))) + val index = createNewIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, + Some(indexDirectory), + Some(new IndexMetadataStore { + def currentState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = + (indexState, None) + + def updateState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = { + events.append((state, time)) + } + + override def initState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = + currentState(datasetRef: DatasetRef, shard: Int) + + override def updateInitState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit + = { + + } + })) + index.closeIndex() + events.toList match { + case (IndexState.Empty, _) :: Nil if indexState != IndexState.Synced && indexState != IndexState.Refreshing => + // The file originally present must not be available + assert(!shardDirectory.list().exists(_.equals("empty"))) + events.clear() + case Nil if indexState == IndexState.Synced + || indexState == IndexState.Refreshing + || indexState == IndexState.Empty => + // Empty state denotes the FS is empty, it is not cleaned up again to ensure its empty + // The file originally present "must" be available, which means no cleanup was done + assert(shardDirectory.list().exists(_.equals("empty"))) + case _ => + fail("Expected an index state Empty after directory cleanup") + } + } + } + + it("Should update the state as empty after the cleanup is from a corrupt index") { + val events = ArrayBuffer.empty[(IndexState.Value, Long)] + IndexState.values.foreach { + indexState => + val indexDirectory = new File( + System.getProperty("java.io.tmpdir"), "part-key-lucene-index-event") + val shardDirectory = new File(indexDirectory, dataset6.ref + File.separator + "0") + // Delete directory to create an index from scratch + scala.reflect.io.Directory(shardDirectory).deleteRecursively() + shardDirectory.mkdirs() + // Validate the file named empty exists + val index = createNewIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, + Some(indexDirectory),None) + // Add some index entries + val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, + "_ns_".utf8 -> "my_ns".utf8) + for { i <- 0 until 1000} { + val counterNum = i % 10 + val partKey = partBuilder.partKeyFromObjects(Schemas.promCounter, s"counter$counterNum", + seriesTags + ("instance".utf8 -> s"instance$i".utf8)) + index.addPartKey(partKeyOnHeap(Schemas.promCounter.partition.binSchema, ZeroPointer, partKey), i, 5)() + } + + index.closeIndex() + // Garble some index files to force a index corruption + // Just add some junk to the end of the segment files + + shardDirectory.listFiles(new FileFilter { + override def accept(pathname: File): Boolean = { + pathname.toString.contains("segment") || pathname.toString.contains(".term") + } + }).foreach( file => { + Files.writeString(file.toPath, "Hello", StandardOpenOption.APPEND) + }) + + createNewIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, + Some(indexDirectory), + Some( new IndexMetadataStore { + def currentState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = + (indexState, None) + + def updateState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = { + assert(shardDirectory.list().length == 0) + events.append((state, time)) + } + + override def initState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = + currentState(datasetRef: DatasetRef, shard: Int) + + override def updateInitState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit + = { + + } + + })).closeIndex() + // For all states, including states where Index is Synced because the index is corrupt, + // the shard directory should be cleared and the new state should be Empty + events.toList match { + case (IndexState.Empty, _) :: Nil => + // The file originally present must not be available + assert(!shardDirectory.list().exists(_.equals("empty"))) + events.clear() + case other => + fail(s"Expected an index state Empty after directory cleanup, got ${other}") + } + } + } + + it("should get a single match for part keys by a filter") { + + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, -1, i, i + 10)( + pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + val filter1 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1), 4, 10) + + partKeyOpt.isDefined shouldBe true + partKeyOpt.get shouldEqual pkrs(7).partKey + + val filter2 = ColumnFilter("Actor2Code", Equals("NonExist".utf8)) + keyIndex.singlePartKeyFromFilters(Seq(filter2), 4, 10) shouldBe None + } + + it("should get a single match for part keys through a field with empty value") { + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.slice(95, 96)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, -1, i, i + 10)( + pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + val filter1_found = ColumnFilter("Actor2Code", Equals("")) + val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1_found), 4, 10) + partKeyOpt.isDefined shouldBe true + partKeyOpt.get shouldEqual pkrs.head.partKey + + val filter2_found = ColumnFilter("Actor2Code", EqualsRegex("")) + val partKeyOpt2 = keyIndex.singlePartKeyFromFilters(Seq(filter2_found), 4, 10) + partKeyOpt2.isDefined shouldBe true + partKeyOpt2.get shouldEqual pkrs.head.partKey + + val filter3_found = ColumnFilter("Actor2Code", EqualsRegex("^$")) + val partKeyOpt3 = keyIndex.singlePartKeyFromFilters(Seq(filter3_found), 4, 10) + partKeyOpt3.isDefined shouldBe true + partKeyOpt3.get shouldEqual pkrs.head.partKey + } + + it("should get a single match for part keys through a non-existing field") { + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(1)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, -1, i, i + 10)( + pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + val filter1_found = ColumnFilter("NonExistingField", Equals("")) + val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1_found), 4, 10) + partKeyOpt.isDefined shouldBe true + partKeyOpt.get shouldEqual pkrs.head.partKey + + val filter2_found = ColumnFilter("NonExistingField", EqualsRegex("")) + val partKeyOpt2 = keyIndex.singlePartKeyFromFilters(Seq(filter2_found), 4, 10) + partKeyOpt2.isDefined shouldBe true + partKeyOpt2.get shouldEqual pkrs.head.partKey + + val filter3_found = ColumnFilter("NonExistingField", EqualsRegex("^$")) + val partKeyOpt3 = keyIndex.singlePartKeyFromFilters(Seq(filter3_found), 4, 10) + partKeyOpt3.isDefined shouldBe true + partKeyOpt3.get shouldEqual pkrs.head.partKey + } + + it("should get a single match for part keys by a regex filter") { + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, -1, i, i + 10)( + pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + val filter1_found = ColumnFilter("Actor2Code", EqualsRegex("""^GO.*$""")) + val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1_found), 4, 10) + partKeyOpt.isDefined shouldBe true + partKeyOpt.get shouldEqual pkrs(7).partKey + + val filter1_not_found = ColumnFilter("Actor2Code", EqualsRegex("""^GO.*\$""")) + keyIndex.singlePartKeyFromFilters(Seq(filter1_not_found), 4, 10) shouldBe None + + val filter2 = ColumnFilter("Actor2Code", NotEqualsRegex("^.*".utf8)) + keyIndex.singlePartKeyFromFilters(Seq(filter2), 4, 10) shouldBe None + } + + it("Should update the state as TriggerRebuild and throw an exception for any error other than CorruptIndexException") + { + val events = ArrayBuffer.empty[(IndexState.Value, Long)] + IndexState.values.foreach { + indexState => + val indexDirectory = new File( + System.getProperty("java.io.tmpdir"), "part-key-lucene-index-event") + val shardDirectory = new File(indexDirectory, dataset6.ref + File.separator + "0") + shardDirectory.mkdirs() + new File(shardDirectory, "empty").createNewFile() + Files.writeString(new File(shardDirectory, "meta.json").toPath, "Hello", StandardOpenOption.CREATE) + // Make directory readonly to for an IOException when attempting to write + shardDirectory.setWritable(false) + // Validate the file named empty exists + assert(shardDirectory.list().exists(_.equals("empty"))) + try { + val index = createNewIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, + Some(indexDirectory), + Some( new IndexMetadataStore { + def currentState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = + (indexState, None) + + def updateState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = { + events.append((state, time)) + } + + override def initState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = + currentState(datasetRef: DatasetRef, shard: Int) + + override def updateInitState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long) + :Unit = { + + } + })) + index.closeIndex() + } catch { + case is: IllegalStateException => + assert(is.getMessage.equals("Unable to clean up index directory")) + + events.toList match { + case (IndexState.TriggerRebuild, _) :: Nil => + // The file originally present would still be there as the directory + // is made readonly + assert(shardDirectory.list().exists(_.equals("empty"))) + events.clear() + case other => + fail(s"Expected an index state Empty after directory cleanup - got ${other}") + } + } finally { + shardDirectory.setWritable(true) + } + + } + } + + it("should match records without label when .* is provided on a non existent label") { + + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.schema.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, i, i, i + 10)() + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + + // Query with just the existing Label name + val filter1 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result1 = keyIndex.partKeyRecordsFromFilters(Seq(filter1), 0, Long.MaxValue) + val expected1 = Seq(pkrs(7), pkrs(8), pkrs(9)) + + result1.map(_.partKey.toSeq) should contain theSameElementsAs expected1.map(_.partKey.toSeq) + result1.map(p => (p.startTime, p.endTime)) should contain theSameElementsAs expected1.map(p => (p.startTime, p.endTime)) + + // Query with non existent label name with an empty regex + val filter2 = ColumnFilter("dummy", EqualsRegex(".*".utf8)) + val filter3 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result2 = keyIndex.partKeyRecordsFromFilters(Seq(filter2, filter3), 0, Long.MaxValue) + val expected2 = Seq(pkrs(7), pkrs(8), pkrs(9)) + + result2.map(_.partKey.toSeq) should contain theSameElementsAs expected2.map(_.partKey.toSeq) + result2.map(p => (p.startTime, p.endTime)) should contain theSameElementsAs expected2.map(p => (p.startTime, p.endTime)) + + // Query with non existent label name with an regex matching at least 1 character + val filter4 = ColumnFilter("dummy", EqualsRegex(".+".utf8)) + val filter5 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result3 = keyIndex.partKeyRecordsFromFilters(Seq(filter4, filter5), 0, Long.MaxValue) + result3 shouldEqual Seq() + + // Query with non existent label name with an empty regex + val filter6 = ColumnFilter("dummy", EqualsRegex("".utf8)) + val filter7 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result4 = keyIndex.partKeyRecordsFromFilters(Seq(filter6, filter7), 0, Long.MaxValue) + val expected4 = Seq(pkrs(7), pkrs(8), pkrs(9)) + result4.map(_.partKey.toSeq) should contain theSameElementsAs expected4.map(_.partKey.toSeq) + result4.map(p => (p.startTime, p.endTime)) should contain theSameElementsAs expected4.map(p => (p.startTime, p.endTime)) + + // Query with non existent label name with an empty equals + val filter8 = ColumnFilter("dummy", Equals("".utf8)) + val filter9 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result5 = keyIndex.partKeyRecordsFromFilters(Seq(filter8, filter9), 0, Long.MaxValue) + val expected5 = Seq(pkrs(7), pkrs(8), pkrs(9)) + result5.map(_.partKey.toSeq) should contain theSameElementsAs expected5.map(_.partKey.toSeq) + result5.map(p => (p.startTime, p.endTime)) should contain theSameElementsAs expected5.map(p => (p.startTime, p.endTime)) + + + val filter10 = ColumnFilter("Actor2Code", EqualsRegex(".*".utf8)) + val result10= keyIndex.partKeyRecordsFromFilters(Seq(filter10), 0, Long.MaxValue) + result10.map(_.partKey.toSeq) should contain theSameElementsAs pkrs.map(_.partKey.toSeq) + result10.map(p => (p.startTime, p.endTime)) should contain theSameElementsAs pkrs.map(p => (p.startTime, p.endTime)) + } + + it("should translate a part key to a part ID") { + val pks = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, i, System.currentTimeMillis())() + + pk + } + keyIndex.refreshReadersBlocking() + + val partKeyOpt = keyIndex.partIdFromPartKeySlow(pks(0), UnsafeUtils.arayOffset) + partKeyOpt.isDefined shouldBe true + partKeyOpt.get shouldEqual 0 + } + } + // scalastyle:on cyclomatic.complexity + // scalastyle:on method.length +} diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala index f39368270c..9c90a7c536 100644 --- a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala @@ -2,26 +2,24 @@ package filodb.core.memstore import com.googlecode.javaewah.IntIterator import filodb.core._ -import filodb.core.binaryrecord2.{RecordBuilder, RecordSchema} +import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.ratelimit.{CardinalityTracker, RocksDbCardinalityStore} -import filodb.core.metadata.{Dataset, DatasetOptions, Schemas} +import filodb.core.metadata.{Dataset, DatasetOptions, PartitionSchema, Schemas} import filodb.core.query.{ColumnFilter, Filter} import filodb.memory.{BinaryRegionConsumer, MemFactory} import filodb.memory.format.UnsafeUtils.ZeroPointer -import filodb.memory.format.UTF8Wrapper import filodb.memory.format.ZeroCopyUTF8String._ import org.apache.lucene.util.BytesRef import org.scalatest.BeforeAndAfter import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers -import java.io.{File, FileFilter} -import java.nio.file.{Files, StandardOpenOption} +import java.io.File import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.util.Random -class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfter { +class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfter with PartKeyIndexRawSpec { import Filter._ import GdeltTestData._ @@ -30,10 +28,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte val partBuilder = new RecordBuilder(TestData.nativeMem) - def partKeyOnHeap(partKeySchema: RecordSchema, - base: Any, - offset: Long): Array[Byte] = partKeySchema.asByteArray(base, offset) - before { keyIndex.reset() keyIndex.refreshReadersBlocking() @@ -51,101 +45,19 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } } - it("should add part keys and parse filters correctly") { - val start = System.currentTimeMillis() - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() - } - val end = System.currentTimeMillis() - keyIndex.refreshReadersBlocking() - - // Should get empty iterator when passing no filters - val partNums1 = keyIndex.partIdsFromFilters(Nil, start, end) - partNums1 shouldEqual debox.Buffer(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) - - // return only 4 partIds - empty filter - val partNumsLimit = keyIndex.partIdsFromFilters(Nil, start, end, 4) - partNumsLimit shouldEqual debox.Buffer(0, 1, 2, 3) - - val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val partNums2 = keyIndex.partIdsFromFilters(Seq(filter2), start, end) - partNums2 shouldEqual debox.Buffer(7, 8, 9) - - // return only 2 partIds - with filter - val partNumsLimitFilter = keyIndex.partIdsFromFilters(Seq(filter2), start, end, 2) - partNumsLimitFilter shouldEqual debox.Buffer(7, 8) - - val filter3 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums3 = keyIndex.partIdsFromFilters(Seq(filter3), start, end) - partNums3 shouldEqual debox.Buffer(8, 9) - - val filter4 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums4 = keyIndex.partIdsFromFilters(Seq(filter4), 10, start-1) - partNums4 shouldEqual debox.Buffer.empty[Int] - - val filter5 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums5 = keyIndex.partIdsFromFilters(Seq(filter5), end + 100, end + 100000) - partNums5 should not equal debox.Buffer.empty[Int] - - val filter6 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums6 = keyIndex.partIdsFromFilters(Seq(filter6), start - 10000, end ) - partNums6 should not equal debox.Buffer.empty[Int] - - val filter7 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums7 = keyIndex.partIdsFromFilters(Seq(filter7), (start + end)/2, end + 1000 ) - partNums7 should not equal debox.Buffer.empty[Int] - - // tests to validate schema ID filter - val filter8 = Seq( ColumnFilter("Actor2Code", Equals("GOV".utf8)), - ColumnFilter("_type_", Equals("schemaID:46894".utf8))) - val partNums8 = keyIndex.partIdsFromFilters(filter8, start, end) - partNums8 shouldEqual debox.Buffer(7, 8, 9) - - val filter9 = Seq( ColumnFilter("Actor2Code", Equals("GOV".utf8)), - ColumnFilter("_type_", Equals("prom-counter".utf8))) - val partNums9 = keyIndex.partIdsFromFilters(filter9, start, end) - partNums9.length shouldEqual 0 - - } - - it("should fetch part key records from filters correctly") { - // Add the first ten keys and row numbers - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, i, i, i + 10)() - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result = keyIndex.partKeyRecordsFromFilters(Seq(filter2), 0, Long.MaxValue) - val expected = Seq(pkrs(7), pkrs(8), pkrs(9)) - - result.map(_.partKey.toSeq) shouldEqual expected.map(_.partKey.toSeq) - result.map( p => (p.startTime, p.endTime)) shouldEqual expected.map( p => (p.startTime, p.endTime)) - } - - it("should fetch only two part key records from filters") { - // Add the first ten keys and row numbers - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, i, i, i + 10)() - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result = keyIndex.partKeyRecordsFromFilters(Seq(filter2), 0, Long.MaxValue, 2) - val expected = Seq(pkrs(7), pkrs(8)) - - result.map(_.partKey.toSeq) shouldEqual expected.map(_.partKey.toSeq) - result.map(p => (p.startTime, p.endTime)) shouldEqual expected.map(p => (p.startTime, p.endTime)) + protected def createNewIndex(ref: DatasetRef, + schema: PartitionSchema, + facetEnabledAllLabels: Boolean, + facetEnabledShardKeyLabels: Boolean, + shardNum: Int, + retentionMillis: Long, + diskLocation: Option[File], + lifecycleManager: Option[IndexMetadataStore]): PartKeyIndexRaw = { + new PartKeyLuceneIndex(ref, schema, facetEnabledAllLabels, facetEnabledShardKeyLabels, shardNum, retentionMillis, + diskLocation, lifecycleManager) } + it should behave like commonPartKeyTests(keyIndex, partBuilder) it("should fetch part key records from filters correctly with index caching disabled") { // Add the first ten keys and row numbers @@ -174,7 +86,7 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte }) } - + it("should fetch part key iterator records from filters correctly") { // Add the first ten keys and row numbers val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) @@ -200,54 +112,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } } - it("should fetch records from filters correctly with a missing label != with a non empty value") { - // Weird case in prometheus where if a non existent label is used with != check with an empty value, - // the returned result includes all results where the label is missing and those where the label exists - // and the value is not equal to the provided value. - // Check next test case for the check for empty value - - // Add the first ten keys and row numbers - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, i, i, i + 10)() - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - // Filter != condition on a field that doesn't exist must not affect the result - - val filter1 = ColumnFilter("some", NotEquals("t".utf8)) - val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result = keyIndex.partKeyRecordsFromFilters(Seq(filter1, filter2), 0, Long.MaxValue) - val expected = Seq(pkrs(7), pkrs(8), pkrs(9)) - - result.map(_.partKey.toSeq) shouldEqual expected.map(_.partKey.toSeq) - result.map( p => (p.startTime, p.endTime)) shouldEqual expected.map( p => (p.startTime, p.endTime)) - } - - it("should not fetch records from filters correctly with a missing label != with an empty value") { - - // Weird case in prometheus where if a non existent label is used with != check with an empty value, - // the returned result is empty - - // Add the first ten keys and row numbers - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, i, i, i + 10)() - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - // Filter != condition on a field that doesn't exist must not affect the result - - val filter1 = ColumnFilter("some", NotEquals("".utf8)) - val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result = keyIndex.partKeyRecordsFromFilters(Seq(filter1, filter2), 0, Long.MaxValue) - result.isEmpty shouldBe true - } - it("should upsert part keys with endtime and foreachPartKeyStillIngesting should work") { // Add the first ten keys and row numbers partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) @@ -269,297 +133,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } - it("should upsert part keys with endtime by partKey should work and return only active partkeys") { - // Add the first ten keys and row numbers - val expectedSHA256PartIds = ArrayBuffer.empty[String] - partKeyFromRecords(dataset6, records(dataset6, uniqueReader.take(10)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - val time = System.currentTimeMillis() - val pkOnHeap = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pkOnHeap, -1, time)( - pkOnHeap.length, - PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pkOnHeap, 0, pkOnHeap.length)) - if (i % 2 == 0) { - keyIndex.upsertPartKey(pkOnHeap, -1, time, time + 300)(pkOnHeap.length, - PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pkOnHeap, 0, pkOnHeap.length)) - } else { - expectedSHA256PartIds.append(PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pkOnHeap, 0, pkOnHeap.length)) - } - } - keyIndex.refreshReadersBlocking() - - keyIndex.indexNumEntries shouldEqual 10 - - val activelyIngestingParts = - keyIndex.partKeyRecordsFromFilters(Nil, System.currentTimeMillis() + 500, Long.MaxValue) - activelyIngestingParts.size shouldBe 5 - - - val actualSha256PartIds = activelyIngestingParts.map(r => { - PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(r.partKey, 0, r.partKey.length) - }) - - expectedSHA256PartIds.toSet shouldEqual actualSha256PartIds.toSet - } - - - - it("should add part keys and fetch startTimes correctly for more than 1024 keys") { - val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit - val start = System.currentTimeMillis() - // we dont care much about the partKey here, but the startTime against partId. - val partKeys = Stream.continually(readers.head).take(numPartIds).toList - partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i)() - } - keyIndex.refreshReadersBlocking() - - val startTimes = keyIndex.startTimeFromPartIds((0 until numPartIds).iterator) - for { i <- 0 until numPartIds} { - startTimes(i) shouldEqual start + i - } - } - - it("should add part keys and fetch partIdsEndedBefore and removePartKeys correctly for more than 1024 keys") { - val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit - val start = 1000 - // we dont care much about the partKey here, but the startTime against partId. - val partKeys = Stream.continually(readers.head).take(numPartIds).toList - partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i, start + i + 100)() - } - keyIndex.refreshReadersBlocking() - - val pIds = keyIndex.partIdsEndedBefore(start + 200) - val pIdsList = pIds.toList() - for { i <- 0 until numPartIds} { - pIdsList.contains(i) shouldEqual (if (i <= 100) true else false) - } - - keyIndex.removePartKeys(pIds) - keyIndex.refreshReadersBlocking() - - for { i <- 0 until numPartIds} { - keyIndex.partKeyFromPartId(i).isDefined shouldEqual (if (i <= 100) false else true) - } - - } - - it("should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys with del count") { - // Identical to test - // it("should add part keys and fetch partIdsEndedBefore and removePartKeys correctly for more than 1024 keys") - // However, combines them not requiring us to get partIds and pass them tpo remove - val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit - val start = 1000 - // we dont care much about the partKey here, but the startTime against partId. - val partKeys = Stream.continually(readers.head).take(numPartIds).toList - partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i, start + i + 100)() - } - keyIndex.refreshReadersBlocking() - val pIdsList = keyIndex.partIdsEndedBefore(start + 200).toList() - for { i <- 0 until numPartIds} { - pIdsList.contains(i) shouldEqual (i <= 100) - } - val numDeleted = keyIndex.removePartitionsEndedBefore(start + 200) - numDeleted shouldEqual pIdsList.size - keyIndex.refreshReadersBlocking() - - // Ensure everything expected is present or deleted - for { i <- 0 until numPartIds} { - // Everything with partId > 100 should be present - keyIndex.partKeyFromPartId(i).isDefined shouldEqual (i > 100) - } - - // Important, while the unit test uses partIds to assert the presence or absence before and after deletion - // it is no longer required to have partIds in the index on non unit test setup - } - - it("should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys w/o del count") { - // identical - // to should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys w/o del count - // except that this one returns 0 for numDocuments deleted - - val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit - val start = 1000 - val partKeys = Stream.continually(readers.head).take(numPartIds).toList - partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, start + i, start + i + 100)() - } - keyIndex.refreshReadersBlocking() - val pIdsList = keyIndex.partIdsEndedBefore(start + 200).toList() - for { i <- 0 until numPartIds} { - pIdsList.contains(i) shouldEqual (i <= 100) - } - val numDeleted = keyIndex.removePartitionsEndedBefore(start + 200, false) - numDeleted shouldEqual 0 - keyIndex.refreshReadersBlocking() - - // Ensure everything expected is present or deleted - for { i <- 0 until numPartIds} { - // Everything with partId > 100 should be present - keyIndex.partKeyFromPartId(i).isDefined shouldEqual (i > 100) - } - - // Important, while the unit test uses partIds to assert the presence or absence before and after deletion - // it is no longer required to have partIds in the index on non unit test setup - } - - it("should update part keys with endtime and parse filters correctly") { - val start = System.currentTimeMillis() - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - val time = System.currentTimeMillis() - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, time)() - keyIndex.refreshReadersBlocking() // updates need to be able to read startTime from index, so commit - keyIndex.updatePartKeyWithEndTime(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, time + 10000)() - } - val end = System.currentTimeMillis() - keyIndex.refreshReadersBlocking() - - // Should get empty iterator when passing no filters - val partNums1 = keyIndex.partIdsFromFilters(Nil, start, end) - partNums1 shouldEqual debox.Buffer(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) - - val filter2 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val partNums2 = keyIndex.partIdsFromFilters(Seq(filter2), start, end) - partNums2 shouldEqual debox.Buffer(7, 8, 9) - - val filter3 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums3 = keyIndex.partIdsFromFilters(Seq(filter3), start, end) - partNums3 shouldEqual debox.Buffer(8, 9) - - val filter4 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums4 = keyIndex.partIdsFromFilters(Seq(filter4), 10, start-1) - partNums4 shouldEqual debox.Buffer.empty[Int] - - val filter5 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums5 = keyIndex.partIdsFromFilters(Seq(filter5), end + 20000, end + 100000) - partNums5 shouldEqual debox.Buffer.empty[Int] - - val filter6 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums6 = keyIndex.partIdsFromFilters(Seq(filter6), start - 10000, end-1 ) - partNums6 should not equal debox.Buffer.empty[Int] - - val filter7 = ColumnFilter("Actor2Name", Equals("REGIME".utf8)) - val partNums7 = keyIndex.partIdsFromFilters(Seq(filter7), (start + end)/2, end + 1000 ) - partNums7 should not equal debox.Buffer.empty[Int] - } - - it("should parse filters with UTF8Wrapper and string correctly") { - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() - } - - keyIndex.refreshReadersBlocking() - - val filter2 = ColumnFilter("Actor2Name", Equals(UTF8Wrapper("REGIME".utf8))) - val partNums2 = keyIndex.partIdsFromFilters(Seq(filter2), 0, Long.MaxValue) - partNums2 shouldEqual debox.Buffer(8, 9) - - val filter3 = ColumnFilter("Actor2Name", Equals("REGIME")) - val partNums3 = keyIndex.partIdsFromFilters(Seq(filter3), 0, Long.MaxValue) - partNums3 shouldEqual debox.Buffer(8, 9) - } - - it("should obtain indexed names and values") { - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() - } - - keyIndex.refreshReadersBlocking() - - keyIndex.indexNames(10).toList shouldEqual Seq("_type_", "Actor2Code", "Actor2Name") - keyIndex.indexValues("not_found").toSeq should equal (Nil) - - val infos = Seq("AFR", "CHN", "COP", "CVL", "EGYEDU").map(_.utf8).map(TermInfo(_, 1)) - val top2infos = Seq(TermInfo("GOV".utf8, 3), TermInfo("AGR".utf8, 2)) - // top 2 items by frequency - keyIndex.indexValues("Actor2Code", 2) shouldEqual top2infos - val allValues = keyIndex.indexValues("Actor2Code") - allValues take 2 shouldEqual top2infos - allValues.drop(2).toSet shouldEqual infos.toSet - } - - it("should be able to AND multiple filters together") { - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() - } - - keyIndex.refreshReadersBlocking() - - val filters1 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), - ColumnFilter("Actor2Name", Equals("REGIME".utf8))) - val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) - partNums1 shouldEqual debox.Buffer(8, 9) - - val filters2 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), - ColumnFilter("Actor2Name", Equals("CHINA".utf8))) - val partNums2 = keyIndex.partIdsFromFilters(filters2, 0, Long.MaxValue) - partNums2 shouldEqual debox.Buffer.empty[Int] - } - - it("should be able to convert pipe regex to TermInSetQuery") { - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() - } - keyIndex.refreshReadersBlocking() - - val filters1 = Seq(ColumnFilter("Actor2Code", EqualsRegex("GOV|KHM|LAB|MED".utf8))) - val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) - partNums1 shouldEqual debox.Buffer(7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 22, 23, 24, 25, 26, 28, 29, 73, 81, 90) - } - - it("should be able to convert prefix regex to PrefixQuery") { - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() - } - keyIndex.refreshReadersBlocking() - - val filters1 = Seq(ColumnFilter("Actor2Name", EqualsRegex("C.*".utf8))) - val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) - partNums1 shouldEqual debox.Buffer(3, 12, 22, 23, 24, 31, 59, 60, 66, 69, 72, 78, 79, 80, 88, 89) - } - - it("should ignore unsupported columns and return empty filter") { - val index2 = new PartKeyLuceneIndex(dataset1.ref, dataset1.schema.partition, true, true, 0, 1.hour.toMillis) - partKeyFromRecords(dataset1, records(dataset1, readers.take(10))).zipWithIndex.foreach { case (addr, i) => - index2.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() - } - keyIndex.refreshReadersBlocking() - - val filters1 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), ColumnFilter("Year", Equals(1979))) - val partNums1 = index2.partIdsFromFilters(filters1, 0, Long.MaxValue) - partNums1 shouldEqual debox.Buffer.empty[Int] - } - - it("should be able to fetch partKey from partId and partId from partKey") { - // Add the first ten keys and row numbers - partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.foreach { case (addr, i) => - val partKeyBytes = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(partKeyBytes, i, System.currentTimeMillis())() - keyIndex.refreshReadersBlocking() - keyIndex.partKeyFromPartId(i).get.bytes shouldEqual partKeyBytes - // keyIndex.partIdFromPartKey(new BytesRef(partKeyBytes)) shouldEqual i - } - } - it("should be able to sort results by endTime, startTime") { val addedKeys = partKeyFromRecords(dataset6, records(dataset6, readers.take(100)), Some(partBuilder)) @@ -586,70 +159,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } } - it("should be able to fetch label names efficiently using facets") { - - val index3 = new PartKeyLuceneIndex(DatasetRef("prometheus"), Schemas.promCounter.partition, - true, true, 0, 1.hour.toMillis) - val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, - "_ns_".utf8 -> "my_ns".utf8) - - // create 1000 time series with 10 metric names - for { i <- 0 until 1000} { - val dynamicLabelNum = i % 5 - val infoLabelNum = i % 10 - val partKey = partBuilder.partKeyFromObjects(Schemas.promCounter, s"counter", - seriesTags + (s"dynamicLabel$dynamicLabelNum".utf8 -> s"dynamicLabelValue".utf8) - + (s"infoLabel$infoLabelNum".utf8 -> s"infoLabelValue".utf8) - ) - index3.addPartKey(partKeyOnHeap(Schemas.promCounter.partition.binSchema, ZeroPointer, partKey), i, 5)() - } - index3.refreshReadersBlocking() - - val filters1 = Seq(ColumnFilter("_ws_", Equals("my_ws")), ColumnFilter("_metric_", Equals("counter"))) - - val partNums1 = index3.partIdsFromFilters(filters1, 0, Long.MaxValue) - partNums1.length shouldEqual 1000 - - val labelValues1 = index3.labelNamesEfficient(filters1, 0, Long.MaxValue) - labelValues1.toSet shouldEqual (0 until 5).map(c => s"dynamicLabel$c").toSet ++ - (0 until 10).map(c => s"infoLabel$c").toSet ++ - Set("_ns_", "_ws_", "_metric_", "_type_") - } - - it("should be able to fetch label values efficiently using facets") { - val index3 = new PartKeyLuceneIndex(DatasetRef("prometheus"), Schemas.promCounter.partition, - true, true, 0, 1.hour.toMillis) - val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, - "_ns_".utf8 -> "my_ns".utf8) - - // create 1000 time series with 10 metric names - for { i <- 0 until 1000} { - val counterNum = i % 10 - val partKey = partBuilder.partKeyFromObjects(Schemas.promCounter, s"counter$counterNum", - seriesTags + ("instance".utf8 -> s"instance$i".utf8)) - index3.addPartKey(partKeyOnHeap(Schemas.promCounter.partition.binSchema, ZeroPointer, partKey), i, 5)() - } - index3.refreshReadersBlocking() - val filters1 = Seq(ColumnFilter("_ws_", Equals("my_ws"))) - - val partNums1 = index3.partIdsFromFilters(filters1, 0, Long.MaxValue) - partNums1.length shouldEqual 1000 - - val labelValues1 = index3.labelValuesEfficient(filters1, 0, Long.MaxValue, "_metric_") - labelValues1.toSet shouldEqual (0 until 10).map(c => s"counter$c").toSet - - val filters2 = Seq(ColumnFilter("_ws_", Equals("NotExist"))) - val labelValues2 = index3.labelValuesEfficient(filters2, 0, Long.MaxValue, "_metric_") - labelValues2.size shouldEqual 0 - - val filters3 = Seq(ColumnFilter("_metric_", Equals("counter1"))) - val labelValues3 = index3.labelValuesEfficient(filters3, 0, Long.MaxValue, "_metric_") - labelValues3 shouldEqual Seq("counter1") - - val labelValues4 = index3.labelValuesEfficient(filters1, 0, Long.MaxValue, "instance", 1000) - labelValues4.toSet shouldEqual (0 until 1000).map(c => s"instance$c").toSet - } - it("should be able to do regular operations when faceting is disabled") { val index3 = new PartKeyLuceneIndex(DatasetRef("prometheus"), Schemas.promCounter.partition, false, false, 0, 1.hour.toMillis) @@ -679,50 +188,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } - // Testcases to test additionalFacet config - - it("should be able to fetch label values efficiently using additonal facets") { - val facetIndex = new PartKeyLuceneIndex(dataset7.ref, dataset7.schema.partition, - true, true, 0, 1.hour.toMillis) - val addedKeys = partKeyFromRecords(dataset7, records(dataset7, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val start = Math.abs(Random.nextLong()) - facetIndex.addPartKey(partKeyOnHeap(dataset7.partKeySchema, ZeroPointer, addr), i, start)() - } - facetIndex.refreshReadersBlocking() - val filters1 = Seq.empty - - val partNums1 = facetIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) - partNums1.length shouldEqual 10 - - val labelValues1 = facetIndex.labelValuesEfficient(filters1, 0, Long.MaxValue, "Actor2Code") - labelValues1.length shouldEqual 7 - - val labelValues2 = facetIndex.labelValuesEfficient(filters1, 0, Long.MaxValue, "Actor2Code-Actor2Name") - labelValues2.length shouldEqual 8 - - val labelValues3 = facetIndex.labelValuesEfficient(filters1, 0, Long.MaxValue, "Actor2Name-Actor2Code") - labelValues3.length shouldEqual 8 - - labelValues1.sorted.toSet shouldEqual labelValues2.map(_.split("\u03C0")(0)).sorted.toSet - labelValues1.sorted.toSet shouldEqual labelValues3.map(_.split("\u03C0")(1)).sorted.toSet - - val filters2 = Seq(ColumnFilter("Actor2Code", Equals("GOV"))) - - val labelValues12 = facetIndex.labelValuesEfficient(filters2, 0, Long.MaxValue, "Actor2Name") - labelValues12.length shouldEqual 2 - - val labelValues22 = facetIndex.labelValuesEfficient(filters2, 0, Long.MaxValue, "Actor2Code-Actor2Name") - labelValues22.length shouldEqual 2 - - val labelValues32 = facetIndex.labelValuesEfficient(filters2, 0, Long.MaxValue, "Actor2Name-Actor2Code") - labelValues32.length shouldEqual 2 - - labelValues12.sorted shouldEqual labelValues22.map(_.split("\u03C0")(1)).sorted - labelValues12.sorted shouldEqual labelValues32.map(_.split("\u03C0")(0)).sorted - - } - it("should be able to do regular operations when faceting is disabled and additional faceting in dataset") { val facetIndex = new PartKeyLuceneIndex(dataset7.ref, dataset7.schema.partition, false, true, 0, 1.hour.toMillis) @@ -742,141 +207,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } should have message "requirement failed: Faceting not enabled for label Actor2Code-Actor2Name; labelValuesEfficient should not have been called" } - it("must clean the input directory for Index state apart from Synced and Refreshing") { - val events = ArrayBuffer.empty[(IndexState.Value, Long)] - IndexState.values.foreach { - indexState => - val indexDirectory = new File( - System.getProperty("java.io.tmpdir"), "part-key-lucene-index-event") - val shardDirectory = new File(indexDirectory, dataset6.ref + File.separator + "0") - shardDirectory.mkdirs() - new File(shardDirectory, "empty").createNewFile() - // Validate the file named empty exists - assert(shardDirectory.list().exists(_.equals("empty"))) - val index = new PartKeyLuceneIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, - Some(indexDirectory), - Some(new IndexMetadataStore { - def currentState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = - (indexState, None) - - def updateState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = { - events.append((state, time)) - } - - override def initState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = - currentState(datasetRef: DatasetRef, shard: Int) - - override def updateInitState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit - = { - - } - })) - index.closeIndex() - events.toList match { - case (IndexState.Empty, _) :: Nil if indexState != IndexState.Synced && indexState != IndexState.Refreshing => - // The file originally present must not be available - assert(!shardDirectory.list().exists(_.equals("empty"))) - events.clear() - case Nil if indexState == IndexState.Synced - || indexState == IndexState.Refreshing - || indexState == IndexState.Empty => - // Empty state denotes the FS is empty, it is not cleaned up again to ensure its empty - // The file originally present "must" be available, which means no cleanup was done - assert(shardDirectory.list().exists(_.equals("empty"))) - case _ => - fail("Expected an index state Empty after directory cleanup") - } - } - } - - - it("Should update the state as empty after the cleanup is from a corrupt index") { - val events = ArrayBuffer.empty[(IndexState.Value, Long)] - IndexState.values.foreach { - indexState => - val indexDirectory = new File( - System.getProperty("java.io.tmpdir"), "part-key-lucene-index-event") - val shardDirectory = new File(indexDirectory, dataset6.ref + File.separator + "0") - // Delete directory to create an index from scratch - scala.reflect.io.Directory(shardDirectory).deleteRecursively() - shardDirectory.mkdirs() - // Validate the file named empty exists - val index = new PartKeyLuceneIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, - Some(indexDirectory),None) - // Add some index entries - val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, - "_ns_".utf8 -> "my_ns".utf8) - for { i <- 0 until 1000} { - val counterNum = i % 10 - val partKey = partBuilder.partKeyFromObjects(Schemas.promCounter, s"counter$counterNum", - seriesTags + ("instance".utf8 -> s"instance$i".utf8)) - index.addPartKey(partKeyOnHeap(Schemas.promCounter.partition.binSchema, ZeroPointer, partKey), i, 5)() - } - - index.closeIndex() - // Garble some index files to force a index corruption - // Just add some junk to the end of the segment files - - shardDirectory.listFiles(new FileFilter { - override def accept(pathname: File): Boolean = pathname.toString.contains("segment") - }).foreach( file => { - Files.writeString(file.toPath, "Hello", StandardOpenOption.APPEND) - }) - - new PartKeyLuceneIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, - Some(indexDirectory), - Some( new IndexMetadataStore { - def currentState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = - (indexState, None) - - def updateState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = { - assert(shardDirectory.list().length == 0) - events.append((state, time)) - } - - override def initState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = - currentState(datasetRef: DatasetRef, shard: Int) - - override def updateInitState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit - = { - - } - - })).closeIndex() - // For all states, including states where Index is Synced because the index is corrupt, - // the shard directory should be cleared and the new state should be Empty - events.toList match { - case (IndexState.Empty, _) :: Nil => - // The file originally present must not be available - assert(!shardDirectory.list().exists(_.equals("empty"))) - events.clear() - case _ => - fail("Expected an index state Empty after directory cleanup") - } - } - } - - it("should get a single match for part keys by a filter") { - - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, -1, i, i + 10)( - pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - val filter1 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1), 4, 10) - - partKeyOpt.isDefined shouldBe true - partKeyOpt.get shouldEqual pkrs(7).partKey - - val filter2 = ColumnFilter("Actor2Code", Equals("NonExist".utf8)) - keyIndex.singlePartKeyFromFilters(Seq(filter2), 4, 10) shouldBe None - } - it("toStringPairsMap should return a map of label key value pairs") { val columns = Seq("timestamp:ts", "min:double", "avg:double", "max:double", "count:long", "tags:map") val options = DatasetOptions.DefaultOptions.copy(metricColumn = "_metric_") @@ -910,153 +240,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } } - it("should match the regex after anchors stripped") { - for ((regex, regexNoAnchors) <- Map( - """^.*$""" -> """.*""", // both anchor are stripped. - """\$""" -> """\$""", // \$ is not removed. - """\\\$""" -> """\\\$""", // \$ is not removed. - """\\$""" -> """\\""", // $ is removed. - """$""" -> """""", // $ is removed. - """\^.*$""" -> """\^.*""", // do not remove \^. - """^ ^.*$""" -> """ ^.*""", // only remove the first ^. - """^.*\$""" -> """.*\$""", // do not remove \$ - """^ $foo""" -> """ $foo""", // the $ is not at the end, keep it. - """.* $ \ $$""" -> """.* $ \ $""", // only remove the last $ - """foo.*\\\ $""" -> """foo.*\\\ """, // remove $ for it at the end and not escaped. - """foo.*\\\$""" -> """foo.*\\\$""", // keep \$. - """foo.*\\$""" -> """foo.*\\""", // remove $ for it at the end and not escaped. - """foo.*$\\\\$""" -> """foo.*$\\\\""", // keep the first $ since it not at the end. - )) { - PartKeyLuceneIndex.removeRegexAnchors(regex) shouldEqual regexNoAnchors - } - } - - it("should get a single match for part keys through a field with empty value") { - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.slice(95, 96)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, -1, i, i + 10)( - pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - val filter1_found = ColumnFilter("Actor2Code", Equals("")) - val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1_found), 4, 10) - partKeyOpt.isDefined shouldBe true - partKeyOpt.get shouldEqual pkrs.head.partKey - - val filter2_found = ColumnFilter("Actor2Code", EqualsRegex("")) - val partKeyOpt2 = keyIndex.singlePartKeyFromFilters(Seq(filter2_found), 4, 10) - partKeyOpt2.isDefined shouldBe true - partKeyOpt2.get shouldEqual pkrs.head.partKey - - val filter3_found = ColumnFilter("Actor2Code", EqualsRegex("^$")) - val partKeyOpt3 = keyIndex.singlePartKeyFromFilters(Seq(filter3_found), 4, 10) - partKeyOpt3.isDefined shouldBe true - partKeyOpt3.get shouldEqual pkrs.head.partKey - } - - it("should get a single match for part keys through a non-existing field") { - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, -1, i, i + 10)( - pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - val filter1_found = ColumnFilter("NonExistingField", Equals("")) - val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1_found), 4, 10) - partKeyOpt.isDefined shouldBe true - partKeyOpt.get shouldEqual pkrs.head.partKey - - val filter2_found = ColumnFilter("NonExistingField", EqualsRegex("")) - val partKeyOpt2 = keyIndex.singlePartKeyFromFilters(Seq(filter2_found), 4, 10) - partKeyOpt2.isDefined shouldBe true - partKeyOpt2.get shouldEqual pkrs.head.partKey - - val filter3_found = ColumnFilter("NonExistingField", EqualsRegex("^$")) - val partKeyOpt3 = keyIndex.singlePartKeyFromFilters(Seq(filter3_found), 4, 10) - partKeyOpt3.isDefined shouldBe true - partKeyOpt3.get shouldEqual pkrs.head.partKey - } - - it("should get a single match for part keys by a regex filter") { - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, -1, i, i + 10)( - pk.length, PartKeyLuceneIndex.partKeyByteRefToSHA256Digest(pk, 0, pk.length)) - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - val filter1_found = ColumnFilter("Actor2Code", EqualsRegex("""^GO.*$""")) - val partKeyOpt = keyIndex.singlePartKeyFromFilters(Seq(filter1_found), 4, 10) - partKeyOpt.isDefined shouldBe true - partKeyOpt.get shouldEqual pkrs(7).partKey - - val filter1_not_found = ColumnFilter("Actor2Code", EqualsRegex("""^GO.*$\$""")) - keyIndex.singlePartKeyFromFilters(Seq(filter1_not_found), 4, 10) shouldBe None - - val filter2 = ColumnFilter("Actor2Code", NotEqualsRegex("^.*".utf8)) - keyIndex.singlePartKeyFromFilters(Seq(filter2), 4, 10) shouldBe None - } - - it("Should update the state as TriggerRebuild and throw an exception for any error other than CorruptIndexException") - { - val events = ArrayBuffer.empty[(IndexState.Value, Long)] - IndexState.values.foreach { - indexState => - val indexDirectory = new File( - System.getProperty("java.io.tmpdir"), "part-key-lucene-index-event") - val shardDirectory = new File(indexDirectory, dataset6.ref + File.separator + "0") - shardDirectory.mkdirs() - new File(shardDirectory, "empty").createNewFile() - // Make directory readonly to for an IOException when attempting to write - shardDirectory.setWritable(false) - // Validate the file named empty exists - assert(shardDirectory.list().exists(_.equals("empty"))) - try { - val index = new PartKeyLuceneIndex(dataset6.ref, dataset6.schema.partition, true, true,0, 1.hour.toMillis, - Some(indexDirectory), - Some( new IndexMetadataStore { - def currentState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = - (indexState, None) - - def updateState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = { - events.append((state, time)) - } - - override def initState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = - currentState(datasetRef: DatasetRef, shard: Int) - - override def updateInitState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long) - :Unit = { - - } - })) - index.closeIndex() - } catch { - case is: IllegalStateException => - assert(is.getMessage.equals("Unable to clean up index directory")) - events.toList match { - case (IndexState.TriggerRebuild, _) :: Nil => - // The file originally present would still be there as the directory - // is made readonly - assert(shardDirectory.list().exists(_.equals("empty"))) - events.clear() - case _ => - fail("Expected an index state Empty after directory cleanup") - } - } finally { - shardDirectory.setWritable(true) - } - } - } - it("should correctly build cardinality count") { // build card tracker val rocksDBStore = new RocksDbCardinalityStore(MachineMetricsData.dataset2.ref, 10) @@ -1098,60 +281,4 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte cardTracker.close() } - it("should match records without label when .* is provided on a non existent label") { - - val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) - .zipWithIndex.map { case (addr, i) => - val pk = partKeyOnHeap(dataset6.schema.partKeySchema, ZeroPointer, addr) - keyIndex.addPartKey(pk, i, i, i + 10)() - PartKeyLuceneIndexRecord(pk, i, i + 10) - } - keyIndex.refreshReadersBlocking() - - - // Query with just the existing Label name - val filter1 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result1 = keyIndex.partKeyRecordsFromFilters(Seq(filter1), 0, Long.MaxValue) - val expected1 = Seq(pkrs(7), pkrs(8), pkrs(9)) - - result1.map(_.partKey.toSeq) shouldEqual expected1.map(_.partKey.toSeq) - result1.map(p => (p.startTime, p.endTime)) shouldEqual expected1.map(p => (p.startTime, p.endTime)) - - // Query with non existent label name with an empty regex - val filter2 = ColumnFilter("dummy", EqualsRegex(".*".utf8)) - val filter3 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result2 = keyIndex.partKeyRecordsFromFilters(Seq(filter2, filter3), 0, Long.MaxValue) - val expected2 = Seq(pkrs(7), pkrs(8), pkrs(9)) - - result2.map(_.partKey.toSeq) shouldEqual expected2.map(_.partKey.toSeq) - result2.map(p => (p.startTime, p.endTime)) shouldEqual expected2.map(p => (p.startTime, p.endTime)) - - // Query with non existent label name with an regex matching at least 1 character - val filter4 = ColumnFilter("dummy", EqualsRegex(".+".utf8)) - val filter5 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result3 = keyIndex.partKeyRecordsFromFilters(Seq(filter4, filter5), 0, Long.MaxValue) - result3 shouldEqual Seq() - - // Query with non existent label name with an empty regex - val filter6 = ColumnFilter("dummy", EqualsRegex("".utf8)) - val filter7 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result4 = keyIndex.partKeyRecordsFromFilters(Seq(filter6, filter7), 0, Long.MaxValue) - val expected4 = Seq(pkrs(7), pkrs(8), pkrs(9)) - result4.map(_.partKey.toSeq) shouldEqual expected4.map(_.partKey.toSeq) - result4.map(p => (p.startTime, p.endTime)) shouldEqual expected4.map(p => (p.startTime, p.endTime)) - - // Query with non existent label name with an empty equals - val filter8 = ColumnFilter("dummy", Equals("".utf8)) - val filter9 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) - val result5 = keyIndex.partKeyRecordsFromFilters(Seq(filter8, filter9), 0, Long.MaxValue) - val expected5 = Seq(pkrs(7), pkrs(8), pkrs(9)) - result5.map(_.partKey.toSeq) shouldEqual expected5.map(_.partKey.toSeq) - result5.map(p => (p.startTime, p.endTime)) shouldEqual expected5.map(p => (p.startTime, p.endTime)) - - - val filter10 = ColumnFilter("Actor2Code", EqualsRegex(".*".utf8)) - val result10= keyIndex.partKeyRecordsFromFilters(Seq(filter10), 0, Long.MaxValue) - result10.map(_.partKey.toSeq) shouldEqual pkrs.map(_.partKey.toSeq) - result10.map(p => (p.startTime, p.endTime)) shouldEqual pkrs.map(p => (p.startTime, p.endTime)) - } } \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyQueryBuilderSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyQueryBuilderSpec.scala new file mode 100644 index 0000000000..26779a78fc --- /dev/null +++ b/core/src/test/scala/filodb.core/memstore/PartKeyQueryBuilderSpec.scala @@ -0,0 +1,27 @@ +package filodb.core.memstore + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +class PartKeyQueryBuilderSpec extends AnyFunSpec with Matchers { + it("should match the regex after anchors stripped") { + for ((regex, regexNoAnchors) <- Map( + """^.*$""" -> """.*""", // both anchor are stripped. + """\$""" -> """\$""", // \$ is not removed. + """\\\$""" -> """\\\$""", // \$ is not removed. + """\\$""" -> """\\""", // $ is removed. + """$""" -> """""", // $ is removed. + """\^.*$""" -> """\^.*""", // do not remove \^. + """^ ^.*$""" -> """ ^.*""", // only remove the first ^. + """^.*\$""" -> """.*\$""", // do not remove \$ + """^ $foo""" -> """ $foo""", // the $ is not at the end, keep it. + """.* $ \ $$""" -> """.* $ \ $""", // only remove the last $ + """foo.*\\\ $""" -> """foo.*\\\ """, // remove $ for it at the end and not escaped. + """foo.*\\\$""" -> """foo.*\\\$""", // keep \$. + """foo.*\\$""" -> """foo.*\\""", // remove $ for it at the end and not escaped. + """foo.*$\\\\$""" -> """foo.*$\\\\""", // keep the first $ since it not at the end. + )) { + PartKeyQueryBuilder.removeRegexAnchors(regex) shouldEqual regexNoAnchors + } + } +} diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyTantivyIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyTantivyIndexSpec.scala new file mode 100644 index 0000000000..5464cb3c91 --- /dev/null +++ b/core/src/test/scala/filodb.core/memstore/PartKeyTantivyIndexSpec.scala @@ -0,0 +1,145 @@ +package filodb.core.memstore + +import filodb.core.GdeltTestData.dataset6 +import filodb.core.{DatasetRef, TestData} +import filodb.core.binaryrecord2.RecordBuilder +import filodb.core.metadata.PartitionSchema +import filodb.core.query.ColumnFilter +import filodb.core.query.Filter.{Equals, EqualsRegex, In} +import org.scalatest.BeforeAndAfter +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers +import org.scalatest.time.SpanSugar.convertIntToGrainOfTime + +import java.io.File + +class PartKeyTantivyIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfter with PartKeyIndexRawSpec { + val keyIndex = new PartKeyTantivyIndex(dataset6.ref, dataset6.schema.partition, 0, 1.hour.toMillis, + Some(new File(System.getProperty("java.io.tmpdir"), "part-key-lucene-index"))) + + val partBuilder = new RecordBuilder(TestData.nativeMem) + + before { + keyIndex.reset() + keyIndex.refreshReadersBlocking() + } + + after { + partBuilder.removeAndFreeContainers(partBuilder.allContainers.length) + } + + protected def createNewIndex(ref: DatasetRef, + schema: PartitionSchema, + facetEnabledAllLabels: Boolean, + facetEnabledShardKeyLabels: Boolean, + shardNum: Int, + retentionMillis: Long, + diskLocation: Option[File], + lifecycleManager: Option[IndexMetadataStore]): PartKeyIndexRaw = { + new PartKeyTantivyIndex(ref, schema, shardNum, retentionMillis, diskLocation, lifecycleManager) + } + + it should behave like commonPartKeyTests(keyIndex, partBuilder) + + it("should encode equals queries correctly") { + val builder = new TantivyQueryBuilder() + + // Simple equals filter + val filters = List(ColumnFilter("col1", Equals("abcd"))) + val query = builder.buildQuery(filters) + + query should contain theSameElementsInOrderAs List(1,// Boolean + 1, // Must + 2, // Equals + 4, 0, // Length 4 + 99, 111, 108, 49, // col1 + 4, 0, // Length 4 + 97, 98, 99, 100, // abcd + 0) // End boolean + } + + it("should encode equals regex correctly") { + val builder = new TantivyQueryBuilder() + + // Simple equals filter + val filters = List(ColumnFilter("col1", EqualsRegex("a.*b"))) + val query = builder.buildQuery(filters) + + query should contain theSameElementsInOrderAs List(1,// Boolean + 1, // Must + 3, // Regex + 4, 0, // Length 4 + 99, 111, 108, 49, // col1 + 4, 0, // Length 4 + 97, 46, 42, 98, // a.*b + 0) // End boolean + } + + it("should encode term in correctly") { + val builder = new TantivyQueryBuilder() + + // Simple equals filter + val filters = List(ColumnFilter("col1", In(Set("a","b")))) + val query = builder.buildQuery(filters) + + query should contain theSameElementsInOrderAs List(1,// Boolean + 1, // Must + 4, // Term In + 4, 0, // Length 4 + 99, 111, 108, 49, // col1 + 2, 0, // Term count 2 + 1, 0, // Length 1 + 97, // a + 1, 0, // Length 1 + 98, // b + 0) // End boolean + } + + it("should encode prefix correctly") { + val builder = new TantivyQueryBuilder() + + // Simple equals filter + val filters = List(ColumnFilter("col1", EqualsRegex("a.*"))) + val query = builder.buildQuery(filters) + + query should contain theSameElementsInOrderAs List(1,// Boolean + 1, // Must + 5, // Prefix + 4, 0, // Length 4 + 99, 111, 108, 49, // col1 + 1, 0, // Length 1 + 97, // a + 0) // End boolean + } + + it("should encode match all correctly") { + val builder = new TantivyQueryBuilder() + + // Simple equals filter + val filters = List(ColumnFilter("col1", EqualsRegex(".*"))) + val query = builder.buildQuery(filters) + + query should contain theSameElementsInOrderAs List(1,// Boolean + 1, // Must + 6, // Match All + 0) // End boolean + } + + it("should encode start and end time properly") { + val builder = new TantivyQueryBuilder() + + // Simple equals filter + val filters = List(ColumnFilter("col1", EqualsRegex(".*"))) + val query = builder.buildQueryWithStartAndEnd(filters, 1, Long.MaxValue) + + query should contain theSameElementsInOrderAs List(1,// Boolean + 1, // Must + 6, // Match All + 1, // Must + 7, // Long Range + 11, 0, 95, 95, 101, 110, 100, 84, 105, 109, 101, 95, 95, // __endTime__ + 1, 0, 0, 0, 0, 0, 0, 0, // 0x1 + -1, -1, -1, -1, -1, -1, -1, 127, // Long.MAX_VALUE + 0) // End boolean + } +} diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index c9ca67e4d2..9903cea13b 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -394,6 +394,40 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } + it("should recover index data from col store ignoring bad schemas") { + val partBuilder = new RecordBuilder(TestData.nativeMem) + + val pkPtrs = GdeltTestData.partKeyFromRecords(dataset1, + records(dataset1, linearMultiSeries().take(2)), Some(partBuilder)) + val pks = pkPtrs.map(dataset1.partKeySchema.asByteArray(_)).toArray + + // Corrupt the schema of the second item + pks(1)(4) = 0xFF.toByte + pks(1)(5) = 0xFF.toByte + + val colStore = new NullColumnStore() { + override def scanPartKeys(ref: DatasetRef, shard: Int): Observable[PartKeyRecord] = { + val keys = Seq( + PartKeyRecord(pks(0), 50, Long.MaxValue, 0), + PartKeyRecord(pks(1), 250, Long.MaxValue, 0) + ) + Observable.fromIterable(keys) + } + } + + val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore()) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2, + diskTTLSeconds = 1.hour.toSeconds.toInt, + flushInterval = 10.minutes), 1) + Thread sleep 1000 + + val tsShard = memStore.asInstanceOf[TimeSeriesMemStore].getShard(dataset1.ref, 0).get + tsShard.recoverIndex().futureValue + + tsShard.partitions.size shouldEqual 1 // only 1, skipping the bad schema + tsShard.partKeyIndex.indexNumEntries shouldEqual 1 // only 1, skipping the bad schema + } + it("should lookupPartitions and return correct PartLookupResult") { memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf, 1) val data = records(dataset2, withMap(linearMultiSeries().take(20))) // 2 records per series x 10 series diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index 41b036bf0c..1129d3055b 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -1,18 +1,18 @@ package filodb.jmh -import java.lang.management.{BufferPoolMXBean, ManagementFactory} import java.util.concurrent.TimeUnit import scala.collection.mutable import scala.concurrent.duration._ import ch.qos.logback.classic.{Level, Logger} +import org.openjdk.jmh.annotations import org.openjdk.jmh.annotations._ import spire.syntax.cfor._ import filodb.core.DatasetRef import filodb.core.binaryrecord2.RecordBuilder -import filodb.core.memstore.PartKeyLuceneIndex +import filodb.core.memstore.{PartKeyIndexRaw, PartKeyLuceneIndex, PartKeyTantivyIndex} import filodb.core.metadata.Schemas import filodb.core.metadata.Schemas.untyped import filodb.core.query.{ColumnFilter, Filter} @@ -21,14 +21,16 @@ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} import filodb.timeseries.TestTimeseriesProducer // scalastyle:off -@State(Scope.Thread) -class PartKeyIndexBenchmark { +@State(Scope.Benchmark) +abstract class PartKeyIndexBenchmark { org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.ERROR) + protected def createPartKeyIndex(): PartKeyIndexRaw + println(s"Building Part Keys") val ref = DatasetRef("prometheus") - val partKeyIndex = new PartKeyLuceneIndex(ref, untyped.partition, true, true,0, 1.hour.toMillis) + val partKeyIndex = createPartKeyIndex() val numSeries = 1000000 val ingestBuilder = new RecordBuilder(MemFactory.onHeapFactory, RecordBuilder.DefaultContainerSize, false) val untypedData = TestTimeseriesProducer.timeSeriesData(0, numSeries, @@ -53,6 +55,20 @@ class PartKeyIndexBenchmark { } } + private var lookupTime = now + 1000 + + // Adjust the time range for every iteration. Without this everything ends up fully covered + // by query caching and you're only testing performance of the cache. + // + // In the real world it's very common to run the same query again and again but with a different time range + // - think cases like a live dashboard or alerting system. + @inline + protected def currentLookupTime(): Long = { + lookupTime += 1 + + lookupTime + } + val start = System.nanoTime() println(s"Indexing started") @@ -61,10 +77,8 @@ class PartKeyIndexBenchmark { val end = System.nanoTime() println(s"Indexing finished. Added $partId part keys Took ${(end-start)/1000000000L}s") - import scala.collection.JavaConverters._ - println(s"Index Memory Map Size: " + - s"${ManagementFactory.getPlatformMXBeans(classOf[BufferPoolMXBean]).asScala.find(_.getName == "mapped").get.getMemoryUsed}") + println(s"Index Memory Map Size: ${partKeyIndex.indexMmapBytes}") @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @@ -77,7 +91,7 @@ class PartKeyIndexBenchmark { ColumnFilter("host", Filter.Equals("H0")), ColumnFilter("_metric_", Filter.Equals("heap_usage0"))), now, - now + 1000) + currentLookupTime()) } } @@ -93,7 +107,7 @@ class PartKeyIndexBenchmark { ColumnFilter("host", Filter.Equals("H0")), ColumnFilter("_metric_", Filter.Equals("heap_usage0"))), now, - now + 1000) + currentLookupTime()) } } @@ -109,7 +123,7 @@ class PartKeyIndexBenchmark { ColumnFilter("_metric_", Filter.Equals("heap_usage0")), ColumnFilter("instance", Filter.EqualsRegex("Instance-2.*"))), now, - now + 1000) + currentLookupTime()) } } @@ -125,7 +139,7 @@ class PartKeyIndexBenchmark { ColumnFilter("_metric_", Filter.Equals("heap_usage0")), ColumnFilter("instance", Filter.EqualsRegex(".*2"))), now, - now + 1000) + currentLookupTime()) } } @@ -144,7 +158,7 @@ class PartKeyIndexBenchmark { "Instance-11|Instance-12|Instance-13|Instance-14|Instance-15|Instance-16|Instance-17|Instance-18|Instance-19|Instance-20|" + "Instance-21|Instance-22|Instance-23|Instance-24|Instance-25|Instance-26|Instance-27|Instance-28|Instance-29|Instance-30"))), now, - now + 1000).length + currentLookupTime()).length } } @@ -160,6 +174,38 @@ class PartKeyIndexBenchmark { } } + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + @OperationsPerInvocation(8) + def indexNames(): Unit = { + cforRange ( 0 until 8 ) { i => + partKeyIndex.indexNames(10000) + } + } + + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + @OperationsPerInvocation(8) + def indexValues(): Unit = { + cforRange ( 0 until 8 ) { i => + partKeyIndex.indexValues("instance", 10000) + } + } + + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + @OperationsPerInvocation(8) + def getLabelNames(): Unit = { + cforRange ( 0 until 8 ) { i => + val filter = Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), + ColumnFilter("_ws_", Filter.Equals("demo"))) + partKeyIndex.labelNamesEfficient(filter, now, currentLookupTime()) + } + } + @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) @@ -168,7 +214,7 @@ class PartKeyIndexBenchmark { cforRange ( 0 until 8 ) { i => val filter = Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), ColumnFilter("_ws_", Filter.Equals("demo"))) - partKeyIndex.labelValuesEfficient(filter, now, now + 1000, "_metric_", 10000) + partKeyIndex.labelValuesEfficient(filter, now, currentLookupTime(), "_metric_", 10000) } } @@ -181,11 +227,51 @@ class PartKeyIndexBenchmark { val filter = Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), ColumnFilter("_ws_", Filter.Equals("demo"))) val res = mutable.HashSet[ZeroCopyUTF8String]() - partKeyIndex.partIdsFromFilters(filter, now, now + 1000).foreach { pId => + partKeyIndex.partIdsFromFilters(filter, now, currentLookupTime()).foreach { pId => val pk = partKeyIndex.partKeyFromPartId(pId) Schemas.promCounter.partition.binSchema.singleColValues(pk.get.bytes, UnsafeUtils.arayOffset, "_metric_", res) } } } + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def partIdsLookupOverTime(): Unit = { + cforRange ( 0 until 8 ) { i => + partKeyIndex.partIdsFromFilters( + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-0")), + ColumnFilter("_ws_", Filter.Equals("demo")), + ColumnFilter("host", Filter.Equals("H0")), + ColumnFilter("_metric_", Filter.Equals("heap_usage0")), + ColumnFilter("instance", Filter.Equals("Instance-1"))), + now, + currentLookupTime()) + } + } + +} + +@State(Scope.Benchmark) +class PartKeyLuceneIndexBenchmark extends PartKeyIndexBenchmark { + override protected def createPartKeyIndex(): PartKeyIndexRaw = { + new PartKeyLuceneIndex(ref, untyped.partition, true, true, 0, 1.hour.toMillis) + } } + +@State(Scope.Benchmark) +class PartKeyTantivyIndexBenchmark extends PartKeyIndexBenchmark { + override protected def createPartKeyIndex(): PartKeyIndexRaw = { + PartKeyTantivyIndex.startMemoryProfiling() + + new PartKeyTantivyIndex(ref, untyped.partition, 0, 1.hour.toMillis) + } + + @TearDown(annotations.Level.Trial) + def teardown(): Unit = { + PartKeyTantivyIndex.stopMemoryProfiling() + val index = partKeyIndex.asInstanceOf[PartKeyTantivyIndex] + + println(s"\nCache stats:\n${index.dumpCacheStats()}\n") + } +} \ No newline at end of file diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexExternalBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexExternalBenchmark.scala new file mode 100644 index 0000000000..1f64af3064 --- /dev/null +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexExternalBenchmark.scala @@ -0,0 +1,215 @@ +package filodb.jmh + +import java.io.File +import java.util.Base64 +import java.util.concurrent.TimeUnit + +import scala.io.Source + +import ch.qos.logback.classic.{Level, Logger} +import org.openjdk.jmh.annotations.{Benchmark, BenchmarkMode, Mode, OutputTimeUnit, Scope, State, TearDown} +import org.openjdk.jmh.annotations +import org.scalatest.time.SpanSugar.convertIntToGrainOfTime + +import filodb.core.DatasetRef +import filodb.core.memstore.{IndexMetadataStore, IndexState, PartKeyIndexRaw, PartKeyLuceneIndex, PartKeyTantivyIndex} +import filodb.core.metadata.Schemas.untyped +import filodb.core.query.{ColumnFilter, Filter} + +/* + A benchmark that loads data from an external file full of part keys. + This is meant to be used with real world exported data to simulate / evaluate changes. + + The input file is specified as a file path and should be a csv with the following columns: + + * partKey (base64 encoded) + * startTime (long) + * endTime (long) + + A header row can be optionally included. + + To use this with a data set change the file path below and benchmark queries as needed. + */ +// scalastyle:off +@State(Scope.Benchmark) +abstract class PartKeyIndexExternalBenchmark { + // File path to load from + final private val inputPath = "partKeys.csv" + // File path to use for index storage + final protected val indexPath = "index/path" + + // Filters to create queries below + private def wsFilter = ColumnFilter("_ws_", Filter.Equals("myws")) + private def nsFilter = ColumnFilter("_ns_", Filter.Equals("myns")) + private def narrowFilter = ColumnFilter("hostname", Filter.Equals("example")) + + org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.ERROR) + + protected def computeIndexPath(): Option[File] = { + val file = new File(s"${indexPath}${this.getClass.getSimpleName}") + + if (!file.exists()) { + new File(s"${file.toPath}/${ref.dataset}/0").mkdirs() + } + + Some(file) + } + + protected def createPartKeyIndex(): PartKeyIndexRaw + + println(s"Building Part Keys") + val ref = DatasetRef("prometheus") + val partKeyIndex = createPartKeyIndex() + + var partId = 1 + + private def load_data(): Unit = { + val source = Source.fromFile(inputPath) + for (line <- source.getLines()) { + if (!line.startsWith("partkey")) { + val parts = line.split(',') + + val partKey = Base64.getDecoder.decode(parts(0)) + val startTime = parts(1).toLong + val endTime = parts(2).toLong + + partKeyIndex.addPartKey(partKey, partId, startTime, endTime)() + partId += 1 + } + } + source.close() + } + + if (partKeyIndex.indexNumEntries == 0) { + val start = System.nanoTime() + println(s"Indexing started at path ${partKeyIndex.indexDiskLocation}") + load_data() + partKeyIndex.refreshReadersBlocking() + val end = System.nanoTime() + + println(s"Indexing finished. Added $partId part keys Took ${(end - start) / 1000000000L}s") + } else { + partKeyIndex.refreshReadersBlocking() + println(s"Loaded existing index with ${partKeyIndex.indexNumEntries} part keys") + } + + @TearDown(annotations.Level.Trial) + def teardown2(): Unit = { + println(s"Ram usage after testing ${partKeyIndex.indexRamBytes}") + println(s"Mmap usage after testing ${partKeyIndex.indexMmapBytes}") + } + + private var lookupTime = 1 + + @inline + private def currentLookupTime(): Long = { + lookupTime += 1 + + lookupTime + } + + // Wide query - matches most documents + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def labelValuesWide(): Unit = { + partKeyIndex.labelValuesEfficient(Seq(wsFilter), + currentLookupTime(), Long.MaxValue, "_ns_") + } + + // Wide query - matches most documents + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def partIdsFromFiltersWide(): Unit = { + partKeyIndex.partIdsFromFilters(Seq(wsFilter, + nsFilter), + currentLookupTime(), Long.MaxValue, 10000) + } + + // Wide query - matches most documents + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def partKeysFromFiltersWide(): Unit = { + partKeyIndex.partKeyRecordsFromFilters(Seq(wsFilter, + nsFilter), + currentLookupTime(), Long.MaxValue, 100) + } + + // Narrow query - matches few (< 10) documents + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def labelValuesNarrow(): Unit = { + partKeyIndex.labelValuesEfficient(Seq(wsFilter, + nsFilter, + narrowFilter), + currentLookupTime(), Long.MaxValue, "pod") + } + + // Narrow query - matches few (< 10) documents + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def partIdsFromFiltersNarrow(): Unit = { + partKeyIndex.partIdsFromFilters(Seq(wsFilter, + nsFilter, + narrowFilter), + currentLookupTime(), Long.MaxValue, 10000) + } + + // Narrow query - matches few (< 10) documents + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def partKeysFromFiltersNarrow(): Unit = { + partKeyIndex.partKeyRecordsFromFilters(Seq(wsFilter, + nsFilter, + narrowFilter), + currentLookupTime(), Long.MaxValue, 100) + } +} + +@State(Scope.Benchmark) +class PartKeyLuceneIndexExternalBenchmark extends PartKeyIndexExternalBenchmark { + override protected def createPartKeyIndex(): PartKeyIndexRaw = { + new PartKeyLuceneIndex(ref, untyped.partition, true, true, 0, 1.hour.toMillis, diskLocation = computeIndexPath(), + lifecycleManager = Some(new MockLifecycleManager())) + } + + @TearDown(annotations.Level.Trial) + def teardown(): Unit = { + // This is needed to keep data consistent between runs with Lucene + partKeyIndex.closeIndex() + } +} + +@State(Scope.Benchmark) +class PartKeyTantivyIndexExternalBenchmark extends PartKeyIndexExternalBenchmark { + override protected def createPartKeyIndex(): PartKeyIndexRaw = { + PartKeyTantivyIndex.startMemoryProfiling() + + new PartKeyTantivyIndex(ref, untyped.partition, 0, 1.hour.toMillis, diskLocation = computeIndexPath(), + lifecycleManager = Some(new MockLifecycleManager())) + } + + @TearDown(annotations.Level.Trial) + def teardown(): Unit = { + PartKeyTantivyIndex.stopMemoryProfiling() + val index = partKeyIndex.asInstanceOf[PartKeyTantivyIndex] + + println(s"\nCache stats:\n${index.dumpCacheStats()}\n") + } +} + +class MockLifecycleManager extends IndexMetadataStore { + + override def initState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = (IndexState.Synced, None) + + override def currentState(datasetRef: DatasetRef, shard: Int): (IndexState.Value, Option[Long]) = (IndexState.Synced, None) + + override def updateState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = {} + + override def updateInitState(datasetRef: DatasetRef, shard: Int, state: IndexState.Value, time: Long): Unit = {} +} \ No newline at end of file diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexIngestionBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexIngestionBenchmark.scala new file mode 100644 index 0000000000..d4dccbad99 --- /dev/null +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexIngestionBenchmark.scala @@ -0,0 +1,87 @@ +package filodb.jmh + +import scala.collection.mutable.ArrayBuffer + +import org.openjdk.jmh.annotations.{Level, Param, Scope, Setup, State, TearDown} +import org.openjdk.jmh.annotations +import org.scalatest.time.SpanSugar.convertIntToGrainOfTime +import spire.syntax.cfor.cforRange + +import filodb.core.memstore.{PartKeyIndexRaw, PartKeyLuceneIndex, PartKeyTantivyIndex} +import filodb.core.metadata.Schemas.untyped +import filodb.memory.BinaryRegionConsumer + +// scalastyle:off +@State(Scope.Benchmark) +abstract class PartKeyIndexIngestionBenchmark extends PartKeyIndexBenchmark { + // How many part keys are added / removed per second + final val itemsPerSecond = 10 + // How often do we commit to disk / refresh readers + final val commitWindowInSeconds = 30 + + // 0 days, 1 day, 5 days, 30 days + @Param(Array("0", "1", "5", "30")) + var durationInDays: Long = _ + + @Setup(Level.Trial) + def setup(): Unit = { + println(s"Simulating $durationInDays days of churn") + val start = System.nanoTime() + val churnSteps = ((durationInDays * 24 * 60 * 60) / commitWindowInSeconds).toInt + var partId = 0 + val itemsPerStep = commitWindowInSeconds * itemsPerSecond + + val partKeys = new ArrayBuffer[Array[Byte]]() + val consumer = new BinaryRegionConsumer { + def onNext(base: Any, offset: Long): Unit = { + val partKey = untyped.partition.binSchema.asByteArray(base, offset) + partKeys += partKey + } + } + partKeyBuilder.allContainers.foreach(_.consumeRecords(consumer)) + + cforRange ( 0 until churnSteps ) { _ => + cforRange ( 0 until itemsPerStep ) { _ => + val partKey = partKeys(partId) + // When we ingested we used 1 based part IDs, not 0 based + partKeyIndex.upsertPartKey(partKey, partId+1, now)() + + partId += 1 + partId = partId % numSeries + } + } + + val end = System.nanoTime() + println(s"Finished ingesting new changes. Took ${(end-start)/1000000000L}s") + + partKeyIndex.refreshReadersBlocking() + val end2 = System.nanoTime() + println(s"Churning finished. Took ${(end2-start)/1000000000L}s") + println(s"New Index Memory Map Size: ${partKeyIndex.indexMmapBytes}") + println(s"Doc count: ${partKeyIndex.indexNumEntries}") + } +} + +@State(Scope.Benchmark) +class PartKeyLuceneIndexIngestionBenchmark extends PartKeyIndexIngestionBenchmark { + override protected def createPartKeyIndex(): PartKeyIndexRaw = { + new PartKeyLuceneIndex(ref, untyped.partition, true, true, 0, 1.hour.toMillis) + } +} + +@State(Scope.Benchmark) +class PartKeyTantivyIndexIngestionBenchmark extends PartKeyIndexIngestionBenchmark { + override protected def createPartKeyIndex(): PartKeyIndexRaw = { + PartKeyTantivyIndex.startMemoryProfiling() + + new PartKeyTantivyIndex(ref, untyped.partition, 0, 1.hour.toMillis) + } + + @TearDown(annotations.Level.Trial) + def teardown(): Unit = { + PartKeyTantivyIndex.stopMemoryProfiling() + val index = partKeyIndex.asInstanceOf[PartKeyTantivyIndex] + + println(s"\nCache stats:\n${index.dumpCacheStats()}\n") + } +} \ No newline at end of file diff --git a/project/Dependencies.scala b/project/Dependencies.scala index ef3d9afa2c..e81eddac8a 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -40,7 +40,8 @@ object Dependencies { val circeParser = "io.circe" %% "circe-parser" % "0.9.3" lazy val commonDeps = Seq( - "io.kamon" %% "kamon-bundle" % kamonBundleVersion, + "io.kamon" %% "kamon-bundle" % kamonBundleVersion, + "io.kamon" %% "kamon-testkit" % kamonBundleVersion % Test, logbackDep % Test, scalaTest % Test, "com.softwaremill.quicklens" %% "quicklens" % "1.4.12" % Test, @@ -80,7 +81,8 @@ object Dependencies { "com.esotericsoftware" % "kryo" % "4.0.0" excludeAll(excludeMinlog), "com.dorkbox" % "MinLog-SLF4J" % "1.12", "com.github.ben-manes.caffeine" % "caffeine" % "3.0.5", - "com.twitter" %% "chill" % "0.9.3" + "com.twitter" %% "chill" % "0.9.3", + "org.apache.commons" % "commons-lang3" % "3.14.0" ) lazy val sparkJobsDeps = commonDeps ++ Seq( diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 24cb7bb26e..fd5ca08f0f 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -1,5 +1,6 @@ import Dependencies._ import FiloSettings._ +import RustPlugin._ import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys.MultiJvm import io.gatling.sbt.GatlingPlugin import pl.project13.scala.sbt.JmhPlugin @@ -45,6 +46,7 @@ object Submodules { ) lazy val core = (project in file("core")) + .enablePlugins(RustPlugin) .dependsOn(memory % "compile->compile; test->test") .settings( commonSettings, diff --git a/project/RustPlugin.scala b/project/RustPlugin.scala new file mode 100644 index 0000000000..2ffc5ab631 --- /dev/null +++ b/project/RustPlugin.scala @@ -0,0 +1,251 @@ +import org.apache.commons.lang3._ +import sbt._ +import sbt.Keys._ +import sbt.io.Path._ +import sbt.nio.Keys._ +import scala.sys.process._ + +/* + Plugin that adds support to build native Rust code as part of a module. + This will build the code, include it in resources, and allow runtime loading. + */ +object RustPlugin extends AutoPlugin { + object autoImport { + // Tasks + val rustCompile = taskKey[Unit]("Compile rust code for this module.") + val rustClean = taskKey[Unit]("Clean rust build for this module.") + val rustGatherLibraries = taskKey[Seq[(File, String)]]("Gather the list of native libraries produced by the build.") + val rustLint = taskKey[Unit]("Run linting on rust code for this module.") + val rustTest = taskKey[Unit]("Test rust code for this module.") + + // Settings + val rustSourceDir = settingKey[File]("Path to base directory with rust code.") + val rustArchitectures = settingKey[Seq[String]]("List of architectures to build for. Takes either a Rust " + + "target tuple or the special key 'host' to build for the current machine. To supply multiple architectures " + + "separate them with a ';' character.") + val rustFeatures = settingKey[String]("Value to pass to cargo's --features option. Defaults to an empty string.") + var rustOptimize = settingKey[Boolean]("Enable optimization during rust builds. Defaults to false for host " + + "only builds and true for any other configuration.") + } + + import autoImport._ + + lazy val settings: Seq[Setting[_]] = Seq( + rustSourceDir := baseDirectory.value / "src" / "rust", + rustArchitectures := { + val archs = Option(System.getProperty("rust.architectures")).getOrElse("host") + + archs.split(';').toSeq + }, + rustFeatures := { + val features = Option(System.getProperty("rust.features")).getOrElse("") + + features + }, + rustOptimize := { + val optimize = Option(System.getProperty("rust.optimize")).getOrElse("true") + + optimize.toBoolean + }, + rustClean := { + val log = streams.value.log + val sourceDir = rustSourceDir.value + + log.info(s"Cleaning rust source at $sourceDir") + + val returnCode = Process(s"cargo clean", sourceDir) ! cargoLog(log) + + if (returnCode != 0) + sys.error(s"cargo clean failed with exit code $returnCode") + }, + rustCompile := { + val log = streams.value.log + val sourceDir = rustSourceDir.value + val features = rustFeatures.value + + for (archTarget <- rustArchitectures.value) { + log.info(s"Compiling rust source at $sourceDir for architecture $archTarget with features '$features'") + + // target setup + val targetCommand = if (archTarget == "host") { + "" + } else { + s"--target $archTarget" + } + + // Use build for the host target, zigbuild for everything else + val buildCommand = if (archTarget == "host") { + "build" + } else { + "zigbuild" + } + + // Check if release build + val buildFlags = if (rustOptimize.value) { + "--release" + } else { + "" + } + + val featureFlags = if (features.isBlank) { + "" + } else { + s" --features $features " + } + + val returnCode = Process(s"cargo $buildCommand $buildFlags $featureFlags $targetCommand", + sourceDir) ! cargoLog(log) + + if (returnCode != 0) + sys.error(s"cargo build failed with exit code $returnCode") + } + }, + rustGatherLibraries := { + val log = streams.value.log + var list: Seq[(File, String)] = Seq() + + // Compile first + rustCompile.value + + val release = rustOptimize.value + val releaseDir = if (release) { + "release" + } else { + "debug" + } + + val targetFolder = rustSourceDir.value / "target" + val fileTree = fileTreeView.value + + // For each architecture find artifacts + for (archTarget <- rustArchitectures.value) { + val normalizedArch = normalizeRustTarget(archTarget) + + // Special case - host + val archFolder = if (normalizedArch == "host") { + targetFolder / releaseDir + } else { + // General case + targetFolder / normalizedArch / releaseDir + } + + // get os arch / kernel, build path + val resourceArchTarget = mapRustTargetToJVMTarget(normalizedArch) + + // Find library files in folder + // We place every produced library in a resource path like + // /native/// + val glob = fileTree.list(Glob(archFolder) / "*.{so,dylib}").collect { + case (path, attributes) if attributes.isRegularFile => file(path.toString) + } + val files = glob.pair(rebase(archFolder, s"/native/$resourceArchTarget")) + + list = list ++ files + } + + list + }, + rustTest := { + val log = streams.value.log + val sourceDir = rustSourceDir.value + + val returnCode = Process(s"cargo test", sourceDir) ! cargoLog(log) + + returnCode match { + case 101 => sys.error("One or more tests failed") + case 0 => () + case x => sys.error(s"cargo test failed with exit code $x") + } + }, + resourceGenerators += Def.task { + val log = streams.value.log + + val libraries: Seq[(File, String)] = rustGatherLibraries.value + val resources: Seq[File] = for ((file, path) <- libraries) yield { + val resource = resourceManaged.value / path + + if (IO.getModifiedTimeOrZero(file) > IO.getModifiedTimeOrZero(resource)) { + IO.copyFile(file, resource, preserveLastModified = true) + } + resource + } + resources + }.taskValue + ) + + lazy val testSettings: Seq[Setting[_]] = Seq( + rustLint := { + val log = streams.value.log + val sourceDir = rustSourceDir.value + + val returnCode = Process(s"cargo clippy --all-targets -- -D warnings", sourceDir) ! cargoLog(log) + if (returnCode != 0) + sys.error(s"cargo clippy failed with exit code $returnCode") + }, + test := { + // Run rust tests and linting + rustLint.value + rustTest.value + // Run base test task + test.value + } + ) + + // Map an input Rust arch tuple to the correct target folder for JVM loading + private def mapRustTargetToJVMTarget(arch: String): String = { + // Rust tuples are basically clang tuples and look like: + // aarch64-apple-darwin + // x86_64-unknown-linux-gnu + // + // We want the architecture (first part) + // and the kernel part (third part) + val RustPattern = "([^-]+)-([^-]+)-([^-]+).*".r + arch match { + case "host" => s"$getHostKernel/${SystemUtils.OS_ARCH}" + // Java uses amd64, Rust uses x86_64 + case RustPattern("x86_64", _, kernel) => s"$kernel/amd64" + case RustPattern(arch, _, kernel) => s"$kernel/$arch" + case x => sys.error(s"Unsupported architecture $x") + } + } + + // Normalize a target string by stripping excess info, like GLIBC version + private def normalizeRustTarget(target: String): String = { + // Handle strings like x86_64-unknown-linux-gnu.2.17, + // which contain a GLIBC version suffix + // + // We want to drop the suffix to get x86_64-unknown-linux-gnu + val RustPattern = "([^-]+)-([^-]+)-([^.]+).*".r + + target match { + // Valid inputs + case "host" => target + case RustPattern(arch, vendor, kernel) => s"$arch-$vendor-$kernel" + // Not matched + case x => sys.error(s"Unsupported target $x") + } + } + + // Get normalized host kernel name + private def getHostKernel: String = { + if (SystemUtils.IS_OS_LINUX) { + "linux" + } else if (SystemUtils.IS_OS_MAC) { + "darwin" + } else if (SystemUtils.IS_OS_WINDOWS) { + "windows" + } else { + sys.error(s"Unhandled platform ${SystemUtils.OS_NAME}") + } + } + + // Cargo logs to both stdout and stderr with normal output + // Log all of these events as info + private def cargoLog(log: sbt.Logger): ProcessLogger = new ProcessLogger { + def out(s: => String): Unit = log.info(s) + def err(s: => String): Unit = log.info(s) + def buffer[T](f: => T): T = f + } + + override lazy val projectSettings = inConfig(Compile)(settings) ++ inConfig(Test)(settings ++ testSettings) +} diff --git a/project/build.sbt b/project/build.sbt new file mode 100644 index 0000000000..7373731fcb --- /dev/null +++ b/project/build.sbt @@ -0,0 +1,2 @@ +// Used by RustPlugin to look at current OS info +libraryDependencies += "org.apache.commons" % "commons-lang3" % "3.14.0" \ No newline at end of file diff --git a/project/plugins.sbt b/project/plugins.sbt index ddeb98941d..8bac19a9b6 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,8 +1,8 @@ addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "1.0.0") -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.5") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.9") -addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.7") +addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.4.7") addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.4.0") diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala index 473d135462..ac7f47596c 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala @@ -9,7 +9,7 @@ import filodb.prometheus.parse.Parser import filodb.query._ object Vectors { - val PromMetricLabel = "__name__" + val PromMetricLabel = GlobalConfig.PromMetricLabel val TypeLabel = Schemas.TypeLabel val BucketFilterLabel = "_bucket_" val conf = GlobalConfig.systemConfig diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala index 5cdcad8e38..684e6d9ee6 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala @@ -60,7 +60,7 @@ trait BaseParser extends JavaTokenParsers with RegexParsers with PackratParsers if (c == '\\') { val next = source.charAt(offset); offset += 1 c = next match { - case '\\' | '\'' | '"' => next + case '\\' | '\'' | '"' | '.' => next case 'f' => '\f' case 'n' => '\n' case 'r' => '\r' diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index f5189592da..6db3f3018f 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -221,7 +221,7 @@ object ParserUtil { if (c == '\\') { val next = str.charAt(offset); offset += 1 c = next match { - case '\\' | '\'' | '"' => next + case '\\' | '\'' | '"' | '.' => next case 'f' => '\f' case 'n' => '\n' case 'r' => '\r' diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 4eae7103c3..a7d15be2b9 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -705,6 +705,8 @@ class ParserSpec extends AnyFunSpec with Matchers { "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(method,Equals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),Some(300000),false),1524855988000,1000000,1524855988000,Some(300000),None),List(),None)", """{__name__="foo\\\"\n\t",job="myjob"}[5m]""" -> "RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo\\\"\n\t)), ColumnFilter(job,Equals(myjob))),List(),Some(300000),None,false)", + """{__name__="foo",job="myjob\."}[5m]""" -> + "RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob.))),List(),Some(300000),None,false)", "{__name__=\"foo\",job=\"myjob\"}" -> "PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob))),List(),Some(300000),None,false),1524855988000,1000000,1524855988000,None,None)", "{__name__=\"foo\",job=\"myjob\"}[5m]" -> diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 1962fdb2d8..2e6c561494 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -1,7 +1,9 @@ package filodb.query +import filodb.core.GlobalConfig import filodb.core.query.{ColumnFilter, RangeParams, RvRange} import filodb.core.query.Filter.Equals +import filodb.query.util.{HierarchicalQueryExperience, HierarchicalQueryExperienceParams} //scalastyle:off number.of.types //scalastyle:off file.size.limit @@ -34,6 +36,21 @@ sealed trait LogicalPlan { case c: TsCardinalities => c // immutable & no members need to be updated } } + + /** + * Optimize the logical plan by using the higher level aggregated metric if applicable + * @param params AggRule object - contains details of the higher level aggregation rule and metric + * @return Updated LogicalPlan if Applicable. Else return the same LogicalPlan + */ + def useHigherLevelAggregatedMetric(params: HierarchicalQueryExperienceParams): LogicalPlan = { + // For now, only PeriodicSeriesPlan and RawSeriesLikePlan are optimized for higher level aggregation + this match { + // We start with no parent plans from the root + case p: PeriodicSeriesPlan => p.useAggregatedMetricIfApplicable(params, Seq()) + case r: RawSeriesLikePlan => r.useAggregatedMetricIfApplicable(params, Seq()) + case _ => this + } + } } /** @@ -44,6 +61,26 @@ sealed trait LogicalPlan { sealed trait RawSeriesLikePlan extends LogicalPlan { def isRaw: Boolean = false def replaceRawSeriesFilters(newFilters: Seq[ColumnFilter]): RawSeriesLikePlan + + /** + * Updates the `metric` ColumnFilter if the raw ColumnFilters satisfy the constraints of the + * higher level aggregation rule. + * + * @param params AggRule object - contains details of the higher level aggregation rule and metric + * @param parentLogicalPlans Seq of parent logical plans name in the query plan tree. + * For example: + * 1. query - sum(rate(foo{}[5m])) + sum(rate(bar{}[5m])) + * 2. parentLogicalPlans when in RawSeriesLikePlan will be: + * Seq(BinaryJoin, Aggregate, PeriodicSeriesWithWindowing) + * @return Updated logical plan if optimized for higher level aggregation. Else return the same logical plan + */ + def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): RawSeriesLikePlan + + /** + * @return Raw series column filters + */ + def rawSeriesFilters(): Seq[ColumnFilter] } sealed trait NonLeafLogicalPlan extends LogicalPlan { @@ -70,6 +107,22 @@ sealed trait PeriodicSeriesPlan extends LogicalPlan { def endMs: Long def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan + + /** + * This function traverses down the logical plan tree to the raw series leaf level and check if the + * aggregation clauses and the column filters satisfies the constraints of the higher level aggregation rule. If so, + * we update the raw series metric column filter with the higher level aggregation metric to reduce number of + * timeseries to be scanned and reduce the query latency. + * @param params AggRule object - contains details of the higher level aggregation rule and metric + * @param parentLogicalPlans Seq of parent logical plans name in the query plan tree. + * For example: + * 1. query - sum(rate(foo{}[5m])) + sum(rate(bar{}[5m])) + * 2. parentLogicalPlans when execution in PeriodicSeriesWithWindowing will be: + * Seq(BinaryJoin, Aggregate) + * @return Updated logical plan if optimized for higher level aggregation. Else return the same logical plan + */ + def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan } sealed trait MetadataQueryPlan extends LogicalPlan { @@ -121,6 +174,34 @@ case class RawSeries(rangeSelector: RangeSelector, val updatedFilters = this.filters.filterNot(f => filterColumns.contains(f.column)) ++ newFilters this.copy(filters = updatedFilters) } + + /** + * Updates the metric ColumnFilter if the higher level aggregation rule is applicable + * @param params AggRule object - contains details of the higher level aggregation rule and metric + * @return Updated RawSeriesLikePlan if Applicable. Else return the same RawSeriesLikePlan + */ + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): RawSeriesLikePlan = { + // Example leaf periodic series plans which has access to raw series - PeriodicSeries, PeriodicSeriesWithWindowing, + // ApplyInstantFunctionRaw. This can be configured as required. + // Only the last plan is checked here for the config. The higher level parent plans is checked in the + // PeriodicSeries and PeriodicSeriesWithWindowing plans + val leafPeriodicSeriesPlan = parentLogicalPlans.lastOption + leafPeriodicSeriesPlan match { + case Some(leafPeriodicPlan) => + // Check 1: Check if the leaf periodic series plan is allowed for raw series update + HierarchicalQueryExperience.isLeafPeriodicSeriesPlanAllowedForRawSeriesUpdate(leafPeriodicPlan) match { + case true => + val updatedFilters = HierarchicalQueryExperience.upsertMetricColumnFilterIfHigherLevelAggregationApplicable( + params, filters) + this.copy(filters = updatedFilters) + case false => this + } + case None => this + } + } + + override def rawSeriesFilters(): Seq[ColumnFilter] = filters } case class LabelValues(labelNames: Seq[String], @@ -145,7 +226,8 @@ case class SeriesKeysByFilters(filters: Seq[ColumnFilter], object TsCardinalities { val LABEL_WORKSPACE = "_ws_" - val SHARD_KEY_LABELS = Seq(LABEL_WORKSPACE, "_ns_", "__name__") + val LABEL_NAMESPACE = "_ns_" + val SHARD_KEY_LABELS = Seq(LABEL_WORKSPACE, LABEL_NAMESPACE, GlobalConfig.PromMetricLabel) } /** @@ -238,6 +320,12 @@ case class RawChunkMeta(rangeSelector: RangeSelector, val updatedFilters = this.filters.filterNot(f => filterColumns.contains(f.column)) ++ newFilters this.copy(filters = updatedFilters) } + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // RawChunkMeta queries are not optimized for higher level aggregation + this + } } /** @@ -261,6 +349,17 @@ case class PeriodicSeries(rawSeries: RawSeriesLikePlan, override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(rawSeries = rawSeries.replaceRawSeriesFilters(filters)) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // Check 1: Check if the parent logical plans are allowed for hierarchical aggregation update + HierarchicalQueryExperience.isParentPeriodicSeriesPlanAllowed(parentLogicalPlans) match { + case true => + this.copy(rawSeries = rawSeries.useAggregatedMetricIfApplicable(params, + parentLogicalPlans :+ this.getClass.getSimpleName)) + case false => this + } + } } /** @@ -323,6 +422,13 @@ case class SubqueryWithWindowing( val updatedFunctionArgs = functionArgs.map(_.replacePeriodicSeriesFilters(filters).asInstanceOf[FunctionArgsPlan]) this.copy(innerPeriodicSeries = updatedInnerPeriodicSeries, functionArgs = updatedFunctionArgs) } + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // recurse to the leaf level + this.copy(innerPeriodicSeries = innerPeriodicSeries.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName)) + } } /** @@ -361,6 +467,13 @@ case class TopLevelSubquery( val updatedInnerPeriodicSeries = innerPeriodicSeries.replacePeriodicSeriesFilters(filters) this.copy(innerPeriodicSeries = updatedInnerPeriodicSeries) } + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // recurse to the leaf level + this.copy(innerPeriodicSeries = innerPeriodicSeries.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName)) + } } /** @@ -390,6 +503,23 @@ case class PeriodicSeriesWithWindowing(series: RawSeriesLikePlan, this.copy(columnFilters = LogicalPlan.overrideColumnFilters(columnFilters, filters), series = series.replaceRawSeriesFilters(filters), functionArgs = functionArgs.map(_.replacePeriodicSeriesFilters(filters).asInstanceOf[FunctionArgsPlan])) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // Checks: + // 1. Check if the range function is allowed + // 2. Check if any of the parent plan is allowed for raw series update + HierarchicalQueryExperience.isRangeFunctionAllowed(function.entryName) && + HierarchicalQueryExperience.isParentPeriodicSeriesPlanAllowed(parentLogicalPlans) match { + case true => + val newRawSeries = series.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName) + this.copy( + columnFilters = LogicalPlan.overrideColumnFilters(columnFilters, newRawSeries.rawSeriesFilters()), + series = newRawSeries) + case false => this + } + } } /** @@ -438,6 +568,20 @@ case class Aggregate(operator: AggregationOperator, override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = vectors.replacePeriodicSeriesFilters(filters)) + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // Modify the map to retain all the AggRules which satisfies the current Aggregate clause labels. + val updatedMap = params.aggRules.filter(x => HierarchicalQueryExperience + .checkAggregateQueryEligibleForHigherLevelAggregatedMetric(x._2, operator, clauseOpt)) + if (updatedMap.isEmpty) { + // none of the aggregation rules matched with the + this + } else { + val updatedParams = params.copy(aggRules = updatedMap) + this.copy(vectors = vectors.useAggregatedMetricIfApplicable( + updatedParams, parentLogicalPlans :+ this.getClass.getSimpleName)) + } + } } /** @@ -468,6 +612,17 @@ case class BinaryJoin(lhs: PeriodicSeriesPlan, override def isRoutable: Boolean = lhs.isRoutable || rhs.isRoutable override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(lhs = lhs.replacePeriodicSeriesFilters(filters), rhs = rhs.replacePeriodicSeriesFilters(filters)) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // No special handling for BinaryJoin. Just pass the call to lhs and rhs recursively + this.copy( + lhs = lhs.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName), + rhs = rhs.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName) + ) + } } /** @@ -485,6 +640,17 @@ case class ScalarVectorBinaryOperation(operator: BinaryOperator, override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vector = vector.replacePeriodicSeriesFilters(filters), scalarArg = scalarArg.replacePeriodicSeriesFilters(filters).asInstanceOf[ScalarPlan]) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // No special handling for ScalarVectorBinaryOperation. Just pass the call to vector and and scalar plan recursively + val parentLogicalPlansUpdated = parentLogicalPlans :+ this.getClass.getSimpleName + this.copy( + vector = vector.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated), + scalarArg = scalarArg.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated).asInstanceOf[ScalarPlan]) + } } /** @@ -503,6 +669,17 @@ case class ApplyInstantFunction(vectors: PeriodicSeriesPlan, override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy( vectors = vectors.replacePeriodicSeriesFilters(filters), functionArgs = functionArgs.map(_.replacePeriodicSeriesFilters(filters).asInstanceOf[FunctionArgsPlan])) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + // No special handling for ApplyInstantFunction. Just pass the call to vectors and functionArgs recursively + val parentLogicalPlansUpdated = parentLogicalPlans :+ this.getClass.getSimpleName + this.copy( + vectors = vectors.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated), + functionArgs = functionArgs.map(_.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated).asInstanceOf[FunctionArgsPlan])) + } } /** @@ -518,6 +695,18 @@ case class ApplyInstantFunctionRaw(vectors: RawSeries, vectors = vectors.replaceRawSeriesFilters(newFilters).asInstanceOf[RawSeries], functionArgs = functionArgs.map(_.replacePeriodicSeriesFilters(newFilters).asInstanceOf[FunctionArgsPlan])) + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): RawSeriesLikePlan = { + val parentLogicalPlansUpdated = parentLogicalPlans :+ this.getClass.getSimpleName + this.copy( + vectors = vectors.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated).asInstanceOf[RawSeries], + functionArgs = functionArgs.map(_.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated).asInstanceOf[FunctionArgsPlan]) + ) + } + + override def rawSeriesFilters(): Seq[ColumnFilter] = vectors.rawSeriesFilters() } /** @@ -533,6 +722,12 @@ case class ApplyMiscellaneousFunction(vectors: PeriodicSeriesPlan, override def endMs: Long = vectors.endMs override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = vectors.replacePeriodicSeriesFilters(filters)) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + this.copy(vectors = vectors.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName)) + } } /** @@ -546,6 +741,12 @@ case class ApplySortFunction(vectors: PeriodicSeriesPlan, override def endMs: Long = vectors.endMs override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = vectors.replacePeriodicSeriesFilters(filters)) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + this.copy(vectors = vectors.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName)) + } } /** @@ -576,6 +777,16 @@ final case class ScalarVaryingDoublePlan(vectors: PeriodicSeriesPlan, override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy( vectors = vectors.replacePeriodicSeriesFilters(filters), functionArgs = functionArgs.map(_.replacePeriodicSeriesFilters(filters).asInstanceOf[FunctionArgsPlan])) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + val parentLogicalPlansUpdated = parentLogicalPlans :+ this.getClass.getSimpleName + this.copy( + vectors = vectors.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated), + functionArgs = functionArgs.map(_.useAggregatedMetricIfApplicable( + params, parentLogicalPlansUpdated).asInstanceOf[FunctionArgsPlan])) + } } /** @@ -588,6 +799,10 @@ final case class ScalarTimeBasedPlan(function: ScalarFunctionId, rangeParams: Ra override def stepMs: Long = rangeParams.stepSecs * 1000 override def endMs: Long = rangeParams.endSecs * 1000 override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this // No Filter + + // No optimization for the scalar plan without raw or periodic series + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = this } /** @@ -602,6 +817,10 @@ final case class ScalarFixedDoublePlan(scalar: Double, override def stepMs: Long = timeStepParams.stepSecs * 1000 override def endMs: Long = timeStepParams.endSecs * 1000 override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this + + // No optimization for the scalar plan without raw or periodic series + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = this } //scalastyle:off number.of.types @@ -617,6 +836,13 @@ final case class VectorPlan(scalars: ScalarPlan) extends PeriodicSeriesPlan with override def isRoutable: Boolean = scalars.isRoutable override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(scalars = scalars.replacePeriodicSeriesFilters(filters).asInstanceOf[ScalarPlan]) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + this.copy( + scalars = scalars.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName).asInstanceOf[ScalarPlan]) + } } /** @@ -637,6 +863,17 @@ case class ScalarBinaryOperation(operator: BinaryOperator, asInstanceOf[ScalarBinaryOperation]) else Left(rhs.left.get) this.copy(lhs = updatedLhs, rhs = updatedRhs) } + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + val parentLogicalPlansUpdated = parentLogicalPlans :+ this.getClass.getSimpleName + val updatedLhs = if (lhs.isRight) Right(lhs.right.get.useAggregatedMetricIfApplicable(params, + parentLogicalPlansUpdated).asInstanceOf[ScalarBinaryOperation]) else Left(lhs.left.get) + + val updatedRhs = if (rhs.isRight) Right(rhs.right.get.useAggregatedMetricIfApplicable(params, + parentLogicalPlansUpdated).asInstanceOf[ScalarBinaryOperation]) else Left(rhs.left.get) + this.copy(lhs = updatedLhs, rhs = updatedRhs) + } } /** @@ -653,6 +890,12 @@ case class ApplyAbsentFunction(vectors: PeriodicSeriesPlan, override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(columnFilters = LogicalPlan.overrideColumnFilters(columnFilters, filters), vectors = vectors.replacePeriodicSeriesFilters(filters)) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + this.copy(vectors = vectors.useAggregatedMetricIfApplicable(params, + parentLogicalPlans :+ this.getClass.getSimpleName)) + } } /** @@ -669,6 +912,12 @@ case class ApplyLimitFunction(vectors: PeriodicSeriesPlan, override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(columnFilters = LogicalPlan.overrideColumnFilters(columnFilters, filters), vectors = vectors.replacePeriodicSeriesFilters(filters)) + + override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, + parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { + this.copy(vectors = vectors.useAggregatedMetricIfApplicable( + params, parentLogicalPlans :+ this.getClass.getSimpleName)) + } } object LogicalPlan { @@ -709,7 +958,6 @@ object LogicalPlan { } } - def getColumnValues(logicalPlan: LogicalPlan, labelName: String): Set[String] = { getColumnValues(getColumnFilterGroup(logicalPlan), labelName) } @@ -732,6 +980,16 @@ object LogicalPlan { }) } + def getColumnValues(columnFilters: Seq[ColumnFilter], labelName: String): Seq[String] = { + columnFilters.flatMap(cFilter => { + if (cFilter.column == labelName) { + cFilter.filter.valuesStrings.map(_.toString) + } else { + Seq.empty + } + }) + } + /** * Given a LogicalPlan, the function finds a Seq of all Child nodes, and returns a Set of ColumnFilters for * each of the Leaf node diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index 9cb38dc997..70c60ec639 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -9,7 +9,6 @@ import filodb.core.query.{ColumnFilter, QueryConfig, QueryContext, QuerySession, import filodb.core.query.Filter.Equals import filodb.core.store._ import filodb.query.Query.qLogger -import filodb.query.TsCardinalities final case class UnknownSchemaQueryErr(id: Int) extends Exception(s"Unknown schema ID $id during query. This likely means a schema config change happened and " + @@ -58,14 +57,15 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, } /** - * @param ws workspace name - * @return true if the config is defined AND ws is not in the list of disabled workspaces. false otherwise + * @param maxMinTenantValue tenant value string such as workspace, app etc + * @return true if the config is defined AND maxMinTenantValue is not in the list of disabled workspaces. + * false otherwise */ - def isMaxMinEnabledForWorkspace(ws: Option[String]) : Boolean = { - ws.isDefined match { + def isMaxMinColumnsEnabled(maxMinTenantValue: Option[String]) : Boolean = { + maxMinTenantValue.isDefined match { // we are making sure that the config is defined to avoid any accidental "turn on" of the feature when not desired case true => (GlobalConfig.workspacesDisabledForMaxMin.isDefined) && - (!GlobalConfig.workspacesDisabledForMaxMin.get.contains(ws.get)) + (!GlobalConfig.workspacesDisabledForMaxMin.get.contains(maxMinTenantValue.get)) case false => false } } @@ -77,7 +77,8 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, Kamon.currentSpan().mark("filtered-partition-scan") var lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod, querySession) val metricName = filters.find(_.column == metricColumn).map(_.filter.valuesStrings.head.toString) - val ws = filters.find(x => x.column == TsCardinalities.LABEL_WORKSPACE && x.filter.isInstanceOf[Equals]) + val maxMinTenantFilter = filters + .find(x => x.column == GlobalConfig.maxMinTenantColumnFilter && x.filter.isInstanceOf[Equals]) .map(_.filter.valuesStrings.head.toString) var newColName = colName @@ -116,14 +117,14 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, // This code is responsible for putting exact IDs needed by any range functions. val colIDs1 = getColumnIDs(sch, newColName.toSeq, rangeVectorTransformers) - val colIDs = isMaxMinEnabledForWorkspace(ws) match { + val colIDs = isMaxMinColumnsEnabled(maxMinTenantFilter) match { case true => addIDsForHistMaxMin(sch, colIDs1) case _ => colIDs1 } // Modify transformers as needed for histogram w/ max, downsample, other schemas val newxformers1 = newXFormersForDownsample(sch, rangeVectorTransformers) - val newxformers = isMaxMinEnabledForWorkspace(ws) match { + val newxformers = isMaxMinColumnsEnabled(maxMinTenantFilter) match { case true => newXFormersForHistMaxMin(sch, colIDs, newxformers1) case _ => newxformers1 } diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala index 7027d5a43c..2d156307a9 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala @@ -173,6 +173,10 @@ abstract class ChunkedRateFunctionBase extends CounterChunkedRangeFunction[Trans startRowNum: Int, endRowNum: Int, startTime: Long, endTime: Long): Unit = { val dblReader = reader.asDoubleReader + // if the chunk is a single row NaN value, then return. Prometheus end of time series marker. + // This is to make sure we don't set highestValue to zero. avoids negative rate/increase values. + if (startRowNum == 0 && endRowNum == 0 && dblReader.apply(acc, vector, startRowNum).isNaN) + return if (startTime < lowestTime || endTime > highestTime) { numSamples += endRowNum - startRowNum + 1 if (startTime < lowestTime) { diff --git a/query/src/main/scala/filodb/query/util/HierarchicalQueryExperience.scala b/query/src/main/scala/filodb/query/util/HierarchicalQueryExperience.scala new file mode 100644 index 0000000000..5b089fa061 --- /dev/null +++ b/query/src/main/scala/filodb/query/util/HierarchicalQueryExperience.scala @@ -0,0 +1,343 @@ +package filodb.query.util + +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon +import scala.jdk.CollectionConverters.asScalaBufferConverter + +import filodb.core.GlobalConfig +import filodb.core.query.ColumnFilter +import filodb.core.query.Filter.Equals +import filodb.query.{AggregateClause, AggregationOperator, LogicalPlan, TsCardinalities} + +/** + * Aggregation rule definition. Contains the following information: + * 1. aggregation metricDelimiter to be matched + * 2. map of current aggregation metric suffix -> nextLevelAggregation's AggRule to be used + * For example: agg -> AggRule { metricSuffix = agg_2, tags = Set("tag1", "tag2") } + */ +case class HierarchicalQueryExperienceParams(metricDelimiter: String, + aggRules: Map[String, AggRule]) { } + +/** + * Aggregation rule definition. Contains the following information: + * 1. metric suffix for the given aggregation rule + * 2. include/exclude tags for the given aggregation rule + */ +sealed trait AggRule { + + val metricSuffix: String + + val tags: Set[String] + def isHigherLevelAggregationApplicable(shardKeyColumns: Set[String], filterTags: Seq[String]): Boolean +} + +/** + * @param metricSuffix - String - Metric suffix for the given aggregation rule + * @param tags - Set[String] - Include tags as specified in the aggregation rule + */ +case class IncludeAggRule(metricSuffix: String, tags: Set[String]) extends AggRule { + + /** + * Checks if the higher level aggregation is applicable with IncludeTags. + * + * @param shardKeyColumns - Seq[String] - List of shard key columns. These columns are not part of check. This + * include tags which are compulsory for the query like _metric_, _ws_, _ns_. + * @param filterTags - Seq[String] - List of filter tags/labels in the query or in the aggregation clause + * @return - Boolean + */ + override def isHigherLevelAggregationApplicable(shardKeyColumns: Set[String], filterTags: Seq[String]): Boolean = { + filterTags.forall( tag => shardKeyColumns.contains(tag) || tags.contains(tag)) + } +} + +/** + * @param metricSuffix - String - Metric suffix for the given aggregation rule + * @param tags - Set[String] - Exclude tags as specified in the aggregation rule + */ +case class ExcludeAggRule(metricSuffix: String, tags: Set[String]) extends AggRule { + + /** + * Checks if the higher level aggregation is applicable with ExcludeTags. Here we need to check if the column filter + * tags present in query or aggregation clause, should not be a part of ExcludeTags. + * + * @param shardKeyColumns - Seq[String] - List of shard key columns. These columns are not part of check. This + * include tags which are compulsory for the query like _metric_, _ws_, _ns_. + * @param filterTags - Seq[String] - List of filter tags/labels in the query or in the aggregation clause + * @return - Boolean + */ + override def isHigherLevelAggregationApplicable(shardKeyColumns: Set[String], filterTags: Seq[String]): Boolean = { + filterTags.forall { tag => shardKeyColumns.contains(tag) || (!tags.contains(tag)) } + } +} + +object HierarchicalQueryExperience extends StrictLogging { + + private val hierarchicalQueryOptimizedCounter = Kamon.counter("hierarchical-query-plans-optimized") + + // Get the shard key columns from the dataset options along with all the metric labels used + private lazy val shardKeyColumnsOption: Option[Set[String]] = GlobalConfig.datasetOptions match { + case Some(datasetOptions) => + Some((datasetOptions.shardKeyColumns ++ Seq( datasetOptions.metricColumn, GlobalConfig.PromMetricLabel)).toSet) + case None => None + } + + // Get the allowed aggregation operators from the hierarchical config + private lazy val allowedAggregationOperators: Option[Set[String]] = GlobalConfig.hierarchicalConfig match { + case Some(hierarchicalConfig) => + Some(hierarchicalConfig.getStringList("allowed-aggregation-operators").asScala.toSet) + case None => None + } + + // Get the allowed range functions from the hierarchical config + private lazy val allowedRangeFunctions: Option[Set[String]] = GlobalConfig.hierarchicalConfig match { + case Some(hierarchicalConfig) => + Some(hierarchicalConfig.getStringList("allowed-range-functions").asScala.toSet) + case None => None + } + + // Get the allowed periodic series plans which have access to RawSeries from the hierarchical config + private lazy val allowedPeriodicSeriesPlansWithRawSeries: Option[Set[String]] = + GlobalConfig.hierarchicalConfig match { + case Some(hierarchicalConfig) => + Some(hierarchicalConfig.getStringList("allowed-periodic-series-plans-with-raw-series").asScala.toSet) + case None => None + } + + // Get the allowed parent logical plans for optimization from the hierarchical config + private lazy val allowedLogicalPlansForOptimization: Option[Set[String]] = GlobalConfig.hierarchicalConfig match { + case Some(hierarchicalConfig) => + Some(hierarchicalConfig.getStringList("allowed-parent-logical-plans").asScala.toSet) + case None => None + } + + /** + * Helper function to get the ColumnFilter tag/label for the metric. This is needed to correctly update the filter. + * @param filterTags - Seq[String] - List of ColumnFilter tags/labels + * @param datasetMetricColumn - String - Metric ColumnFilter tag/label from the configured dataset options + * @return - String - ColumnFilter tag/label for the metric + */ + def getMetricColumnFilterTag(filterTags: Seq[String], datasetMetricColumn: String): String = { + // get metric name filter i.e either datasetOptions.get.metricColumn or PromMetricLabel - We need to support both + // the cases + filterTags.find( tag => tag == datasetMetricColumn || tag == GlobalConfig.PromMetricLabel) + .getOrElse(datasetMetricColumn) + } + + /** + * Helper function to update the filters with new filters. Example: + * filters = Seq( + * ColumnFilter("tag1", Equals("value1")), + * ColumnFilter("tag2", Equals("value2"))) + * + * newFilters = Seq( + * ColumnFilter("tag2", Equals("value2Updated")), + * ColumnFilter("tag4", Equals("value4"))) + * + * Updated filters = Seq( + * ColumnFilter("tag1", Equals("value1")), + * ColumnFilter("tag2", Equals("value2Updated")), + * ColumnFilter("tag4", Equals("value4"))) + * + * @param filters - Seq[ColumnFilter] - Existing filters + * @param newFilters - Seq[ColumnFilter] - New filters to be added/updated + * @return - Seq[ColumnFilter] - Updated filters + */ + private def upsertFilters(filters: Seq[ColumnFilter], newFilters: Seq[ColumnFilter]): Seq[ColumnFilter] = { + val filterColumns = newFilters.map(_.column) + val updatedFilters = filters.filterNot(f => filterColumns.contains(f.column)) ++ newFilters + updatedFilters + } + + /** Checks if the higher level aggregation is applicable for the given Include/Exclude tags. + * @param params - AggRule - Include or Exclude AggRule + * @param filterTags - Seq[String] - List of filter tags/labels in the query or in the aggregation clause + * @return - Boolean + */ + def isHigherLevelAggregationApplicable(params: AggRule, + filterTags: Seq[String]): Boolean = { + shardKeyColumnsOption match { + case None => + logger.info("[HierarchicalQueryExperience] Dataset options config not found. Skipping optimization !") + false + case Some(shardKeyColumns) => + params.isHigherLevelAggregationApplicable(shardKeyColumns, filterTags) + } + } + + /** Returns the next level aggregated metric name. Example + * metricDelimiter = ::: + * metricSuffix = agg_2 + * Existing metric name - metric1:::agg + * After update - metric1:::agg -> metric1:::agg_2 + * @param metricName - String - Metric ColumnFilter tag/label + * @param metricDelimiter - HierarchicalQueryExperience - Contains + * @param metricSuffix - Seq[ColumnFilter] - label filters of the query/lp + * @return - Option[String] - Next level aggregated metric name + */ + def getNextLevelAggregatedMetricName(metricName : String, metricDelimiter: String, metricSuffix: String): String = { + metricName.replaceFirst(metricDelimiter + ".*", metricDelimiter + metricSuffix) + } + + /** Gets the current metric name from the given metricColumnFilter and filters + * + * @param metricColumnFilter - String - Metric ColumnFilter tag/label + * @param filters - Seq[ColumnFilter] - label filters of the query/lp + * @return - Option[String] - Next level aggregated metric name + */ + def getMetricName(metricColumnFilter: String, filters: Seq[ColumnFilter]): Option[String] = { + // Get the metric name from the filters + val metricNameSeq = LogicalPlan.getColumnValues(filters, metricColumnFilter) + metricNameSeq match { + case Seq() => None + case _ => Some(metricNameSeq.head) + } + } + + /** + * Checks if the given aggregation operator is allowed for next level aggregated metric. + * @param operatorName - String - Aggregation operator name. Ex - sum, avg, min, max, count. + * @return - Boolean + */ + def isAggregationOperatorAllowed(operatorName: String): Boolean = allowedAggregationOperators match { + case Some(allowedAggregationOperatorsSet) => allowedAggregationOperatorsSet.contains(operatorName) + case None => false + } + + /** + * Checks if the query with the given range function is allowed for next level aggregated metric. + * @param rangeFunctionEntryName - String - Range function name. Ex - rate, increase etc. + * @return - Boolean + */ + def isRangeFunctionAllowed(rangeFunctionEntryName: String): Boolean = allowedRangeFunctions match { + case Some(allowedRangeFunctionsSet) => allowedRangeFunctionsSet.contains(rangeFunctionEntryName) + case None => false + } + + /** + * Checks if at least one of the parent LogicalPlan(s) is allowed to update/optimize the metric name. If there are + * No parent LogicalPlans, then it returns false + * @param parentLogicalPlans - Seq[String] - List of parent LogicalPlans. Ex - Seq("BinaryJoin", "Aggregate") + * @return - Boolean + */ + def isParentPeriodicSeriesPlanAllowed(parentLogicalPlans: Seq[String]): Boolean = + allowedLogicalPlansForOptimization match { + case Some(allowedLogicalPlans) => parentLogicalPlans.exists(allowedLogicalPlans.contains) + case None => false + } + + /** + * Checks if the PeriodicSeriesPlan which has access to RawSeries is allowed to update/optimize the metric name + * @param logicalPlanName - PeriodicSeriesPlan name. Ex - PeriodicSeriesWithWindowing, PeriodicSeries + * @return - Boolean + */ + def isLeafPeriodicSeriesPlanAllowedForRawSeriesUpdate(logicalPlanName: String): Boolean = + allowedPeriodicSeriesPlansWithRawSeries match { + case Some(allowedPeriodSeriesPlans) => allowedPeriodSeriesPlans.contains(logicalPlanName) + case None => false + } + + /** + * Updates the metric column filter if higher level aggregation is applicable + * @param params - HierarchicalQueryExperienceParams - Contains metricDelimiter and aggRules + * @param filters - Seq[ColumnFilter] - label filters of the query/lp + * @return - Seq[ColumnFilter] - Updated filters + */ + def upsertMetricColumnFilterIfHigherLevelAggregationApplicable(params: HierarchicalQueryExperienceParams, + filters: Seq[ColumnFilter]): Seq[ColumnFilter] = { + val filterTags = filters.map(x => x.column) + val metricColumnFilter = getMetricColumnFilterTag(filterTags, GlobalConfig.datasetOptions.get.metricColumn) + val currentMetricName = getMetricName(metricColumnFilter, filters) + if (currentMetricName.isDefined) { + params.aggRules.find( x => currentMetricName.get.endsWith(x._1)) match { + case Some(aggRule) => + if (isHigherLevelAggregationApplicable(aggRule._2, filterTags)) { + val updatedMetricName = getNextLevelAggregatedMetricName( + currentMetricName.get, params.metricDelimiter, aggRule._2.metricSuffix) + val updatedFilters = upsertFilters( + filters, Seq(ColumnFilter(metricColumnFilter, Equals(updatedMetricName)))) + logger.info(s"[HierarchicalQueryExperience] Query optimized with filters: ${updatedFilters.toString()}") + incrementHierarchicalQueryOptimizedCounter(updatedFilters) + updatedFilters + } else { + filters + } + case None => filters + } + } else { + filters + } + } + + /** + * Track the queries optimized by workspace and namespace + * @param filters - Seq[ColumnFilter] - label filters of the query/lp + */ + private def incrementHierarchicalQueryOptimizedCounter(filters: Seq[ColumnFilter]): Unit = { + // track query optimized per workspace and namespace in the counter + val metric_ws = LogicalPlan.getColumnValues(filters, TsCardinalities.LABEL_WORKSPACE) match { + case Seq() => "" + case ws => ws.head + } + val metric_ns = LogicalPlan.getColumnValues(filters, TsCardinalities.LABEL_NAMESPACE) match { + case Seq() => "" + case ns => ns.head + } + hierarchicalQueryOptimizedCounter + .withTag("metric_ws", metric_ws) + .withTag("metric_ns", metric_ns) + .increment() + } + + /** + * Helper function to check the following: + * Check 1: Check if the aggregation operator is enabled + * Check 2: Check if the `by` and `without` clause labels satisfy the include/exclude tag constraints + * + * @param params higher level aggregation rule + * @param operator Aggregation operator like sum, avg, min, max, count + * @param clauseOpt AggregateClause - by or without clause + * @return true if the current aggregate query can be optimized, false otherwise + */ + def checkAggregateQueryEligibleForHigherLevelAggregatedMetric(params: AggRule, + operator: AggregationOperator, + clauseOpt: Option[AggregateClause]): Boolean = { + HierarchicalQueryExperience.isAggregationOperatorAllowed(operator.entryName) match { + case true => + clauseOpt match { + case Some(clause) => + clause.clauseType match { + case AggregateClause.ClauseType.By => + // Check if ALL the tags present in the by clause are part of includeTags or are not part of excludeTags + if (HierarchicalQueryExperience.isHigherLevelAggregationApplicable(params, clause.labels)) { + true + } + else { + // can't be optimized further as by labels not present in the higher level metric include tags + false + } + case AggregateClause.ClauseType.Without => + // This is a slightly more tricky case than the by clause. Following are the scenarios: + // 1. For excludeTags scenario: + // - We need to check if ALL the excludeTags is subset of the without clause's tags/labels + // - This ensures that we are not using a tag/label which is not part of the higher level metric + // 2. For includeTags scenario: + // - We need to check if all tags/labels, which are NOT present in the without clause, must be part of + // the includeTags. This requires the knowledge of all the tags/labels which are being published + // for a metric. This info is not available during planning and hence we can't optimize this scenario. + params match { + case IncludeAggRule( _, _) => + // can't optimize this scenario as we don't have the required info at the planning stage + false + case ExcludeAggRule(_, excludeTags) => + if (excludeTags.subsetOf(clause.labels.toSet)) { true } + else { false } + } + } + case None => + // No aggregation clause present. Check raw-series plan to see if we can use the next aggregation level + true + } + case _ => false + } + } +} diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index 03110e1f6f..f8558e9173 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -5,7 +5,6 @@ import filodb.core.query.Filter.{Equals, EqualsRegex, In, NotEquals, NotEqualsRe import filodb.query.BinaryOperator.DIV import filodb.query.Cardinality.OneToOne import filodb.query.RangeFunctionId.SumOverTime - import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 203b2d59f6..d8a1d23574 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -458,9 +458,9 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF it("test isMaxMinEnabledForWorkspace correctly returns expected values") { val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, MMD.histMaxMinDS.ref, 0, Seq(), AllChunkScan, "_metric_", colName = Some("h")) - execPlan.isMaxMinEnabledForWorkspace(Some("demo")) shouldEqual true - execPlan.isMaxMinEnabledForWorkspace(Some(GlobalConfig.workspacesDisabledForMaxMin.get.head)) shouldEqual false - execPlan.isMaxMinEnabledForWorkspace(None) shouldEqual false + execPlan.isMaxMinColumnsEnabled(Some("demo")) shouldEqual true + execPlan.isMaxMinColumnsEnabled(Some(GlobalConfig.workspacesDisabledForMaxMin.get.head)) shouldEqual false + execPlan.isMaxMinColumnsEnabled(None) shouldEqual false } it("should return chunk metadata from MemStore") { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala index bc98e0630e..36e7f25048 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -188,6 +188,54 @@ class RateFunctionsSpec extends RawDataWindowingSpec { } } + it("rate should work for even with single value NaN end of timeseries chunks") { + val data = (1 to 500).map(_ * 10 + rand.nextInt(10)).map(_.toDouble) + val tuples = data.zipWithIndex.map { case (d, t) => (defaultStartTS + t * pubFreq, d) } + val rv = timeValueRVPk(tuples) // should be a couple chunks + + // simulate creation of chunk with single NaN value + addChunkToRV(rv, Seq(defaultStartTS + 500 * pubFreq -> Double.NaN)) + + // add single row NaN chunk + // addChunkToRV(rv, tuples.takeRight(1)) + + (0 until 10).foreach { x => + val windowSize = rand.nextInt(100) + 10 + val step = rand.nextInt(50) + 5 + info(s" iteration $x windowSize=$windowSize step=$step") + + val slidingRate = slidingWindowIt(data, rv, RateFunction, windowSize, step) + val slidingResults = slidingRate.map(_.getDouble(1)).toBuffer + slidingRate.close() + + val rateChunked = chunkedWindowIt(data, rv, new ChunkedRateFunction, windowSize, step) + val resultRows = rateChunked.map { r => (r.getLong(0), r.getDouble(1)) }.toBuffer + val rates = resultRows.map(_._2) + + // Since the input data and window sizes are randomized, it is not possible to precompute results + // beforehand. Coming up with a formula to figure out the right rate is really hard. + // Thus we take an approach of comparing the sliding and chunked results to ensure they are identical. + + // val windowTime = (windowSize.toLong - 1) * pubFreq + // val expected = tuples.sliding(windowSize, step).toBuffer + // .zip(resultRows).map { case (w, (ts, _)) => + // // For some reason rate is based on window, not timestamps - so not w.last._1 + // (w.last._2 - w.head._2) / (windowTime) * 1000 + // // (w.last._2 - w.head._2) / (w.last._1 - w.head._1) * 1000 + // } + rates.dropRight(1) shouldEqual slidingResults.dropRight(1) + + // rate should be positive + val resultLen = rates.length + rates(resultLen - 1) shouldBe > (0d) // positive + + // sliding window is not used for rate/increase. We use ChunkedRateFunction. There may be slight difference + // in the way NaN is handled, which is okay. + val percentError: Double = math.abs(rates(resultLen - 1) - slidingResults(resultLen - 1))/slidingResults(resultLen - 1) + percentError shouldBe < (0.1d) + } + } + val promHistDS = Dataset("histogram", Seq("metric:string", "tags:map"), Seq("timestamp:ts", "count:long", "sum:long", "h:hist:counter=true")) val histBufferPool = new WriteBufferPool(TestData.nativeMem, promHistDS.schema.data, TestData.storeConf) diff --git a/query/src/test/scala/filodb/query/util/HierarchicalQueryExperienceSpec.scala b/query/src/test/scala/filodb/query/util/HierarchicalQueryExperienceSpec.scala new file mode 100644 index 0000000000..a3d4720076 --- /dev/null +++ b/query/src/test/scala/filodb/query/util/HierarchicalQueryExperienceSpec.scala @@ -0,0 +1,122 @@ +package filodb.query.util + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers +import kamon.Kamon +import kamon.testkit.InstrumentInspection.Syntax.counterInstrumentInspection + +import filodb.core.query.ColumnFilter +import filodb.core.query.Filter.Equals + +class HierarchicalQueryExperienceSpec extends AnyFunSpec with Matchers { + + it("getMetricColumnFilterTag should return expected column") { + HierarchicalQueryExperience.getMetricColumnFilterTag(Seq("tag1", "__name__"), "_metric_") shouldEqual "__name__" + HierarchicalQueryExperience.getMetricColumnFilterTag(Seq("tag1", "_metric_"), "_metric_") shouldEqual "_metric_" + HierarchicalQueryExperience.getMetricColumnFilterTag(Seq("tag1", "tag2"), "_metric_") shouldEqual "_metric_" + HierarchicalQueryExperience.getMetricColumnFilterTag(Seq("tag1", "tag2"), "__name__") shouldEqual "__name__" + } + + it("getNextLevelAggregatedMetricName should return expected metric name") { + + val params = IncludeAggRule("agg_2", Set("job", "instance")) + + // Case 1: Should not update if metric doesn't have the aggregated metric identifier + HierarchicalQueryExperience.getNextLevelAggregatedMetricName( + "metric1", ":::", params.metricSuffix) shouldEqual "metric1" + + // Case 2: Should update if metric has the aggregated metric identifier + HierarchicalQueryExperience.getNextLevelAggregatedMetricName( + "metric1:::agg", ":::", params.metricSuffix) shouldEqual "metric1:::agg_2" + } + + it("isParentPeriodicSeriesPlanAllowedForRawSeriesUpdateForHigherLevelAggregatedMetric return expected values") { + HierarchicalQueryExperience.isParentPeriodicSeriesPlanAllowed( + Seq("BinaryJoin", "Aggregate", "ScalarOperation")) shouldEqual true + + HierarchicalQueryExperience.isParentPeriodicSeriesPlanAllowed( + Seq("BinaryJoin", "ScalarOperation")) shouldEqual false + } + + it("isRangeFunctionAllowed should return expected values") { + HierarchicalQueryExperience.isRangeFunctionAllowed("rate") shouldEqual true + HierarchicalQueryExperience.isRangeFunctionAllowed("increase") shouldEqual true + HierarchicalQueryExperience.isRangeFunctionAllowed("sum_over_time") shouldEqual false + HierarchicalQueryExperience.isRangeFunctionAllowed("last") shouldEqual false + } + + it("isAggregationOperatorAllowed should return expected values") { + HierarchicalQueryExperience.isAggregationOperatorAllowed("sum") shouldEqual true + HierarchicalQueryExperience.isAggregationOperatorAllowed("min") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("max") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("avg") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("count") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("topk") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("bottomk") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("stddev") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("stdvar") shouldEqual false + HierarchicalQueryExperience.isAggregationOperatorAllowed("quantile") shouldEqual false + } + + it("should check if higher level aggregation is applicable with IncludeTags") { + HierarchicalQueryExperience.isHigherLevelAggregationApplicable( + IncludeAggRule("agg_2", Set("tag1", "tag2")), Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual true + + HierarchicalQueryExperience.isHigherLevelAggregationApplicable( + IncludeAggRule("agg_2", Set("tag1", "tag2", "tag3")), Seq("tag1", "tag2", "_ws_", "_ns_", "__name__")) shouldEqual true + + HierarchicalQueryExperience.isHigherLevelAggregationApplicable( + IncludeAggRule("agg_2", Set("tag1", "tag2", "tag3")), Seq("tag3", "tag4", "_ws_", "_ns_", "__name__")) shouldEqual false + } + + it("should check if higher level aggregation is applicable with ExcludeTags") { + HierarchicalQueryExperience.isHigherLevelAggregationApplicable( + ExcludeAggRule("agg_2", Set("tag1", "tag2")),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false + + HierarchicalQueryExperience.isHigherLevelAggregationApplicable( + ExcludeAggRule("agg_2", Set("tag1", "tag3")),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false + + HierarchicalQueryExperience.isHigherLevelAggregationApplicable( + ExcludeAggRule("agg_2", Set("tag1", "tag2")),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false + + HierarchicalQueryExperience.isHigherLevelAggregationApplicable( + ExcludeAggRule("agg_2", Set("tag3", "tag4")), Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual true + } + + it("checkAggregateQueryEligibleForHigherLevelAggregatedMetric should increment counter if metric updated") { + val excludeRule = ExcludeAggRule("agg_2", Set("notAggTag1", "notAggTag2")) + val params = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeRule)) + Kamon.init() + var counter = Kamon.counter("hierarchical-query-plans-optimized") + + // CASE 1: Should update if metric have the aggregated metric identifier + counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").value shouldEqual 0 + var updatedFilters = HierarchicalQueryExperience.upsertMetricColumnFilterIfHigherLevelAggregationApplicable( + params, Seq( + ColumnFilter("__name__", Equals("metric1:::agg")), + ColumnFilter("_ws_", Equals("testws")), + ColumnFilter("_ns_", Equals("testns")), + ColumnFilter("aggTag", Equals("value")))) + updatedFilters.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("metric1:::agg_2") + counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").value shouldEqual 1 + + + // CASE 2: Should not update if metric doesn't have the aggregated metric identifier + // reset the counter + counter = Kamon.counter("hierarchical-query-plans-optimized") + counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").value shouldEqual 0 + updatedFilters = HierarchicalQueryExperience.upsertMetricColumnFilterIfHigherLevelAggregationApplicable( + params, Seq( + ColumnFilter("__name__", Equals("metric1:::agg")), + ColumnFilter("_ws_", Equals("testws")), + ColumnFilter("_ns_", Equals("testns")), + ColumnFilter("notAggTag1", Equals("value")))) // using exclude tag, so should not optimize + updatedFilters.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] + .shouldEqual("metric1:::agg") + // count should not increment + counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").value shouldEqual 0 + + Kamon.stop() + } +} diff --git a/run_benchmarks.sh b/run_benchmarks.sh index 0572870994..30acb54659 100755 --- a/run_benchmarks.sh +++ b/run_benchmarks.sh @@ -1,5 +1,5 @@ #!/bin/bash -sbt "jmh/jmh:run -rf json -i 5 -wi 3 -f 1 -jvmArgsAppend -XX:MaxInlineLevel=20 \ +sbt -Drust.optimize=true "jmh/jmh:run -rf json -i 5 -wi 3 -f 1 -jvmArgsAppend -XX:MaxInlineLevel=20 \ -jvmArgsAppend -Xmx4g -jvmArgsAppend -XX:MaxInlineSize=99 \ filodb.jmh.QueryHiCardInMemoryBenchmark \ filodb.jmh.QueryInMemoryBenchmark \ @@ -7,4 +7,5 @@ sbt "jmh/jmh:run -rf json -i 5 -wi 3 -f 1 -jvmArgsAppend -XX:MaxInlineLevel=20 \ filodb.jmh.IngestionBenchmark \ filodb.jmh.QueryOnDemandBenchmark \ filodb.jmh.GatewayBenchmark \ - filodb.jmh.PartKeyIndexBenchmark" + filodb.jmh.PartKeyLuceneIndexBenchmark \ + filodb.jmh.PartKeyTantivyIndexBenchmark" diff --git a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala index 9697b57555..82b1c73042 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala @@ -1,6 +1,7 @@ package filodb.downsampler.index import scala.concurrent.Await +import scala.concurrent.duration.DurationInt import kamon.Kamon import kamon.metric.MeasurementUnit @@ -88,8 +89,11 @@ class DSIndexJob(dsSettings: DownsamplerSettings, sparkTasksFailed.increment throw e } + + // quick & dirty hack to ensure that the completed metric gets published if (dsSettings.shouldSleepForMetricsFlush) - Thread.sleep(62000) // quick & dirty hack to ensure that the completed metric gets published + Await.result(Kamon.stopModules(), 62.seconds) + } def migrateWithDownsamplePartKeys(partKeys: Observable[PartKeyRecord], shard: Int,