-
Notifications
You must be signed in to change notification settings - Fork 224
/
QueryActor.scala
237 lines (210 loc) · 10.2 KB
/
QueryActor.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
package filodb.coordinator
import java.lang.Thread.UncaughtExceptionHandler
import java.util.concurrent.{ScheduledThreadPoolExecutor, ThreadFactory}
import java.util.concurrent.atomic.AtomicLong
import scala.util.control.NonFatal
import akka.actor.{ActorRef, ActorSystem, Props}
import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox}
import com.typesafe.config.Config
import kamon.Kamon
import kamon.instrumentation.executor.ExecutorInstrumentation
import kamon.tag.TagSet
import monix.execution.Scheduler
import monix.execution.schedulers.SchedulerService
import net.ceedubs.ficus.Ficus._
import net.ceedubs.ficus.readers.ValueReader
import filodb.coordinator.queryplanner.SingleClusterPlanner
import filodb.core._
import filodb.core.memstore.{FiloSchedulers, MemStore, TermInfo}
import filodb.core.metadata.Schemas
import filodb.core.query.{QueryConfig, QueryContext, QuerySession}
import filodb.core.store.CorruptVectorException
import filodb.query._
import filodb.query.exec.ExecPlan
object QueryCommandPriority extends java.util.Comparator[Envelope] {
override def compare(o1: Envelope, o2: Envelope): Int = {
(o1.message, o2.message) match {
case (q1: QueryCommand, q2: QueryCommand) => q1.submitTime.compareTo(q2.submitTime)
case (_, _: QueryCommand) => -1 // non-query commands are admin and have higher priority
case (_: QueryCommand, _) => 1 // non-query commands are admin and have higher priority
case _ => 0
}
}
}
class QueryActorMailbox(settings: ActorSystem.Settings, config: Config)
extends UnboundedStablePriorityMailbox(QueryCommandPriority)
object QueryActor {
private val nextId = new AtomicLong()
def nextQueryId: Long = nextId.getAndIncrement
final case class ThrowException(dataset: DatasetRef)
def props(memStore: MemStore, dsRef: DatasetRef,
schemas: Schemas, shardMapFunc: => ShardMapper,
earliestRawTimestampFn: => Long): Props =
Props(new QueryActor(memStore, dsRef, schemas,
shardMapFunc, earliestRawTimestampFn)).withMailbox("query-actor-mailbox")
}
/**
* Translates external query API calls into internal ColumnStore calls.
*
* The actual reading of data structures and aggregation is performed asynchronously by Observables,
* so it is probably fine for there to be just one QueryActor per dataset.
*/
final class QueryActor(memStore: MemStore,
dsRef: DatasetRef,
schemas: Schemas,
shardMapFunc: => ShardMapper,
earliestRawTimestampFn: => Long) extends BaseActor {
import QueryActor._
import client.QueryCommands._
import filodb.core.memstore.FiloSchedulers._
val config = context.system.settings.config
val dsOptions = schemas.part.options
var filodbSpreadMap = new collection.mutable.HashMap[collection.Map[String, String], Int]
val applicationShardKeyNames = dsOptions.nonMetricShardColumns
val defaultSpread = config.getInt("filodb.spread-default")
implicit val spreadOverrideReader: ValueReader[SpreadAssignment] = ValueReader.relative { spreadAssignmentConfig =>
SpreadAssignment(
shardKeysMap = dsOptions.nonMetricShardColumns.map(x =>
(x, spreadAssignmentConfig.getString(x))).toMap[String, String],
spread = spreadAssignmentConfig.getInt("_spread_")
)
}
val spreadAssignment : List[SpreadAssignment]= config.as[List[SpreadAssignment]]("filodb.spread-assignment")
spreadAssignment.foreach{ x => filodbSpreadMap.put(x.shardKeysMap, x.spread)}
val spreadFunc = QueryContext.simpleMapSpreadFunc(applicationShardKeyNames, filodbSpreadMap, defaultSpread)
val functionalSpreadProvider = FunctionalSpreadProvider(spreadFunc)
logger.info(s"Starting QueryActor and QueryEngine for ds=$dsRef schemas=$schemas")
val queryConfig = new QueryConfig(config.getConfig("filodb.query"))
val queryPlanner = new SingleClusterPlanner(dsRef, schemas, shardMapFunc,
earliestRawTimestampFn, queryConfig, functionalSpreadProvider)
val queryScheduler = createInstrumentedQueryScheduler()
private val tags = Map("dataset" -> dsRef.toString)
private val lpRequests = Kamon.counter("queryactor-logicalPlan-requests").withTags(TagSet.from(tags))
private val epRequests = Kamon.counter("queryactor-execplan-requests").withTags(TagSet.from(tags))
private val resultVectors = Kamon.histogram("queryactor-result-num-rvs").withTags(TagSet.from(tags))
private val queryErrors = Kamon.counter("queryactor-query-errors").withTags(TagSet.from(tags))
/**
* Instrumentation adds following metrics on the Query Scheduler
*
* # Counter
* executor_tasks_submitted_total{type="ThreadPoolExecutor",name="query-sched-prometheus"}
* # Counter
* executor_tasks_completed_total{type="ThreadPoolExecutor",name="query-sched-prometheus"}
* # Histogram
* executor_threads_active{type="ThreadPoolExecutor",name="query-sched-prometheus"}
* # Histogram
* executor_queue_size_count{type="ThreadPoolExecutor",name="query-sched-prometheus"}
*
*/
private def createInstrumentedQueryScheduler(): SchedulerService = {
val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor")
* sys.runtime.availableProcessors).toInt
val schedName = s"$QuerySchedName-$dsRef"
val thFactory = new ThreadFactory {
def newThread(r: Runnable) = {
val thread = new Thread(r)
thread.setName(s"$schedName-${thread.getId}")
thread.setDaemon(true)
thread.setUncaughtExceptionHandler(
new UncaughtExceptionHandler {
override def uncaughtException(t: Thread, e: Throwable): Unit =
logger.error("Uncaught Exception in Query Scheduler", e)
})
thread
}
}
// TODO retaining old fixed size pool for now - later change to fork join pool.
val executor = new ScheduledThreadPoolExecutor(numSchedThreads, thFactory)
Scheduler.apply(ExecutorInstrumentation.instrument(executor, schedName))
}
def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = {
if (checkTimeout(q.queryContext, replyTo)) {
epRequests.increment()
Kamon.currentSpan().tag("query", q.getClass.getSimpleName)
Kamon.currentSpan().tag("query-id", q.queryContext.queryId)
val querySession = QuerySession(q.queryContext, queryConfig)
q.execute(memStore, querySession)(queryScheduler)
.foreach { res =>
FiloSchedulers.assertThreadName(QuerySchedName)
querySession.close()
replyTo ! res
res match {
case QueryResult(_, _, vectors) => resultVectors.record(vectors.length)
case e: QueryError =>
queryErrors.increment()
logger.debug(s"queryId ${q.queryContext.queryId} Normal QueryError returned from query execution: $e")
e.t match {
case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dsRef, cve)
case t: Throwable =>
}
}
}(queryScheduler).recover { case ex =>
querySession.close()
// Unhandled exception in query, should be rare
logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error: ", ex)
replyTo ! QueryError(q.queryContext.queryId, ex)
}(queryScheduler)
}
}
private def processLogicalPlan2Query(q: LogicalPlan2Query, replyTo: ActorRef) = {
if (checkTimeout(q.qContext, replyTo)) {
// This is for CLI use only. Always prefer clients to materialize logical plan
lpRequests.increment()
try {
val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext)
self forward execPlan
} catch {
case NonFatal(ex) =>
if (!ex.isInstanceOf[BadQueryException]) // dont log user errors
logger.error(s"Exception while materializing logical plan", ex)
replyTo ! QueryError("unknown", ex)
}
}
}
private def processExplainPlanQuery(q: ExplainPlan2Query, replyTo: ActorRef): Unit = {
if (checkTimeout(q.qContext, replyTo)) {
try {
val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext)
replyTo ! execPlan
} catch {
case NonFatal(ex) =>
if (!ex.isInstanceOf[BadQueryException]) // dont log user errors
logger.error(s"Exception while materializing logical plan", ex)
replyTo ! QueryError("unknown", ex)
}
}
}
private def processIndexValues(g: GetIndexValues, originator: ActorRef): Unit = {
val localShards = memStore.activeShards(g.dataset)
if (localShards contains g.shard) {
originator ! memStore.labelValues(g.dataset, g.shard, g.indexName, g.limit)
.map { case TermInfo(term, freq) => (term.toString, freq) }
} else {
val destNode = shardMapFunc.coordForShard(g.shard)
if (destNode != ActorRef.noSender) { destNode.forward(g) }
else { originator ! BadArgument(s"Shard ${g.shard} is not assigned") }
}
}
def checkTimeout(queryContext: QueryContext, replyTo: ActorRef): Boolean = {
// timeout can occur here if there is a build up in actor mailbox queue and delayed delivery
val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime
if (queryTimeElapsed >= queryContext.queryTimeoutMillis) {
replyTo ! QueryError(s"Query timeout, $queryTimeElapsed ms > ${queryContext.queryTimeoutMillis}",
QueryTimeoutException(queryTimeElapsed, this.getClass.getName))
false
} else true
}
def receive: Receive = {
case q: LogicalPlan2Query => val replyTo = sender()
processLogicalPlan2Query(q, replyTo)
case q: ExplainPlan2Query => val replyTo = sender()
processExplainPlanQuery(q, replyTo)
case q: ExecPlan => execPhysicalPlan2(q, sender())
case GetIndexNames(ref, limit, _) =>
sender() ! memStore.indexNames(ref, limit).map(_._1).toBuffer
case g: GetIndexValues => processIndexValues(g, sender())
case ThrowException(dataset) =>
logger.warn(s"Throwing exception for dataset $dataset. QueryActor will be killed")
throw new RuntimeException
}
}