Skip to content

Commit

Permalink
eval: add a rewrite module to handle queries with namespaces (#1675)
Browse files Browse the repository at this point in the history
If configured via `atlas.eval.stream.rewrite-url`, data source URIs are sent to a
rewriting service. Responses with errors are then dropped and the caller notified.
Valid rewrites continue processing.

Some uses, e.g. the atlas-stream case, entail updating the data sources for a
flow when new subscriptions are added or existing subscriptions dropped.
This will return cached rewrites for individual queries in the event the
rewrite backend is unavailable. Calls will continue for the rewrite service
until a successful response is received, at which point it will update the
cache and forward the rewrites downstream.
  • Loading branch information
manolama authored Aug 20, 2024
1 parent a11c6d0 commit 48d16a8
Show file tree
Hide file tree
Showing 7 changed files with 871 additions and 5 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,237 @@
/*
* Copyright 2014-2024 Netflix, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.netflix.atlas.eval.stream

import com.netflix.atlas.eval.stream.Evaluator.DataSource
import com.netflix.atlas.eval.stream.Evaluator.DataSources
import com.netflix.atlas.eval.stream.HostSource.unzipIfNeeded
import com.netflix.atlas.json.Json
import com.netflix.atlas.pekko.DiagnosticMessage
import com.netflix.atlas.pekko.OpportunisticEC.ec
import com.netflix.atlas.pekko.PekkoHttpClient
import com.netflix.atlas.pekko.StreamOps
import com.netflix.spectator.api.Registry
import com.typesafe.config.Config
import com.typesafe.scalalogging.StrictLogging
import org.apache.pekko.NotUsed
import org.apache.pekko.actor.ActorSystem
import org.apache.pekko.http.scaladsl.model.ContentTypes
import org.apache.pekko.http.scaladsl.model.HttpEntity
import org.apache.pekko.http.scaladsl.model.HttpMethods
import org.apache.pekko.http.scaladsl.model.HttpRequest
import org.apache.pekko.http.scaladsl.model.StatusCodes
import org.apache.pekko.stream.scaladsl.BroadcastHub
import org.apache.pekko.stream.scaladsl.Flow
import org.apache.pekko.stream.scaladsl.Keep
import org.apache.pekko.stream.scaladsl.RetryFlow
import org.apache.pekko.stream.scaladsl.Source

import java.io.ByteArrayOutputStream
import java.util.concurrent.ConcurrentHashMap
import scala.concurrent.duration.DurationInt
import scala.jdk.CollectionConverters.CollectionHasAsScala
import scala.util.Failure
import scala.util.Success
import scala.util.Using

class DataSourceRewriter(
config: Config,
registry: Registry,
implicit val system: ActorSystem
) extends StrictLogging {

private val (enabled, rewriteUrl) = {
val enabled = config.hasPath("atlas.eval.stream.rewrite-url")
val url = if (enabled) config.getString("atlas.eval.stream.rewrite-url") else ""
if (enabled) {
logger.info(s"Rewriting enabled with url: ${url}")
} else {
logger.info("Rewriting is disabled")
}
(enabled, url)
}

private val client = PekkoHttpClient
.create("datasource-rewrite", system)
.superPool[List[DataSource]]()

private val rewriteCache = new ConcurrentHashMap[String, String]()

private val rewriteSuccess = registry.counter("atlas.eval.stream.rewrite.success")
private val rewriteFailures = registry.createId("atlas.eval.stream.rewrite.failures")
private val rewriteCacheHits = registry.counter("atlas.eval.stream.rewrite.cache", "id", "hits")

private val rewriteCacheMisses =
registry.counter("atlas.eval.stream.rewrite.cache", "id", "misses")

def rewrite(
context: StreamContext,
keepRetrying: Boolean = true
): Flow[DataSources, DataSources, NotUsed] = {
rewrite(client, context, keepRetrying)
}

def rewrite(
client: SuperPoolClient,
context: StreamContext,
keepRetrying: Boolean
): Flow[DataSources, DataSources, NotUsed] = {
if (!enabled) {
return Flow[DataSources]
}

val (cachedQueue, cachedSource) = StreamOps
.blockingQueue[DataSources](registry, "cachedRewrites", 1)
.toMat(BroadcastHub.sink(1))(Keep.both)
.run()
var sentCacheData = false

val retryFlow = RetryFlow
.withBackoff(
minBackoff = 100.milliseconds,
maxBackoff = 5.second,
randomFactor = 0.35,
maxRetries = if (keepRetrying) -1 else 0,
flow = httpFlow(client, context)
) {
case (original, resp) =>
resp match {
case Success(_) => None
case Failure(ex) =>
val (request, dsl) = original
logger.debug("Retrying the rewrite request due to error", ex)
if (!sentCacheData) {
if (!cachedQueue.offer(returnFromCache(dsl))) {
// note that this should never happen.
logger.error("Unable to send cached results to queue.")
} else {
sentCacheData = true
}
}
Some(request -> dsl)
}

}
.watchTermination() { (_, f) =>
f.onComplete { _ =>
cachedQueue.complete()
}
}

Flow[DataSources]
.map(_.sources().asScala.toList)
.map(dsl => constructRequest(dsl) -> dsl)
.via(retryFlow)
.filter(_.isSuccess)
.map {
// reset the cached flag
sentCacheData = false
_.get
}
.merge(cachedSource)
}

private[stream] def httpFlow(client: SuperPoolClient, context: StreamContext) = {
Flow[(HttpRequest, List[DataSource])]
.via(client)
.flatMapConcat {
case (Success(resp), dsl) =>
unzipIfNeeded(resp)
.map(_.utf8String)
.map { body =>
resp.status match {
case StatusCodes.OK =>
val rewrites = List.newBuilder[DataSource]
Json
.decode[List[Rewrite]](body)
.zip(dsl)
.map {
case (r, ds) =>
if (!r.status.equals("OK")) {
val msg =
DiagnosticMessage.error(s"failed rewrite of ${ds.uri()}: ${r.message}")
context.dsLogger(ds, msg)
} else {
rewriteCache.put(ds.uri(), r.rewrite)
rewrites += new DataSource(ds.id, ds.step(), r.rewrite)
}
}
.toArray
// NOTE: We're assuming that the number of items returned will be the same as the
// number of uris sent to the rewrite service. If they differ, data sources may be
// mapped to IDs and steps incorrectly.
rewriteSuccess.increment()
Success(DataSources.of(rewrites.result().toArray: _*))
case _ =>
logger.error(
"Error from rewrite service. status={}, resp={}",
resp.status,
body
)
registry
.counter(
rewriteFailures.withTags("status", resp.status.toString(), "exception", "NA")
)
.increment()
Failure(
new RuntimeException(
s"Error from rewrite service. status=${resp.status}, resp=$body"
)
)
}
}
case (Failure(ex), _) =>
logger.error("Failure from rewrite service", ex)
registry
.counter(
rewriteFailures.withTags("status", "0", "exception", ex.getClass.getSimpleName)
)
.increment()
Source.single(Failure(ex))
}
}

private[stream] def returnFromCache(dsl: List[DataSource]): DataSources = {
val rewrites = dsl.flatMap { ds =>
val rewrite = rewriteCache.get(ds.uri())
if (rewrite == null) {
rewriteCacheMisses.increment()
None
} else {
rewriteCacheHits.increment()
Some(new DataSource(ds.id, ds.step(), rewrite))
}
}
DataSources.of(rewrites: _*)
}

private[stream] def constructRequest(dss: List[DataSource]): HttpRequest = {
val baos = new ByteArrayOutputStream
Using(Json.newJsonGenerator(baos)) { json =>
json.writeStartArray()
dss.foreach(s => json.writeString(s.uri()))
json.writeEndArray()
}
HttpRequest(
uri = rewriteUrl,
method = HttpMethods.POST,
entity = HttpEntity(ContentTypes.`application/json`, baos.toByteArray)
)
}

}

case class Rewrite(status: String, rewrite: String, original: String, message: String)
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ import com.netflix.atlas.eval.stream.Evaluator.MessageEnvelope
import com.netflix.atlas.json.Json
import com.netflix.atlas.json.JsonSupport
import com.netflix.atlas.pekko.ClusterOps
import com.netflix.atlas.pekko.DiagnosticMessage
import com.netflix.atlas.pekko.StreamOps
import com.netflix.atlas.pekko.ThreadPools
import com.netflix.spectator.api.Registry
Expand All @@ -86,8 +87,11 @@ private[stream] abstract class EvaluatorImpl(

private val logger = LoggerFactory.getLogger(getClass)

// Calls out to a rewrite service in case URIs need mutating to pick the proper backend.
private[stream] var dataSourceRewriter = new DataSourceRewriter(config, registry, system)

// Cached context instance used for things like expression validation.
private val validationStreamContext = newStreamContext()
private val validationStreamContext = newStreamContext(new ThrowingDSLogger)

// Timeout for DataSources unique operator: emit repeating DataSources after timeout exceeds
private val uniqueTimeout: Long = config.getDuration("atlas.eval.stream.unique-timeout").toMillis
Expand Down Expand Up @@ -129,7 +133,13 @@ private[stream] abstract class EvaluatorImpl(
}

protected def validateImpl(ds: DataSource): Unit = {
validationStreamContext.validateDataSource(ds).get
val future = Source
.single(DataSources.of(ds))
.via(dataSourceRewriter.rewrite(validationStreamContext, false))
.map(_.sources().asScala.map(validationStreamContext.validateDataSource).map(_.get))
.toMat(Sink.head)(Keep.right)
.run()
Await.result(future, 60.seconds)
}

protected def writeInputToFileImpl(uri: String, file: Path, duration: Duration): Unit = {
Expand Down Expand Up @@ -212,6 +222,7 @@ private[stream] abstract class EvaluatorImpl(
def createStreamsFlow: Flow[DataSources, MessageEnvelope, NotUsed] = {
val (logSrc, context) = createStreamContextSource
Flow[DataSources]
.via(dataSourceRewriter.rewrite(context))
.map(dss => groupByHost(dss))
// Emit empty DataSource if no more DataSource for a host, so that the sub-stream get the info
.via(new FillRemovedKeysWith[String, DataSources](_ => DataSources.empty()))
Expand Down Expand Up @@ -581,4 +592,16 @@ private[stream] abstract class EvaluatorImpl(
private def isPrintable(c: Int): Boolean = {
c >= 32 && c < 127
}

private class ThrowingDSLogger extends DataSourceLogger {

override def apply(ds: DataSource, msg: JsonSupport): Unit = {
msg match {
case dsg: DiagnosticMessage =>
throw new IllegalArgumentException(dsg.message)
}
}

override def close(): Unit = {}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ private[stream] object HostSource extends StrictLogging {
Source.empty[ByteString]
}

private def unzipIfNeeded(res: HttpResponse): Source[ByteString, Any] = {
def unzipIfNeeded(res: HttpResponse): Source[ByteString, Any] = {
val isCompressed = res.headers.contains(`Content-Encoding`(HttpEncodings.gzip))
val dataBytes = res.entity.withoutSizeLimit().dataBytes
if (isCompressed) dataBytes.via(Compression.gunzip()) else dataBytes
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,5 +30,8 @@ package object stream {

type SimpleClient = Flow[HttpRequest, Try[HttpResponse], NotUsed]

type SuperPoolClient =
Flow[(HttpRequest, List[DataSource]), (Try[HttpResponse], List[DataSource]), NotUsed]

type SourcesAndGroups = (DataSources, EddaSource.Groups)
}
Loading

0 comments on commit 48d16a8

Please sign in to comment.