From f04e3ff2193af6cdb9bc063509e4f1c14bc2e47d Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Thu, 20 Aug 2020 13:38:37 -0700 Subject: [PATCH 01/53] misc(query): match nonleaf child query parallelism to number of children (#865) --- query/src/main/scala/filodb/query/exec/ExecPlan.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index c776746cbc..76cf79c7bb 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -371,7 +371,7 @@ abstract class NonLeafExecPlan extends ExecPlan { // Create tasks for all results. // NOTE: It's really important to preserve the "index" of the child task, as joins depend on it val childTasks = Observable.fromIterable(children.zipWithIndex) - .mapAsync(Runtime.getRuntime.availableProcessors()) { case (plan, i) => + .mapAsync(children.length) { case (plan, i) => dispatchRemotePlan(plan, parentSpan).map((_, i)) } From 3dec2c767a8930d2c2a2bc8146f3b357a4b12a60 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Mon, 24 Aug 2020 11:48:23 -0700 Subject: [PATCH 02/53] bug(query): binary join with AND has LHS range vectors when corresponding RHS range vector is empty (#867) --- .../filodb/query/exec/SetOperatorExec.scala | 42 +++-- .../exec/BinaryJoinSetOperatorSpec.scala | 171 ++++++++++++++---- 2 files changed, 160 insertions(+), 53 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index 9735d60cc6..a22889044c 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -53,20 +53,23 @@ final case class SetOperatorExec(queryContext: QueryContext, firstSchema: Task[ResultSchema], querySession: QuerySession): Observable[RangeVector] = { val taskOfResults = childResponses.map { - case (QueryResult(_, _, result), i) => (result, i) - case (QueryError(_, ex), _) => throw ex + case (QueryResult(_, schema, result), i) => (schema, result, i) + case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => // NOTE: We can't require this any more, as multischema queries may result in not a QueryResult if the // filter returns empty results. The reason is that the schema will be undefined. // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") - val lhsRvs = resp.filter(_._2 < lhs.size).flatMap(_._1) - val rhsRvs = resp.filter(_._2 >= lhs.size).flatMap(_._1) - - val results: List[RangeVector] = binaryOp match { - case LAND => setOpAnd(lhsRvs, rhsRvs) - case LOR => setOpOr(lhsRvs, rhsRvs) + // Resp is segregated based on index of child plans + val lhsRvs = resp.filter(_._3 < lhs.size).flatMap(_._2) + val rhsResp = resp.filter(_._3 >= lhs.size) + val rhsRvs = rhsResp.flatMap(_._2) + + val results: List[RangeVector] = binaryOp match { + case LAND => val rhsSchema = if (rhsResp.map(_._1).nonEmpty) rhsResp.map(_._1).head else ResultSchema.empty + setOpAnd(lhsRvs, rhsRvs, rhsSchema) + case LOR => setOpOr(lhsRvs, rhsRvs) case LUnless => setOpUnless(lhsRvs, rhsRvs) - case _ => throw new IllegalArgumentException("requirement failed: " + "Only and, or and unless are supported ") + case _ => throw new IllegalArgumentException("requirement failed: Only and, or and unless are supported ") } Observable.fromIterable(results) @@ -79,21 +82,30 @@ final case class SetOperatorExec(queryContext: QueryContext, else rvk.labelValues.filterNot(lv => ignoringLabels.contains(lv._1)) } - private def setOpAnd(lhsRvs: List[RangeVector] - , rhsRvs: List[RangeVector]): List[RangeVector] = { + /*** + * Returns true when range vector does not have any values + */ + private def isEmpty(rv: RangeVector, schema: ResultSchema) = { + if (schema.isHistogram) rv.rows.map(_.getHistogram(1)).filter(_.numBuckets > 0).isEmpty + else rv.rows.filter(!_.getDouble(1).isNaN).isEmpty + } + + private def setOpAnd(lhsRvs: List[RangeVector], rhsRvs: List[RangeVector], + rhsSchema: ResultSchema): List[RangeVector] = { val rhsKeysSet = new mutable.HashSet[Map[Utf8Str, Utf8Str]]() var result = new ListBuffer[RangeVector]() rhsRvs.foreach { rv => val jk = joinKeys(rv.key) - if (jk.nonEmpty) + // Don't add range vector if it is empty + if (jk.nonEmpty && !isEmpty(rv, rhsSchema)) rhsKeysSet += jk } lhsRvs.foreach { lhs => val jk = joinKeys(lhs.key) - // Add range vectors from lhs which are present in lhs and rhs both - // Result should also have range vectors for which rhs does not have any keys - if (rhsKeysSet.contains(jk) || rhsKeysSet.isEmpty) { + // Add range vectors from lhs which are present in lhs and rhs both or when jk is empty + // "up AND ON (dummy) vector(1)" should be equivalent to up as there's no dummy label + if (rhsKeysSet.contains(jk) || jk.isEmpty) { result += lhs } } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index b24febfb94..cab77deb86 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -173,6 +173,46 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur new TransientRow(1L, 200)).iterator } ) + + val sampleWithNaN: Array[RangeVector] = Array( + new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"http_requests".utf8, + "job".utf8 -> s"api-server".utf8, + "instance".utf8 -> "0".utf8, + "group".utf8 -> s"production".utf8) + ) + + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( + new TransientRow(1L, 100)).iterator + }, + new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"http_requests".utf8, + "job".utf8 -> s"api-server".utf8, + "instance".utf8 -> "1".utf8, + "group".utf8 -> s"production".utf8) + ) + + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( + new TransientRow(1L, Double.NaN)).iterator + }) + val sampleAllNaN : Array[RangeVector] = Array( + new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"http_requests".utf8, + "job".utf8 -> s"api-server".utf8, + "instance".utf8 -> "0".utf8, + "group".utf8 -> s"production".utf8) + ) + + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( + new TransientRow(1L, Double.NaN)).iterator + }) + val sampleCanary = sampleHttpRequests.filter(_.key.labelValues.get(ZeroCopyUTF8String("group")).get. toString.equals("canary")) val sampleProduction = sampleHttpRequests.filter(_.key.labelValues.get(ZeroCopyUTF8String("group")).get. @@ -197,8 +237,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Nil, Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -234,8 +274,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -271,8 +311,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -308,8 +348,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -344,8 +384,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -379,8 +419,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -412,8 +452,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Seq("dummy"), Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -434,8 +474,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Nil, Seq("group", "instance", "job"), "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -456,8 +496,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Nil, Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -486,8 +526,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -546,8 +586,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) - val rhs1 = QueryResult("someId", null, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) + val lhs1 = QueryResult("someId", tvSchema, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) + val rhs1 = QueryResult("someId", tvSchema, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -559,8 +599,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Seq("instance"), Nil, "__name__") // scalastyle:off - val lhs2 = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs2 = QueryResult("someId", null, result1.map(rv => SerializedRangeVector(rv, schema))) + val lhs2 = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs2 = QueryResult("someId", tvSchema, result1.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -617,8 +657,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off - val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) - val rhs1 = QueryResult("someId", null, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) + val lhs1 = QueryResult("someId", tvSchema, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) + val rhs1 = QueryResult("someId", tvSchema, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -630,8 +670,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Nil, Seq("l", "group", "job"), "__name__") // scalastyle:off - val lhs2 = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) - val rhs2 = QueryResult("someId", null, result1.map(rv => SerializedRangeVector(rv, schema))) + val lhs2 = QueryResult("someId", tvSchema, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) + val rhs2 = QueryResult("someId", tvSchema, result1.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -685,8 +725,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Nil, Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -719,8 +759,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Seq("job"), Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -750,8 +790,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Seq("job", "instance"), Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -785,8 +825,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Seq("job"), Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -817,8 +857,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur Seq("job", "instance"), Nil, "__name__") // scalastyle:off - val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) - val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) + val lhs = QueryResult("someId", tvSchema, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -841,4 +881,59 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur result(0).rows.map(_.getDouble(1)).toList shouldEqual List(400) result(1).rows.map(_.getDouble(1)).toList shouldEqual List(800) } + + it("AND should not return rv's when RHS has only NaN") { + + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, + Array(dummyPlan), + new Array[ExecPlan](1), + BinaryOperator.LAND, + Nil, Nil, "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", tvSchema, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleAllNaN.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) + .toListL.runAsync.futureValue + + val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("api-server"), + ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("0"), + ZeroCopyUTF8String("group") -> ZeroCopyUTF8String("canary") + ), + Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("app-server"), + ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("0"), + ZeroCopyUTF8String("group") -> ZeroCopyUTF8String("canary") + )) + + result.size shouldEqual 0 + } + + it("AND should return only non NaN RangeVectors") { + + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, + Array(dummyPlan), + new Array[ExecPlan](1), + BinaryOperator.LAND, + Nil, Nil, "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", tvSchema, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleWithNaN.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) + .toListL.runAsync.futureValue + + val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("api-server"), + ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("0"), + ZeroCopyUTF8String("group") -> ZeroCopyUTF8String("production") + )) + + result.size shouldEqual 1 // second RV in sampleWithNaN has all Nan's + result.map(_.key.labelValues) sameElements (expectedLabels) shouldEqual true + result(0).rows.map(_.getDouble(1)).toList shouldEqual List(100) + } } From bd629df7f6f4e82fef19ec7dd4b410f9e2e86eff Mon Sep 17 00:00:00 2001 From: Szymon Matejczyk Date: Mon, 24 Aug 2020 22:40:37 +0200 Subject: [PATCH 03/53] misc(cli): use scallop instead of sumac (#854) --- cli/src/main/scala/filodb.cli/CliMain.scala | 144 ++++++++++---------- project/Dependencies.scala | 2 +- 2 files changed, 74 insertions(+), 72 deletions(-) diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 2a8fa9745b..0c2ba5fdd5 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -8,8 +8,8 @@ import scala.concurrent.duration._ import scala.util.Try import com.opencsv.CSVWriter -import com.quantifind.sumac.{ArgMain, FieldArgs} import monix.reactive.Observable +import org.rogach.scallop.ScallopConf import org.scalactic._ import filodb.coordinator._ @@ -27,41 +27,42 @@ import filodb.prometheus.parse.Parser import filodb.query._ // scalastyle:off -class Arguments extends FieldArgs { - var dataset: Option[String] = None - var database: Option[String] = None - var command: Option[String] = None - var filename: Option[String] = None - var configPath: Option[String] = None +class Arguments(args: Seq[String]) extends ScallopConf(args) { + val dataset = opt[String]() + val database = opt[String]() + val command = opt[String]() + val filename = opt[String]() + val configPath = opt[String]() // max # of results returned. Don't make it too high. - var limit: Int = 200 - var sampleLimit: Int = 1000000 - var timeoutSeconds: Int = 60 - var outfile: Option[String] = None - var delimiter: String = "," - var indexName: Option[String] = None - var host: Option[String] = None - var port: Int = 2552 - var promql: Option[String] = None - var schema: Option[String] = None - var hexPk: Option[String] = None - var hexVector: Option[String] = None - var hexChunkInfo: Option[String] = None - var vectorType: Option[String] = None - var matcher: Option[String] = None - var labelNames: Seq[String] = Seq.empty - var labelFilter: Map[String, String] = Map.empty - var start: Long = System.currentTimeMillis() / 1000 // promql argument is seconds since epoch - var end: Long = System.currentTimeMillis() / 1000 // promql argument is seconds since epoch - var minutes: Option[String] = None - var step: Long = 10 // in seconds - var chunks: Option[String] = None // select either "memory" or "buffers" chunks only - var everyNSeconds: Option[String] = None - var shards: Option[Seq[String]] = None - var spread: Option[Integer] = None + val limit = opt[Int](default = Some(200)) + val sampleLimit = opt[Int](default = Some(1000000)) + val timeoutSeconds = opt[Int](default = Some(60)) + val outfile = opt[String]() + val delimiter = opt[String](default = Some(",")).apply() + val indexName = opt[String]() + val host = opt[String]() + val port = opt[Int](default = Some(2552)) + val promql = opt[String]() + val schema = opt[String]() + val hexPk = opt[String]() + val hexVector = opt[String]() + val hexChunkInfo = opt[String]() + val vectorType = opt[String]() + val matcher = opt[String]() + val labelNames = opt[List[String]](default = Some(List())) + val labelFilter = opt[Map[String, String]](default = Some(Map.empty)) + val start: Long = System.currentTimeMillis() / 1000 // promql argument is seconds since epoch + val end: Long = System.currentTimeMillis() / 1000 // promql argument is seconds since epoch + val minutes = opt[String]() + val step = opt[Long](default = Some(10)) // in seconds + val chunks = opt[String]() // select either "memory" or "buffers" chunks only + val everyNSeconds = opt[String]() + val shards = opt[List[String]]() + val spread = opt[Int]() + verify() } -object CliMain extends ArgMain[Arguments] with FilodbClusterNode { +object CliMain extends FilodbClusterNode { var exitCode = 0 override val role = ClusterRole.Cli @@ -99,30 +100,31 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { println(" options: ./filo-cli -Dakka.loglevel=DEBUG -Dakka.actor.debug.receive=on -Dakka.actor.debug.autoreceive=on --command importcsv ...") } - def getRef(args: Arguments): DatasetRef = DatasetRef(args.dataset.get, args.database) + def getRef(args: Arguments): DatasetRef = DatasetRef(args.dataset(), args.database.toOption) def getClientAndRef(args: Arguments): (LocalClient, DatasetRef) = { - require(args.host.nonEmpty && args.dataset.nonEmpty, "--host and --dataset must be defined") - val remote = Client.standaloneClient(system, args.host.get, args.port) - (remote, DatasetRef(args.dataset.get)) + require(args.host.isDefined && args.dataset.isDefined, "--host and --dataset must be defined") + val remote = Client.standaloneClient(system, args.host(), args.port()) + (remote, DatasetRef(args.dataset())) } def getQueryRange(args: Arguments): TimeRangeParams = args.chunks.filter { cOpt => cOpt == "memory" || cOpt == "buffers" } .map { - case "memory" => InMemoryParam(args.step) - case "buffers" => WriteBuffersParam(args.step) + case "memory" => InMemoryParam(args.step()) + case "buffers" => WriteBuffersParam(args.step()) }.getOrElse { args.minutes.map { minArg => val end = System.currentTimeMillis() / 1000 - TimeStepParams(end - minArg.toInt * 60, args.step, end) - }.getOrElse(TimeStepParams(args.start, args.step, args.end)) + TimeStepParams(end - minArg.toInt * 60, args.step(), end) + }.getOrElse(TimeStepParams(args.start, args.step(), args.end)) } - def main(args: Arguments): Unit = { + def main(rawArgs: Array[String]): Unit = { + val args = new Arguments(rawArgs) try { - val timeout = args.timeoutSeconds.seconds - args.command match { + val timeout = args.timeoutSeconds().seconds + args.command.toOption match { case Some("init") => println("Initializing FiloDB Admin keyspace and tables...") parse(metaStore.initialize(), timeout) { @@ -137,7 +139,7 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { case Some("list") => args.host.map { server => - listRegisteredDatasets(Client.standaloneClient(system, server, args.port)) + listRegisteredDatasets(Client.standaloneClient(system, server, args.port())) } case Some("indexnames") => @@ -146,10 +148,10 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { names.foreach(println) case Some("indexvalues") => - require(args.indexName.nonEmpty, "--indexName required") - require(args.shards.nonEmpty, "--shards required") + require(args.indexName.isDefined, "--indexName required") + require(args.shards.isDefined, "--shards required") val (remote, ref) = getClientAndRef(args) - val values = remote.getIndexValues(ref, args.indexName.get, args.shards.get.head.toInt, args.limit) + val values = remote.getIndexValues(ref, args.indexName(), args.shards().head.toInt, args.limit()) values.foreach { case (term, freq) => println(f"$term%40s\t$freq") } case Some("status") => @@ -159,45 +161,45 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { case Some("validateSchemas") => validateSchemas() case Some("promFilterToPartKeyBR") => - require(args.promql.nonEmpty && args.schema.nonEmpty, "--promql and --schema must be defined") - promFilterToPartKeyBr(args.promql.get, args.schema.get) + require(args.promql.isDefined && args.schema.isDefined, "--promql and --schema must be defined") + promFilterToPartKeyBr(args.promql(), args.schema()) case Some("partKeyBrAsString") => - require(args.hexPk.nonEmpty, "--hexPk must be defined") - partKeyBrAsString(args.hexPk.get) + require(args.hexPk.isDefined, "--hexPk must be defined") + partKeyBrAsString(args.hexPk()) case Some("decodeChunkInfo") => - require(args.hexChunkInfo.nonEmpty, "--hexChunkInfo must be defined") - decodeChunkInfo(args.hexChunkInfo.get) + require(args.hexChunkInfo.isDefined, "--hexChunkInfo must be defined") + decodeChunkInfo(args.hexChunkInfo()) case Some("decodeVector") => - require(args.hexVector.nonEmpty && args.vectorType.nonEmpty, "--hexVector and --vectorType must be defined") - decodeVector(args.hexVector.get, args.vectorType.get) + require(args.hexVector.isDefined && args.vectorType.isDefined, "--hexVector and --vectorType must be defined") + decodeVector(args.hexVector(), args.vectorType()) case Some("timeseriesMetadata") => - require(args.host.nonEmpty && args.dataset.nonEmpty && args.matcher.nonEmpty, "--host, --dataset and --matcher must be defined") - val remote = Client.standaloneClient(system, args.host.get, args.port) - val options = QOptions(args.limit, args.sampleLimit, args.everyNSeconds.map(_.toInt), - timeout, args.shards.map(_.map(_.toInt)), args.spread) - parseTimeSeriesMetadataQuery(remote, args.matcher.get, args.dataset.get, + require(args.host.isDefined && args.dataset.isDefined && args.matcher.isDefined, "--host, --dataset and --matcher must be defined") + val remote = Client.standaloneClient(system, args.host(), args.port()) + val options = QOptions(args.limit(), args.sampleLimit(), args.everyNSeconds.map(_.toInt).toOption, + timeout, args.shards.map(_.map(_.toInt)).toOption, args.spread.toOption.map(Integer.valueOf)) + parseTimeSeriesMetadataQuery(remote, args.matcher(), args.dataset(), getQueryRange(args), true, options) case Some("labelValues") => - require(args.host.nonEmpty && args.dataset.nonEmpty && args.labelNames.nonEmpty, "--host, --dataset and --labelName must be defined") - val remote = Client.standaloneClient(system, args.host.get, args.port) - val options = QOptions(args.limit, args.sampleLimit, args.everyNSeconds.map(_.toInt), - timeout, args.shards.map(_.map(_.toInt)), args.spread) - parseLabelValuesQuery(remote, args.labelNames, args.labelFilter, args.dataset.get, + require(args.host.isDefined && args.dataset.isDefined && args.labelNames.isDefined, "--host, --dataset and --labelName must be defined") + val remote = Client.standaloneClient(system, args.host(), args.port()) + val options = QOptions(args.limit(), args.sampleLimit(), args.everyNSeconds.map(_.toInt).toOption, + timeout, args.shards.map(_.map(_.toInt)).toOption, args.spread.toOption.map(Integer.valueOf)) + parseLabelValuesQuery(remote, args.labelNames(), args.labelFilter(), args.dataset(), getQueryRange(args), options) case x: Any => // This will soon be deprecated args.promql.map { query => - require(args.host.nonEmpty && args.dataset.nonEmpty, "--host and --dataset must be defined") - val remote = Client.standaloneClient(system, args.host.get, args.port) - val options = QOptions(args.limit, args.sampleLimit, args.everyNSeconds.map(_.toInt), - timeout, args.shards.map(_.map(_.toInt)), args.spread) - parsePromQuery2(remote, query, args.dataset.get, getQueryRange(args), options) + require(args.host.isDefined && args.dataset.isDefined, "--host and --dataset must be defined") + val remote = Client.standaloneClient(system, args.host(), args.port()) + val options = QOptions(args.limit(), args.sampleLimit(), args.everyNSeconds.toOption.map(_.toInt), + timeout, args.shards.toOption.map(_.map(_.toInt)), args.spread.toOption.map(Integer.valueOf)) + parsePromQuery2(remote, query, args.dataset(), getQueryRange(args), options) } .getOrElse(printHelp) } diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 8e13f5b3d6..26bf9837d5 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -120,7 +120,7 @@ object Dependencies { lazy val cliDeps = Seq( logbackDep, "io.kamon" %% "kamon-bundle" % kamonBundleVersion, - "com.quantifind" %% "sumac" % "0.3.0" + "org.rogach" %% "scallop" % "3.1.1" ) lazy val kafkaDeps = Seq( From 85c3feac4af5d8a89e989d67200bd9b243df8538 Mon Sep 17 00:00:00 2001 From: Szymon Matejczyk Date: Tue, 25 Aug 2020 22:35:32 +0200 Subject: [PATCH 04/53] misc: remove scalaxy, use spire.cforRange (#866) --- .../multijvm/BaseAkkaBootstrapperSpec.scala | 1 - .../filodb.coordinator/client/Client.scala | 2 +- core/src/main/scala/filodb.core/KeyType.scala | 7 +++--- .../binaryrecord2/RecordBuilder.scala | 8 +++---- .../binaryrecord2/RecordComparator.scala | 6 ++--- .../binaryrecord2/RecordSchema.scala | 4 ++-- .../downsample/DownsamplePeriodMarker.scala | 4 ++-- .../legacy/binaryrecord/RecordBuilder.scala | 10 ++++---- .../binaryrecord/RecordComparator.scala | 6 ++--- .../memstore/PartKeyLuceneIndex.scala | 8 +++---- .../memstore/PartitionKeyIndex.scala | 6 ++--- .../memstore/TimeSeriesPartition.scala | 4 ++-- .../memstore/TimeSeriesShard.scala | 10 ++++---- .../memstore/WriteBufferPool.scala | 6 ++--- .../filodb.core/metadata/ComputedColumn.scala | 4 +--- .../scala/filodb.core/query/KeyFilter.scala | 2 +- .../query/PartitionTimeRangeReader.scala | 6 ++--- .../conversion/InfluxProtocolParser.scala | 8 +++---- .../legacy/conversion/InputRecord.scala | 6 ++--- .../scala/filodb.jmh/BasicFiloBenchmark.scala | 12 ++++------ .../filodb.jmh/DictStringBenchmark.scala | 10 ++++---- .../scala/filodb.jmh/EncodingBenchmark.scala | 16 ++++++------- .../filodb.jmh/HistogramIngestBenchmark.scala | 8 +++---- .../filodb.jmh/IntSumReadBenchmark.scala | 8 +++---- .../filodb.jmh/PartKeyIndexBenchmark.scala | 15 ++++++------ .../filodb.jmh/PartitionListBenchmark.scala | 12 ++++------ .../filodb.memory/format/BinaryVector.scala | 10 ++++---- .../filodb.memory/format/NibblePack.scala | 24 +++++++++---------- .../filodb.memory/format/RowReader.scala | 6 ++--- .../format/RowToVectorBuilder.scala | 8 +++---- .../filodb.memory/format/UnsafeUtils.scala | 5 ++-- .../filodb.memory/format/ZeroCopyBinary.scala | 8 +++---- .../format/vectors/DeltaDeltaVector.scala | 6 ++--- .../format/vectors/DictUTF8Vector.scala | 4 ++-- .../format/vectors/DoubleVector.scala | 12 +++++----- .../format/vectors/Histogram.scala | 24 +++++++++---------- .../format/vectors/HistogramCompressor.scala | 8 +++---- .../format/vectors/HistogramVector.scala | 6 ++--- .../format/vectors/IntBinaryVector.scala | 8 +++---- .../format/vectors/LongBinaryVector.scala | 8 +++---- .../format/vectors/UTF8Vector.scala | 8 +++---- project/Dependencies.scala | 16 ++++--------- .../scala/filodb/query/PromCirceSupport.scala | 3 +-- .../query/exec/AggrOverRangeVectors.scala | 6 ++--- .../query/exec/HistogramQuantileMapper.scala | 4 ++-- .../query/exec/RangeVectorTransformer.scala | 4 ++-- .../query/exec/rangefn/InstantFunction.scala | 6 ++--- .../query/exec/rangefn/RateFunctions.scala | 4 ++-- .../downsampler/chunk/BatchDownsampler.scala | 6 ++--- .../scala/filodb.stress/MemStoreStress.scala | 4 +--- 50 files changed, 176 insertions(+), 211 deletions(-) diff --git a/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala b/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala index 83a8aa10d7..283ddff216 100644 --- a/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala +++ b/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala @@ -1,7 +1,6 @@ package filodb.akkabootstrapper.multijvm import scala.concurrent.duration._ -import scala.language.postfixOps import akka.actor.AddressFromURIString import akka.cluster.Cluster import akka.http.scaladsl.Http diff --git a/coordinator/src/main/scala/filodb.coordinator/client/Client.scala b/coordinator/src/main/scala/filodb.coordinator/client/Client.scala index 56edcaba49..2b3448dc0c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/Client.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/Client.scala @@ -146,4 +146,4 @@ class ClusterClient(nodeClusterActor: ActorRef, def sendAllIngestors(msg: Any): Unit = nodeClusterActor ! Broadcast(ingestionRole, msg) val clusterActor = Some(nodeClusterActor) -} \ No newline at end of file +} diff --git a/core/src/main/scala/filodb.core/KeyType.scala b/core/src/main/scala/filodb.core/KeyType.scala index 6ab1c7af2c..161fc8d216 100644 --- a/core/src/main/scala/filodb.core/KeyType.scala +++ b/core/src/main/scala/filodb.core/KeyType.scala @@ -2,11 +2,10 @@ package filodb.core import java.sql.Timestamp -import scala.language.postfixOps import scala.math.Ordering import org.joda.time.DateTime -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.format.{vectors => bv, RowReader, ZeroCopyUTF8String} import filodb.memory.format.RowReader._ @@ -60,7 +59,7 @@ abstract class SingleKeyTypeBase[K : Ordering : TypedFieldExtractor] extends Key case class CompositeOrdering(atomTypes: Seq[SingleKeyType]) extends Ordering[Seq[_]] { override def compare(x: Seq[_], y: Seq[_]): Int = { if (x.length == y.length && x.length == atomTypes.length) { - for { i <- 0 until x.length optimized } { + cforRange { 0 until x.length } { i => val keyType = atomTypes(i) val res = keyType.ordering.compare(x(i).asInstanceOf[keyType.T], y(i).asInstanceOf[keyType.T]) @@ -76,7 +75,7 @@ case class CompositeReaderOrdering(atomTypes: Seq[SingleKeyType]) extends Orderi private final val extractors = atomTypes.map(_.extractor).toArray private final val numAtoms = atomTypes.length def compare(a: RowReader, b: RowReader): Int = { - for { i <- 0 until numAtoms optimized } { + cforRange { 0 until numAtoms } { i => val res = extractors(i).compare(a, b, i) if (res != 0) return res } diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala index c3a76a04b4..dc63aced93 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala @@ -2,7 +2,7 @@ package filodb.core.binaryrecord2 import com.typesafe.scalalogging.StrictLogging import org.agrona.DirectBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.binaryrecord2.RecordSchema.schemaID import filodb.core.metadata.{Column, DatasetOptions, PartitionSchema, Schema, Schemas} @@ -283,7 +283,7 @@ class RecordBuilder(memFactory: MemFactory, */ final def addFromReader(row: RowReader, schema: RecordSchema, schemID: Int): Long = { startNewRecord(schema, schemID) - for { pos <- 0 until schema.numFields optimized } { + cforRange { 0 until schema.numFields } { pos => schema.builderAdders(pos)(row, this) } endRecord() @@ -615,7 +615,7 @@ object RecordBuilder { final def sortAndComputeHashes(pairs: java.util.ArrayList[(String, String)]): Array[Int] = { pairs.sort(stringPairComparator) val hashes = new Array[Int](pairs.size) - for { i <- 0 until pairs.size optimized } { + cforRange { 0 until pairs.size } { i => val (k, v) = pairs.get(i) // This is not very efficient, we have to convert String to bytes first to get the hash // TODO: work on different API which is far more efficient and saves memory allocation @@ -644,7 +644,7 @@ object RecordBuilder { hashes: Array[Int], excludeKeys: Set[String]): Int = { var hash = 7 - for { i <- 0 until sortedPairs.size optimized } { + cforRange { 0 until sortedPairs.size } { i => if (!(excludeKeys contains sortedPairs.get(i)._1)) hash = combineHash(hash, hashes(i)) } diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala index 76e90762fb..9eed71f5b6 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala @@ -1,6 +1,6 @@ package filodb.core.binaryrecord2 -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.format.UnsafeUtils @@ -130,7 +130,7 @@ final class RecordComparator(ingestSchema: RecordSchema) { // adjust offsets to var fields val adjustment = partVarAreaOffset - ingestVarAreaOffset - for { i <- 0 until fixedAreaNumWords optimized } { + cforRange { 0 until fixedAreaNumWords } { i => if ((compareBitmap & (1 << i)) == 0) { // not a primitive field, but an offset to String or Map builder.adjustFieldOffset(i, adjustment) } @@ -138,4 +138,4 @@ final class RecordComparator(ingestSchema: RecordSchema) { builder.endRecord(writeHash = false) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index 20e23e440d..d71b391a1f 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -4,7 +4,7 @@ import scala.collection.mutable.ArrayBuffer import org.agrona.DirectBuffer import org.agrona.concurrent.UnsafeBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.metadata.{Column, Schemas} import filodb.core.metadata.Column.ColumnType.{LongColumn, MapColumn, TimestampColumn} @@ -430,7 +430,7 @@ object RecordSchema { // Creates a Long from a byte array private def eightBytesToLong(bytes: Array[Byte], index: Int, len: Int): Long = { var num = 0L - for { i <- 0 until len optimized } { + cforRange { 0 until len } { i => num = (num << 8) ^ (bytes(index + i) & 0x00ff) } num diff --git a/core/src/main/scala/filodb.core/downsample/DownsamplePeriodMarker.scala b/core/src/main/scala/filodb.core/downsample/DownsamplePeriodMarker.scala index ad5b5e6bcb..c933a4354e 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsamplePeriodMarker.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsamplePeriodMarker.scala @@ -1,7 +1,7 @@ package filodb.core.downsample import enumeratum.{Enum, EnumEntry} -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.metadata.DataSchema import filodb.core.store.{ChunkSetInfoReader, ReadablePartition} @@ -125,7 +125,7 @@ class CounterDownsamplePeriodMarker(val inputColId: Int) extends DownsamplePerio case r: DoubleVectorDataReader => if (PrimitiveVectorReader.dropped(ctrVecAcc, ctrVecPtr)) { // counter dip detected val drops = r.asInstanceOf[CorrectingDoubleVectorReader].dropPositions(ctrVecAcc, ctrVecPtr) - for {i <- 0 until drops.length optimized} { + cforRange { 0 until drops.length } { i => if (drops(i) <= endRow) { result += drops(i) - 1 result += drops(i) diff --git a/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordBuilder.scala b/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordBuilder.scala index 6b164eb557..6b223ad310 100644 --- a/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordBuilder.scala @@ -2,7 +2,7 @@ package filodb.core.legacy.binaryrecord import com.typesafe.scalalogging.StrictLogging import org.agrona.DirectBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.metadata.{Column, Dataset} import filodb.core.metadata.Column.ColumnType.{DoubleColumn, LongColumn, MapColumn, StringColumn} @@ -236,7 +236,7 @@ final class RecordBuilder(memFactory: MemFactory, */ final def addFromReader(row: RowReader): Long = { startNewRecord() - for { pos <- 0 until schema.numFields optimized } { + cforRange { 0 until schema.numFields } { pos => schema.builderAdders(pos)(row, this) } endRecord() @@ -557,7 +557,7 @@ object RecordBuilder { final def sortAndComputeHashes(pairs: java.util.ArrayList[(String, String)]): Array[Int] = { pairs.sort(stringPairComparator) val hashes = new Array[Int](pairs.size) - for { i <- 0 until pairs.size optimized } { + cforRange { 0 until pairs.size } { i => val (k, v) = pairs.get(i) // This is not very efficient, we have to convert String to bytes first to get the hash // TODO: work on different API which is far more efficient and saves memory allocation @@ -586,7 +586,7 @@ object RecordBuilder { hashes: Array[Int], excludeKeys: Set[String]): Int = { var hash = 7 - for { i <- 0 until sortedPairs.size optimized } { + cforRange { 0 until sortedPairs.size } { i => if (!(excludeKeys contains sortedPairs.get(i)._1)) hash = combineHash(hash, hashes(i)) } @@ -632,4 +632,4 @@ object RecordBuilder { } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordComparator.scala b/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordComparator.scala index bda1b4f017..229f0db2b1 100644 --- a/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordComparator.scala +++ b/core/src/main/scala/filodb.core/legacy/binaryrecord/RecordComparator.scala @@ -1,6 +1,6 @@ package filodb.core.legacy.binaryrecord -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.format.UnsafeUtils @@ -129,7 +129,7 @@ final class RecordComparator(ingestSchema: RecordSchema) { // adjust offsets to var fields val adjustment = partVarAreaOffset - ingestVarAreaOffset - for { i <- 0 until fixedAreaNumWords optimized } { + cforRange { 0 until fixedAreaNumWords } { i => if ((compareBitmap & (1 << i)) == 0) { // not a primitive field, but an offset to String or Map builder.adjustFieldOffset(i, adjustment) } @@ -137,4 +137,4 @@ final class RecordComparator(ingestSchema: RecordSchema) { builder.endRecord(writeHash = false) } -} \ 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 5827b62a57..e260e18ead 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -21,7 +21,7 @@ import org.apache.lucene.search.BooleanClause.Occur import org.apache.lucene.store.MMapDirectory import org.apache.lucene.util.{BytesRef, InfoStream} import org.apache.lucene.util.automaton.RegExp -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.{concurrentCache, DatasetRef} import filodb.core.Types.PartitionKey @@ -184,7 +184,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, */ def removePartKeys(partIds: debox.Buffer[Int]): Unit = { val terms = new util.ArrayList[BytesRef]() - for { i <- 0 until partIds.length optimized } { + cforRange { 0 until partIds.length } { i => terms.add(new BytesRef(partIds(i).toString.getBytes)) } indexWriter.deleteDocuments(new TermInSetQuery(PART_ID, terms)) @@ -317,7 +317,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, // Currently there is a bit of leak in abstraction of Binary Record processing in this class. luceneDocument.set(document) // threadlocal since we are not able to pass the document into mapconsumer - for { i <- 0 until numPartColumns optimized } { + cforRange { 0 until numPartColumns } { i => indexers(i).fromPartKey(partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset), partId) } // partId @@ -779,4 +779,4 @@ class LuceneMetricsRouter(ref: DatasetRef, shard: Int) extends InfoStream with S } override def isEnabled(component: String): Boolean = true override def close(): Unit = {} -} \ No newline at end of file +} diff --git a/core/src/main/scala/filodb.core/memstore/PartitionKeyIndex.scala b/core/src/main/scala/filodb.core/memstore/PartitionKeyIndex.scala index 935986b18b..b739044c52 100644 --- a/core/src/main/scala/filodb.core/memstore/PartitionKeyIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartitionKeyIndex.scala @@ -3,7 +3,7 @@ package filodb.core.memstore import com.googlecode.javaewah.{EWAHCompressedBitmap, IntIterator} import com.typesafe.scalalogging.StrictLogging import org.jctools.maps.NonBlockingHashMap -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.Types.PartitionKey import filodb.core.binaryrecord2.MapItemConsumer @@ -71,7 +71,7 @@ class PartitionKeyIndex(dataset: Dataset) extends StrictLogging { * Adds fields from a partition key to the index */ def addPartKey(base: Any, offset: Long, partIndex: Int): Unit = { - for { i <- 0 until numPartColumns optimized } { + cforRange { 0 until numPartColumns } { i => indexers(i).fromPartKey(base, offset, partIndex) } } @@ -134,4 +134,4 @@ class PartitionKeyIndex(dataset: Dataset) extends StrictLogging { def reset(): Unit = { indices.clear() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 871ae227c5..07a1d42413 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -1,7 +1,7 @@ package filodb.core.memstore import com.typesafe.scalalogging.StrictLogging -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.DatasetRef import filodb.core.Types._ @@ -140,7 +140,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { // we have reached maximum userTime in chunk. switch buffers, start a new chunk and ingest switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) } else { - for { col <- 0 until schema.numDataColumns optimized} { + cforRange { 0 until schema.numDataColumns } { col => currentChunks(col).addFromReaderNoNA(row, col) match { case r: VectorTooSmall => switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 5e9c194bb4..a2cfdc5c03 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -21,7 +21,7 @@ import monix.execution.{Scheduler, UncaughtExceptionReporter} import monix.execution.atomic.AtomicBoolean import monix.reactive.Observable import org.jctools.maps.NonBlockingHashMapLong -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.{ErrorResponse, _} import filodb.core.binaryrecord2._ @@ -114,7 +114,7 @@ object TimeSeriesShard { def writeMeta(addr: Long, partitionID: Int, info: ChunkSetInfo, vectors: Array[BinaryVectorPtr]): Unit = { UnsafeUtils.setInt(UnsafeUtils.ZeroPointer, addr, partitionID) ChunkSetInfo.copy(info, addr + 4) - for { i <- 0 until vectors.size optimized } { + cforRange { 0 until vectors.size } { i => ChunkSetInfo.setVectorPtr(addr + 4, i, vectors(i)) } } @@ -125,7 +125,7 @@ object TimeSeriesShard { def writeMeta(addr: Long, partitionID: Int, bytes: Array[Byte], vectors: Array[BinaryVectorPtr]): Unit = { UnsafeUtils.setInt(UnsafeUtils.ZeroPointer, addr, partitionID) ChunkSetInfo.copy(bytes, addr + 4) - for { i <- 0 until vectors.size optimized } { + cforRange { 0 until vectors.size } { i => ChunkSetInfo.setVectorPtr(addr + 4, i, vectors(i)) } } @@ -135,7 +135,7 @@ object TimeSeriesShard { */ def writeMetaWithoutPartId(addr: Long, bytes: Array[Byte], vectors: Array[BinaryVectorPtr]): Unit = { ChunkSetInfo.copy(bytes, addr) - for { i <- 0 until vectors.size optimized } { + cforRange { 0 until vectors.size } { i => ChunkSetInfo.setVectorPtr(addr, i, vectors(i)) } } @@ -809,7 +809,7 @@ class TimeSeriesShard(val ref: DatasetRef, var newTimestamp = ingestionTime if (newTimestamp > oldTimestamp && oldTimestamp != Long.MinValue) { - for (group <- 0 until numGroups optimized) { + cforRange ( 0 until numGroups ) { group => /* Logically, the task creation filter is as follows: // Compute the time offset relative to the group number. 0 min, 1 min, 2 min, etc. diff --git a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala index 4d9a705d88..74ddcadb89 100644 --- a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala +++ b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala @@ -2,7 +2,7 @@ package filodb.core.memstore import com.typesafe.scalalogging.StrictLogging import org.jctools.queues.MpscUnboundedArrayQueue -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.metadata.DataSchema import filodb.core.store.{ChunkSetInfo, StoreConfig} @@ -46,7 +46,7 @@ class WriteBufferPool(memFactory: MemFactory, val builders = MemStore.getAppendables(memFactory, schema, storeConf) val info = ChunkSetInfo(memFactory, schema, 0, 0, 0, Long.MaxValue) // Point vectors in chunkset metadata to builders addresses - for { colNo <- 0 until schema.columns.length optimized } { + cforRange { 0 until schema.columns.length } { colNo => ChunkSetInfo.setVectorPtr(info.infoAddr, colNo, builders(colNo).addr) } queue.add((info.infoAddr, builders)) @@ -77,7 +77,7 @@ class WriteBufferPool(memFactory: MemFactory, def release(metaAddr: NativePointer, appenders: AppenderArray): Unit = { if (poolSize >= storeConf.maxBufferPoolSize) { // pool is at max size, release extra so memory can be shared. Be sure to release each vector's memory - for { colNo <- 0 until schema.columns.length optimized } { + cforRange { 0 until schema.columns.length } { colNo => memFactory.freeMemory(ChunkSetInfo.getVectorPtr(metaAddr, colNo)) } memFactory.freeMemory(metaAddr) diff --git a/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala b/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala index e1e4a9a8d5..a57d3a1ace 100644 --- a/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala +++ b/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala @@ -1,7 +1,5 @@ package filodb.core.metadata -import scala.language.existentials - import com.typesafe.config.Config import org.scalactic._ @@ -163,4 +161,4 @@ trait SingleColumnComputation extends ColumnComputation { colType: Column.ColumnType, extractor: TypedFieldExtractor[_]): ComputedColumn = computedColumn(expr, dataset, Seq(c.colIndex), colType, extractor) -} \ No newline at end of file +} diff --git a/core/src/main/scala/filodb.core/query/KeyFilter.scala b/core/src/main/scala/filodb.core/query/KeyFilter.scala index b10a59533f..084800e7dc 100644 --- a/core/src/main/scala/filodb.core/query/KeyFilter.scala +++ b/core/src/main/scala/filodb.core/query/KeyFilter.scala @@ -121,7 +121,7 @@ object KeyFilter { // val funcs = positionsAndFuncs.collect { case (pos, func) if pos >= 0 => func }.toArray // def partFunc(p: PartitionKey): Boolean = { - // for { i <- 0 until positions.size optimized } { + // cforRange { 0 until positions.size } { i => // val bool = funcs(i)(p.getAny(positions(i))) // // Short circuit when any filter returns false // if (!bool) return false diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala index a115b88aa4..e4b336c5da 100644 --- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala +++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala @@ -1,6 +1,6 @@ package filodb.core.query -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.metadata.Dataset import filodb.core.store.{ChunkInfoIterator, ChunkSetInfoReader, ReadablePartition} @@ -45,7 +45,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, private def populateIterators(info: ChunkSetInfoReader): Unit = { setChunkStartEnd(info) - for { pos <- 0 until columnIDs.size optimized } { + cforRange { 0 until columnIDs.size } { pos => val colID = columnIDs(pos) if (Dataset.isPartitionID(colID)) { // Look up the TypedIterator for that partition key @@ -101,4 +101,4 @@ final class PartitionTimeRangeReader(part: ReadablePartition, final def close(): Unit = { infos.close() } -} \ No newline at end of file +} diff --git a/gateway/src/main/scala/filodb/gateway/conversion/InfluxProtocolParser.scala b/gateway/src/main/scala/filodb/gateway/conversion/InfluxProtocolParser.scala index 69cf933453..1c53b35df2 100644 --- a/gateway/src/main/scala/filodb/gateway/conversion/InfluxProtocolParser.scala +++ b/gateway/src/main/scala/filodb/gateway/conversion/InfluxProtocolParser.scala @@ -1,11 +1,9 @@ package filodb.gateway.conversion -import scala.language.postfixOps - import com.typesafe.scalalogging.StrictLogging import debox.Buffer import org.jboss.netty.buffer.ChannelBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.binaryrecord2.RecordBuilder import filodb.memory.format.UnsafeUtils @@ -169,7 +167,7 @@ object InfluxProtocolParser extends StrictLogging { // relies on offsets being in the same order as keys def parseKeyValues(bytes: Array[Byte], offsets: Buffer[Int], endOffset: Int, visitor: KVVisitor): Unit = { val last = offsets.length - 1 - for { i <- 0 to last optimized } { + cforRange { 0 to last } { i => val fieldEnd = if (i < last) keyOffset(offsets(i + 1)) - 1 else endOffset val curOffsetInt = offsets(i) val valueOffset = valOffset(curOffsetInt) @@ -233,4 +231,4 @@ object InfluxProtocolParser extends StrictLogging { } dblNum } -} \ No newline at end of file +} diff --git a/gateway/src/main/scala/filodb/gateway/legacy/conversion/InputRecord.scala b/gateway/src/main/scala/filodb/gateway/legacy/conversion/InputRecord.scala index 38e170b0d5..1dc709f416 100644 --- a/gateway/src/main/scala/filodb/gateway/legacy/conversion/InputRecord.scala +++ b/gateway/src/main/scala/filodb/gateway/legacy/conversion/InputRecord.scala @@ -1,9 +1,7 @@ package filodb.gateway.legacy.conversion -import scala.language.postfixOps - import remote.RemoteStorage.TimeSeries -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.legacy.binaryrecord.RecordBuilder import filodb.core.metadata.Dataset @@ -79,7 +77,7 @@ case class PrometheusInputRecord(tags: Map[String, String], val metricBytes = metric.getBytes builder.addMapKeyValueHash(dataset.options.metricBytes, dataset.options.metricHash, metricBytes, 0, metricBytes.size) - for { i <- 0 until javaTags.size optimized } { + cforRange { 0 until javaTags.size } { i => val (k, v) = javaTags.get(i) builder.addMapKeyValue(k.getBytes, v.getBytes) builder.updatePartitionHash(hashes(i)) diff --git a/jmh/src/main/scala/filodb.jmh/BasicFiloBenchmark.scala b/jmh/src/main/scala/filodb.jmh/BasicFiloBenchmark.scala index d06d3bb98f..c117522729 100644 --- a/jmh/src/main/scala/filodb.jmh/BasicFiloBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/BasicFiloBenchmark.scala @@ -2,10 +2,8 @@ package filodb.jmh import java.util.concurrent.TimeUnit -import scala.language.postfixOps - import org.openjdk.jmh.annotations._ -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.NativeMemoryManager import filodb.memory.format.MemoryReader._ @@ -62,7 +60,7 @@ class BasicFiloBenchmark { def sumAllLongsApply(): Long = { var total = 0L val acc2 = acc // local variable to make the scala compiler not use virtual invoke - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => total += ivReader(acc2, iv, i) } total @@ -74,7 +72,7 @@ class BasicFiloBenchmark { def sumAllLongsIterate(): Long = { var total = 0L val it = ivReader.iterate(acc, iv) - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => total += it.next } total @@ -107,7 +105,7 @@ class BasicFiloBenchmark { def sumTimeSeriesBytesApply(): Long = { var total = 0L val acc2 = acc // local variable to make the scala compiler not use virtual invoke - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => total += byteReader(acc2, byteVect, i) } total @@ -119,7 +117,7 @@ class BasicFiloBenchmark { def sumTimeSeriesBytesIterate(): Long = { var total = 0L val it = byteReader.iterate(acc, byteVect) - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => total += it.next } total diff --git a/jmh/src/main/scala/filodb.jmh/DictStringBenchmark.scala b/jmh/src/main/scala/filodb.jmh/DictStringBenchmark.scala index 6b6da60322..89b6fe1a98 100644 --- a/jmh/src/main/scala/filodb.jmh/DictStringBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/DictStringBenchmark.scala @@ -2,10 +2,8 @@ package filodb.jmh import java.util.concurrent.TimeUnit -import scala.language.postfixOps - import org.openjdk.jmh.annotations._ -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.NativeMemoryManager import filodb.memory.format._ @@ -58,7 +56,7 @@ class DictStringBenchmark { @OutputTimeUnit(TimeUnit.MICROSECONDS) def rawStringLengthTotal(): Int = { var totalLen = 0 - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => totalLen += scNoNA(i).length } totalLen @@ -75,9 +73,9 @@ class DictStringBenchmark { val reader = UTF8Vector(acc, scNAPtr) val it = reader.iterate(acc, scNAPtr) - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => totalLen += it.next.length } totalLen } -} \ No newline at end of file +} diff --git a/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala b/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala index c6894bbf2e..261a7762a1 100644 --- a/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala @@ -2,13 +2,11 @@ package filodb.jmh import java.util.concurrent.TimeUnit -import scala.language.postfixOps - import org.openjdk.jmh.annotations.{Mode, Scope, State} import org.openjdk.jmh.annotations.Benchmark import org.openjdk.jmh.annotations.BenchmarkMode import org.openjdk.jmh.annotations.OutputTimeUnit -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.NativeMemoryManager import filodb.memory.format._ @@ -53,7 +51,7 @@ class EncodingBenchmark { @OutputTimeUnit(TimeUnit.SECONDS) def newIntVectorEncoding(): Unit = { val cb = IntBinaryVector.appendingVector(memFactory, numValues) - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => cb.addData(intArray(i)) } cb.optimize(memFactory) @@ -69,7 +67,7 @@ class EncodingBenchmark { @OutputTimeUnit(TimeUnit.SECONDS) def growableIntVectorAddData(): Unit = { cbAdder.reset() - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => cbAdder.addData(intArray(i)) } } @@ -81,7 +79,7 @@ class EncodingBenchmark { @OutputTimeUnit(TimeUnit.SECONDS) def noNAIntVectorAddData(): Unit = { noNAAdder.reset() - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => noNAAdder.addData(intArray(i)) } } @@ -93,7 +91,7 @@ class EncodingBenchmark { @OutputTimeUnit(TimeUnit.SECONDS) def newUtf8VectorEncoding(): Unit = { val cb = UTF8Vector.appendingVector(memFactory, numValues, maxStringLength * numUniqueStrings) - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => cb.addData(utf8strings(i)) } cb.optimize(memFactory) @@ -102,7 +100,7 @@ class EncodingBenchmark { // TODO: RowReader based vector building val utf8cb = UTF8Vector.appendingVector(memFactory, numValues, maxStringLength * numUniqueStrings) - for { i <- 0 until numValues optimized } { + cforRange { 0 until numValues } { i => utf8cb.addData(utf8strings(i)) } @@ -122,4 +120,4 @@ class EncodingBenchmark { val hint = Encodings.AutoDictString(samplingRate = 0.5) UTF8Vector(memFactory, utf8strings).optimize(memFactory, hint) } -} \ No newline at end of file +} diff --git a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala index 46b4ec9138..e4eb963d14 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala @@ -2,14 +2,12 @@ package filodb.jmh import java.util.concurrent.TimeUnit -import scala.language.postfixOps - import ch.qos.logback.classic.{Level, Logger} import com.typesafe.config.ConfigFactory import org.agrona.{ExpandableArrayBuffer, ExpandableDirectByteBuffer} import org.agrona.concurrent.UnsafeBuffer import org.openjdk.jmh.annotations.{Level => JMHLevel, _} -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.{MachineMetricsData, MetricsTestData, TestData} import filodb.core.binaryrecord2.RecordBuilder @@ -188,7 +186,7 @@ class HistogramIngestBenchmark { ddsink.writePos = 0 java.util.Arrays.fill(ddsink.lastHistDeltas, 0) var lastPos = 0 - for { i <- 0 until numInputs optimized } { + cforRange { 0 until numInputs } { i => ddSlice.wrap(increasingBuf, lastPos, increasingHistPos(i) - lastPos) val res = NibblePack.unpackToSink(ddSlice, ddsink, inputs.size) require(res == NibblePack.Ok) @@ -196,4 +194,4 @@ class HistogramIngestBenchmark { ddsink.reset() } } -} \ No newline at end of file +} diff --git a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala index 4238390b06..37dc562614 100644 --- a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala @@ -2,11 +2,9 @@ package filodb.jmh import java.util.concurrent.TimeUnit -import scala.language.postfixOps - import ch.qos.logback.classic.{Level, Logger} import org.openjdk.jmh.annotations._ -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.{NamesTestData, TestData} import filodb.core.metadata.{Dataset, DatasetOptions} @@ -52,7 +50,7 @@ class IntSumReadBenchmark { def applyVectorScan(): Int = { var total = 0 val acc2 = acc // local variable to make the scala compiler not use virtual invoke - for { i <- 0 until NumRows optimized } { + cforRange { 0 until NumRows } { i => total += intReader(acc2, intVectAddr, i) } total @@ -67,7 +65,7 @@ class IntSumReadBenchmark { def iterateScan(): Int = { val it = intReader.iterate(acc, intVectAddr, 0) var sum = 0 - for { i <- 0 until NumRows optimized } { + cforRange { 0 until NumRows } { i => sum += it.next } sum diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index 8067b8f7f7..8aad7cc6d1 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -3,11 +3,10 @@ package filodb.jmh import java.util.concurrent.TimeUnit import scala.concurrent.duration._ -import scala.language.postfixOps import ch.qos.logback.classic.{Level, Logger} import org.openjdk.jmh.annotations._ -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.DatasetRef import filodb.core.binaryrecord2.RecordBuilder @@ -53,7 +52,7 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithEqualsFilters(): Unit = { - for ( i <- 0 until 8 optimized) { + cforRange ( 0 until 8 ) { i => partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), ColumnFilter("_ws_", Filter.Equals("demo")), @@ -68,7 +67,7 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def emptyPartIdsLookupWithEqualsFilters(): Unit = { - for ( i <- 0 until 8 optimized) { + cforRange ( 0 until 8 ) { i => partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-${i + 200}")), ColumnFilter("_ws_", Filter.Equals("demo")), @@ -83,7 +82,7 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithSuffixRegexFilters(): Unit = { - for ( i <- 0 until 8 optimized) { + cforRange ( 0 until 8 ) { i => partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), ColumnFilter("_ws_", Filter.Equals("demo")), @@ -98,7 +97,7 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithPrefixRegexFilters(): Unit = { - for ( i <- 0 until 8 optimized) { + cforRange ( 0 until 8 ) { i => partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), ColumnFilter("_ws_", Filter.Equals("demo")), @@ -113,9 +112,9 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def startTimeLookupWithPartId(): Unit = { - for ( i <- 0 until 8 optimized) { + cforRange ( 0 until 8 ) { i => val pIds = debox.Buffer.empty[Int] - for ( j <- i * 1000 to i * 1000 + 1000 optimized) { pIds += j } + cforRange ( i * 1000 to i * 1000 + 1000 ) { j => pIds += j } partKeyIndex.startTimeFromPartIds(pIds.iterator()) } } diff --git a/jmh/src/main/scala/filodb.jmh/PartitionListBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartitionListBenchmark.scala index 0e3c81cac6..88f1c3626f 100644 --- a/jmh/src/main/scala/filodb.jmh/PartitionListBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartitionListBenchmark.scala @@ -2,10 +2,8 @@ package filodb.jmh import java.util.concurrent.TimeUnit -import scala.language.postfixOps - import org.openjdk.jmh.annotations._ -import scalaxy.loops._ +import spire.syntax.cfor._ /** * Microbenchmark involving comparison of data structures for holding partition lists in a shard. @@ -18,10 +16,10 @@ import scalaxy.loops._ @State(Scope.Thread) class PartitionListBenchmark { val lhm = new collection.mutable.LinkedHashMap[Int, String] - for { i <- 0 until 1000000 optimized } { lhm(i) = "shoo" } + cforRange { 0 until 1000000 } { i => lhm(i) = "shoo" } val jlhm = new java.util.LinkedHashMap[Int, String] - for { i <- 0 until 1000000 optimized } { jlhm.put(i, "shoo") } + cforRange { 0 until 1000000 } { i => jlhm.put(i, "shoo") } val abuf = collection.mutable.ArrayBuffer.fill(1000000)("shoo") @@ -52,8 +50,8 @@ class PartitionListBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def javaLinkedMapRemove100(): Unit = { - for { i <- 5100 to 5199 optimized } { + cforRange { 5100 to 5199 } { i => jlhm.remove(i) } } -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala index 7b01ad5f85..7b54a46962 100644 --- a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala @@ -3,7 +3,7 @@ package filodb.memory.format import java.nio.ByteBuffer import debox.Buffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format.Encodings._ @@ -331,7 +331,7 @@ trait BinaryAppendableVector[@specialized(Int, Long, Double, Boolean) A] { * The default implementation is generic but inefficient: does not take advantage of data layout */ def addVector(other: BinaryAppendableVector[A]): Unit = { - for { i <- 0 until other.length optimized } { + cforRange { 0 until other.length } { i => if (other.isAvailable(i)) { addData(other(i))} else { addNA() } } @@ -571,7 +571,7 @@ extends OptimizingPrimitiveAppender[A] { // Optimization: this vector does not support NAs so just add the data require(numBytes + (nbits * v.length / 8) <= maxBytes, s"Not enough space to add ${v.length} elems; nbits=$nbits; need ${maxBytes-numBytes} bytes") - for { i <- 0 until v.length optimized } { addData(v(i)) } + cforRange { 0 until v.length } { i => addData(v(i)) } } final def isAllNA: Boolean = (length == 0) @@ -669,7 +669,7 @@ extends BinaryAppendableVector[A] { } final def isAllNA: Boolean = { - for { word <- 0 until curBitmapOffset/8 optimized } { + cforRange { 0 until curBitmapOffset/8 } { word => if (nativePtrReader.getLong(bitmapOffset + word * 8) != -1L) return false } val naMask = curMask - 1 @@ -677,7 +677,7 @@ extends BinaryAppendableVector[A] { } final def noNAs: Boolean = { - for { word <- 0 until curBitmapOffset/8 optimized } { + cforRange { 0 until curBitmapOffset/8 } { word => if (nativePtrReader.getLong(bitmapOffset + word * 8) != 0) return false } val naMask = curMask - 1 diff --git a/memory/src/main/scala/filodb.memory/format/NibblePack.scala b/memory/src/main/scala/filodb.memory/format/NibblePack.scala index 6a58b986e9..7e00cf5bf5 100644 --- a/memory/src/main/scala/filodb.memory/format/NibblePack.scala +++ b/memory/src/main/scala/filodb.memory/format/NibblePack.scala @@ -3,7 +3,7 @@ package filodb.memory.format import java.nio.ByteOrder.LITTLE_ENDIAN import org.agrona.{DirectBuffer, MutableDirectBuffer} -import scalaxy.loops._ +import spire.syntax.cfor._ /** * An implementation of the NibblePack algorithm for efficient encoding, see [compression.md](doc/compression.md). @@ -56,7 +56,7 @@ object NibblePack { @inline private def packRemainder(input: Array[Long], buf: MutableDirectBuffer, pos: Int, i: Int): Int = if (i % 8 != 0) { - for { j <- (i % 8) until 8 optimized } { input(j) = 0 } + cforRange { (i % 8) until 8 } { j => input(j) = 0 } pack8(input, buf, pos) } else { pos @@ -87,7 +87,7 @@ object NibblePack { // Flush remainder - if any left if (i % 8 != 0) { - for { j <- (i % 8) until 8 optimized } { inputArray(j) = 0 } + cforRange { (i % 8) until 8 } { j => inputArray(j) = 0 } pos = pack8(inputArray, buf, pos) } @@ -108,7 +108,7 @@ object NibblePack { var bitmask = 0 // Figure out which words are nonzero, pack bitmask - for { i <- 0 until 8 optimized } { + cforRange { 0 until 8 } { i => if (input(i) != 0) bitmask |= 1 << i } buf.putByte(bufpos, bitmask.toByte) @@ -118,7 +118,7 @@ object NibblePack { // figure out min # of nibbles to represent nonzero words var minLeadingZeros = 64 var minTrailingZeros = 64 - for { i <- 0 until 8 optimized } { + cforRange { 0 until 8 } { i => minLeadingZeros = Math.min(minLeadingZeros, java.lang.Long.numberOfLeadingZeros(input(i))) minTrailingZeros = Math.min(minTrailingZeros, java.lang.Long.numberOfTrailingZeros(input(i))) } @@ -145,7 +145,7 @@ object NibblePack { var outWord = 0L var bitCursor = 0 - for { i <- 0 until 8 optimized } { + cforRange { 0 until 8 } { i => val input = inputs(i) if (input != 0) { val remaining = 64 - bitCursor @@ -210,7 +210,7 @@ object NibblePack { // user might not intuitively allocate output arrays in elements of 8. val numElems = Math.min(outArray.size - i, 8) require(numElems > 0) - for { n <- 0 until numElems optimized } { + cforRange { 0 until numElems } { n => current += data(n) outArray(i + n) = current } @@ -227,7 +227,7 @@ object NibblePack { private var pos: Int = 1 def process(data: Array[Long]): Unit = { val numElems = Math.min(outArray.size - pos, 8) - for { n <- 0 until numElems optimized } { + cforRange { 0 until numElems } { n => val nextBits = lastBits ^ data(n) outArray(pos + n) = java.lang.Double.longBitsToDouble(nextBits) lastBits = nextBits @@ -271,7 +271,7 @@ object NibblePack { final def process(data: Array[Long]): Unit = { val numElems = Math.min(lastHistDeltas.size - i, 8) - for { n <- 0 until numElems optimized } { + cforRange { 0 until numElems } { n => val diff = data(n) - lastHistDeltas(i + n) if (diff < 0) valueDropped = true packArray(n) = diff @@ -310,7 +310,7 @@ object NibblePack { final def process(data: Array[Long]): Unit = { val numElems = Math.min(numBuckets - i, 8) - for { n <- 0 until numElems optimized } { + cforRange { 0 until numElems } { n => if (data(n) < lastHistDeltas(i + n)) valueDropped = true packArray(n) = data(n) - originalDeltas(i + n) } @@ -389,7 +389,7 @@ object NibblePack { var inWord = readLong(compressed, bufIndex) bufIndex += 8 - for { bit <- 0 until 8 optimized } { + cforRange { 0 until 8 } { bit => if ((nonzeroMask & (1 << bit)) != 0) { val remaining = 64 - bitCursor @@ -446,4 +446,4 @@ object NibblePack { outWord } } -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/format/RowReader.scala b/memory/src/main/scala/filodb.memory/format/RowReader.scala index b29270802f..ce7872262e 100644 --- a/memory/src/main/scala/filodb.memory/format/RowReader.scala +++ b/memory/src/main/scala/filodb.memory/format/RowReader.scala @@ -8,7 +8,7 @@ import scala.reflect.ClassTag import org.agrona.DirectBuffer import org.agrona.concurrent.UnsafeBuffer import org.joda.time.DateTime -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.format.vectors.Histogram @@ -88,7 +88,7 @@ trait SchemaRowReader extends RowReader { // or slow functional code here. override def hashCode: Int = { var hash = 0 - for { i <- 0 until extractors.size optimized } { + cforRange { 0 until extractors.size } { i => hash ^= extractors(i).getField(this, i).hashCode } hash @@ -96,7 +96,7 @@ trait SchemaRowReader extends RowReader { override def equals(other: Any): Boolean = other match { case reader: RowReader => - for { i <- 0 until extractors.size optimized } { + cforRange { 0 until extractors.size } { i => if (extractors(i).compare(this, reader, i) != 0) return false } true diff --git a/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala b/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala index ca123ee1b5..9c9c0b960b 100644 --- a/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala +++ b/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala @@ -2,9 +2,7 @@ package filodb.memory.format import java.nio.ByteBuffer -import scala.language.existentials - -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.MemFactory import filodb.memory.format.Encodings.{AutoDetect, EncodingHint} @@ -73,7 +71,7 @@ class RowToVectorBuilder(schema: Seq[VectorInfo], memFactory: MemFactory) { * @param row the row of data to transpose. Each column will be added to the right Builders. */ def addRow(row: RowReader): Unit = { - for { i <- 0 until numColumns optimized } { + cforRange { 0 until numColumns } { i => builders(i).addFromReader(row, i) } } @@ -86,4 +84,4 @@ class RowToVectorBuilder(schema: Seq[VectorInfo], memFactory: MemFactory) { private def unsupportedInput(typ: Any) = throw new RuntimeException("Unsupported input type " + typ) -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala b/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala index 315904a7ab..7f2766e061 100644 --- a/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala +++ b/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala @@ -4,8 +4,7 @@ import java.nio.ByteBuffer import com.kenai.jffi.MemoryIO import org.agrona.DirectBuffer -import scalaxy.loops._ - +import spire.implicits.cforRange // scalastyle:off number.of.methods object UnsafeUtils { val unsafe = scala.concurrent.util.Unsafe.instance @@ -145,7 +144,7 @@ object UnsafeUtils { if (wordComp == 0) { var pointer1 = offset1 + minLenAligned var pointer2 = offset2 + minLenAligned - for { i <- minLenAligned until minLen optimized } { + cforRange { minLenAligned until minLen } { _ => val res = (getByte(base1, pointer1) & 0xff) - (getByte(base2, pointer2) & 0xff) if (res != 0) return res pointer1 += 1 diff --git a/memory/src/main/scala/filodb.memory/format/ZeroCopyBinary.scala b/memory/src/main/scala/filodb.memory/format/ZeroCopyBinary.scala index 990a97ee24..605b50ea2b 100644 --- a/memory/src/main/scala/filodb.memory/format/ZeroCopyBinary.scala +++ b/memory/src/main/scala/filodb.memory/format/ZeroCopyBinary.scala @@ -1,7 +1,7 @@ package filodb.memory.format import net.jpountz.xxhash.XXHashFactory -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{MemFactory, UTF8StringMedium} @@ -29,7 +29,7 @@ trait ZeroCopyBinary extends Ordered[ZeroCopyBinary] { val minLenAligned = minLen & -4 val wordComp = UnsafeUtils.wordCompare(base, offset, other.base, other.offset, minLenAligned) if (wordComp == 0) { - for { i <- minLenAligned until minLen optimized } { + cforRange { minLenAligned until minLen } { i => val res = getByte(i) - other.getByte(i) if (res != 0) return res } @@ -174,7 +174,7 @@ final class ZeroCopyUTF8String(val base: Any, val offset: Long, val numBytes: In if (substring.numBytes == 0) { true } else { val firstByte = substring.getByte(0) - for { i <- 0 to (numBytes - substring.numBytes) optimized } { + cforRange { 0 to (numBytes - substring.numBytes) } { i => if (getByte(i) == firstByte && matchAt(substring, i)) return true } false @@ -233,4 +233,4 @@ object ZeroCopyUTF8String { implicit object ZeroCopyUTF8BinaryOrdering extends Ordering[ZeroCopyUTF8String] { def compare(a: ZeroCopyUTF8String, b: ZeroCopyUTF8String): Int = a.compare(b) } -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala index 1c4c37d613..918bd6ce58 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala @@ -1,7 +1,7 @@ package filodb.memory.format.vectors import debox.Buffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format._ @@ -119,7 +119,7 @@ object DeltaDeltaVector { var baseValue: Long = inputVect(0) var max = Int.MinValue var min = Int.MaxValue - for { i <- 1 until inputVect.length optimized } { + cforRange { 1 until inputVect.length } { i => baseValue += slope val delta = inputVect(i) - baseValue if (delta > Int.MaxValue || delta < Int.MinValue) return None // will not fit in 32 bits, just quit @@ -216,7 +216,7 @@ object DeltaDeltaDataReader extends LongVectorDataReader { val itr = iterate(acc, vector, start) var prevVector: Long = prev var changes = 0 - for {i <- start until end + 1 optimized} { + cforRange { start until end + 1 } { i => val cur = itr.next if (i == start && ignorePrev) //Initialize prev prevVector = cur diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala index 36ecc91437..a6f0556dea 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala @@ -2,7 +2,7 @@ package filodb.memory.format.vectors import java.util.HashMap -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.MemFactory import filodb.memory.format._ @@ -46,7 +46,7 @@ object DictUTF8Vector { val codeVect = IntBinaryVector.appendingVectorNoNA(memFactory, sourceLen) dictVect.addNA() // first code point 0 == NA - for { i <- 0 until sourceLen optimized } { + cforRange { 0 until sourceLen } { i => if (sourceVector.isAvailable(i)) { val item = sourceVector(i) val newCode = codeMap.size + 1 diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala index 48fb52013a..3e72de6444 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -3,7 +3,7 @@ package filodb.memory.format.vectors import java.nio.ByteBuffer import debox.Buffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format._ @@ -167,7 +167,7 @@ trait DoubleVectorDataReader extends CounterVectorReader { val dataIt = iterate(acc, vector, startElement) val availIt = iterateAvailable(acc, vector, startElement) val len = length(acc, vector) - for { n <- startElement until len optimized } { + cforRange { startElement until len } { n => val item = dataIt.next if (availIt.next) newBuf += item } @@ -320,7 +320,7 @@ extends DoubleVectorDataReader { val _corrected = new Array[Double](length(acc, vect)) val it = iterate(acc, vect, 0) var last = Double.MinValue - for { pos <- 0 until length(acc, vect) optimized } { + cforRange { 0 until length(acc, vect) } { pos => val nextVal = it.next if (nextVal < last) { // reset! _correction += last @@ -400,7 +400,7 @@ extends PrimitiveAppendableVector[Double](addr, maxBytes, 64, true) { final def minMax: (Double, Double) = { var min = Double.MaxValue var max = Double.MinValue - for { index <- 0 until length optimized } { + cforRange { 0 until length } { index => val data = apply(index) if (data < min) min = data if (data > max) max = data @@ -457,7 +457,7 @@ BitmapMaskAppendableVector[Double](addr, maxElements) with OptimizingPrimitiveAp final def minMax: (Double, Double) = { var min = Double.MaxValue var max = Double.MinValue - for { index <- 0 until length optimized } { + cforRange { 0 until length } { index => if (isAvailable(index)) { val data = subVect.apply(index) if (data < min) min = data @@ -489,7 +489,7 @@ extends AppendableVectorWrapper[Long, Double] { val MaxLongDouble = Long.MaxValue.toDouble final def nonIntegrals: Int = { var nonInts = 0 - for { index <- 0 until length optimized } { + cforRange { 0 until length } { index => if (inner.isAvailable(index)) { val data = inner.apply(index) if (data > MaxLongDouble || (Math.rint(data) != data)) nonInts += 1 diff --git a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala index f37e41d245..102230a4da 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -3,7 +3,7 @@ package filodb.memory.format.vectors import java.nio.ByteOrder.LITTLE_ENDIAN import org.agrona.{DirectBuffer, MutableDirectBuffer} -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.format._ @@ -93,10 +93,10 @@ trait Histogram extends Ordered[Histogram] { */ def compare(other: Histogram): Int = { if (numBuckets != other.numBuckets) return topBucketValue compare other.topBucketValue - for { b <- 0 until numBuckets optimized } { + cforRange { 0 until numBuckets } { b => if (bucketTop(b) != other.bucketTop(b)) return topBucketValue compare other.topBucketValue } - for { b <- (numBuckets - 1) to 0 by -1 optimized } { + cforRange { (numBuckets - 1) to 0 by -1 } {b => val countComp = bucketValue(b) compare other.bucketValue(b) if (countComp != 0) return countComp } @@ -113,7 +113,7 @@ trait Histogram extends Ordered[Histogram] { override def hashCode: Int = { var hash = 7.0 - for { b <- 0 until numBuckets optimized } { + cforRange { 0 until numBuckets } { b => hash = (31 * bucketTop(b) + hash) * 31 + bucketValue(b) } java.lang.Double.doubleToLongBits(hash).toInt @@ -130,7 +130,7 @@ trait HistogramWithBuckets extends Histogram { final def bucketTop(no: Int): Double = buckets.bucketTop(no) final def valueArray: Array[Double] = { val values = new Array[Double](numBuckets) - for { b <- 0 until numBuckets optimized } { + cforRange { 0 until numBuckets } { b => values(b) = bucketValue(b) } values @@ -155,7 +155,7 @@ final case class LongHistogram(buckets: HistogramBuckets, values: Array[Long]) e */ final def add(other: LongHistogram): Unit = { assert(other.buckets == buckets) - for { b <- 0 until numBuckets optimized } { + cforRange { 0 until numBuckets } { b => values(b) += other.values(b) } } @@ -208,7 +208,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl case m: MutableHistogram => System.arraycopy(m.values, 0, values, 0, values.size) case l: LongHistogram => - for { n <- 0 until values.size optimized } { + cforRange { 0 until values.size } { n => values(n) = l.values(n).toDouble } } @@ -225,7 +225,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl if (buckets.similarForMath(other.buckets)) { // If it was NaN before, reset to 0 to sum another hist if (values(0).isNaN) java.util.Arrays.fill(values, 0.0) - for { b <- 0 until numBuckets optimized } { + cforRange { 0 until numBuckets } { b => values(b) += other.bucketValue(b) } } else { @@ -234,7 +234,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl // NOTE: there are two issues here: below add picks the existing bucket scheme (not commutative) // and the newer different buckets are lost (one may want more granularity) // var ourBucketNo = 0 - // for { b <- 0 until other.numBuckets optimized } { + // cforRange { 0 until other.numBuckets } { b => // // Find our first bucket greater than or equal to their bucket // while (ourBucketNo < numBuckets && bucketTop(ourBucketNo) < other.bucketTop(b)) ourBucketNo += 1 // if (ourBucketNo < numBuckets) { @@ -260,7 +260,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl */ final def makeMonotonic(): Unit = { var max = 0d - for { b <- 0 until values.size optimized } { + cforRange { 0 until values.size } { b => // When bucket no longer used NaN will be seen. Non-increasing values can be seen when // newer buckets are introduced and not all instances are updated with that bucket. if (values(b) < max || values(b).isNaN) values(b) = max // assign previous max @@ -345,7 +345,7 @@ sealed trait HistogramBuckets { */ final def allBucketTops: Array[Double] = { val tops = new Array[Double](numBuckets) - for { b <- 0 until numBuckets optimized } { + cforRange { 0 until numBuckets } { b => tops(b) = bucketTop(b) } tops @@ -353,7 +353,7 @@ sealed trait HistogramBuckets { final def bucketSet: debox.Set[Double] = { val tops = debox.Set.empty[Double] - for { b <- 0 until numBuckets optimized } { + cforRange { 0 until numBuckets } { b => tops += bucketTop(b) } tops diff --git a/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala b/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala index b3663ccad1..fe7edae94b 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala @@ -4,7 +4,7 @@ import scala.io.Source import org.agrona.{ExpandableArrayBuffer, ExpandableDirectByteBuffer} import org.agrona.concurrent.UnsafeBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.NativeMemoryManager import filodb.memory.format._ @@ -157,7 +157,7 @@ object HistogramIngestBench extends App with CompressorAnalyzer { ddsink.writePos = 0 java.util.Arrays.fill(ddsink.lastHistDeltas, 0) var lastPos = 0 - for { i <- 0 until numHist optimized } { + cforRange { 0 until numHist } { i => ddSlice.wrap(increasingBuf, lastPos, increasingHistPos(i) - lastPos) val res = NibblePack.unpackToSink(ddSlice, ddsink, bucketDef.numBuckets) require(res == NibblePack.Ok) @@ -189,7 +189,7 @@ object PromCompressor extends App with CompressorAnalyzer { histograms.foreach { h => numRecords += 1 - for { b <- 0 until bucketDef.numBuckets optimized } { + cforRange { 0 until bucketDef.numBuckets } { b => appenders(b).addData(h.values(b)) match { case Ack => // data added, no problem case VectorTooSmall(_, _) => // not enough space. Encode, aggregate, and add to a new appender @@ -213,4 +213,4 @@ object PromCompressor extends App with CompressorAnalyzer { } analyze() -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala index 5df2d5cf4b..a416f377f9 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -6,7 +6,7 @@ import com.typesafe.scalalogging.StrictLogging import debox.Buffer import org.agrona.{DirectBuffer, ExpandableArrayBuffer, MutableDirectBuffer} import org.agrona.concurrent.UnsafeBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format._ @@ -544,7 +544,7 @@ class RowHistogramReader(val acc: MemoryReader, histVect: Ptr.U8) extends Histog final def sum(start: Int, end: Int): MutableHistogram = { require(length > 0 && start >= 0 && end < length) val summedHist = MutableHistogram.empty(buckets) - for { i <- start to end optimized } { + cforRange { start to end } { i => summedHist.addNoCorrection(apply(i)) } summedHist @@ -658,4 +658,4 @@ class SectDeltaHistogramReader(acc2: MemoryReader, histVect: Ptr.U8) h } } -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala index 2976a2c1cf..b4e1e1de07 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala @@ -3,7 +3,7 @@ package filodb.memory.format.vectors import java.nio.ByteBuffer import debox.Buffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format._ @@ -295,7 +295,7 @@ trait IntVectorDataReader extends VectorDataReader { val dataIt = iterate(acc, vector, startElement) val availIt = iterateAvailable(acc, vector, startElement) val len = length(acc, vector) - for { n <- startElement until len optimized } { + cforRange { startElement until len } { n => val item = dataIt.next if (availIt.next) newBuf += item } @@ -490,7 +490,7 @@ extends PrimitiveAppendableVector[Int](addr, maxBytes, nbits, signed) { final def minMax: (Int, Int) = { var min = Int.MaxValue var max = Int.MinValue - for { index <- 0 until length optimized } { + cforRange { 0 until length } { index => val data = reader.apply(nativePtrReader, addr, index) if (data < min) min = data if (data > max) max = data @@ -523,7 +523,7 @@ BitmapMaskAppendableVector[Int](addr, maxElements) with OptimizingPrimitiveAppen final def minMax: (Int, Int) = { var min = Int.MaxValue var max = Int.MinValue - for { index <- 0 until length optimized } { + cforRange { 0 until length } { index => if (isAvailable(index)) { val data = subVect.apply(index) if (data < min) min = data diff --git a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala index 065d25a439..a4dfd6648a 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala @@ -3,7 +3,7 @@ package filodb.memory.format.vectors import java.nio.ByteBuffer import debox.Buffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format._ @@ -178,7 +178,7 @@ trait LongVectorDataReader extends VectorDataReader { val dataIt = iterate(acc, vector, startElement) val availIt = iterateAvailable(acc, vector, startElement) val len = length(acc, vector) - for { n <- startElement until len optimized } { + cforRange { startElement until len } { n => val item = dataIt.next if (availIt.next) newBuf += item } @@ -313,7 +313,7 @@ extends PrimitiveAppendableVector[Long](addr, maxBytes, 64, true) { final def minMax: (Long, Long) = { var min = Long.MaxValue var max = Long.MinValue - for { index <- 0 until length optimized } { + cforRange { 0 until length } { index => val data = apply(index) if (data < min) min = data if (data > max) max = data @@ -355,7 +355,7 @@ BitmapMaskAppendableVector[Long](addr, maxElements) with OptimizingPrimitiveAppe final def minMax: (Long, Long) = { var min = Long.MaxValue var max = Long.MinValue - for { index <- 0 until length optimized } { + cforRange { 0 until length } { index => if (isAvailable(index)) { val data = subVect.apply(index) if (data < min) min = data diff --git a/memory/src/main/scala/filodb.memory/format/vectors/UTF8Vector.scala b/memory/src/main/scala/filodb.memory/format/vectors/UTF8Vector.scala index 2405fcac60..1c314e9824 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/UTF8Vector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/UTF8Vector.scala @@ -3,7 +3,7 @@ package filodb.memory.format.vectors import java.nio.ByteBuffer import debox.Buffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format._ @@ -132,7 +132,7 @@ trait UTF8VectorDataReader extends VectorDataReader { val dataIt = iterate(acc, vector, startElement) val availIt = iterateAvailable(acc, vector, startElement) val len = length(acc, vector) - for { n <- startElement until len optimized } { + cforRange { startElement until len } { n => val item = dataIt.next if (availIt.next) newBuf += item } @@ -286,7 +286,7 @@ class UTF8AppendableVector(val addr: BinaryRegion.NativePointer, final def minMaxStrLen: (Int, Int) = { var min = Int.MaxValue var max = 0 - for {index <- 0 until _len optimized} { + cforRange { 0 until _len } { index => val fixedData = UnsafeUtils.getInt(addr + 12 + index * 4) if (fixedData != NABlob) { val utf8len = if (fixedData < 0) fixedData & MaxSmallLen else UnsafeUtils.getInt(addr + fixedData) @@ -332,7 +332,7 @@ class UTF8AppendableVector(val addr: BinaryRegion.NativePointer, // WARNING: no checking for if delta pushes small offsets out. Intended for compactions only. private def adjustOffsets(newBase: Any, newOff: Long, delta: Int): Unit = { - for {i <- 0 until _len optimized} { + cforRange { 0 until _len } { i => val fixedData = UnsafeUtils.getInt(newBase, newOff + 12 + i * 4) val newData = if (fixedData < 0) { if (fixedData == NABlob) { diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 26bf9837d5..a350f2ac31 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -47,8 +47,6 @@ object Dependencies { scalaTestPlus % Test ) - lazy val scalaxyDep = "com.nativelibs4java" %% "scalaxy-loops" % "0.3.3" % "provided" - lazy val memoryDeps = commonDeps ++ Seq( "com.github.jnr" % "jnr-ffi" % "2.1.6", "joda-time" % "joda-time" % "2.2" withJavadoc(), @@ -57,8 +55,7 @@ object Dependencies { "org.agrona" % "agrona" % "0.9.35", "org.jctools" % "jctools-core" % "2.0.1" withJavadoc(), "org.spire-math" %% "debox" % "0.8.0" withJavadoc(), - scalaLoggingDep, - scalaxyDep + scalaLoggingDep ) lazy val coreDeps = commonDeps ++ Seq( @@ -74,16 +71,14 @@ object Dependencies { "com.github.rholder.fauxflake" % "fauxflake-core" % "1.1.0", "org.scalactic" %% "scalactic" % "3.2.0" withJavadoc(), "org.apache.lucene" % "lucene-core" % "7.3.0" withJavadoc(), - "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0", - scalaxyDep + "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0" ) lazy val sparkJobsDeps = commonDeps ++ Seq( "org.apache.spark" %% "spark-core" % sparkVersion % Provided, "org.apache.spark" %% "spark-sql" % sparkVersion % Provided, "org.apache.spark" %% "spark-core" % sparkVersion % Test excludeAll(excludeNetty), - "org.apache.spark" %% "spark-sql" % sparkVersion % Test excludeAll(excludeNetty), - scalaxyDep + "org.apache.spark" %% "spark-sql" % sparkVersion % Test excludeAll(excludeNetty) ) lazy val cassDeps = commonDeps ++ Seq( @@ -99,8 +94,7 @@ object Dependencies { "com.softwaremill.sttp" %% "circe" % sttpVersion , "com.softwaremill.sttp" %% "async-http-client-backend-future" % sttpVersion, "com.softwaremill.sttp" %% "core" % sttpVersion, - circeGeneric, - scalaxyDep + circeGeneric ) lazy val coordDeps = commonDeps ++ Seq( @@ -137,7 +131,6 @@ object Dependencies { ) lazy val gatewayDeps = commonDeps ++ Seq( - scalaxyDep, logbackDep, "io.monix" %% "monix-kafka-1x" % monixKafkaVersion, "org.rogach" %% "scallop" % "3.1.1" @@ -190,7 +183,6 @@ object Dependencies { // ) lazy val jmhDeps = Seq( - scalaxyDep, "org.apache.spark" %% "spark-sql" % sparkVersion excludeAll(excludeSlf4jLog4j, excludeZK, excludeJersey) ) diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala index ed6f47d3ae..99a4d09da4 100644 --- a/query/src/main/scala/filodb/query/PromCirceSupport.scala +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -5,7 +5,6 @@ import io.circe.syntax._ object PromCirceSupport { import cats.syntax.either._ - // necessary to encode sample in promql response as an array with long and double value as string // Specific encoders for *Sampl types implicit val encodeSampl: Encoder[DataSampl] = Encoder.instance { @@ -31,4 +30,4 @@ object PromCirceSupport { } } } -} \ No newline at end of file +} diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index aa93c72525..5825fc297a 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import monix.reactive.Observable -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.query._ import filodb.memory.format.ZeroCopyUTF8String @@ -246,7 +246,7 @@ object RangeVectorAggregator extends StrictLogging { count += 1 val rowIter = rv.rows toClose += rowIter - for { i <- 0 until outputLen optimized } { + cforRange { 0 until outputLen } { i => accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next) } accs @@ -257,7 +257,7 @@ object RangeVectorAggregator extends StrictLogging { count += 1 val rowIter = rv.rows toClose += rowIter - for { i <- 0 until outputLen optimized } { + cforRange { 0 until outputLen } { i => val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i)) accs(i) = rowAgg.reduceMappedRow(accs(i), mapped) } diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 4928e2dd3e..03dd29545b 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -2,7 +2,7 @@ package filodb.query.exec import monix.reactive.Observable import org.agrona.MutableDirectBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} @@ -79,7 +79,7 @@ final case class HistogramQuantileMapper(funcParams: Seq[FuncArgs]) extends Rang val row = new TransientRow() override def hasNext: Boolean = samples.forall(_.hasNext) override def next(): RowReader = { - for { i <- 0 until samples.size optimized } { + cforRange { 0 until samples.size } { i => val nxt = samples(i).next() buckets(i).rate = nxt.getDouble(1) row.timestamp = nxt.getLong(0) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index dfc838e7f6..39e893bb9f 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -2,7 +2,7 @@ package filodb.query.exec import monix.reactive.Observable import scala.collection.mutable.ListBuffer -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.PartitionSchema @@ -442,7 +442,7 @@ final case class HistToPromSeriesMapper(sch: PartitionSchema) extends RangeVecto } timestamps += row.getLong(0) - for { b <- 0 until hist.numBuckets optimized } { + cforRange { 0 until hist.numBuckets } { b => buckets(hist.bucketTop(b)) += hist.bucketValue(b) } emptyBuckets.foreach { b => buckets(b) += Double.NaN } diff --git a/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala index c1e7db267a..a332e3e976 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala @@ -2,7 +2,7 @@ package filodb.query.exec.rangefn import java.time.{Instant, LocalDateTime, YearMonth, ZoneId, ZoneOffset} -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.memory.format.vectors.{Histogram, MaxHistogram, MutableHistogram} import filodb.query.InstantFunctionId @@ -372,7 +372,7 @@ final case class HistogramBucketImpl() extends HistToDoubleIFunction { if (value.bucketTop(value.numBuckets - 1) == Double.PositiveInfinity) value.topBucketValue else throw new IllegalArgumentException(s"+Inf bucket not in the last position!") } else { - for { b <- 0 until value.numBuckets optimized } { + cforRange { 0 until value.numBuckets } { b => // This comparison does not work for +Inf if (Math.abs(value.bucketTop(b) - bucket) <= 1E-10) return value.bucketValue(b) } @@ -380,4 +380,4 @@ final case class HistogramBucketImpl() extends HistToDoubleIFunction { } } -} \ No newline at end of file +} 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 417f653158..e99575579a 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala @@ -1,6 +1,6 @@ package filodb.query.exec.rangefn -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.core.query.{QueryConfig, TransientHistRow, TransientRow} import filodb.memory.format.{CounterVectorReader, MemoryReader} @@ -272,7 +272,7 @@ abstract class HistogramRateFunctionBase extends CounterChunkedRangeFunction[Tra // TODO: handle case where schemas are different and we need to interpolate schemas if (highestValue.buckets == lowestValue.buckets) { val rateArray = new Array[Double](lowestValue.numBuckets) - for { b <- 0 until rateArray.size optimized } { + cforRange { 0 until rateArray.size } { b => rateArray(b) = RateFunctions.extrapolatedRate( windowStart - 1, windowEnd, numSamples, lowestTime, lowestValue.bucketValue(b), diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala index df28263981..7a7ffb970f 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -6,7 +6,7 @@ import scala.concurrent.duration.FiniteDuration import kamon.Kamon import monix.reactive.Observable -import scalaxy.loops._ +import spire.syntax.cfor._ import filodb.cassandra.columnstore.CassandraColumnStore import filodb.core.{DatasetRef, ErrorResponse, Instance} @@ -305,12 +305,12 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri try { // for each downsample period var first = startRow - for {i <- 0 until downsamplePeriods.length optimized} { + cforRange { 0 until downsamplePeriods.length } { i => val last = downsamplePeriods(i) dsRecordBuilder.startNewRecord(part.schema) // for each column, add downsample column value - for {col <- 0 until downsamplers.length optimized} { + cforRange { 0 until downsamplers.length } { col => val downsampler = downsamplers(col) downsampler match { case t: TimeChunkDownsampler => diff --git a/stress/src/main/scala/filodb.stress/MemStoreStress.scala b/stress/src/main/scala/filodb.stress/MemStoreStress.scala index 5211903bca..a6256763c3 100644 --- a/stress/src/main/scala/filodb.stress/MemStoreStress.scala +++ b/stress/src/main/scala/filodb.stress/MemStoreStress.scala @@ -2,8 +2,6 @@ package filodb.stress import scala.concurrent.Await import scala.concurrent.duration._ -import scala.language.postfixOps - import akka.pattern.ask import akka.util.Timeout import monix.eval.Task @@ -111,4 +109,4 @@ object MemStoreStress extends App { FiloDriver.shutdown() FiloExecutor.shutdown() sess.stop() -} \ No newline at end of file +} From 176323833e17b42de8b18e73c066bf1a1f9c01f3 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 26 Aug 2020 15:59:45 -0700 Subject: [PATCH 05/53] Change querycontext for lhs and rhs of binary join (#871) --- .../queryplanner/SinglePartitionPlanner.scala | 15 ++++++++++----- .../SinglePartitionPlannerSpec.scala | 17 +++++++++++++++-- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index 3c39af6988..d880826bcd 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -1,6 +1,6 @@ package filodb.coordinator.queryplanner -import filodb.core.query.QueryContext +import filodb.core.query.{PromQlQueryParams, QueryContext} import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters, SetOperator} import filodb.query.exec._ @@ -43,14 +43,19 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelecto private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) + val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) + val lhsExec = logicalPlan.lhs match { - case b: BinaryJoin => materializeBinaryJoin(b, qContext) - case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, qContext) + case b: BinaryJoin => materializeBinaryJoin(b, lhsQueryContext) + case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, lhsQueryContext) } val rhsExec = logicalPlan.rhs match { - case b: BinaryJoin => materializeBinaryJoin(b, qContext) - case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, qContext) + case b: BinaryJoin => materializeBinaryJoin(b, rhsQueryContext) + case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, rhsQueryContext) } if (logicalPlan.operator.isInstanceOf[SetOperator]) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 5c16c94c06..674e8f9ea3 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -4,7 +4,6 @@ import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import monix.execution.Scheduler - import filodb.coordinator.ShardMapper import filodb.core.{DatasetRef, MetricsTestData} import filodb.core.metadata.Schemas @@ -48,7 +47,7 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { Seq(FailureTimeRange("local", datasetRef, - TimeRange(100, 10000), false)) + TimeRange(300000, 400000), false)) } } @@ -160,5 +159,19 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual true } + it("should generate BinaryJoin Exec with remote exec's having lhs or rhs query") { + val lp = Parser.queryRangeToLogicalPlan("""test1{job = "app"} + test2{job = "app"}""", TimeStepParams(300, 20, 500)) + + val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 300, 20, 500) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + println(execPlan.printTree()) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual(true) + execPlan.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual(true) + + // LHS should have only LHS query and RHS should have oly RHS query + execPlan.children(0).asInstanceOf[PromQlRemoteExec].params.promQl shouldEqual("""test1{job="app"}""") + execPlan.children(1).asInstanceOf[PromQlRemoteExec].params.promQl shouldEqual("""test2{job="app"}""") + } } From 6ce613daaa871fb193e0faa9e5310560349cf1c3 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Thu, 27 Aug 2020 11:43:18 -0700 Subject: [PATCH 06/53] misc(core): Update monix. (#873) Co-authored-by: Brian O'Neill --- project/Dependencies.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index a350f2ac31..53c70769ed 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -63,7 +63,7 @@ object Dependencies { "io.kamon" %% "kamon-zipkin" % kamonZipkinVersion, "org.slf4j" % "slf4j-api" % "1.7.10", "com.beachape" %% "enumeratum" % "1.5.10", - "io.monix" %% "monix" % "2.3.0", + "io.monix" %% "monix" % "2.3.3", "com.googlecode.concurrentlinkedhashmap" % "concurrentlinkedhashmap-lru" % "1.4", "com.iheart" %% "ficus" % ficusVersion, "io.fastjson" % "boon" % "0.33", From 6024f15952dc83cc2c359c1eb11addf44aea5c3f Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 28 Aug 2020 12:41:00 -0700 Subject: [PATCH 07/53] perf(cass): Increase splits for all token scan CQL queries (#875) Now all token scan queries will use common splits configuration defined for the cassandra cluster. Earlier, splits were being applied only for the downsample scan queries. This is being done, after discovering that index bootstrap scan CQL queries are also latent if cassandra cluster is overloaded. --- .../columnstore/CassandraColumnStore.scala | 3 ++- core/src/main/resources/filodb-defaults.conf | 11 ++++++----- core/src/test/resources/application_test.conf | 1 + .../filodb/downsampler/chunk/DownsamplerMain.scala | 5 ++--- .../downsampler/chunk/DownsamplerSettings.scala | 2 -- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala index d71b58cf31..de2f9059cc 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala @@ -64,6 +64,7 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { private val pkByUTNumSplits = cassandraConfig.getInt("pk-by-updated-time-table-num-splits") private val pkByUTTtlSeconds = cassandraConfig.getDuration("pk-by-updated-time-table-ttl", TimeUnit.SECONDS).toInt private val createTablesEnabled = cassandraConfig.getBoolean("create-tables-enabled") + private val numTokenRangeSplitsForScans = cassandraConfig.getInt("num-token-range-splits-for-scans") val sinkStats = new ChunkSinkStats @@ -316,7 +317,7 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { * @param splitsPerNode - how much parallelism or ways to divide a token range on each node * @return each split will have token_start, token_end, replicas filled in */ - def getScanSplits(dataset: DatasetRef, splitsPerNode: Int = 1): Seq[ScanSplit] = { + def getScanSplits(dataset: DatasetRef, splitsPerNode: Int = numTokenRangeSplitsForScans): Seq[ScanSplit] = { val keyspace = clusterConnector.keyspace require(splitsPerNode >= 1, s"Must specify at least 1 splits_per_node, got $splitsPerNode") diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 52c79461c0..037327e308 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -228,6 +228,12 @@ filodb { # Creation of tables is enabled. Do not set to true in production to avoid # multiple nodes trying to create table at once create-tables-enabled = false + + # amount of parallelism to introduce in the token scan queries. This controls number of spark partitions + # increase if the number of splits seen in cassandra reads is low and spark jobs are slow, or + # if we see Cassandra read timeouts in token range scans. + num-token-range-splits-for-scans = 10 + } downsampler { @@ -241,11 +247,6 @@ filodb { # Number of time series to operate on at one time. Reduce if there is much less memory available cass-write-batch-size = 250 - # amount of parallelism to introduce in the spark job. This controls number of spark partitions - # increase if the number of splits seen in cassandra reads is low and spark jobs are slow, or - # if we see Cassandra read timeouts in token range scans. - splits-per-node = 10 - # Number of rows to read in one fetch. Reduce if we see Cassandra read timeouts cass-read-fetch-size = 5000 diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index 05700e113a..ec2b5ceede 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -17,6 +17,7 @@ filodb { pk-by-updated-time-table-num-splits = 200 pk-by-updated-time-table-ttl = 1 day create-tables-enabled = true + num-token-range-splits-for-scans = 1 } shard-manager { diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala index 8be3f07d7a..597b00e2a0 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala @@ -96,10 +96,9 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp DownsamplerContext.dsLogger.info(s"To rerun this job add the following spark config: " + s""""spark.filodb.downsampler.userTimeOverride": "${java.time.Instant.ofEpochMilli(userTimeInPeriod)}"""") - val splits = batchDownsampler.rawCassandraColStore.getScanSplits(batchDownsampler.rawDatasetRef, - settings.splitsPerNode) + val splits = batchDownsampler.rawCassandraColStore.getScanSplits(batchDownsampler.rawDatasetRef) DownsamplerContext.dsLogger.info(s"Cassandra split size: ${splits.size}. We will have this many spark " + - s"partitions. Tune splitsPerNode which was ${settings.splitsPerNode} if parallelism is low") + s"partitions. Tune num-token-range-splits-for-scans if parallelism is low or latency is high") KamonShutdownHook.registerShutdownHook() diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala index c698b55e03..35975a62f0 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala @@ -56,8 +56,6 @@ class DownsamplerSettings(conf: Config = ConfigFactory.empty()) extends Serializ @transient lazy val cassFetchSize = downsamplerConfig.getInt("cass-read-fetch-size") - @transient lazy val splitsPerNode = downsamplerConfig.getInt("splits-per-node") - @transient lazy val cassWriteTimeout = downsamplerConfig.as[FiniteDuration]("cassandra-write-timeout") @transient lazy val widenIngestionTimeRangeBy = downsamplerConfig.as[FiniteDuration]("widen-ingestion-time-range-by") From 9db87f7a20740721212430755f23a4e09b73dd22 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 31 Aug 2020 12:49:54 -0700 Subject: [PATCH 08/53] fix(prometheus): Parse labels without stack overflow. (#869) --- .../scala/filodb/prometheus/ast/Vectors.scala | 5 +- .../filodb/prometheus/parse/Parser.scala | 66 +++++++++++++++++-- .../filodb/prometheus/parse/ParserSpec.scala | 11 ++++ 3 files changed, 74 insertions(+), 8 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala index 8151d101a4..7eb20c181f 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala @@ -139,10 +139,7 @@ trait Vectors extends Scalars with TimeUnits with Base { false } else true }.map { labelMatch => - val labelVal = labelMatch.value.replace("\\\\", "\\") - .replace("\\\"", "\"") - .replace("\\n", "\n") - .replace("\\t", "\t") + val labelVal = labelMatch.value val labelValue = if (labelMatch.label == PromMetricLabel) { val (newValue, colNameOpt) = extractStripColumn(labelVal) colNameOpt.foreach { col => column = Some(col) } diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index 7325867905..488bd80b8c 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -5,6 +5,10 @@ import scala.util.parsing.combinator.{JavaTokenParsers, PackratParsers, RegexPar import filodb.prometheus.ast.{Expressions, TimeRangeParams, TimeStepParams} import filodb.query._ +object BaseParser { + val whiteSpace = "[ \t\r\f\n]+".r +} + trait BaseParser extends Expressions with JavaTokenParsers with RegexParsers with PackratParsers { lazy val labelNameIdentifier: PackratParser[Identifier] = { @@ -15,8 +19,62 @@ trait BaseParser extends Expressions with JavaTokenParsers with RegexParsers wit "[a-zA-Z_:][a-zA-Z0-9_:\\-\\.]*".r ^^ { str => Identifier(str) } } - protected lazy val labelValueIdentifier: PackratParser[Identifier] = - "([\"'])(?:\\\\\\1|.)*?\\1".r ^^ { str => Identifier(str.substring(1, str.size-1)) } //remove quotes + protected lazy val labelValueIdentifier: PackratParser[Identifier] = { + // Parse a quoted identifier, supporting escapes, with quotes removed. Note that this + // originally relied on a complex regex with capturing groups. The way capturing groups are + // processed by the Java regex class results in deep recursion and a stack overflow error + // for long identifiers. In addition, the regex could only detect escape patterns, but it + // couldn't replace them. As a result, an additional step was required to parse the string + // again, searching and replacing the escapes. Parsers for "real" programming languages + // never use regular expressions, because they are limited in capability. Custom code is + // certainly "bigger", but it's much more flexible overall. This also makes it easier to + // support additional types of promql strings that aren't supported as of yet. For example, + // additional escapes, and backtick quotes which don't do escape processing. + + new PackratParser[Identifier]() { + def apply(in: Input): ParseResult[Identifier] = { + val source = in.source + var offset = in.offset + + (whiteSpace findPrefixMatchOf (source.subSequence(offset, source.length))) match { + case Some(matched) => offset += matched.end + case None => + } + + val quote = source.charAt(offset); offset += 1 + + if (quote != '\'' && quote != '"') { + return Failure("quote character expected", in) + } + + val bob = new StringBuilder() + + while (offset < source.length) { + var c = source.charAt(offset); offset += 1 + + if (c == quote) { + return Success(Identifier(bob.toString()), in.drop(offset - in.offset)) + } + + if (c == '\\') { + val next = source.charAt(offset); offset += 1 + c = next match { + case '\\' | '\'' | '"' => next + case 'f' => '\f' + case 'n' => '\n' + case 'r' => '\r' + case 't' => '\t' + case _ => return Error("illegal string escape: " + next, in) + } + } + + bob.append(c) + } + + return Error("unfinished quoted identifier", in) + } + } + } protected val OFFSET = Keyword("OFFSET") protected val IGNORING = Keyword("IGNORING") @@ -320,7 +378,7 @@ object Parser extends Expression { */ override lazy val skipWhitespace: Boolean = true - override val whiteSpace = "[ \t\r\f\n]+".r + override val whiteSpace = BaseParser.whiteSpace def parseQuery(query: String): Expression = { parseAll(expression, query) match { @@ -417,4 +475,4 @@ object Parser extends Expression { throw new IllegalArgumentException(msg) } -} \ No newline at end of file +} diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index af8de6a378..2aafdc6629 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -383,6 +383,17 @@ class ParserSpec extends AnyFunSpec with Matchers { parseError("timestamp(some_metric, hello)") // reason : Expected only 1 arg, got 2 } + it("parse long identifiers") { + // This should not cause a stack overflow error. + + val bob = new StringBuilder().append("requests{job=\"") + for (i <- 1 to 100) { + bob.append("abcdefghijklmnopqrstuvwxyz_abcdefghijklmnopqrstuvwxyz_") + } + + parseSuccessfully(bob.append("\"}").toString()) + } + it("Should be able to make logical plans for Series Expressions") { val queryToLpString = Map( "http_requests_total + time()" -> "ScalarVectorBinaryOperation(ADD,ScalarTimeBasedPlan(Time,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false)", From 6303b3d655a5e67bfc83c1b324f6fdbff95efa8c Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Tue, 1 Sep 2020 12:19:49 -0700 Subject: [PATCH 09/53] feat(query): Split long-range queries into smaller time-range queries (#832) --- .../queryplanner/LogicalPlanUtils.scala | 45 +- .../queryplanner/SingleClusterPlanner.scala | 42 +- .../SingleClusterPlannerSplitSpec.scala | 697 ++++++++++++++++++ .../main/scala/filodb/query/LogicalPlan.scala | 8 + .../filodb/query/exec/DistConcatExec.scala | 11 + .../scala/filodb/query/exec/ExecPlan.scala | 14 +- 6 files changed, 807 insertions(+), 10 deletions(-) create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 2ca0a508b3..a5d48d2a14 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -1,11 +1,15 @@ package filodb.coordinator.queryplanner -import filodb.core.query.RangeParams +import scala.collection.mutable.ArrayBuffer + +import com.typesafe.scalalogging.StrictLogging + +import filodb.core.query.{QueryContext, RangeParams} import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants import filodb.query._ -object LogicalPlanUtils { +object LogicalPlanUtils extends StrictLogging { /** * Check whether all child logical plans have same start and end time @@ -181,4 +185,41 @@ object LogicalPlanUtils { } else { labels } + + /** + * Split query if the time range is greater than the threshold. It clones the given LogicalPlan with the smaller + * time ranges, creates an execPlan using the provided planner and finally returns Stitch ExecPlan. + * @param lPlan LogicalPlan to be split + * @param qContext QueryContext + * @param timeSplitEnabled split based on longer time range + * @param minTimeRangeForSplitMs if time range is longer than this, plan will be split into multiple plans + * @param splitSizeMs time range for each split, if plan needed to be split + */ + def splitPlans(lPlan: LogicalPlan, + qContext: QueryContext, + timeSplitEnabled: Boolean, + minTimeRangeForSplitMs: Long, + splitSizeMs: Long): Seq[LogicalPlan] = { + val lp = lPlan.asInstanceOf[PeriodicSeriesPlan] + if (timeSplitEnabled && lp.isTimeSplittable && lp.endMs - lp.startMs > minTimeRangeForSplitMs + && lp.stepMs <= splitSizeMs) { + logger.info(s"Splitting query queryId=${qContext.queryId} start=${lp.startMs}" + + s" end=${lp.endMs} step=${lp.stepMs} splitThresholdMs=$minTimeRangeForSplitMs splitSizeMs=$splitSizeMs") + val numStepsPerSplit = splitSizeMs/lp.stepMs + var startTime = lp.startMs + var endTime = Math.min(lp.startMs + numStepsPerSplit * lp.stepMs, lp.endMs) + val splitPlans: ArrayBuffer[LogicalPlan] = ArrayBuffer.empty + while (endTime < lp.endMs ) { + splitPlans += copyWithUpdatedTimeRange(lp, TimeRange(startTime, endTime)) + startTime = endTime + lp.stepMs + endTime = Math.min(startTime + numStepsPerSplit*lp.stepMs, lp.endMs) + } + // when endTime == lp.endMs - exit condition + splitPlans += copyWithUpdatedTimeRange(lp, TimeRange(startTime, endTime)) + logger.info(s"splitsize queryId=${qContext.queryId} numWindows=${splitPlans.length}") + splitPlans + } else { + Seq(lp) + } + } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index ee37e7f227..dfbdb4e066 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -1,5 +1,7 @@ package filodb.coordinator.queryplanner +import scala.concurrent.duration._ + import akka.actor.ActorRef import com.typesafe.scalalogging.StrictLogging import kamon.Kamon @@ -13,8 +15,8 @@ import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext} import filodb.core.store.{AllChunkScan, ChunkScanMethod, InMemoryChunkScan, TimeRangeChunkScan, WriteBufferChunkScan} import filodb.prometheus.ast.Vectors.{PromMetricLabel, TypeLabel} import filodb.prometheus.ast.WindowConstants -import filodb.query._ -import filodb.query.exec._ +import filodb.query.{exec, _} +import filodb.query.exec.{LocalPartitionDistConcatExec, _} object SingleClusterPlanner { private val mdNoShardKeyFilterRequests = Kamon.counter("queryengine-metadata-no-shardkey-requests").withoutTags @@ -23,17 +25,23 @@ object SingleClusterPlanner { /** * Responsible for query planning within single FiloDB cluster * - * @param dsRef dataset - * @param schema schema instance, used to extract partKey schema - * @param spreadProvider used to get spread + * @param dsRef dataset + * @param schema schema instance, used to extract partKey schema + * @param spreadProvider used to get spread * @param shardMapperFunc used to get shard locality + * @param timeSplitEnabled split based on longer time range + * @param minTimeRangeForSplitMs if time range is longer than this, plan will be split into multiple plans + * @param splitSizeMs time range for each split, if plan needed to be split */ class SingleClusterPlanner(dsRef: DatasetRef, schema: Schemas, shardMapperFunc: => ShardMapper, earliestRetainedTimestampFn: => Long, queryConfig: QueryConfig, - spreadProvider: SpreadProvider = StaticSpreadProvider()) + spreadProvider: SpreadProvider = StaticSpreadProvider(), + timeSplitEnabled: Boolean = false, + minTimeRangeForSplitMs: => Long = 1.day.toMillis, + splitSizeMs: => Long = 1.day.toMillis) extends QueryPlanner with StrictLogging with PlannerMaterializer { override val schemas = schema @@ -53,6 +61,27 @@ class SingleClusterPlanner(dsRef: DatasetRef, def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { if (shardMapperFunc.numShards <= 0) throw new IllegalStateException("No shards available") + val logicalPlans = if (logicalPlan.isInstanceOf[PeriodicSeriesPlan]) + LogicalPlanUtils.splitPlans(logicalPlan, qContext, timeSplitEnabled, minTimeRangeForSplitMs, splitSizeMs) + else + Seq(logicalPlan) + val materialized = logicalPlans match { + case Seq(one) => materializeTimeSplitPlan(one, qContext) + case many => + val meterializedPlans = many.map(materializeTimeSplitPlan(_, qContext)) + val targetActor = pickDispatcher(meterializedPlans) + + // create SplitLocalPartitionDistConcatExec that will execute child execplanss sequentially and stitches + // results back with StitchRvsMapper transformer. + val stitchPlan = SplitLocalPartitionDistConcatExec(qContext, targetActor, meterializedPlans) + stitchPlan + } + logger.debug(s"Materialized logical plan for dataset=$dsRef :" + + s" $logicalPlan to \n${materialized.printTree()}") + materialized + } + + private def materializeTimeSplitPlan(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { val materialized = walkLogicalPlanTree(logicalPlan, qContext) match { case PlanResult(Seq(justOne), stitch) => @@ -72,7 +101,6 @@ class SingleClusterPlanner(dsRef: DatasetRef, logger.debug(s"Materialized logical plan for dataset=$dsRef :" + s" $logicalPlan to \n${materialized.printTree()}") materialized - } private def shardsFromFilters(filters: Seq[ColumnFilter], diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala new file mode 100644 index 0000000000..00c8f10c16 --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala @@ -0,0 +1,697 @@ +package filodb.coordinator.queryplanner + +import scala.concurrent.duration._ +import scala.math.min + +import akka.actor.ActorSystem +import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +import filodb.coordinator.ShardMapper +import filodb.coordinator.client.QueryCommands.{FunctionalSpreadProvider, StaticSpreadProvider} +import filodb.core.{MetricsTestData, SpreadChange} +import filodb.core.metadata.Schemas +import filodb.core.query._ +import filodb.core.store.TimeRangeChunkScan +import filodb.prometheus.ast.{TimeStepParams, WindowConstants} +import filodb.prometheus.parse.Parser +import filodb.query._ +import filodb.query.exec._ + +class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaFutures { + + implicit val system = ActorSystem() + private val node = TestProbe().ref + + private val mapper = new ShardMapper(32) + for { i <- 0 until 32 } mapper.registerNode(Seq(i), node) + + private def mapperRef = mapper + + private val dataset = MetricsTestData.timeseriesDataset + private val dsRef = dataset.ref + private val schemas = Schemas(dataset.schema) + + private val config = ConfigFactory.load("application_test.conf") + private val queryConfig = new QueryConfig(config.getConfig("filodb.query")) + + private val splitThresholdMs = 20000 + private val splitSizeMs = 10000 + + // Splitting timewindow is enabled with threshold at 20secs with splitsize as 10sec + private val engine = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = splitThresholdMs, splitSizeMs = splitSizeMs) + + /* + This is the PromQL + + sum(rate(http_request_duration_seconds_bucket{job="myService",le="0.3"}[5m])) by (job) + / + sum(rate(http_request_duration_seconds_count{job="myService"}[5m])) by (job) + */ + + val f1 = Seq(ColumnFilter("__name__", Filter.Equals("http_request_duration_seconds_bucket")), + ColumnFilter("job", Filter.Equals("myService")), + ColumnFilter("le", Filter.Equals("0.3"))) + + val to = System.currentTimeMillis() + val from = to - 50000 + + val intervalSelector = IntervalSelector(from, to) + + val raw1 = RawSeries(rangeSelector = intervalSelector, filters= f1, columns = Seq("value")) + val windowed1 = PeriodicSeriesWithWindowing(raw1, from, 1000, to, 5000, RangeFunctionId.Rate) + val summed1 = Aggregate(AggregationOperator.Sum, windowed1, Nil, Seq("job")) + + val f2 = Seq(ColumnFilter("__name__", Filter.Equals("http_request_duration_seconds_count")), + ColumnFilter("job", Filter.Equals("myService"))) + val raw2 = RawSeries(rangeSelector = intervalSelector, filters= f2, columns = Seq("value")) + val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate) + val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) + val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000) + + it ("should generate SplitLocalPartitionDistConcatExec plan with LocalPartitionDistConcatExec child plans" + + " for LogicalPlan") { + // final logical plan + val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) + + // materialized exec plan + val parentExecPlan = engine.materialize(logicalPlan, QueryContext(origQueryParams = promQlQueryParams)) + + /* + Since threshold + BinaryJoinExec will be divided into 5 time windows and aggregated + with LocalPartitionDistConcatExec using StitchRvsMapper transformation + + Following ExecPlan should be generated: + + T~StitchRvsMapper() + -E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + 1# + --E~BinaryJoinExec(binaryOp=DIV, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + ---T~AggregatePresenter(aggrOp=Sum, aggrParams=List()) + ----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173425483, step=1000, end=1597173435483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(1597173125483,1597173435483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket)), ColumnFilter(job,Equals(myService)), ColumnFilter(le,Equals(0.3))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173425483, step=1000, end=1597173435483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(1597173125483,1597173435483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket)), ColumnFilter(job,Equals(myService)), ColumnFilter(le,Equals(0.3))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + ---T~AggregatePresenter(aggrOp=Sum, aggrParams=List()) + ----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173425483, step=1000, end=1597173435483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(1597173125483,1597173435483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_count)), ColumnFilter(job,Equals(myService))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173425483, step=1000, end=1597173435483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(1597173125483,1597173435483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_count)), ColumnFilter(job,Equals(myService))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + 2# + --E~BinaryJoinExec(binaryOp=DIV, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + ---T~AggregatePresenter(aggrOp=Sum, aggrParams=List()) + ----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173436483, step=1000, end=1597173446483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(1597173136483,1597173446483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket)), ColumnFilter(job,Equals(myService)), ColumnFilter(le,Equals(0.3))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173436483, step=1000, end=1597173446483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(1597173136483,1597173446483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket)), ColumnFilter(job,Equals(myService)), ColumnFilter(le,Equals(0.3))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + ---T~AggregatePresenter(aggrOp=Sum, aggrParams=List()) + ----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173436483, step=1000, end=1597173446483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(1597173136483,1597173446483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_count)), ColumnFilter(job,Equals(myService))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + -----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) + ------T~PeriodicSamplesMapper(start=1597173436483, step=1000, end=1597173446483, window=Some(5000), functionId=Some(Rate), rawSource=true, offsetMs=None) + -------E~MultiSchemaPartitionsExec.(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(1597173136483,1597173446483), filters=List(ColumnFilter(__name__,Equals(http_request_duration_seconds_count)), ColumnFilter(job,Equals(myService))), colName=Some(value), schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-111191042]) + 3#... + 4#... + 5#... + */ + + parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + parentExecPlan.children.size shouldEqual 5 // Split query + parentExecPlan.children.foreach { execPlan => + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true + execPlan.children.foreach { l1 => + // Now there should be single level of reduce because we have 2 shards + l1.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + l1.children.foreach { l2 => + l2.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l2.rangeVectorTransformers.size shouldEqual 2 + l2.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l2.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + } + } + } + + } + + it ("should generate SplitLocalPartitionDistConcatExec wrapper plan and parallelize underlying" + + " binary join aggregation") { + val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) + + // materialized exec plan + val parentExecPlan = engine.materialize(logicalPlan, + QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + + parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + parentExecPlan.children.foreach { execPlan => + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true + + // Now there should be multiple levels of reduce because we have 16 shards + execPlan.children.foreach { l1 => + l1.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + l1.children.foreach { l2 => + l2.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + l2.children.foreach { l3 => + l3.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l3.rangeVectorTransformers.size shouldEqual 2 + l3.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l3.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + } + } + } + } + } + + it("should generate SplitLocalPartitionDistConcatExec wrapper plan and" + + " materialize underlying ExecPlans correctly for _bucket_ histogram queries") { + val lp = Parser.queryRangeToLogicalPlan("""rate(foo{job="bar",_bucket_="2.5"}[5m])""", + TimeStepParams(310, 10, 400)) + + info(s"LogicalPlan is $lp") + lp match { + case p: PeriodicSeriesWithWindowing => p.series.isInstanceOf[ApplyInstantFunctionRaw] shouldEqual true + case _ => throw new IllegalArgumentException(s"Unexpected LP $lp") + } + + val parentExecPlan = engine.materialize(lp, + QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + + info(s"First inner child plan: ${parentExecPlan.children.head.children.head.printTree()}") + parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + parentExecPlan.children.size shouldEqual 5 // 1 + (to-from)/splitThreshold + parentExecPlan.children.foreach { execPlan => + execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + execPlan.children.foreach { l1 => + l1.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l1.rangeVectorTransformers.size shouldEqual 2 + l1.rangeVectorTransformers(0).isInstanceOf[InstantVectorFunctionMapper] shouldEqual true + l1.rangeVectorTransformers(1).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l1.rangeVectorTransformers(1).asInstanceOf[PeriodicSamplesMapper].rawSource shouldEqual false + } + } + + } + + it("should generate SplitLocalPartitionDistConcatExec wrapper plan with appropriate splits," + + "should generate child ExecPlans with appropriate shards for windows when there is a change in spread") { + var filodbSpreadMap = new collection.mutable.HashMap[collection.Map[String, String], Int] + filodbSpreadMap.put(collection.Map(("job" -> "myService")), 2) + + val stepMs = 1000 + + val spreadFunc = QueryContext.simpleMapSpreadFunc(Seq("job"), filodbSpreadMap, 1) + + // final logical plan + val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) + + // materialized exec plan + val parentExecPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spreadFunc)), 1000000)) + parentExecPlan.printTree() + + parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + parentExecPlan.children.size shouldEqual 5 // 5 splits = ((from - to) / splitSize) + + // assert that all window splits have proper start/end times + parentExecPlan.children.zipWithIndex.foreach { case (exec, i) => + val rangeFrom = from + (splitSizeMs * i) + (stepMs * i) + val rangeTo = min(rangeFrom + splitSizeMs, to) + exec.children.head.children.head.rangeVectorTransformers + .head.asInstanceOf[PeriodicSamplesMapper].start shouldEqual rangeFrom + exec.children.head.children.head.rangeVectorTransformers + .head.asInstanceOf[PeriodicSamplesMapper].end shouldEqual rangeTo + } + + // assert that all child plans are not altered + parentExecPlan.children.foreach { execPlan => + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true + execPlan.children should have length (2) + execPlan.children.foreach { reduceAggPlan => + reduceAggPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + reduceAggPlan.children should have length (4) // spread=2 means 4 shards + } + } + } + + it("should generate SplitExec wrapper and should stitch child execplan results" + + "when spread changes during query window") { + val lp = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", TimeStepParams(310, 10, 400)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(350000, 2)) // spread change time is in ms + } + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, + Some(FunctionalSpreadProvider(spread)), 1000000)) + execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans + execPlan.children should have length (5) + + // Inner StitchPlan due to spread change + execPlan.children.head.rangeVectorTransformers.isEmpty shouldEqual true + execPlan.children(1).rangeVectorTransformers.isEmpty shouldEqual true + execPlan.children(2).rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Spread changes here + execPlan.children(3).rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true + execPlan.children.last.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true + } + + it("should generate SplitExec wrapper and should not stitch results when spread has not changed in query range") { + val lp = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", TimeStepParams(310, 10, 400)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(450000, 2)) // spread change time is in ms + } + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, + Some(FunctionalSpreadProvider(spread)), 1000000)) + execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans + execPlan.children should have length (5) + + // no StitchPlan since there are no spread changes in the window + execPlan.children.head.rangeVectorTransformers.isEmpty shouldEqual true + execPlan.children(1).rangeVectorTransformers.isEmpty shouldEqual true + execPlan.children(2).rangeVectorTransformers.isEmpty shouldEqual true + execPlan.children(3).rangeVectorTransformers.isEmpty shouldEqual true + execPlan.children.last.rangeVectorTransformers.isEmpty shouldEqual true + } + + it("should generate SplitExec wrapper with appropriate splits " + + "and should stitch child results before binary join when spread changed in query range") { + val lp = Parser.queryRangeToLogicalPlan("""count(foo{job="bar"} + baz{job="bar"})""", + TimeStepParams(310, 10, 400)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(350000, 2)) + } + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), + 1000000)) + + execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans + execPlan.children should have length (5) + + val binaryJoinNode0 = execPlan.children.head.children.head + val binaryJoinNode1 = execPlan.children(1).children.head + val binaryJoinNode2 = execPlan.children(2).children.head // Spread changes here + val binaryJoinNode3 = execPlan.children(3).children.head + val binaryJoinNode4 = execPlan.children(4).children.head + + binaryJoinNode0.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode0.children.foreach(_.isInstanceOf[StitchRvsExec] shouldEqual false) + + binaryJoinNode1.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode1.children.foreach(_.isInstanceOf[StitchRvsExec] shouldEqual false) + + binaryJoinNode2.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode2.children.foreach(_.isInstanceOf[StitchRvsExec] shouldEqual true) // Spread changes here + + binaryJoinNode3.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode3.children.foreach(_.isInstanceOf[StitchRvsExec] shouldEqual true) + + binaryJoinNode4.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode4.children.foreach(_.isInstanceOf[StitchRvsExec] shouldEqual true) + } + + it("should generate SplitExec wrapper with appropriate splits and" + + " should not stitch child results before binary join when spread has not changed in query range") { + val lp = Parser.queryRangeToLogicalPlan("""count(foo{job="bar"} + baz{job="bar"})""", + TimeStepParams(310, 10, 400)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(450000, 2)) + } + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), + 1000000)) + + execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans + execPlan.children should have length (5) + + execPlan.children.foreach { localPartPlan => + val binaryJoinNode = localPartPlan.children.head + binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode.children should have length(8) // lhs(4) + rhs(4) + binaryJoinNode.children.foreach(_.isInstanceOf[StitchRvsExec] shouldEqual false) + } + } + + it ("should generate SplitExec wrapper with appropriate splits and" + + " should generate child SetOperatorExec for LogicalPlan with Set operator") { + // final logical plan + val logicalPlan = BinaryJoin(summed1, BinaryOperator.LAND, Cardinality.ManyToMany, summed2) + + // materialized exec plan + val parentExecPlan = engine.materialize(logicalPlan, QueryContext(origQueryParams = promQlQueryParams)) + + parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + parentExecPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true + + parentExecPlan.children.foreach { execPlan => + execPlan.isInstanceOf[SetOperatorExec] shouldEqual true + execPlan.children.foreach { l1 => + // Now there should be single level of reduce because we have 2 shards + l1.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + l1.children.foreach { l2 => + l2.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l2.rangeVectorTransformers.size shouldEqual 2 + l2.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l2.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + } + } + } + + } + + it("should generate SplitExec wrapper with appropriate splits " + + " and generate child EmptyResultExec if range is outside retention period " + + " should bound queries until retention period and drop instants outside retention period") { + val nowSeconds = System.currentTimeMillis() / 1000 + val planner = new SingleClusterPlanner(dsRef, schemas, mapperRef, + earliestRetainedTimestampFn = nowSeconds * 1000 - 3.days.toMillis, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = 1.day.toMillis, splitSizeMs = 1.day.toMillis) + + // Case 1: no offset or window + val logicalPlan1 = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", + TimeStepParams(nowSeconds - 4.days.toSeconds, 1.minute.toSeconds, nowSeconds)) + + val splitEp1 = planner.materialize(logicalPlan1, QueryContext()).asInstanceOf[SplitLocalPartitionDistConcatExec] + splitEp1.children should have length(4) + + splitEp1.children.head.isInstanceOf[EmptyResultExec] shouldEqual true // outside retention period + + val ep11 = splitEp1.children(1) + val psm11 = ep11.children.head.asInstanceOf[MultiSchemaPartitionsExec] + .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] + psm11.start shouldEqual (nowSeconds * 1000 + - 3.days.toMillis // retention + + 1.minute.toMillis // step + + WindowConstants.staleDataLookbackMillis) // default window + psm11.end shouldEqual psm11.start - WindowConstants.staleDataLookbackMillis + 1.day.toMillis + + val ep12 = splitEp1.children(2) + val psm12 = ep12.children.head.asInstanceOf[MultiSchemaPartitionsExec] + .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] + psm12.start shouldEqual psm11.end + 1.minute.toMillis // step + psm12.end shouldEqual psm12.start + 1.day.toMillis + + val ep13 = splitEp1.children(3) + val psm13 = ep13.children.head.asInstanceOf[MultiSchemaPartitionsExec] + .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] + psm13.start shouldEqual psm12.end + 1.minute.toMillis // step + psm13.end shouldEqual min(psm13.start + 1.day.toMillis, nowSeconds*1000) + + // Case 2: no offset, some window + val logicalPlan2 = Parser.queryRangeToLogicalPlan("""rate(foo{job="bar"}[20m])""", + TimeStepParams(nowSeconds - 4.days.toSeconds, 1.minute.toSeconds, nowSeconds)) + val splitEp2 = planner.materialize(logicalPlan2, QueryContext()).asInstanceOf[SplitLocalPartitionDistConcatExec] + splitEp2.children should have length(4) + + splitEp2.children.head.isInstanceOf[EmptyResultExec] shouldEqual true + val ep21 = splitEp2.children(1) + val psm21 = ep21.children.head.asInstanceOf[MultiSchemaPartitionsExec] + .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] + psm21.start shouldEqual (nowSeconds * 1000 + - 3.days.toMillis // retention + + 1.minute.toMillis // step + + 20.minutes.toMillis) // window + psm21.end shouldEqual psm21.start - 20.minutes.toMillis + 1.day.toMillis + + // Case 3: offset and some window + val logicalPlan3 = Parser.queryRangeToLogicalPlan("""rate(foo{job="bar"}[20m] offset 15m)""", + TimeStepParams(nowSeconds - 4.days.toSeconds, 1.minute.toSeconds, nowSeconds)) + + val splitEp3 = planner.materialize(logicalPlan3, QueryContext()).asInstanceOf[SplitLocalPartitionDistConcatExec] + splitEp3.children should have length(4) + + splitEp3.children.head.isInstanceOf[EmptyResultExec] shouldEqual true + val ep31 = splitEp3.children(1) + val psm31 = ep31.children.head.asInstanceOf[MultiSchemaPartitionsExec] + .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] + psm31.start shouldEqual (nowSeconds * 1000 + - 3.days.toMillis // retention + + 1.minute.toMillis // step + + 20.minutes.toMillis // window + + 15.minutes.toMillis) // offset + psm31.end shouldEqual psm31.start - 20.minutes.toMillis - 15.minutes.toMillis + 1.day.toMillis + + // Case 4: outside retention + val logicalPlan4 = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", + TimeStepParams(nowSeconds - 10.days.toSeconds, 1.minute.toSeconds, nowSeconds - 5.days.toSeconds)) + val ep4 = planner.materialize(logicalPlan4, QueryContext()) + ep4.children should have length (5) + ep4.children.foreach { childPlan => + childPlan.isInstanceOf[EmptyResultExec] shouldEqual true + import filodb.core.GlobalScheduler._ + val res = childPlan.dispatcher.dispatch(childPlan).runAsync.futureValue.asInstanceOf[QueryResult] + res.result.isEmpty shouldEqual true + } + } + + it("should generate SplitExec wrapper with appropriate splits " + + " and should not split underlying queries with step > splitSize and materialize instant queries " + + "with lookback == retention correctly") { + val nowSeconds = System.currentTimeMillis() / 1000 + val planner = new SingleClusterPlanner(dsRef, schemas, mapperRef, + earliestRetainedTimestampFn = nowSeconds * 1000 - 3.days.toMillis, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = 1.day.toMillis, splitSizeMs = 1.day.toMillis) + + val logicalPlan = Parser.queryRangeToLogicalPlan("""sum(rate(foo{job="bar"}[3d]))""", + TimeStepParams(nowSeconds, 1.minute.toSeconds, nowSeconds)) + + val ep = planner.materialize(logicalPlan, QueryContext()).asInstanceOf[LocalPartitionReduceAggregateExec] + val psm = ep.children.head.asInstanceOf[MultiSchemaPartitionsExec] + .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] + psm.start shouldEqual (nowSeconds * 1000) + psm.end shouldEqual (nowSeconds * 1000) + } + + it("should generate SplitExec wrapper with appropriate splits and should generate child execPlans with offset") { + val t = TimeStepParams(700, 1000, 10000) + val lp = Parser.queryRangeToLogicalPlan("http_requests_total{job = \"app\"} offset 5m", t) + val periodicSeries = lp.asInstanceOf[PeriodicSeries] + periodicSeries.startMs shouldEqual 700000 + periodicSeries.endMs shouldEqual 10000000 + periodicSeries.stepMs shouldEqual 1000000 + + val engine2 = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = 5000000, splitSizeMs = 2000000) + + val parentExecPlan = engine2.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + parentExecPlan.children should have length(4) + parentExecPlan.children.zipWithIndex.foreach { case(execPlan, i) => + val stepAndSplit = (2000 + 1000) * i + val expChunkScanStart = (t.start - 300 - 300 + stepAndSplit).seconds.toMillis + val expChunkScanEnd = min(expChunkScanStart + (300 + 2000).seconds.toMillis, periodicSeries.endMs - 300000) + val expRvtStart = (t.start + stepAndSplit).seconds.toMillis + val expRvtEnd = min((expRvtStart + 2000000), periodicSeries.endMs) + val expRvtStartOffset = (t.start - 300 + stepAndSplit).seconds.toMillis + val expRvtEndOffset = min(expRvtStartOffset + 2000000, periodicSeries.endMs - 300000) + execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual(true) + val multiSchemaExec = execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec] + multiSchemaExec.chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual(expChunkScanStart) //(700 - 300 - 300)*1000 + multiSchemaExec.chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual(expChunkScanEnd) //(700 + 2000 - 300)*1000 + + multiSchemaExec.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual(true) + val rvt = multiSchemaExec.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper] + rvt.offsetMs.get shouldEqual 300000 + rvt.startWithOffset shouldEqual(expRvtStartOffset) // (700 - 300) * 1000 + rvt.endWithOffset shouldEqual (expRvtEndOffset) // (10000 - 300) * 1000 + rvt.start shouldEqual expRvtStart // start and end should be same as query TimeStepParams + rvt.end shouldEqual expRvtEnd + rvt.step shouldEqual 1000000 + } + + } + + it("should generate SplitExec wrapper with appropriate splits " + + "and should generate child execPlans with offset with window") { + val t = TimeStepParams(700, 1000, 10000) + val lp = Parser.queryRangeToLogicalPlan("rate(http_requests_total{job = \"app\"}[5m] offset 5m)", t) + val periodicSeries = lp.asInstanceOf[PeriodicSeriesWithWindowing] + periodicSeries.startMs shouldEqual 700000 + periodicSeries.endMs shouldEqual 10000000 + periodicSeries.stepMs shouldEqual 1000000 + + val engine2 = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = 5000000, splitSizeMs = 2000000) + + val parentExecPlan = engine2.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + parentExecPlan.children should have length(4) + parentExecPlan.children.zipWithIndex.foreach { case(execPlan, i) => + val stepAndSplit = (2000 + 1000) * i + val expChunkScanStart = (t.start - 300 - 300 + stepAndSplit).seconds.toMillis + val expChunkScanEnd = min(expChunkScanStart + (300 + 2000).seconds.toMillis, periodicSeries.endMs - 300000) + val expRvtStart = (t.start + stepAndSplit).seconds.toMillis + val expRvtEnd = min((expRvtStart + 2000000), periodicSeries.endMs) + val expRvtStartOffset = (t.start - 300 + stepAndSplit).seconds.toMillis + val expRvtEndOffset = min(expRvtStartOffset + 2000000, periodicSeries.endMs - 300000) + execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual(true) + val multiSchemaExec = execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec] + multiSchemaExec.chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual(expChunkScanStart) //(700 - 300 - 300)*1000 + multiSchemaExec.chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual(expChunkScanEnd) //(700 + 2000 - 300)*1000 + + multiSchemaExec.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual(true) + val rvt = multiSchemaExec.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper] + rvt.offsetMs.get shouldEqual 300000 + rvt.startWithOffset shouldEqual(expRvtStartOffset) // (700 - 300) * 1000 + rvt.endWithOffset shouldEqual (expRvtEndOffset) // (10000 - 300) * 1000 + rvt.start shouldEqual expRvtStart // start and end should be same as query TimeStepParams + rvt.end shouldEqual expRvtEnd + rvt.step shouldEqual 1000000 + } + } + + it ("should generate SplitExec wrapper with appropriate splits" + + " and should replace __name__ with _metric_ in by and without in underlying execplans") { + val dataset = MetricsTestData.timeseriesDatasetWithMetric + val dsRef = dataset.ref + val schemas = Schemas(dataset.schema) + + val engine = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = 200000, splitSizeMs = 100000) + + val logicalPlan1 = Parser.queryRangeToLogicalPlan("""sum(foo{_ns_="bar", _ws_="test"}) by (__name__)""", + TimeStepParams(1000, 20, 2000)) + + val execPlan1 = engine.materialize(logicalPlan1, QueryContext(origQueryParams = promQlQueryParams)) + execPlan1.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + + execPlan1.children.foreach { childPlan => + childPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + childPlan.children.foreach { l1 => + l1.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + l1.rangeVectorTransformers(1).asInstanceOf[AggregateMapReduce].by shouldEqual List("_metric_") + } + } + + val logicalPlan2 = Parser.queryRangeToLogicalPlan( + """sum(foo{_ns_="bar", _ws_="test"}) + |without (__name__, instance)""".stripMargin, + TimeStepParams(1000, 20, 2000)) + + // materialized exec plan + val execPlan2 = engine.materialize(logicalPlan2, QueryContext(origQueryParams = promQlQueryParams)) + execPlan2.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + + execPlan2.children.foreach { childPlan => + childPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + childPlan.children.foreach { l1 => + l1.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + l1.rangeVectorTransformers(1).asInstanceOf[AggregateMapReduce].without shouldEqual List("_metric_", "instance") + } + } + } + + it ("should generate SplitExec wrapper with appropriate splits" + + " and should replace __name__ with _metric_ in ignoring and group_left/group_right in child execplans") { + val dataset = MetricsTestData.timeseriesDatasetWithMetric + val dsRef = dataset.ref + val schemas = Schemas(dataset.schema) + + val engine = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = 200000, splitSizeMs = 100000) + + val logicalPlan1 = Parser.queryRangeToLogicalPlan( + """sum(foo{_ns_="bar1", _ws_="test"}) + ignoring(__name__) + | sum(foo{_ns_="bar2", _ws_="test"})""".stripMargin, + TimeStepParams(1000, 20, 2000)) + val execPlan1 = engine.materialize(logicalPlan1, QueryContext(origQueryParams = promQlQueryParams)) + execPlan1.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + execPlan1.children should have length(9) + + execPlan1.children.foreach { childPlan => + childPlan.isInstanceOf[BinaryJoinExec] shouldEqual true + childPlan.asInstanceOf[BinaryJoinExec].ignoring shouldEqual Seq("_metric_") + } + + val logicalPlan2 = Parser.queryRangeToLogicalPlan( + """sum(foo{_ns_="bar1", _ws_="test"}) + group_left(__name__) + | sum(foo{_ns_="bar2", _ws_="test"})""".stripMargin, + TimeStepParams(1000, 20, 2000)) + val execPlan2 = engine.materialize(logicalPlan2, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan2.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + execPlan2.children should have length(9) + + execPlan2.children.foreach { childPlan => + childPlan.isInstanceOf[BinaryJoinExec] shouldEqual true + childPlan.asInstanceOf[BinaryJoinExec].include shouldEqual Seq("_metric_") + } + } + + it("should generate SplitExec wrapper with appropriate splits" + + " and should generate execPlan for binary join with offset in underlying child plans") { + val t = TimeStepParams(700, 1000, 10000) + val lp = Parser.queryRangeToLogicalPlan("rate(http_requests_total{job = \"app\"}[5m] offset 5m) / " + + "rate(http_requests_total{job = \"app\"}[5m])", t) + + val engine2 = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig, + timeSplitEnabled = true, minTimeRangeForSplitMs = 5000000, splitSizeMs = 2000000) + + val periodicSeriesPlan = lp.asInstanceOf[BinaryJoin] + periodicSeriesPlan.startMs shouldEqual 700000 + periodicSeriesPlan.endMs shouldEqual 10000000 + periodicSeriesPlan.stepMs shouldEqual 1000000 + + val parentExecPlan = engine2.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true + parentExecPlan.children should have length(4) + + parentExecPlan.children.zipWithIndex.foreach { case(execPlan, i) => + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) + val binaryJoin = execPlan.asInstanceOf[BinaryJoinExec] + val stepAndSplit = (2000 + 1000) * i + val expChunkScanStartOffset = (t.start - 300 - 300 + stepAndSplit).seconds.toMillis + val expChunkScanStart = (t.start - 300 + stepAndSplit).seconds.toMillis + val expChunkScanEndOffset = min(expChunkScanStartOffset + (300 + 2000).seconds.toMillis, periodicSeriesPlan.endMs - 300000) + val expChunkScanEnd = min(expChunkScanStart + (300 + 2000).seconds.toMillis, periodicSeriesPlan.endMs) + val expRvtStart = (t.start + stepAndSplit).seconds.toMillis + val expRvtEnd = min((expRvtStart + 2000000), periodicSeriesPlan.endMs) + val expRvtStartOffset = (t.start - 300 + stepAndSplit).seconds.toMillis + val expRvtEndOffset = min(expRvtStartOffset + 2000000, periodicSeriesPlan.endMs - 300000) + binaryJoin.lhs(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual(true) + val multiSchemaExec1 = binaryJoin.lhs(0).asInstanceOf[MultiSchemaPartitionsExec] + multiSchemaExec1.chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual(expChunkScanStartOffset) + multiSchemaExec1.chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual(expChunkScanEndOffset) + + multiSchemaExec1.rangeVectorTransformers.head.isInstanceOf[PeriodicSamplesMapper] shouldEqual(true) + val rvt1 = multiSchemaExec1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper] + rvt1.offsetMs.get shouldEqual(300000) + rvt1.startWithOffset shouldEqual(expRvtStartOffset) // (700 - 300) * 1000 + rvt1.endWithOffset shouldEqual (expRvtEndOffset) // (10000 - 300) * 1000 + rvt1.start shouldEqual expRvtStart + rvt1.end shouldEqual expRvtEnd + rvt1.step shouldEqual 1000000 + + binaryJoin.rhs(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual(true) + val multiSchemaExec2 = binaryJoin.rhs(0).asInstanceOf[MultiSchemaPartitionsExec] + multiSchemaExec2.chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual(expChunkScanStart) // (700 - 300) * 1000 + multiSchemaExec2.chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual(expChunkScanEnd) + + multiSchemaExec2.rangeVectorTransformers.head.isInstanceOf[PeriodicSamplesMapper] shouldEqual(true) + val rvt2 = multiSchemaExec2.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper] + // No offset in rhs + rvt2.offsetMs.isEmpty shouldEqual true + rvt2.startWithOffset shouldEqual(expRvtStart) + rvt2.endWithOffset shouldEqual (expRvtEnd) + rvt2.start shouldEqual expRvtStart + rvt2.end shouldEqual expRvtEnd + rvt2.step shouldEqual 1000000 + } + + } + +} diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 8c8a2a4cf4..444526f2dc 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -11,6 +11,13 @@ sealed trait LogicalPlan { */ def isRoutable: Boolean = true + /** + * Whether to Time-Split queries into smaller range queries if the range exceeds configured limit. + * This flag will be overridden by plans, which either do not support splitting or will not help in improving + * performance. For e.g. metadata query plans. + */ + def isTimeSplittable: Boolean = true + /** * Replace filters present in logical plan */ @@ -63,6 +70,7 @@ sealed trait PeriodicSeriesPlan extends LogicalPlan { sealed trait MetadataQueryPlan extends LogicalPlan { override def isRoutable: Boolean = false + override def isTimeSplittable: Boolean = false } /** diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index 19eb1d6e3e..f5a98e2906 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -31,6 +31,17 @@ final case class LocalPartitionDistConcatExec(queryContext: QueryContext, dispatcher: PlanDispatcher, children: Seq[ExecPlan]) extends DistConcatExec +/** + * Wrapper/Nonleaf execplan to split long range PeriodicPlan to multiple smaller execs. + * It executes child plans sequentially and merges results using StitchRvsMapper + */ +final case class SplitLocalPartitionDistConcatExec(queryContext: QueryContext, + dispatcher: PlanDispatcher, + children: Seq[ExecPlan], + override val parallelChildTasks: Boolean = false) extends DistConcatExec { + addRangeVectorTransformer(StitchRvsMapper()) +} + /** * Use when child ExecPlan's span multiple partitions */ diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 76cf79c7bb..4c255320dd 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -343,6 +343,10 @@ abstract class NonLeafExecPlan extends ExecPlan { final def submitTime: Long = children.head.queryContext.submitTime + // flag to override child task execution behavior. If it is false, child tasks get executed sequentially. + // Use-cases include splitting longer range query into multiple smaller range queries. + def parallelChildTasks: Boolean = true + private def dispatchRemotePlan(plan: ExecPlan, span: kamon.trace.Span) (implicit sched: Scheduler) = { // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated @@ -368,10 +372,18 @@ abstract class NonLeafExecPlan extends ExecPlan { (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() parentSpan.mark("create-child-tasks") + + // whether child tasks need to be executed sequentially. + // parallelism 1 means, only one worker thread to process underlying tasks. + val parallelism: Int = if (parallelChildTasks) + children.length + else + 1 + // Create tasks for all results. // NOTE: It's really important to preserve the "index" of the child task, as joins depend on it val childTasks = Observable.fromIterable(children.zipWithIndex) - .mapAsync(children.length) { case (plan, i) => + .mapAsync(parallelism) { case (plan, i) => dispatchRemotePlan(plan, parentSpan).map((_, i)) } From 32549be0cb4ae64a4832d20a929d16211cbee812 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 2 Sep 2020 15:14:03 -0700 Subject: [PATCH 10/53] feat(query): support failure routing for metadata and instant queries (#876) --- .../multijvm/BaseAkkaBootstrapperSpec.scala | 2 + .../HighAvailabilityPlanner.scala | 46 ++++++++++++++----- .../queryplanner/LogicalPlanUtils.scala | 29 ++++++++---- .../queryplanner/FailureProviderSpec.scala | 2 +- .../HighAvailabilityPlannerSpec.scala | 25 ++++++++++ .../main/scala/filodb/query/LogicalPlan.scala | 2 - 6 files changed, 84 insertions(+), 22 deletions(-) diff --git a/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala b/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala index 283ddff216..b5ed00307a 100644 --- a/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala +++ b/akka-bootstrapper/src/multi-jvm/scala/filodb/akkabootstrapper/multijvm/BaseAkkaBootstrapperSpec.scala @@ -1,6 +1,8 @@ package filodb.akkabootstrapper.multijvm import scala.concurrent.duration._ +import scala.language.postfixOps + import akka.actor.AddressFromURIString import akka.cluster.Cluster import akka.http.scaladsl.Http diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index 4348ba5967..a3860caca5 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -4,8 +4,8 @@ import com.typesafe.scalalogging.StrictLogging import filodb.core.DatasetRef import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} -import filodb.query.LogicalPlan -import filodb.query.exec.{ExecPlan, InProcessPlanDispatcher, PromQlRemoteExec, StitchRvsExec} +import filodb.query.{LabelValues, LogicalPlan, SeriesKeysByFilters} +import filodb.query.exec._ /** * HighAvailabilityPlanner responsible for using underlying local planner and FailureProvider @@ -32,6 +32,25 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, val remoteHttpTimeoutMs: Long = queryConfig.routingConfig.config.as[Option[Long]]("remote.http.timeout").getOrElse(60000) + private def stitchPlans(rootLogicalPlan: LogicalPlan, + execPlans: Seq[ExecPlan], + queryContext: QueryContext)= { + rootLogicalPlan match { + case lp: LabelValues => LabelValuesDistConcatExec(queryContext, InProcessPlanDispatcher, + execPlans.sortWith((x, y) => !x.isInstanceOf[MetadataRemoteExec])) + case lp: SeriesKeysByFilters => PartKeysDistConcatExec(queryContext, InProcessPlanDispatcher, + execPlans.sortWith((x, y) => !x.isInstanceOf[MetadataRemoteExec])) + case _ => StitchRvsExec(queryContext, InProcessPlanDispatcher, + execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlRemoteExec])) + // ^^ Stitch RemoteExec plan results with local using InProcessPlanDispatcher + // Sort to move RemoteExec in end as it does not have schema + } + } + + private def getLabelValuesUrlParams(lp: LabelValues) = Map("filter" -> lp.filters.map{f => f.column + + f.filter.operatorString + f.filter.valuesStrings.head}.mkString(","), + "labels" -> lp.labelNames.mkString(",")) + /** * Converts Route objects returned by FailureProvider to ExecPlan */ @@ -59,21 +78,26 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, (timeRange.startMs + offsetMs) / 1000, queryParams.stepSecs, (timeRange.endMs + offsetMs) / 1000, queryParams.spread, processFailure = false) logger.debug("PromQlExec params:" + promQlParams) - PromQlRemoteExec(remoteHttpEndpoint, remoteHttpTimeoutMs, - qContext, InProcessPlanDispatcher, dsRef, promQlParams) + val httpEndpoint = remoteHttpEndpoint + queryParams.remoteQueryPath.getOrElse("") + rootLogicalPlan match { + case lp: LabelValues => MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, + getLabelValuesUrlParams(lp), qContext, InProcessPlanDispatcher, + dsRef, promQlParams) + case lp: SeriesKeysByFilters => val urlParams = Map("match[]" -> queryParams.promQl) + MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, + urlParams, qContext, InProcessPlanDispatcher, dsRef, promQlParams) + case _ => PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, + qContext, InProcessPlanDispatcher, dsRef, promQlParams) + } + } } if (execPlans.size == 1) execPlans.head - else StitchRvsExec(qContext, - InProcessPlanDispatcher, - execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlRemoteExec])) - // ^^ Stitch RemoteExec plan results with local using InProcessPlanDispatcher - // Sort to move RemoteExec in end as it does not have schema - + else stitchPlans(rootLogicalPlan, execPlans, qContext) } - def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { // lazy because we want to fetch failures only if needed lazy val offsetMillis = LogicalPlanUtils.getOffsetMillis(logicalPlan) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index a5d48d2a14..01561b1d9b 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -27,14 +27,18 @@ object LogicalPlanUtils extends StrictLogging { /** * Retrieve start and end time from LogicalPlan */ + // scalastyle:off cyclomatic.complexity def getTimeFromLogicalPlan(logicalPlan: LogicalPlan): TimeRange = { logicalPlan match { case lp: PeriodicSeries => TimeRange(lp.startMs, lp.endMs) case lp: PeriodicSeriesWithWindowing => TimeRange(lp.startMs, lp.endMs) case lp: ApplyInstantFunction => getTimeFromLogicalPlan(lp.vectors) case lp: Aggregate => getTimeFromLogicalPlan(lp.vectors) - case lp: BinaryJoin => // can assume lhs & rhs have same time - getTimeFromLogicalPlan(lp.lhs) + case lp: BinaryJoin => val lhsTime = getTimeFromLogicalPlan(lp.lhs) + val rhsTime = getTimeFromLogicalPlan(lp.rhs) + if (lhsTime != rhsTime) throw new UnsupportedOperationException( + "Binary Join has different LHS and RHS times") + else lhsTime case lp: ScalarVectorBinaryOperation => getTimeFromLogicalPlan(lp.vector) case lp: ApplyMiscellaneousFunction => getTimeFromLogicalPlan(lp.vectors) case lp: ApplySortFunction => getTimeFromLogicalPlan(lp.vectors) @@ -46,9 +50,17 @@ object LogicalPlanUtils extends StrictLogging { case i: IntervalSelector => TimeRange(i.from, i.to) case _ => throw new BadQueryException(s"Invalid logical plan") } - case _ => throw new BadQueryException(s"Invalid logical plan ${logicalPlan}") + case lp: LabelValues => TimeRange(lp.startMs, lp.endMs) + case lp: SeriesKeysByFilters => TimeRange(lp.startMs, lp.endMs) + case lp: ApplyInstantFunctionRaw => getTimeFromLogicalPlan(lp.vectors) + case lp: ScalarBinaryOperation => TimeRange(lp.rangeParams.startSecs * 1000, lp.rangeParams.endSecs * 1000) + case lp: ScalarFixedDoublePlan => TimeRange(lp.timeStepParams.startSecs * 1000, + lp.timeStepParams.endSecs * 1000) + case lp: RawChunkMeta => throw new UnsupportedOperationException(s"RawChunkMeta does not have " + + s"time") } } + // scalastyle:on cyclomatic.complexity /** * Used to change start and end time(TimeRange) of LogicalPlan @@ -57,9 +69,10 @@ object LogicalPlanUtils extends StrictLogging { def copyLogicalPlanWithUpdatedTimeRange(logicalPlan: LogicalPlan, timeRange: TimeRange): LogicalPlan = { logicalPlan match { - case lp: PeriodicSeriesPlan => copyWithUpdatedTimeRange(lp, timeRange) - case lp: RawSeriesLikePlan => copyNonPeriodicWithUpdatedTimeRange(lp, timeRange) - case _ => throw new UnsupportedOperationException("Logical plan not supported for copy") + case lp: PeriodicSeriesPlan => copyWithUpdatedTimeRange(lp, timeRange) + case lp: RawSeriesLikePlan => copyNonPeriodicWithUpdatedTimeRange(lp, timeRange) + case lp: LabelValues => lp.copy(startMs = timeRange.startMs, endMs = timeRange.endMs) + case lp: SeriesKeysByFilters => lp.copy(startMs = timeRange.startMs, endMs = timeRange.endMs) } } @@ -72,8 +85,8 @@ object LogicalPlanUtils extends StrictLogging { timeRange: TimeRange): PeriodicSeriesPlan = { logicalPlan match { case lp: PeriodicSeries => lp.copy(startMs = timeRange.startMs, - endMs = timeRange.endMs, - rawSeries = copyNonPeriodicWithUpdatedTimeRange(lp.rawSeries, + endMs = timeRange.endMs, + rawSeries = copyNonPeriodicWithUpdatedTimeRange(lp.rawSeries, timeRange).asInstanceOf[RawSeries]) case lp: PeriodicSeriesWithWindowing => lp.copy(startMs = timeRange.startMs, endMs = timeRange.endMs, diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala index cd31e789da..af77ddc6fa 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala @@ -34,7 +34,7 @@ class FailureProviderSpec extends AnyFunSpec with Matchers { it("should check for PeriodicSeries plan") { summed1.isRoutable shouldEqual (true) - raw2.isRoutable shouldEqual (false) + raw2.isRoutable shouldEqual (true) } it("should not allow Binary Joins with different time ranges in lhs/rhs") { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index ef61567221..602b43ca79 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -13,6 +13,7 @@ import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query._ import filodb.query.exec._ + import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers @@ -422,4 +423,28 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { execPlan2.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(700) execPlan2.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(10000) } + + it("should generate PromQlExec for metadata queries") { + val to = 10000 + val from = 100 + val intervalSelector = IntervalSelector(from, to) + val lp = Parser.metadataQueryToLogicalPlan("http_requests_total{job=\"prometheus\", method=\"GET\"}", + TimeStepParams(from, 20, to)) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(from * 1000, (from + 200) * 1000), false)) + } + } + + val engine = new HighAvailabilityPlanner(dsRef, localPlanner, failureProvider, queryConfig) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.isInstanceOf[MetadataRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[MetadataRemoteExec].params.startSecs shouldEqual (from) + execPlan.asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual (to) + + } } diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 444526f2dc..9bffe34e73 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -37,7 +37,6 @@ sealed trait LogicalPlan { * not in the same time cadence as user query windowing. */ sealed trait RawSeriesLikePlan extends LogicalPlan { - override def isRoutable: Boolean = false def isRaw: Boolean = false def replaceRawSeriesFilters(newFilters: Seq[ColumnFilter]): RawSeriesLikePlan } @@ -69,7 +68,6 @@ sealed trait PeriodicSeriesPlan extends LogicalPlan { } sealed trait MetadataQueryPlan extends LogicalPlan { - override def isRoutable: Boolean = false override def isTimeSplittable: Boolean = false } From 7e975c5ca11e8ecf05236690ad97e8d148b59ec1 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 3 Sep 2020 16:26:42 -0700 Subject: [PATCH 11/53] bug(query): AND returns LHS row values if corresponding RHS row value is NaN (#872) --- .../filodb/query/exec/SetOperatorExec.scala | 33 ++++++++-- .../exec/BinaryJoinSetOperatorSpec.scala | 61 ++++++++++++++++++- 2 files changed, 88 insertions(+), 6 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index a22889044c..3a28f231d3 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -8,7 +8,7 @@ import monix.reactive.Observable import filodb.core.memstore.SchemaMismatch import filodb.core.query._ -import filodb.memory.format.{ZeroCopyUTF8String => Utf8Str} +import filodb.memory.format.{RowReader, ZeroCopyUTF8String => Utf8Str} import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ import filodb.query.BinaryOperator.{LAND, LOR, LUnless} @@ -92,20 +92,43 @@ final case class SetOperatorExec(queryContext: QueryContext, private def setOpAnd(lhsRvs: List[RangeVector], rhsRvs: List[RangeVector], rhsSchema: ResultSchema): List[RangeVector] = { - val rhsKeysSet = new mutable.HashSet[Map[Utf8Str, Utf8Str]]() + // isEmpty method consumes rhs range vector + require(rhsRvs.forall(_.isInstanceOf[SerializedRangeVector]), "RHS should be SerializedRangeVector") + val rhsMap = new mutable.HashMap[Map[Utf8Str, Utf8Str], RangeVector]() var result = new ListBuffer[RangeVector]() rhsRvs.foreach { rv => val jk = joinKeys(rv.key) // Don't add range vector if it is empty if (jk.nonEmpty && !isEmpty(rv, rhsSchema)) - rhsKeysSet += jk + rhsMap.put(jk, rv) } lhsRvs.foreach { lhs => val jk = joinKeys(lhs.key) // Add range vectors from lhs which are present in lhs and rhs both or when jk is empty - // "up AND ON (dummy) vector(1)" should be equivalent to up as there's no dummy label - if (rhsKeysSet.contains(jk) || jk.isEmpty) { + if (rhsMap.contains(jk)) { + val lhsRows = lhs.rows + val rhsRows = rhsMap.get(jk).get.rows + + val rows = new RangeVectorCursor { + val cur = new TransientRow() + override def hasNext: Boolean = lhsRows.hasNext && rhsRows.hasNext + override def next(): RowReader = { + val lhsRow = lhsRows.next() + val rhsRow = rhsRows.next() + // LHS row should not be added to result if corresponding RHS row does not exist + val res = if (rhsRow.getDouble(1).isNaN) Double.NaN else lhsRow.getDouble(1) + cur.setValues(lhsRow.getLong(0), res) + cur + } + override def close(): Unit = { + lhsRows.close() + rhsRows.close() + } + } + result += IteratorBackedRangeVector(lhs.key, rows) + } else if (jk.isEmpty) { + // "up AND ON (dummy) vector(1)" should be equivalent to up as there's no dummy label result += lhs } } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index cab77deb86..f0f4bed677 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -185,7 +185,8 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur import NoCloseCursor._ override def rows(): RangeVectorCursor = Seq( - new TransientRow(1L, 100)).iterator + new TransientRow(1L, 100), + new TransientRow(2L, Double.NaN)).iterator }, new RangeVector { val key: RangeVectorKey = CustomRangeVectorKey( @@ -199,6 +200,7 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, Double.NaN)).iterator }) + val sampleAllNaN : Array[RangeVector] = Array( new RangeVector { val key: RangeVectorKey = CustomRangeVectorKey( @@ -213,6 +215,34 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur new TransientRow(1L, Double.NaN)).iterator }) + val sampleMultipleRows: Array[RangeVector] = Array( + new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"http_requests".utf8, + "job".utf8 -> s"api-server".utf8, + "instance".utf8 -> "0".utf8, + "group".utf8 -> s"production".utf8) + ) + + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( + new TransientRow(1L, 100), + new TransientRow(2L, 300)).iterator + }, + new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"http_requests".utf8, + "job".utf8 -> s"api-server".utf8, + "instance".utf8 -> "1".utf8, + "group".utf8 -> s"production".utf8) + ) + + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( + new TransientRow(1L, 200), + new TransientRow(2L, 400)).iterator + }) + val sampleCanary = sampleHttpRequests.filter(_.key.labelValues.get(ZeroCopyUTF8String("group")).get. toString.equals("canary")) val sampleProduction = sampleHttpRequests.filter(_.key.labelValues.get(ZeroCopyUTF8String("group")).get. @@ -936,4 +966,33 @@ class BinaryJoinSetOperatorSpec extends AnyFunSpec with Matchers with ScalaFutur result.map(_.key.labelValues) sameElements (expectedLabels) shouldEqual true result(0).rows.map(_.getDouble(1)).toList shouldEqual List(100) } + + it("AND should return NaN when rhs sample has Nan even when LHS is not NaN ") { + + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, + Array(dummyPlan), + new Array[ExecPlan](1), + BinaryOperator.LAND, + Nil, Nil, "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", tvSchema, sampleMultipleRows.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", tvSchema, sampleWithNaN.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) + .toListL.runAsync.futureValue + + val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("api-server"), + ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("0"), + ZeroCopyUTF8String("group") -> ZeroCopyUTF8String("production") + )) + + result.size shouldEqual 1 // second RV in sampleWithNaN has all Nan's + result.map(_.key.labelValues) sameElements (expectedLabels) shouldEqual true + val rowValues = result(0).rows.map(_.getDouble(1)).toList + rowValues.head shouldEqual 100 + // LHS second RV has value 300 for 2L, however RHS has Double.NaN for 2L so RHS value is picked + rowValues(1).isNaN shouldEqual true + } } From eaae30641f649d9534ee5a3bc0b35cf2f068c776 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Wed, 9 Sep 2020 07:50:20 -0700 Subject: [PATCH 12/53] refactor(query): move shardkeyfilters to logical plan util (#879) --- .../queryplanner/ShardKeyRegexPlanner.scala | 22 +++++++------------ .../main/scala/filodb/query/LogicalPlan.scala | 15 +++++++++++++ .../filodb/query/exec/MetadataExecPlan.scala | 4 ++-- 3 files changed, 25 insertions(+), 16 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index 7884a7bb81..e9dab4ffa8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -38,7 +38,8 @@ class ShardKeyRegexPlanner(dataset: Dataset, * @return materialized Execution Plan which can be dispatched */ override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - if (hasShardKeyEqualsOnly(logicalPlan)) queryPlanner.materialize(logicalPlan, qContext) + if (LogicalPlan.hasShardKeyEqualsOnly(logicalPlan, dataset.options.nonMetricShardColumns)) + queryPlanner.materialize(logicalPlan, qContext) else walkLogicalPlanTree(logicalPlan, qContext).plans.head } @@ -59,16 +60,6 @@ class ShardKeyRegexPlanner(dataset: Dataset, } } - private def getNonMetricShardKeyFilters(logicalPlan: LogicalPlan): Seq[Seq[ColumnFilter]] = - LogicalPlan.getRawSeriesFilters(logicalPlan) - .map { s => s.filter(f => dataset.options.nonMetricShardColumns.contains(f.column))} - - /** - * Returns true when all shard key filters have Equals - */ - private def hasShardKeyEqualsOnly(logicalPlan: LogicalPlan) = getNonMetricShardKeyFilters(logicalPlan). - forall(_.forall(f => f.filter.isInstanceOf[filodb.core.query.Filter.Equals])) - private def generateExecWithoutRegex(logicalPlan: LogicalPlan, nonMetricShardKeyFilters: Seq[ColumnFilter], qContext: QueryContext): Seq[ExecPlan] = { val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] @@ -87,7 +78,8 @@ class ShardKeyRegexPlanner(dataset: Dataset, * For binary join queries like test1{_ws_ = "demo", _ns_ =~ "App.*"} + test2{_ws_ = "demo", _ns_ =~ "App.*"}) */ private def materializeBinaryJoin(binaryJoin: BinaryJoin, qContext: QueryContext): PlanResult = { - if (hasShardKeyEqualsOnly(binaryJoin)) PlanResult(Seq(queryPlanner.materialize(binaryJoin, qContext))) + if (LogicalPlan.hasShardKeyEqualsOnly(binaryJoin, dataset.options.nonMetricShardColumns)) + PlanResult(Seq(queryPlanner.materialize(binaryJoin, qContext))) else throw new UnsupportedOperationException("Regex not supported for Binary Join") } @@ -97,7 +89,8 @@ class ShardKeyRegexPlanner(dataset: Dataset, * Sub query could be across multiple partitions so aggregate using MultiPartitionReduceAggregateExec * */ private def materializeAggregate(aggregate: Aggregate, queryContext: QueryContext): PlanResult = { - val execPlans = generateExecWithoutRegex(aggregate, getNonMetricShardKeyFilters(aggregate).head, queryContext) + val execPlans = generateExecWithoutRegex(aggregate, + LogicalPlan.getNonMetricShardKeyFilters(aggregate, dataset.options.nonMetricShardColumns).head, queryContext) val exec = if (execPlans.size == 1) execPlans.head else { val reducer = MultiPartitionReduceAggregateExec(queryContext, InProcessPlanDispatcher, @@ -114,7 +107,8 @@ class ShardKeyRegexPlanner(dataset: Dataset, * Sub query could be across multiple partitions so concatenate using MultiPartitionDistConcatExec * */ private def materializeOthers(logicalPlan: LogicalPlan, queryContext: QueryContext): PlanResult = { - val nonMetricShardKeyFilters = getNonMetricShardKeyFilters(logicalPlan) + val nonMetricShardKeyFilters = + LogicalPlan.getNonMetricShardKeyFilters(logicalPlan, dataset.options.nonMetricShardColumns) // For queries which don't have RawSeries filters like metadata and fixed scalar queries val exec = if (nonMetricShardKeyFilters.head.isEmpty) queryPlanner.materialize(logicalPlan, queryContext) else { diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 9bffe34e73..0587b18d39 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -480,5 +480,20 @@ object LogicalPlan { } } } + + /** + * Returns all nonMetricShardKey column filters + */ + def getNonMetricShardKeyFilters(logicalPlan: LogicalPlan, + nonMetricShardColumns: Seq[String]): Seq[Seq[ColumnFilter]] = + getRawSeriesFilters(logicalPlan).map { s => s.filter(f => nonMetricShardColumns.contains(f.column))} + + /** + * Returns true when all shard key filters have Equals + */ + def hasShardKeyEqualsOnly(logicalPlan: LogicalPlan, nonMetricShardColumns: Seq[String]): Boolean = + getNonMetricShardKeyFilters(logicalPlan: LogicalPlan, nonMetricShardColumns: Seq[String]). + forall(_.forall(f => f.filter.isInstanceOf[filodb.core.query.Filter.Equals])) + } //scalastyle:on number.of.types \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 0e68125d2d..7474216224 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -94,7 +94,7 @@ final case class PartKeysExec(queryContext: QueryContext, Observable.empty } Kamon.currentSpan().mark("creating-resultschema") - val sch = new ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) + val sch = ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) ExecResult(rvs, Task.eval(sch)) } @@ -135,7 +135,7 @@ final case class LabelValuesExec(queryContext: QueryContext, Observable.empty } parentSpan.mark("creating-resultschema") - val sch = new ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) + val sch = ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) ExecResult(rvs, Task.eval(sch)) } From facafff2aac866561d73571788b6a4881a183ac5 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 9 Sep 2020 15:57:50 -0700 Subject: [PATCH 13/53] bug(core): Revert _step_ predefined key since we already use it; fix binary joins (#880) _step_ tag was earlier introduced as predefined tag. As a result new partKeys are created which aren't equivalent to old ones. In addition, binary joins have issues since the one-side in OneToOne or OneToMany joins now have duplicate join keys. With this PR, we rename _step_ tag with unused _pi_ (stands for publish interval). Test case included to ensure name update works fine. Include tags like _step_ and _pi_ in the `on` specifier of joins --- core/src/main/resources/filodb-defaults.conf | 10 +- .../PrometheusInputRecordSpec.scala | 40 +++++- .../filodb/query/exec/BinaryJoinExec.scala | 10 +- .../query/exec/BinaryJoinExecSpec.scala | 119 ++++++++++++++++++ 4 files changed, 169 insertions(+), 10 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 037327e308..c30da8d737 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -25,9 +25,13 @@ filodb { columns = ["_metric_:string", "tags:map"] # Predefined keys allow space to be saved for over the wire tags with the given keys - # WARN: It is suggested to only ADD to predefined keys. - # Changing/Renaming existing keys will cause incorrect tags to be reported and probably require a clean wipe. - predefined-keys = ["_ws_", "_ns_", "app", "__name__", "instance", "dc", "le", "job", "exporter", "_step_"] + # WARN: 1. It is suggested to only ADD to predefined keys that are not used already, + # which means only new tags with underscore prefixed and suffixed. + # 2. Changing/Renaming existing keys will cause incorrect tags to be reported and probably require a clean wipe. + # 3. Even when adding new keys, be careful. If that tag is already being used, there is no + # equality of partKeys between old and new time series. + predefined-keys = ["_ws_", "_ns_", "app", "__name__", "instance", "dc", "le", "job", "exporter", "_pi_"] + # FYI: _pi_ stands for publish interval; _ws_ stands for workspace, _ns_ stands for namespace options { # Copy tags can be used to create a new labelPair from one of the existing labelPair in a TimeSeries. diff --git a/gateway/src/test/scala/filodb/gateway/conversion/PrometheusInputRecordSpec.scala b/gateway/src/test/scala/filodb/gateway/conversion/PrometheusInputRecordSpec.scala index 1a4c0b733a..1b40374d43 100644 --- a/gateway/src/test/scala/filodb/gateway/conversion/PrometheusInputRecordSpec.scala +++ b/gateway/src/test/scala/filodb/gateway/conversion/PrometheusInputRecordSpec.scala @@ -1,13 +1,13 @@ package filodb.gateway.conversion - +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers import remote.RemoteStorage.{LabelPair, Sample, TimeSeries} import filodb.core.binaryrecord2.{RecordBuilder, StringifyMapItemConsumer} import filodb.core.metadata.Schemas import filodb.memory.MemFactory -import org.scalatest.funspec.AnyFunSpec -import org.scalatest.matchers.should.Matchers +import filodb.memory.format.ZeroCopyUTF8String._ object TimeSeriesFixture { // "num_partitions,dataset=timeseries,host=MacBook-Pro-229.local,shard=0,_ws_=demo,_ns_=filodb counter=0 1536790212000000000", @@ -53,6 +53,40 @@ class PrometheusInputRecordSpec extends AnyFunSpec with Matchers { } } + it("should be able to change the name of predefined tag and be able to read old part keys with new tag") { + val builder = new RecordBuilder(MemFactory.onHeapFactory) + val oldSchema = Schemas.promCounter.copy( + partition = Schemas.promCounter.partition.copy( + predefinedKeys = Schemas.promCounter.partition.predefinedKeys.updated(9, "_step_"))) + + val records = Seq(new MetricTagInputRecord( + Seq[Any](1000000L, 1.1d), + "num_partitions", + Map("_step_".utf8 -> "0".utf8, "_ns_".utf8 -> "filodb".utf8, "_ws_".utf8 -> "demo".utf8), + oldSchema + )) + + records should have length (1) + val record1 = records.head + record1.getMetric shouldEqual "num_partitions" + record1.nonMetricShardValues shouldEqual Seq("filodb", "demo") + + record1.shardKeyHash shouldEqual RecordBuilder.shardKeyHash(Seq("filodb", "demo"), "num_partitions") + + record1.addToBuilder(builder) + builder.allContainers.head.foreach { case (base, offset) => + schema.ingestionSchema.partitionHash(base, offset) should not equal (7) + schema.ingestionSchema.getLong(base, offset, 0) shouldEqual 1000000L + schema.ingestionSchema.getDouble(base, offset, 1) shouldEqual 1.1d + schema.ingestionSchema.asJavaString(base, offset, 2) shouldEqual "num_partitions" + + val consumer = new StringifyMapItemConsumer() + schema.ingestionSchema.consumeMapItems(base, offset, 3, consumer) + // if there is an undefined key, it should result in empty tag + consumer.stringPairs.toMap shouldEqual Map("_pi_" -> "0", "_ns_" -> "filodb", "_ws_" -> "demo") + } + } + it("should not return any records if metric missing") { val proto1 = TimeSeriesFixture.timeseries(0, baseTags) val records = PrometheusInputRecord(proto1) diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index ed2404649b..62f27e837e 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -51,11 +51,12 @@ final case class BinaryJoinExec(queryContext: QueryContext, require(!on.contains(metricColumn), "On cannot contain metric name") val onLabels = on.map(Utf8Str(_)).toSet + // publishInterval and step tags always needs to be included in join key + val withExtraOnLabels = onLabels ++ Seq("_pi_", "_step_") val ignoringLabels = ignoring.map(Utf8Str(_)).toSet val ignoringLabelsForJoin = ignoringLabels + metricColumn.utf8 // if onLabels is non-empty, we are doing matching based on on-label, otherwise we are // doing matching based on ignoringLabels even if it is empty - val onMatching = onLabels.nonEmpty def children: Seq[ExecPlan] = lhs ++ rhs @@ -120,7 +121,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, } private def joinKeys(rvk: RangeVectorKey): Map[Utf8Str, Utf8Str] = { - if (onLabels.nonEmpty) rvk.labelValues.filter(lv => onLabels.contains(lv._1)) + if (onLabels.nonEmpty) rvk.labelValues.filter(lv => withExtraOnLabels.contains(lv._1)) else rvk.labelValues.filterNot(lv => ignoringLabelsForJoin.contains(lv._1)) } @@ -131,8 +132,9 @@ final case class BinaryJoinExec(queryContext: QueryContext, if (binaryOp.isInstanceOf[MathOperator]) result = result - Utf8Str(metricColumn) if (cardinality == Cardinality.OneToOne) { - result = if (onLabels.nonEmpty) result.filter(lv => onLabels.contains(lv._1)) // retain what is in onLabel list - else result.filterNot(lv => ignoringLabels.contains(lv._1)) // remove the labels in ignoring label list + result = + if (onLabels.nonEmpty) result.filter(lv => withExtraOnLabels.contains(lv._1)) // retain what is in onLabel list + else result.filterNot(lv => ignoringLabels.contains(lv._1)) // remove the labels in ignoring label list } else if (cardinality == Cardinality.OneToMany || cardinality == Cardinality.ManyToOne) { // For group_left/group_right add labels in include from one side. Result should have all keys from many side include.foreach { x => diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index 1e18c42483..e3f1abfd1f 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -143,6 +143,125 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { result.map(_.key).toSet.size shouldEqual 100 } + it("should implictly add step and pi tag as join key on OneToOne joins") { + val lhs1: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricLhs".utf8, "_pi_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val lhs2: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricLhs".utf8, "_step_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val rhs1: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricRhs".utf8,"_pi_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val rhs2: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricRhs".utf8, "_step_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, + Array(dummyPlan), // cannot be empty as some compose's rely on the schema + Array(dummyPlan), // empty since we test compose, not execute or doExecute + BinaryOperator.ADD, + Cardinality.OneToOne, + Nil, Nil, Nil, "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", null, Seq(lhs1, lhs2).map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, Seq(rhs1, rhs2).map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) + .toListL.runAsync.futureValue + + result.size shouldEqual 2 + result(0).key.labelValues.contains("_pi_".utf8) shouldEqual true + result(1).key.labelValues.contains("_step_".utf8) shouldEqual true + + } + + it("should implictly add step and pi tag as join key on OneToMany joins") { + val lhs1: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricLhs".utf8, "_pi_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val lhs2: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricLhs".utf8, "_step_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val rhs1: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricRhs".utf8, "_pi_".utf8 -> "0".utf8, + "tag2".utf8 -> "tag2Val".utf8, "tag1".utf8 -> "tag1Val1".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val rhs2: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricRhs".utf8, "_step_".utf8 -> "0".utf8, + "tag2".utf8 -> "tag2Val".utf8, "tag1".utf8 -> "tag1Val1".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val rhs3: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricRhs".utf8, "_pi_".utf8 -> "0".utf8, + "tag2".utf8 -> "tag2Val".utf8, "tag1".utf8 -> "tag1Val2".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val rhs4: RangeVector = new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricRhs".utf8, "_step_".utf8 -> "0".utf8, + "tag2".utf8 -> "tag2Val".utf8, "tag1".utf8 -> "tag1Val2".utf8)) + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator + } + + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, + Array(dummyPlan), // cannot be empty as some compose's rely on the schema + Array(dummyPlan), // empty since we test compose, not execute or doExecute + BinaryOperator.ADD, + Cardinality.OneToMany, + Nil, ignoring = Seq("tag1"), include = Seq("tag2"), "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", null, Seq(lhs1, lhs2).map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, Seq(rhs1, rhs2, rhs3, rhs4).map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) + .toListL.runAsync.futureValue + + result.size shouldEqual 4 + Seq("_pi_".utf8, "tag1".utf8).forall(result(0).key.labelValues.contains) shouldEqual true + Seq("_step_".utf8, "tag1".utf8).forall(result(1).key.labelValues.contains) shouldEqual true + Seq("_pi_".utf8, "tag1".utf8).forall(result(2).key.labelValues.contains) shouldEqual true + Seq("_step_".utf8, "tag1".utf8).forall(result(3).key.labelValues.contains) shouldEqual true + } + it("should throw error if OneToOne cardinality passed, but OneToMany") { val duplicate: RangeVector = new RangeVector { From 26c78706232ef3ccdd0ff01caa3d111bde892c12 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 9 Sep 2020 18:37:37 -0700 Subject: [PATCH 14/53] bug(core): More Binary join fixes to include step and pi tags in join key (#882) * Use utf8 string instead of regular string when adding extra join keys * Apply previous fix to set join operators --- query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala | 2 +- query/src/main/scala/filodb/query/exec/SetOperatorExec.scala | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index 62f27e837e..9321916a8a 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -52,7 +52,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, val onLabels = on.map(Utf8Str(_)).toSet // publishInterval and step tags always needs to be included in join key - val withExtraOnLabels = onLabels ++ Seq("_pi_", "_step_") + val withExtraOnLabels = onLabels ++ Seq("_pi_".utf8, "_step_".utf8) val ignoringLabels = ignoring.map(Utf8Str(_)).toSet val ignoringLabelsForJoin = ignoringLabels + metricColumn.utf8 // if onLabels is non-empty, we are doing matching based on on-label, otherwise we are diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index 3a28f231d3..e0326bfb8a 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -40,10 +40,11 @@ final case class SetOperatorExec(queryContext: QueryContext, require(!on.contains(metricColumn), "On cannot contain metric name") val onLabels = on.map(Utf8Str(_)).toSet + // TODO Add unit tests for automatic inclusion of _pi_ and _step_ in the join key + val withExtraOnLabels = onLabels ++ Seq("_pi_".utf8, "_step_".utf8) val ignoringLabels = ignoring.map(Utf8Str(_)).toSet + metricColumn.utf8 // if onLabels is non-empty, we are doing matching based on on-label, otherwise we are // doing matching based on ignoringLabels even if it is empty - val onMatching = onLabels.nonEmpty def children: Seq[ExecPlan] = lhs ++ rhs @@ -78,7 +79,7 @@ final case class SetOperatorExec(queryContext: QueryContext, } private def joinKeys(rvk: RangeVectorKey): Map[Utf8Str, Utf8Str] = { - if (onLabels.nonEmpty) rvk.labelValues.filter(lv => onLabels.contains(lv._1)) + if (onLabels.nonEmpty) rvk.labelValues.filter(lv => withExtraOnLabels.contains(lv._1)) else rvk.labelValues.filterNot(lv => ignoringLabels.contains(lv._1)) } From 92ecaa2b7131e65519d7891d2be44175bbcab21b Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 10 Sep 2020 17:25:54 -0700 Subject: [PATCH 15/53] log(query): Log BinaryJoin queries between Aggregate and NonAggregate (#884) --- .../queryplanner/SingleClusterPlanner.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index dfbdb4e066..2bb31897a0 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -5,6 +5,7 @@ import scala.concurrent.duration._ import akka.actor.ActorRef import com.typesafe.scalalogging.StrictLogging import kamon.Kamon +import kamon.tag.TagSet import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands.StaticSpreadProvider @@ -48,6 +49,8 @@ class SingleClusterPlanner(dsRef: DatasetRef, val shardColumns = dsOptions.shardKeyColumns.sorted import SingleClusterPlanner._ + val bjBetweenAggAndNonAgg = Kamon.counter("join-between-agg-non-agg") + .withTags(TagSet.of("dataset", dsRef.toString)) private def dispatcherForShard(shard: Int): PlanDispatcher = { val targetActor = shardMapperFunc.coordForShard(shard) @@ -201,6 +204,19 @@ class SingleClusterPlanner(dsRef: DatasetRef, val rhs = walkLogicalPlanTree(lp.rhs, qContext) val stitchedRhs = if (rhs.needsStitch) Seq(StitchRvsExec(qContext, pickDispatcher(rhs.plans), rhs.plans)) else rhs.plans + + if (lhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && + !rhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { + logger.info(s"Saw Binary Join between aggregate(lhs) and non-aggregate (rhs). ${qContext.origQueryParams}") + bjBetweenAggAndNonAgg.increment() + } + + if (!lhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && + rhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { + logger.info(s"Saw Binary Join between non-aggregate(lhs) and aggregate(rhs): ${qContext.origQueryParams}") + bjBetweenAggAndNonAgg.increment() + } + // TODO Currently we create separate exec plan node for stitching. // Ideally, we can go one step further and add capability to NonLeafNode plans to pre-process // and transform child results individually before composing child results together. From 496d4bd47947753647a2b00f62f6eb55897951cd Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Fri, 11 Sep 2020 14:15:13 -0700 Subject: [PATCH 16/53] log(query): log BinaryJoin queries between Aggr and NonAgg for SinglePartitionPlanner (#885) --- .../queryplanner/PlannerMaterializer.scala | 22 +++++++++++++++++++ .../queryplanner/SingleClusterPlanner.scala | 16 ++------------ .../queryplanner/SinglePartitionPlanner.scala | 2 ++ 3 files changed, 26 insertions(+), 14 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index 395f737396..8b85d427b3 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -1,6 +1,8 @@ package filodb.coordinator.queryplanner +import com.typesafe.scalalogging.StrictLogging import java.util.concurrent.ThreadLocalRandom +import kamon.Kamon import filodb.core.metadata.{DatasetOptions, Schemas} import filodb.core.query.{QueryContext, RangeParams} @@ -139,3 +141,23 @@ trait PlannerMaterializer { } } } + +object PlannerUtil extends StrictLogging { + + val bjBetweenAggAndNonAgg = Kamon.counter("join-between-agg-non-agg").withoutTags() + + def validateBinaryJoin(lhs: Seq[ExecPlan], rhs: Seq[ExecPlan], queryContext: QueryContext): Any = { + + if (lhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && + !rhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { + logger.info(s"Saw Binary Join between aggregate(lhs) and non-aggregate (rhs). ${queryContext.origQueryParams}") + bjBetweenAggAndNonAgg.increment() + } + + if (!lhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && + rhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { + logger.info(s"Saw Binary Join between non-aggregate(lhs) and aggregate(rhs): ${queryContext.origQueryParams}") + bjBetweenAggAndNonAgg.increment() + } + } +} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 2bb31897a0..6cd765a21c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -5,7 +5,6 @@ import scala.concurrent.duration._ import akka.actor.ActorRef import com.typesafe.scalalogging.StrictLogging import kamon.Kamon -import kamon.tag.TagSet import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands.StaticSpreadProvider @@ -43,14 +42,12 @@ class SingleClusterPlanner(dsRef: DatasetRef, timeSplitEnabled: Boolean = false, minTimeRangeForSplitMs: => Long = 1.day.toMillis, splitSizeMs: => Long = 1.day.toMillis) - extends QueryPlanner with StrictLogging with PlannerMaterializer { + extends QueryPlanner with StrictLogging with PlannerMaterializer { override val schemas = schema val shardColumns = dsOptions.shardKeyColumns.sorted import SingleClusterPlanner._ - val bjBetweenAggAndNonAgg = Kamon.counter("join-between-agg-non-agg") - .withTags(TagSet.of("dataset", dsRef.toString)) private def dispatcherForShard(shard: Int): PlanDispatcher = { val targetActor = shardMapperFunc.coordForShard(shard) @@ -205,17 +202,8 @@ class SingleClusterPlanner(dsRef: DatasetRef, val stitchedRhs = if (rhs.needsStitch) Seq(StitchRvsExec(qContext, pickDispatcher(rhs.plans), rhs.plans)) else rhs.plans - if (lhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && - !rhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { - logger.info(s"Saw Binary Join between aggregate(lhs) and non-aggregate (rhs). ${qContext.origQueryParams}") - bjBetweenAggAndNonAgg.increment() - } - if (!lhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && - rhs.plans.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { - logger.info(s"Saw Binary Join between non-aggregate(lhs) and aggregate(rhs): ${qContext.origQueryParams}") - bjBetweenAggAndNonAgg.increment() - } + PlannerUtil.validateBinaryJoin(lhs.plans, rhs.plans, qContext) // TODO Currently we create separate exec plan node for stitching. // Ideally, we can go one step further and add capability to NonLeafNode plans to pre-process diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index d880826bcd..d1a5839787 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -58,6 +58,8 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelecto case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, rhsQueryContext) } + PlannerUtil.validateBinaryJoin(Seq(lhsExec), Seq(rhsExec), qContext) + if (logicalPlan.operator.isInstanceOf[SetOperator]) SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), From 16b7537d2776e65ceac1ee4af36f0f13c44c25c6 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 14 Sep 2020 12:12:16 -0700 Subject: [PATCH 17/53] measure(query): Measure latency of scans separately (#888) Step2 latency measured includes monix task wait time in queue We need to measure scan latency separately. This is the non-asynchronous part of the query processing which should ideally remain the same (per sample scanned) irrespective of spread changes. --- .../src/main/scala/filodb.core/query/RangeVector.scala | 10 +++++++++- query/src/main/scala/filodb/query/exec/ExecPlan.scala | 2 +- .../scala/filodb/query/exec/MetadataRemoteExec.scala | 2 +- .../scala/filodb/query/exec/PromQlRemoteExec.scala | 4 ++-- .../filodb/query/exec/RangeVectorTransformer.scala | 2 +- .../exec/aggregator/CountValuesRowAggregator.scala | 4 ++++ .../exec/aggregator/TopBottomKRowAggregator.scala | 4 ++++ 7 files changed, 22 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index d0d8b68f55..f07e595b23 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -7,6 +7,7 @@ import scala.collection.Iterator import com.typesafe.scalalogging.StrictLogging import debox.Buffer import kamon.Kamon +import kamon.trace.Span import org.joda.time.DateTime import filodb.core.binaryrecord2.{MapItemConsumer, RecordBuilder, RecordContainer, RecordSchema} @@ -313,10 +314,15 @@ object SerializedRangeVector extends StrictLogging { * shared correctly. * The containers are sent whole as most likely more than one would be sent, so they should mostly be packed. */ + // scalastyle:off null def apply(rv: RangeVector, builder: RecordBuilder, schema: RecordSchema, - execPlan: String): SerializedRangeVector = { + execPlanName: String, + span: Span = null): SerializedRangeVector = { + var spanBldr = Kamon.spanBuilder(s"execplan-scan-latency-$execPlanName") + if (span != null) { spanBldr = spanBldr.asChildOf(span) } + val scanSpan = spanBldr.start() var numRows = 0 val oldContainerOpt = builder.currentContainer val startRecordNo = oldContainerOpt.map(_.numRecords).getOrElse(0) @@ -339,8 +345,10 @@ object SerializedRangeVector extends StrictLogging { case None => builder.allContainers case Some(firstContainer) => builder.allContainers.dropWhile(_ != firstContainer) } + scanSpan.finish() new SerializedRangeVector(rv.key, numRows, containers, schema, startRecordNo) } + // scalastyle:on null /** * Creates a SerializedRangeVector out of another RV and ColumnInfo schema. Convenient but no sharing. diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 4c255320dd..04ff22dc06 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -150,7 +150,7 @@ trait ExecPlan extends QueryCommand { srv case rv: RangeVector => // materialize, and limit rows per RV - val srv = SerializedRangeVector(rv, builder, recSchema, printTree(false)) + val srv = SerializedRangeVector(rv, builder, recSchema, getClass.getSimpleName, span) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results if (enforceLimit && numResultSamples > queryContext.sampleLimit) diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index fbe5b01316..667b23f398 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -48,7 +48,7 @@ case class MetadataRemoteExec(queryEndpoint: String, val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(iteratorMap.toIterator)) - val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, printTree(false))) + val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, this.getClass.getSimpleName, span)) span.finish() QueryResult(id, resultSchema, srvSeq) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index 6c0a5623f4..d20ff7e8c2 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -162,7 +162,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, defaultRecSchema, printTree(useNewline = false)) + SerializedRangeVector(rv, builder, defaultRecSchema, "PromQlRemoteExec-default") // TODO: Handle stitching with verbose flag } QueryResult(id, defaultResultSchema, rangeVectors) @@ -196,7 +196,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, histRecSchema, printTree(useNewline = false)) + SerializedRangeVector(rv, builder, histRecSchema, "PromQlRemoteExec-hist") // TODO: Handle stitching with verbose flag } QueryResult(id, histResultSchema, rangeVectors) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 39e893bb9f..cf2e9be525 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -279,7 +279,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto // Create SerializedRangeVector so that sorting does not consume rows iterator val resultRv = source.toListL.map { rvs => - rvs.map(SerializedRangeVector(_, builder, recSchema, "sortExecPlan")). + rvs.map(SerializedRangeVector(_, builder, recSchema, getClass.getSimpleName)). sortBy { rv => if (rv.rows.hasNext) rv.rows.next().getDouble(1) else Double.NaN }(ordering) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala index e98513dcdb..2eff37b6e4 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -2,6 +2,8 @@ package filodb.query.exec.aggregator import scala.collection.mutable +import kamon.Kamon + import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers import filodb.core.memstore.FiloSchedulers.QuerySchedName @@ -86,6 +88,7 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() + val span = Kamon.spanBuilder(s"execplan-scan-latency-TopBottomK").start() try { FiloSchedulers.assertThreadName(QuerySchedName) // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release @@ -108,6 +111,7 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr aggRangeVector.rows.close() ChunkMap.releaseAllSharedLocks() } + span.finish() resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords()).sum new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala index fbbd276865..43d37c5b0b 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -2,6 +2,8 @@ package filodb.query.exec.aggregator import scala.collection.mutable +import kamon.Kamon + import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers import filodb.core.memstore.FiloSchedulers.QuerySchedName @@ -86,6 +88,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() + val span = Kamon.spanBuilder(s"execplan-scan-latency-TopBottomK").start() try { FiloSchedulers.assertThreadName(QuerySchedName) ChunkMap.validateNoSharedLocks() @@ -108,6 +111,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { aggRangeVector.rows().close() ChunkMap.releaseAllSharedLocks() } + span.finish() resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords).sum new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0) From 014d79073e1469a523638cd0490aa9dd21c309b7 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 14 Sep 2020 12:56:24 -0700 Subject: [PATCH 18/53] fix(query): Fix class cast exception when using absent function over an aggregate (#881) --- .../queryplanner/LogicalPlanParserSpec.scala | 3 +++ .../scala/filodb/prometheus/ast/Functions.scala | 7 ++++--- .../query/exec/rangefn/AbsentFunctionSpec.scala | 14 ++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala index e22622982e..0a685c58da 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala @@ -57,6 +57,9 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { parseAndAssertResult("""count_values("freq",http_requests_total{job="app"})""") parseAndAssertResult("""timestamp(http_requests_total{job="app"})""") parseAndAssertResult("""absent(http_requests_total{job="app"})""") + parseAndAssertResult("""absent(sum(http_requests_total{job="app"}))""") + parseAndAssertResult("""absent(sum_over_time(http_requests_total{job="app"}[5s]))""") + parseAndAssertResult("""absent(rate(http_requests_total{job="app"}[5s] offset 200s))""") } it("should generate query from LogicalPlan having offset") { diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala index a7891cd7de..363fef265c 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala @@ -1,6 +1,6 @@ package filodb.prometheus.ast -import filodb.core.query.RangeParams +import filodb.core.query.{ColumnFilter, RangeParams} import filodb.query._ import filodb.query.RangeFunctionId.Timestamp @@ -165,7 +165,8 @@ trait Functions extends Base with Operators with Vectors { val periodicSeriesPlan = seriesParam.asInstanceOf[PeriodicSeries].toSeriesPlan(timeParams) ApplySortFunction(periodicSeriesPlan, sortFunctionId) } else if (absentFunctionIdOpt.isDefined) { - val columnFilter = seriesParam.asInstanceOf[InstantExpression].columnFilters + val columnFilter = if (seriesParam.isInstanceOf[InstantExpression]) + seriesParam.asInstanceOf[InstantExpression].columnFilters else Seq.empty[ColumnFilter] val periodicSeriesPlan = seriesParam.asInstanceOf[PeriodicSeries].toSeriesPlan(timeParams) ApplyAbsentFunction(periodicSeriesPlan, columnFilter, RangeParams(timeParams.start, timeParams.step, timeParams.end)) @@ -189,4 +190,4 @@ trait Functions extends Base with Operators with Vectors { } } } -} \ No newline at end of file +} diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala index 852f15ecb3..70262e2327 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala @@ -140,4 +140,18 @@ class AbsentFunctionSpec extends AnyFunSpec with Matchers with ScalaFutures with keys.head shouldEqual expectedKeys rows shouldEqual expectedRows } + + it("should not have keys when ColumnFilter is empty") { + val columnFilter = Seq.empty[ColumnFilter] + val expectedRows = List(1.0, 1.0, 1.0, 1.0, 1.0, 1.0) + val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 2, 11), "metric") + val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) + val result = resultObs.toListL.runAsync.futureValue + result.size shouldEqual (1) + val keys = result.map(_.key.labelValues) + val rows = result.flatMap(_.rows.map(_.getDouble(1)).toList) + keys.head.isEmpty shouldEqual true + rows shouldEqual expectedRows + } + } From 21443c3088a797a23a8e1d7ecbbbfb3e9fe45402 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Mon, 14 Sep 2020 13:12:53 -0700 Subject: [PATCH 19/53] feat(parser): Add support to parse LabelValues query filters (#887) --- .../queryplanner/SingleClusterPlanner.scala | 7 ++-- .../filodb/prometheus/parse/Parser.scala | 33 +++++++++++++++++++ .../filodb/prometheus/parse/ParserSpec.scala | 25 ++++++++++++++ .../main/scala/filodb/query/LogicalPlan.scala | 8 ++--- 4 files changed, 66 insertions(+), 7 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 6cd765a21c..b2822dc70d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -371,15 +371,16 @@ class SingleClusterPlanner(dsRef: DatasetRef, val labelNames = if (metricLabelIndex > -1 && dsOptions.metricColumn != PromMetricLabel) lp.labelNames.updated(metricLabelIndex, dsOptions.metricColumn) else lp.labelNames - val shardsToHit = if (shardColumns.toSet.subsetOf(lp.filters.map(_.column).toSet)) { - shardsFromFilters(lp.filters, qContext) + val renamedFilters = renameMetricFilter(lp.filters) + val shardsToHit = if (shardColumns.toSet.subsetOf(renamedFilters.map(_.column).toSet)) { + shardsFromFilters(renamedFilters, qContext) } else { mdNoShardKeyFilterRequests.increment() shardMapperFunc.assignedShards } val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) - exec.LabelValuesExec(qContext, dispatcher, dsRef, shard, lp.filters, labelNames, lp.startMs, lp.endMs) + exec.LabelValuesExec(qContext, dispatcher, dsRef, shard, renamedFilters, labelNames, lp.startMs, lp.endMs) } PlanResult(metaExec, false) } diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index 488bd80b8c..ba3a356ea7 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -2,6 +2,7 @@ package filodb.prometheus.parse import scala.util.parsing.combinator.{JavaTokenParsers, PackratParsers, RegexParsers} +import filodb.core.query.{ColumnFilter, Filter} import filodb.prometheus.ast.{Expressions, TimeRangeParams, TimeStepParams} import filodb.query._ @@ -136,6 +137,11 @@ trait Operator extends BaseParser { Seq() ++ _ } + lazy val labelValues: PackratParser[Seq[LabelMatch]] = + repsep(labelMatch, ",") ^^ { + Seq() ++ _ + } + lazy val add = "+" ^^ (_ => Add) lazy val sub = "-" ^^ (_ => Sub) @@ -399,6 +405,14 @@ object Parser extends Expression { } } + def parseLabelValueFilter(query: String): Seq[LabelMatch] = { + parseAll(labelValues, query) match { + case s: Success[_] => s.get.asInstanceOf[Seq[LabelMatch]] + case e: Error => handleError(e, query) + case f: Failure => handleFailure(f, query) + } + } + def metadataQueryToLogicalPlan(query: String, timeParams: TimeRangeParams, fetchFirstLastSampleTimes: Boolean = false): LogicalPlan = { val expression = parseQuery(query) @@ -408,6 +422,25 @@ object Parser extends Expression { } } + def labelValuesQueryToLogicalPlan(labelNames: Seq[String], filterQuery: Option[String], + timeParams: TimeRangeParams): LogicalPlan = { + filterQuery match { + case Some(filter) => + val columnFilters = parseLabelValueFilter(filter).map { l => + l.labelMatchOp match { + case EqualMatch => ColumnFilter(l.label, Filter.Equals(l.value)) + case NotRegexMatch => ColumnFilter(l.label, Filter.NotEqualsRegex(l.value)) + case RegexMatch => ColumnFilter(l.label, Filter.EqualsRegex(l.value)) + case NotEqual(false) => ColumnFilter(l.label, Filter.NotEquals(l.value)) + case other: Any => throw new IllegalArgumentException(s"Unknown match operator $other") + } + } + LabelValues(labelNames, columnFilters, timeParams.start * 1000, timeParams.end * 1000) + case _ => + LabelValues(labelNames, Seq.empty, timeParams.start * 1000, timeParams.end * 1000) + } + } + def queryToLogicalPlan(query: String, queryTimestamp: Long, step: Long): LogicalPlan = { // Remember step matters here in instant query, when lookback is provided in step factor notation as in [5i] val defaultQueryParams = TimeStepParams(queryTimestamp, step, queryTimestamp) diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 2aafdc6629..177e141319 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -23,6 +23,21 @@ class ParserSpec extends AnyFunSpec with Matchers { lp.toString shouldEqual queryToLpString._2 } + it("labelvalues filter query") { + parseLabelValueSuccessfully("job=\"prometheus\", method=\"GET\"") + parseLabelValueSuccessfully("job=\"prometheus\", method=\"GET\"") + parseLabelValueSuccessfully("job=\"prometheus\", method!=\"GET\"") + parseLabelValueError("http_requests_total{job=\"prometheus\", method!=\"GET\"}") + parseLabelValueError("{__name__=\"prometheus\"}") + parseLabelValueError("job[__name__=\"prometheus\"]") + val queryToLpString = ("job=\"prometheus\", method!=\"GET\"" -> + "LabelValues(List(_ns_),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(method,NotEquals(GET))),1524855988000,1524855988000)") + val start: Long = 1524855988L + val end: Long = 1524855988L + val lp = Parser.labelValuesQueryToLogicalPlan(Seq("_ns_"), Some(queryToLpString._1), TimeStepParams(start, -1, end)) + lp.toString shouldEqual queryToLpString._2 + } + it("parse basic scalar expressions") { parseSuccessfully("1") // parse("+Inf") @@ -594,4 +609,14 @@ class ParserSpec extends AnyFunSpec with Matchers { Parser.parseQuery(query) } } + + private def parseLabelValueSuccessfully(query: String) = { + Parser.parseLabelValueFilter(query) + } + + private def parseLabelValueError(query: String) = { + intercept[IllegalArgumentException] { + Parser.parseLabelValueFilter(query) + } + } } diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 0587b18d39..36b23c4fd7 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -473,10 +473,10 @@ object LogicalPlan { def getRawSeriesFilters(logicalPlan: LogicalPlan): Seq[Seq[ColumnFilter]] = { LogicalPlan.findLeafLogicalPlans(logicalPlan).map { l => - l match - { - case lp: RawSeries => lp.filters - case _ => Seq.empty + l match { + case lp: RawSeries => lp.filters + case lp: LabelValues => lp.filters + case _ => Seq.empty } } } From a926d7468d6a08440c8f9e29c3adced284cbd39b Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Mon, 14 Sep 2020 14:08:23 -0700 Subject: [PATCH 20/53] fix(query): Remove unused param from PartKeysExec (#889) --- .../queryplanner/SingleClusterPlanner.scala | 2 +- .../src/main/scala/filodb/query/exec/MetadataExecPlan.scala | 2 -- .../src/test/scala/filodb/query/exec/MetadataExecSpec.scala | 6 +++--- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index b2822dc70d..befd47aec1 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -398,7 +398,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, } val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) - PartKeysExec(qContext, dispatcher, dsRef, shard, schemas.part, renamedFilters, + PartKeysExec(qContext, dispatcher, dsRef, shard, renamedFilters, lp.fetchFirstLastSampleTimes, lp.startMs, lp.endMs) } PlanResult(metaExec, false) diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 7474216224..bfff4d3d90 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -9,7 +9,6 @@ import filodb.core.DatasetRef import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.memstore.MemStore import filodb.core.metadata.Column.ColumnType -import filodb.core.metadata.PartitionSchema import filodb.core.query._ import filodb.core.store.ChunkSource import filodb.memory.format.{UTF8MapIteratorRowReader, ZeroCopyUTF8String} @@ -72,7 +71,6 @@ final case class PartKeysExec(queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, shard: Int, - partSchema: PartitionSchema, filters: Seq[ColumnFilter], fetchFirstLastSampleTimes: Boolean, start: Long, diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index c1ee1d2ca6..e5803b17b6 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -108,7 +108,7 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, - timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) + timeseriesDataset.ref, 0, filters, false, now-5000, now) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue resp match { @@ -122,7 +122,7 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B val filters = Seq (ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, - timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) + timeseriesDataset.ref, 0, filters, false, now-5000, now) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { @@ -143,7 +143,7 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B //Reducing limit results in truncated metadata response val execPlan = PartKeysExec(QueryContext(sampleLimit = limit-1), dummyDispatcher, - timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) + timeseriesDataset.ref, 0, filters, false, now-5000, now) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { From d3c57f1db2d4937141a6f5f471cf3d058f255a23 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 16 Sep 2020 12:05:31 -0700 Subject: [PATCH 21/53] correct prev value being returned (#890) --- .../filodb.memory/format/vectors/DoubleVector.scala | 2 +- .../exec/rangefn/AggrOverTimeFunctionsSpec.scala | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala index 3e72de6444..a1808c20ea 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -528,6 +528,6 @@ object DoubleLongWrapDataReader extends DoubleVectorDataReader { val ignorePrev = if (prev.isNaN) true else false val changes = LongBinaryVector(acc, vector).changes(acc, vector, start, end, prev.toLong, ignorePrev) - (changes._1.toDouble, changes._1.toDouble) + (changes._1.toDouble, changes._2.toDouble) } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index 9ea5ff0063..ce74dbb4d0 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -707,4 +707,15 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { aggregated shouldEqual List((100000, 0.0), (120000, 2.0), (140000, 2.0)) } should have message "Query timeout in filodb.core.store.WindowedChunkIterator after 180 seconds" } + + it("should return 0 for changes on constant value") { + val data = List.fill(1000)(1.586365307E9) + val startTS = 1599071100L + val tuples = data.zipWithIndex.map { case (d, t) => (startTS + t * 15, d) } + val rv = timeValueRVPk(tuples) + val chunkedIt = new ChunkedWindowIteratorD(rv, 1599073500L, 300000, 1599678300L, 10800000, + new ChangesChunkedFunctionD(), querySession) + val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList + aggregated.foreach(x => x._2 shouldEqual(0)) + } } From 11c373f829e4c9ffdb298f5bc7754480f5313645 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 16 Sep 2020 20:57:22 -0700 Subject: [PATCH 22/53] perf(conf): Adding dev gatling performance test config (#893) --- conf/logback-perf.xml | 23 +++++++++++++++++++ conf/promperf-filodb-server.conf | 13 +++++++++++ conf/promperf-source.conf | 13 +++++++++++ .../timeseries/TestTimeseriesProducer.scala | 8 +++---- 4 files changed, 53 insertions(+), 4 deletions(-) create mode 100644 conf/logback-perf.xml create mode 100644 conf/promperf-filodb-server.conf create mode 100644 conf/promperf-source.conf diff --git a/conf/logback-perf.xml b/conf/logback-perf.xml new file mode 100644 index 0000000000..4d9dd95f2c --- /dev/null +++ b/conf/logback-perf.xml @@ -0,0 +1,23 @@ + + + + + + + [%date{ISO8601}] %-5level %thread %logger{26} [%X{akkaSource}] - %msg%n + + + + + + + + + + + + + + + + diff --git a/conf/promperf-filodb-server.conf b/conf/promperf-filodb-server.conf new file mode 100644 index 0000000000..6865bfad28 --- /dev/null +++ b/conf/promperf-filodb-server.conf @@ -0,0 +1,13 @@ +include "timeseries-filodb-server.conf" + +filodb { + dataset-configs = [ + "conf/promperf-source.conf" + ] + + spread-default = 3 + + # Override default spread for application using override block which will have non metric shard keys and spread. + spread-assignment = [] + +} \ No newline at end of file diff --git a/conf/promperf-source.conf b/conf/promperf-source.conf new file mode 100644 index 0000000000..781f1be88f --- /dev/null +++ b/conf/promperf-source.conf @@ -0,0 +1,13 @@ +include "timeseries-dev-source.conf" + +dataset = "promperf" +num-shards = 8 +min-num-nodes = 1 +sourceconfig { + filo-topic-name = "prom-perf" + store { + flush-interval = 2m + disk-time-to-live = 720 hours // 30 days + shard-mem-size = 128MB + } +} diff --git a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala index 03ac37dd5c..10bd3f136a 100644 --- a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala +++ b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala @@ -65,8 +65,8 @@ object TestTimeseriesProducer extends StrictLogging { logger.info(s"Finished producing $numSamples records for ${samplesDuration / 1000} seconds") val startQuery = startTime / 1000 - val endQuery = startQuery + 300 - val periodicPromQL = """heap_usage{dc="DC0",_ns_="App-0",_ws_="demo"}""" + val endQuery = startQuery + (numSamples / numTimeSeries) * 10 + val periodicPromQL = """heap_usage{_ns_="App-0",_ws_="demo"}""" val query = s"""./filo-cli '-Dakka.remote.netty.tcp.hostname=127.0.0.1' --host 127.0.0.1 --dataset prometheus """ + s"""--promql '$periodicPromQL' --start $startQuery --end $endQuery --limit 15""" @@ -77,12 +77,12 @@ object TestTimeseriesProducer extends StrictLogging { s"query=$periodicSamplesQ&start=$startQuery&end=$endQuery&step=15" logger.info(s"Periodic Samples query URL: \n$periodicSamplesUrl") - val rawSamplesQ = URLEncoder.encode("""heap_usage{dc="DC0",_ws_="demo",_ns_="App-0"}[2m]""", + val rawSamplesQ = URLEncoder.encode("""heap_usage{_ws_="demo",_ns_="App-0"}[2m]""", StandardCharsets.UTF_8.toString) val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$rawSamplesQ&time=$endQuery" logger.info(s"Raw Samples query URL: \n$rawSamplesUrl") - val downsampledQ = URLEncoder.encode("""heap_usage::sum{dc="DC0",_ws_="demo",_ns_="App-0"}[2m]""", + val downsampledQ = URLEncoder.encode("""heap_usage::sum{_ws_="demo",_ns_="App-0"}[2m]""", StandardCharsets.UTF_8.toString) val downsampledSamplesUrl = s"http://localhost:8080/promql/prometheus_ds_1m/api/v1/query?" + s"query=$downsampledQ&time=$endQuery" From 6b6b402b4c78818f1827c690bc808daaf81c3b5f Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 18 Sep 2020 15:14:30 -0700 Subject: [PATCH 23/53] measure(query): Measure time elapsed for different points in execplan (#894) --- .../filodb/query/exec/BinaryJoinExec.scala | 4 ++++ .../scala/filodb/query/exec/ExecPlan.scala | 22 ++++++++++++++++--- .../filodb/query/exec/PlanDispatcher.scala | 2 +- .../filodb/query/exec/SetOperatorExec.scala | 4 ++++ .../aggregator/CountValuesRowAggregator.scala | 2 +- 5 files changed, 29 insertions(+), 5 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index 9321916a8a..dc7ebacf67 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -2,6 +2,7 @@ package filodb.query.exec import scala.collection.mutable +import kamon.Kamon import monix.eval.Task import monix.reactive.Observable @@ -74,6 +75,9 @@ final case class BinaryJoinExec(queryContext: QueryContext, case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => + Kamon.histogram("query-execute-time-elapsed-step2-child-results-available") + .withTag("plan", getClass.getSimpleName) + .record(System.currentTimeMillis - queryContext.submitTime) // NOTE: We can't require this any more, as multischema queries may result in not a QueryResult if the // filter returns empty results. The reason is that the schema will be undefined. // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 04ff22dc06..fe0c5e8f16 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -96,11 +96,14 @@ trait ExecPlan extends QueryCommand { querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { + val startExecute = querySession.qContext.submitTime + val parentSpan = Kamon.currentSpan() // NOTE: we launch the preparatory steps as a Task too. This is important because scanPartitions, // Lucene index lookup, and On-Demand Paging orchestration work could suck up nontrivial time and // we don't want these to happen in a single thread. - // Step 1: initiate doExecute, get schema + + // Step 1: initiate doExecute: make result schema and set up the async monix pipeline to create RVs lazy val step1 = Task { val span = Kamon.spanBuilder(s"execute-step1-${getClass.getSimpleName}") .asChildOf(parentSpan) @@ -111,12 +114,19 @@ trait ExecPlan extends QueryCommand { // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. Kamon.runWithSpan(span, true) { - doExecute(source, querySession) + val doEx = doExecute(source, querySession) + Kamon.histogram("query-execute-time-elapsed-step1-done") + .withTag("plan", getClass.getSimpleName) + .record(System.currentTimeMillis - startExecute) + doEx } } - // Step 2: Set up transformers and loop over all rangevectors, creating the result + // Step 2: Run connect monix pipeline to transformers, materialize the result def step2(res: ExecResult) = res.schema.map { resSchema => + Kamon.histogram("query-execute-time-elapsed-step2-start") + .withTag("plan", getClass.getSimpleName) + .record(System.currentTimeMillis - startExecute) val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") .asChildOf(parentSpan) .tag("query-id", queryContext.queryId) @@ -137,6 +147,9 @@ trait ExecPlan extends QueryCommand { paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) + Kamon.histogram("query-execute-time-elapsed-step2-transformer-pipeline-setup") + .withTag("plan", getClass.getSimpleName) + .record(System.currentTimeMillis - startExecute) val builder = SerializedRangeVector.newBuilder() @volatile var numResultSamples = 0 // BEWARE - do not modify concurrently!! finalRes._1 @@ -160,6 +173,9 @@ trait ExecPlan extends QueryCommand { } .toListL .map { r => + Kamon.histogram("query-execute-time-elapsed-step2-result-materialized") + .withTag("plan", getClass.getSimpleName) + .record(System.currentTimeMillis - startExecute) val numBytes = builder.allContainers.map(_.numBytes).sum SerializedRangeVector.queryResultBytes.record(numBytes) span.mark(s"num-bytes: $numBytes") diff --git a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala index 9036959e19..2fb68aa7b8 100644 --- a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala @@ -33,7 +33,7 @@ case class ActorPlanDispatcher(target: ActorRef) extends PlanDispatcher { val remainingTime = plan.queryContext.queryTimeoutMillis - queryTimeElapsed // Don't send if time left is very small if (remainingTime < 1) { - Task.raiseError(QueryTimeoutException(remainingTime, this.getClass.getName)) + Task.raiseError(QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) } else { val t = Timeout(FiniteDuration(remainingTime, TimeUnit.MILLISECONDS)) val fut = (target ? plan)(t).map { diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index e0326bfb8a..0a2137990e 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -3,6 +3,7 @@ package filodb.query.exec import scala.collection.mutable import scala.collection.mutable.ListBuffer +import kamon.Kamon import monix.eval.Task import monix.reactive.Observable @@ -57,6 +58,9 @@ final case class SetOperatorExec(queryContext: QueryContext, case (QueryResult(_, schema, result), i) => (schema, result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => + Kamon.histogram("query-execute-time-elapsed-step2-child-results-available") + .withTag("plan", getClass.getSimpleName) + .record(System.currentTimeMillis - queryContext.submitTime) // NOTE: We can't require this any more, as multischema queries may result in not a QueryResult if the // filter returns empty results. The reason is that the schema will be undefined. // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala index 2eff37b6e4..cb74a54df8 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -88,7 +88,7 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() - val span = Kamon.spanBuilder(s"execplan-scan-latency-TopBottomK").start() + val span = Kamon.spanBuilder(s"execplan-scan-latency-CountValues").start() try { FiloSchedulers.assertThreadName(QuerySchedName) // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release From 0e4bd74fc72cb0a4b157357d43bff081819f02e9 Mon Sep 17 00:00:00 2001 From: whizkido Date: Tue, 22 Sep 2020 22:50:26 -0700 Subject: [PATCH 24/53] Fix bugs with usage to scallops in filo cli (#895) * Fix bugs with usage to scallops in filo cli --- cli/src/main/scala/filodb.cli/CliMain.scala | 83 +++++++++++-------- .../test/scala/filodb/cli/FilodbCliSpec.scala | 33 ++++++++ 2 files changed, 80 insertions(+), 36 deletions(-) diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 0c2ba5fdd5..0a7774ad2e 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -10,6 +10,7 @@ import scala.util.Try import com.opencsv.CSVWriter import monix.reactive.Observable import org.rogach.scallop.ScallopConf +import org.rogach.scallop.exceptions.ScallopException import org.scalactic._ import filodb.coordinator._ @@ -28,38 +29,49 @@ import filodb.query._ // scalastyle:off class Arguments(args: Seq[String]) extends ScallopConf(args) { + + val dataset = opt[String]() val database = opt[String]() val command = opt[String]() val filename = opt[String]() - val configPath = opt[String]() + val configpath = opt[String]() // max # of results returned. Don't make it too high. val limit = opt[Int](default = Some(200)) - val sampleLimit = opt[Int](default = Some(1000000)) - val timeoutSeconds = opt[Int](default = Some(60)) + val samplelimit = opt[Int](default = Some(1000000)) + val timeoutseconds = opt[Int](default = Some(60)) val outfile = opt[String]() - val delimiter = opt[String](default = Some(",")).apply() - val indexName = opt[String]() + val indexname = opt[String]() val host = opt[String]() val port = opt[Int](default = Some(2552)) val promql = opt[String]() val schema = opt[String]() - val hexPk = opt[String]() - val hexVector = opt[String]() - val hexChunkInfo = opt[String]() - val vectorType = opt[String]() + val hexpk = opt[String]() + val hexvector = opt[String]() + val hexchunkinfo = opt[String]() + val vectortype = opt[String]() val matcher = opt[String]() - val labelNames = opt[List[String]](default = Some(List())) - val labelFilter = opt[Map[String, String]](default = Some(Map.empty)) - val start: Long = System.currentTimeMillis() / 1000 // promql argument is seconds since epoch - val end: Long = System.currentTimeMillis() / 1000 // promql argument is seconds since epoch + val labelnames = opt[List[String]](default = Some(List())) + val labelfilter = opt[Map[String, String]](default = Some(Map.empty)) + val currentTime = System.currentTimeMillis()/1000 + +// val starts = opt[Long](default = Some(currentTime)) + val start = opt[Long](default = Some(currentTime))// promql argument is seconds since epoch + val end = opt[Long](default = Some(currentTime))// promql argument is seconds since epoch val minutes = opt[String]() val step = opt[Long](default = Some(10)) // in seconds val chunks = opt[String]() // select either "memory" or "buffers" chunks only - val everyNSeconds = opt[String]() + val everynseconds = opt[String]() val shards = opt[List[String]]() val spread = opt[Int]() verify() + + override def onError(e: Throwable): Unit = e match { + + case ScallopException(message) => throw e + case other => throw other + } + } object CliMain extends FilodbClusterNode { @@ -84,16 +96,15 @@ object CliMain extends FilodbClusterNode { println("\nStandalone client commands:") println(" --host [--port ...] --command indexnames --dataset ") println(" --host [--port ...] --command indexvalues --indexname --dataset --shards SS") - println(" --host [--port ...] [--metricColumn ] --dataset --promql --start --step --end ") - println(" --host [--port ...] --command setup --filename | --configPath ") + println(" --host [--port ...] --dataset --promql --start --step --end ") + println(" --host [--port ...] --command setup --filename | --configpath ") println(" --host [--port ...] --command list") println(" --host [--port ...] --command status --dataset ") - println(" --host [--port ...] --command timeseriesMetadata --matcher --dataset --start --end ") - println(" --host [--port ...] --command labelValues --labelName --labelFilter --dataset ") + println(" --host [--port ...] --command labelvalues --labelName --labelfilter --dataset ") println(""" --command promFilterToPartKeyBR --promql "myMetricName{_ws_='myWs',_ns_='myNs'}" --schema prom-counter""") - println(""" --command partKeyBrAsString --hexPk 0x2C0000000F1712000000200000004B8B36940C006D794D65747269634E616D650E00C104006D794E73C004006D795773""") - println(""" --command decodeChunkInfo --hexChunkInfo 0x12e8253a267ea2db060000005046fc896e0100005046fc896e010000""") - println(""" --command decodeVector --hexVector 0x1b000000080800000300000000000000010000000700000006080400109836 --vectorType d""") + println(""" --command partKeyBrAsString --hexpk 0x2C0000000F1712000000200000004B8B36940C006D794D65747269634E616D650E00C104006D794E73C004006D795773""") + println(""" --command decodeChunkInfo --hexchunkinfo 0x12e8253a267ea2db060000005046fc896e0100005046fc896e010000""") + println(""" --command decodeVector --hexvector 0x1b000000080800000300000000000000010000000700000006080400109836 --vectortype d""") println("\nTo change config: pass -Dconfig.file=/path/to/config as first arg or set $FILO_CONFIG_FILE") println(" or override any config by passing -Dconfig.path=newvalue as first args") println("\nFor detailed debugging, uncomment the TRACE/DEBUG loggers in logback.xml and add these ") @@ -117,13 +128,13 @@ object CliMain extends FilodbClusterNode { args.minutes.map { minArg => val end = System.currentTimeMillis() / 1000 TimeStepParams(end - minArg.toInt * 60, args.step(), end) - }.getOrElse(TimeStepParams(args.start, args.step(), args.end)) + }.getOrElse(TimeStepParams(args.start(), args.step(), args.end())) } def main(rawArgs: Array[String]): Unit = { val args = new Arguments(rawArgs) try { - val timeout = args.timeoutSeconds().seconds + val timeout = args.timeoutseconds().seconds args.command.toOption match { case Some("init") => println("Initializing FiloDB Admin keyspace and tables...") @@ -148,10 +159,10 @@ object CliMain extends FilodbClusterNode { names.foreach(println) case Some("indexvalues") => - require(args.indexName.isDefined, "--indexName required") + require(args.indexname.isDefined, "--indexName required") require(args.shards.isDefined, "--shards required") val (remote, ref) = getClientAndRef(args) - val values = remote.getIndexValues(ref, args.indexName(), args.shards().head.toInt, args.limit()) + val values = remote.getIndexValues(ref, args.indexname(), args.shards().head.toInt, args.limit()) values.foreach { case (term, freq) => println(f"$term%40s\t$freq") } case Some("status") => @@ -165,31 +176,31 @@ object CliMain extends FilodbClusterNode { promFilterToPartKeyBr(args.promql(), args.schema()) case Some("partKeyBrAsString") => - require(args.hexPk.isDefined, "--hexPk must be defined") - partKeyBrAsString(args.hexPk()) + require(args.hexpk.isDefined, "--hexPk must be defined") + partKeyBrAsString(args.hexpk()) case Some("decodeChunkInfo") => - require(args.hexChunkInfo.isDefined, "--hexChunkInfo must be defined") - decodeChunkInfo(args.hexChunkInfo()) + require(args.hexchunkinfo.isDefined, "--hexChunkInfo must be defined") + decodeChunkInfo(args.hexchunkinfo()) case Some("decodeVector") => - require(args.hexVector.isDefined && args.vectorType.isDefined, "--hexVector and --vectorType must be defined") - decodeVector(args.hexVector(), args.vectorType()) + require(args.hexvector.isDefined && args.vectortype.isDefined, "--hexVector and --vectorType must be defined") + decodeVector(args.hexvector(), args.vectortype()) case Some("timeseriesMetadata") => require(args.host.isDefined && args.dataset.isDefined && args.matcher.isDefined, "--host, --dataset and --matcher must be defined") val remote = Client.standaloneClient(system, args.host(), args.port()) - val options = QOptions(args.limit(), args.sampleLimit(), args.everyNSeconds.map(_.toInt).toOption, + val options = QOptions(args.limit(), args.samplelimit(), args.everynseconds.map(_.toInt).toOption, timeout, args.shards.map(_.map(_.toInt)).toOption, args.spread.toOption.map(Integer.valueOf)) parseTimeSeriesMetadataQuery(remote, args.matcher(), args.dataset(), getQueryRange(args), true, options) case Some("labelValues") => - require(args.host.isDefined && args.dataset.isDefined && args.labelNames.isDefined, "--host, --dataset and --labelName must be defined") + require(args.host.isDefined && args.dataset.isDefined && args.labelnames.isDefined, "--host, --dataset and --labelName must be defined") val remote = Client.standaloneClient(system, args.host(), args.port()) - val options = QOptions(args.limit(), args.sampleLimit(), args.everyNSeconds.map(_.toInt).toOption, + val options = QOptions(args.limit(), args.samplelimit(), args.everynseconds.map(_.toInt).toOption, timeout, args.shards.map(_.map(_.toInt)).toOption, args.spread.toOption.map(Integer.valueOf)) - parseLabelValuesQuery(remote, args.labelNames(), args.labelFilter(), args.dataset(), + parseLabelValuesQuery(remote, args.labelnames(), args.labelfilter(), args.dataset(), getQueryRange(args), options) case x: Any => @@ -197,7 +208,7 @@ object CliMain extends FilodbClusterNode { args.promql.map { query => require(args.host.isDefined && args.dataset.isDefined, "--host and --dataset must be defined") val remote = Client.standaloneClient(system, args.host(), args.port()) - val options = QOptions(args.limit(), args.sampleLimit(), args.everyNSeconds.toOption.map(_.toInt), + val options = QOptions(args.limit(), args.samplelimit(), args.everynseconds.toOption.map(_.toInt), timeout, args.shards.toOption.map(_.map(_.toInt)), args.spread.toOption.map(Integer.valueOf)) parsePromQuery2(remote, query, args.dataset(), getQueryRange(args), options) } diff --git a/cli/src/test/scala/filodb/cli/FilodbCliSpec.scala b/cli/src/test/scala/filodb/cli/FilodbCliSpec.scala index d9eea605b1..a2f80b4bae 100644 --- a/cli/src/test/scala/filodb/cli/FilodbCliSpec.scala +++ b/cli/src/test/scala/filodb/cli/FilodbCliSpec.scala @@ -1,10 +1,14 @@ package filodb.cli +import org.rogach.scallop.exceptions.ScallopException + import filodb.coordinator.{ActorName, ClusterRole, RunnableSpec} class FilodbCliSpec extends RunnableSpec { "A Filodb Cli" must { "initialize" in { + + testScallopOptions() eventually(CliMain.cluster.isInitialized) } "create and setup the coordinatorActor and clusterActor" in { @@ -20,4 +24,33 @@ class FilodbCliSpec extends RunnableSpec { eventually(CliMain.cluster.isTerminated) } } + + def testScallopOptions(): Unit = { + + + parseSucessFully("--host localhost --command indexnames --dataset prometheus") + parseSucessFully("--host localhost --port 6564 --command indexvalues --indexname asdasd --dataset prometheus --shards SS") + parseSucessFully("""--host localhost --port 6564 --dataset "adadasd" --promql "myMetricName{_ws_='myWs',_ns_='myNs'}" --start 1212 --step 5555 --end 1212""") + parseSucessFully("--host localhost --port 6564 --command timeseriesmetadata --matcher a=b --dataset prometheus --start 123123 --end 13123") + parseSucessFully("--host localhost --port 6564 --command labelvalues --labelnames a --labelfilter a=b --dataset prometheus") + parseSucessFully("""--command promFilterToPartKeyBR --promql "myMetricName{_ws_='myWs',_ns_='myNs'}" --schema prom-counter""") + parseSucessFully("""--command partKeyBrAsString --hexpk 0x2C0000000F1712000000200000004B8B36940C006D794D65747269634E616D650E00C104006D794E73C004006D795773""") + parseSucessFully("""--command decodeChunkInfo --hexchunkinfo 0x12e8253a267ea2db060000005046fc896e0100005046fc896e010000""") + parseSucessFully("""--command decodeVector --hexvector 0x1b000000080800000300000000000000010000000700000006080400109836 --vectortype d""") + + parserError("""--host localhost --port 6564 --metriccolumn adasdasd --dataset "adadasd" --promql "myMetricName{_ws_='myWs',_ns_='myNs'}" --start 1231673123675123 --step 13131312313123123 --end 5""") + parserError("""--command partKeyBrAsString --hexPk 0x2C0000000F1712000000200000004B8B36940C006D794D65747269634E616D650E00C104006D794E73C004006D795773""") + parserError("""--command decodeChunkInfo --hexChunkInfo 0x12e8253a267ea2db060000005046fc896e0100005046fc896e010000""") + parserError("""--command decodeVector --hexVector 0x1b000000080800000300000000000000010000000700000006080400109836 --vectortype d""") + + } + def parseSucessFully(commandLine: String): Unit = { + new Arguments(commandLine.split(" ")) + } + + def parserError(commandLine: String):Unit = { + intercept[ScallopException]{ + new Arguments(commandLine.split(" ")) + } + } } From dbafce94a675cea06b8dbe7abbbdc938aca0bc22 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 23 Sep 2020 10:41:53 -0700 Subject: [PATCH 25/53] bug(coord): Add extra by and on keys in QueryPlanner for specific time ranges (#901) * Extra join-on or group-by keys are added during planning time and not during runtime. * BinaryJoinExec changes for this are removed. * Keys are added during planning only if query overlaps with configured time ranges * Keeps ExecPlans clean, and improves unit testability --- .../queryplanner/LogicalPlanUtils.scala | 50 +++++++++++++++ .../queryplanner/PlannerMaterializer.scala | 22 ------- .../queryplanner/SingleClusterPlanner.scala | 11 ++-- .../queryplanner/SinglePartitionPlanner.scala | 14 +++-- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 61 +++++++++++++++++++ .../SinglePartitionPlannerSpec.scala | 3 +- core/src/main/resources/filodb-defaults.conf | 6 ++ .../scala/filodb.core/query/QueryConfig.scala | 5 ++ core/src/test/resources/application_test.conf | 1 + .../filodb/query/exec/BinaryJoinExec.scala | 9 +-- .../filodb/query/exec/SetOperatorExec.scala | 4 +- .../query/exec/BinaryJoinExecSpec.scala | 6 +- 12 files changed, 146 insertions(+), 46 deletions(-) create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 01561b1d9b..871277169c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.ArrayBuffer import com.typesafe.scalalogging.StrictLogging +import filodb.coordinator.queryplanner.LogicalPlanUtils.{getLookBackMillis, getTimeFromLogicalPlan} import filodb.core.query.{QueryContext, RangeParams} import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants @@ -236,3 +237,52 @@ object LogicalPlanUtils extends StrictLogging { } } } + +/** + * Temporary utility to modify plan to add extra join-on keys or group-by keys + * for specific time ranges. + */ +object ExtraOnByKeysUtil { + + def getRealOnLabels(lp: BinaryJoin, addStepKeyTimeRanges: Seq[Seq[Long]]): Seq[String] = { + if (shouldAddExtraKeys(lp.lhs, addStepKeyTimeRanges: Seq[Seq[Long]]) || + shouldAddExtraKeys(lp.rhs, addStepKeyTimeRanges: Seq[Seq[Long]])) { + // add extra keys if ignoring clause is not specified + if (lp.ignoring.isEmpty) lp.on ++ extraByOnKeys + else lp.on + } else { + lp.on + } + } + + def getRealByLabels(lp: Aggregate, addStepKeyTimeRanges: Seq[Seq[Long]]): Seq[String] = { + if (shouldAddExtraKeys(lp, addStepKeyTimeRanges)) { + // add extra keys if without clause is not specified + if (lp.without.isEmpty) lp.by ++ extraByOnKeys + else lp.by + } else { + lp.by + } + } + + private def shouldAddExtraKeys(lp: LogicalPlan, addStepKeyTimeRanges: Seq[Seq[Long]]): Boolean = { + // need to check if raw time range in query overlaps with configured addStepKeyTimeRanges + val range = getTimeFromLogicalPlan(lp) + val lookback = getLookBackMillis(lp) + queryTimeRangeRequiresExtraKeys(range.startMs - lookback, range.endMs, addStepKeyTimeRanges) + } + + val extraByOnKeys = Seq("_pi_", "_step_") + /** + * Returns true if two time ranges (x1, x2) and (y1, y2) overlap + */ + private def rangeOverlaps(x1: Long, x2: Long, y1: Long, y2: Long): Boolean = { + Math.max(x1, y1) <= Math.min(x2, y2) + } + + private def queryTimeRangeRequiresExtraKeys(rawStartMs: Long, + rawEndMs: Long, + addStepKeyTimeRanges: Seq[Seq[Long]]): Boolean = { + addStepKeyTimeRanges.exists { r => rangeOverlaps(rawStartMs, rawEndMs, r(0), r(1)) } + } +} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index 8b85d427b3..395f737396 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -1,8 +1,6 @@ package filodb.coordinator.queryplanner -import com.typesafe.scalalogging.StrictLogging import java.util.concurrent.ThreadLocalRandom -import kamon.Kamon import filodb.core.metadata.{DatasetOptions, Schemas} import filodb.core.query.{QueryContext, RangeParams} @@ -141,23 +139,3 @@ trait PlannerMaterializer { } } } - -object PlannerUtil extends StrictLogging { - - val bjBetweenAggAndNonAgg = Kamon.counter("join-between-agg-non-agg").withoutTags() - - def validateBinaryJoin(lhs: Seq[ExecPlan], rhs: Seq[ExecPlan], queryContext: QueryContext): Any = { - - if (lhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && - !rhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { - logger.info(s"Saw Binary Join between aggregate(lhs) and non-aggregate (rhs). ${queryContext.origQueryParams}") - bjBetweenAggAndNonAgg.increment() - } - - if (!lhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && - rhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { - logger.info(s"Saw Binary Join between non-aggregate(lhs) and aggregate(rhs): ${queryContext.origQueryParams}") - bjBetweenAggAndNonAgg.increment() - } - } -} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index befd47aec1..09e4fc0739 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -202,8 +202,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, val stitchedRhs = if (rhs.needsStitch) Seq(StitchRvsExec(qContext, pickDispatcher(rhs.plans), rhs.plans)) else rhs.plans - - PlannerUtil.validateBinaryJoin(lhs.plans, rhs.plans, qContext) + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(lp, queryConfig.addExtraOnByKeysTimeRanges) // TODO Currently we create separate exec plan node for stitching. // Ideally, we can go one step further and add capability to NonLeafNode plans to pre-process @@ -214,11 +213,11 @@ class SingleClusterPlanner(dsRef: DatasetRef, val targetActor = pickDispatcher(stitchedLhs ++ stitchedRhs) val joined = if (lp.operator.isInstanceOf[SetOperator]) Seq(exec.SetOperatorExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, - LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(onKeysReal, dsOptions.metricColumn), LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), dsOptions.metricColumn)) else Seq(BinaryJoinExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.cardinality, - LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(onKeysReal, dsOptions.metricColumn), LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), LogicalPlanUtils.renameLabels(lp.include, dsOptions.metricColumn), dsOptions.metricColumn)) PlanResult(joined, false) @@ -227,6 +226,8 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def materializeAggregate(qContext: QueryContext, lp: Aggregate): PlanResult = { val toReduceLevel1 = walkLogicalPlanTree(lp.vectors, qContext) + val byKeysReal = ExtraOnByKeysUtil.getRealByLabels(lp, queryConfig.addExtraOnByKeysTimeRanges) + // Now we have one exec plan per shard /* * Note that in order for same overlapping RVs to not be double counted when spread is increased, @@ -242,7 +243,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, toReduceLevel1.plans.foreach { _.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, LogicalPlanUtils.renameLabels(lp.without, dsOptions.metricColumn), - LogicalPlanUtils.renameLabels(lp.by, dsOptions.metricColumn))) + LogicalPlanUtils.renameLabels(byKeysReal, dsOptions.metricColumn))) } val toReduceLevel2 = diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index d1a5839787..7c27d89a1f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -1,6 +1,6 @@ package filodb.coordinator.queryplanner -import filodb.core.query.{PromQlQueryParams, QueryContext} +import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters, SetOperator} import filodb.query.exec._ @@ -12,8 +12,10 @@ import filodb.query.exec._ * @param plannerSelector a function that selects the planner name given the metric name * */ -class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelector: String => String, - datasetMetricColumn: String) +class SinglePartitionPlanner(planners: Map[String, QueryPlanner], + plannerSelector: String => String, + datasetMetricColumn: String, + queryConfig: QueryConfig) extends QueryPlanner { def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { @@ -58,15 +60,15 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelecto case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, rhsQueryContext) } - PlannerUtil.validateBinaryJoin(Seq(lhsExec), Seq(rhsExec), qContext) + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) if (logicalPlan.operator.isInstanceOf[SetOperator]) SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) else BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - logicalPlan.cardinality, LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala new file mode 100644 index 0000000000..9dcfd2af2c --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -0,0 +1,61 @@ +package filodb.coordinator.queryplanner + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +import filodb.prometheus.ast.TimeStepParams +import filodb.prometheus.parse.Parser +import filodb.query.{Aggregate, BinaryJoin} + +class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { + + import ExtraOnByKeysUtil._ + + val extraKeysTimeRange = Seq(Seq(25000000L, 30000000L)) + + it("should add extra by keys for aggregate when no keys present") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m]))""", TimeStepParams(20000, 100, 30000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual extraByOnKeys + } + + it("should not add extra by keys for aggregate when without present") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m])) without (pod)""", + TimeStepParams(20000, 100, 30000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty + } + + it("should add extra by keys for aggregate when on already keys present") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m])) by (pod)""", + TimeStepParams(20000, 100, 30000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq("pod") ++ extraByOnKeys + } + + it("should add extra on keys for binary join when no keys present") { + val lp = Parser.queryRangeToLogicalPlan("""foo + bar """, + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual extraByOnKeys + } + + it("should add extra on keys for binary join when on already keys present") { + val lp = Parser.queryRangeToLogicalPlan("""foo + on(pod) bar """, + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq("pod") ++ extraByOnKeys + } + + it("should not add extra on keys for binary join when ignoring present") { + val lp = Parser.queryRangeToLogicalPlan("""foo + ignoring(pod) bar """, + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty + } + + it("should add extra keys even with overlap is inside of the first lookback range") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m]))""", TimeStepParams(30005L, 100, 40000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual extraByOnKeys + } + + it("should not add extra keys when no overlap with configured time ranges") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m]))""", TimeStepParams(40000L, 100, 50000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty + } + +} diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 674e8f9ea3..60a7b98382 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -4,6 +4,7 @@ import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import monix.execution.Scheduler + import filodb.coordinator.ShardMapper import filodb.core.{DatasetRef, MetricsTestData} import filodb.core.metadata.Schemas @@ -80,7 +81,7 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val plannerSelector = (metricName: String) => { if (metricName.equals("rr1")) "rules1" else if (metricName.equals("rr2")) "rules2" else "local" } - val engine = new SinglePartitionPlanner(planners, plannerSelector, "_metric_") + val engine = new SinglePartitionPlanner(planners, plannerSelector, "_metric_", queryConfig) it("should generate Exec plan for simple query") { val lp = Parser.queryToLogicalPlan("test{job = \"app\"}", 1000, 1000) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index c30da8d737..a5b0ec168e 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -174,6 +174,12 @@ filodb { # Enable faster rate/increase/delta calculations. Depends on drop detection in chunks (detectDrops=true) faster-rate = true + # Time ranges for which additional join/by keys need to be added implicitly. Add as a 2D array, example: + # [ + # [1600224662, 1600229662], + # [1600204662, 1600209662] + # ] + add-extra-by-on-key-time-ranges = [] } shard-manager { diff --git a/core/src/main/scala/filodb.core/query/QueryConfig.scala b/core/src/main/scala/filodb.core/query/QueryConfig.scala index 08f14e29c4..b40d291b2d 100644 --- a/core/src/main/scala/filodb.core/query/QueryConfig.scala +++ b/core/src/main/scala/filodb.core/query/QueryConfig.scala @@ -15,6 +15,11 @@ class QueryConfig(queryConfig: Config) { lazy val minStepMs = queryConfig.getDuration("min-step").toMillis lazy val fastReduceMaxWindows = queryConfig.getInt("fastreduce-max-windows") lazy val routingConfig = queryConfig.getConfig("routing") + lazy val addExtraOnByKeysTimeRanges = { + val v = queryConfig.as[Seq[Seq[Long]]]("add-extra-by-on-key-time-ranges") + require(v.forall(r => r.size == 2 && r(0) < r(1))) + v + } /** * Feature flag test: returns true if the config has an entry with "true", "t" etc diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index ec2b5ceede..7699534503 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -63,6 +63,7 @@ filodb { min-step = 1 ms faster-rate = true fastreduce-max-windows = 50 + add-extra-by-on-key-time-ranges = [] } spread-default = 1 diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index dc7ebacf67..a4d390a241 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -52,8 +52,6 @@ final case class BinaryJoinExec(queryContext: QueryContext, require(!on.contains(metricColumn), "On cannot contain metric name") val onLabels = on.map(Utf8Str(_)).toSet - // publishInterval and step tags always needs to be included in join key - val withExtraOnLabels = onLabels ++ Seq("_pi_".utf8, "_step_".utf8) val ignoringLabels = ignoring.map(Utf8Str(_)).toSet val ignoringLabelsForJoin = ignoringLabels + metricColumn.utf8 // if onLabels is non-empty, we are doing matching based on on-label, otherwise we are @@ -125,7 +123,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, } private def joinKeys(rvk: RangeVectorKey): Map[Utf8Str, Utf8Str] = { - if (onLabels.nonEmpty) rvk.labelValues.filter(lv => withExtraOnLabels.contains(lv._1)) + if (onLabels.nonEmpty) rvk.labelValues.filter(lv => onLabels.contains(lv._1)) else rvk.labelValues.filterNot(lv => ignoringLabelsForJoin.contains(lv._1)) } @@ -136,9 +134,8 @@ final case class BinaryJoinExec(queryContext: QueryContext, if (binaryOp.isInstanceOf[MathOperator]) result = result - Utf8Str(metricColumn) if (cardinality == Cardinality.OneToOne) { - result = - if (onLabels.nonEmpty) result.filter(lv => withExtraOnLabels.contains(lv._1)) // retain what is in onLabel list - else result.filterNot(lv => ignoringLabels.contains(lv._1)) // remove the labels in ignoring label list + result = if (onLabels.nonEmpty) result.filter(lv => onLabels.contains(lv._1)) // retain what is in onLabel list + else result.filterNot(lv => ignoringLabels.contains(lv._1)) // remove the labels in ignoring label list } else if (cardinality == Cardinality.OneToMany || cardinality == Cardinality.ManyToOne) { // For group_left/group_right add labels in include from one side. Result should have all keys from many side include.foreach { x => diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index 0a2137990e..d210bf150c 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -41,8 +41,6 @@ final case class SetOperatorExec(queryContext: QueryContext, require(!on.contains(metricColumn), "On cannot contain metric name") val onLabels = on.map(Utf8Str(_)).toSet - // TODO Add unit tests for automatic inclusion of _pi_ and _step_ in the join key - val withExtraOnLabels = onLabels ++ Seq("_pi_".utf8, "_step_".utf8) val ignoringLabels = ignoring.map(Utf8Str(_)).toSet + metricColumn.utf8 // if onLabels is non-empty, we are doing matching based on on-label, otherwise we are // doing matching based on ignoringLabels even if it is empty @@ -83,7 +81,7 @@ final case class SetOperatorExec(queryContext: QueryContext, } private def joinKeys(rvk: RangeVectorKey): Map[Utf8Str, Utf8Str] = { - if (onLabels.nonEmpty) rvk.labelValues.filter(lv => withExtraOnLabels.contains(lv._1)) + if (onLabels.nonEmpty) rvk.labelValues.filter(lv => onLabels.contains(lv._1)) else rvk.labelValues.filterNot(lv => ignoringLabels.contains(lv._1)) } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index e3f1abfd1f..82dcf6d8f3 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -143,7 +143,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { result.map(_.key).toSet.size shouldEqual 100 } - it("should implictly add step and pi tag as join key on OneToOne joins") { + it("should deal with additional step and pi tag as join key on OneToOne joins") { val lhs1: RangeVector = new RangeVector { val key: RangeVectorKey = CustomRangeVectorKey( Map("__name__".utf8 -> s"someMetricLhs".utf8, "_pi_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) @@ -177,7 +177,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { Array(dummyPlan), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, Cardinality.OneToOne, - Nil, Nil, Nil, "__name__") + Seq("_step_", "_pi_"), Nil, Nil, "__name__") // scalastyle:off val lhs = QueryResult("someId", null, Seq(lhs1, lhs2).map(rv => SerializedRangeVector(rv, schema))) @@ -193,7 +193,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } - it("should implictly add step and pi tag as join key on OneToMany joins") { + it("should deal with implictly added step and pi tag as join key on OneToMany joins") { val lhs1: RangeVector = new RangeVector { val key: RangeVectorKey = CustomRangeVectorKey( Map("__name__".utf8 -> s"someMetricLhs".utf8, "_pi_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) From 3cf6724374b1d37b004e8dafb573d3384d48e235 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 23 Sep 2020 19:12:11 -0700 Subject: [PATCH 26/53] bug(coord): Extra on-keys should be added only if on is nonEmpty (#904) Query Planner changes to add join key were not done equivalently. Now adding extra on-keys only if on specifier is non-empty already --- .../filodb.coordinator/queryplanner/LogicalPlanUtils.scala | 4 ++-- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 871277169c..2fad2394ec 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -247,8 +247,8 @@ object ExtraOnByKeysUtil { def getRealOnLabels(lp: BinaryJoin, addStepKeyTimeRanges: Seq[Seq[Long]]): Seq[String] = { if (shouldAddExtraKeys(lp.lhs, addStepKeyTimeRanges: Seq[Seq[Long]]) || shouldAddExtraKeys(lp.rhs, addStepKeyTimeRanges: Seq[Seq[Long]])) { - // add extra keys if ignoring clause is not specified - if (lp.ignoring.isEmpty) lp.on ++ extraByOnKeys + // add extra keys if ignoring clause is not specified and on is specified + if (lp.on.nonEmpty) lp.on ++ extraByOnKeys else lp.on } else { lp.on diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala index 9dcfd2af2c..f3d5bd60df 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -30,10 +30,10 @@ class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq("pod") ++ extraByOnKeys } - it("should add extra on keys for binary join when no keys present") { + it("should not add extra on keys for binary join when no join keys present") { val lp = Parser.queryRangeToLogicalPlan("""foo + bar """, TimeStepParams(20000, 100, 30000)) - getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual extraByOnKeys + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty } it("should add extra on keys for binary join when on already keys present") { From d21f39043adb60c8b59481da7080c7aa680b762a Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 24 Sep 2020 12:02:04 -0700 Subject: [PATCH 27/53] bug(query): head call on empty list in LogicalPlanUtils (#906) --- .../queryplanner/LogicalPlanUtils.scala | 31 ++++++++++++------- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 6 ++++ 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 2fad2394ec..c8a553f3cc 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -155,27 +155,36 @@ object LogicalPlanUtils extends StrictLogging { * NOTE: Plan should be PeriodicSeriesPlan */ def getRawSeriesStartTime(logicalPlan: LogicalPlan): Option[Long] = { - LogicalPlan.findLeafLogicalPlans(logicalPlan).head match { - case lp: RawSeries => lp.rangeSelector match { - case rs: IntervalSelector => Some(rs.from) - case _ => None + val leaf = LogicalPlan.findLeafLogicalPlans(logicalPlan) + if (leaf.isEmpty) None else { + leaf.head match { + case lp: RawSeries => lp.rangeSelector match { + case rs: IntervalSelector => Some(rs.from) + case _ => None + } + case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") } - case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") } } def getOffsetMillis(logicalPlan: LogicalPlan): Long = { - LogicalPlan.findLeafLogicalPlans(logicalPlan).head match { - case lp: RawSeries => lp.offsetMs.getOrElse(0) - case _ => 0 + val leaf = LogicalPlan.findLeafLogicalPlans(logicalPlan) + if (leaf.isEmpty) 0 else { + leaf.head match { + case lp: RawSeries => lp.offsetMs.getOrElse(0) + case _ => 0 + } } } def getLookBackMillis(logicalPlan: LogicalPlan): Long = { val staleDataLookbackMillis = WindowConstants.staleDataLookbackMillis - LogicalPlan.findLeafLogicalPlans(logicalPlan).head match { - case lp: RawSeries => lp.lookbackMs.getOrElse(staleDataLookbackMillis) - case _ => 0 + val leaf = LogicalPlan.findLeafLogicalPlans(logicalPlan) + if (leaf.isEmpty) 0 else { + leaf.head match { + case lp: RawSeries => lp.lookbackMs.getOrElse(staleDataLookbackMillis) + case _ => 0 + } } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala index f3d5bd60df..249ed56a0d 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -58,4 +58,10 @@ class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty } +// TODO remove test when unnecessary binary join bug is fixed + it("should not add extra on keys for nested scalar queries") { + val lp = Parser.queryRangeToLogicalPlan("""foo + 10/2""", + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty + } } From 570e2f1adfa3b329741a7e6ca1b57c0974e64f05 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 24 Sep 2020 16:07:54 -0700 Subject: [PATCH 28/53] bug(query): BinaryJoinExec is created for queries having nested scalar expressions (#903) --- .../queryplanner/PlannerMaterializer.scala | 5 ++- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 3 +- .../queryplanner/ScalarQueriesSpec.scala | 19 ++++++++ .../filodb/prometheus/ast/Expressions.scala | 44 ++++++++++++------- .../filodb/prometheus/parse/ParserSpec.scala | 7 ++- 5 files changed, 55 insertions(+), 23 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index 395f737396..2f3abeb918 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -48,9 +48,10 @@ trait PlannerMaterializer { param match { case num: ScalarFixedDoublePlan => StaticFuncArgs(num.scalar, num.timeStepParams) case s: ScalarVaryingDoublePlan => ExecPlanFuncArgs(materialize(s, qContext), - RangeParams(s.startMs, s.stepMs, s.endMs)) + RangeParams(s.startMs, s.stepMs, s.endMs)) case t: ScalarTimeBasedPlan => TimeFuncArgs(t.rangeParams) - case _ => throw new UnsupportedOperationException("Invalid logical plan") + case s: ScalarBinaryOperation => ExecPlanFuncArgs(materialize(s, qContext), + RangeParams(s.startMs, s.stepMs, s.endMs)) } } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala index 249ed56a0d..a676686b95 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -58,9 +58,8 @@ class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty } -// TODO remove test when unnecessary binary join bug is fixed it("should not add extra on keys for nested scalar queries") { - val lp = Parser.queryRangeToLogicalPlan("""foo + 10/2""", + val lp = Parser.queryRangeToLogicalPlan("""foo * 10/2 + bar""", TimeStepParams(20000, 100, 30000)) getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala index a2d11c5252..22e7811e68 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala @@ -471,4 +471,23 @@ class ScalarQueriesSpec extends AnyFunSpec with Matchers { |on InProcessPlanDispatcher""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } + + it("should generate execPlan for binary join with nested scalar query") { + val lp = Parser.queryToLogicalPlan("""sum(http_requests_total{job = "app"}) - 10/2""", 1000, 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val expected = + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~ + |-E~ScalarBinaryOperationExec(params = RangeParams(1000,1000,1000), operator = DIV, lhs = Left(10.0), rhs = Left(2.0)) on InProcessPlanDispatcher + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List()) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-669137818]) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-669137818]) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-669137818])""".stripMargin + maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) + } } diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala index 9329f64207..f213bc6392 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala @@ -3,7 +3,6 @@ package filodb.prometheus.ast import filodb.core.query.RangeParams import filodb.query._ - trait Expressions extends Aggregates with Functions { case class UnaryExpression(operator: Operator, operand: Expression) extends Expression { @@ -38,7 +37,9 @@ trait Expressions extends Aggregates with Functions { case _ => false } } + // scalastyle:off method.length + // scalastyle:off cyclomatic.complexity override def toSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { if (hasScalarResult(lhs) && hasScalarResult(rhs)) { val rangeParams = RangeParams(timeParams.start, timeParams.step, timeParams.end) @@ -95,26 +96,35 @@ trait Expressions extends Aggregates with Functions { // node_info + http_requests case (lh: PeriodicSeries, rh: PeriodicSeries) => - val seriesPlanLhs = lh.toSeriesPlan(timeParams) - val seriesPlanRhs = rh.toSeriesPlan(timeParams) - val cardinality = if (operator.getPlanOperator.isInstanceOf[SetOperator]) - Cardinality.ManyToMany - else - vectorMatch.map(_.cardinality.cardinality).getOrElse(Cardinality.OneToOne) - - val matcher = vectorMatch.flatMap(_.matching) - val onLabels = matcher.filter(_.isInstanceOf[On]).map(_.labels) - val ignoringLabels = matcher.filter(_.isInstanceOf[Ignoring]).map(_.labels) - - BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs, - onLabels.getOrElse(Nil), ignoringLabels.getOrElse(Nil), - vectorMatch.flatMap(_.grouping).map(_.labels).getOrElse(Nil)) - + //10/2 + foo + if (hasScalarResult(lh)) { + val scalar = lh.toSeriesPlan(timeParams).asInstanceOf[ScalarPlan] + val seriesPlan = rh.toSeriesPlan(timeParams) + ScalarVectorBinaryOperation(operator.getPlanOperator, scalar, seriesPlan, scalarIsLhs = true) + } else if (hasScalarResult(rh)) { // foo + 10/2 + val scalar = rh.toSeriesPlan(timeParams).asInstanceOf[ScalarPlan] + val seriesPlan = lh.toSeriesPlan(timeParams) + ScalarVectorBinaryOperation(operator.getPlanOperator, scalar, seriesPlan, scalarIsLhs = false) + } else { + val seriesPlanLhs = lh.toSeriesPlan(timeParams) + val seriesPlanRhs = rh.toSeriesPlan(timeParams) + val cardinality = if (operator.getPlanOperator.isInstanceOf[SetOperator]) + Cardinality.ManyToMany + else + vectorMatch.map(_.cardinality.cardinality).getOrElse(Cardinality.OneToOne) + + val matcher = vectorMatch.flatMap(_.matching) + val onLabels = matcher.filter(_.isInstanceOf[On]).map(_.labels) + val ignoringLabels = matcher.filter(_.isInstanceOf[Ignoring]).map(_.labels) + BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs, + onLabels.getOrElse(Nil), ignoringLabels.getOrElse(Nil), + vectorMatch.flatMap(_.grouping).map(_.labels).getOrElse(Nil)) + } case _ => throw new UnsupportedOperationException("Invalid operands") } } } // scalastyle:on method.length + // scalastyle:on cyclomatic.complexity } - } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 177e141319..e53a1833b6 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -550,7 +550,10 @@ class ParserSpec extends AnyFunSpec with Matchers { "PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),Some(2000000)),1524855988000,1000000,1524855988000,Some(2000000))", "sum(rate(foo{job=\"SNRT-App-0\"}[5i]))" -> "Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(5000000),None),1524855988000,1000000,1524855988000,5000000,Rate,true,List(),None),List(),List(),List())", "rate(foo{job=\"SNRT-App-0\"}[5i]) + rate(bar{job=\"SNRT-App-0\"}[4i])" -> "BinaryJoin(PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(5000000),None),1524855988000,1000000,1524855988000,5000000,Rate,true,List(),None),ADD,OneToOne,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(bar))),List(),Some(4000000),None),1524855988000,1000000,1524855988000,4000000,Rate,true,List(),None),List(),List(),List())", - "sum(rate(foo{job=\"SNRT-App-0\"}[0.5i]))" -> "Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(500000),None),1524855988000,1000000,1524855988000,500000,Rate,true,List(),None),List(),List(),List())" + "sum(rate(foo{job=\"SNRT-App-0\"}[0.5i]))" -> "Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(500000),None),1524855988000,1000000,1524855988000,500000,Rate,true,List(),None),List(),List(),List())", + "http_requests_total - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false)", + "foo - http_requests_total * 2^3" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),SUB,OneToOne,ScalarVectorBinaryOperation(MUL,ScalarBinaryOperation(POW,Left(2.0),Left(3.0),RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false),List(),List(),List())", + "sum(http_requests_total) - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false)" ) val qts: Long = 1524855988L @@ -559,7 +562,7 @@ class ParserSpec extends AnyFunSpec with Matchers { info(s"Parsing $q") val lp = Parser.queryToLogicalPlan(q, qts, step) if (lp.isInstanceOf[BinaryJoin]) - printBinaryJoin(lp) + printBinaryJoin(lp) lp.toString shouldEqual (e) } } From 183915f6ba1bfe9600ebea429161f70160739a3a Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 30 Sep 2020 14:13:16 -0700 Subject: [PATCH 29/53] bug(query): lucene matches timeseries for notEquals filter even if the column is not present (#878) --- .../memstore/PartKeyLuceneIndex.scala | 4 +-- .../exec/MultiSchemaPartitionsExecSpec.scala | 36 +++++++++++++++++-- 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index e260e18ead..e064a1ae9e 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -451,9 +451,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, new TermQuery(term) case NotEquals(value) => val term = new Term(column, value.toString) - val allDocs = new MatchAllDocsQuery val booleanQuery = new BooleanQuery.Builder - booleanQuery.add(allDocs, Occur.FILTER) + val termAll = new Term(column, ".*") + booleanQuery.add(new RegexpQuery(termAll, RegExp.NONE), Occur.FILTER) booleanQuery.add(new TermQuery(term), Occur.MUST_NOT) booleanQuery.build() case In(values) => diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 49925f7531..63a7bcbbec 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -51,7 +51,7 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) val metric = "http_req_total" - val partKeyLabelValues = Map("job" -> "myCoolService", "instance" -> "someHost:8787") + val partKeyLabelValues = Map("job" -> "myCoolService", "instance" -> "someHost:8787", "host" -> "host-1") val partKeyKVWithMetric = partKeyLabelValues ++ Map("_metric_" -> metric) val partTagsUTF8 = partKeyLabelValues.map { case (k, v) => (k.utf8, v.utf8) } val now = System.currentTimeMillis() @@ -119,7 +119,8 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF it ("should read raw samples from Memstore using IntervalSelector") { import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), - ColumnFilter("job", Filter.Equals("myCoolService".utf8))) + ColumnFilter("job", Filter.Equals("myCoolService".utf8)), + ColumnFilter("instance", Filter.NotEquals("SomeJob".utf8))) // read from an interval of 100000ms, resulting in 11 samples val startTime = now - numRawSamples * reportingInterval val endTime = now - (numRawSamples-10) * reportingInterval @@ -470,5 +471,36 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF thrown.getCause.getMessage shouldEqual "Query timeout in filodb.query.exec.MultiSchemaPartitionsExec after 180 seconds" } + it ("""should not return range vectors with !="" where column is not present""") { + import ZeroCopyUTF8String._ + val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), + ColumnFilter("job", Filter.Equals("myCoolService".utf8)), + ColumnFilter("dc", Filter.NotEquals("".utf8))) + val startTime = now - numRawSamples * reportingInterval + val endTime = now - (numRawSamples-10) * reportingInterval + + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, + dsRef, 0, filters, TimeRangeChunkScan(startTime, endTime)) + + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue + val result = resp.asInstanceOf[QueryResult] + result.result.size shouldEqual 0 + } + + it ("""should return range vectors when it satisfies NotEquals condition""") { + import ZeroCopyUTF8String._ + val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), + ColumnFilter("job", Filter.Equals("myCoolService".utf8)), + ColumnFilter("host", Filter.NotEquals("host".utf8))) + val startTime = now - numRawSamples * reportingInterval + val endTime = now - (numRawSamples-10) * reportingInterval + + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, + dsRef, 0, filters, TimeRangeChunkScan(startTime, endTime)) + + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue + val result = resp.asInstanceOf[QueryResult] + result.result.size shouldEqual 1 + } } From 19b890137d217d48ce76464f863d5d2a50457f70 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Thu, 1 Oct 2020 09:25:25 -0700 Subject: [PATCH 30/53] fix(core,memory): Allow blocks to be reclaimed when ODP fails. (#911) --- .../memstore/DemandPagedChunkStore.scala | 29 ++++++++++++++----- .../memstore/TimeSeriesShard.scala | 2 +- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index e6945bb55f..14048ba2ba 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -2,11 +2,13 @@ package filodb.core.memstore import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import org.jctools.maps.NonBlockingHashMapLong +import spire.syntax.cfor._ import filodb.core.store._ import filodb.memory.{BlockManager, BlockMemFactory} @@ -65,6 +67,7 @@ extends RawToPartitionMaker with StrictLogging { /** * Stores raw chunks into offheap memory and populates chunks into partition */ + //scalastyle:off def populateRawChunks(rawPartition: RawPartData): Task[ReadablePartition] = Task { FiloSchedulers.assertThreadName(FiloSchedulers.PopulateChunksSched) // Find the right partition given the partition key @@ -85,10 +88,15 @@ extends RawToPartitionMaker with StrictLogging { val chunkID = ChunkSetInfo.getChunkID(infoBytes) if (!tsPart.chunkmapContains(chunkID)) { + val chunkPtrs = new ArrayBuffer[BinaryVectorPtr](rawVectors.length) memFactory.startMetaSpan() - val chunkPtrs = copyToOffHeap(rawVectors, memFactory) - val metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), - tsPart.schema.data.blockMetaSize.toShort) + var metaAddr: Long = 0 + try { + copyToOffHeap(rawVectors, memFactory, chunkPtrs) + } finally { + metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), + tsPart.schema.data.blockMetaSize.toShort) + } require(metaAddr != 0) val infoAddr = metaAddr + 4 // Important: don't point at partID val inserted = tsPart.addChunkInfoIfAbsent(chunkID, infoAddr) @@ -109,6 +117,7 @@ extends RawToPartitionMaker with StrictLogging { throw new RuntimeException(s"Partition [${new String(rawPartition.partitionKey)}] not found, this is bad") } } + //scalastyle:on /** * For a given chunkset, this method calculates the time bucket the chunks fall in. @@ -118,16 +127,20 @@ extends RawToPartitionMaker with StrictLogging { (ChunkSetInfo.getEndTime(infoBytes) / flushIntervalMillis) * flushIntervalMillis /** - * Copies the onHeap contents read from ColStore into off-heap using the given memFactory + * Copies the onHeap contents read from ColStore into off-heap using the given memFactory. + * If an exception is thrown by this method, the tail of chunkPtrs sequence isn't filled in. + * + * @param chunkPtrs filled in by this method */ private def copyToOffHeap(buffers: Array[ByteBuffer], - memFactory: BlockMemFactory): Array[BinaryVectorPtr] = { - buffers.map { buf => - // TODO: if in case the buffer is offheap/direct buffer already, maybe we don't need to copy it? + memFactory: BlockMemFactory, + chunkPtrs: ArrayBuffer[BinaryVectorPtr]): Unit = { + cforRange { 0 until buffers.length } { i => + val buf = buffers(i) val (bufBase, bufOffset, bufLen) = UnsafeUtils.BOLfromBuffer(buf) val vectorAddr = memFactory.allocateOffheap(bufLen) UnsafeUtils.unsafe.copyMemory(bufBase, bufOffset, UnsafeUtils.ZeroPointer, vectorAddr, bufLen) - vectorAddr + chunkPtrs += vectorAddr } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index a2cfdc5c03..0a449da1a6 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -122,7 +122,7 @@ object TimeSeriesShard { /** * Copies serialized ChunkSetInfo bytes from persistent storage / on-demand paging. */ - def writeMeta(addr: Long, partitionID: Int, bytes: Array[Byte], vectors: Array[BinaryVectorPtr]): Unit = { + def writeMeta(addr: Long, partitionID: Int, bytes: Array[Byte], vectors: ArrayBuffer[BinaryVectorPtr]): Unit = { UnsafeUtils.setInt(UnsafeUtils.ZeroPointer, addr, partitionID) ChunkSetInfo.copy(bytes, addr + 4) cforRange { 0 until vectors.size } { i => From 5ed6c8ca5b7e293621b50dabd7a1c537d9666489 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Thu, 1 Oct 2020 12:16:52 -0700 Subject: [PATCH 31/53] fix(core): Shutdown process if headroom task is failing. (#912) --- .../filodb.core/memstore/TimeSeriesShard.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 0a449da1a6..eec1fd3e3b 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -30,6 +30,7 @@ import filodb.core.metadata.{Schema, Schemas} import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory._ +import filodb.memory.data.ChunkMap import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} import filodb.memory.format.BinaryVector.BinaryVectorPtr import filodb.memory.format.ZeroCopyUTF8String._ @@ -1500,7 +1501,21 @@ class TimeSeriesShard(val ref: DatasetRef, private def startHeadroomTask(sched: Scheduler): Unit = { sched.scheduleWithFixedDelay(1, 1, TimeUnit.MINUTES, new Runnable { - def run() = blockStore.ensureHeadroom(storeConfig.ensureHeadroomPercent) + var numFailures = 0 + + def run() = { + val numFree = blockStore.ensureHeadroom(storeConfig.ensureHeadroomPercent) + if (numFree > 0) { + numFailures = 0 + } else { + numFailures += 1 + if (numFailures >= 5) { + logger.error(s"Headroom task was unable to free memory for $numFailures consecutive attempts. " + + s"Shutting down process. shard=$shardNum") + ChunkMap.haltAndCatchFire() + } + } + } }) } From f57823ca9e57cf3ae44f5039df3185d1a3c059a6 Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Thu, 1 Oct 2020 15:38:48 -0700 Subject: [PATCH 32/53] misc(query): ignore fixed vector length during reduce phase in SplitDistConcatExec plan (#892) --- .../filodb/query/exec/DistConcatExec.scala | 5 + ...plitLocalPartitionDistConcatExecSpec.scala | 127 ++++++++++++++++++ 2 files changed, 132 insertions(+) create mode 100644 query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index f5a98e2906..30cd315db6 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -39,7 +39,12 @@ final case class SplitLocalPartitionDistConcatExec(queryContext: QueryContext, dispatcher: PlanDispatcher, children: Seq[ExecPlan], override val parallelChildTasks: Boolean = false) extends DistConcatExec { + addRangeVectorTransformer(StitchRvsMapper()) + + // overriden since it can reduce schemas with different vector lengths as long as the columns are same + override def reduceSchemas(rs: ResultSchema, resp: QueryResult): ResultSchema = + IgnoreFixedVectorLenAndColumnNamesSchemaReducer.reduceSchema(rs, resp) } /** diff --git a/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala b/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala new file mode 100644 index 0000000000..75b5c532f1 --- /dev/null +++ b/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala @@ -0,0 +1,127 @@ +package filodb.query.exec + +import scala.concurrent.duration._ + +import com.typesafe.config.ConfigFactory +import monix.eval.Task +import monix.execution.Scheduler +import monix.execution.Scheduler.Implicits.global +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers +import org.scalatest.BeforeAndAfterAll +import org.scalatest.time.{Millis, Seconds, Span} + +import filodb.core.binaryrecord2.RecordBuilder +import filodb.core.metadata.Schemas +import filodb.core.query._ +import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore} +import filodb.core.{DatasetRef, TestData} +import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} +import filodb.core.metadata.Column.ColumnType.{DoubleColumn, TimestampColumn} +import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} +import filodb.memory.MemFactory +import filodb.query.{QueryResponse, QueryResult} + +object SplitLocalPartitionDistConcatExecSpec { + val dummyDispatcher = new PlanDispatcher { + override def dispatch(plan: ExecPlan) + (implicit sched: Scheduler): Task[QueryResponse] = ??? + } + + val dsRef = DatasetRef("raw-metrics") + val dummyPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, Nil, AllChunkScan) + + val builder = new RecordBuilder(MemFactory.onHeapFactory) +} + +class SplitLocalPartitionDistConcatExecSpec extends AnyFunSpec with Matchers with ScalaFutures with BeforeAndAfterAll { + import ZeroCopyUTF8String._ + import filodb.core.{MachineMetricsData => MMD} + import Schemas.promCounter + import SplitLocalPartitionDistConcatExecSpec._ + + implicit val defaultPatience = PatienceConfig(timeout = Span(30, Seconds), interval = Span(250, Millis)) + + val config = ConfigFactory.load("application_test.conf").getConfig("filodb") + val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val policy = new FixedMaxPartitionsEvictionPolicy(20) + val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) + + val metric = "http_req_total" + val partKeyLabelValues = Map("job" -> "myCoolService", "instance" -> "someHost:8787") + val partKeyKVWithMetric = partKeyLabelValues ++ Map("_metric_" -> metric) + val partTagsUTF8 = partKeyLabelValues.map { case (k, v) => (k.utf8, v.utf8) } + val now = System.currentTimeMillis() + val numRawSamples = 1000 + val reportingInterval = 10000 + val tuples = (numRawSamples until 0).by(-1).map { n => + (now - n * reportingInterval, n.toDouble) + } + val schemas = Schemas(promCounter.partition, + Map(promCounter.name -> promCounter, + "histogram" -> MMD.histDataset.schema, + Schemas.dsGauge.name -> Schemas.dsGauge)) + + // NOTE: due to max-chunk-size in storeConf = 100, this will make (numRawSamples / 100) chunks + // Be sure to reset the builder; it is in an Object so static and shared amongst tests + builder.reset() + tuples.map { t => SeqRowReader(Seq(t._1, t._2, metric.utf8, partTagsUTF8)) } + .foreach(builder.addFromReader(_, promCounter)) + val container = builder.allContainers.head + + val mmdBuilder = new RecordBuilder(MemFactory.onHeapFactory) + val mmdTuples = MMD.linearMultiSeries().take(100) + val mmdSomeData = MMD.records(MMD.dataset1, mmdTuples) + + implicit val execTimeout = 5.seconds + + override def beforeAll(): Unit = { + memStore.setup(dsRef, schemas, 0, TestData.storeConf) + memStore.ingest(dsRef, 0, SomeData(container, 0)) + + memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf) + memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) + + memStore.refreshIndexForTesting(dsRef) + memStore.refreshIndexForTesting(MMD.dataset1.ref) + } + + override def afterAll(): Unit = { + memStore.shutdown() + } + + it ("should stitch the child exec results with different splits, result schema should have fixedVectorLen of both results") { + import ZeroCopyUTF8String._ + + val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), + ColumnFilter("job", Filter.Equals("myCoolService".utf8))) + val step = reportingInterval*100 + val start1 = now - numRawSamples * reportingInterval + step + val end1 = start1 + step * 3 // 4 steps + val start2 = end1 + step + val end2 = now // 6 steps + val dispacher = DummyDispatcher(memStore, querySession) + + val execPlan1 = MultiSchemaPartitionsExec(QueryContext(), dispacher, + dsRef, 0, filters, AllChunkScan) + execPlan1.addRangeVectorTransformer(new PeriodicSamplesMapper(start1, step, end1, Some(reportingInterval * 100), + Some(InternalRangeFunction.SumOverTime), QueryContext())) + + val execPlan2 = MultiSchemaPartitionsExec(QueryContext(), dispacher, + dsRef, 0, filters, AllChunkScan) + execPlan2.addRangeVectorTransformer(new PeriodicSamplesMapper(start2, step, end2, Some(reportingInterval * 100), + Some(InternalRangeFunction.SumOverTime), QueryContext())) + + val distConcatExec = SplitLocalPartitionDistConcatExec(QueryContext(), dispacher, Seq(execPlan1, execPlan2)) + + val resp = distConcatExec.execute(memStore, querySession).runAsync.futureValue + val result = resp.asInstanceOf[QueryResult] + result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) + result.result.size shouldEqual 1 + val partKeyRead = result.result(0).key.labelValues.map(lv => (lv._1.asNewString, lv._2.asNewString)) + partKeyRead shouldEqual partKeyKVWithMetric + result.resultSchema.fixedVectorLen shouldEqual Some(4) // first schema vectorLength + } +} From 7e7b75ece732b88f097846976068b470ae7e865b Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 6 Oct 2020 14:58:58 -0700 Subject: [PATCH 33/53] fix(memory): Forcibly reclaim time-ordered-blocks when running out of memory. (#914) --- .../scala/filodb.memory/BlockManager.scala | 59 ++++++++++++------- 1 file changed, 38 insertions(+), 21 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 2f8edbb5aa..de2ef21c63 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -397,40 +397,56 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, reclaimLog += event } + //scalastyle:off protected def tryReclaim(num: Int): Unit = { var reclaimed = 0 - var currList = 0 - val timeOrderedListIt = usedBlocksTimeOrdered.entrySet.iterator - while ( reclaimed < num && - timeOrderedListIt.hasNext ) { - val entry = timeOrderedListIt.next - val prevReclaimed = reclaimed - val removed = reclaimFrom(entry.getValue, stats.timeOrderedBlocksReclaimedMetric) - if (removed.nonEmpty) { - logger.info(s"timeBlockReclaim: Reclaimed ${removed.length} time ordered blocks " + - s"from list at t=${entry.getKey} (${(System.currentTimeMillis - entry.getKey)/3600000} hrs ago) " + - s"\nReclaimed blocks: ${removed.map(b => jLong.toHexString(b.address)).mkString(" ")}") + + // First reclaim time-ordered blocks which are marked as reclaimable. + reclaimTimeOrdered(false); + + if (reclaimed < num) { + // Not enough reclaimed, so try reclaiming non-time-ordered blocks which are marked as reclaimable. + reclaimFrom(usedBlocks, stats.blocksReclaimedMetric, false) + + if (reclaimed < num) { + // Still not enough? Forcibly reclaim time-ordered blocks. + reclaimTimeOrdered(true); + + if (reclaimed < num) { + // Still not enough, but forcibly reclaiming non-time-ordered blocks is dangerous. + logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. usedblocks=${usedBlocks.size} " + + s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}") + } } - // If the block list is now empty, remove it from tree map - if (entry.getValue.isEmpty) timeOrderedListIt.remove() } - if (reclaimed < num) reclaimFrom(usedBlocks, stats.blocksReclaimedMetric) - // if we do not get required blocks even after reclaim call - if (reclaimed < num) { - logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. usedblocks=${usedBlocks.size} " + - s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}") + + def reclaimTimeOrdered(forced: Boolean): Unit = { + val timeOrderedListIt = usedBlocksTimeOrdered.entrySet.iterator + while ( reclaimed < num && + timeOrderedListIt.hasNext ) { + val entry = timeOrderedListIt.next + val prevReclaimed = reclaimed + val removed = reclaimFrom(entry.getValue, stats.timeOrderedBlocksReclaimedMetric, forced) + if (removed.nonEmpty) { + logger.info(s"timeBlockReclaim: Reclaimed ${removed.length} time ordered blocks " + + s"from list at t=${entry.getKey} (${(System.currentTimeMillis - entry.getKey)/3600000} hrs ago) " + + s"\nReclaimed blocks: ${removed.map(b => jLong.toHexString(b.address)).mkString(" ")}") + } + // If the block list is now empty, remove it from tree map + if (entry.getValue.isEmpty) timeOrderedListIt.remove() + } } - def reclaimFrom(list: util.ArrayDeque[Block], reclaimedCounter: Counter): Seq[Block] = { + def reclaimFrom(list: util.ArrayDeque[Block], reclaimedCounter: Counter, forced: Boolean): Seq[Block] = { val entries = list.iterator val removed = new collection.mutable.ArrayBuffer[Block] while (entries.hasNext && reclaimed < num) { val block = entries.next - if (block.canReclaim) { + if (forced || block.canReclaim) { entries.remove() removed += block addToReclaimLog(block) - block.reclaim() + block.reclaim(forced) block.clearOwner() freeBlocks.add(block) stats.freeBlocksMetric.update(freeBlocks.size()) @@ -441,6 +457,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, removed } } + //scalastyle:on def numTimeOrderedBlocks: Int = usedBlocksTimeOrdered.values.asScala.map(_.size).sum From 11d3ecf1034b5b56bab6078b127573fb691e86d6 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 6 Oct 2020 15:16:20 -0700 Subject: [PATCH 34/53] bug(core): Shutdown when out of ingestion blocks; Mark ingestion blocks reclaimable on cassandra failure (#915) --- .../memstore/TimeSeriesPartition.scala | 33 +++++++++++-------- .../memstore/TimeSeriesShard.scala | 22 ++++++++----- .../scala/filodb.memory/data/ChunkMap.scala | 17 +++------- .../scala/filodb.memory/data/Shutdown.scala | 16 +++++++++ 4 files changed, 54 insertions(+), 34 deletions(-) create mode 100644 memory/src/main/scala/filodb.memory/data/Shutdown.scala diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 07a1d42413..9e325cbf84 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -8,7 +8,7 @@ import filodb.core.Types._ import filodb.core.metadata.{Column, PartitionSchema, Schema} import filodb.core.store._ import filodb.memory.{BinaryRegion, BinaryRegionLarge, BlockMemFactory, MemFactory} -import filodb.memory.data.ChunkMap +import filodb.memory.data.{ChunkMap, Shutdown} import filodb.memory.format._ import filodb.memory.format.MemoryReader._ @@ -221,20 +221,27 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { */ private def encodeOneChunkset(info: ChunkSetInfo, appenders: AppenderArray, blockHolder: BlockMemFactory) = { blockHolder.startMetaSpan() - // optimize and compact chunks - val frozenVectors = appenders.zipWithIndex.map { case (appender, i) => - // This assumption cannot break. We should ensure one vector can be written - // to one block always atleast as per the current design. - // If this gets triggered, decrease the max writebuffer size so smaller chunks are encoded - require(blockHolder.blockAllocationSize() > appender.frozenSize) - val optimized = appender.optimize(blockHolder) - shardStats.encodedBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) - if (schema.data.columns(i).columnType == Column.ColumnType.HistogramColumn) - shardStats.encodedHistBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) - optimized + val frozenVectors = try { + // optimize and compact chunks + appenders.zipWithIndex.map { case (appender, i) => + // This assumption cannot break. We should ensure one vector can be written + // to one block always atleast as per the current design. + // If this gets triggered, decrease the max writebuffer size so smaller chunks are encoded + require(blockHolder.blockAllocationSize() > appender.frozenSize) + val optimized = appender.optimize(blockHolder) + shardStats.encodedBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) + if (schema.data.columns(i).columnType == Column.ColumnType.HistogramColumn) + shardStats.encodedHistBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) + optimized + } + } catch { case e: Exception => + // Shutdown process right away! Reaching this state means that we could not reclaim + // a whole bunch of blocks possibly because they were not marked as reclaimable, + // because of some bug. Cleanup or rollback at this point is not viable. + Shutdown.haltAndCatchFire(new RuntimeException("Error occurred when encoding vectors", e)) + throw e } shardStats.numSamplesEncoded.increment(info.numRows) - // Now, write metadata into offheap block metadata space and update infosChunks val metaAddr = blockHolder.endMetaSpan(TimeSeriesShard.writeMeta(_, partID, info, frozenVectors), schema.data.blockMetaSize.toShort) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index eec1fd3e3b..3f8724d314 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -30,7 +30,7 @@ import filodb.core.metadata.{Schema, Schemas} import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory._ -import filodb.memory.data.ChunkMap +import filodb.memory.data.Shutdown import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} import filodb.memory.format.BinaryVector.BinaryVectorPtr import filodb.memory.format.ZeroCopyUTF8String._ @@ -332,7 +332,7 @@ class TimeSeriesShard(val ref: DatasetRef, private[memstore] final val reclaimLock = blockStore.reclaimLock // Requires blockStore. - startHeadroomTask(ingestSched) + private val headroomTask = startHeadroomTask(ingestSched) // Each shard has a single ingestion stream at a time. This BlockMemFactory is used for buffer overflow encoding // strictly during ingest() and switchBuffers(). @@ -951,8 +951,7 @@ class TimeSeriesShard(val ref: DatasetRef, val result = Future.sequence(Seq(writeChunksFuture, writeDirtyPartKeysFuture, pubDownsampleFuture)).map { _.find(_.isInstanceOf[ErrorResponse]).getOrElse(Success) }.flatMap { - case Success => blockHolder.markUsedBlocksReclaimable() - commitCheckpoint(ref, shardNum, flushGroup) + case Success => commitCheckpoint(ref, shardNum, flushGroup) case er: ErrorResponse => Future.successful(er) }.recover { case e => logger.error(s"Internal Error when persisting chunks in dataset=$ref shard=$shardNum - should " + @@ -962,6 +961,13 @@ class TimeSeriesShard(val ref: DatasetRef, result.onComplete { resp => assertThreadName(IngestSchedName) try { + // COMMENTARY ON BUG FIX DONE: Mark used blocks as reclaimable even on failure. Even if cassandra write fails + // or other errors occur, we cannot leave blocks as not reclaimable and also release the factory back into pool. + // Earlier, we were not calling this with the hope that next use of the blockMemFactory will mark them + // as reclaimable. But the factory could be used for a different flush group. Not the same one. It can + // succeed, and the wrong blocks can be marked as reclaimable. + // Can try out tracking unreclaimed blockMemFactories without releasing, but it needs to be separate PR. + blockHolder.markUsedBlocksReclaimable() blockFactoryPool.release(blockHolder) flushDoneTasks(flushGroup, resp) tracer.finish() @@ -1499,7 +1505,7 @@ class TimeSeriesShard(val ref: DatasetRef, }) } - private def startHeadroomTask(sched: Scheduler): Unit = { + private def startHeadroomTask(sched: Scheduler) = { sched.scheduleWithFixedDelay(1, 1, TimeUnit.MINUTES, new Runnable { var numFailures = 0 @@ -1510,9 +1516,8 @@ class TimeSeriesShard(val ref: DatasetRef, } else { numFailures += 1 if (numFailures >= 5) { - logger.error(s"Headroom task was unable to free memory for $numFailures consecutive attempts. " + - s"Shutting down process. shard=$shardNum") - ChunkMap.haltAndCatchFire() + Shutdown.haltAndCatchFire(new RuntimeException(s"Headroom task was unable to free memory " + + s"for $numFailures consecutive attempts. Shutting down process. shard=$shardNum")) } } } @@ -1556,6 +1561,7 @@ class TimeSeriesShard(val ref: DatasetRef, method to ensure that no threads are accessing the memory before it's freed. blockStore.releaseBlocks() */ + headroomTask.cancel() ingestSched.shutdown() } } diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 86d9a5f1f6..1b036b2e27 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -113,20 +113,12 @@ object ChunkMap extends StrictLogging { def validateNoSharedLocks(unitTest: Boolean = false): Unit = { val numLocksReleased = ChunkMap.releaseAllSharedLocks() if (numLocksReleased > 0) { - val msg = s"Number of locks was non-zero: $numLocksReleased. " + - s"This is indicative of a possible lock acquisition/release bug." - if (unitTest) { - throw new Error(msg) - } - logger.error(msg) - haltAndCatchFire() + val ex = new RuntimeException(s"Number of locks was non-zero: $numLocksReleased. " + + s"This is indicative of a possible lock acquisition/release bug.") + Shutdown.haltAndCatchFire(ex) } } - def haltAndCatchFire(): Unit = { - logger.error(s"Shutting down process since it may be in an unstable/corrupt state.") - Runtime.getRuntime.halt(1) - } } /** @@ -273,8 +265,7 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) { warned = true } else if (warned && timeoutNanos >= MaxExclusiveRetryTimeoutNanos) { val lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) - _logger.error(s"Unable to acquire exclusive lock: $lockState") - haltAndCatchFire() + Shutdown.haltAndCatchFire(new RuntimeException(s"Unable to acquire exclusive lock: $lockState")) } } } diff --git a/memory/src/main/scala/filodb.memory/data/Shutdown.scala b/memory/src/main/scala/filodb.memory/data/Shutdown.scala new file mode 100644 index 0000000000..3fdf2d6aff --- /dev/null +++ b/memory/src/main/scala/filodb.memory/data/Shutdown.scala @@ -0,0 +1,16 @@ +package filodb.memory.data + +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon + +object Shutdown extends StrictLogging { + + val forcedShutdowns = Kamon.counter("forced-shutdowns").withoutTags() + def haltAndCatchFire(e: Exception, unitTest: Boolean = false): Unit = { + forcedShutdowns.increment() + if (unitTest) throw e + logger.error(s"Shutting down process since it may be in an unstable/corrupt state", e) + Runtime.getRuntime.halt(189) + } + +} From 79f6769fe1048a2ada784828b02c751f364a6bc1 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 8 Oct 2020 10:06:16 -0700 Subject: [PATCH 35/53] maint(core): Remove and deprecate downsampling v1 (#919) Remove code for old downsampling mechanism. Replace docs with new approach. --- conf/timeseries-dev-source.conf | 21 ------ conf/timeseries-ds-1m-dev-source.conf | 66 ----------------- .../downsample/DownsampleConfig.scala | 12 ---- .../downsample/DownsamplePublisher.scala | 53 -------------- .../downsample/ShardDownsampler.scala | 17 ----- .../scala/filodb.core/memstore/MemStore.scala | 1 - .../memstore/OnDemandPagingShard.scala | 8 +-- .../memstore/TimeSeriesMemStore.scala | 18 +---- .../memstore/TimeSeriesShard.scala | 42 +---------- doc/downsampling.md | 72 ++++--------------- .../timeseries/TestTimeseriesProducer.scala | 6 -- .../downsample/GaugeDownsampleValidator.scala | 2 + .../kafka/KafkaDownsamplePublisher.scala | 61 ---------------- 13 files changed, 21 insertions(+), 358 deletions(-) delete mode 100644 conf/timeseries-ds-1m-dev-source.conf delete mode 100644 core/src/main/scala/filodb.core/downsample/DownsamplePublisher.scala delete mode 100644 kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index 17cfd9ea8a..246e2b9245 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -99,32 +99,11 @@ max-data-per-shard-query = 50 MB } downsample { - # can be disabled by setting this flag to false - enabled = false # Resolutions for downsampled data ** in ascending order ** resolutions = [ 1 minute, 5 minutes ] # Retention of downsampled data for the corresponding resolution ttls = [ 30 days, 183 days ] # Raw schemas from which to downsample raw-schema-names = [ "gauge", "untyped", "prom-counter", "prom-histogram"] - # class implementing the dispatch of downsample metrics to another dataset - publisher-class = "filodb.kafka.KafkaDownsamplePublisher" - publisher-config { - # kafka properties that will be used for the producer - kafka { - bootstrap.servers = "localhost:9092" - group.id = "filo-db-timeseries-downsample" - linger.ms=1000 - batch.size=786432 // three times record container size - acks = "-1" - retries = 3 - max.in.flight.requests.per.connection=1 - } - # map of millisecond resolution to the kafka topic for publishing downsample data - # should have one topic per defined resolution above - topics { - 60000 = "timeseries-dev-ds-1m" - } - } } } \ No newline at end of file diff --git a/conf/timeseries-ds-1m-dev-source.conf b/conf/timeseries-ds-1m-dev-source.conf deleted file mode 100644 index 7f3f397dd6..0000000000 --- a/conf/timeseries-ds-1m-dev-source.conf +++ /dev/null @@ -1,66 +0,0 @@ - dataset = "prometheus_ds_1m" - schema = "ds-gauge" - - num-shards = 4 - min-num-nodes = 2 - # Length of chunks to be written, roughly - sourcefactory = "filodb.kafka.KafkaIngestionStreamFactory" - - sourceconfig { - # Required FiloDB configurations - filo-topic-name = "timeseries-dev-ds-1m" - - # Standard kafka configurations, e.g. - # This accepts both the standard kafka value of a comma-separated - # string and a Typesafe list of String values - # EXCEPT: do not populate value.deserializer, as the Kafka format is fixed in FiloDB to be messages of RecordContainer's - bootstrap.servers = "localhost:9092" - group.id = "filo-db-timeseries-ds-ingestion" - - # Values controlling in-memory store chunking, flushing, etc. - store { - # Interval it takes to flush ALL time series in a shard. This time is further divided by groups-per-shard - flush-interval = 1h - - # TTL for on-disk / C* data. Data older than this may be purged. - disk-time-to-live = 24 hours - - # amount of time paged chunks should be retained in memory. - # We need to have a minimum of x hours free blocks or else init won't work. - demand-paged-chunk-retention-period = 12 hours - - max-chunks-size = 400 - - # Write buffer size, in bytes, for blob columns (histograms, UTF8Strings). Since these are variable data types, - # we need a maximum size, not a maximum number of items. - max-blob-buffer-size = 15000 - - # Number of bytes of offheap mem to allocate to chunk storage in each shard. Ex. 1000MB, 1G, 2GB - # Assume 5 bytes per sample, should be roughly equal to (# samples per time series) * (# time series) - shard-mem-size = 512MB - - # Number of bytes of offheap mem to allocate to write buffers for all shards. Ex. 1000MB, 1G, 2GB - ingestion-buffer-mem-size = 200MB - - # Number of time series to evict at a time. - # num-partitions-to-evict = 1000 - - # Number of subgroups within each shard. Persistence to a ChunkSink occurs one subgroup at a time, as does - # recovery from failure. This many batches of flushes must occur to cover persistence of every partition - groups-per-shard = 20 - - # Use a "MultiPartitionScan" or Cassandra MULTIGET for on-demand paging. Might improve performance. - multi-partition-odp = false - - # Amount of parallelism during on-demand paging - # demand-paging-parallelism = 4 - - # Number of retries for IngestionSource/Kafka initialization - # failure-retries = 3 - - # Amount of time to delay before retrying - # retry-delay = 15s - - evicted-pk-bloom-filter-capacity = 50000 - } - } \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/downsample/DownsampleConfig.scala b/core/src/main/scala/filodb.core/downsample/DownsampleConfig.scala index 4fe55fc026..3af8ec452e 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampleConfig.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampleConfig.scala @@ -8,8 +8,6 @@ import net.ceedubs.ficus.Ficus._ import filodb.core.DatasetRef final case class DownsampleConfig(config: Config) { - val enabled = config.hasPath("enabled") && config.getBoolean("enabled") - /** * Resolutions to downsample at */ @@ -37,16 +35,6 @@ final case class DownsampleConfig(config: Config) { } } - def makePublisher(): DownsamplePublisher = { - if (!enabled) { - NoOpDownsamplePublisher - } else { - val publisherClass = config.getString("publisher-class") - val pub = Class.forName(publisherClass).getDeclaredConstructor(classOf[Config]) - .newInstance(config).asInstanceOf[DownsamplePublisher] - pub - } - } } object DownsampleConfig { diff --git a/core/src/main/scala/filodb.core/downsample/DownsamplePublisher.scala b/core/src/main/scala/filodb.core/downsample/DownsamplePublisher.scala deleted file mode 100644 index 378ba4ee2d..0000000000 --- a/core/src/main/scala/filodb.core/downsample/DownsamplePublisher.scala +++ /dev/null @@ -1,53 +0,0 @@ -package filodb.core.downsample - -import scala.concurrent.Future - -import com.typesafe.scalalogging.StrictLogging - -import filodb.core.{Response, Success} - -/** - * Dispatches downsample data to the FiloDB datasets holding downsampled data. - */ -trait DownsamplePublisher { - /** - * Start the downsample publish pipeline/thread - */ - def start(): Unit - - /** - * Cleanly stop the downsample publish task. Typically called on shutdown. - */ - def stop(): Unit - - /** - * Dispatch samples to the downsampling dataset. - * The publisher needs to take care of retry logic, and acks if any. - * Flush pipeline in TimeSeriesShard is expected to move on and not hold off the ingestion pipeline - * if there is any slowdown here. - * - * For now, there is a possibility of loss of samples in memory that have not been dispatched. - * - * @param shardNum - * @param resolution - * @param records each Array of Byte is a bunch of Binary Records that can be deserialized by the ingestion pipeline. - * @return Future of Success if all is good. - */ - def publish(shardNum: Int, resolution: Int, records: Seq[Array[Byte]]): Future[Response] -} - -/** - * Typically used when downsampling is disabled. - */ -object NoOpDownsamplePublisher extends DownsamplePublisher with StrictLogging { - override def publish(shardNum: Int, resolution: Int, - records: Seq[Array[Byte]]): Future[Response] = Future.successful(Success) - - override def start(): Unit = { - logger.info("Starting NoOpDownsamplePublisher since downsampling is disabled") - } - - override def stop(): Unit = { - logger.info("Stopping NoOpDownsamplePublisher since downsampling is disabled") - } -} diff --git a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala index efe9e0138d..1277f5bb35 100644 --- a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala +++ b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala @@ -1,11 +1,8 @@ package filodb.core.downsample -import scala.concurrent.{ExecutionContext, Future} - import com.typesafe.scalalogging.StrictLogging import kamon.Kamon -import filodb.core.{DatasetRef, ErrorResponse, Response, Success} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesShardStats} import filodb.core.metadata.Schema @@ -34,20 +31,6 @@ object ShardDownsampler extends StrictLogging { } } - /** - * Publishes the current data in downsample builders, typically to Kafka - */ - def publishToDownsampleDataset(dsRecords: Seq[DownsampleRecords], - publisher: DownsamplePublisher, ref: DatasetRef, shard: Int) - (implicit sched: ExecutionContext): Future[Response] = { - val responses = dsRecords.map { rec => - val containers = rec.builder.optimalContainerBytes(true) - logger.debug(s"Publishing ${containers.size} downsample record containers " + - s"of dataset=$ref shard=$shard for resolution ${rec.resolution}") - publisher.publish(shard, rec.resolution, containers) - } - Future.sequence(responses).map(_.find(_.isInstanceOf[ErrorResponse]).getOrElse(Success)) - } } /** diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala index 5336a562f5..ff84e13793 100644 --- a/core/src/main/scala/filodb.core/memstore/MemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala @@ -57,7 +57,6 @@ trait MemStore extends ChunkSource { * @param storeConf the store configuration for that dataset. Each dataset may have a different mem config. * See sourceconfig.store section in conf/timeseries-dev-source.conf * @param downsampleConfig configuration for downsampling operation. By default it is disabled. - * @throws filodb.core.memstore.ShardAlreadySetup */ def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 5c6d63f702..7ce4d506d6 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -14,7 +14,6 @@ import monix.reactive.{Observable, OverflowStrategy} import filodb.core.{DatasetRef, Types} import filodb.core.binaryrecord2.RecordSchema -import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas import filodb.core.query.QuerySession import filodb.core.store._ @@ -31,12 +30,9 @@ class OnDemandPagingShard(ref: DatasetRef, bufferMemoryManager: MemFactory, rawStore: ColumnStore, metastore: MetaStore, - evictionPolicy: PartitionEvictionPolicy, - downsampleConfig: DownsampleConfig, - downsamplePublisher: DownsamplePublisher) + evictionPolicy: PartitionEvictionPolicy) (implicit ec: ExecutionContext) extends -TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawStore, metastore, evictionPolicy, - downsampleConfig, downsamplePublisher)(ec) { +TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawStore, metastore, evictionPolicy)(ec) { import TimeSeriesShard._ import FiloSchedulers._ diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 365c47d6f3..af94403ce2 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -12,7 +12,7 @@ import monix.reactive.Observable import org.jctools.maps.NonBlockingHashMapLong import filodb.core.{DatasetRef, Response, Types} -import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} +import filodb.core.downsample.DownsampleConfig import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -31,11 +31,6 @@ extends MemStore with StrictLogging { private val datasets = new HashMap[DatasetRef, Shards] private val datasetMemFactories = new HashMap[DatasetRef, MemFactory] - /** - * The Downsample Publisher is per dataset on the memstore and is shared among all shards of the dataset - */ - private val downsamplePublishers = new HashMap[DatasetRef, DownsamplePublisher] - val stats = new ChunkSourceStats private val numParallelFlushes = config.getInt("memstore.flush-task-parallelism") @@ -46,12 +41,6 @@ extends MemStore with StrictLogging { def isDownsampleStore: Boolean = false - private def makeAndStartPublisher(downsample: DownsampleConfig): DownsamplePublisher = { - val pub = downsample.makePublisher() - pub.start() - pub - } - // TODO: Change the API to return Unit Or ShardAlreadySetup, instead of throwing. Make idempotent. def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, downsample: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized { @@ -66,9 +55,8 @@ extends MemStore with StrictLogging { new NativeMemoryManager(bufferMemorySize, tags) }) - val publisher = downsamplePublishers.getOrElseUpdate(ref, makeAndStartPublisher(downsample)) val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, shard, memFactory, store, metastore, - partEvictionPolicy, downsample, publisher) + partEvictionPolicy) shards.put(shard, tsdb) } } @@ -254,8 +242,6 @@ extends MemStore with StrictLogging { def reset(): Unit = { datasets.clear() - downsamplePublishers.valuesIterator.foreach { _.stop() } - downsamplePublishers.clear() store.reset() } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 3f8724d314..d38a204883 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -25,7 +25,6 @@ import spire.syntax.cfor._ import filodb.core.{ErrorResponse, _} import filodb.core.binaryrecord2._ -import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher, ShardDownsampler} import filodb.core.metadata.{Schema, Schemas} import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -217,8 +216,6 @@ object SchemaMismatch { * @param bufferMemoryManager Unencoded/unoptimized ingested data is stored in buffers that are allocated from this * memory pool. This pool is also used to store partition keys. * @param storeConfig the store portion of the sourceconfig, not the global FiloDB application config - * @param downsampleConfig configuration for downsample operations - * @param downsamplePublisher is shared among all shards of the dataset on the node */ class TimeSeriesShard(val ref: DatasetRef, val schemas: Schemas, @@ -227,9 +224,7 @@ class TimeSeriesShard(val ref: DatasetRef, val bufferMemoryManager: MemFactory, colStore: ColumnStore, metastore: MetaStore, - evictionPolicy: PartitionEvictionPolicy, - downsampleConfig: DownsampleConfig, - downsamplePublisher: DownsamplePublisher) + evictionPolicy: PartitionEvictionPolicy) (implicit val ioPool: ExecutionContext) extends StrictLogging { import collection.JavaConverters._ @@ -405,17 +400,6 @@ class TimeSeriesShard(val ref: DatasetRef, // scheduled once a minute. private val flushOffsetMillis = flushBoundaryMillis / numGroups - /** - * Helper for downsampling ingested data for long term retention. - */ - private final val shardDownsamplers = { - val downsamplers = schemas.schemas.values.map { s => - s.schemaHash -> new ShardDownsampler(ref.dataset, shardNum, - s, s.downsample.getOrElse(s), downsampleConfig.enabled, shardStats) - } - DMap(downsamplers.toSeq: _*) - } - private[memstore] val evictedPartKeys = BloomFilter[PartKey](storeConfig.evictedPkBfCapacity, falsePositiveRate = 0.01)(new CanGenerateHashFrom[PartKey] { override def generateHash(from: PartKey): Long = { @@ -894,13 +878,6 @@ class TimeSeriesShard(val ref: DatasetRef, // Only allocate the blockHolder when we actually have chunks/partitions to flush val blockHolder = blockFactoryPool.checkout(Map("flushGroup" -> flushGroup.groupNum.toString)) - // This initializes the containers for the downsample records. Yes, we create new containers - // and not reuse them at the moment and there is allocation for every call of this method - // (once per minute). We can perhaps use a thread-local or a pool if necessary after testing. - val downsampleRecords = ShardDownsampler - .newEmptyDownsampleRecords(downsampleConfig.resolutions.map(_.toMillis.toInt), - downsampleConfig.enabled) - val chunkSetIter = partitionIt.flatMap { p => // TODO re-enable following assertion. Am noticing that monix uses TrampolineExecutionContext // causing the iterator to be consumed synchronously in some cases. It doesnt @@ -913,10 +890,6 @@ class TimeSeriesShard(val ref: DatasetRef, /* VERY IMPORTANT: This block is lazy and is executed when chunkSetIter is consumed in writeChunksFuture below */ - /* Step 3: Add downsample records for the chunks into the downsample record builders */ - val ds = shardDownsamplers(p.schema.schemaHash) - ds.populateDownsampleRecords(p, p.infosToBeFlushed, downsampleRecords) - /* Step 4: Update endTime of all partKeys that stopped ingesting in this flush period. */ updateIndexWithEndTime(p, chunks, flushGroup.dirtyPartsToFlush) chunks @@ -928,17 +901,6 @@ class TimeSeriesShard(val ref: DatasetRef, /* Step 1: Kick off partition iteration to persist chunks to column store */ val writeChunksFuture = writeChunks(flushGroup, chunkSetIter, partitionIt, blockHolder) - /* Step 5.1: Publish the downsample record data collected to the downsample dataset. - * We recover future since we want to proceed to publish downsample data even if chunk flush failed. - * This is done after writeChunksFuture because chunkSetIter is lazy. */ - val pubDownsampleFuture = writeChunksFuture.recover {case _ => Success} - .flatMap { _ => - assertThreadName(IOSchedName) - if (downsampleConfig.enabled) - ShardDownsampler.publishToDownsampleDataset(downsampleRecords, downsamplePublisher, ref, shardNum) - else Future.successful(Success) - } - /* Step 5.2: We flush dirty part keys in the one designated group for each shard. * We recover future since we want to proceed to write dirty part keys even if chunk flush failed. * This is done after writeChunksFuture because chunkSetIter is lazy. More partKeys could @@ -948,7 +910,7 @@ class TimeSeriesShard(val ref: DatasetRef, .flatMap( _=> writeDirtyPartKeys(flushGroup)) /* Step 6: Checkpoint after dirty part keys and chunks are flushed */ - val result = Future.sequence(Seq(writeChunksFuture, writeDirtyPartKeysFuture, pubDownsampleFuture)).map { + val result = Future.sequence(Seq(writeChunksFuture, writeDirtyPartKeysFuture)).map { _.find(_.isInstanceOf[ErrorResponse]).getOrElse(Success) }.flatMap { case Success => commitCheckpoint(ref, shardNum, flushGroup) diff --git a/doc/downsampling.md b/doc/downsampling.md index 222d559650..72e18c91b0 100644 --- a/doc/downsampling.md +++ b/doc/downsampling.md @@ -76,68 +76,22 @@ Additional configuration like downsample resolutions, how/where data is to be pu is supplied via the ingestion config file for the dataset. See [timeseries-dev-source.conf](../conf/timeseries-dev-source.conf) for more details. +## Downsampled Data Computation -The downsample dataset needs to be created as follows with the downsample columns in the same -order as the downsample type configuration. For the above example, we would create the downsample -dataset as: - -``` -./filo-cli -Dconfig.file=conf/timeseries-filodb-server.conf --command create --dataset prometheus_ds_1m --dataColumns timestamp:ts,min:double,max:double,sum:double,count:double,avg:double --partitionColumns tags:map --shardKeyColumns __name__,_ns_,_ws_ -``` - -Note that there is no downsampling configuration here in the above dataset. Note that partition -key for the downsample dataset is the same as the raw dataset's. - -## Downsample Data Generation -Downsampling is done at chunk flush time, when data is moved out of write buffers into block memory and cassandra. -At this time, we iterate through all of the assigned downsampler algorithms and resolution -combinations and do a query on the chunk for each period. - -For example, if the chunk spans from 10:07am to 11:08am with 10-secondly data -and we were to downsample (min, max, sum, count) at 15m interval, we'd -have the following data - -* One row each corresponding to the 10:15am, 10:30am, 10:45am, 11am, 11:15am periods. -* Timestamp on those rows would be the last timestamp found in the raw data for those -periods. -* One column for each downsampler chosen. In this case we would have 4 more data columns in -the downsample dataset besides the timestamp. -* Each row would be assigned the same partition key as the time series parition the chunk -belonged to. - -## Best Practices - -* As a result of the downsample data being generated at chunk flush time, there may be a -delay in the generation of downsample data, and it may extend up to the dataset's flush -interval. Hence if you are thinking of cascading the downsample resolutions, you need to -accept the delay in generation. -* Cascading of downsampling can be done, but the implications should be understood clearly. For example, -5m downsample is calculated from 1m data, 15m is calculated from 5m, 1hr is calculated from 15m etc. -This is possible, but remember to choose the right downsamplers. For example, `dAvg` downsampling will -not be accurate when done in a cascaded fashion since average of averages is not the correct average. -Instead you would choose to calculate average using average and count columns by using the `dAvgAc` -downsampler on the downsampled dataset. +This happens in a spark job that runs every 6 hours. There are two jobs: +1. First Spark job downsamples the data ingested in last 6 hours by reading the raw data cassandra keyspace, + compresses it into one chunkset for each time series partition and writes them to the Cassandra + tables of downsample keyspace. See [DownsamplerMain](spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala) + for more details on how it works. +2. Second Spark job copies the partition keys table updates from raw data keyspace, and copies any modified entries into the + downsample keyspace tables. ## Querying of Downsample Data - -Downsampled data for individual time series can be queried from the downsampled dataset. The downsampled dataset schema varies by schema type. For gauges, the min, max, sum, count, and avergage are computed and stored in separate columns in the `ds-gauge` schema. The FiloDB Query Engine automatically translates queries to select the right column under the hood. -For example `min_over_time(heap_usage{_ws_="demo",_ns_="myApp"})` is roughly converted to something like `heap_usage::min{_ws_="demo",_ns_="myApp"}`. -## Validation of Downsample Results - -Run main class [filodb.prom.downsample.GaugeDownsampleValidator](../http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala) with following system property arguments: - -``` --Dquery-endpoint=https://myFiloDbEndpoint.com --Draw-data-promql=jvm_threads::value{_ws_=\"demo\",_ns_=\"myApplication\",measure=\"daemon\"}[@@@@s] --Dflush-interval=12h --Dquery-range=6h -``` +A separate FiloDB cluster setup using [DonwsampledTimeSeriesStore](core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala) +configuration allows queries of downsampled data written by the downsampler jobs to the downsample cassandra keyspace. -Notes: -* `raw-data-promql` system property value should end with `[@@@@s]`. -* `raw-data-promql` needs to have `::value` column selector suffix at end of metric name as it gets replaced with diff column names -* The lookback window `@@@@` is replaced with downsample period by validation tool when running the query. +The FiloDB Chunk scans automatically translates queries to select the right column under the hood when querying downsampled +data. For example `min_over_time(heap_usage{_ws_="demo",_ns_="myApp"})` is roughly converted to something like +`heap_usage::min{_ws_="demo",_ns_="myApp"}` so the min column is chosen. -This will perform validation of min, max, sum and count downsamplers by issuing same query to both datasets -and making sure results are consistent. diff --git a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala index 10bd3f136a..67538ee8de 100644 --- a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala +++ b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala @@ -81,12 +81,6 @@ object TestTimeseriesProducer extends StrictLogging { StandardCharsets.UTF_8.toString) val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$rawSamplesQ&time=$endQuery" logger.info(s"Raw Samples query URL: \n$rawSamplesUrl") - - val downsampledQ = URLEncoder.encode("""heap_usage::sum{_ws_="demo",_ns_="App-0"}[2m]""", - StandardCharsets.UTF_8.toString) - val downsampledSamplesUrl = s"http://localhost:8080/promql/prometheus_ds_1m/api/v1/query?" + - s"query=$downsampledQ&time=$endQuery" - logger.info(s"Downsampled Samples query URL: \n$downsampledSamplesUrl") } def metricsToContainerStream(startTime: Long, diff --git a/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala b/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala index 88dc1979c5..0cc52a93a8 100644 --- a/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala +++ b/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala @@ -31,6 +31,8 @@ import filodb.query.SuccessResponse * raw-data-promql property value should end with '}[@@@@s]'. * The lookback window is replaced by validation tool when running the query. * + * TODO This validator needs to be modified to reflect the newer approach to store downsampled data + * */ object GaugeDownsampleValidator extends App with StrictLogging { diff --git a/kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala b/kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala deleted file mode 100644 index 730e026ff1..0000000000 --- a/kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala +++ /dev/null @@ -1,61 +0,0 @@ -package filodb.kafka - -import java.lang.{Long => JLong} - -import scala.collection.JavaConverters._ -import scala.concurrent.Future - -import com.typesafe.config.Config -import com.typesafe.scalalogging.StrictLogging -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} -import org.apache.kafka.clients.producer.internals.DefaultPartitioner -import org.apache.kafka.common.serialization.{ByteArraySerializer, LongSerializer} - -import filodb.core.{Response, Success} -import filodb.core.downsample.DownsamplePublisher - -class KafkaDownsamplePublisher(downsampleConfig: Config) extends DownsamplePublisher with StrictLogging { - - private val kafkaConfig = propsFromConfig(downsampleConfig.getConfig("publisher-config.kafka")) - - private val topics: Map[Int, String] = downsampleConfig.getConfig("publisher-config.topics") - .entrySet().asScala.map { e => e.getKey.toInt -> e.getValue.unwrapped().toString }.toMap - - private var producer: KafkaProducer[JLong, Array[Byte]] = _ - - override def publish(shardNum: Int, resolution: Int, records: Seq[Array[Byte]]): Future[Response] = { - topics.get(resolution) match { - case Some(topic) => - records.foreach { bytes => - val rec = new ProducerRecord[JLong, Array[Byte]](topic, shardNum, shardNum.toLong: JLong, - bytes) - producer.send(rec) - } - Future.successful(Success) - case None => - Future.failed(new IllegalArgumentException(s"Unregistered resolution $resolution")) - } - } - - def propsFromConfig(config: Config): Map[String, Object] = { - - val map = config.entrySet().asScala.map({ entry => - entry.getKey -> entry.getValue.unwrapped() - }).toMap - - // immutable properties to be overwritten - map ++ Map( "value.serializer" -> classOf[ByteArraySerializer], - "key.serializer" -> classOf[LongSerializer], - "partitioner.class" -> classOf[DefaultPartitioner]) - } - - override def start(): Unit = { - logger.info(s"Starting Kafka Downsampling Publisher. Will be publishing to $topics with config: $kafkaConfig") - producer = new KafkaProducer(kafkaConfig.asJava) - } - - override def stop(): Unit = { - logger.info("Stopping Kafka Downsampling Publisher") - producer.close() - } -} From 73a288792ef84d61a4151c4cd2ece2748e81201c Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 8 Oct 2020 10:45:22 -0700 Subject: [PATCH 36/53] bug(query): assign precedence to child Binary expression's (#910) --- .../scala/filodb/prometheus/parse/Parser.scala | 17 +++++++++++------ .../filodb/prometheus/parse/ParserSpec.scala | 4 +++- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index ba3a356ea7..1923281998 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -449,19 +449,24 @@ object Parser extends Expression { def queryRangeToLogicalPlan(query: String, timeParams: TimeRangeParams): LogicalPlan = { val expression = parseQuery(query) - val expressionWithPrecedence = expression match { - case binaryExpression: BinaryExpression => assignPrecedence(binaryExpression.lhs, binaryExpression.operator, - binaryExpression.vectorMatch, binaryExpression.rhs) - case _ => expression - } - expressionWithPrecedence match { + assignPrecedence(expression) match { case p: PeriodicSeries => p.toSeriesPlan(timeParams) case r: SimpleSeries => r.toSeriesPlan(timeParams, isRoot = true) case _ => throw new UnsupportedOperationException() } } + def assignPrecedence(expression: Expression): Expression = { + expression match { + case f: Function => f.copy(allParams = f.allParams.map(assignPrecedence(_))) + case a: AggregateExpression => a.copy(params = a.params.map(assignPrecedence(_)), altFunctionParams = a. + altFunctionParams.map(assignPrecedence(_))) + case b: BinaryExpression => assignPrecedence(b.lhs, b.operator, b.vectorMatch, b.rhs) + case _ => expression + } + } + /** * Recursively assign precedence to BinaryExpression by creating new BinaryExpression with inner expressions * rearranged based on precedence diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index e53a1833b6..4f9cd78d99 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -553,7 +553,9 @@ class ParserSpec extends AnyFunSpec with Matchers { "sum(rate(foo{job=\"SNRT-App-0\"}[0.5i]))" -> "Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(500000),None),1524855988000,1000000,1524855988000,500000,Rate,true,List(),None),List(),List(),List())", "http_requests_total - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false)", "foo - http_requests_total * 2^3" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),SUB,OneToOne,ScalarVectorBinaryOperation(MUL,ScalarBinaryOperation(POW,Left(2.0),Left(3.0),RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false),List(),List(),List())", - "sum(http_requests_total) - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false)" + "sum(http_requests_total) - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false)", + "ceil(vector(100) / 10 / 10)" -> "ApplyInstantFunction(ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),VectorPlan(ScalarFixedDoublePlan(100.0,RangeParams(1524855988,1000,1524855988))),false),false),Ceil,List())", + "ceil(sum(foo) / 10 / 10 / 10)" -> "ApplyInstantFunction(ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false),false),false),Ceil,List())" ) val qts: Long = 1524855988L From 932e18e2e9f82935874d26751348173ac45a3491 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 23 Sep 2020 10:41:53 -0700 Subject: [PATCH 37/53] bug(coord): Add extra by and on keys in QueryPlanner for specific time ranges (#901) * Extra join-on or group-by keys are added during planning time and not during runtime. * BinaryJoinExec changes for this are removed. * Keys are added during planning only if query overlaps with configured time ranges * Keeps ExecPlans clean, and improves unit testability --- .../queryplanner/LogicalPlanUtils.scala | 50 +++++++++++++++ .../queryplanner/PlannerMaterializer.scala | 22 ------- .../queryplanner/SingleClusterPlanner.scala | 11 ++-- .../queryplanner/SinglePartitionPlanner.scala | 16 ++--- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 61 +++++++++++++++++++ .../SinglePartitionPlannerSpec.scala | 2 +- core/src/main/resources/filodb-defaults.conf | 6 ++ .../scala/filodb.core/query/QueryConfig.scala | 5 ++ core/src/test/resources/application_test.conf | 1 + .../filodb/query/exec/BinaryJoinExec.scala | 9 +-- .../filodb/query/exec/SetOperatorExec.scala | 4 +- .../query/exec/BinaryJoinExecSpec.scala | 6 +- 12 files changed, 146 insertions(+), 47 deletions(-) create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 2ca0a508b3..9c5c7cab36 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -1,5 +1,6 @@ package filodb.coordinator.queryplanner +import filodb.coordinator.queryplanner.LogicalPlanUtils.{getLookBackMillis, getTimeFromLogicalPlan} import filodb.core.query.RangeParams import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants @@ -182,3 +183,52 @@ object LogicalPlanUtils { labels } } + +/** + * Temporary utility to modify plan to add extra join-on keys or group-by keys + * for specific time ranges. + */ +object ExtraOnByKeysUtil { + + def getRealOnLabels(lp: BinaryJoin, addStepKeyTimeRanges: Seq[Seq[Long]]): Seq[String] = { + if (shouldAddExtraKeys(lp.lhs, addStepKeyTimeRanges: Seq[Seq[Long]]) || + shouldAddExtraKeys(lp.rhs, addStepKeyTimeRanges: Seq[Seq[Long]])) { + // add extra keys if ignoring clause is not specified + if (lp.ignoring.isEmpty) lp.on ++ extraByOnKeys + else lp.on + } else { + lp.on + } + } + + def getRealByLabels(lp: Aggregate, addStepKeyTimeRanges: Seq[Seq[Long]]): Seq[String] = { + if (shouldAddExtraKeys(lp, addStepKeyTimeRanges)) { + // add extra keys if without clause is not specified + if (lp.without.isEmpty) lp.by ++ extraByOnKeys + else lp.by + } else { + lp.by + } + } + + private def shouldAddExtraKeys(lp: LogicalPlan, addStepKeyTimeRanges: Seq[Seq[Long]]): Boolean = { + // need to check if raw time range in query overlaps with configured addStepKeyTimeRanges + val range = getTimeFromLogicalPlan(lp) + val lookback = getLookBackMillis(lp) + queryTimeRangeRequiresExtraKeys(range.startMs - lookback, range.endMs, addStepKeyTimeRanges) + } + + val extraByOnKeys = Seq("_pi_", "_step_") + /** + * Returns true if two time ranges (x1, x2) and (y1, y2) overlap + */ + private def rangeOverlaps(x1: Long, x2: Long, y1: Long, y2: Long): Boolean = { + Math.max(x1, y1) <= Math.min(x2, y2) + } + + private def queryTimeRangeRequiresExtraKeys(rawStartMs: Long, + rawEndMs: Long, + addStepKeyTimeRanges: Seq[Seq[Long]]): Boolean = { + addStepKeyTimeRanges.exists { r => rangeOverlaps(rawStartMs, rawEndMs, r(0), r(1)) } + } +} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index 8b85d427b3..395f737396 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -1,8 +1,6 @@ package filodb.coordinator.queryplanner -import com.typesafe.scalalogging.StrictLogging import java.util.concurrent.ThreadLocalRandom -import kamon.Kamon import filodb.core.metadata.{DatasetOptions, Schemas} import filodb.core.query.{QueryContext, RangeParams} @@ -141,23 +139,3 @@ trait PlannerMaterializer { } } } - -object PlannerUtil extends StrictLogging { - - val bjBetweenAggAndNonAgg = Kamon.counter("join-between-agg-non-agg").withoutTags() - - def validateBinaryJoin(lhs: Seq[ExecPlan], rhs: Seq[ExecPlan], queryContext: QueryContext): Any = { - - if (lhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && - !rhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { - logger.info(s"Saw Binary Join between aggregate(lhs) and non-aggregate (rhs). ${queryContext.origQueryParams}") - bjBetweenAggAndNonAgg.increment() - } - - if (!lhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec]) && - rhs.exists(_.isInstanceOf[LocalPartitionReduceAggregateExec])) { - logger.info(s"Saw Binary Join between non-aggregate(lhs) and aggregate(rhs): ${queryContext.origQueryParams}") - bjBetweenAggAndNonAgg.increment() - } - } -} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 47d9e0dba9..5e49958232 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -174,8 +174,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, val stitchedRhs = if (rhs.needsStitch) Seq(StitchRvsExec(qContext, pickDispatcher(rhs.plans), rhs.plans)) else rhs.plans - - PlannerUtil.validateBinaryJoin(lhs.plans, rhs.plans, qContext) + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(lp, queryConfig.addExtraOnByKeysTimeRanges) // TODO Currently we create separate exec plan node for stitching. // Ideally, we can go one step further and add capability to NonLeafNode plans to pre-process @@ -186,11 +185,11 @@ class SingleClusterPlanner(dsRef: DatasetRef, val targetActor = pickDispatcher(stitchedLhs ++ stitchedRhs) val joined = if (lp.operator.isInstanceOf[SetOperator]) Seq(exec.SetOperatorExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, - LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(onKeysReal, dsOptions.metricColumn), LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), dsOptions.metricColumn)) else Seq(BinaryJoinExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.cardinality, - LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(onKeysReal, dsOptions.metricColumn), LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), LogicalPlanUtils.renameLabels(lp.include, dsOptions.metricColumn), dsOptions.metricColumn)) PlanResult(joined, false) @@ -199,6 +198,8 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def materializeAggregate(qContext: QueryContext, lp: Aggregate): PlanResult = { val toReduceLevel1 = walkLogicalPlanTree(lp.vectors, qContext) + val byKeysReal = ExtraOnByKeysUtil.getRealByLabels(lp, queryConfig.addExtraOnByKeysTimeRanges) + // Now we have one exec plan per shard /* * Note that in order for same overlapping RVs to not be double counted when spread is increased, @@ -214,7 +215,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, toReduceLevel1.plans.foreach { _.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, LogicalPlanUtils.renameLabels(lp.without, dsOptions.metricColumn), - LogicalPlanUtils.renameLabels(lp.by, dsOptions.metricColumn))) + LogicalPlanUtils.renameLabels(byKeysReal, dsOptions.metricColumn))) } val toReduceLevel2 = diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index ab9f9c2135..445b61fcd7 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -1,7 +1,7 @@ package filodb.coordinator.queryplanner -import filodb.core.query.QueryContext -import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters, SetOperator} +import filodb.core.query.{QueryConfig, QueryContext} +import filodb.query._ import filodb.query.exec._ /** @@ -12,8 +12,10 @@ import filodb.query.exec._ * @param plannerSelector a function that selects the planner name given the metric name * */ -class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelector: String => String, - datasetMetricColumn: String) +class SinglePartitionPlanner(planners: Map[String, QueryPlanner], + plannerSelector: String => String, + datasetMetricColumn: String, + queryConfig: QueryConfig) extends QueryPlanner { def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { @@ -53,15 +55,15 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelecto case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, qContext) } - PlannerUtil.validateBinaryJoin(Seq(lhsExec), Seq(rhsExec), qContext) + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) if (logicalPlan.operator.isInstanceOf[SetOperator]) SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) else BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - logicalPlan.cardinality, LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala new file mode 100644 index 0000000000..9dcfd2af2c --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -0,0 +1,61 @@ +package filodb.coordinator.queryplanner + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +import filodb.prometheus.ast.TimeStepParams +import filodb.prometheus.parse.Parser +import filodb.query.{Aggregate, BinaryJoin} + +class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { + + import ExtraOnByKeysUtil._ + + val extraKeysTimeRange = Seq(Seq(25000000L, 30000000L)) + + it("should add extra by keys for aggregate when no keys present") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m]))""", TimeStepParams(20000, 100, 30000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual extraByOnKeys + } + + it("should not add extra by keys for aggregate when without present") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m])) without (pod)""", + TimeStepParams(20000, 100, 30000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty + } + + it("should add extra by keys for aggregate when on already keys present") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m])) by (pod)""", + TimeStepParams(20000, 100, 30000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq("pod") ++ extraByOnKeys + } + + it("should add extra on keys for binary join when no keys present") { + val lp = Parser.queryRangeToLogicalPlan("""foo + bar """, + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual extraByOnKeys + } + + it("should add extra on keys for binary join when on already keys present") { + val lp = Parser.queryRangeToLogicalPlan("""foo + on(pod) bar """, + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq("pod") ++ extraByOnKeys + } + + it("should not add extra on keys for binary join when ignoring present") { + val lp = Parser.queryRangeToLogicalPlan("""foo + ignoring(pod) bar """, + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty + } + + it("should add extra keys even with overlap is inside of the first lookback range") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m]))""", TimeStepParams(30005L, 100, 40000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual extraByOnKeys + } + + it("should not add extra keys when no overlap with configured time ranges") { + val lp = Parser.queryRangeToLogicalPlan("""sum(rate(foo[5m]))""", TimeStepParams(40000L, 100, 50000)) + getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty + } + +} diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 5c16c94c06..e85faecd92 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -81,7 +81,7 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val plannerSelector = (metricName: String) => { if (metricName.equals("rr1")) "rules1" else if (metricName.equals("rr2")) "rules2" else "local" } - val engine = new SinglePartitionPlanner(planners, plannerSelector, "_metric_") + val engine = new SinglePartitionPlanner(planners, plannerSelector, "_metric_", queryConfig) it("should generate Exec plan for simple query") { val lp = Parser.queryToLogicalPlan("test{job = \"app\"}", 1000, 1000) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 69a64640e9..af9d6d1605 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -174,6 +174,12 @@ filodb { # Enable faster rate/increase/delta calculations. Depends on drop detection in chunks (detectDrops=true) faster-rate = true + # Time ranges for which additional join/by keys need to be added implicitly. Add as a 2D array, example: + # [ + # [1600224662, 1600229662], + # [1600204662, 1600209662] + # ] + add-extra-by-on-key-time-ranges = [] } shard-manager { diff --git a/core/src/main/scala/filodb.core/query/QueryConfig.scala b/core/src/main/scala/filodb.core/query/QueryConfig.scala index 08f14e29c4..b40d291b2d 100644 --- a/core/src/main/scala/filodb.core/query/QueryConfig.scala +++ b/core/src/main/scala/filodb.core/query/QueryConfig.scala @@ -15,6 +15,11 @@ class QueryConfig(queryConfig: Config) { lazy val minStepMs = queryConfig.getDuration("min-step").toMillis lazy val fastReduceMaxWindows = queryConfig.getInt("fastreduce-max-windows") lazy val routingConfig = queryConfig.getConfig("routing") + lazy val addExtraOnByKeysTimeRanges = { + val v = queryConfig.as[Seq[Seq[Long]]]("add-extra-by-on-key-time-ranges") + require(v.forall(r => r.size == 2 && r(0) < r(1))) + v + } /** * Feature flag test: returns true if the config has an entry with "true", "t" etc diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index 05700e113a..9243aeec1c 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -62,6 +62,7 @@ filodb { min-step = 1 ms faster-rate = true fastreduce-max-windows = 50 + add-extra-by-on-key-time-ranges = [] } spread-default = 1 diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index 9321916a8a..c97bd86c05 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -51,8 +51,6 @@ final case class BinaryJoinExec(queryContext: QueryContext, require(!on.contains(metricColumn), "On cannot contain metric name") val onLabels = on.map(Utf8Str(_)).toSet - // publishInterval and step tags always needs to be included in join key - val withExtraOnLabels = onLabels ++ Seq("_pi_".utf8, "_step_".utf8) val ignoringLabels = ignoring.map(Utf8Str(_)).toSet val ignoringLabelsForJoin = ignoringLabels + metricColumn.utf8 // if onLabels is non-empty, we are doing matching based on on-label, otherwise we are @@ -121,7 +119,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, } private def joinKeys(rvk: RangeVectorKey): Map[Utf8Str, Utf8Str] = { - if (onLabels.nonEmpty) rvk.labelValues.filter(lv => withExtraOnLabels.contains(lv._1)) + if (onLabels.nonEmpty) rvk.labelValues.filter(lv => onLabels.contains(lv._1)) else rvk.labelValues.filterNot(lv => ignoringLabelsForJoin.contains(lv._1)) } @@ -132,9 +130,8 @@ final case class BinaryJoinExec(queryContext: QueryContext, if (binaryOp.isInstanceOf[MathOperator]) result = result - Utf8Str(metricColumn) if (cardinality == Cardinality.OneToOne) { - result = - if (onLabels.nonEmpty) result.filter(lv => withExtraOnLabels.contains(lv._1)) // retain what is in onLabel list - else result.filterNot(lv => ignoringLabels.contains(lv._1)) // remove the labels in ignoring label list + result = if (onLabels.nonEmpty) result.filter(lv => onLabels.contains(lv._1)) // retain what is in onLabel list + else result.filterNot(lv => ignoringLabels.contains(lv._1)) // remove the labels in ignoring label list } else if (cardinality == Cardinality.OneToMany || cardinality == Cardinality.ManyToOne) { // For group_left/group_right add labels in include from one side. Result should have all keys from many side include.foreach { x => diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index fbac85df65..34a5a4590e 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -40,8 +40,6 @@ final case class SetOperatorExec(queryContext: QueryContext, require(!on.contains(metricColumn), "On cannot contain metric name") val onLabels = on.map(Utf8Str(_)).toSet - // TODO Add unit tests for automatic inclusion of _pi_ and _step_ in the join key - val withExtraOnLabels = onLabels ++ Seq("_pi_".utf8, "_step_".utf8) val ignoringLabels = ignoring.map(Utf8Str(_)).toSet + metricColumn.utf8 // if onLabels is non-empty, we are doing matching based on on-label, otherwise we are // doing matching based on ignoringLabels even if it is empty @@ -76,7 +74,7 @@ final case class SetOperatorExec(queryContext: QueryContext, } private def joinKeys(rvk: RangeVectorKey): Map[Utf8Str, Utf8Str] = { - if (onLabels.nonEmpty) rvk.labelValues.filter(lv => withExtraOnLabels.contains(lv._1)) + if (onLabels.nonEmpty) rvk.labelValues.filter(lv => onLabels.contains(lv._1)) else rvk.labelValues.filterNot(lv => ignoringLabels.contains(lv._1)) } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index e3f1abfd1f..82dcf6d8f3 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -143,7 +143,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { result.map(_.key).toSet.size shouldEqual 100 } - it("should implictly add step and pi tag as join key on OneToOne joins") { + it("should deal with additional step and pi tag as join key on OneToOne joins") { val lhs1: RangeVector = new RangeVector { val key: RangeVectorKey = CustomRangeVectorKey( Map("__name__".utf8 -> s"someMetricLhs".utf8, "_pi_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) @@ -177,7 +177,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { Array(dummyPlan), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, Cardinality.OneToOne, - Nil, Nil, Nil, "__name__") + Seq("_step_", "_pi_"), Nil, Nil, "__name__") // scalastyle:off val lhs = QueryResult("someId", null, Seq(lhs1, lhs2).map(rv => SerializedRangeVector(rv, schema))) @@ -193,7 +193,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } - it("should implictly add step and pi tag as join key on OneToMany joins") { + it("should deal with implictly added step and pi tag as join key on OneToMany joins") { val lhs1: RangeVector = new RangeVector { val key: RangeVectorKey = CustomRangeVectorKey( Map("__name__".utf8 -> s"someMetricLhs".utf8, "_pi_".utf8 -> "0".utf8, "tag2".utf8 -> "tag2Val".utf8)) From 2d4f5ff1afa92a069d7e72d8c73cb7f8464ecd50 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 23 Sep 2020 19:12:11 -0700 Subject: [PATCH 38/53] bug(coord): Extra on-keys should be added only if on is nonEmpty (#904) Query Planner changes to add join key were not done equivalently. Now adding extra on-keys only if on specifier is non-empty already --- .../filodb.coordinator/queryplanner/LogicalPlanUtils.scala | 4 ++-- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 9c5c7cab36..d0ac4ac68b 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -193,8 +193,8 @@ object ExtraOnByKeysUtil { def getRealOnLabels(lp: BinaryJoin, addStepKeyTimeRanges: Seq[Seq[Long]]): Seq[String] = { if (shouldAddExtraKeys(lp.lhs, addStepKeyTimeRanges: Seq[Seq[Long]]) || shouldAddExtraKeys(lp.rhs, addStepKeyTimeRanges: Seq[Seq[Long]])) { - // add extra keys if ignoring clause is not specified - if (lp.ignoring.isEmpty) lp.on ++ extraByOnKeys + // add extra keys if ignoring clause is not specified and on is specified + if (lp.on.nonEmpty) lp.on ++ extraByOnKeys else lp.on } else { lp.on diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala index 9dcfd2af2c..f3d5bd60df 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -30,10 +30,10 @@ class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq("pod") ++ extraByOnKeys } - it("should add extra on keys for binary join when no keys present") { + it("should not add extra on keys for binary join when no join keys present") { val lp = Parser.queryRangeToLogicalPlan("""foo + bar """, TimeStepParams(20000, 100, 30000)) - getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual extraByOnKeys + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty } it("should add extra on keys for binary join when on already keys present") { From 263d15328d41556db624a41771b809f193c26c1f Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 24 Sep 2020 12:02:04 -0700 Subject: [PATCH 39/53] bug(query): head call on empty list in LogicalPlanUtils (#906) --- .../queryplanner/LogicalPlanUtils.scala | 31 ++++++++++++------- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 6 ++++ 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index d0ac4ac68b..925f1c49d4 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -138,27 +138,36 @@ object LogicalPlanUtils { * NOTE: Plan should be PeriodicSeriesPlan */ def getRawSeriesStartTime(logicalPlan: LogicalPlan): Option[Long] = { - LogicalPlan.findLeafLogicalPlans(logicalPlan).head match { - case lp: RawSeries => lp.rangeSelector match { - case rs: IntervalSelector => Some(rs.from) - case _ => None + val leaf = LogicalPlan.findLeafLogicalPlans(logicalPlan) + if (leaf.isEmpty) None else { + leaf.head match { + case lp: RawSeries => lp.rangeSelector match { + case rs: IntervalSelector => Some(rs.from) + case _ => None + } + case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") } - case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") } } def getOffsetMillis(logicalPlan: LogicalPlan): Long = { - LogicalPlan.findLeafLogicalPlans(logicalPlan).head match { - case lp: RawSeries => lp.offsetMs.getOrElse(0) - case _ => 0 + val leaf = LogicalPlan.findLeafLogicalPlans(logicalPlan) + if (leaf.isEmpty) 0 else { + leaf.head match { + case lp: RawSeries => lp.offsetMs.getOrElse(0) + case _ => 0 + } } } def getLookBackMillis(logicalPlan: LogicalPlan): Long = { val staleDataLookbackMillis = WindowConstants.staleDataLookbackMillis - LogicalPlan.findLeafLogicalPlans(logicalPlan).head match { - case lp: RawSeries => lp.lookbackMs.getOrElse(staleDataLookbackMillis) - case _ => 0 + val leaf = LogicalPlan.findLeafLogicalPlans(logicalPlan) + if (leaf.isEmpty) 0 else { + leaf.head match { + case lp: RawSeries => lp.lookbackMs.getOrElse(staleDataLookbackMillis) + case _ => 0 + } } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala index f3d5bd60df..249ed56a0d 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -58,4 +58,10 @@ class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty } +// TODO remove test when unnecessary binary join bug is fixed + it("should not add extra on keys for nested scalar queries") { + val lp = Parser.queryRangeToLogicalPlan("""foo + 10/2""", + TimeStepParams(20000, 100, 30000)) + getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty + } } From 59f538d46efdbd028e9ad42c6b76a67f28cd42bb Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 24 Sep 2020 16:07:54 -0700 Subject: [PATCH 40/53] bug(query): BinaryJoinExec is created for queries having nested scalar expressions (#903) --- .../queryplanner/PlannerMaterializer.scala | 5 ++- .../queryplanner/ExtraOnByKeysUtilSpec.scala | 3 +- .../queryplanner/ScalarQueriesSpec.scala | 19 ++++++++ .../filodb/prometheus/ast/Expressions.scala | 44 ++++++++++++------- .../filodb/prometheus/parse/ParserSpec.scala | 7 ++- 5 files changed, 55 insertions(+), 23 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index 395f737396..2f3abeb918 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -48,9 +48,10 @@ trait PlannerMaterializer { param match { case num: ScalarFixedDoublePlan => StaticFuncArgs(num.scalar, num.timeStepParams) case s: ScalarVaryingDoublePlan => ExecPlanFuncArgs(materialize(s, qContext), - RangeParams(s.startMs, s.stepMs, s.endMs)) + RangeParams(s.startMs, s.stepMs, s.endMs)) case t: ScalarTimeBasedPlan => TimeFuncArgs(t.rangeParams) - case _ => throw new UnsupportedOperationException("Invalid logical plan") + case s: ScalarBinaryOperation => ExecPlanFuncArgs(materialize(s, qContext), + RangeParams(s.startMs, s.stepMs, s.endMs)) } } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala index 249ed56a0d..a676686b95 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ExtraOnByKeysUtilSpec.scala @@ -58,9 +58,8 @@ class ExtraOnByKeysUtilSpec extends AnyFunSpec with Matchers { getRealByLabels(lp.asInstanceOf[Aggregate], extraKeysTimeRange) shouldEqual Seq.empty } -// TODO remove test when unnecessary binary join bug is fixed it("should not add extra on keys for nested scalar queries") { - val lp = Parser.queryRangeToLogicalPlan("""foo + 10/2""", + val lp = Parser.queryRangeToLogicalPlan("""foo * 10/2 + bar""", TimeStepParams(20000, 100, 30000)) getRealOnLabels(lp.asInstanceOf[BinaryJoin], extraKeysTimeRange) shouldEqual Seq.empty } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala index a2d11c5252..22e7811e68 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala @@ -471,4 +471,23 @@ class ScalarQueriesSpec extends AnyFunSpec with Matchers { |on InProcessPlanDispatcher""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } + + it("should generate execPlan for binary join with nested scalar query") { + val lp = Parser.queryToLogicalPlan("""sum(http_requests_total{job = "app"}) - 10/2""", 1000, 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val expected = + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~ + |-E~ScalarBinaryOperationExec(params = RangeParams(1000,1000,1000), operator = DIV, lhs = Left(10.0), rhs = Left(2.0)) on InProcessPlanDispatcher + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List()) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-669137818]) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-669137818]) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-669137818])""".stripMargin + maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) + } } diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala index 9329f64207..f213bc6392 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala @@ -3,7 +3,6 @@ package filodb.prometheus.ast import filodb.core.query.RangeParams import filodb.query._ - trait Expressions extends Aggregates with Functions { case class UnaryExpression(operator: Operator, operand: Expression) extends Expression { @@ -38,7 +37,9 @@ trait Expressions extends Aggregates with Functions { case _ => false } } + // scalastyle:off method.length + // scalastyle:off cyclomatic.complexity override def toSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { if (hasScalarResult(lhs) && hasScalarResult(rhs)) { val rangeParams = RangeParams(timeParams.start, timeParams.step, timeParams.end) @@ -95,26 +96,35 @@ trait Expressions extends Aggregates with Functions { // node_info + http_requests case (lh: PeriodicSeries, rh: PeriodicSeries) => - val seriesPlanLhs = lh.toSeriesPlan(timeParams) - val seriesPlanRhs = rh.toSeriesPlan(timeParams) - val cardinality = if (operator.getPlanOperator.isInstanceOf[SetOperator]) - Cardinality.ManyToMany - else - vectorMatch.map(_.cardinality.cardinality).getOrElse(Cardinality.OneToOne) - - val matcher = vectorMatch.flatMap(_.matching) - val onLabels = matcher.filter(_.isInstanceOf[On]).map(_.labels) - val ignoringLabels = matcher.filter(_.isInstanceOf[Ignoring]).map(_.labels) - - BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs, - onLabels.getOrElse(Nil), ignoringLabels.getOrElse(Nil), - vectorMatch.flatMap(_.grouping).map(_.labels).getOrElse(Nil)) - + //10/2 + foo + if (hasScalarResult(lh)) { + val scalar = lh.toSeriesPlan(timeParams).asInstanceOf[ScalarPlan] + val seriesPlan = rh.toSeriesPlan(timeParams) + ScalarVectorBinaryOperation(operator.getPlanOperator, scalar, seriesPlan, scalarIsLhs = true) + } else if (hasScalarResult(rh)) { // foo + 10/2 + val scalar = rh.toSeriesPlan(timeParams).asInstanceOf[ScalarPlan] + val seriesPlan = lh.toSeriesPlan(timeParams) + ScalarVectorBinaryOperation(operator.getPlanOperator, scalar, seriesPlan, scalarIsLhs = false) + } else { + val seriesPlanLhs = lh.toSeriesPlan(timeParams) + val seriesPlanRhs = rh.toSeriesPlan(timeParams) + val cardinality = if (operator.getPlanOperator.isInstanceOf[SetOperator]) + Cardinality.ManyToMany + else + vectorMatch.map(_.cardinality.cardinality).getOrElse(Cardinality.OneToOne) + + val matcher = vectorMatch.flatMap(_.matching) + val onLabels = matcher.filter(_.isInstanceOf[On]).map(_.labels) + val ignoringLabels = matcher.filter(_.isInstanceOf[Ignoring]).map(_.labels) + BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs, + onLabels.getOrElse(Nil), ignoringLabels.getOrElse(Nil), + vectorMatch.flatMap(_.grouping).map(_.labels).getOrElse(Nil)) + } case _ => throw new UnsupportedOperationException("Invalid operands") } } } // scalastyle:on method.length + // scalastyle:on cyclomatic.complexity } - } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index af8de6a378..50066fe010 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -524,7 +524,10 @@ class ParserSpec extends AnyFunSpec with Matchers { "PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),Some(2000000)),1524855988000,1000000,1524855988000,Some(2000000))", "sum(rate(foo{job=\"SNRT-App-0\"}[5i]))" -> "Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(5000000),None),1524855988000,1000000,1524855988000,5000000,Rate,true,List(),None),List(),List(),List())", "rate(foo{job=\"SNRT-App-0\"}[5i]) + rate(bar{job=\"SNRT-App-0\"}[4i])" -> "BinaryJoin(PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(5000000),None),1524855988000,1000000,1524855988000,5000000,Rate,true,List(),None),ADD,OneToOne,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(bar))),List(),Some(4000000),None),1524855988000,1000000,1524855988000,4000000,Rate,true,List(),None),List(),List(),List())", - "sum(rate(foo{job=\"SNRT-App-0\"}[0.5i]))" -> "Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(500000),None),1524855988000,1000000,1524855988000,500000,Rate,true,List(),None),List(),List(),List())" + "sum(rate(foo{job=\"SNRT-App-0\"}[0.5i]))" -> "Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(SNRT-App-0)), ColumnFilter(__name__,Equals(foo))),List(),Some(500000),None),1524855988000,1000000,1524855988000,500000,Rate,true,List(),None),List(),List(),List())", + "http_requests_total - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false)", + "foo - http_requests_total * 2^3" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),SUB,OneToOne,ScalarVectorBinaryOperation(MUL,ScalarBinaryOperation(POW,Left(2.0),Left(3.0),RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false),List(),List(),List())", + "sum(http_requests_total) - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false)" ) val qts: Long = 1524855988L @@ -533,7 +536,7 @@ class ParserSpec extends AnyFunSpec with Matchers { info(s"Parsing $q") val lp = Parser.queryToLogicalPlan(q, qts, step) if (lp.isInstanceOf[BinaryJoin]) - printBinaryJoin(lp) + printBinaryJoin(lp) lp.toString shouldEqual (e) } } From 61b0c10afd66f3c24b6b7c6d293997a2c54b9e2b Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Sun, 11 Oct 2020 07:44:08 -0700 Subject: [PATCH 41/53] Version change 0.9.9.10.integration --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index df06e356e3..b3567421e6 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.9.integration-SNAPSHOT" +version in ThisBuild := "0.9.10.integration-SNAPSHOT" From 21e3afdba93a576017adc68f108f0707665cfdec Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 14 Oct 2020 08:56:15 -0700 Subject: [PATCH 42/53] maint(core): Strongly type NativeMemoryMgr usage (#923) Strongly type usage of NativeMemoryManager to make reclaim requirements pop out in code. Helps establish correctness in code more easily. --- .../scala/filodb.core/memstore/OnDemandPagingShard.scala | 7 +++---- .../scala/filodb.core/memstore/TimeSeriesMemStore.scala | 4 ++-- .../scala/filodb.core/memstore/TimeSeriesPartition.scala | 6 +++--- .../main/scala/filodb.core/memstore/TimeSeriesShard.scala | 2 +- .../main/scala/filodb.core/memstore/WriteBufferPool.scala | 4 ++-- memory/src/main/scala/filodb.memory/data/ChunkMap.scala | 5 ++--- 6 files changed, 13 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 7ce4d506d6..e036791fd7 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -1,11 +1,10 @@ package filodb.core.memstore -import java.util - import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext import debox.Buffer +import java.util import kamon.Kamon import kamon.trace.Span import monix.eval.Task @@ -17,7 +16,7 @@ import filodb.core.binaryrecord2.RecordSchema import filodb.core.metadata.Schemas import filodb.core.query.QuerySession import filodb.core.store._ -import filodb.memory.MemFactory +import filodb.memory.NativeMemoryManager /** * Extends TimeSeriesShard with on-demand paging functionality by populating in-memory partitions with chunks from @@ -27,7 +26,7 @@ class OnDemandPagingShard(ref: DatasetRef, schemas: Schemas, storeConfig: StoreConfig, shardNum: Int, - bufferMemoryManager: MemFactory, + bufferMemoryManager: NativeMemoryManager, rawStore: ColumnStore, metastore: MetaStore, evictionPolicy: PartitionEvictionPolicy) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index af94403ce2..4423f4ca92 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -16,7 +16,7 @@ import filodb.core.downsample.DownsampleConfig import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ -import filodb.memory.{MemFactory, NativeMemoryManager} +import filodb.memory.NativeMemoryManager import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} class TimeSeriesMemStore(config: Config, @@ -29,7 +29,7 @@ extends MemStore with StrictLogging { type Shards = NonBlockingHashMapLong[TimeSeriesShard] private val datasets = new HashMap[DatasetRef, Shards] - private val datasetMemFactories = new HashMap[DatasetRef, MemFactory] + private val datasetMemFactories = new HashMap[DatasetRef, NativeMemoryManager] val stats = new ChunkSourceStats diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 9e325cbf84..1cf9988924 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -7,7 +7,7 @@ import filodb.core.DatasetRef import filodb.core.Types._ import filodb.core.metadata.{Column, PartitionSchema, Schema} import filodb.core.store._ -import filodb.memory.{BinaryRegion, BinaryRegionLarge, BlockMemFactory, MemFactory} +import filodb.memory.{BinaryRegion, BinaryRegionLarge, BlockMemFactory, NativeMemoryManager} import filodb.memory.data.{ChunkMap, Shutdown} import filodb.memory.format._ import filodb.memory.format.MemoryReader._ @@ -67,7 +67,7 @@ class TimeSeriesPartition(val partID: Int, val shard: Int, bufferPool: WriteBufferPool, val shardStats: TimeSeriesShardStats, - memFactory: MemFactory, + memFactory: NativeMemoryManager, initMapSize: Int) extends ChunkMap(memFactory, initMapSize) with ReadablePartition { import TimeSeriesPartition._ @@ -462,7 +462,7 @@ class TracingTimeSeriesPartition(partID: Int, shard: Int, bufferPool: WriteBufferPool, shardStats: TimeSeriesShardStats, - memFactory: MemFactory, + memFactory: NativeMemoryManager, initMapSize: Int) extends TimeSeriesPartition(partID, schema, partitionKey, shard, bufferPool, shardStats, memFactory, initMapSize) { import TimeSeriesPartition._ diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index d38a204883..002da1343c 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -221,7 +221,7 @@ class TimeSeriesShard(val ref: DatasetRef, val schemas: Schemas, val storeConfig: StoreConfig, val shardNum: Int, - val bufferMemoryManager: MemFactory, + val bufferMemoryManager: NativeMemoryManager, colStore: ColumnStore, metastore: MetaStore, evictionPolicy: PartitionEvictionPolicy) diff --git a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala index 74ddcadb89..afdd92c26a 100644 --- a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala +++ b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala @@ -7,7 +7,7 @@ import spire.syntax.cfor._ import filodb.core.metadata.DataSchema import filodb.core.store.{ChunkSetInfo, StoreConfig} import filodb.memory.BinaryRegion.NativePointer -import filodb.memory.MemFactory +import filodb.memory.NativeMemoryManager object WriteBufferPool { /** @@ -31,7 +31,7 @@ object WriteBufferPool { * * @param storeConf the StoreConfig containing parameters for configuring write buffers, etc. */ -class WriteBufferPool(memFactory: MemFactory, +class WriteBufferPool(memFactory: NativeMemoryManager, val schema: DataSchema, storeConf: StoreConfig) extends StrictLogging { import TimeSeriesPartition._ diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 1b036b2e27..94c96031a8 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -6,9 +6,8 @@ import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging import kamon.Kamon +import filodb.memory.{NativeMemoryManager, OutOfOffheapMemoryException} import filodb.memory.BinaryRegion.NativePointer -import filodb.memory.MemFactory -import filodb.memory.OutOfOffheapMemoryException import filodb.memory.format.UnsafeUtils /** @@ -125,7 +124,7 @@ object ChunkMap extends StrictLogging { * @param memFactory a THREAD-SAFE factory for allocating offheap space * @param capacity initial capacity of the map; must be more than 0 */ -class ChunkMap(val memFactory: MemFactory, var capacity: Int) { +class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { require(capacity > 0) private var lockState: Int = 0 From c099498893b450dc9a6c8211492e1090d55804c3 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 20 Oct 2020 17:27:43 -0700 Subject: [PATCH 43/53] bug(memory): Blocks need to be marked reclaimable correctly (#925) Address cases where blocks were not being marked as reclaimable in ingestion pipeline causing blocks to leak. --- .../memstore/TimeSeriesShard.scala | 2 +- .../memstore/TimeSeriesPartitionSpec.scala | 2 +- .../scala/filodb.memory/BlockManager.scala | 3 +- .../filodb.memory/BlockMemFactoryPool.scala | 2 +- .../main/scala/filodb.memory/MemFactory.scala | 17 ++++--- .../filodb.memory/BlockMemFactorySpec.scala | 49 +++++++++++++++++++ 6 files changed, 63 insertions(+), 12 deletions(-) create mode 100644 memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 002da1343c..b84183dbf2 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -929,7 +929,7 @@ class TimeSeriesShard(val ref: DatasetRef, // as reclaimable. But the factory could be used for a different flush group. Not the same one. It can // succeed, and the wrong blocks can be marked as reclaimable. // Can try out tracking unreclaimed blockMemFactories without releasing, but it needs to be separate PR. - blockHolder.markUsedBlocksReclaimable() + blockHolder.markFullBlocksReclaimable() blockFactoryPool.release(blockHolder) flushDoneTasks(flushGroup, resp) tracer.finish() diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index 4d3001677b..e5f6e5e851 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -255,7 +255,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part.unflushedChunksets shouldEqual 1 val currBlock = blockHolder.currentBlock // hang on to these; we'll later test reclaiming them manually - blockHolder.markUsedBlocksReclaimable() + blockHolder.markFullBlocksReclaimable() // Now, switch buffers and flush again, ingesting 5 more rows // There should now be 3 chunks total, the current write buffers plus the two flushed ones diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index de2ef21c63..eff2d484e0 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -398,7 +398,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } //scalastyle:off - protected def tryReclaim(num: Int): Unit = { + protected[memory] def tryReclaim(num: Int): Int = { var reclaimed = 0 // First reclaim time-ordered blocks which are marked as reclaimable. @@ -456,6 +456,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } removed } + reclaimed } //scalastyle:on diff --git a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala index 308e7d7928..736744f91f 100644 --- a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala +++ b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala @@ -40,7 +40,7 @@ class BlockMemFactoryPool(blockStore: BlockManager, def blocksContainingPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = factoryPool.flatMap { bmf => - val blocks = bmf.fullBlocks ++ Option(bmf.currentBlock).toList + val blocks = bmf.fullBlocksToBeMarkedAsReclaimable ++ Option(bmf.currentBlock).toList BlockDetective.containsPtr(ptr, blocks) } } diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index 1852d5764f..a3325aef04 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -217,7 +217,7 @@ class BlockMemFactory(blockStore: BlockManager, val optionSelf = Some(this) // tracks fully populated blocks not marked reclaimable yet (typically waiting for flush) - val fullBlocks = ListBuffer[Block]() + val fullBlocksToBeMarkedAsReclaimable = ListBuffer[Block]() // tracks block currently being populated var currentBlock = requestBlock() @@ -250,7 +250,7 @@ class BlockMemFactory(blockStore: BlockManager, */ def tryMarkReclaimable(): Unit = synchronized { if (now - lastUsedNanos > BlockMemFactory.USED_THRESHOLD_NANOS) { - markUsedBlocksReclaimable() + markFullBlocksReclaimable() if (currentBlock != null) { currentBlock.markReclaimable() currentBlock = null @@ -294,9 +294,10 @@ class BlockMemFactory(blockStore: BlockManager, metadataWriter(metaAddr) if (blk != metadataSpan.last) { if (markFullBlocksAsReclaimable) { + // We know that all the blocks in the span except the last one is full, so mark them reclaimable blk.markReclaimable() } else synchronized { - fullBlocks += blk + fullBlocksToBeMarkedAsReclaimable += blk } } } @@ -311,9 +312,9 @@ class BlockMemFactory(blockStore: BlockManager, metaAddr } - def markUsedBlocksReclaimable(): Unit = synchronized { - fullBlocks.foreach(_.markReclaimable()) - fullBlocks.clear() + def markFullBlocksReclaimable(): Unit = synchronized { + fullBlocksToBeMarkedAsReclaimable.foreach(_.markReclaimable()) + fullBlocksToBeMarkedAsReclaimable.clear() } protected def ensureCapacity(forSize: Long): Block = synchronized { @@ -325,11 +326,11 @@ class BlockMemFactory(blockStore: BlockManager, } } else { val newBlock = requestBlock() - if (!metadataSpanActive) { + if (!metadataSpanActive || metadataSpan.isEmpty) { if (markFullBlocksAsReclaimable) { block.markReclaimable() } else { - fullBlocks += block + fullBlocksToBeMarkedAsReclaimable += block } } block = newBlock diff --git a/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala b/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala new file mode 100644 index 0000000000..c79f14c66e --- /dev/null +++ b/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala @@ -0,0 +1,49 @@ +package filodb.memory + +import scala.collection.JavaConverters._ + +import com.kenai.jffi.PageManager +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should.Matchers + +import filodb.memory.PageAlignedBlockManagerSpec.testReclaimer + +class BlockMemFactorySpec extends AnyFlatSpec with Matchers { + + val pageSize = PageManager.getInstance().pageSize() + + it should "Mark all blocks of BlockMemFactory as reclaimable when used as done in ingestion pipeline" in { + val stats = new MemoryStats(Map("test1" -> "test1")) + val blockManager = new PageAlignedBlockManager(2048 * 1024, stats, testReclaimer, 1) + val bmf = new BlockMemFactory(blockManager, None, 50, Map("test" -> "val"), false) + + // simulate encoding of multiple ts partitions in flush group + + for { flushGroup <- 0 to 1 } { + for {tsParts <- 0 to 5} { + bmf.startMetaSpan() + for {chunks <- 0 to 3} { + bmf.allocateOffheap(1000) + } + bmf.endMetaSpan(d => {}, 45) + } + // full blocks are tracked as they are allocated + flushGroup match { + case 0 => bmf.fullBlocksToBeMarkedAsReclaimable.size shouldEqual 5 + case 1 => bmf.fullBlocksToBeMarkedAsReclaimable.size shouldEqual 6 + } + // full blocks are marked as reclaimable + bmf.markFullBlocksReclaimable() + } + + // only the current block is not reclaimable + blockManager.usedBlocks.asScala.count(!_.canReclaim) shouldEqual 1 + + blockManager.usedBlocks.size shouldEqual 12 + blockManager.tryReclaim(3) shouldEqual 3 + blockManager.usedBlocks.size shouldEqual 9 // 3 are reclaimed + + blockManager.releaseBlocks() + } + +} From 55b95908ccb528193629c193249c432d28d04acb Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 21 Oct 2020 12:29:47 -0700 Subject: [PATCH 44/53] Add precedenceExpression to allow brackets to override operator precedence (#926) --- .../filodb/prometheus/ast/Expressions.scala | 23 +++++++++++++++---- .../filodb/prometheus/parse/Parser.scala | 19 ++++++++++----- .../filodb/prometheus/parse/ParserSpec.scala | 10 +++++++- 3 files changed, 40 insertions(+), 12 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala index f213bc6392..2a0eac15ed 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala @@ -9,6 +9,8 @@ trait Expressions extends Aggregates with Functions { //TODO Need to pass an operator to a series } + case class PrecedenceExpression(expression: Expression) extends Expression + case class BinaryExpression(lhs: Expression, operator: Operator, vectorMatch: Option[VectorMatch], @@ -41,10 +43,22 @@ trait Expressions extends Aggregates with Functions { // scalastyle:off method.length // scalastyle:off cyclomatic.complexity override def toSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { - if (hasScalarResult(lhs) && hasScalarResult(rhs)) { + val lhsWithPrecedence = lhs match { + case p: PrecedenceExpression => p.expression + case _ => lhs + + } + + val rhsWithPrecedence = rhs match { + case p: PrecedenceExpression => p.expression + case _ => rhs + + } + + if (hasScalarResult(lhsWithPrecedence) && hasScalarResult(rhsWithPrecedence)) { val rangeParams = RangeParams(timeParams.start, timeParams.step, timeParams.end) - (lhs, rhs) match { + (lhsWithPrecedence, rhsWithPrecedence) match { // 3 + 4 case (lh: ScalarExpression, rh: ScalarExpression) => ScalarBinaryOperation(operator.getPlanOperator, Left(lh.toScalar), Left(rh.toScalar), rangeParams) @@ -60,8 +74,7 @@ trait Expressions extends Aggregates with Functions { Right(rh.toSeriesPlan(timeParams).asInstanceOf[ScalarBinaryOperation]), rangeParams) } } else { - - (lhs, rhs) match { + (lhsWithPrecedence, rhsWithPrecedence) match { // scalar(http_requests) + scalar(node_info) case (lh: Function, rh: Function) if lh.isScalarFunction() && rh.isScalarFunction() => val scalar = lh.toSeriesPlan(timeParams).asInstanceOf[ScalarPlan] @@ -123,8 +136,8 @@ trait Expressions extends Aggregates with Functions { case _ => throw new UnsupportedOperationException("Invalid operands") } } + } } // scalastyle:on method.length // scalastyle:on cyclomatic.complexity - } } diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index 1923281998..d8a9adde42 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -342,6 +342,12 @@ trait Expression extends Aggregates with Selector with Numeric with Join { case lhs ~ op ~ vm ~ rhs => BinaryExpression(lhs, op, vm, rhs) } + lazy val precedenceExpression: PackratParser[PrecedenceExpression] = { + + "(" ~ expression ~ ")" ^^ { + case "(" ~ ep ~ ")" => PrecedenceExpression(ep) + } + } lazy val functionParams: PackratParser[Seq[Expression]] = "(" ~> repsep(expression, ",") <~ ")" ^^ { @@ -372,7 +378,7 @@ trait Expression extends Aggregates with Selector with Numeric with Join { lazy val expression: PackratParser[Expression] = binaryExpression | aggregateExpression2 | aggregateExpression1 | - function | unaryExpression | vector | numericalExpression | simpleSeries | "(" ~> expression <~ ")" + function | unaryExpression | vector | numericalExpression | simpleSeries | precedenceExpression } @@ -449,7 +455,6 @@ object Parser extends Expression { def queryRangeToLogicalPlan(query: String, timeParams: TimeRangeParams): LogicalPlan = { val expression = parseQuery(query) - assignPrecedence(expression) match { case p: PeriodicSeries => p.toSeriesPlan(timeParams) case r: SimpleSeries => r.toSeriesPlan(timeParams, isRoot = true) @@ -459,11 +464,13 @@ object Parser extends Expression { def assignPrecedence(expression: Expression): Expression = { expression match { - case f: Function => f.copy(allParams = f.allParams.map(assignPrecedence(_))) - case a: AggregateExpression => a.copy(params = a.params.map(assignPrecedence(_)), altFunctionParams = a. + case f: Function => f.copy(allParams = f.allParams.map(assignPrecedence(_))) + case a: AggregateExpression => a.copy(params = a.params.map(assignPrecedence(_)), altFunctionParams = a. altFunctionParams.map(assignPrecedence(_))) - case b: BinaryExpression => assignPrecedence(b.lhs, b.operator, b.vectorMatch, b.rhs) - case _ => expression + case b: BinaryExpression => assignPrecedence(b.lhs, b.operator, b.vectorMatch, b.rhs) + + case p: PrecedenceExpression => assignPrecedence(p.expression) + case _ => expression } } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 4f9cd78d99..78e741582f 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -555,7 +555,15 @@ class ParserSpec extends AnyFunSpec with Matchers { "foo - http_requests_total * 2^3" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),SUB,OneToOne,ScalarVectorBinaryOperation(MUL,ScalarBinaryOperation(POW,Left(2.0),Left(3.0),RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false),List(),List(),List())", "sum(http_requests_total) - 10/2" -> "ScalarVectorBinaryOperation(SUB,ScalarBinaryOperation(DIV,Left(10.0),Left(2.0),RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false)", "ceil(vector(100) / 10 / 10)" -> "ApplyInstantFunction(ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),VectorPlan(ScalarFixedDoublePlan(100.0,RangeParams(1524855988,1000,1524855988))),false),false),Ceil,List())", - "ceil(sum(foo) / 10 / 10 / 10)" -> "ApplyInstantFunction(ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false),false),false),Ceil,List())" + "ceil(sum(foo) / 10 / 10 / 10)" -> "ApplyInstantFunction(ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),ScalarVectorBinaryOperation(DIV,ScalarFixedDoublePlan(10.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),false),false),false),Ceil,List())", + "metric1 * metric2 + metric3" -> "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric1))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric2))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric3))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List())", + "metric1 * (metric2 + metric3)" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric1))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric2))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric3))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),List(),List(),List())", + + "(metric1 + (metric2 * metric3)) + metric4" -> "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric1))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),ADD,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric2))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric3))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),List(),List(),List()),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric4))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List())", + "metric1 + metric2 * metric3 + metric4" -> "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric1))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),ADD,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric2))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric3))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),List(),List(),List()),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric4))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List())", + + "(metric1 + metric2) * (metric3 + metric4)" -> "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric1))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric2))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),MUL,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric3))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(metric4))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List()),List(),List(),List())" + ) val qts: Long = 1524855988L From 0ee7207cb0e7ff640b0a73b5a4e6534c6a2ab5db Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 21 Oct 2020 13:04:05 -0700 Subject: [PATCH 45/53] bug(query): add quotes to filter value (#924) --- .../HighAvailabilityPlanner.scala | 8 +--- .../queryplanner/MultiPartitionPlanner.scala | 4 +- .../queryplanner/PlannerMaterializer.scala | 15 +++++++- .../MultiPartitionPlannerSpec.scala | 37 +++++++++++++++++++ 4 files changed, 54 insertions(+), 10 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index a3860caca5..ba4a7c7108 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -47,10 +47,6 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, } } - private def getLabelValuesUrlParams(lp: LabelValues) = Map("filter" -> lp.filters.map{f => f.column + - f.filter.operatorString + f.filter.valuesStrings.head}.mkString(","), - "labels" -> lp.labelNames.mkString(",")) - /** * Converts Route objects returned by FailureProvider to ExecPlan */ @@ -81,8 +77,8 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, val httpEndpoint = remoteHttpEndpoint + queryParams.remoteQueryPath.getOrElse("") rootLogicalPlan match { case lp: LabelValues => MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - getLabelValuesUrlParams(lp), qContext, InProcessPlanDispatcher, - dsRef, promQlParams) + PlannerUtil.getLabelValuesUrlParams(lp, queryParams), qContext, + InProcessPlanDispatcher, dsRef, promQlParams) case lp: SeriesKeysByFilters => val urlParams = Map("match[]" -> queryParams.promQl) MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, urlParams, qContext, InProcessPlanDispatcher, dsRef, promQlParams) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 197987fc20..2b1cbd5392 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -175,9 +175,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider if (p.partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(lp.copy(startMs = p.timeRange.startMs, endMs = p.timeRange.endMs), qContext) else - createMetadataRemoteExec(qContext, queryParams, p, - Map("filter" -> lp.filters.map{f => f.column + f.filter.operatorString + f.filter.valuesStrings.head}. - mkString(","), "labels" -> lp.labelNames.mkString(","))) + createMetadataRemoteExec(qContext, queryParams, p, PlannerUtil.getLabelValuesUrlParams(lp, queryParams)) } if (execPlans.size == 1) execPlans.head else LabelValuesDistConcatExec(qContext, InProcessPlanDispatcher, diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index 2f3abeb918..b5ed10a9ca 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -3,7 +3,7 @@ package filodb.coordinator.queryplanner import java.util.concurrent.ThreadLocalRandom import filodb.core.metadata.{DatasetOptions, Schemas} -import filodb.core.query.{QueryContext, RangeParams} +import filodb.core.query.{PromQlQueryParams, QueryContext, RangeParams} import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.query._ import filodb.query.exec._ @@ -140,3 +140,16 @@ trait PlannerMaterializer { } } } + +object PlannerUtil { + /** + * Returns URL params for label values which is used to create Metadata remote exec plan + */ + def getLabelValuesUrlParams(lp: LabelValues, queryParams: PromQlQueryParams): Map[String, String] = { + val quote = if (queryParams.remoteQueryPath.get.contains("""/v2/label/""")) """"""" else "" + // Filter value should be enclosed in quotes for label values v2 endpoint + val filters = lp.filters.map{ f => s"""${f.column}${f.filter.operatorString}$quote${f.filter.valuesStrings. + head}$quote"""}.mkString(",") + Map("filter" -> filters, "labels" -> lp.labelNames.mkString(",")) + } +} diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index a96936c92e..66ddffc688 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -418,4 +418,41 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { } + it ("should generate Exec plan for Metadata Label values query") { + def partitions(timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("remote", "remote-url", + TimeRange(startSeconds * 1000, localPartitionStart * 1000 - 1)), + PartitionAssignment("local", "local-url", TimeRange(localPartitionStart * 1000, endSeconds * 1000))) + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo""""), TimeStepParams(startSeconds, step, endSeconds) ) + + val promQlQueryParams = PromQlQueryParams( + "", startSeconds, step, endSeconds, None, Some("/api/v2/label/values"), + processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[LabelValuesDistConcatExec] shouldEqual(true) + execPlan.children(1).isInstanceOf[MetadataRemoteExec] shouldEqual(true) + + val expectedUrlParams = Map("filter" -> """_ws_="demo"""", "labels" -> "__metric__") + execPlan.children(1).asInstanceOf[MetadataRemoteExec].urlParams shouldEqual(expectedUrlParams) // Filter values + // should have quotes + execPlan.children(1).asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual(localPartitionStart - 1) + execPlan.children(0).asInstanceOf[LabelValuesDistConcatExec].children(0).asInstanceOf[LabelValuesExec].startMs shouldEqual + (localPartitionStart * 1000) + execPlan.children(0).asInstanceOf[LabelValuesDistConcatExec].children(0).asInstanceOf[LabelValuesExec].endMs shouldEqual + (endSeconds * 1000) + } + } From dfef03be94388fbecb4990d6198b5ce543f3bbf0 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 21 Oct 2020 17:09:27 -0700 Subject: [PATCH 46/53] use multipartitionPlanner for shard key regex metadata queries (#929) --- .../queryplanner/ShardKeyRegexPlanner.scala | 2 ++ .../queryplanner/ShardKeyRegexPlannerSpec.scala | 14 ++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index e9dab4ffa8..b1ccf0ecb6 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -56,6 +56,8 @@ class ShardKeyRegexPlanner(dataset: Dataset, case lp: VectorPlan => materializeVectorPlan(qContext, lp) case lp: Aggregate => materializeAggregate(lp, qContext) case lp: BinaryJoin => materializeBinaryJoin(lp, qContext) + case lp: LabelValues => PlanResult(Seq(queryPlanner.materialize(lp, qContext))) + case lp: SeriesKeysByFilters => PlanResult(Seq(queryPlanner.materialize(lp, qContext))) case _ => materializeOthers(logicalPlan, qContext) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index 6be473bf79..a03acc976d 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -234,4 +234,18 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture multiPartitionExec.children(0).children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual "test{_ws_=\"demo\",_ns_=\"App-2\"}" } + + it ("should generate Exec plan for Metadata Label values query") { + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq.empty + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo", _ns_=~".*" """), + TimeStepParams(1000, 20, 5000) ) + + val promQlQueryParams = PromQlQueryParams( + "", 1000, 20, 5000, None, Some("/api/v2/label/values"), + processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) + } } From 20edbb7132e2c659d91101be37bd6b3aae9849fc Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 26 Oct 2020 13:12:59 -0700 Subject: [PATCH 47/53] bug(query): Small misc low-risk cleanup changes (#934) --- .../filodb.core/memstore/TimeSeriesShard.scala | 7 +++---- .../main/scala/filodb.memory/MemFactory.scala | 5 +++++ .../main/scala/filodb/query/exec/ExecPlan.scala | 16 +++++++++------- .../filodb/query/exec/MetadataExecPlan.scala | 2 -- .../filodb/query/exec/SelectChunkInfosExec.scala | 2 -- .../query/exec/TimeScalarGeneratorExec.scala | 1 - 6 files changed, 17 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index b84183dbf2..baff77d164 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -282,12 +282,11 @@ class TimeSeriesShard(val ref: DatasetRef, // the process to be safe and log details for further debugging. val chunkID = UnsafeUtils.getLong(metaAddr + 4) if (numBytes != partition.schema.data.blockMetaSize) { - logger.error(f"POSSIBLE CORRUPTION DURING onReclaim(metaAddr=0x$metaAddr%08x, numBytes=$numBytes)" + + Shutdown.haltAndCatchFire( new RuntimeException(f"POSSIBLE CORRUPTION DURING onReclaim(" + + f"metaAddr=0x$metaAddr%08x, numBytes=$numBytes)" + s"Expected meta size: ${partition.schema.data.blockMetaSize} for schema=${partition.schema}" + s" Reclaiming chunk chunkID=$chunkID from shard=$shardNum " + - s"partID=$partID ${partition.stringPartition}") - logger.warn("Halting FiloDB...") - sys.exit(33) // Special onReclaim corruption exit code + s"partID=$partID ${partition.stringPartition}")) } partition.removeChunksAt(chunkID) logger.debug(s"Reclaiming chunk chunkID=$chunkID from shard=$shardNum " + diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index a3325aef04..f6d59d8938 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -317,6 +317,11 @@ class BlockMemFactory(blockStore: BlockManager, fullBlocksToBeMarkedAsReclaimable.clear() } + /** + * If current block has the capacity, no-op + * Otherwise, move to next block. As a result, "currentBlock" + * now points to a new block + */ protected def ensureCapacity(forSize: Long): Block = synchronized { var block = accessCurrentBlock() if (block.hasCapacity(forSize)) { diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index fe0c5e8f16..921070b451 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -115,7 +116,8 @@ trait ExecPlan extends QueryCommand { // kamon uses thread-locals. Kamon.runWithSpan(span, true) { val doEx = doExecute(source, querySession) - Kamon.histogram("query-execute-time-elapsed-step1-done") + Kamon.histogram("query-execute-time-elapsed-step1-done", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) doEx @@ -124,7 +126,8 @@ trait ExecPlan extends QueryCommand { // Step 2: Run connect monix pipeline to transformers, materialize the result def step2(res: ExecResult) = res.schema.map { resSchema => - Kamon.histogram("query-execute-time-elapsed-step2-start") + Kamon.histogram("query-execute-time-elapsed-step2-start", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") @@ -147,7 +150,8 @@ trait ExecPlan extends QueryCommand { paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) - Kamon.histogram("query-execute-time-elapsed-step2-transformer-pipeline-setup") + Kamon.histogram("query-execute-time-elapsed-step2-transformer-pipeline-setup", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) val builder = SerializedRangeVector.newBuilder() @@ -173,7 +177,8 @@ trait ExecPlan extends QueryCommand { } .toListL .map { r => - Kamon.histogram("query-execute-time-elapsed-step2-result-materialized") + Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) val numBytes = builder.allContainers.map(_.numBytes).sum @@ -387,7 +392,6 @@ abstract class NonLeafExecPlan extends ExecPlan { querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() - parentSpan.mark("create-child-tasks") // whether child tasks need to be executed sequentially. // parallelism 1 means, only one worker thread to process underlying tasks. @@ -418,9 +422,7 @@ abstract class NonLeafExecPlan extends ExecPlan { val outputSchema = processedTasks.collect { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) - parentSpan.mark("output-compose") val outputRvs = compose(processedTasks, outputSchema, querySession) - parentSpan.mark("return-results") ExecResult(outputRvs, outputSchema) } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index bfff4d3d90..47f2b112e2 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -91,7 +91,6 @@ final case class PartKeysExec(queryContext: QueryContext, case other => Observable.empty } - Kamon.currentSpan().mark("creating-resultschema") val sch = ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) ExecResult(rvs, Task.eval(sch)) } @@ -132,7 +131,6 @@ final case class LabelValuesExec(queryContext: QueryContext, } else { Observable.empty } - parentSpan.mark("creating-resultschema") val sch = ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) ExecResult(rvs, Task.eval(sch)) } diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index 915624457d..27e51f3b6c 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -53,7 +52,6 @@ final case class SelectChunkInfosExec(queryContext: QueryContext, val dataColumn = dataSchema.data.columns(colID) val partCols = dataSchema.partitionInfos val numGroups = source.groupsInDataset(dataset) - Kamon.currentSpan().mark("creating-scanpartitions") val rvs = source.scanPartitions(dataset, lookupRes, Seq.empty, querySession) .filter(_.hasChunks(chunkMethod)) .map { partition => diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index 49d9212385..f59026d62d 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -70,7 +70,6 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, .tag("query-id", queryContext.queryId) .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => - span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) From c7b167b7bb78f07373d0fddcc6765cd8fdec1110 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 28 Oct 2020 10:05:34 -0700 Subject: [PATCH 48/53] bug(query): Remove span per result RV; add spans/markers to track child plans (#935) * Removed scan span which was per result RV. It created too many spans - was not intended earlier. * Added spans and markers to track scans, and child result retrieval. * Fixed config for local zipkin testing --- conf/timeseries-filodb-server.conf | 27 +++++-------- .../scala/filodb.core/query/RangeVector.scala | 11 +----- .../filodb.core/query/RangeVectorSpec.scala | 2 +- .../filodb/query/exec/BinaryJoinExec.scala | 8 +++- .../scala/filodb/query/exec/ExecPlan.scala | 38 ++++++++++++------- .../query/exec/MetadataRemoteExec.scala | 2 +- .../filodb/query/exec/PromQlRemoteExec.scala | 4 +- .../query/exec/RangeVectorTransformer.scala | 2 +- .../filodb/query/exec/SetOperatorExec.scala | 8 +++- .../query/exec/AggrOverRangeVectorsSpec.scala | 2 +- scripts/schema-truncate.sh | 2 +- 11 files changed, 55 insertions(+), 51 deletions(-) diff --git a/conf/timeseries-filodb-server.conf b/conf/timeseries-filodb-server.conf index e458a4ee3f..2800835194 100644 --- a/conf/timeseries-filodb-server.conf +++ b/conf/timeseries-filodb-server.conf @@ -48,24 +48,15 @@ kamon { port = 9095 } modules { - metriclog-reporter { - enabled = true - name = "MetricLog Reporter" - description = "Log all Metrics" - factory = "filodb.coordinator.KamonLogger$MetricsLogFactory" - } - spanlog-reporter { - enabled = true - name = "SpanLog Reporter" - description = "Log all traced Spans" - factory = "filodb.coordinator.KamonLogger$SpanLogFactory" - } - status-page { - enabled = false - } - zipkin-reporter { - enabled = false - } + zipkin-reporter.enabled = false + prometheus-reporter.enabled = false + status-page.enabled = false + } + + zipkin { + url = "https://localhost:9411/api/v2/spans" + max.requests = 128 + message.max.bytes = 131072 } metric.tick-interval = 60s diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index f07e595b23..9bc146ae5b 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -7,7 +7,6 @@ import scala.collection.Iterator import com.typesafe.scalalogging.StrictLogging import debox.Buffer import kamon.Kamon -import kamon.trace.Span import org.joda.time.DateTime import filodb.core.binaryrecord2.{MapItemConsumer, RecordBuilder, RecordContainer, RecordSchema} @@ -317,12 +316,7 @@ object SerializedRangeVector extends StrictLogging { // scalastyle:off null def apply(rv: RangeVector, builder: RecordBuilder, - schema: RecordSchema, - execPlanName: String, - span: Span = null): SerializedRangeVector = { - var spanBldr = Kamon.spanBuilder(s"execplan-scan-latency-$execPlanName") - if (span != null) { spanBldr = spanBldr.asChildOf(span) } - val scanSpan = spanBldr.start() + schema: RecordSchema): SerializedRangeVector = { var numRows = 0 val oldContainerOpt = builder.currentContainer val startRecordNo = oldContainerOpt.map(_.numRecords).getOrElse(0) @@ -345,7 +339,6 @@ object SerializedRangeVector extends StrictLogging { case None => builder.allContainers case Some(firstContainer) => builder.allContainers.dropWhile(_ != firstContainer) } - scanSpan.finish() new SerializedRangeVector(rv.key, numRows, containers, schema, startRecordNo) } // scalastyle:on null @@ -356,7 +349,7 @@ object SerializedRangeVector extends StrictLogging { */ def apply(rv: RangeVector, cols: Seq[ColumnInfo]): SerializedRangeVector = { val schema = toSchema(cols) - apply(rv, newBuilder(), schema, "Test-Only-Plan") + apply(rv, newBuilder(), schema) } // TODO: make this configurable.... diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala index 4bdeaf56aa..8c15be7390 100644 --- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala @@ -50,7 +50,7 @@ class RangeVectorSpec extends AnyFunSpec with Matchers { val builder = SerializedRangeVector.newBuilder() // Sharing one builder across multiple input RangeVectors - val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema, "Unit-test")) + val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema)) // Now verify each of them val observedTs = srvs(0).rows.toSeq.map(_.getLong(0)) diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index a4d390a241..d4cc446df4 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -3,6 +3,7 @@ package filodb.query.exec import scala.collection.mutable import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.reactive.Observable @@ -65,6 +66,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], querySession: QuerySession): Observable[RangeVector] = { + val span = Kamon.currentSpan() val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) if (result.size > queryContext.joinQueryCardLimit && cardinality == Cardinality.OneToOne) => @@ -73,9 +75,11 @@ final case class BinaryJoinExec(queryContext: QueryContext, case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => - Kamon.histogram("query-execute-time-elapsed-step2-child-results-available") + span.mark("binary-join-child-results-available") + Kamon.histogram("query-execute-time-elapsed-step1-child-results-available", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - queryContext.submitTime) + .record(Math.max(0, System.currentTimeMillis - queryContext.submitTime)) // NOTE: We can't require this any more, as multischema queries may result in not a QueryResult if the // filter returns empty results. The reason is that the schema will be undefined. // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 921070b451..b9053aac69 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -119,17 +119,16 @@ trait ExecPlan extends QueryCommand { Kamon.histogram("query-execute-time-elapsed-step1-done", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) doEx } } // Step 2: Run connect monix pipeline to transformers, materialize the result def step2(res: ExecResult) = res.schema.map { resSchema => - Kamon.histogram("query-execute-time-elapsed-step2-start", - MeasurementUnit.time.milliseconds) + Kamon.histogram("query-execute-time-elapsed-step2-start", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") .asChildOf(parentSpan) .tag("query-id", queryContext.queryId) @@ -144,7 +143,6 @@ trait ExecPlan extends QueryCommand { Task.eval(QueryResult(queryContext.queryId, resSchema, Nil)) } else { val finalRes = allTransformers.foldLeft((res.rvs, resSchema)) { (acc, transf) => - span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) @@ -153,10 +151,12 @@ trait ExecPlan extends QueryCommand { Kamon.histogram("query-execute-time-elapsed-step2-transformer-pipeline-setup", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) + span.mark("step2-transformer-pipeline-setup") val builder = SerializedRangeVector.newBuilder() @volatile var numResultSamples = 0 // BEWARE - do not modify concurrently!! finalRes._1 + .doOnStart(_ => span.mark("before-first-materialized-result-rv")) .map { case srv: SerializableRangeVector => numResultSamples += srv.numRowsInt @@ -167,7 +167,7 @@ trait ExecPlan extends QueryCommand { srv case rv: RangeVector => // materialize, and limit rows per RV - val srv = SerializedRangeVector(rv, builder, recSchema, getClass.getSimpleName, span) + val srv = SerializedRangeVector(rv, builder, recSchema) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results if (enforceLimit && numResultSamples > queryContext.sampleLimit) @@ -175,12 +175,13 @@ trait ExecPlan extends QueryCommand { s"Try applying more filters or reduce time range.") srv } + .doOnTerminate(_ => span.mark("after-last-materialized-result-rv")) .toListL .map { r => Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", - MeasurementUnit.time.milliseconds) + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) val numBytes = builder.allContainers.map(_.numBytes).sum SerializedRangeVector.queryResultBytes.record(numBytes) span.mark(s"num-bytes: $numBytes") @@ -393,6 +394,11 @@ abstract class NonLeafExecPlan extends ExecPlan { (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() + val span = Kamon.spanBuilder(s"execute-step1-child-result-composition-${getClass.getSimpleName}") + .asChildOf(parentSpan) + .tag("query-id", queryContext.queryId) + .start() + // whether child tasks need to be executed sequentially. // parallelism 1 means, only one worker thread to process underlying tasks. val parallelism: Int = if (parallelChildTasks) @@ -404,13 +410,16 @@ abstract class NonLeafExecPlan extends ExecPlan { // NOTE: It's really important to preserve the "index" of the child task, as joins depend on it val childTasks = Observable.fromIterable(children.zipWithIndex) .mapAsync(parallelism) { case (plan, i) => - dispatchRemotePlan(plan, parentSpan).map((_, i)) + dispatchRemotePlan(plan, span).map((_, i)) } // The first valid schema is returned as the Task. If all results are empty, then return // an empty schema. Validate that the other schemas are the same. Skip over empty schemas. var sch = ResultSchema.empty - val processedTasks = childTasks.collect { + val processedTasks = childTasks + .doOnStart(_ => span.mark("first-child-result-received")) + .doOnTerminate(_ => span.mark("last-child-result-received")) + .collect { case (res @ QueryResult(_, schema, _), i) if schema != ResultSchema.empty => sch = reduceSchemas(sch, res) (res, i.toInt) @@ -422,8 +431,11 @@ abstract class NonLeafExecPlan extends ExecPlan { val outputSchema = processedTasks.collect { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) - val outputRvs = compose(processedTasks, outputSchema, querySession) - ExecResult(outputRvs, outputSchema) + Kamon.runWithSpan(span, false) { + val outputRvs = compose(processedTasks, outputSchema, querySession) + .doOnTerminate(_ => span.finish()) + ExecResult(outputRvs, outputSchema) + } } /** diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 667b23f398..5b54519f1f 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -48,7 +48,7 @@ case class MetadataRemoteExec(queryEndpoint: String, val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(iteratorMap.toIterator)) - val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, this.getClass.getSimpleName, span)) + val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema)) span.finish() QueryResult(id, resultSchema, srvSeq) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index d20ff7e8c2..2b800faa29 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -162,7 +162,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, defaultRecSchema, "PromQlRemoteExec-default") + SerializedRangeVector(rv, builder, defaultRecSchema) // TODO: Handle stitching with verbose flag } QueryResult(id, defaultResultSchema, rangeVectors) @@ -196,7 +196,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, histRecSchema, "PromQlRemoteExec-hist") + SerializedRangeVector(rv, builder, histRecSchema) // TODO: Handle stitching with verbose flag } QueryResult(id, histResultSchema, rangeVectors) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index cf2e9be525..06506327c7 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -279,7 +279,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto // Create SerializedRangeVector so that sorting does not consume rows iterator val resultRv = source.toListL.map { rvs => - rvs.map(SerializedRangeVector(_, builder, recSchema, getClass.getSimpleName)). + rvs.map(SerializedRangeVector(_, builder, recSchema)). sortBy { rv => if (rv.rows.hasNext) rv.rows.next().getDouble(1) else Double.NaN }(ordering) diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index d210bf150c..f8c1318359 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -4,6 +4,7 @@ import scala.collection.mutable import scala.collection.mutable.ListBuffer import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.reactive.Observable @@ -52,13 +53,16 @@ final case class SetOperatorExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], querySession: QuerySession): Observable[RangeVector] = { + val span = Kamon.currentSpan() val taskOfResults = childResponses.map { case (QueryResult(_, schema, result), i) => (schema, result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => - Kamon.histogram("query-execute-time-elapsed-step2-child-results-available") + span.mark("binary-join-child-results-available") + Kamon.histogram("query-execute-time-elapsed-step1-child-results-available", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - queryContext.submitTime) + .record(Math.max(0, System.currentTimeMillis - queryContext.submitTime)) // NOTE: We can't require this any more, as multischema queries may result in not a QueryResult if the // filter returns empty results. The reason is that the schema will be undefined. // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 80c1e2f921..d33e86d497 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -287,7 +287,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val recSchema = SerializedRangeVector.toSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("tdig", ColumnType.StringColumn))) val builder = SerializedRangeVector.newBuilder() - val srv = SerializedRangeVector(result7(0), builder, recSchema, "Unit-Test") + val srv = SerializedRangeVector(result7(0), builder, recSchema) val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000) val finalResult = resultObs7b.toListL.runAsync.futureValue diff --git a/scripts/schema-truncate.sh b/scripts/schema-truncate.sh index 82e37d2050..fdad7c306d 100755 --- a/scripts/schema-truncate.sh +++ b/scripts/schema-truncate.sh @@ -39,7 +39,7 @@ done if [[ "${KEYSP}" != "${FILO_DOWNSAMPLE_KEYSPACE}" ]]; then cat << EOF -TRUNCATE ${KEYSP}.${DSET}_pks_by_update_time +TRUNCATE ${KEYSP}.${DSET}_pks_by_update_time; EOF fi From 59e3b0b63ce2bbfeb2811aaa31d36ee6c2ed12e0 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 28 Oct 2020 12:05:37 -0700 Subject: [PATCH 49/53] debug(query): Marking spans when dispatch of child plan is done (#936) --- query/src/main/scala/filodb/query/exec/ExecPlan.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index b9053aac69..c153fefe58 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -410,7 +410,9 @@ abstract class NonLeafExecPlan extends ExecPlan { // NOTE: It's really important to preserve the "index" of the child task, as joins depend on it val childTasks = Observable.fromIterable(children.zipWithIndex) .mapAsync(parallelism) { case (plan, i) => - dispatchRemotePlan(plan, span).map((_, i)) + val task = dispatchRemotePlan(plan, span).map((_, i)) + span.mark(s"plan-dispatched-${plan.getClass.getSimpleName}") + task } // The first valid schema is returned as the Task. If all results are empty, then return From ea1b615a04fd73a75d57789705a3ba80b954df32 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 4 Nov 2020 12:03:51 -0800 Subject: [PATCH 50/53] fix(query,memory): Lenient lock validation check to account for possible re-entrancy. --- .../main/scala/filodb.memory/data/ChunkMap.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 94c96031a8..6bd905216c 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -110,9 +110,19 @@ object ChunkMap extends StrictLogging { * it is quite possible a lock acquire or release bug exists */ def validateNoSharedLocks(unitTest: Boolean = false): Unit = { - val numLocksReleased = ChunkMap.releaseAllSharedLocks() - if (numLocksReleased > 0) { - val ex = new RuntimeException(s"Number of locks was non-zero: $numLocksReleased. " + + // Count up the number of held locks. + var total = 0 + val countMap = sharedLockCounts.get + if (countMap != null) { + for ((inst, amt) <- countMap) { + if (amt > 0) { + total += amt + } + } + } + + if (total > 10) { // lenient check for now + val ex = new RuntimeException(s"Number of locks lingering: $total. " + s"This is indicative of a possible lock acquisition/release bug.") Shutdown.haltAndCatchFire(ex) } From 33d9a558c59197c3f9efe3f89ce69a11bd252b91 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 4 Nov 2020 12:23:56 -0800 Subject: [PATCH 51/53] Simply add TimeSeriesPartition.toString for better debug log --- .../main/scala/filodb.core/memstore/TimeSeriesPartition.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 1cf9988924..2aed0a93dc 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -444,6 +444,10 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { memFactory.freeMemory(partKeyOffset) if (currentInfo != nullInfo) bufferPool.release(currentInfo.infoAddr, currentChunks) } + + override def toString: String = { + s"TimeSeriesPartition(shard=$shard,partId=$partID){$stringPartition}" + } } /** From 23d850d7c3cce04335b4a8e7a78758035f7e6783 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 5 Nov 2020 11:34:29 -0800 Subject: [PATCH 52/53] bug(memory): Log lingering locks for debugging purposes (#947) --- memory/src/main/scala/filodb.memory/data/ChunkMap.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 6bd905216c..4d4cbc1602 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -117,6 +117,8 @@ object ChunkMap extends StrictLogging { for ((inst, amt) <- countMap) { if (amt > 0) { total += amt + sharedLockLingering.increment(amt) + _logger.warn(s"THIS IS A BUG. ChunkMap shared locks was not released for: $inst, amount: $amt") } } } From 5da5091a2a9029b7f801e13d3b7ea4d8f570190c Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 6 Nov 2020 13:40:24 -0800 Subject: [PATCH 53/53] debug(core): TracingTimeSeriesPartition trace chunkmap locks (#950) (#952) --- .../memstore/TimeSeriesPartition.scala | 22 ++++++++++++++++++- .../scala/filodb.memory/data/ChunkMap.scala | 8 +++---- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 2aed0a93dc..7afbc4678c 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -494,8 +494,28 @@ TimeSeriesPartition(partID, schema, partitionKey, shard, bufferPool, shardStats, _log.info(s"dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + s"newly created ChunkInfo ${currentInfo.debugString}") } -} + override def chunkmapAcquireShared(): Unit = { + super.chunkmapAcquireShared() + _log.info(s"SHARED LOCK ACQUIRED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + + override def chunkmapReleaseShared(): Unit = { + super.chunkmapReleaseShared() + _log.info(s"SHARED LOCK RELEASED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + + override def chunkmapAcquireExclusive(): Unit = { + super.chunkmapAcquireExclusive() + _log.info(s"EXCLUSIVE LOCK ACQUIRED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + + override def chunkmapReleaseExclusive(): Unit = { + super.chunkmapReleaseExclusive() + _log.info(s"EXCLUSIVE LOCK RELEASED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + +} final case class PartKeyRowReader(records: Iterator[PartKeyWithTimes]) extends Iterator[RowReader] { var currVal: PartKeyWithTimes = _ diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 4d4cbc1602..4c310790ee 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -243,7 +243,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Acquire exclusive access to this map, spinning if necessary. Exclusive lock isn't re-entrant. */ - final def chunkmapAcquireExclusive(): Unit = { + def chunkmapAcquireExclusive(): Unit = { // Spin-lock implementation. Because the owner of the shared lock might be blocked by this // thread as it waits for an exclusive lock, deadlock is possible. To mitigate this problem, // timeout and retry, allowing shared lock waiters to make progress. The timeout doubles @@ -332,7 +332,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Release an acquired exclusive lock. */ - final def chunkmapReleaseExclusive(): Unit = { + def chunkmapReleaseExclusive(): Unit = { UnsafeUtils.setIntVolatile(this, lockStateOffset, 0) } @@ -351,7 +351,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Acquire shared access to this map, spinning if necessary. */ - final def chunkmapAcquireShared(): Unit = { + def chunkmapAcquireShared(): Unit = { // Spin-lock implementation. var lockState = 0 @@ -371,7 +371,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Release an acquired shared lock. */ - final def chunkmapReleaseShared(): Unit = { + def chunkmapReleaseShared(): Unit = { var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset)