-
Notifications
You must be signed in to change notification settings - Fork 53
Expand file tree
/
Copy pathSQLGen.scala
More file actions
714 lines (608 loc) · 27.1 KB
/
SQLGen.scala
File metadata and controls
714 lines (608 loc) · 27.1 KB
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
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
package com.singlestore.spark
import java.sql.{Date, Timestamp}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.types._
import org.slf4j.{Logger, LoggerFactory}
import com.singlestore.spark.JdbcHelpers.getDMLConnProperties
import org.apache.spark.sql.execution.datasources.LogicalRelation
import scala.collection.immutable.HashMap
import scala.collection.mutable
object SQLGen extends LazyLogging {
type VariableList = List[Var[_]]
trait Joinable {
def +(j: Joinable): Statement
def +(s: String): Statement = this + Raw(s)
}
trait Chunk extends Joinable {
def +(j: Joinable): Statement = j match {
case Statement(list, output) => Statement(list ::: this :: Nil, output)
case c: Chunk => Statement(c :: this :: Nil)
}
def toSQL(fieldMap: Map[ExprId, Attribute]): String
}
case class Statement(list: List[Chunk], output: Seq[AttributeReference] = Nil)
extends Joinable
with LazyLogging {
lazy val reverseList: List[Chunk] = list.reverse
lazy val relations: Seq[Relation] = reverseList.collect {
case r: Relation => r
}
lazy val fieldMap: Map[ExprId, Attribute] = relations
.flatMap(_.output)
.map(a => (a.exprId, a))
.toMap
lazy val variables: VariableList =
reverseList.collect {
case r: Relation => r.reader.variables
case r: Var[_] => Iterator(r)
}.flatten
lazy val sql: String = reverseList.map(_.toSQL(fieldMap)).mkString(" ")
def asLogicalPlan(isFinal: Boolean = false): LogicalPlan =
relations.head.toLogicalPlan(output, sql, variables, isFinal, relations.head.reader.context)
private def newlineIfEmpty: String = list match {
case Nil => ""
case _ => "\n"
}
// ------------------------------------
// Builder functions for easy chaining
// ------------------------------------
def +(j: Joinable): Statement = j match {
case Statement(otherList, _) => copy(otherList ::: list)
case c: Chunk => copy(c :: list)
}
private val MAX_PLAN_FIELDS: Int = Int.MaxValue
def withLogicalPlanComment(plan: LogicalPlan): Statement =
if (log.isTraceEnabled()) {
this + s"${newlineIfEmpty}-- Spark LogicalPlan: ${plan.simpleString(MAX_PLAN_FIELDS).replace("\n", "\n-- ")}"
} else {
this
}
def selectAll(): Statement = this + "\nSELECT *"
def select(c: Joinable): Statement = this + "\nSELECT" + c
def select(c: Option[Joinable]): Statement = this + "\nSELECT" + c.getOrElse(Raw("*"))
def from(c: Joinable): Statement = this + "\nFROM" + c
def join(c: Joinable, joinType: JoinType): Statement =
joinType match {
case Inner => this + "\nINNER JOIN" + c
case Cross => this + "\nCROSS JOIN" + c
case LeftOuter => this + "\nLEFT OUTER JOIN" + c
case RightOuter => this + "\nRIGHT OUTER JOIN" + c
case FullOuter => this + "\nFULL OUTER JOIN" + c
case NaturalJoin(Inner) => this + "\nNATURAL JOIN" + c
case NaturalJoin(LeftOuter) => this + "\nNATURAL LEFT OUTER JOIN" + c
case NaturalJoin(RightOuter) => this + "\nNATURAL RIGHT OUTER JOIN" + c
case NaturalJoin(FullOuter) => this + "\nNATURAL FULL OUTER JOIN" + c
case _ => throw new IllegalArgumentException(s"join type $joinType not supported")
}
def on(c: Joinable): Statement = this + "ON" + c
def on(c: Option[Joinable]): Statement = c.map(on).getOrElse(this)
def where(c: Joinable): Statement = this + "\nWHERE" + c
def groupby(c: Joinable): Statement = this + "\nGROUP BY" + c
def groupby(c: Option[Joinable]): Statement = c.map(groupby).getOrElse(this)
def limit(c: Joinable): Statement = this + "\nLIMIT" + c
def orderby(c: Joinable): Statement = this + "\nORDER BY" + c
def orderby(c: Option[Joinable]): Statement = c.map(orderby).getOrElse(this)
def output(o: Seq[Attribute], updateFromFieldMap: Boolean = true): Statement =
copy(
output = o.map(
f => {
val target = if (updateFromFieldMap) fieldMap.getOrElse(f.exprId, f) else f
AttributeReference(target.name, f.dataType, f.nullable, f.metadata)(
f.exprId,
f.qualifier
)
}
)
)
}
// ----------------------------------
// Primary Chunk Types
// ----------------------------------
trait SQLChunk extends Chunk {
val sql: String
override def toSQL(fieldMap: Map[ExprId, Attribute]): String = sql
}
case class Raw(override val sql: String) extends SQLChunk
case class Ident(name: String) extends SQLChunk {
override val sql: String = SinglestoreDialect.quoteIdentifier(name)
// it's not clear that we ever need to fully-qualify references since we do field renames with expr-ids
// If this changes then you can change this code to something like this:
// (and grab the qualifier when creating Ident)
// qualifier
// .map(q => s"${SinglestoreDialect.quoteIdentifier(q)}.")
// .getOrElse("") + SinglestoreDialect.quoteIdentifier(name)
}
case class Relation(
rawOutput: Seq[Attribute],
reader: SinglestoreReader,
name: String,
toLogicalPlan: (Seq[AttributeReference],
String,
VariableList,
Boolean,
SQLGenContext) => LogicalPlan
) extends SQLChunk {
val isFinal = reader.isFinal
val output = rawOutput.map(
a => AttributeReference(a.name, a.dataType, a.nullable, a.metadata)(a.exprId)
)
override val sql: String = {
var inAttributeName: Boolean = false
// Add indentation after new line character if it is not in the attribute name.
// We are inside of the Attribute if the number of backticks we already processed is odd.
// Example:
// "select id as `name\n\n``name` from \n table" -> "select id as `name\n\n``name` from \n table"
val indentedQuery = reader.query
.map({
case '`' =>
inAttributeName = !inAttributeName
"`"
case '\n' =>
if (inAttributeName) {
"\n"
} else {
"\n "
}
case c => c.toString
})
.mkString
val alias = SinglestoreDialect.quoteIdentifier(name)
s"(\n $indentedQuery\n) AS $alias"
}
def renameOutput: LogicalPlan =
select(
output
.map(a =>
alias(SinglestoreDialect.quoteIdentifier(a.name), a.name, a.exprId, reader.context))
.reduce(_ + "," + _))
.from(this)
.output(output)
.asLogicalPlan()
def castOutputAndFinalize: LogicalPlan = {
val schema = try {
reader.schema
} catch {
case e: Exception => {
log.error(s"Failed to compute schema for reader:\n${reader.toString}")
throw e
}
}
val castedOutputExpr = output
.zip(schema)
.map({
case (a, f) if a.dataType != f.dataType =>
Alias(Cast(a, a.dataType), a.name)(a.exprId, a.qualifier, Some(a.metadata))
case (a, _) => a
})
val expressionExtractor = ExpressionExtractor(reader.context)
select(castedOutputExpr match {
case expressionExtractor(expr) => expr
case _ => None
}).from(this)
.output(output)
.asLogicalPlan(true)
}
}
object Relation {
def unapply(source: LogicalPlan): Option[Relation] = {
source match {
case lr: LogicalRelation if lr.relation.isInstanceOf[SinglestoreReader] => {
val reader = lr.relation.asInstanceOf[SinglestoreReader]
def convertBack(output: Seq[AttributeReference],
sql: String,
variables: VariableList,
isFinal: Boolean,
context: SQLGenContext): LogicalPlan = {
LogicalRelationCompat.copyWithReader(lr,
reader.copy(query = sql,
variables = variables,
isFinal = isFinal,
expectedOutput = output,
context = context))
}
Some(Relation(lr.output, reader, reader.context.nextAlias(), convertBack))
}
case _ => None
}
}
}
// In some versions of Spark, LogicalRelation has an extra `stream`
// argument, while in others it does not. This extractor abstracts away the differences
// and safely retrieves the common fields. It will also extract the `stream` attribute only if it exists
// in the current runtime class, avoiding compile-time errors and runtime crashes.
object LogicalRelationCompat {
def copyWithReader(lr: LogicalRelation, reader: SinglestoreReader): LogicalRelation = {
val cls = lr.getClass
val copies = cls.getMethods.iterator
.filter(m => m.getName == "copy")
.toList
.sortBy(_.getParameterCount)
// pick the most specific/longest copy; we’ll fill defaults for trailing params
val copyM = copies.lastOption.getOrElse {
throw new NoSuchMethodError("LogicalRelation.copy not found.")
}
// The first four params are stable across versions:
// relation: BaseRelation
// output: Seq[AttributeReference]
// catalogTable: Option[CatalogTable]
// isStreaming: Boolean
val baseArgs: Array[AnyRef] = Array(
reader.asInstanceOf[AnyRef],
reader.expectedOutput,
lr.catalogTable.asInstanceOf[AnyRef],
java.lang.Boolean.valueOf(lr.isStreaming)
)
val paramCount = copyM.getParameterCount
val args =
if (paramCount <= 4) baseArgs
else {
// Fill in extra params with their default values: copy$default$5, copy$default$6, ...
val extras = (5 to paramCount).map { i =>
val dm = cls.getMethod(s"copy$$default$$$i")
dm.invoke(lr)
}
baseArgs ++ extras
}
copyM.invoke(lr, args: _*).asInstanceOf[LogicalRelation]
}
}
case class Attr(a: Attribute, context: SQLGenContext) extends Chunk {
override def toSQL(fieldMap: Map[ExprId, Attribute]): String = {
val target = fieldMap.getOrElse(a.exprId, a)
context.ident(target.name, target.exprId)
}
}
// ----------------------------------
// Variables
// ----------------------------------
sealed trait Var[T] extends SQLChunk {
override val sql: String = "?"
val variable: T
}
case class StringVar(override val variable: String) extends Var[String]
case class IntVar(override val variable: Int) extends Var[Int]
case class LongVar(override val variable: Long) extends Var[Long]
case class ShortVar(override val variable: Short) extends Var[Short]
case class FloatVar(override val variable: Float) extends Var[Float]
case class DoubleVar(override val variable: Double) extends Var[Double]
case class DecimalVar(override val variable: Decimal) extends Var[Decimal]
case class BooleanVar(override val variable: Boolean) extends Var[Boolean]
case class ByteVar(override val variable: Byte) extends Var[Byte]
case class DateVar(override val variable: Date) extends Var[Date]
case class TimestampVar(override val variable: Timestamp) extends Var[Timestamp]
// ----------------------------------
// Builder functions and constants
// ----------------------------------
final val empty: Statement = Statement(Nil)
implicit def stringToJoinable(s: String): Joinable = Raw(s)
def block(j: Joinable): Statement = Raw("(") + j + ")"
def alias(j: Joinable, n: String, e: ExprId, context: SQLGenContext): Statement =
block(j) + "AS" + context.ident(n, e)
def func(n: String, j: Joinable): Statement = Raw(n) + block(j)
def func(n: String, j: Joinable*): Statement = Raw(n) + block(j.reduce(_ + "," + _))
def cast(j: Joinable, t: Joinable): Statement = func("CONVERT", j, t)
def newStatement(sourcePlan: LogicalPlan): Statement = empty.withLogicalPlanComment(sourcePlan)
def selectAll: Statement = Statement(Raw("SELECT *") :: Nil)
def select(c: Joinable): Statement = Raw("SELECT") + c
def select(c: Option[Joinable]): Statement = Raw("SELECT") + c.getOrElse(Raw("*"))
def sqlMapValueCaseInsensitive(value: Joinable,
mappings: Map[String, String],
default: Joinable): Joinable =
value match {
case StringVar(s) => mappings.get(s.toLowerCase).map(StringVar).getOrElse(default)
case _ =>
block(mappings.foldLeft(Raw("CASE") + func("LOWER", value))({
case (memo, (key, value)) =>
memo + "WHEN" + StringVar(key.toLowerCase) + "THEN" + StringVar(value.toLowerCase)
}) + "ELSE" + default + "END")
}
case class SortPredicates(expressionExtractor: ExpressionExtractor) {
def joinPredicates(predicates: Seq[Option[Joinable]], operation: String): Option[Joinable] = {
predicates
.sortWith((p1, p2) => p1.toString < p2.toString)
.reduce[Option[Joinable]] {
case (Some(left), Some(right)) => Some(ExpressionGen.op(operation, left, right))
case _ => None
}
}
def extractOr(expr: Expression): Seq[Option[Joinable]] = expr match {
case Or(left, right) => extractOr(left) ++ extractOr(right)
case And(left, right) => Seq(joinPredicates(extractAnd(left) ++ extractAnd(right), "AND"))
case expressionExtractor(expr) => Seq(Some(expr))
case _ => Seq(None)
}
def extractAnd(expr: Expression): Seq[Option[Joinable]] = expr match {
case And(left, right) => extractAnd(left) ++ extractAnd(right)
case Or(left, right) => Seq(joinPredicates(extractOr(left) ++ extractOr(right), "OR"))
case expressionExtractor(expr) => Seq(Some(expr))
case _ => Seq(None)
}
def unapply(expr: Expression): Option[Joinable] = {
joinPredicates(extractAnd(expr), "AND")
}
// None -> Some(None) nothing to compile results in no SQL
// Some(good expr) -> Some(Some(sql)) we can compile, results in SQL
// Some(bad expr) -> None failed to compile, unapply does not match
def unapply(expr: Option[Expression]): Option[Option[Joinable]] = expr match {
case None => Some(None)
case Some(expression) => joinPredicates(extractAnd(expression), "AND").map(j => Some(j))
}
}
case class StatementWithOrder(expressionExtractor: ExpressionExtractor) {
def unapply(source: LogicalPlan): Option[(LogicalPlan, Seq[SortOrder])] = {
val limitWithOrder = LimitWithOrder(expressionExtractor)
source match {
case plan @ VersionSpecificSortExtractor(order @ expressionExtractor(expr),
true,
Relation(relation)) =>
Some(
newStatement(plan)
.selectAll()
.from(relation)
.orderby(expr)
// For now - we add a huge limit to all sort queries which forces SingleStore to preserve the order by.
// fromTopLevelSort and fromLimit will handle pushing down sort without a max-int limit.
.limit(Long.MaxValue.toString)
.output(plan.output)
.asLogicalPlan(),
order
)
case limitWithOrder(logicalPlan, order) => Some(logicalPlan, order)
case _ => None
}
}
}
case class LimitWithOrder(expressionExtractor: ExpressionExtractor) {
def unapply(source: LogicalPlan): Option[(LogicalPlan, Seq[SortOrder])] = {
source match {
case plan @ Limit(expressionExtractor(limitExpr),
innerPlan @ VersionSpecificSortExtractor(order @ expressionExtractor(
sortExpr),
true,
Relation(relation))) =>
Some(
newStatement(plan)
.withLogicalPlanComment(innerPlan)
.selectAll()
.from(relation)
.orderby(sortExpr)
.limit(limitExpr)
.output(plan.output)
.asLogicalPlan(),
order
)
case _ => None
}
}
}
case class RelationOrSort(expressionExtractor: ExpressionExtractor) {
def unapply(source: LogicalPlan): Option[Relation] = {
val statementWithOrder = StatementWithOrder(expressionExtractor)
source match {
case Relation(relation) => Some(relation)
case statementWithOrder(Relation(relation), _) => Some(relation)
case _ => None
}
}
}
def fromLogicalPlan(
expressionExtractor: ExpressionExtractor): PartialFunction[LogicalPlan, Statement] = {
val sortPredicates = SortPredicates(expressionExtractor)
val relationOrSort = RelationOrSort(expressionExtractor)
return {
case plan @ Project(expressionExtractor(expr), Relation(relation)) =>
newStatement(plan)
.select(expr)
.from(relation)
.output(plan.output)
case plan @ Limit(expressionExtractor(expr), Relation(relation)) =>
newStatement(plan)
.selectAll()
.from(relation)
.limit(expr)
.output(plan.output)
case plan @ Filter(sortPredicates(filter), relationOrSort(relation)) => {
newStatement(plan)
.selectAll()
.from(relation)
.where(filter)
.output(plan.output)
}
case plan @ VersionSpecificAggregateExtractor(expressionExtractor(groupingExpr),
expressionExtractor(aggregateExpr),
relationOrSort(relation)) =>
newStatement(plan)
.select(aggregateExpr)
.from(relation)
.groupby(groupingExpr)
.output(plan.output)
case plan @ VersionSpecificWindowExtractor(expressionExtractor(windowExpressions),
_,
_,
relationOrSort(relation)) => {
newStatement(plan)
.select(windowExpressions.map(exp => Raw("*,") + exp))
.from(relation)
.output(plan.output)
}
// the last parameter is a spark hint for join
// SingleStore does its own optimizations under the hood, so we can safely ignore this parameter
case plan @ Join(relationOrSort(left),
relationOrSort(right),
joinType @ (Inner | Cross),
sortPredicates(condition),
_)
if getDMLConnProperties(left.reader.options, isOnExecutor = false) == getDMLConnProperties(
right.reader.options,
isOnExecutor = false) =>
newStatement(plan)
.selectAll()
.from(left)
.join(right, joinType)
.on(condition)
.output(plan.output)
// condition is required for {Left, Right, Full} outer joins
// the last parameter is a spark hint for join
// SingleStore does its own optimizations under the hood, so we can safely ignore this parameter
case plan @ Join(relationOrSort(left),
relationOrSort(right),
joinType @ (LeftOuter | RightOuter | FullOuter),
Some(sortPredicates(condition)),
_)
if getDMLConnProperties(left.reader.options, isOnExecutor = false) == getDMLConnProperties(
right.reader.options,
isOnExecutor = false) =>
newStatement(plan)
.selectAll()
.from(left)
.join(right, joinType)
.on(condition)
.output(plan.output)
// condition is not allowed for natural joins
// the last parameter is a spark hint for join
// SingleStore does its own optimizations under the hood, so we can safely ignore this parameter
case plan @ Join(relationOrSort(left), relationOrSort(right), NaturalJoin(joinType), None, _)
if getDMLConnProperties(left.reader.options, isOnExecutor = false) == getDMLConnProperties(
right.reader.options,
isOnExecutor = false) =>
newStatement(plan)
.selectAll()
.from(left)
.join(right, joinType)
.output(plan.output)
}
}
def fromTopLevelSort(
expressionExtractor: ExpressionExtractor): PartialFunction[LogicalPlan, LogicalPlan] = {
val statementWithOrder = StatementWithOrder(expressionExtractor)
val limitWithOrder = LimitWithOrder(expressionExtractor)
return {
// for Disabled and AutomaticLite option do pushdown of the top-level sort expression
// parallel read won't be done in this case
case statementWithOrder(plan @ Relation(relation), _)
if relation.reader.options.enableParallelRead == Disabled ||
relation.reader.options.enableParallelRead == AutomaticLite => {
relation.reader.resultMustBeSorted = true
plan
}
// for Automatic and Forced option pushdown sort with limit but add top-level sort
// which will be done on a spark side
case limitWithOrder(plan @ Relation(relation), order)
if relation.reader.options.enableParallelRead == Automatic ||
relation.reader.options.enableParallelRead == Forced =>
Sort(order, global = true, plan)
}
}
// SQLGenContext is used to generate aliases during the codegen
// normalizedExprIdMap is a map from ExprId to its normalized index
// It is needed to make generated SQL queries deterministic
case class SQLGenContext(normalizedExprIdMap: HashMap[ExprId, Int],
singlestoreVersion: SinglestoreVersion) {
val aliasGen: Iterator[String] = Iterator.from(1).map(i => s"a$i")
def nextAlias(): String = aliasGen.next()
def singlestoreVersionAtLeast(version: String): Boolean =
singlestoreVersion.atLeast(version)
def ident(name: String, exprId: ExprId): String =
if (normalizedExprIdMap.contains(exprId)) {
Ident(s"${name.substring(0, Math.min(name.length, 10))}#${normalizedExprIdMap(exprId)}").sql
} else {
Ident(s"${name.substring(0, Math.min(name.length, 10))}#${exprId.id}").sql
}
}
object SQLGenContext {
var singlestoreVersion: Option[String] = None
private def getSinglestoreVersion(options: SinglestoreOptions): SinglestoreVersion =
singlestoreVersion match {
case Some(version) => SinglestoreVersion(version)
case None =>
singlestoreVersion = Some(JdbcHelpers.getSinglestoreVersion(options))
SinglestoreVersion(singlestoreVersion.get)
}
def apply(root: LogicalPlan, options: SinglestoreOptions): SQLGenContext = {
var normalizedExprIdMap = scala.collection.immutable.HashMap[ExprId, Int]()
val nextId = Iterator.from(1)
root.foreach(plan =>
plan.output.foreach(f => {
if (!normalizedExprIdMap.contains(f.exprId)) {
normalizedExprIdMap = normalizedExprIdMap + (f.exprId -> nextId.next())
}
}))
new SQLGenContext(normalizedExprIdMap, getSinglestoreVersion(options))
}
def apply(options: SinglestoreOptions): SQLGenContext =
new SQLGenContext(HashMap.empty, getSinglestoreVersion(options))
}
case class SinglestoreVersion(major: Int, minor: Int, patch: Int) {
implicit val ordering: Ordering[SinglestoreVersion] =
Ordering.by(v => (v.major, v.minor, v.patch))
import Ordering.Implicits.infixOrderingOps
def atLeast(version: SinglestoreVersion): Boolean = {
this >= version
}
def atLeast(version: String): Boolean = {
atLeast(SinglestoreVersion(version))
}
override def toString: String = s"${this.major}.${this.minor}.${this.patch}"
}
object SinglestoreVersion {
def apply(version: String): SinglestoreVersion = {
val versionParts = version.split("\\.")
if (versionParts.size != 3)
throw new IllegalArgumentException(
"Singlestore version should contain three parts (major, minor, patch)")
new SinglestoreVersion(Integer.parseInt(versionParts(0)),
Integer.parseInt(versionParts(1)),
Integer.parseInt(versionParts(2)))
}
}
case class ExpressionExtractor(context: SQLGenContext) {
protected lazy val log: Logger = LoggerFactory.getLogger(getClass.getName)
def unapply(arg: Expression): Option[Joinable] = {
val out = ExpressionGen.apply(this).lift(arg)
if (out.isEmpty && log.isTraceEnabled) {
val argStr: String = try {
arg.asCode
} catch {
case e: NullPointerException =>
s"${arg.prettyName} (failed to convert expression to string)"
}
log.trace(s"Warning: SingleStore SQL pushdown was unable to convert expression: $argStr")
}
out
}
// None -> Some(None) nothing to compile results in no SQL
// Some(good expr) -> Some(Some(sql)) we can compile, results in SQL
// Some(bad expr) -> None failed to compile, unapply does not match
def unapply(arg: Option[Expression]): Option[Option[Joinable]] = arg match {
case None => Some(None)
case Some(expression) => ExpressionGen.apply(this).lift(expression).map(j => Some(j))
}
// Seq() -> Some(None) nothing to compile results in no SQL
// Seq(good expressions) -> Some(Some(sql)) we can compile, results in SQL
// Seq(at least one bad expression) -> None failed to compile, unapply does not match
def unapply(args: Seq[Expression]): Option[Option[Joinable]] = {
if (args.isEmpty) {
Some(None)
} else {
// TODO: PLAT-4670 check how this can be improved to enable pushdown for queries like "SELECT CONCAT(first_name, first_name) FROM users"
if (args.lengthCompare(1) > 0) {
val expressionNames = new mutable.HashSet[String]()
val hasDuplicates = args.exists({
case a @ NamedExpression(name, _) => !expressionNames.add(s"${name}#${a.exprId.id}")
case _ => false
})
if (hasDuplicates) return None
}
args
.map(ExpressionGen.apply(this).lift)
.reduce[Option[Joinable]] {
case (Some(left), Some(right)) => Some(left + "," + right)
case _ => None
}
.map(j => Some(j))
}
}
}
}