-
Notifications
You must be signed in to change notification settings - Fork 229
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(query): namespace cardinality publisher (#1294)
- Loading branch information
1 parent
a0558cc
commit 95b91c4
Showing
5 changed files
with
133 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
89 changes: 89 additions & 0 deletions
89
coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,89 @@ | ||
package filodb.coordinator | ||
|
||
import java.util.concurrent.TimeUnit | ||
|
||
import scala.concurrent.duration.FiniteDuration | ||
import scala.util.{Failure, Success} | ||
|
||
import akka.actor.ActorRef | ||
import com.typesafe.scalalogging.StrictLogging | ||
import kamon.Kamon | ||
import kamon.tag.TagSet | ||
import monix.execution.Scheduler.Implicits.{global => scheduler} | ||
|
||
import filodb.coordinator.client.Client | ||
import filodb.coordinator.client.QueryCommands.LogicalPlan2Query | ||
import filodb.core.DatasetRef | ||
import filodb.query.{QueryError, QueryResult, TsCardinalities} | ||
|
||
/** | ||
* Periodically queries a node for all namespace cardinalities. | ||
* Kamon gauges are updated with the response data. | ||
* | ||
* The intent is to publish a low-cardinality metric such that namespace | ||
* cardinality queries can be efficiently answered. | ||
* | ||
* @param dsIterProducer produces an iterator to step through all datasets. | ||
* @param coordActorProducer produces a single actor to ask a query. Actors are | ||
* queried in the order they're returned from this function. | ||
*/ | ||
case class TenantIngestionMetering(settings: FilodbSettings, | ||
dsIterProducer: () => Iterator[DatasetRef], | ||
coordActorProducer: () => ActorRef) extends StrictLogging{ | ||
|
||
private val ASK_TIMEOUT = FiniteDuration( | ||
settings.config.getDuration("metering-query-interval").toSeconds, | ||
TimeUnit.SECONDS) | ||
private val SCHED_INIT_DELAY = ASK_TIMEOUT // time until first job is scheduled | ||
private val SCHED_DELAY = ASK_TIMEOUT // time between all jobs after the first | ||
|
||
private val CLUSTER_TYPE = settings.config.getString("cluster-type") | ||
|
||
private val METRIC_ACTIVE = "active_timeseries_by_tenant" | ||
private val METRIC_TOTAL = "total_timeseries_by_tenant" | ||
|
||
def schedulePeriodicPublishJob() : Unit = { | ||
// NOTE: the FiniteDuration overload of scheduleWithFixedDelay | ||
// does not work. Unsure why, but that's why these FiniteDurations are | ||
// awkwardly parsed into seconds. | ||
scheduler.scheduleWithFixedDelay( | ||
SCHED_INIT_DELAY.toSeconds, | ||
SCHED_DELAY.toSeconds, | ||
TimeUnit.SECONDS, | ||
() => queryAndSchedulePublish()) | ||
} | ||
|
||
/** | ||
* For each dataset, ask a Coordinator with a TsCardinalities LogicalPlan. | ||
* Schedules a job to publish the Coordinator's response. | ||
*/ | ||
private def queryAndSchedulePublish() : Unit = { | ||
import filodb.query.exec.TsCardExec._ | ||
val groupDepth = 1 // group cardinalities at the second level (i.e. ws & ns) | ||
val prefix = Nil // query for cardinalities regardless of first-level name (i.e. ws name) | ||
dsIterProducer().foreach { dsRef => | ||
val fut = Client.asyncAsk( | ||
coordActorProducer(), | ||
LogicalPlan2Query(dsRef, TsCardinalities(prefix, groupDepth)), | ||
ASK_TIMEOUT) | ||
fut.onComplete { | ||
case Success(QueryResult(_, _, rv, _, _, _)) => | ||
rv.foreach(_.rows().foreach{ rr => | ||
// publish a cardinality metric for each namespace | ||
val data = RowData.fromRowReader(rr) | ||
val prefix = data.group.toString.split(PREFIX_DELIM) | ||
val tags = Map("metric_ws" -> prefix(0), | ||
"metric_ns" -> prefix(1), | ||
"dataset" -> dsRef.dataset, | ||
"cluster_type" -> CLUSTER_TYPE) | ||
Kamon.gauge(METRIC_ACTIVE).withTags(TagSet.from(tags)).update(data.counts.active.toDouble) | ||
Kamon.gauge(METRIC_TOTAL).withTags(TagSet.from(tags)).update(data.counts.total.toDouble) | ||
}) | ||
case Success(QueryError(_, _, t)) => logger.warn("QueryError: " + t.getMessage) | ||
case Failure(t) => logger.warn("Failure: " + t.getMessage) | ||
// required to compile | ||
case _ => throw new IllegalArgumentException("should never reach here; attempted to match: " + fut) | ||
} | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
29 changes: 29 additions & 0 deletions
29
core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreSpec.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
package filodb.core.memstore.ratelimit | ||
|
||
import org.scalatest.funspec.AnyFunSpec | ||
import org.scalatest.matchers.should.Matchers | ||
|
||
import filodb.core.memstore.ratelimit.CardinalityStore._ | ||
import filodb.core.MetricsTestData | ||
|
||
class RocksDbCardinalityStoreSpec extends AnyFunSpec with Matchers { | ||
it ("should correctly return overflow record") { | ||
val dset = MetricsTestData.timeseriesDatasetMultipleShardKeys | ||
val shard = 0 | ||
val numOverflow = 123 | ||
|
||
val db = new RocksDbCardinalityStore(dset.ref, shard) | ||
|
||
(0 until MAX_RESULT_SIZE + numOverflow).foreach{ i => | ||
val prefix = Seq("ws", "ns", s"metric-$i") | ||
db.store(CardinalityRecord(shard, prefix, 1, 1, 1, 1)) | ||
} | ||
|
||
Seq(Nil, Seq("ws"), Seq("ws", "ns")).foreach{ prefix => | ||
val res = db.scanChildren(prefix, 3) | ||
res.size shouldEqual MAX_RESULT_SIZE + 1 // one extra for the overflow CardinalityRecord | ||
res.contains(CardinalityRecord(shard, OVERFLOW_PREFIX, | ||
numOverflow, numOverflow, numOverflow, numOverflow)) shouldEqual true | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters