sqlglot.dialects.clickhouse
1from __future__ import annotations 2 3import typing as t 4 5from sqlglot import exp, generator, parser, tokens 6from sqlglot.dialects.dialect import ( 7 Dialect, 8 NormalizationStrategy, 9 arg_max_or_min_no_count, 10 build_date_delta, 11 build_formatted_time, 12 inline_array_sql, 13 json_extract_segments, 14 json_path_key_only_name, 15 no_pivot_sql, 16 build_json_extract_path, 17 rename_func, 18 sha256_sql, 19 var_map_sql, 20 timestamptrunc_sql, 21 unit_to_var, 22) 23from sqlglot.generator import Generator 24from sqlglot.helper import is_int, seq_get 25from sqlglot.tokens import Token, TokenType 26 27DATEΤΙΜΕ_DELTA = t.Union[exp.DateAdd, exp.DateDiff, exp.DateSub, exp.TimestampSub, exp.TimestampAdd] 28 29 30def _build_date_format(args: t.List) -> exp.TimeToStr: 31 expr = build_formatted_time(exp.TimeToStr, "clickhouse")(args) 32 33 timezone = seq_get(args, 2) 34 if timezone: 35 expr.set("timezone", timezone) 36 37 return expr 38 39 40def _unix_to_time_sql(self: ClickHouse.Generator, expression: exp.UnixToTime) -> str: 41 scale = expression.args.get("scale") 42 timestamp = expression.this 43 44 if scale in (None, exp.UnixToTime.SECONDS): 45 return self.func("fromUnixTimestamp", exp.cast(timestamp, exp.DataType.Type.BIGINT)) 46 if scale == exp.UnixToTime.MILLIS: 47 return self.func("fromUnixTimestamp64Milli", exp.cast(timestamp, exp.DataType.Type.BIGINT)) 48 if scale == exp.UnixToTime.MICROS: 49 return self.func("fromUnixTimestamp64Micro", exp.cast(timestamp, exp.DataType.Type.BIGINT)) 50 if scale == exp.UnixToTime.NANOS: 51 return self.func("fromUnixTimestamp64Nano", exp.cast(timestamp, exp.DataType.Type.BIGINT)) 52 53 return self.func( 54 "fromUnixTimestamp", 55 exp.cast( 56 exp.Div(this=timestamp, expression=exp.func("POW", 10, scale)), exp.DataType.Type.BIGINT 57 ), 58 ) 59 60 61def _lower_func(sql: str) -> str: 62 index = sql.index("(") 63 return sql[:index].lower() + sql[index:] 64 65 66def _quantile_sql(self: ClickHouse.Generator, expression: exp.Quantile) -> str: 67 quantile = expression.args["quantile"] 68 args = f"({self.sql(expression, 'this')})" 69 70 if isinstance(quantile, exp.Array): 71 func = self.func("quantiles", *quantile) 72 else: 73 func = self.func("quantile", quantile) 74 75 return func + args 76 77 78def _build_count_if(args: t.List) -> exp.CountIf | exp.CombinedAggFunc: 79 if len(args) == 1: 80 return exp.CountIf(this=seq_get(args, 0)) 81 82 return exp.CombinedAggFunc(this="countIf", expressions=args, parts=("count", "If")) 83 84 85def _build_str_to_date(args: t.List) -> exp.Cast | exp.Anonymous: 86 if len(args) == 3: 87 return exp.Anonymous(this="STR_TO_DATE", expressions=args) 88 89 strtodate = exp.StrToDate.from_arg_list(args) 90 return exp.cast(strtodate, exp.DataType.build(exp.DataType.Type.DATETIME)) 91 92 93def _datetime_delta_sql(name: str) -> t.Callable[[Generator, DATEΤΙΜΕ_DELTA], str]: 94 def _delta_sql(self: Generator, expression: DATEΤΙΜΕ_DELTA) -> str: 95 if not expression.unit: 96 return rename_func(name)(self, expression) 97 98 return self.func( 99 name, 100 unit_to_var(expression), 101 expression.expression, 102 expression.this, 103 ) 104 105 return _delta_sql 106 107 108class ClickHouse(Dialect): 109 NORMALIZE_FUNCTIONS: bool | str = False 110 NULL_ORDERING = "nulls_are_last" 111 SUPPORTS_USER_DEFINED_TYPES = False 112 SAFE_DIVISION = True 113 LOG_BASE_FIRST: t.Optional[bool] = None 114 FORCE_EARLY_ALIAS_REF_EXPANSION = True 115 116 # https://github.com/ClickHouse/ClickHouse/issues/33935#issue-1112165779 117 NORMALIZATION_STRATEGY = NormalizationStrategy.CASE_SENSITIVE 118 119 UNESCAPED_SEQUENCES = { 120 "\\0": "\0", 121 } 122 123 CREATABLE_KIND_MAPPING = {"DATABASE": "SCHEMA"} 124 125 class Tokenizer(tokens.Tokenizer): 126 COMMENTS = ["--", "#", "#!", ("/*", "*/")] 127 IDENTIFIERS = ['"', "`"] 128 STRING_ESCAPES = ["'", "\\"] 129 BIT_STRINGS = [("0b", "")] 130 HEX_STRINGS = [("0x", ""), ("0X", "")] 131 HEREDOC_STRINGS = ["$"] 132 133 KEYWORDS = { 134 **tokens.Tokenizer.KEYWORDS, 135 "ATTACH": TokenType.COMMAND, 136 "DATE32": TokenType.DATE32, 137 "DATETIME64": TokenType.DATETIME64, 138 "DICTIONARY": TokenType.DICTIONARY, 139 "ENUM8": TokenType.ENUM8, 140 "ENUM16": TokenType.ENUM16, 141 "FINAL": TokenType.FINAL, 142 "FIXEDSTRING": TokenType.FIXEDSTRING, 143 "FLOAT32": TokenType.FLOAT, 144 "FLOAT64": TokenType.DOUBLE, 145 "GLOBAL": TokenType.GLOBAL, 146 "INT256": TokenType.INT256, 147 "LOWCARDINALITY": TokenType.LOWCARDINALITY, 148 "MAP": TokenType.MAP, 149 "NESTED": TokenType.NESTED, 150 "SAMPLE": TokenType.TABLE_SAMPLE, 151 "TUPLE": TokenType.STRUCT, 152 "UINT128": TokenType.UINT128, 153 "UINT16": TokenType.USMALLINT, 154 "UINT256": TokenType.UINT256, 155 "UINT32": TokenType.UINT, 156 "UINT64": TokenType.UBIGINT, 157 "UINT8": TokenType.UTINYINT, 158 "IPV4": TokenType.IPV4, 159 "IPV6": TokenType.IPV6, 160 "AGGREGATEFUNCTION": TokenType.AGGREGATEFUNCTION, 161 "SIMPLEAGGREGATEFUNCTION": TokenType.SIMPLEAGGREGATEFUNCTION, 162 "SYSTEM": TokenType.COMMAND, 163 "PREWHERE": TokenType.PREWHERE, 164 } 165 KEYWORDS.pop("/*+") 166 167 SINGLE_TOKENS = { 168 **tokens.Tokenizer.SINGLE_TOKENS, 169 "$": TokenType.HEREDOC_STRING, 170 } 171 172 class Parser(parser.Parser): 173 # Tested in ClickHouse's playground, it seems that the following two queries do the same thing 174 # * select x from t1 union all select x from t2 limit 1; 175 # * select x from t1 union all (select x from t2 limit 1); 176 MODIFIERS_ATTACHED_TO_SET_OP = False 177 INTERVAL_SPANS = False 178 179 FUNCTIONS = { 180 **parser.Parser.FUNCTIONS, 181 "ANY": exp.AnyValue.from_arg_list, 182 "ARRAYSUM": exp.ArraySum.from_arg_list, 183 "COUNTIF": _build_count_if, 184 "DATE_ADD": build_date_delta(exp.DateAdd, default_unit=None), 185 "DATEADD": build_date_delta(exp.DateAdd, default_unit=None), 186 "DATE_DIFF": build_date_delta(exp.DateDiff, default_unit=None), 187 "DATEDIFF": build_date_delta(exp.DateDiff, default_unit=None), 188 "DATE_FORMAT": _build_date_format, 189 "DATE_SUB": build_date_delta(exp.DateSub, default_unit=None), 190 "DATESUB": build_date_delta(exp.DateSub, default_unit=None), 191 "FORMATDATETIME": _build_date_format, 192 "JSONEXTRACTSTRING": build_json_extract_path( 193 exp.JSONExtractScalar, zero_based_indexing=False 194 ), 195 "MAP": parser.build_var_map, 196 "MATCH": exp.RegexpLike.from_arg_list, 197 "RANDCANONICAL": exp.Rand.from_arg_list, 198 "STR_TO_DATE": _build_str_to_date, 199 "TUPLE": exp.Struct.from_arg_list, 200 "TIMESTAMP_SUB": build_date_delta(exp.TimestampSub, default_unit=None), 201 "TIMESTAMPSUB": build_date_delta(exp.TimestampSub, default_unit=None), 202 "TIMESTAMP_ADD": build_date_delta(exp.TimestampAdd, default_unit=None), 203 "TIMESTAMPADD": build_date_delta(exp.TimestampAdd, default_unit=None), 204 "UNIQ": exp.ApproxDistinct.from_arg_list, 205 "XOR": lambda args: exp.Xor(expressions=args), 206 "MD5": exp.MD5Digest.from_arg_list, 207 "SHA256": lambda args: exp.SHA2(this=seq_get(args, 0), length=exp.Literal.number(256)), 208 "SHA512": lambda args: exp.SHA2(this=seq_get(args, 0), length=exp.Literal.number(512)), 209 } 210 211 AGG_FUNCTIONS = { 212 "count", 213 "min", 214 "max", 215 "sum", 216 "avg", 217 "any", 218 "stddevPop", 219 "stddevSamp", 220 "varPop", 221 "varSamp", 222 "corr", 223 "covarPop", 224 "covarSamp", 225 "entropy", 226 "exponentialMovingAverage", 227 "intervalLengthSum", 228 "kolmogorovSmirnovTest", 229 "mannWhitneyUTest", 230 "median", 231 "rankCorr", 232 "sumKahan", 233 "studentTTest", 234 "welchTTest", 235 "anyHeavy", 236 "anyLast", 237 "boundingRatio", 238 "first_value", 239 "last_value", 240 "argMin", 241 "argMax", 242 "avgWeighted", 243 "topK", 244 "topKWeighted", 245 "deltaSum", 246 "deltaSumTimestamp", 247 "groupArray", 248 "groupArrayLast", 249 "groupUniqArray", 250 "groupArrayInsertAt", 251 "groupArrayMovingAvg", 252 "groupArrayMovingSum", 253 "groupArraySample", 254 "groupBitAnd", 255 "groupBitOr", 256 "groupBitXor", 257 "groupBitmap", 258 "groupBitmapAnd", 259 "groupBitmapOr", 260 "groupBitmapXor", 261 "sumWithOverflow", 262 "sumMap", 263 "minMap", 264 "maxMap", 265 "skewSamp", 266 "skewPop", 267 "kurtSamp", 268 "kurtPop", 269 "uniq", 270 "uniqExact", 271 "uniqCombined", 272 "uniqCombined64", 273 "uniqHLL12", 274 "uniqTheta", 275 "quantile", 276 "quantiles", 277 "quantileExact", 278 "quantilesExact", 279 "quantileExactLow", 280 "quantilesExactLow", 281 "quantileExactHigh", 282 "quantilesExactHigh", 283 "quantileExactWeighted", 284 "quantilesExactWeighted", 285 "quantileTiming", 286 "quantilesTiming", 287 "quantileTimingWeighted", 288 "quantilesTimingWeighted", 289 "quantileDeterministic", 290 "quantilesDeterministic", 291 "quantileTDigest", 292 "quantilesTDigest", 293 "quantileTDigestWeighted", 294 "quantilesTDigestWeighted", 295 "quantileBFloat16", 296 "quantilesBFloat16", 297 "quantileBFloat16Weighted", 298 "quantilesBFloat16Weighted", 299 "simpleLinearRegression", 300 "stochasticLinearRegression", 301 "stochasticLogisticRegression", 302 "categoricalInformationValue", 303 "contingency", 304 "cramersV", 305 "cramersVBiasCorrected", 306 "theilsU", 307 "maxIntersections", 308 "maxIntersectionsPosition", 309 "meanZTest", 310 "quantileInterpolatedWeighted", 311 "quantilesInterpolatedWeighted", 312 "quantileGK", 313 "quantilesGK", 314 "sparkBar", 315 "sumCount", 316 "largestTriangleThreeBuckets", 317 "histogram", 318 "sequenceMatch", 319 "sequenceCount", 320 "windowFunnel", 321 "retention", 322 "uniqUpTo", 323 "sequenceNextNode", 324 "exponentialTimeDecayedAvg", 325 } 326 327 AGG_FUNCTIONS_SUFFIXES = [ 328 "If", 329 "Array", 330 "ArrayIf", 331 "Map", 332 "SimpleState", 333 "State", 334 "Merge", 335 "MergeState", 336 "ForEach", 337 "Distinct", 338 "OrDefault", 339 "OrNull", 340 "Resample", 341 "ArgMin", 342 "ArgMax", 343 ] 344 345 FUNC_TOKENS = { 346 *parser.Parser.FUNC_TOKENS, 347 TokenType.SET, 348 } 349 350 RESERVED_TOKENS = parser.Parser.RESERVED_TOKENS - {TokenType.SELECT} 351 352 ID_VAR_TOKENS = { 353 *parser.Parser.ID_VAR_TOKENS, 354 TokenType.LIKE, 355 } 356 357 AGG_FUNC_MAPPING = ( 358 lambda functions, suffixes: { 359 f"{f}{sfx}": (f, sfx) for sfx in (suffixes + [""]) for f in functions 360 } 361 )(AGG_FUNCTIONS, AGG_FUNCTIONS_SUFFIXES) 362 363 FUNCTIONS_WITH_ALIASED_ARGS = {*parser.Parser.FUNCTIONS_WITH_ALIASED_ARGS, "TUPLE"} 364 365 FUNCTION_PARSERS = { 366 **parser.Parser.FUNCTION_PARSERS, 367 "ARRAYJOIN": lambda self: self.expression(exp.Explode, this=self._parse_expression()), 368 "QUANTILE": lambda self: self._parse_quantile(), 369 } 370 371 FUNCTION_PARSERS.pop("MATCH") 372 373 NO_PAREN_FUNCTION_PARSERS = parser.Parser.NO_PAREN_FUNCTION_PARSERS.copy() 374 NO_PAREN_FUNCTION_PARSERS.pop("ANY") 375 376 RANGE_PARSERS = { 377 **parser.Parser.RANGE_PARSERS, 378 TokenType.GLOBAL: lambda self, this: self._match(TokenType.IN) 379 and self._parse_in(this, is_global=True), 380 } 381 382 # The PLACEHOLDER entry is popped because 1) it doesn't affect Clickhouse (it corresponds to 383 # the postgres-specific JSONBContains parser) and 2) it makes parsing the ternary op simpler. 384 COLUMN_OPERATORS = parser.Parser.COLUMN_OPERATORS.copy() 385 COLUMN_OPERATORS.pop(TokenType.PLACEHOLDER) 386 387 JOIN_KINDS = { 388 *parser.Parser.JOIN_KINDS, 389 TokenType.ANY, 390 TokenType.ASOF, 391 TokenType.ARRAY, 392 } 393 394 TABLE_ALIAS_TOKENS = parser.Parser.TABLE_ALIAS_TOKENS - { 395 TokenType.ANY, 396 TokenType.ARRAY, 397 TokenType.FINAL, 398 TokenType.FORMAT, 399 TokenType.SETTINGS, 400 } 401 402 ALIAS_TOKENS = parser.Parser.ALIAS_TOKENS - { 403 TokenType.FORMAT, 404 } 405 406 LOG_DEFAULTS_TO_LN = True 407 408 QUERY_MODIFIER_PARSERS = { 409 **parser.Parser.QUERY_MODIFIER_PARSERS, 410 TokenType.SETTINGS: lambda self: ( 411 "settings", 412 self._advance() or self._parse_csv(self._parse_assignment), 413 ), 414 TokenType.FORMAT: lambda self: ("format", self._advance() or self._parse_id_var()), 415 } 416 417 CONSTRAINT_PARSERS = { 418 **parser.Parser.CONSTRAINT_PARSERS, 419 "INDEX": lambda self: self._parse_index_constraint(), 420 "CODEC": lambda self: self._parse_compress(), 421 } 422 423 ALTER_PARSERS = { 424 **parser.Parser.ALTER_PARSERS, 425 "REPLACE": lambda self: self._parse_alter_table_replace(), 426 } 427 428 SCHEMA_UNNAMED_CONSTRAINTS = { 429 *parser.Parser.SCHEMA_UNNAMED_CONSTRAINTS, 430 "INDEX", 431 } 432 433 def _parse_types( 434 self, check_func: bool = False, schema: bool = False, allow_identifiers: bool = True 435 ) -> t.Optional[exp.Expression]: 436 dtype = super()._parse_types( 437 check_func=check_func, schema=schema, allow_identifiers=allow_identifiers 438 ) 439 if isinstance(dtype, exp.DataType): 440 # Mark every type as non-nullable which is ClickHouse's default. This marker 441 # helps us transpile types from other dialects to ClickHouse, so that we can 442 # e.g. produce `CAST(x AS Nullable(String))` from `CAST(x AS TEXT)`. If there 443 # is a `NULL` value in `x`, the former would fail in ClickHouse without the 444 # `Nullable` type constructor 445 dtype.set("nullable", False) 446 447 return dtype 448 449 def _parse_extract(self) -> exp.Extract | exp.Anonymous: 450 index = self._index 451 this = self._parse_bitwise() 452 if self._match(TokenType.FROM): 453 self._retreat(index) 454 return super()._parse_extract() 455 456 # We return Anonymous here because extract and regexpExtract have different semantics, 457 # so parsing extract(foo, bar) into RegexpExtract can potentially break queries. E.g., 458 # `extract('foobar', 'b')` works, but ClickHouse crashes for `regexpExtract('foobar', 'b')`. 459 # 460 # TODO: can we somehow convert the former into an equivalent `regexpExtract` call? 461 self._match(TokenType.COMMA) 462 return self.expression( 463 exp.Anonymous, this="extract", expressions=[this, self._parse_bitwise()] 464 ) 465 466 def _parse_assignment(self) -> t.Optional[exp.Expression]: 467 this = super()._parse_assignment() 468 469 if self._match(TokenType.PLACEHOLDER): 470 return self.expression( 471 exp.If, 472 this=this, 473 true=self._parse_assignment(), 474 false=self._match(TokenType.COLON) and self._parse_assignment(), 475 ) 476 477 return this 478 479 def _parse_placeholder(self) -> t.Optional[exp.Expression]: 480 """ 481 Parse a placeholder expression like SELECT {abc: UInt32} or FROM {table: Identifier} 482 https://clickhouse.com/docs/en/sql-reference/syntax#defining-and-using-query-parameters 483 """ 484 if not self._match(TokenType.L_BRACE): 485 return None 486 487 this = self._parse_id_var() 488 self._match(TokenType.COLON) 489 kind = self._parse_types(check_func=False, allow_identifiers=False) or ( 490 self._match_text_seq("IDENTIFIER") and "Identifier" 491 ) 492 493 if not kind: 494 self.raise_error("Expecting a placeholder type or 'Identifier' for tables") 495 elif not self._match(TokenType.R_BRACE): 496 self.raise_error("Expecting }") 497 498 return self.expression(exp.Placeholder, this=this, kind=kind) 499 500 def _parse_in(self, this: t.Optional[exp.Expression], is_global: bool = False) -> exp.In: 501 this = super()._parse_in(this) 502 this.set("is_global", is_global) 503 return this 504 505 def _parse_table( 506 self, 507 schema: bool = False, 508 joins: bool = False, 509 alias_tokens: t.Optional[t.Collection[TokenType]] = None, 510 parse_bracket: bool = False, 511 is_db_reference: bool = False, 512 parse_partition: bool = False, 513 ) -> t.Optional[exp.Expression]: 514 this = super()._parse_table( 515 schema=schema, 516 joins=joins, 517 alias_tokens=alias_tokens, 518 parse_bracket=parse_bracket, 519 is_db_reference=is_db_reference, 520 ) 521 522 if self._match(TokenType.FINAL): 523 this = self.expression(exp.Final, this=this) 524 525 return this 526 527 def _parse_position(self, haystack_first: bool = False) -> exp.StrPosition: 528 return super()._parse_position(haystack_first=True) 529 530 # https://clickhouse.com/docs/en/sql-reference/statements/select/with/ 531 def _parse_cte(self) -> exp.CTE: 532 # WITH <identifier> AS <subquery expression> 533 cte: t.Optional[exp.CTE] = self._try_parse(super()._parse_cte) 534 535 if not cte: 536 # WITH <expression> AS <identifier> 537 cte = self.expression( 538 exp.CTE, 539 this=self._parse_assignment(), 540 alias=self._parse_table_alias(), 541 scalar=True, 542 ) 543 544 return cte 545 546 def _parse_join_parts( 547 self, 548 ) -> t.Tuple[t.Optional[Token], t.Optional[Token], t.Optional[Token]]: 549 is_global = self._match(TokenType.GLOBAL) and self._prev 550 kind_pre = self._match_set(self.JOIN_KINDS, advance=False) and self._prev 551 552 if kind_pre: 553 kind = self._match_set(self.JOIN_KINDS) and self._prev 554 side = self._match_set(self.JOIN_SIDES) and self._prev 555 return is_global, side, kind 556 557 return ( 558 is_global, 559 self._match_set(self.JOIN_SIDES) and self._prev, 560 self._match_set(self.JOIN_KINDS) and self._prev, 561 ) 562 563 def _parse_join( 564 self, skip_join_token: bool = False, parse_bracket: bool = False 565 ) -> t.Optional[exp.Join]: 566 join = super()._parse_join(skip_join_token=skip_join_token, parse_bracket=True) 567 if join: 568 join.set("global", join.args.pop("method", None)) 569 570 return join 571 572 def _parse_function( 573 self, 574 functions: t.Optional[t.Dict[str, t.Callable]] = None, 575 anonymous: bool = False, 576 optional_parens: bool = True, 577 any_token: bool = False, 578 ) -> t.Optional[exp.Expression]: 579 expr = super()._parse_function( 580 functions=functions, 581 anonymous=anonymous, 582 optional_parens=optional_parens, 583 any_token=any_token, 584 ) 585 586 func = expr.this if isinstance(expr, exp.Window) else expr 587 588 # Aggregate functions can be split in 2 parts: <func_name><suffix> 589 parts = ( 590 self.AGG_FUNC_MAPPING.get(func.this) if isinstance(func, exp.Anonymous) else None 591 ) 592 593 if parts: 594 params = self._parse_func_params(func) 595 596 kwargs = { 597 "this": func.this, 598 "expressions": func.expressions, 599 } 600 if parts[1]: 601 kwargs["parts"] = parts 602 exp_class = exp.CombinedParameterizedAgg if params else exp.CombinedAggFunc 603 else: 604 exp_class = exp.ParameterizedAgg if params else exp.AnonymousAggFunc 605 606 kwargs["exp_class"] = exp_class 607 if params: 608 kwargs["params"] = params 609 610 func = self.expression(**kwargs) 611 612 if isinstance(expr, exp.Window): 613 # The window's func was parsed as Anonymous in base parser, fix its 614 # type to be ClickHouse style CombinedAnonymousAggFunc / AnonymousAggFunc 615 expr.set("this", func) 616 elif params: 617 # Params have blocked super()._parse_function() from parsing the following window 618 # (if that exists) as they're standing between the function call and the window spec 619 expr = self._parse_window(func) 620 else: 621 expr = func 622 623 return expr 624 625 def _parse_func_params( 626 self, this: t.Optional[exp.Func] = None 627 ) -> t.Optional[t.List[exp.Expression]]: 628 if self._match_pair(TokenType.R_PAREN, TokenType.L_PAREN): 629 return self._parse_csv(self._parse_lambda) 630 631 if self._match(TokenType.L_PAREN): 632 params = self._parse_csv(self._parse_lambda) 633 self._match_r_paren(this) 634 return params 635 636 return None 637 638 def _parse_quantile(self) -> exp.Quantile: 639 this = self._parse_lambda() 640 params = self._parse_func_params() 641 if params: 642 return self.expression(exp.Quantile, this=params[0], quantile=this) 643 return self.expression(exp.Quantile, this=this, quantile=exp.Literal.number(0.5)) 644 645 def _parse_wrapped_id_vars(self, optional: bool = False) -> t.List[exp.Expression]: 646 return super()._parse_wrapped_id_vars(optional=True) 647 648 def _parse_primary_key( 649 self, wrapped_optional: bool = False, in_props: bool = False 650 ) -> exp.PrimaryKeyColumnConstraint | exp.PrimaryKey: 651 return super()._parse_primary_key( 652 wrapped_optional=wrapped_optional or in_props, in_props=in_props 653 ) 654 655 def _parse_on_property(self) -> t.Optional[exp.Expression]: 656 index = self._index 657 if self._match_text_seq("CLUSTER"): 658 this = self._parse_id_var() 659 if this: 660 return self.expression(exp.OnCluster, this=this) 661 else: 662 self._retreat(index) 663 return None 664 665 def _parse_index_constraint( 666 self, kind: t.Optional[str] = None 667 ) -> exp.IndexColumnConstraint: 668 # INDEX name1 expr TYPE type1(args) GRANULARITY value 669 this = self._parse_id_var() 670 expression = self._parse_assignment() 671 672 index_type = self._match_text_seq("TYPE") and ( 673 self._parse_function() or self._parse_var() 674 ) 675 676 granularity = self._match_text_seq("GRANULARITY") and self._parse_term() 677 678 return self.expression( 679 exp.IndexColumnConstraint, 680 this=this, 681 expression=expression, 682 index_type=index_type, 683 granularity=granularity, 684 ) 685 686 def _parse_partition(self) -> t.Optional[exp.Partition]: 687 # https://clickhouse.com/docs/en/sql-reference/statements/alter/partition#how-to-set-partition-expression 688 if not self._match(TokenType.PARTITION): 689 return None 690 691 if self._match_text_seq("ID"): 692 # Corresponds to the PARTITION ID <string_value> syntax 693 expressions: t.List[exp.Expression] = [ 694 self.expression(exp.PartitionId, this=self._parse_string()) 695 ] 696 else: 697 expressions = self._parse_expressions() 698 699 return self.expression(exp.Partition, expressions=expressions) 700 701 def _parse_alter_table_replace(self) -> t.Optional[exp.Expression]: 702 partition = self._parse_partition() 703 704 if not partition or not self._match(TokenType.FROM): 705 return None 706 707 return self.expression( 708 exp.ReplacePartition, expression=partition, source=self._parse_table_parts() 709 ) 710 711 def _parse_projection_def(self) -> t.Optional[exp.ProjectionDef]: 712 if not self._match_text_seq("PROJECTION"): 713 return None 714 715 return self.expression( 716 exp.ProjectionDef, 717 this=self._parse_id_var(), 718 expression=self._parse_wrapped(self._parse_statement), 719 ) 720 721 def _parse_constraint(self) -> t.Optional[exp.Expression]: 722 return super()._parse_constraint() or self._parse_projection_def() 723 724 class Generator(generator.Generator): 725 QUERY_HINTS = False 726 STRUCT_DELIMITER = ("(", ")") 727 NVL2_SUPPORTED = False 728 TABLESAMPLE_REQUIRES_PARENS = False 729 TABLESAMPLE_SIZE_IS_ROWS = False 730 TABLESAMPLE_KEYWORDS = "SAMPLE" 731 LAST_DAY_SUPPORTS_DATE_PART = False 732 CAN_IMPLEMENT_ARRAY_ANY = True 733 SUPPORTS_TO_NUMBER = False 734 JOIN_HINTS = False 735 TABLE_HINTS = False 736 EXPLICIT_SET_OP = True 737 GROUPINGS_SEP = "" 738 SET_OP_MODIFIERS = False 739 SUPPORTS_TABLE_ALIAS_COLUMNS = False 740 VALUES_AS_TABLE = False 741 742 STRING_TYPE_MAPPING = { 743 exp.DataType.Type.CHAR: "String", 744 exp.DataType.Type.LONGBLOB: "String", 745 exp.DataType.Type.LONGTEXT: "String", 746 exp.DataType.Type.MEDIUMBLOB: "String", 747 exp.DataType.Type.MEDIUMTEXT: "String", 748 exp.DataType.Type.TINYBLOB: "String", 749 exp.DataType.Type.TINYTEXT: "String", 750 exp.DataType.Type.TEXT: "String", 751 exp.DataType.Type.VARBINARY: "String", 752 exp.DataType.Type.VARCHAR: "String", 753 } 754 755 SUPPORTED_JSON_PATH_PARTS = { 756 exp.JSONPathKey, 757 exp.JSONPathRoot, 758 exp.JSONPathSubscript, 759 } 760 761 TYPE_MAPPING = { 762 **generator.Generator.TYPE_MAPPING, 763 **STRING_TYPE_MAPPING, 764 exp.DataType.Type.ARRAY: "Array", 765 exp.DataType.Type.BIGINT: "Int64", 766 exp.DataType.Type.DATE32: "Date32", 767 exp.DataType.Type.DATETIME64: "DateTime64", 768 exp.DataType.Type.TIMESTAMPTZ: "TIMESTAMP", 769 exp.DataType.Type.DOUBLE: "Float64", 770 exp.DataType.Type.ENUM: "Enum", 771 exp.DataType.Type.ENUM8: "Enum8", 772 exp.DataType.Type.ENUM16: "Enum16", 773 exp.DataType.Type.FIXEDSTRING: "FixedString", 774 exp.DataType.Type.FLOAT: "Float32", 775 exp.DataType.Type.INT: "Int32", 776 exp.DataType.Type.MEDIUMINT: "Int32", 777 exp.DataType.Type.INT128: "Int128", 778 exp.DataType.Type.INT256: "Int256", 779 exp.DataType.Type.LOWCARDINALITY: "LowCardinality", 780 exp.DataType.Type.MAP: "Map", 781 exp.DataType.Type.NESTED: "Nested", 782 exp.DataType.Type.NULLABLE: "Nullable", 783 exp.DataType.Type.SMALLINT: "Int16", 784 exp.DataType.Type.STRUCT: "Tuple", 785 exp.DataType.Type.TINYINT: "Int8", 786 exp.DataType.Type.UBIGINT: "UInt64", 787 exp.DataType.Type.UINT: "UInt32", 788 exp.DataType.Type.UINT128: "UInt128", 789 exp.DataType.Type.UINT256: "UInt256", 790 exp.DataType.Type.USMALLINT: "UInt16", 791 exp.DataType.Type.UTINYINT: "UInt8", 792 exp.DataType.Type.IPV4: "IPv4", 793 exp.DataType.Type.IPV6: "IPv6", 794 exp.DataType.Type.AGGREGATEFUNCTION: "AggregateFunction", 795 exp.DataType.Type.SIMPLEAGGREGATEFUNCTION: "SimpleAggregateFunction", 796 } 797 798 TRANSFORMS = { 799 **generator.Generator.TRANSFORMS, 800 exp.AnyValue: rename_func("any"), 801 exp.ApproxDistinct: rename_func("uniq"), 802 exp.ArrayFilter: lambda self, e: self.func("arrayFilter", e.expression, e.this), 803 exp.ArraySize: rename_func("LENGTH"), 804 exp.ArraySum: rename_func("arraySum"), 805 exp.ArgMax: arg_max_or_min_no_count("argMax"), 806 exp.ArgMin: arg_max_or_min_no_count("argMin"), 807 exp.Array: inline_array_sql, 808 exp.CastToStrType: rename_func("CAST"), 809 exp.CountIf: rename_func("countIf"), 810 exp.CompressColumnConstraint: lambda self, 811 e: f"CODEC({self.expressions(e, key='this', flat=True)})", 812 exp.ComputedColumnConstraint: lambda self, 813 e: f"{'MATERIALIZED' if e.args.get('persisted') else 'ALIAS'} {self.sql(e, 'this')}", 814 exp.CurrentDate: lambda self, e: self.func("CURRENT_DATE"), 815 exp.DateAdd: _datetime_delta_sql("DATE_ADD"), 816 exp.DateDiff: _datetime_delta_sql("DATE_DIFF"), 817 exp.DateStrToDate: rename_func("toDate"), 818 exp.DateSub: _datetime_delta_sql("DATE_SUB"), 819 exp.Explode: rename_func("arrayJoin"), 820 exp.Final: lambda self, e: f"{self.sql(e, 'this')} FINAL", 821 exp.IsNan: rename_func("isNaN"), 822 exp.JSONExtract: json_extract_segments("JSONExtractString", quoted_index=False), 823 exp.JSONExtractScalar: json_extract_segments("JSONExtractString", quoted_index=False), 824 exp.JSONPathKey: json_path_key_only_name, 825 exp.JSONPathRoot: lambda *_: "", 826 exp.Map: lambda self, e: _lower_func(var_map_sql(self, e)), 827 exp.Nullif: rename_func("nullIf"), 828 exp.PartitionedByProperty: lambda self, e: f"PARTITION BY {self.sql(e, 'this')}", 829 exp.Pivot: no_pivot_sql, 830 exp.Quantile: _quantile_sql, 831 exp.RegexpLike: lambda self, e: self.func("match", e.this, e.expression), 832 exp.Rand: rename_func("randCanonical"), 833 exp.StartsWith: rename_func("startsWith"), 834 exp.StrPosition: lambda self, e: self.func( 835 "position", e.this, e.args.get("substr"), e.args.get("position") 836 ), 837 exp.TimeToStr: lambda self, e: self.func( 838 "DATE_FORMAT", e.this, self.format_time(e), e.args.get("timezone") 839 ), 840 exp.TimestampAdd: _datetime_delta_sql("TIMESTAMP_ADD"), 841 exp.TimestampSub: _datetime_delta_sql("TIMESTAMP_SUB"), 842 exp.VarMap: lambda self, e: _lower_func(var_map_sql(self, e)), 843 exp.Xor: lambda self, e: self.func("xor", e.this, e.expression, *e.expressions), 844 exp.MD5Digest: rename_func("MD5"), 845 exp.MD5: lambda self, e: self.func("LOWER", self.func("HEX", self.func("MD5", e.this))), 846 exp.SHA: rename_func("SHA1"), 847 exp.SHA2: sha256_sql, 848 exp.UnixToTime: _unix_to_time_sql, 849 exp.TimestampTrunc: timestamptrunc_sql(zone=True), 850 exp.Variance: rename_func("varSamp"), 851 exp.SchemaCommentProperty: lambda self, e: self.naked_property(e), 852 exp.Stddev: rename_func("stddevSamp"), 853 } 854 855 PROPERTIES_LOCATION = { 856 **generator.Generator.PROPERTIES_LOCATION, 857 exp.VolatileProperty: exp.Properties.Location.UNSUPPORTED, 858 exp.PartitionedByProperty: exp.Properties.Location.POST_SCHEMA, 859 exp.OnCluster: exp.Properties.Location.POST_NAME, 860 } 861 862 # There's no list in docs, but it can be found in Clickhouse code 863 # see `ClickHouse/src/Parsers/ParserCreate*.cpp` 864 ON_CLUSTER_TARGETS = { 865 "DATABASE", 866 "TABLE", 867 "VIEW", 868 "DICTIONARY", 869 "INDEX", 870 "FUNCTION", 871 "NAMED COLLECTION", 872 } 873 874 # https://clickhouse.com/docs/en/sql-reference/data-types/nullable 875 NON_NULLABLE_TYPES = { 876 exp.DataType.Type.ARRAY, 877 exp.DataType.Type.MAP, 878 exp.DataType.Type.NULLABLE, 879 exp.DataType.Type.STRUCT, 880 } 881 882 def strtodate_sql(self, expression: exp.StrToDate) -> str: 883 strtodate_sql = self.function_fallback_sql(expression) 884 885 if not isinstance(expression.parent, exp.Cast): 886 # StrToDate returns DATEs in other dialects (eg. postgres), so 887 # this branch aims to improve the transpilation to clickhouse 888 return f"CAST({strtodate_sql} AS DATE)" 889 890 return strtodate_sql 891 892 def cast_sql(self, expression: exp.Cast, safe_prefix: t.Optional[str] = None) -> str: 893 this = expression.this 894 895 if isinstance(this, exp.StrToDate) and expression.to == exp.DataType.build("datetime"): 896 return self.sql(this) 897 898 return super().cast_sql(expression, safe_prefix=safe_prefix) 899 900 def trycast_sql(self, expression: exp.TryCast) -> str: 901 dtype = expression.to 902 if not dtype.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True): 903 # Casting x into Nullable(T) appears to behave similarly to TRY_CAST(x AS T) 904 dtype.set("nullable", True) 905 906 return super().cast_sql(expression) 907 908 def _jsonpathsubscript_sql(self, expression: exp.JSONPathSubscript) -> str: 909 this = self.json_path_part(expression.this) 910 return str(int(this) + 1) if is_int(this) else this 911 912 def likeproperty_sql(self, expression: exp.LikeProperty) -> str: 913 return f"AS {self.sql(expression, 'this')}" 914 915 def _any_to_has( 916 self, 917 expression: exp.EQ | exp.NEQ, 918 default: t.Callable[[t.Any], str], 919 prefix: str = "", 920 ) -> str: 921 if isinstance(expression.left, exp.Any): 922 arr = expression.left 923 this = expression.right 924 elif isinstance(expression.right, exp.Any): 925 arr = expression.right 926 this = expression.left 927 else: 928 return default(expression) 929 930 return prefix + self.func("has", arr.this.unnest(), this) 931 932 def eq_sql(self, expression: exp.EQ) -> str: 933 return self._any_to_has(expression, super().eq_sql) 934 935 def neq_sql(self, expression: exp.NEQ) -> str: 936 return self._any_to_has(expression, super().neq_sql, "NOT ") 937 938 def regexpilike_sql(self, expression: exp.RegexpILike) -> str: 939 # Manually add a flag to make the search case-insensitive 940 regex = self.func("CONCAT", "'(?i)'", expression.expression) 941 return self.func("match", expression.this, regex) 942 943 def datatype_sql(self, expression: exp.DataType) -> str: 944 # String is the standard ClickHouse type, every other variant is just an alias. 945 # Additionally, any supplied length parameter will be ignored. 946 # 947 # https://clickhouse.com/docs/en/sql-reference/data-types/string 948 if expression.this in self.STRING_TYPE_MAPPING: 949 dtype = "String" 950 else: 951 dtype = super().datatype_sql(expression) 952 953 # This section changes the type to `Nullable(...)` if the following conditions hold: 954 # - It's marked as nullable - this ensures we won't wrap ClickHouse types with `Nullable` 955 # and change their semantics 956 # - It's not the key type of a `Map`. This is because ClickHouse enforces the following 957 # constraint: "Type of Map key must be a type, that can be represented by integer or 958 # String or FixedString (possibly LowCardinality) or UUID or IPv6" 959 # - It's not a composite type, e.g. `Nullable(Array(...))` is not a valid type 960 parent = expression.parent 961 if ( 962 expression.args.get("nullable") is not False 963 and not ( 964 isinstance(parent, exp.DataType) 965 and parent.is_type(exp.DataType.Type.MAP, check_nullable=True) 966 and expression.index in (None, 0) 967 ) 968 and not expression.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True) 969 ): 970 dtype = f"Nullable({dtype})" 971 972 return dtype 973 974 def cte_sql(self, expression: exp.CTE) -> str: 975 if expression.args.get("scalar"): 976 this = self.sql(expression, "this") 977 alias = self.sql(expression, "alias") 978 return f"{this} AS {alias}" 979 980 return super().cte_sql(expression) 981 982 def after_limit_modifiers(self, expression: exp.Expression) -> t.List[str]: 983 return super().after_limit_modifiers(expression) + [ 984 ( 985 self.seg("SETTINGS ") + self.expressions(expression, key="settings", flat=True) 986 if expression.args.get("settings") 987 else "" 988 ), 989 ( 990 self.seg("FORMAT ") + self.sql(expression, "format") 991 if expression.args.get("format") 992 else "" 993 ), 994 ] 995 996 def parameterizedagg_sql(self, expression: exp.ParameterizedAgg) -> str: 997 params = self.expressions(expression, key="params", flat=True) 998 return self.func(expression.name, *expression.expressions) + f"({params})" 999 1000 def anonymousaggfunc_sql(self, expression: exp.AnonymousAggFunc) -> str: 1001 return self.func(expression.name, *expression.expressions) 1002 1003 def combinedaggfunc_sql(self, expression: exp.CombinedAggFunc) -> str: 1004 return self.anonymousaggfunc_sql(expression) 1005 1006 def combinedparameterizedagg_sql(self, expression: exp.CombinedParameterizedAgg) -> str: 1007 return self.parameterizedagg_sql(expression) 1008 1009 def placeholder_sql(self, expression: exp.Placeholder) -> str: 1010 return f"{{{expression.name}: {self.sql(expression, 'kind')}}}" 1011 1012 def oncluster_sql(self, expression: exp.OnCluster) -> str: 1013 return f"ON CLUSTER {self.sql(expression, 'this')}" 1014 1015 def createable_sql(self, expression: exp.Create, locations: t.DefaultDict) -> str: 1016 if expression.kind in self.ON_CLUSTER_TARGETS and locations.get( 1017 exp.Properties.Location.POST_NAME 1018 ): 1019 this_name = self.sql( 1020 expression.this if isinstance(expression.this, exp.Schema) else expression, 1021 "this", 1022 ) 1023 this_properties = " ".join( 1024 [self.sql(prop) for prop in locations[exp.Properties.Location.POST_NAME]] 1025 ) 1026 this_schema = self.schema_columns_sql(expression.this) 1027 return f"{this_name}{self.sep()}{this_properties}{self.sep()}{this_schema}" 1028 1029 return super().createable_sql(expression, locations) 1030 1031 def create_sql(self, expression: exp.Create) -> str: 1032 # The comment property comes last in CTAS statements, i.e. after the query 1033 query = expression.expression 1034 if isinstance(query, exp.Query): 1035 comment_prop = expression.find(exp.SchemaCommentProperty) 1036 if comment_prop: 1037 comment_prop.pop() 1038 query.replace(exp.paren(query)) 1039 else: 1040 comment_prop = None 1041 1042 create_sql = super().create_sql(expression) 1043 1044 comment_sql = self.sql(comment_prop) 1045 comment_sql = f" {comment_sql}" if comment_sql else "" 1046 1047 return f"{create_sql}{comment_sql}" 1048 1049 def prewhere_sql(self, expression: exp.PreWhere) -> str: 1050 this = self.indent(self.sql(expression, "this")) 1051 return f"{self.seg('PREWHERE')}{self.sep()}{this}" 1052 1053 def indexcolumnconstraint_sql(self, expression: exp.IndexColumnConstraint) -> str: 1054 this = self.sql(expression, "this") 1055 this = f" {this}" if this else "" 1056 expr = self.sql(expression, "expression") 1057 expr = f" {expr}" if expr else "" 1058 index_type = self.sql(expression, "index_type") 1059 index_type = f" TYPE {index_type}" if index_type else "" 1060 granularity = self.sql(expression, "granularity") 1061 granularity = f" GRANULARITY {granularity}" if granularity else "" 1062 1063 return f"INDEX{this}{expr}{index_type}{granularity}" 1064 1065 def partition_sql(self, expression: exp.Partition) -> str: 1066 return f"PARTITION {self.expressions(expression, flat=True)}" 1067 1068 def partitionid_sql(self, expression: exp.PartitionId) -> str: 1069 return f"ID {self.sql(expression.this)}" 1070 1071 def replacepartition_sql(self, expression: exp.ReplacePartition) -> str: 1072 return ( 1073 f"REPLACE {self.sql(expression.expression)} FROM {self.sql(expression, 'source')}" 1074 ) 1075 1076 def projectiondef_sql(self, expression: exp.ProjectionDef) -> str: 1077 return f"PROJECTION {self.sql(expression.this)} {self.wrap(expression.expression)}"
109class ClickHouse(Dialect): 110 NORMALIZE_FUNCTIONS: bool | str = False 111 NULL_ORDERING = "nulls_are_last" 112 SUPPORTS_USER_DEFINED_TYPES = False 113 SAFE_DIVISION = True 114 LOG_BASE_FIRST: t.Optional[bool] = None 115 FORCE_EARLY_ALIAS_REF_EXPANSION = True 116 117 # https://github.com/ClickHouse/ClickHouse/issues/33935#issue-1112165779 118 NORMALIZATION_STRATEGY = NormalizationStrategy.CASE_SENSITIVE 119 120 UNESCAPED_SEQUENCES = { 121 "\\0": "\0", 122 } 123 124 CREATABLE_KIND_MAPPING = {"DATABASE": "SCHEMA"} 125 126 class Tokenizer(tokens.Tokenizer): 127 COMMENTS = ["--", "#", "#!", ("/*", "*/")] 128 IDENTIFIERS = ['"', "`"] 129 STRING_ESCAPES = ["'", "\\"] 130 BIT_STRINGS = [("0b", "")] 131 HEX_STRINGS = [("0x", ""), ("0X", "")] 132 HEREDOC_STRINGS = ["$"] 133 134 KEYWORDS = { 135 **tokens.Tokenizer.KEYWORDS, 136 "ATTACH": TokenType.COMMAND, 137 "DATE32": TokenType.DATE32, 138 "DATETIME64": TokenType.DATETIME64, 139 "DICTIONARY": TokenType.DICTIONARY, 140 "ENUM8": TokenType.ENUM8, 141 "ENUM16": TokenType.ENUM16, 142 "FINAL": TokenType.FINAL, 143 "FIXEDSTRING": TokenType.FIXEDSTRING, 144 "FLOAT32": TokenType.FLOAT, 145 "FLOAT64": TokenType.DOUBLE, 146 "GLOBAL": TokenType.GLOBAL, 147 "INT256": TokenType.INT256, 148 "LOWCARDINALITY": TokenType.LOWCARDINALITY, 149 "MAP": TokenType.MAP, 150 "NESTED": TokenType.NESTED, 151 "SAMPLE": TokenType.TABLE_SAMPLE, 152 "TUPLE": TokenType.STRUCT, 153 "UINT128": TokenType.UINT128, 154 "UINT16": TokenType.USMALLINT, 155 "UINT256": TokenType.UINT256, 156 "UINT32": TokenType.UINT, 157 "UINT64": TokenType.UBIGINT, 158 "UINT8": TokenType.UTINYINT, 159 "IPV4": TokenType.IPV4, 160 "IPV6": TokenType.IPV6, 161 "AGGREGATEFUNCTION": TokenType.AGGREGATEFUNCTION, 162 "SIMPLEAGGREGATEFUNCTION": TokenType.SIMPLEAGGREGATEFUNCTION, 163 "SYSTEM": TokenType.COMMAND, 164 "PREWHERE": TokenType.PREWHERE, 165 } 166 KEYWORDS.pop("/*+") 167 168 SINGLE_TOKENS = { 169 **tokens.Tokenizer.SINGLE_TOKENS, 170 "$": TokenType.HEREDOC_STRING, 171 } 172 173 class Parser(parser.Parser): 174 # Tested in ClickHouse's playground, it seems that the following two queries do the same thing 175 # * select x from t1 union all select x from t2 limit 1; 176 # * select x from t1 union all (select x from t2 limit 1); 177 MODIFIERS_ATTACHED_TO_SET_OP = False 178 INTERVAL_SPANS = False 179 180 FUNCTIONS = { 181 **parser.Parser.FUNCTIONS, 182 "ANY": exp.AnyValue.from_arg_list, 183 "ARRAYSUM": exp.ArraySum.from_arg_list, 184 "COUNTIF": _build_count_if, 185 "DATE_ADD": build_date_delta(exp.DateAdd, default_unit=None), 186 "DATEADD": build_date_delta(exp.DateAdd, default_unit=None), 187 "DATE_DIFF": build_date_delta(exp.DateDiff, default_unit=None), 188 "DATEDIFF": build_date_delta(exp.DateDiff, default_unit=None), 189 "DATE_FORMAT": _build_date_format, 190 "DATE_SUB": build_date_delta(exp.DateSub, default_unit=None), 191 "DATESUB": build_date_delta(exp.DateSub, default_unit=None), 192 "FORMATDATETIME": _build_date_format, 193 "JSONEXTRACTSTRING": build_json_extract_path( 194 exp.JSONExtractScalar, zero_based_indexing=False 195 ), 196 "MAP": parser.build_var_map, 197 "MATCH": exp.RegexpLike.from_arg_list, 198 "RANDCANONICAL": exp.Rand.from_arg_list, 199 "STR_TO_DATE": _build_str_to_date, 200 "TUPLE": exp.Struct.from_arg_list, 201 "TIMESTAMP_SUB": build_date_delta(exp.TimestampSub, default_unit=None), 202 "TIMESTAMPSUB": build_date_delta(exp.TimestampSub, default_unit=None), 203 "TIMESTAMP_ADD": build_date_delta(exp.TimestampAdd, default_unit=None), 204 "TIMESTAMPADD": build_date_delta(exp.TimestampAdd, default_unit=None), 205 "UNIQ": exp.ApproxDistinct.from_arg_list, 206 "XOR": lambda args: exp.Xor(expressions=args), 207 "MD5": exp.MD5Digest.from_arg_list, 208 "SHA256": lambda args: exp.SHA2(this=seq_get(args, 0), length=exp.Literal.number(256)), 209 "SHA512": lambda args: exp.SHA2(this=seq_get(args, 0), length=exp.Literal.number(512)), 210 } 211 212 AGG_FUNCTIONS = { 213 "count", 214 "min", 215 "max", 216 "sum", 217 "avg", 218 "any", 219 "stddevPop", 220 "stddevSamp", 221 "varPop", 222 "varSamp", 223 "corr", 224 "covarPop", 225 "covarSamp", 226 "entropy", 227 "exponentialMovingAverage", 228 "intervalLengthSum", 229 "kolmogorovSmirnovTest", 230 "mannWhitneyUTest", 231 "median", 232 "rankCorr", 233 "sumKahan", 234 "studentTTest", 235 "welchTTest", 236 "anyHeavy", 237 "anyLast", 238 "boundingRatio", 239 "first_value", 240 "last_value", 241 "argMin", 242 "argMax", 243 "avgWeighted", 244 "topK", 245 "topKWeighted", 246 "deltaSum", 247 "deltaSumTimestamp", 248 "groupArray", 249 "groupArrayLast", 250 "groupUniqArray", 251 "groupArrayInsertAt", 252 "groupArrayMovingAvg", 253 "groupArrayMovingSum", 254 "groupArraySample", 255 "groupBitAnd", 256 "groupBitOr", 257 "groupBitXor", 258 "groupBitmap", 259 "groupBitmapAnd", 260 "groupBitmapOr", 261 "groupBitmapXor", 262 "sumWithOverflow", 263 "sumMap", 264 "minMap", 265 "maxMap", 266 "skewSamp", 267 "skewPop", 268 "kurtSamp", 269 "kurtPop", 270 "uniq", 271 "uniqExact", 272 "uniqCombined", 273 "uniqCombined64", 274 "uniqHLL12", 275 "uniqTheta", 276 "quantile", 277 "quantiles", 278 "quantileExact", 279 "quantilesExact", 280 "quantileExactLow", 281 "quantilesExactLow", 282 "quantileExactHigh", 283 "quantilesExactHigh", 284 "quantileExactWeighted", 285 "quantilesExactWeighted", 286 "quantileTiming", 287 "quantilesTiming", 288 "quantileTimingWeighted", 289 "quantilesTimingWeighted", 290 "quantileDeterministic", 291 "quantilesDeterministic", 292 "quantileTDigest", 293 "quantilesTDigest", 294 "quantileTDigestWeighted", 295 "quantilesTDigestWeighted", 296 "quantileBFloat16", 297 "quantilesBFloat16", 298 "quantileBFloat16Weighted", 299 "quantilesBFloat16Weighted", 300 "simpleLinearRegression", 301 "stochasticLinearRegression", 302 "stochasticLogisticRegression", 303 "categoricalInformationValue", 304 "contingency", 305 "cramersV", 306 "cramersVBiasCorrected", 307 "theilsU", 308 "maxIntersections", 309 "maxIntersectionsPosition", 310 "meanZTest", 311 "quantileInterpolatedWeighted", 312 "quantilesInterpolatedWeighted", 313 "quantileGK", 314 "quantilesGK", 315 "sparkBar", 316 "sumCount", 317 "largestTriangleThreeBuckets", 318 "histogram", 319 "sequenceMatch", 320 "sequenceCount", 321 "windowFunnel", 322 "retention", 323 "uniqUpTo", 324 "sequenceNextNode", 325 "exponentialTimeDecayedAvg", 326 } 327 328 AGG_FUNCTIONS_SUFFIXES = [ 329 "If", 330 "Array", 331 "ArrayIf", 332 "Map", 333 "SimpleState", 334 "State", 335 "Merge", 336 "MergeState", 337 "ForEach", 338 "Distinct", 339 "OrDefault", 340 "OrNull", 341 "Resample", 342 "ArgMin", 343 "ArgMax", 344 ] 345 346 FUNC_TOKENS = { 347 *parser.Parser.FUNC_TOKENS, 348 TokenType.SET, 349 } 350 351 RESERVED_TOKENS = parser.Parser.RESERVED_TOKENS - {TokenType.SELECT} 352 353 ID_VAR_TOKENS = { 354 *parser.Parser.ID_VAR_TOKENS, 355 TokenType.LIKE, 356 } 357 358 AGG_FUNC_MAPPING = ( 359 lambda functions, suffixes: { 360 f"{f}{sfx}": (f, sfx) for sfx in (suffixes + [""]) for f in functions 361 } 362 )(AGG_FUNCTIONS, AGG_FUNCTIONS_SUFFIXES) 363 364 FUNCTIONS_WITH_ALIASED_ARGS = {*parser.Parser.FUNCTIONS_WITH_ALIASED_ARGS, "TUPLE"} 365 366 FUNCTION_PARSERS = { 367 **parser.Parser.FUNCTION_PARSERS, 368 "ARRAYJOIN": lambda self: self.expression(exp.Explode, this=self._parse_expression()), 369 "QUANTILE": lambda self: self._parse_quantile(), 370 } 371 372 FUNCTION_PARSERS.pop("MATCH") 373 374 NO_PAREN_FUNCTION_PARSERS = parser.Parser.NO_PAREN_FUNCTION_PARSERS.copy() 375 NO_PAREN_FUNCTION_PARSERS.pop("ANY") 376 377 RANGE_PARSERS = { 378 **parser.Parser.RANGE_PARSERS, 379 TokenType.GLOBAL: lambda self, this: self._match(TokenType.IN) 380 and self._parse_in(this, is_global=True), 381 } 382 383 # The PLACEHOLDER entry is popped because 1) it doesn't affect Clickhouse (it corresponds to 384 # the postgres-specific JSONBContains parser) and 2) it makes parsing the ternary op simpler. 385 COLUMN_OPERATORS = parser.Parser.COLUMN_OPERATORS.copy() 386 COLUMN_OPERATORS.pop(TokenType.PLACEHOLDER) 387 388 JOIN_KINDS = { 389 *parser.Parser.JOIN_KINDS, 390 TokenType.ANY, 391 TokenType.ASOF, 392 TokenType.ARRAY, 393 } 394 395 TABLE_ALIAS_TOKENS = parser.Parser.TABLE_ALIAS_TOKENS - { 396 TokenType.ANY, 397 TokenType.ARRAY, 398 TokenType.FINAL, 399 TokenType.FORMAT, 400 TokenType.SETTINGS, 401 } 402 403 ALIAS_TOKENS = parser.Parser.ALIAS_TOKENS - { 404 TokenType.FORMAT, 405 } 406 407 LOG_DEFAULTS_TO_LN = True 408 409 QUERY_MODIFIER_PARSERS = { 410 **parser.Parser.QUERY_MODIFIER_PARSERS, 411 TokenType.SETTINGS: lambda self: ( 412 "settings", 413 self._advance() or self._parse_csv(self._parse_assignment), 414 ), 415 TokenType.FORMAT: lambda self: ("format", self._advance() or self._parse_id_var()), 416 } 417 418 CONSTRAINT_PARSERS = { 419 **parser.Parser.CONSTRAINT_PARSERS, 420 "INDEX": lambda self: self._parse_index_constraint(), 421 "CODEC": lambda self: self._parse_compress(), 422 } 423 424 ALTER_PARSERS = { 425 **parser.Parser.ALTER_PARSERS, 426 "REPLACE": lambda self: self._parse_alter_table_replace(), 427 } 428 429 SCHEMA_UNNAMED_CONSTRAINTS = { 430 *parser.Parser.SCHEMA_UNNAMED_CONSTRAINTS, 431 "INDEX", 432 } 433 434 def _parse_types( 435 self, check_func: bool = False, schema: bool = False, allow_identifiers: bool = True 436 ) -> t.Optional[exp.Expression]: 437 dtype = super()._parse_types( 438 check_func=check_func, schema=schema, allow_identifiers=allow_identifiers 439 ) 440 if isinstance(dtype, exp.DataType): 441 # Mark every type as non-nullable which is ClickHouse's default. This marker 442 # helps us transpile types from other dialects to ClickHouse, so that we can 443 # e.g. produce `CAST(x AS Nullable(String))` from `CAST(x AS TEXT)`. If there 444 # is a `NULL` value in `x`, the former would fail in ClickHouse without the 445 # `Nullable` type constructor 446 dtype.set("nullable", False) 447 448 return dtype 449 450 def _parse_extract(self) -> exp.Extract | exp.Anonymous: 451 index = self._index 452 this = self._parse_bitwise() 453 if self._match(TokenType.FROM): 454 self._retreat(index) 455 return super()._parse_extract() 456 457 # We return Anonymous here because extract and regexpExtract have different semantics, 458 # so parsing extract(foo, bar) into RegexpExtract can potentially break queries. E.g., 459 # `extract('foobar', 'b')` works, but ClickHouse crashes for `regexpExtract('foobar', 'b')`. 460 # 461 # TODO: can we somehow convert the former into an equivalent `regexpExtract` call? 462 self._match(TokenType.COMMA) 463 return self.expression( 464 exp.Anonymous, this="extract", expressions=[this, self._parse_bitwise()] 465 ) 466 467 def _parse_assignment(self) -> t.Optional[exp.Expression]: 468 this = super()._parse_assignment() 469 470 if self._match(TokenType.PLACEHOLDER): 471 return self.expression( 472 exp.If, 473 this=this, 474 true=self._parse_assignment(), 475 false=self._match(TokenType.COLON) and self._parse_assignment(), 476 ) 477 478 return this 479 480 def _parse_placeholder(self) -> t.Optional[exp.Expression]: 481 """ 482 Parse a placeholder expression like SELECT {abc: UInt32} or FROM {table: Identifier} 483 https://clickhouse.com/docs/en/sql-reference/syntax#defining-and-using-query-parameters 484 """ 485 if not self._match(TokenType.L_BRACE): 486 return None 487 488 this = self._parse_id_var() 489 self._match(TokenType.COLON) 490 kind = self._parse_types(check_func=False, allow_identifiers=False) or ( 491 self._match_text_seq("IDENTIFIER") and "Identifier" 492 ) 493 494 if not kind: 495 self.raise_error("Expecting a placeholder type or 'Identifier' for tables") 496 elif not self._match(TokenType.R_BRACE): 497 self.raise_error("Expecting }") 498 499 return self.expression(exp.Placeholder, this=this, kind=kind) 500 501 def _parse_in(self, this: t.Optional[exp.Expression], is_global: bool = False) -> exp.In: 502 this = super()._parse_in(this) 503 this.set("is_global", is_global) 504 return this 505 506 def _parse_table( 507 self, 508 schema: bool = False, 509 joins: bool = False, 510 alias_tokens: t.Optional[t.Collection[TokenType]] = None, 511 parse_bracket: bool = False, 512 is_db_reference: bool = False, 513 parse_partition: bool = False, 514 ) -> t.Optional[exp.Expression]: 515 this = super()._parse_table( 516 schema=schema, 517 joins=joins, 518 alias_tokens=alias_tokens, 519 parse_bracket=parse_bracket, 520 is_db_reference=is_db_reference, 521 ) 522 523 if self._match(TokenType.FINAL): 524 this = self.expression(exp.Final, this=this) 525 526 return this 527 528 def _parse_position(self, haystack_first: bool = False) -> exp.StrPosition: 529 return super()._parse_position(haystack_first=True) 530 531 # https://clickhouse.com/docs/en/sql-reference/statements/select/with/ 532 def _parse_cte(self) -> exp.CTE: 533 # WITH <identifier> AS <subquery expression> 534 cte: t.Optional[exp.CTE] = self._try_parse(super()._parse_cte) 535 536 if not cte: 537 # WITH <expression> AS <identifier> 538 cte = self.expression( 539 exp.CTE, 540 this=self._parse_assignment(), 541 alias=self._parse_table_alias(), 542 scalar=True, 543 ) 544 545 return cte 546 547 def _parse_join_parts( 548 self, 549 ) -> t.Tuple[t.Optional[Token], t.Optional[Token], t.Optional[Token]]: 550 is_global = self._match(TokenType.GLOBAL) and self._prev 551 kind_pre = self._match_set(self.JOIN_KINDS, advance=False) and self._prev 552 553 if kind_pre: 554 kind = self._match_set(self.JOIN_KINDS) and self._prev 555 side = self._match_set(self.JOIN_SIDES) and self._prev 556 return is_global, side, kind 557 558 return ( 559 is_global, 560 self._match_set(self.JOIN_SIDES) and self._prev, 561 self._match_set(self.JOIN_KINDS) and self._prev, 562 ) 563 564 def _parse_join( 565 self, skip_join_token: bool = False, parse_bracket: bool = False 566 ) -> t.Optional[exp.Join]: 567 join = super()._parse_join(skip_join_token=skip_join_token, parse_bracket=True) 568 if join: 569 join.set("global", join.args.pop("method", None)) 570 571 return join 572 573 def _parse_function( 574 self, 575 functions: t.Optional[t.Dict[str, t.Callable]] = None, 576 anonymous: bool = False, 577 optional_parens: bool = True, 578 any_token: bool = False, 579 ) -> t.Optional[exp.Expression]: 580 expr = super()._parse_function( 581 functions=functions, 582 anonymous=anonymous, 583 optional_parens=optional_parens, 584 any_token=any_token, 585 ) 586 587 func = expr.this if isinstance(expr, exp.Window) else expr 588 589 # Aggregate functions can be split in 2 parts: <func_name><suffix> 590 parts = ( 591 self.AGG_FUNC_MAPPING.get(func.this) if isinstance(func, exp.Anonymous) else None 592 ) 593 594 if parts: 595 params = self._parse_func_params(func) 596 597 kwargs = { 598 "this": func.this, 599 "expressions": func.expressions, 600 } 601 if parts[1]: 602 kwargs["parts"] = parts 603 exp_class = exp.CombinedParameterizedAgg if params else exp.CombinedAggFunc 604 else: 605 exp_class = exp.ParameterizedAgg if params else exp.AnonymousAggFunc 606 607 kwargs["exp_class"] = exp_class 608 if params: 609 kwargs["params"] = params 610 611 func = self.expression(**kwargs) 612 613 if isinstance(expr, exp.Window): 614 # The window's func was parsed as Anonymous in base parser, fix its 615 # type to be ClickHouse style CombinedAnonymousAggFunc / AnonymousAggFunc 616 expr.set("this", func) 617 elif params: 618 # Params have blocked super()._parse_function() from parsing the following window 619 # (if that exists) as they're standing between the function call and the window spec 620 expr = self._parse_window(func) 621 else: 622 expr = func 623 624 return expr 625 626 def _parse_func_params( 627 self, this: t.Optional[exp.Func] = None 628 ) -> t.Optional[t.List[exp.Expression]]: 629 if self._match_pair(TokenType.R_PAREN, TokenType.L_PAREN): 630 return self._parse_csv(self._parse_lambda) 631 632 if self._match(TokenType.L_PAREN): 633 params = self._parse_csv(self._parse_lambda) 634 self._match_r_paren(this) 635 return params 636 637 return None 638 639 def _parse_quantile(self) -> exp.Quantile: 640 this = self._parse_lambda() 641 params = self._parse_func_params() 642 if params: 643 return self.expression(exp.Quantile, this=params[0], quantile=this) 644 return self.expression(exp.Quantile, this=this, quantile=exp.Literal.number(0.5)) 645 646 def _parse_wrapped_id_vars(self, optional: bool = False) -> t.List[exp.Expression]: 647 return super()._parse_wrapped_id_vars(optional=True) 648 649 def _parse_primary_key( 650 self, wrapped_optional: bool = False, in_props: bool = False 651 ) -> exp.PrimaryKeyColumnConstraint | exp.PrimaryKey: 652 return super()._parse_primary_key( 653 wrapped_optional=wrapped_optional or in_props, in_props=in_props 654 ) 655 656 def _parse_on_property(self) -> t.Optional[exp.Expression]: 657 index = self._index 658 if self._match_text_seq("CLUSTER"): 659 this = self._parse_id_var() 660 if this: 661 return self.expression(exp.OnCluster, this=this) 662 else: 663 self._retreat(index) 664 return None 665 666 def _parse_index_constraint( 667 self, kind: t.Optional[str] = None 668 ) -> exp.IndexColumnConstraint: 669 # INDEX name1 expr TYPE type1(args) GRANULARITY value 670 this = self._parse_id_var() 671 expression = self._parse_assignment() 672 673 index_type = self._match_text_seq("TYPE") and ( 674 self._parse_function() or self._parse_var() 675 ) 676 677 granularity = self._match_text_seq("GRANULARITY") and self._parse_term() 678 679 return self.expression( 680 exp.IndexColumnConstraint, 681 this=this, 682 expression=expression, 683 index_type=index_type, 684 granularity=granularity, 685 ) 686 687 def _parse_partition(self) -> t.Optional[exp.Partition]: 688 # https://clickhouse.com/docs/en/sql-reference/statements/alter/partition#how-to-set-partition-expression 689 if not self._match(TokenType.PARTITION): 690 return None 691 692 if self._match_text_seq("ID"): 693 # Corresponds to the PARTITION ID <string_value> syntax 694 expressions: t.List[exp.Expression] = [ 695 self.expression(exp.PartitionId, this=self._parse_string()) 696 ] 697 else: 698 expressions = self._parse_expressions() 699 700 return self.expression(exp.Partition, expressions=expressions) 701 702 def _parse_alter_table_replace(self) -> t.Optional[exp.Expression]: 703 partition = self._parse_partition() 704 705 if not partition or not self._match(TokenType.FROM): 706 return None 707 708 return self.expression( 709 exp.ReplacePartition, expression=partition, source=self._parse_table_parts() 710 ) 711 712 def _parse_projection_def(self) -> t.Optional[exp.ProjectionDef]: 713 if not self._match_text_seq("PROJECTION"): 714 return None 715 716 return self.expression( 717 exp.ProjectionDef, 718 this=self._parse_id_var(), 719 expression=self._parse_wrapped(self._parse_statement), 720 ) 721 722 def _parse_constraint(self) -> t.Optional[exp.Expression]: 723 return super()._parse_constraint() or self._parse_projection_def() 724 725 class Generator(generator.Generator): 726 QUERY_HINTS = False 727 STRUCT_DELIMITER = ("(", ")") 728 NVL2_SUPPORTED = False 729 TABLESAMPLE_REQUIRES_PARENS = False 730 TABLESAMPLE_SIZE_IS_ROWS = False 731 TABLESAMPLE_KEYWORDS = "SAMPLE" 732 LAST_DAY_SUPPORTS_DATE_PART = False 733 CAN_IMPLEMENT_ARRAY_ANY = True 734 SUPPORTS_TO_NUMBER = False 735 JOIN_HINTS = False 736 TABLE_HINTS = False 737 EXPLICIT_SET_OP = True 738 GROUPINGS_SEP = "" 739 SET_OP_MODIFIERS = False 740 SUPPORTS_TABLE_ALIAS_COLUMNS = False 741 VALUES_AS_TABLE = False 742 743 STRING_TYPE_MAPPING = { 744 exp.DataType.Type.CHAR: "String", 745 exp.DataType.Type.LONGBLOB: "String", 746 exp.DataType.Type.LONGTEXT: "String", 747 exp.DataType.Type.MEDIUMBLOB: "String", 748 exp.DataType.Type.MEDIUMTEXT: "String", 749 exp.DataType.Type.TINYBLOB: "String", 750 exp.DataType.Type.TINYTEXT: "String", 751 exp.DataType.Type.TEXT: "String", 752 exp.DataType.Type.VARBINARY: "String", 753 exp.DataType.Type.VARCHAR: "String", 754 } 755 756 SUPPORTED_JSON_PATH_PARTS = { 757 exp.JSONPathKey, 758 exp.JSONPathRoot, 759 exp.JSONPathSubscript, 760 } 761 762 TYPE_MAPPING = { 763 **generator.Generator.TYPE_MAPPING, 764 **STRING_TYPE_MAPPING, 765 exp.DataType.Type.ARRAY: "Array", 766 exp.DataType.Type.BIGINT: "Int64", 767 exp.DataType.Type.DATE32: "Date32", 768 exp.DataType.Type.DATETIME64: "DateTime64", 769 exp.DataType.Type.TIMESTAMPTZ: "TIMESTAMP", 770 exp.DataType.Type.DOUBLE: "Float64", 771 exp.DataType.Type.ENUM: "Enum", 772 exp.DataType.Type.ENUM8: "Enum8", 773 exp.DataType.Type.ENUM16: "Enum16", 774 exp.DataType.Type.FIXEDSTRING: "FixedString", 775 exp.DataType.Type.FLOAT: "Float32", 776 exp.DataType.Type.INT: "Int32", 777 exp.DataType.Type.MEDIUMINT: "Int32", 778 exp.DataType.Type.INT128: "Int128", 779 exp.DataType.Type.INT256: "Int256", 780 exp.DataType.Type.LOWCARDINALITY: "LowCardinality", 781 exp.DataType.Type.MAP: "Map", 782 exp.DataType.Type.NESTED: "Nested", 783 exp.DataType.Type.NULLABLE: "Nullable", 784 exp.DataType.Type.SMALLINT: "Int16", 785 exp.DataType.Type.STRUCT: "Tuple", 786 exp.DataType.Type.TINYINT: "Int8", 787 exp.DataType.Type.UBIGINT: "UInt64", 788 exp.DataType.Type.UINT: "UInt32", 789 exp.DataType.Type.UINT128: "UInt128", 790 exp.DataType.Type.UINT256: "UInt256", 791 exp.DataType.Type.USMALLINT: "UInt16", 792 exp.DataType.Type.UTINYINT: "UInt8", 793 exp.DataType.Type.IPV4: "IPv4", 794 exp.DataType.Type.IPV6: "IPv6", 795 exp.DataType.Type.AGGREGATEFUNCTION: "AggregateFunction", 796 exp.DataType.Type.SIMPLEAGGREGATEFUNCTION: "SimpleAggregateFunction", 797 } 798 799 TRANSFORMS = { 800 **generator.Generator.TRANSFORMS, 801 exp.AnyValue: rename_func("any"), 802 exp.ApproxDistinct: rename_func("uniq"), 803 exp.ArrayFilter: lambda self, e: self.func("arrayFilter", e.expression, e.this), 804 exp.ArraySize: rename_func("LENGTH"), 805 exp.ArraySum: rename_func("arraySum"), 806 exp.ArgMax: arg_max_or_min_no_count("argMax"), 807 exp.ArgMin: arg_max_or_min_no_count("argMin"), 808 exp.Array: inline_array_sql, 809 exp.CastToStrType: rename_func("CAST"), 810 exp.CountIf: rename_func("countIf"), 811 exp.CompressColumnConstraint: lambda self, 812 e: f"CODEC({self.expressions(e, key='this', flat=True)})", 813 exp.ComputedColumnConstraint: lambda self, 814 e: f"{'MATERIALIZED' if e.args.get('persisted') else 'ALIAS'} {self.sql(e, 'this')}", 815 exp.CurrentDate: lambda self, e: self.func("CURRENT_DATE"), 816 exp.DateAdd: _datetime_delta_sql("DATE_ADD"), 817 exp.DateDiff: _datetime_delta_sql("DATE_DIFF"), 818 exp.DateStrToDate: rename_func("toDate"), 819 exp.DateSub: _datetime_delta_sql("DATE_SUB"), 820 exp.Explode: rename_func("arrayJoin"), 821 exp.Final: lambda self, e: f"{self.sql(e, 'this')} FINAL", 822 exp.IsNan: rename_func("isNaN"), 823 exp.JSONExtract: json_extract_segments("JSONExtractString", quoted_index=False), 824 exp.JSONExtractScalar: json_extract_segments("JSONExtractString", quoted_index=False), 825 exp.JSONPathKey: json_path_key_only_name, 826 exp.JSONPathRoot: lambda *_: "", 827 exp.Map: lambda self, e: _lower_func(var_map_sql(self, e)), 828 exp.Nullif: rename_func("nullIf"), 829 exp.PartitionedByProperty: lambda self, e: f"PARTITION BY {self.sql(e, 'this')}", 830 exp.Pivot: no_pivot_sql, 831 exp.Quantile: _quantile_sql, 832 exp.RegexpLike: lambda self, e: self.func("match", e.this, e.expression), 833 exp.Rand: rename_func("randCanonical"), 834 exp.StartsWith: rename_func("startsWith"), 835 exp.StrPosition: lambda self, e: self.func( 836 "position", e.this, e.args.get("substr"), e.args.get("position") 837 ), 838 exp.TimeToStr: lambda self, e: self.func( 839 "DATE_FORMAT", e.this, self.format_time(e), e.args.get("timezone") 840 ), 841 exp.TimestampAdd: _datetime_delta_sql("TIMESTAMP_ADD"), 842 exp.TimestampSub: _datetime_delta_sql("TIMESTAMP_SUB"), 843 exp.VarMap: lambda self, e: _lower_func(var_map_sql(self, e)), 844 exp.Xor: lambda self, e: self.func("xor", e.this, e.expression, *e.expressions), 845 exp.MD5Digest: rename_func("MD5"), 846 exp.MD5: lambda self, e: self.func("LOWER", self.func("HEX", self.func("MD5", e.this))), 847 exp.SHA: rename_func("SHA1"), 848 exp.SHA2: sha256_sql, 849 exp.UnixToTime: _unix_to_time_sql, 850 exp.TimestampTrunc: timestamptrunc_sql(zone=True), 851 exp.Variance: rename_func("varSamp"), 852 exp.SchemaCommentProperty: lambda self, e: self.naked_property(e), 853 exp.Stddev: rename_func("stddevSamp"), 854 } 855 856 PROPERTIES_LOCATION = { 857 **generator.Generator.PROPERTIES_LOCATION, 858 exp.VolatileProperty: exp.Properties.Location.UNSUPPORTED, 859 exp.PartitionedByProperty: exp.Properties.Location.POST_SCHEMA, 860 exp.OnCluster: exp.Properties.Location.POST_NAME, 861 } 862 863 # There's no list in docs, but it can be found in Clickhouse code 864 # see `ClickHouse/src/Parsers/ParserCreate*.cpp` 865 ON_CLUSTER_TARGETS = { 866 "DATABASE", 867 "TABLE", 868 "VIEW", 869 "DICTIONARY", 870 "INDEX", 871 "FUNCTION", 872 "NAMED COLLECTION", 873 } 874 875 # https://clickhouse.com/docs/en/sql-reference/data-types/nullable 876 NON_NULLABLE_TYPES = { 877 exp.DataType.Type.ARRAY, 878 exp.DataType.Type.MAP, 879 exp.DataType.Type.NULLABLE, 880 exp.DataType.Type.STRUCT, 881 } 882 883 def strtodate_sql(self, expression: exp.StrToDate) -> str: 884 strtodate_sql = self.function_fallback_sql(expression) 885 886 if not isinstance(expression.parent, exp.Cast): 887 # StrToDate returns DATEs in other dialects (eg. postgres), so 888 # this branch aims to improve the transpilation to clickhouse 889 return f"CAST({strtodate_sql} AS DATE)" 890 891 return strtodate_sql 892 893 def cast_sql(self, expression: exp.Cast, safe_prefix: t.Optional[str] = None) -> str: 894 this = expression.this 895 896 if isinstance(this, exp.StrToDate) and expression.to == exp.DataType.build("datetime"): 897 return self.sql(this) 898 899 return super().cast_sql(expression, safe_prefix=safe_prefix) 900 901 def trycast_sql(self, expression: exp.TryCast) -> str: 902 dtype = expression.to 903 if not dtype.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True): 904 # Casting x into Nullable(T) appears to behave similarly to TRY_CAST(x AS T) 905 dtype.set("nullable", True) 906 907 return super().cast_sql(expression) 908 909 def _jsonpathsubscript_sql(self, expression: exp.JSONPathSubscript) -> str: 910 this = self.json_path_part(expression.this) 911 return str(int(this) + 1) if is_int(this) else this 912 913 def likeproperty_sql(self, expression: exp.LikeProperty) -> str: 914 return f"AS {self.sql(expression, 'this')}" 915 916 def _any_to_has( 917 self, 918 expression: exp.EQ | exp.NEQ, 919 default: t.Callable[[t.Any], str], 920 prefix: str = "", 921 ) -> str: 922 if isinstance(expression.left, exp.Any): 923 arr = expression.left 924 this = expression.right 925 elif isinstance(expression.right, exp.Any): 926 arr = expression.right 927 this = expression.left 928 else: 929 return default(expression) 930 931 return prefix + self.func("has", arr.this.unnest(), this) 932 933 def eq_sql(self, expression: exp.EQ) -> str: 934 return self._any_to_has(expression, super().eq_sql) 935 936 def neq_sql(self, expression: exp.NEQ) -> str: 937 return self._any_to_has(expression, super().neq_sql, "NOT ") 938 939 def regexpilike_sql(self, expression: exp.RegexpILike) -> str: 940 # Manually add a flag to make the search case-insensitive 941 regex = self.func("CONCAT", "'(?i)'", expression.expression) 942 return self.func("match", expression.this, regex) 943 944 def datatype_sql(self, expression: exp.DataType) -> str: 945 # String is the standard ClickHouse type, every other variant is just an alias. 946 # Additionally, any supplied length parameter will be ignored. 947 # 948 # https://clickhouse.com/docs/en/sql-reference/data-types/string 949 if expression.this in self.STRING_TYPE_MAPPING: 950 dtype = "String" 951 else: 952 dtype = super().datatype_sql(expression) 953 954 # This section changes the type to `Nullable(...)` if the following conditions hold: 955 # - It's marked as nullable - this ensures we won't wrap ClickHouse types with `Nullable` 956 # and change their semantics 957 # - It's not the key type of a `Map`. This is because ClickHouse enforces the following 958 # constraint: "Type of Map key must be a type, that can be represented by integer or 959 # String or FixedString (possibly LowCardinality) or UUID or IPv6" 960 # - It's not a composite type, e.g. `Nullable(Array(...))` is not a valid type 961 parent = expression.parent 962 if ( 963 expression.args.get("nullable") is not False 964 and not ( 965 isinstance(parent, exp.DataType) 966 and parent.is_type(exp.DataType.Type.MAP, check_nullable=True) 967 and expression.index in (None, 0) 968 ) 969 and not expression.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True) 970 ): 971 dtype = f"Nullable({dtype})" 972 973 return dtype 974 975 def cte_sql(self, expression: exp.CTE) -> str: 976 if expression.args.get("scalar"): 977 this = self.sql(expression, "this") 978 alias = self.sql(expression, "alias") 979 return f"{this} AS {alias}" 980 981 return super().cte_sql(expression) 982 983 def after_limit_modifiers(self, expression: exp.Expression) -> t.List[str]: 984 return super().after_limit_modifiers(expression) + [ 985 ( 986 self.seg("SETTINGS ") + self.expressions(expression, key="settings", flat=True) 987 if expression.args.get("settings") 988 else "" 989 ), 990 ( 991 self.seg("FORMAT ") + self.sql(expression, "format") 992 if expression.args.get("format") 993 else "" 994 ), 995 ] 996 997 def parameterizedagg_sql(self, expression: exp.ParameterizedAgg) -> str: 998 params = self.expressions(expression, key="params", flat=True) 999 return self.func(expression.name, *expression.expressions) + f"({params})" 1000 1001 def anonymousaggfunc_sql(self, expression: exp.AnonymousAggFunc) -> str: 1002 return self.func(expression.name, *expression.expressions) 1003 1004 def combinedaggfunc_sql(self, expression: exp.CombinedAggFunc) -> str: 1005 return self.anonymousaggfunc_sql(expression) 1006 1007 def combinedparameterizedagg_sql(self, expression: exp.CombinedParameterizedAgg) -> str: 1008 return self.parameterizedagg_sql(expression) 1009 1010 def placeholder_sql(self, expression: exp.Placeholder) -> str: 1011 return f"{{{expression.name}: {self.sql(expression, 'kind')}}}" 1012 1013 def oncluster_sql(self, expression: exp.OnCluster) -> str: 1014 return f"ON CLUSTER {self.sql(expression, 'this')}" 1015 1016 def createable_sql(self, expression: exp.Create, locations: t.DefaultDict) -> str: 1017 if expression.kind in self.ON_CLUSTER_TARGETS and locations.get( 1018 exp.Properties.Location.POST_NAME 1019 ): 1020 this_name = self.sql( 1021 expression.this if isinstance(expression.this, exp.Schema) else expression, 1022 "this", 1023 ) 1024 this_properties = " ".join( 1025 [self.sql(prop) for prop in locations[exp.Properties.Location.POST_NAME]] 1026 ) 1027 this_schema = self.schema_columns_sql(expression.this) 1028 return f"{this_name}{self.sep()}{this_properties}{self.sep()}{this_schema}" 1029 1030 return super().createable_sql(expression, locations) 1031 1032 def create_sql(self, expression: exp.Create) -> str: 1033 # The comment property comes last in CTAS statements, i.e. after the query 1034 query = expression.expression 1035 if isinstance(query, exp.Query): 1036 comment_prop = expression.find(exp.SchemaCommentProperty) 1037 if comment_prop: 1038 comment_prop.pop() 1039 query.replace(exp.paren(query)) 1040 else: 1041 comment_prop = None 1042 1043 create_sql = super().create_sql(expression) 1044 1045 comment_sql = self.sql(comment_prop) 1046 comment_sql = f" {comment_sql}" if comment_sql else "" 1047 1048 return f"{create_sql}{comment_sql}" 1049 1050 def prewhere_sql(self, expression: exp.PreWhere) -> str: 1051 this = self.indent(self.sql(expression, "this")) 1052 return f"{self.seg('PREWHERE')}{self.sep()}{this}" 1053 1054 def indexcolumnconstraint_sql(self, expression: exp.IndexColumnConstraint) -> str: 1055 this = self.sql(expression, "this") 1056 this = f" {this}" if this else "" 1057 expr = self.sql(expression, "expression") 1058 expr = f" {expr}" if expr else "" 1059 index_type = self.sql(expression, "index_type") 1060 index_type = f" TYPE {index_type}" if index_type else "" 1061 granularity = self.sql(expression, "granularity") 1062 granularity = f" GRANULARITY {granularity}" if granularity else "" 1063 1064 return f"INDEX{this}{expr}{index_type}{granularity}" 1065 1066 def partition_sql(self, expression: exp.Partition) -> str: 1067 return f"PARTITION {self.expressions(expression, flat=True)}" 1068 1069 def partitionid_sql(self, expression: exp.PartitionId) -> str: 1070 return f"ID {self.sql(expression.this)}" 1071 1072 def replacepartition_sql(self, expression: exp.ReplacePartition) -> str: 1073 return ( 1074 f"REPLACE {self.sql(expression.expression)} FROM {self.sql(expression, 'source')}" 1075 ) 1076 1077 def projectiondef_sql(self, expression: exp.ProjectionDef) -> str: 1078 return f"PROJECTION {self.sql(expression.this)} {self.wrap(expression.expression)}"
Determines how function names are going to be normalized.
Possible values:
"upper" or True: Convert names to uppercase. "lower": Convert names to lowercase. False: Disables function name normalization.
Default NULL
ordering method to use if not explicitly set.
Possible values: "nulls_are_small"
, "nulls_are_large"
, "nulls_are_last"
Whether the base comes first in the LOG
function.
Possible values: True
, False
, None
(two arguments are not supported by LOG
)
Whether alias reference expansion (_expand_alias_refs()) should run before column qualification (_qualify_columns()).
For example:
WITH data AS ( SELECT 1 AS id, 2 AS my_id ) SELECT id AS my_id FROM data WHERE my_id = 1 GROUP BY my_id, HAVING my_id = 1
In most dialects "my_id" would refer to "data.my_id" (which is done in _qualify_columns()) across the query, except: - BigQuery, which will forward the alias to GROUP BY + HAVING clauses i.e it resolves to "WHERE my_id = 1 GROUP BY id HAVING id = 1" - Clickhouse, which will forward the alias across the query i.e it resolves to "WHERE id = 1 GROUP BY id HAVING id = 1"
Specifies the strategy according to which identifiers should be normalized.
Mapping of an escaped sequence (\n
) to its unescaped version (
).
Helper for dialects that use a different name for the same creatable kind. For example, the Clickhouse equivalent of CREATE SCHEMA is CREATE DATABASE.
Inherited Members
- sqlglot.dialects.dialect.Dialect
- Dialect
- INDEX_OFFSET
- WEEK_OFFSET
- UNNEST_COLUMN_ONLY
- ALIAS_POST_TABLESAMPLE
- TABLESAMPLE_SIZE_IS_PERCENT
- IDENTIFIERS_CAN_START_WITH_DIGIT
- DPIPE_IS_STRING_CONCAT
- STRICT_STRING_CONCAT
- SUPPORTS_SEMI_ANTI_JOIN
- COPY_PARAMS_ARE_CSV
- TYPED_DIVISION
- CONCAT_COALESCE
- HEX_LOWERCASE
- DATE_FORMAT
- DATEINT_FORMAT
- TIME_FORMAT
- TIME_MAPPING
- FORMAT_MAPPING
- PSEUDOCOLUMNS
- PREFER_CTE_ALIAS_COLUMN
- EXPAND_ALIAS_REFS_EARLY_ONLY_IN_GROUP_BY
- SUPPORTS_ORDER_BY_ALL
- HAS_DISTINCT_ARRAY_CONSTRUCTORS
- SUPPORTS_FIXED_SIZE_ARRAYS
- DATE_PART_MAPPING
- TYPE_TO_EXPRESSIONS
- ANNOTATORS
- get_or_raise
- format_time
- settings
- normalize_identifier
- case_sensitive
- can_identify
- quote_identifier
- to_json_path
- parse
- parse_into
- generate
- transpile
- tokenize
- tokenizer
- jsonpath_tokenizer
- parser
- generator
126 class Tokenizer(tokens.Tokenizer): 127 COMMENTS = ["--", "#", "#!", ("/*", "*/")] 128 IDENTIFIERS = ['"', "`"] 129 STRING_ESCAPES = ["'", "\\"] 130 BIT_STRINGS = [("0b", "")] 131 HEX_STRINGS = [("0x", ""), ("0X", "")] 132 HEREDOC_STRINGS = ["$"] 133 134 KEYWORDS = { 135 **tokens.Tokenizer.KEYWORDS, 136 "ATTACH": TokenType.COMMAND, 137 "DATE32": TokenType.DATE32, 138 "DATETIME64": TokenType.DATETIME64, 139 "DICTIONARY": TokenType.DICTIONARY, 140 "ENUM8": TokenType.ENUM8, 141 "ENUM16": TokenType.ENUM16, 142 "FINAL": TokenType.FINAL, 143 "FIXEDSTRING": TokenType.FIXEDSTRING, 144 "FLOAT32": TokenType.FLOAT, 145 "FLOAT64": TokenType.DOUBLE, 146 "GLOBAL": TokenType.GLOBAL, 147 "INT256": TokenType.INT256, 148 "LOWCARDINALITY": TokenType.LOWCARDINALITY, 149 "MAP": TokenType.MAP, 150 "NESTED": TokenType.NESTED, 151 "SAMPLE": TokenType.TABLE_SAMPLE, 152 "TUPLE": TokenType.STRUCT, 153 "UINT128": TokenType.UINT128, 154 "UINT16": TokenType.USMALLINT, 155 "UINT256": TokenType.UINT256, 156 "UINT32": TokenType.UINT, 157 "UINT64": TokenType.UBIGINT, 158 "UINT8": TokenType.UTINYINT, 159 "IPV4": TokenType.IPV4, 160 "IPV6": TokenType.IPV6, 161 "AGGREGATEFUNCTION": TokenType.AGGREGATEFUNCTION, 162 "SIMPLEAGGREGATEFUNCTION": TokenType.SIMPLEAGGREGATEFUNCTION, 163 "SYSTEM": TokenType.COMMAND, 164 "PREWHERE": TokenType.PREWHERE, 165 } 166 KEYWORDS.pop("/*+") 167 168 SINGLE_TOKENS = { 169 **tokens.Tokenizer.SINGLE_TOKENS, 170 "$": TokenType.HEREDOC_STRING, 171 }
Inherited Members
- sqlglot.tokens.Tokenizer
- Tokenizer
- BYTE_STRINGS
- RAW_STRINGS
- UNICODE_STRINGS
- IDENTIFIER_ESCAPES
- QUOTES
- VAR_SINGLE_TOKENS
- HEREDOC_TAG_IS_IDENTIFIER
- HEREDOC_STRING_ALTERNATIVE
- STRING_ESCAPES_ALLOWED_IN_RAW_STRINGS
- WHITE_SPACE
- COMMANDS
- COMMAND_PREFIX_TOKENS
- NUMERIC_LITERALS
- dialect
- reset
- tokenize
- tokenize_rs
- size
- sql
- tokens
173 class Parser(parser.Parser): 174 # Tested in ClickHouse's playground, it seems that the following two queries do the same thing 175 # * select x from t1 union all select x from t2 limit 1; 176 # * select x from t1 union all (select x from t2 limit 1); 177 MODIFIERS_ATTACHED_TO_SET_OP = False 178 INTERVAL_SPANS = False 179 180 FUNCTIONS = { 181 **parser.Parser.FUNCTIONS, 182 "ANY": exp.AnyValue.from_arg_list, 183 "ARRAYSUM": exp.ArraySum.from_arg_list, 184 "COUNTIF": _build_count_if, 185 "DATE_ADD": build_date_delta(exp.DateAdd, default_unit=None), 186 "DATEADD": build_date_delta(exp.DateAdd, default_unit=None), 187 "DATE_DIFF": build_date_delta(exp.DateDiff, default_unit=None), 188 "DATEDIFF": build_date_delta(exp.DateDiff, default_unit=None), 189 "DATE_FORMAT": _build_date_format, 190 "DATE_SUB": build_date_delta(exp.DateSub, default_unit=None), 191 "DATESUB": build_date_delta(exp.DateSub, default_unit=None), 192 "FORMATDATETIME": _build_date_format, 193 "JSONEXTRACTSTRING": build_json_extract_path( 194 exp.JSONExtractScalar, zero_based_indexing=False 195 ), 196 "MAP": parser.build_var_map, 197 "MATCH": exp.RegexpLike.from_arg_list, 198 "RANDCANONICAL": exp.Rand.from_arg_list, 199 "STR_TO_DATE": _build_str_to_date, 200 "TUPLE": exp.Struct.from_arg_list, 201 "TIMESTAMP_SUB": build_date_delta(exp.TimestampSub, default_unit=None), 202 "TIMESTAMPSUB": build_date_delta(exp.TimestampSub, default_unit=None), 203 "TIMESTAMP_ADD": build_date_delta(exp.TimestampAdd, default_unit=None), 204 "TIMESTAMPADD": build_date_delta(exp.TimestampAdd, default_unit=None), 205 "UNIQ": exp.ApproxDistinct.from_arg_list, 206 "XOR": lambda args: exp.Xor(expressions=args), 207 "MD5": exp.MD5Digest.from_arg_list, 208 "SHA256": lambda args: exp.SHA2(this=seq_get(args, 0), length=exp.Literal.number(256)), 209 "SHA512": lambda args: exp.SHA2(this=seq_get(args, 0), length=exp.Literal.number(512)), 210 } 211 212 AGG_FUNCTIONS = { 213 "count", 214 "min", 215 "max", 216 "sum", 217 "avg", 218 "any", 219 "stddevPop", 220 "stddevSamp", 221 "varPop", 222 "varSamp", 223 "corr", 224 "covarPop", 225 "covarSamp", 226 "entropy", 227 "exponentialMovingAverage", 228 "intervalLengthSum", 229 "kolmogorovSmirnovTest", 230 "mannWhitneyUTest", 231 "median", 232 "rankCorr", 233 "sumKahan", 234 "studentTTest", 235 "welchTTest", 236 "anyHeavy", 237 "anyLast", 238 "boundingRatio", 239 "first_value", 240 "last_value", 241 "argMin", 242 "argMax", 243 "avgWeighted", 244 "topK", 245 "topKWeighted", 246 "deltaSum", 247 "deltaSumTimestamp", 248 "groupArray", 249 "groupArrayLast", 250 "groupUniqArray", 251 "groupArrayInsertAt", 252 "groupArrayMovingAvg", 253 "groupArrayMovingSum", 254 "groupArraySample", 255 "groupBitAnd", 256 "groupBitOr", 257 "groupBitXor", 258 "groupBitmap", 259 "groupBitmapAnd", 260 "groupBitmapOr", 261 "groupBitmapXor", 262 "sumWithOverflow", 263 "sumMap", 264 "minMap", 265 "maxMap", 266 "skewSamp", 267 "skewPop", 268 "kurtSamp", 269 "kurtPop", 270 "uniq", 271 "uniqExact", 272 "uniqCombined", 273 "uniqCombined64", 274 "uniqHLL12", 275 "uniqTheta", 276 "quantile", 277 "quantiles", 278 "quantileExact", 279 "quantilesExact", 280 "quantileExactLow", 281 "quantilesExactLow", 282 "quantileExactHigh", 283 "quantilesExactHigh", 284 "quantileExactWeighted", 285 "quantilesExactWeighted", 286 "quantileTiming", 287 "quantilesTiming", 288 "quantileTimingWeighted", 289 "quantilesTimingWeighted", 290 "quantileDeterministic", 291 "quantilesDeterministic", 292 "quantileTDigest", 293 "quantilesTDigest", 294 "quantileTDigestWeighted", 295 "quantilesTDigestWeighted", 296 "quantileBFloat16", 297 "quantilesBFloat16", 298 "quantileBFloat16Weighted", 299 "quantilesBFloat16Weighted", 300 "simpleLinearRegression", 301 "stochasticLinearRegression", 302 "stochasticLogisticRegression", 303 "categoricalInformationValue", 304 "contingency", 305 "cramersV", 306 "cramersVBiasCorrected", 307 "theilsU", 308 "maxIntersections", 309 "maxIntersectionsPosition", 310 "meanZTest", 311 "quantileInterpolatedWeighted", 312 "quantilesInterpolatedWeighted", 313 "quantileGK", 314 "quantilesGK", 315 "sparkBar", 316 "sumCount", 317 "largestTriangleThreeBuckets", 318 "histogram", 319 "sequenceMatch", 320 "sequenceCount", 321 "windowFunnel", 322 "retention", 323 "uniqUpTo", 324 "sequenceNextNode", 325 "exponentialTimeDecayedAvg", 326 } 327 328 AGG_FUNCTIONS_SUFFIXES = [ 329 "If", 330 "Array", 331 "ArrayIf", 332 "Map", 333 "SimpleState", 334 "State", 335 "Merge", 336 "MergeState", 337 "ForEach", 338 "Distinct", 339 "OrDefault", 340 "OrNull", 341 "Resample", 342 "ArgMin", 343 "ArgMax", 344 ] 345 346 FUNC_TOKENS = { 347 *parser.Parser.FUNC_TOKENS, 348 TokenType.SET, 349 } 350 351 RESERVED_TOKENS = parser.Parser.RESERVED_TOKENS - {TokenType.SELECT} 352 353 ID_VAR_TOKENS = { 354 *parser.Parser.ID_VAR_TOKENS, 355 TokenType.LIKE, 356 } 357 358 AGG_FUNC_MAPPING = ( 359 lambda functions, suffixes: { 360 f"{f}{sfx}": (f, sfx) for sfx in (suffixes + [""]) for f in functions 361 } 362 )(AGG_FUNCTIONS, AGG_FUNCTIONS_SUFFIXES) 363 364 FUNCTIONS_WITH_ALIASED_ARGS = {*parser.Parser.FUNCTIONS_WITH_ALIASED_ARGS, "TUPLE"} 365 366 FUNCTION_PARSERS = { 367 **parser.Parser.FUNCTION_PARSERS, 368 "ARRAYJOIN": lambda self: self.expression(exp.Explode, this=self._parse_expression()), 369 "QUANTILE": lambda self: self._parse_quantile(), 370 } 371 372 FUNCTION_PARSERS.pop("MATCH") 373 374 NO_PAREN_FUNCTION_PARSERS = parser.Parser.NO_PAREN_FUNCTION_PARSERS.copy() 375 NO_PAREN_FUNCTION_PARSERS.pop("ANY") 376 377 RANGE_PARSERS = { 378 **parser.Parser.RANGE_PARSERS, 379 TokenType.GLOBAL: lambda self, this: self._match(TokenType.IN) 380 and self._parse_in(this, is_global=True), 381 } 382 383 # The PLACEHOLDER entry is popped because 1) it doesn't affect Clickhouse (it corresponds to 384 # the postgres-specific JSONBContains parser) and 2) it makes parsing the ternary op simpler. 385 COLUMN_OPERATORS = parser.Parser.COLUMN_OPERATORS.copy() 386 COLUMN_OPERATORS.pop(TokenType.PLACEHOLDER) 387 388 JOIN_KINDS = { 389 *parser.Parser.JOIN_KINDS, 390 TokenType.ANY, 391 TokenType.ASOF, 392 TokenType.ARRAY, 393 } 394 395 TABLE_ALIAS_TOKENS = parser.Parser.TABLE_ALIAS_TOKENS - { 396 TokenType.ANY, 397 TokenType.ARRAY, 398 TokenType.FINAL, 399 TokenType.FORMAT, 400 TokenType.SETTINGS, 401 } 402 403 ALIAS_TOKENS = parser.Parser.ALIAS_TOKENS - { 404 TokenType.FORMAT, 405 } 406 407 LOG_DEFAULTS_TO_LN = True 408 409 QUERY_MODIFIER_PARSERS = { 410 **parser.Parser.QUERY_MODIFIER_PARSERS, 411 TokenType.SETTINGS: lambda self: ( 412 "settings", 413 self._advance() or self._parse_csv(self._parse_assignment), 414 ), 415 TokenType.FORMAT: lambda self: ("format", self._advance() or self._parse_id_var()), 416 } 417 418 CONSTRAINT_PARSERS = { 419 **parser.Parser.CONSTRAINT_PARSERS, 420 "INDEX": lambda self: self._parse_index_constraint(), 421 "CODEC": lambda self: self._parse_compress(), 422 } 423 424 ALTER_PARSERS = { 425 **parser.Parser.ALTER_PARSERS, 426 "REPLACE": lambda self: self._parse_alter_table_replace(), 427 } 428 429 SCHEMA_UNNAMED_CONSTRAINTS = { 430 *parser.Parser.SCHEMA_UNNAMED_CONSTRAINTS, 431 "INDEX", 432 } 433 434 def _parse_types( 435 self, check_func: bool = False, schema: bool = False, allow_identifiers: bool = True 436 ) -> t.Optional[exp.Expression]: 437 dtype = super()._parse_types( 438 check_func=check_func, schema=schema, allow_identifiers=allow_identifiers 439 ) 440 if isinstance(dtype, exp.DataType): 441 # Mark every type as non-nullable which is ClickHouse's default. This marker 442 # helps us transpile types from other dialects to ClickHouse, so that we can 443 # e.g. produce `CAST(x AS Nullable(String))` from `CAST(x AS TEXT)`. If there 444 # is a `NULL` value in `x`, the former would fail in ClickHouse without the 445 # `Nullable` type constructor 446 dtype.set("nullable", False) 447 448 return dtype 449 450 def _parse_extract(self) -> exp.Extract | exp.Anonymous: 451 index = self._index 452 this = self._parse_bitwise() 453 if self._match(TokenType.FROM): 454 self._retreat(index) 455 return super()._parse_extract() 456 457 # We return Anonymous here because extract and regexpExtract have different semantics, 458 # so parsing extract(foo, bar) into RegexpExtract can potentially break queries. E.g., 459 # `extract('foobar', 'b')` works, but ClickHouse crashes for `regexpExtract('foobar', 'b')`. 460 # 461 # TODO: can we somehow convert the former into an equivalent `regexpExtract` call? 462 self._match(TokenType.COMMA) 463 return self.expression( 464 exp.Anonymous, this="extract", expressions=[this, self._parse_bitwise()] 465 ) 466 467 def _parse_assignment(self) -> t.Optional[exp.Expression]: 468 this = super()._parse_assignment() 469 470 if self._match(TokenType.PLACEHOLDER): 471 return self.expression( 472 exp.If, 473 this=this, 474 true=self._parse_assignment(), 475 false=self._match(TokenType.COLON) and self._parse_assignment(), 476 ) 477 478 return this 479 480 def _parse_placeholder(self) -> t.Optional[exp.Expression]: 481 """ 482 Parse a placeholder expression like SELECT {abc: UInt32} or FROM {table: Identifier} 483 https://clickhouse.com/docs/en/sql-reference/syntax#defining-and-using-query-parameters 484 """ 485 if not self._match(TokenType.L_BRACE): 486 return None 487 488 this = self._parse_id_var() 489 self._match(TokenType.COLON) 490 kind = self._parse_types(check_func=False, allow_identifiers=False) or ( 491 self._match_text_seq("IDENTIFIER") and "Identifier" 492 ) 493 494 if not kind: 495 self.raise_error("Expecting a placeholder type or 'Identifier' for tables") 496 elif not self._match(TokenType.R_BRACE): 497 self.raise_error("Expecting }") 498 499 return self.expression(exp.Placeholder, this=this, kind=kind) 500 501 def _parse_in(self, this: t.Optional[exp.Expression], is_global: bool = False) -> exp.In: 502 this = super()._parse_in(this) 503 this.set("is_global", is_global) 504 return this 505 506 def _parse_table( 507 self, 508 schema: bool = False, 509 joins: bool = False, 510 alias_tokens: t.Optional[t.Collection[TokenType]] = None, 511 parse_bracket: bool = False, 512 is_db_reference: bool = False, 513 parse_partition: bool = False, 514 ) -> t.Optional[exp.Expression]: 515 this = super()._parse_table( 516 schema=schema, 517 joins=joins, 518 alias_tokens=alias_tokens, 519 parse_bracket=parse_bracket, 520 is_db_reference=is_db_reference, 521 ) 522 523 if self._match(TokenType.FINAL): 524 this = self.expression(exp.Final, this=this) 525 526 return this 527 528 def _parse_position(self, haystack_first: bool = False) -> exp.StrPosition: 529 return super()._parse_position(haystack_first=True) 530 531 # https://clickhouse.com/docs/en/sql-reference/statements/select/with/ 532 def _parse_cte(self) -> exp.CTE: 533 # WITH <identifier> AS <subquery expression> 534 cte: t.Optional[exp.CTE] = self._try_parse(super()._parse_cte) 535 536 if not cte: 537 # WITH <expression> AS <identifier> 538 cte = self.expression( 539 exp.CTE, 540 this=self._parse_assignment(), 541 alias=self._parse_table_alias(), 542 scalar=True, 543 ) 544 545 return cte 546 547 def _parse_join_parts( 548 self, 549 ) -> t.Tuple[t.Optional[Token], t.Optional[Token], t.Optional[Token]]: 550 is_global = self._match(TokenType.GLOBAL) and self._prev 551 kind_pre = self._match_set(self.JOIN_KINDS, advance=False) and self._prev 552 553 if kind_pre: 554 kind = self._match_set(self.JOIN_KINDS) and self._prev 555 side = self._match_set(self.JOIN_SIDES) and self._prev 556 return is_global, side, kind 557 558 return ( 559 is_global, 560 self._match_set(self.JOIN_SIDES) and self._prev, 561 self._match_set(self.JOIN_KINDS) and self._prev, 562 ) 563 564 def _parse_join( 565 self, skip_join_token: bool = False, parse_bracket: bool = False 566 ) -> t.Optional[exp.Join]: 567 join = super()._parse_join(skip_join_token=skip_join_token, parse_bracket=True) 568 if join: 569 join.set("global", join.args.pop("method", None)) 570 571 return join 572 573 def _parse_function( 574 self, 575 functions: t.Optional[t.Dict[str, t.Callable]] = None, 576 anonymous: bool = False, 577 optional_parens: bool = True, 578 any_token: bool = False, 579 ) -> t.Optional[exp.Expression]: 580 expr = super()._parse_function( 581 functions=functions, 582 anonymous=anonymous, 583 optional_parens=optional_parens, 584 any_token=any_token, 585 ) 586 587 func = expr.this if isinstance(expr, exp.Window) else expr 588 589 # Aggregate functions can be split in 2 parts: <func_name><suffix> 590 parts = ( 591 self.AGG_FUNC_MAPPING.get(func.this) if isinstance(func, exp.Anonymous) else None 592 ) 593 594 if parts: 595 params = self._parse_func_params(func) 596 597 kwargs = { 598 "this": func.this, 599 "expressions": func.expressions, 600 } 601 if parts[1]: 602 kwargs["parts"] = parts 603 exp_class = exp.CombinedParameterizedAgg if params else exp.CombinedAggFunc 604 else: 605 exp_class = exp.ParameterizedAgg if params else exp.AnonymousAggFunc 606 607 kwargs["exp_class"] = exp_class 608 if params: 609 kwargs["params"] = params 610 611 func = self.expression(**kwargs) 612 613 if isinstance(expr, exp.Window): 614 # The window's func was parsed as Anonymous in base parser, fix its 615 # type to be ClickHouse style CombinedAnonymousAggFunc / AnonymousAggFunc 616 expr.set("this", func) 617 elif params: 618 # Params have blocked super()._parse_function() from parsing the following window 619 # (if that exists) as they're standing between the function call and the window spec 620 expr = self._parse_window(func) 621 else: 622 expr = func 623 624 return expr 625 626 def _parse_func_params( 627 self, this: t.Optional[exp.Func] = None 628 ) -> t.Optional[t.List[exp.Expression]]: 629 if self._match_pair(TokenType.R_PAREN, TokenType.L_PAREN): 630 return self._parse_csv(self._parse_lambda) 631 632 if self._match(TokenType.L_PAREN): 633 params = self._parse_csv(self._parse_lambda) 634 self._match_r_paren(this) 635 return params 636 637 return None 638 639 def _parse_quantile(self) -> exp.Quantile: 640 this = self._parse_lambda() 641 params = self._parse_func_params() 642 if params: 643 return self.expression(exp.Quantile, this=params[0], quantile=this) 644 return self.expression(exp.Quantile, this=this, quantile=exp.Literal.number(0.5)) 645 646 def _parse_wrapped_id_vars(self, optional: bool = False) -> t.List[exp.Expression]: 647 return super()._parse_wrapped_id_vars(optional=True) 648 649 def _parse_primary_key( 650 self, wrapped_optional: bool = False, in_props: bool = False 651 ) -> exp.PrimaryKeyColumnConstraint | exp.PrimaryKey: 652 return super()._parse_primary_key( 653 wrapped_optional=wrapped_optional or in_props, in_props=in_props 654 ) 655 656 def _parse_on_property(self) -> t.Optional[exp.Expression]: 657 index = self._index 658 if self._match_text_seq("CLUSTER"): 659 this = self._parse_id_var() 660 if this: 661 return self.expression(exp.OnCluster, this=this) 662 else: 663 self._retreat(index) 664 return None 665 666 def _parse_index_constraint( 667 self, kind: t.Optional[str] = None 668 ) -> exp.IndexColumnConstraint: 669 # INDEX name1 expr TYPE type1(args) GRANULARITY value 670 this = self._parse_id_var() 671 expression = self._parse_assignment() 672 673 index_type = self._match_text_seq("TYPE") and ( 674 self._parse_function() or self._parse_var() 675 ) 676 677 granularity = self._match_text_seq("GRANULARITY") and self._parse_term() 678 679 return self.expression( 680 exp.IndexColumnConstraint, 681 this=this, 682 expression=expression, 683 index_type=index_type, 684 granularity=granularity, 685 ) 686 687 def _parse_partition(self) -> t.Optional[exp.Partition]: 688 # https://clickhouse.com/docs/en/sql-reference/statements/alter/partition#how-to-set-partition-expression 689 if not self._match(TokenType.PARTITION): 690 return None 691 692 if self._match_text_seq("ID"): 693 # Corresponds to the PARTITION ID <string_value> syntax 694 expressions: t.List[exp.Expression] = [ 695 self.expression(exp.PartitionId, this=self._parse_string()) 696 ] 697 else: 698 expressions = self._parse_expressions() 699 700 return self.expression(exp.Partition, expressions=expressions) 701 702 def _parse_alter_table_replace(self) -> t.Optional[exp.Expression]: 703 partition = self._parse_partition() 704 705 if not partition or not self._match(TokenType.FROM): 706 return None 707 708 return self.expression( 709 exp.ReplacePartition, expression=partition, source=self._parse_table_parts() 710 ) 711 712 def _parse_projection_def(self) -> t.Optional[exp.ProjectionDef]: 713 if not self._match_text_seq("PROJECTION"): 714 return None 715 716 return self.expression( 717 exp.ProjectionDef, 718 this=self._parse_id_var(), 719 expression=self._parse_wrapped(self._parse_statement), 720 ) 721 722 def _parse_constraint(self) -> t.Optional[exp.Expression]: 723 return super()._parse_constraint() or self._parse_projection_def()
Parser consumes a list of tokens produced by the Tokenizer and produces a parsed syntax tree.
Arguments:
- error_level: The desired error level. Default: ErrorLevel.IMMEDIATE
- error_message_context: The amount of context to capture from a query string when displaying the error message (in number of characters). Default: 100
- max_errors: Maximum number of error messages to include in a raised ParseError. This is only relevant if error_level is ErrorLevel.RAISE. Default: 3
Inherited Members
- sqlglot.parser.Parser
- Parser
- NO_PAREN_FUNCTIONS
- STRUCT_TYPE_TOKENS
- NESTED_TYPE_TOKENS
- ENUM_TYPE_TOKENS
- AGGREGATE_TYPE_TOKENS
- TYPE_TOKENS
- SIGNED_TO_UNSIGNED_TYPE_TOKEN
- SUBQUERY_PREDICATES
- DB_CREATABLES
- CREATABLES
- ALTERABLES
- INTERVAL_VARS
- ARRAY_CONSTRUCTORS
- COMMENT_TABLE_ALIAS_TOKENS
- UPDATE_ALIAS_TOKENS
- TRIM_TYPES
- CONJUNCTION
- ASSIGNMENT
- DISJUNCTION
- EQUALITY
- COMPARISON
- BITWISE
- TERM
- FACTOR
- EXPONENT
- TIMES
- TIMESTAMPS
- SET_OPERATIONS
- JOIN_METHODS
- JOIN_SIDES
- JOIN_HINTS
- LAMBDAS
- EXPRESSION_PARSERS
- STATEMENT_PARSERS
- UNARY_PARSERS
- STRING_PARSERS
- NUMERIC_PARSERS
- PRIMARY_PARSERS
- PLACEHOLDER_PARSERS
- PROPERTY_PARSERS
- ALTER_ALTER_PARSERS
- INVALID_FUNC_NAME_TOKENS
- KEY_VALUE_DEFINITIONS
- SET_PARSERS
- SHOW_PARSERS
- TYPE_LITERAL_PARSERS
- TYPE_CONVERTERS
- DDL_SELECT_TOKENS
- PRE_VOLATILE_TOKENS
- TRANSACTION_KIND
- TRANSACTION_CHARACTERISTICS
- CONFLICT_ACTIONS
- CREATE_SEQUENCE
- ISOLATED_LOADING_OPTIONS
- USABLES
- CAST_ACTIONS
- SCHEMA_BINDING_OPTIONS
- KEY_CONSTRAINT_OPTIONS
- INSERT_ALTERNATIVES
- CLONE_KEYWORDS
- HISTORICAL_DATA_PREFIX
- HISTORICAL_DATA_KIND
- OPCLASS_FOLLOW_KEYWORDS
- OPTYPE_FOLLOW_TOKENS
- TABLE_INDEX_HINT_TOKENS
- VIEW_ATTRIBUTES
- WINDOW_ALIAS_TOKENS
- WINDOW_BEFORE_PAREN_TOKENS
- WINDOW_SIDES
- JSON_KEY_VALUE_SEPARATOR_TOKENS
- FETCH_TOKENS
- ADD_CONSTRAINT_TOKENS
- DISTINCT_TOKENS
- NULL_TOKENS
- UNNEST_OFFSET_ALIAS_TOKENS
- SELECT_START_TOKENS
- COPY_INTO_VARLEN_OPTIONS
- STRICT_CAST
- PREFIXED_PIVOT_COLUMNS
- IDENTIFY_PIVOT_STRINGS
- ALTER_TABLE_ADD_REQUIRED_FOR_EACH_COLUMN
- TABLESAMPLE_CSV
- DEFAULT_SAMPLING_METHOD
- SET_REQUIRES_ASSIGNMENT_DELIMITER
- TRIM_PATTERN_FIRST
- STRING_ALIASES
- SET_OP_MODIFIERS
- NO_PAREN_IF_COMMANDS
- JSON_ARROWS_REQUIRE_JSON_TYPE
- COLON_IS_VARIANT_EXTRACT
- VALUES_FOLLOWED_BY_PAREN
- SUPPORTS_IMPLICIT_UNNEST
- SUPPORTS_PARTITION_SELECTION
- error_level
- error_message_context
- max_errors
- dialect
- reset
- parse
- parse_into
- check_errors
- raise_error
- expression
- validate_expression
- errors
- sql
725 class Generator(generator.Generator): 726 QUERY_HINTS = False 727 STRUCT_DELIMITER = ("(", ")") 728 NVL2_SUPPORTED = False 729 TABLESAMPLE_REQUIRES_PARENS = False 730 TABLESAMPLE_SIZE_IS_ROWS = False 731 TABLESAMPLE_KEYWORDS = "SAMPLE" 732 LAST_DAY_SUPPORTS_DATE_PART = False 733 CAN_IMPLEMENT_ARRAY_ANY = True 734 SUPPORTS_TO_NUMBER = False 735 JOIN_HINTS = False 736 TABLE_HINTS = False 737 EXPLICIT_SET_OP = True 738 GROUPINGS_SEP = "" 739 SET_OP_MODIFIERS = False 740 SUPPORTS_TABLE_ALIAS_COLUMNS = False 741 VALUES_AS_TABLE = False 742 743 STRING_TYPE_MAPPING = { 744 exp.DataType.Type.CHAR: "String", 745 exp.DataType.Type.LONGBLOB: "String", 746 exp.DataType.Type.LONGTEXT: "String", 747 exp.DataType.Type.MEDIUMBLOB: "String", 748 exp.DataType.Type.MEDIUMTEXT: "String", 749 exp.DataType.Type.TINYBLOB: "String", 750 exp.DataType.Type.TINYTEXT: "String", 751 exp.DataType.Type.TEXT: "String", 752 exp.DataType.Type.VARBINARY: "String", 753 exp.DataType.Type.VARCHAR: "String", 754 } 755 756 SUPPORTED_JSON_PATH_PARTS = { 757 exp.JSONPathKey, 758 exp.JSONPathRoot, 759 exp.JSONPathSubscript, 760 } 761 762 TYPE_MAPPING = { 763 **generator.Generator.TYPE_MAPPING, 764 **STRING_TYPE_MAPPING, 765 exp.DataType.Type.ARRAY: "Array", 766 exp.DataType.Type.BIGINT: "Int64", 767 exp.DataType.Type.DATE32: "Date32", 768 exp.DataType.Type.DATETIME64: "DateTime64", 769 exp.DataType.Type.TIMESTAMPTZ: "TIMESTAMP", 770 exp.DataType.Type.DOUBLE: "Float64", 771 exp.DataType.Type.ENUM: "Enum", 772 exp.DataType.Type.ENUM8: "Enum8", 773 exp.DataType.Type.ENUM16: "Enum16", 774 exp.DataType.Type.FIXEDSTRING: "FixedString", 775 exp.DataType.Type.FLOAT: "Float32", 776 exp.DataType.Type.INT: "Int32", 777 exp.DataType.Type.MEDIUMINT: "Int32", 778 exp.DataType.Type.INT128: "Int128", 779 exp.DataType.Type.INT256: "Int256", 780 exp.DataType.Type.LOWCARDINALITY: "LowCardinality", 781 exp.DataType.Type.MAP: "Map", 782 exp.DataType.Type.NESTED: "Nested", 783 exp.DataType.Type.NULLABLE: "Nullable", 784 exp.DataType.Type.SMALLINT: "Int16", 785 exp.DataType.Type.STRUCT: "Tuple", 786 exp.DataType.Type.TINYINT: "Int8", 787 exp.DataType.Type.UBIGINT: "UInt64", 788 exp.DataType.Type.UINT: "UInt32", 789 exp.DataType.Type.UINT128: "UInt128", 790 exp.DataType.Type.UINT256: "UInt256", 791 exp.DataType.Type.USMALLINT: "UInt16", 792 exp.DataType.Type.UTINYINT: "UInt8", 793 exp.DataType.Type.IPV4: "IPv4", 794 exp.DataType.Type.IPV6: "IPv6", 795 exp.DataType.Type.AGGREGATEFUNCTION: "AggregateFunction", 796 exp.DataType.Type.SIMPLEAGGREGATEFUNCTION: "SimpleAggregateFunction", 797 } 798 799 TRANSFORMS = { 800 **generator.Generator.TRANSFORMS, 801 exp.AnyValue: rename_func("any"), 802 exp.ApproxDistinct: rename_func("uniq"), 803 exp.ArrayFilter: lambda self, e: self.func("arrayFilter", e.expression, e.this), 804 exp.ArraySize: rename_func("LENGTH"), 805 exp.ArraySum: rename_func("arraySum"), 806 exp.ArgMax: arg_max_or_min_no_count("argMax"), 807 exp.ArgMin: arg_max_or_min_no_count("argMin"), 808 exp.Array: inline_array_sql, 809 exp.CastToStrType: rename_func("CAST"), 810 exp.CountIf: rename_func("countIf"), 811 exp.CompressColumnConstraint: lambda self, 812 e: f"CODEC({self.expressions(e, key='this', flat=True)})", 813 exp.ComputedColumnConstraint: lambda self, 814 e: f"{'MATERIALIZED' if e.args.get('persisted') else 'ALIAS'} {self.sql(e, 'this')}", 815 exp.CurrentDate: lambda self, e: self.func("CURRENT_DATE"), 816 exp.DateAdd: _datetime_delta_sql("DATE_ADD"), 817 exp.DateDiff: _datetime_delta_sql("DATE_DIFF"), 818 exp.DateStrToDate: rename_func("toDate"), 819 exp.DateSub: _datetime_delta_sql("DATE_SUB"), 820 exp.Explode: rename_func("arrayJoin"), 821 exp.Final: lambda self, e: f"{self.sql(e, 'this')} FINAL", 822 exp.IsNan: rename_func("isNaN"), 823 exp.JSONExtract: json_extract_segments("JSONExtractString", quoted_index=False), 824 exp.JSONExtractScalar: json_extract_segments("JSONExtractString", quoted_index=False), 825 exp.JSONPathKey: json_path_key_only_name, 826 exp.JSONPathRoot: lambda *_: "", 827 exp.Map: lambda self, e: _lower_func(var_map_sql(self, e)), 828 exp.Nullif: rename_func("nullIf"), 829 exp.PartitionedByProperty: lambda self, e: f"PARTITION BY {self.sql(e, 'this')}", 830 exp.Pivot: no_pivot_sql, 831 exp.Quantile: _quantile_sql, 832 exp.RegexpLike: lambda self, e: self.func("match", e.this, e.expression), 833 exp.Rand: rename_func("randCanonical"), 834 exp.StartsWith: rename_func("startsWith"), 835 exp.StrPosition: lambda self, e: self.func( 836 "position", e.this, e.args.get("substr"), e.args.get("position") 837 ), 838 exp.TimeToStr: lambda self, e: self.func( 839 "DATE_FORMAT", e.this, self.format_time(e), e.args.get("timezone") 840 ), 841 exp.TimestampAdd: _datetime_delta_sql("TIMESTAMP_ADD"), 842 exp.TimestampSub: _datetime_delta_sql("TIMESTAMP_SUB"), 843 exp.VarMap: lambda self, e: _lower_func(var_map_sql(self, e)), 844 exp.Xor: lambda self, e: self.func("xor", e.this, e.expression, *e.expressions), 845 exp.MD5Digest: rename_func("MD5"), 846 exp.MD5: lambda self, e: self.func("LOWER", self.func("HEX", self.func("MD5", e.this))), 847 exp.SHA: rename_func("SHA1"), 848 exp.SHA2: sha256_sql, 849 exp.UnixToTime: _unix_to_time_sql, 850 exp.TimestampTrunc: timestamptrunc_sql(zone=True), 851 exp.Variance: rename_func("varSamp"), 852 exp.SchemaCommentProperty: lambda self, e: self.naked_property(e), 853 exp.Stddev: rename_func("stddevSamp"), 854 } 855 856 PROPERTIES_LOCATION = { 857 **generator.Generator.PROPERTIES_LOCATION, 858 exp.VolatileProperty: exp.Properties.Location.UNSUPPORTED, 859 exp.PartitionedByProperty: exp.Properties.Location.POST_SCHEMA, 860 exp.OnCluster: exp.Properties.Location.POST_NAME, 861 } 862 863 # There's no list in docs, but it can be found in Clickhouse code 864 # see `ClickHouse/src/Parsers/ParserCreate*.cpp` 865 ON_CLUSTER_TARGETS = { 866 "DATABASE", 867 "TABLE", 868 "VIEW", 869 "DICTIONARY", 870 "INDEX", 871 "FUNCTION", 872 "NAMED COLLECTION", 873 } 874 875 # https://clickhouse.com/docs/en/sql-reference/data-types/nullable 876 NON_NULLABLE_TYPES = { 877 exp.DataType.Type.ARRAY, 878 exp.DataType.Type.MAP, 879 exp.DataType.Type.NULLABLE, 880 exp.DataType.Type.STRUCT, 881 } 882 883 def strtodate_sql(self, expression: exp.StrToDate) -> str: 884 strtodate_sql = self.function_fallback_sql(expression) 885 886 if not isinstance(expression.parent, exp.Cast): 887 # StrToDate returns DATEs in other dialects (eg. postgres), so 888 # this branch aims to improve the transpilation to clickhouse 889 return f"CAST({strtodate_sql} AS DATE)" 890 891 return strtodate_sql 892 893 def cast_sql(self, expression: exp.Cast, safe_prefix: t.Optional[str] = None) -> str: 894 this = expression.this 895 896 if isinstance(this, exp.StrToDate) and expression.to == exp.DataType.build("datetime"): 897 return self.sql(this) 898 899 return super().cast_sql(expression, safe_prefix=safe_prefix) 900 901 def trycast_sql(self, expression: exp.TryCast) -> str: 902 dtype = expression.to 903 if not dtype.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True): 904 # Casting x into Nullable(T) appears to behave similarly to TRY_CAST(x AS T) 905 dtype.set("nullable", True) 906 907 return super().cast_sql(expression) 908 909 def _jsonpathsubscript_sql(self, expression: exp.JSONPathSubscript) -> str: 910 this = self.json_path_part(expression.this) 911 return str(int(this) + 1) if is_int(this) else this 912 913 def likeproperty_sql(self, expression: exp.LikeProperty) -> str: 914 return f"AS {self.sql(expression, 'this')}" 915 916 def _any_to_has( 917 self, 918 expression: exp.EQ | exp.NEQ, 919 default: t.Callable[[t.Any], str], 920 prefix: str = "", 921 ) -> str: 922 if isinstance(expression.left, exp.Any): 923 arr = expression.left 924 this = expression.right 925 elif isinstance(expression.right, exp.Any): 926 arr = expression.right 927 this = expression.left 928 else: 929 return default(expression) 930 931 return prefix + self.func("has", arr.this.unnest(), this) 932 933 def eq_sql(self, expression: exp.EQ) -> str: 934 return self._any_to_has(expression, super().eq_sql) 935 936 def neq_sql(self, expression: exp.NEQ) -> str: 937 return self._any_to_has(expression, super().neq_sql, "NOT ") 938 939 def regexpilike_sql(self, expression: exp.RegexpILike) -> str: 940 # Manually add a flag to make the search case-insensitive 941 regex = self.func("CONCAT", "'(?i)'", expression.expression) 942 return self.func("match", expression.this, regex) 943 944 def datatype_sql(self, expression: exp.DataType) -> str: 945 # String is the standard ClickHouse type, every other variant is just an alias. 946 # Additionally, any supplied length parameter will be ignored. 947 # 948 # https://clickhouse.com/docs/en/sql-reference/data-types/string 949 if expression.this in self.STRING_TYPE_MAPPING: 950 dtype = "String" 951 else: 952 dtype = super().datatype_sql(expression) 953 954 # This section changes the type to `Nullable(...)` if the following conditions hold: 955 # - It's marked as nullable - this ensures we won't wrap ClickHouse types with `Nullable` 956 # and change their semantics 957 # - It's not the key type of a `Map`. This is because ClickHouse enforces the following 958 # constraint: "Type of Map key must be a type, that can be represented by integer or 959 # String or FixedString (possibly LowCardinality) or UUID or IPv6" 960 # - It's not a composite type, e.g. `Nullable(Array(...))` is not a valid type 961 parent = expression.parent 962 if ( 963 expression.args.get("nullable") is not False 964 and not ( 965 isinstance(parent, exp.DataType) 966 and parent.is_type(exp.DataType.Type.MAP, check_nullable=True) 967 and expression.index in (None, 0) 968 ) 969 and not expression.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True) 970 ): 971 dtype = f"Nullable({dtype})" 972 973 return dtype 974 975 def cte_sql(self, expression: exp.CTE) -> str: 976 if expression.args.get("scalar"): 977 this = self.sql(expression, "this") 978 alias = self.sql(expression, "alias") 979 return f"{this} AS {alias}" 980 981 return super().cte_sql(expression) 982 983 def after_limit_modifiers(self, expression: exp.Expression) -> t.List[str]: 984 return super().after_limit_modifiers(expression) + [ 985 ( 986 self.seg("SETTINGS ") + self.expressions(expression, key="settings", flat=True) 987 if expression.args.get("settings") 988 else "" 989 ), 990 ( 991 self.seg("FORMAT ") + self.sql(expression, "format") 992 if expression.args.get("format") 993 else "" 994 ), 995 ] 996 997 def parameterizedagg_sql(self, expression: exp.ParameterizedAgg) -> str: 998 params = self.expressions(expression, key="params", flat=True) 999 return self.func(expression.name, *expression.expressions) + f"({params})" 1000 1001 def anonymousaggfunc_sql(self, expression: exp.AnonymousAggFunc) -> str: 1002 return self.func(expression.name, *expression.expressions) 1003 1004 def combinedaggfunc_sql(self, expression: exp.CombinedAggFunc) -> str: 1005 return self.anonymousaggfunc_sql(expression) 1006 1007 def combinedparameterizedagg_sql(self, expression: exp.CombinedParameterizedAgg) -> str: 1008 return self.parameterizedagg_sql(expression) 1009 1010 def placeholder_sql(self, expression: exp.Placeholder) -> str: 1011 return f"{{{expression.name}: {self.sql(expression, 'kind')}}}" 1012 1013 def oncluster_sql(self, expression: exp.OnCluster) -> str: 1014 return f"ON CLUSTER {self.sql(expression, 'this')}" 1015 1016 def createable_sql(self, expression: exp.Create, locations: t.DefaultDict) -> str: 1017 if expression.kind in self.ON_CLUSTER_TARGETS and locations.get( 1018 exp.Properties.Location.POST_NAME 1019 ): 1020 this_name = self.sql( 1021 expression.this if isinstance(expression.this, exp.Schema) else expression, 1022 "this", 1023 ) 1024 this_properties = " ".join( 1025 [self.sql(prop) for prop in locations[exp.Properties.Location.POST_NAME]] 1026 ) 1027 this_schema = self.schema_columns_sql(expression.this) 1028 return f"{this_name}{self.sep()}{this_properties}{self.sep()}{this_schema}" 1029 1030 return super().createable_sql(expression, locations) 1031 1032 def create_sql(self, expression: exp.Create) -> str: 1033 # The comment property comes last in CTAS statements, i.e. after the query 1034 query = expression.expression 1035 if isinstance(query, exp.Query): 1036 comment_prop = expression.find(exp.SchemaCommentProperty) 1037 if comment_prop: 1038 comment_prop.pop() 1039 query.replace(exp.paren(query)) 1040 else: 1041 comment_prop = None 1042 1043 create_sql = super().create_sql(expression) 1044 1045 comment_sql = self.sql(comment_prop) 1046 comment_sql = f" {comment_sql}" if comment_sql else "" 1047 1048 return f"{create_sql}{comment_sql}" 1049 1050 def prewhere_sql(self, expression: exp.PreWhere) -> str: 1051 this = self.indent(self.sql(expression, "this")) 1052 return f"{self.seg('PREWHERE')}{self.sep()}{this}" 1053 1054 def indexcolumnconstraint_sql(self, expression: exp.IndexColumnConstraint) -> str: 1055 this = self.sql(expression, "this") 1056 this = f" {this}" if this else "" 1057 expr = self.sql(expression, "expression") 1058 expr = f" {expr}" if expr else "" 1059 index_type = self.sql(expression, "index_type") 1060 index_type = f" TYPE {index_type}" if index_type else "" 1061 granularity = self.sql(expression, "granularity") 1062 granularity = f" GRANULARITY {granularity}" if granularity else "" 1063 1064 return f"INDEX{this}{expr}{index_type}{granularity}" 1065 1066 def partition_sql(self, expression: exp.Partition) -> str: 1067 return f"PARTITION {self.expressions(expression, flat=True)}" 1068 1069 def partitionid_sql(self, expression: exp.PartitionId) -> str: 1070 return f"ID {self.sql(expression.this)}" 1071 1072 def replacepartition_sql(self, expression: exp.ReplacePartition) -> str: 1073 return ( 1074 f"REPLACE {self.sql(expression.expression)} FROM {self.sql(expression, 'source')}" 1075 ) 1076 1077 def projectiondef_sql(self, expression: exp.ProjectionDef) -> str: 1078 return f"PROJECTION {self.sql(expression.this)} {self.wrap(expression.expression)}"
Generator converts a given syntax tree to the corresponding SQL string.
Arguments:
- pretty: Whether to format the produced SQL string. Default: False.
- identify: Determines when an identifier should be quoted. Possible values are: False (default): Never quote, except in cases where it's mandatory by the dialect. True or 'always': Always quote. 'safe': Only quote identifiers that are case insensitive.
- normalize: Whether to normalize identifiers to lowercase. Default: False.
- pad: The pad size in a formatted string. For example, this affects the indentation of a projection in a query, relative to its nesting level. Default: 2.
- indent: The indentation size in a formatted string. For example, this affects the
indentation of subqueries and filters under a
WHERE
clause. Default: 2. - normalize_functions: How to normalize function names. Possible values are: "upper" or True (default): Convert names to uppercase. "lower": Convert names to lowercase. False: Disables function name normalization.
- unsupported_level: Determines the generator's behavior when it encounters unsupported expressions. Default ErrorLevel.WARN.
- max_unsupported: Maximum number of unsupported messages to include in a raised UnsupportedError. This is only relevant if unsupported_level is ErrorLevel.RAISE. Default: 3
- leading_comma: Whether the comma is leading or trailing in select expressions. This is only relevant when generating in pretty mode. Default: False
- max_text_width: The max number of characters in a segment before creating new lines in pretty mode. The default is on the smaller end because the length only represents a segment and not the true line length. Default: 80
- comments: Whether to preserve comments in the output SQL code. Default: True
883 def strtodate_sql(self, expression: exp.StrToDate) -> str: 884 strtodate_sql = self.function_fallback_sql(expression) 885 886 if not isinstance(expression.parent, exp.Cast): 887 # StrToDate returns DATEs in other dialects (eg. postgres), so 888 # this branch aims to improve the transpilation to clickhouse 889 return f"CAST({strtodate_sql} AS DATE)" 890 891 return strtodate_sql
893 def cast_sql(self, expression: exp.Cast, safe_prefix: t.Optional[str] = None) -> str: 894 this = expression.this 895 896 if isinstance(this, exp.StrToDate) and expression.to == exp.DataType.build("datetime"): 897 return self.sql(this) 898 899 return super().cast_sql(expression, safe_prefix=safe_prefix)
901 def trycast_sql(self, expression: exp.TryCast) -> str: 902 dtype = expression.to 903 if not dtype.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True): 904 # Casting x into Nullable(T) appears to behave similarly to TRY_CAST(x AS T) 905 dtype.set("nullable", True) 906 907 return super().cast_sql(expression)
944 def datatype_sql(self, expression: exp.DataType) -> str: 945 # String is the standard ClickHouse type, every other variant is just an alias. 946 # Additionally, any supplied length parameter will be ignored. 947 # 948 # https://clickhouse.com/docs/en/sql-reference/data-types/string 949 if expression.this in self.STRING_TYPE_MAPPING: 950 dtype = "String" 951 else: 952 dtype = super().datatype_sql(expression) 953 954 # This section changes the type to `Nullable(...)` if the following conditions hold: 955 # - It's marked as nullable - this ensures we won't wrap ClickHouse types with `Nullable` 956 # and change their semantics 957 # - It's not the key type of a `Map`. This is because ClickHouse enforces the following 958 # constraint: "Type of Map key must be a type, that can be represented by integer or 959 # String or FixedString (possibly LowCardinality) or UUID or IPv6" 960 # - It's not a composite type, e.g. `Nullable(Array(...))` is not a valid type 961 parent = expression.parent 962 if ( 963 expression.args.get("nullable") is not False 964 and not ( 965 isinstance(parent, exp.DataType) 966 and parent.is_type(exp.DataType.Type.MAP, check_nullable=True) 967 and expression.index in (None, 0) 968 ) 969 and not expression.is_type(*self.NON_NULLABLE_TYPES, check_nullable=True) 970 ): 971 dtype = f"Nullable({dtype})" 972 973 return dtype
983 def after_limit_modifiers(self, expression: exp.Expression) -> t.List[str]: 984 return super().after_limit_modifiers(expression) + [ 985 ( 986 self.seg("SETTINGS ") + self.expressions(expression, key="settings", flat=True) 987 if expression.args.get("settings") 988 else "" 989 ), 990 ( 991 self.seg("FORMAT ") + self.sql(expression, "format") 992 if expression.args.get("format") 993 else "" 994 ), 995 ]
1016 def createable_sql(self, expression: exp.Create, locations: t.DefaultDict) -> str: 1017 if expression.kind in self.ON_CLUSTER_TARGETS and locations.get( 1018 exp.Properties.Location.POST_NAME 1019 ): 1020 this_name = self.sql( 1021 expression.this if isinstance(expression.this, exp.Schema) else expression, 1022 "this", 1023 ) 1024 this_properties = " ".join( 1025 [self.sql(prop) for prop in locations[exp.Properties.Location.POST_NAME]] 1026 ) 1027 this_schema = self.schema_columns_sql(expression.this) 1028 return f"{this_name}{self.sep()}{this_properties}{self.sep()}{this_schema}" 1029 1030 return super().createable_sql(expression, locations)
1032 def create_sql(self, expression: exp.Create) -> str: 1033 # The comment property comes last in CTAS statements, i.e. after the query 1034 query = expression.expression 1035 if isinstance(query, exp.Query): 1036 comment_prop = expression.find(exp.SchemaCommentProperty) 1037 if comment_prop: 1038 comment_prop.pop() 1039 query.replace(exp.paren(query)) 1040 else: 1041 comment_prop = None 1042 1043 create_sql = super().create_sql(expression) 1044 1045 comment_sql = self.sql(comment_prop) 1046 comment_sql = f" {comment_sql}" if comment_sql else "" 1047 1048 return f"{create_sql}{comment_sql}"
1054 def indexcolumnconstraint_sql(self, expression: exp.IndexColumnConstraint) -> str: 1055 this = self.sql(expression, "this") 1056 this = f" {this}" if this else "" 1057 expr = self.sql(expression, "expression") 1058 expr = f" {expr}" if expr else "" 1059 index_type = self.sql(expression, "index_type") 1060 index_type = f" TYPE {index_type}" if index_type else "" 1061 granularity = self.sql(expression, "granularity") 1062 granularity = f" GRANULARITY {granularity}" if granularity else "" 1063 1064 return f"INDEX{this}{expr}{index_type}{granularity}"
Inherited Members
- sqlglot.generator.Generator
- Generator
- NULL_ORDERING_SUPPORTED
- IGNORE_NULLS_IN_FUNC
- LOCKING_READS_SUPPORTED
- WRAP_DERIVED_VALUES
- CREATE_FUNCTION_RETURN_AS
- MATCHED_BY_SOURCE
- SINGLE_STRING_INTERVAL
- INTERVAL_ALLOWS_PLURAL_FORM
- LIMIT_FETCH
- LIMIT_ONLY_LITERALS
- RENAME_TABLE_WITH_DB
- INDEX_ON
- QUERY_HINT_SEP
- IS_BOOL_ALLOWED
- DUPLICATE_KEY_UPDATE_WITH_SET
- LIMIT_IS_TOP
- RETURNING_END
- EXTRACT_ALLOWS_QUOTES
- TZ_TO_WITH_TIME_ZONE
- ALTER_TABLE_INCLUDE_COLUMN_KEYWORD
- UNNEST_WITH_ORDINALITY
- AGGREGATE_FILTER_SUPPORTED
- SEMI_ANTI_JOIN_WITH_SIDE
- COMPUTED_COLUMN_WITH_TYPE
- SUPPORTS_TABLE_COPY
- TABLESAMPLE_WITH_METHOD
- TABLESAMPLE_SEED_KEYWORD
- COLLATE_IS_FUNC
- DATA_TYPE_SPECIFIERS_ALLOWED
- ENSURE_BOOLS
- CTE_RECURSIVE_KEYWORD_REQUIRED
- SUPPORTS_SINGLE_ARG_CONCAT
- UNPIVOT_ALIASES_ARE_IDENTIFIERS
- JSON_KEY_VALUE_PAIR_SEP
- INSERT_OVERWRITE
- SUPPORTS_SELECT_INTO
- SUPPORTS_UNLOGGED_TABLES
- SUPPORTS_CREATE_TABLE_LIKE
- LIKE_PROPERTY_INSIDE_SCHEMA
- MULTI_ARG_DISTINCT
- JSON_TYPE_REQUIRED_FOR_EXTRACTION
- JSON_PATH_BRACKETED_KEY_SUPPORTED
- JSON_PATH_SINGLE_QUOTE_ESCAPE
- COPY_PARAMS_ARE_WRAPPED
- COPY_PARAMS_EQ_REQUIRED
- COPY_HAS_INTO_KEYWORD
- STAR_EXCEPT
- HEX_FUNC
- WITH_PROPERTIES_PREFIX
- QUOTE_JSON_PATH
- PAD_FILL_PATTERN_IS_REQUIRED
- SUPPORTS_EXPLODING_PROJECTIONS
- ARRAY_CONCAT_IS_VAR_LEN
- SUPPORTS_CONVERT_TIMEZONE
- SUPPORTS_NULLABLE_TYPES
- PARSE_JSON_NAME
- TIME_PART_SINGULARS
- TOKEN_MAPPING
- PARAMETER_TOKEN
- NAMED_PLACEHOLDER_TOKEN
- RESERVED_KEYWORDS
- WITH_SEPARATED_COMMENTS
- EXCLUDE_COMMENTS
- UNWRAPPED_INTERVAL_VALUES
- PARAMETERIZABLE_TEXT_TYPES
- EXPRESSIONS_WITHOUT_NESTED_CTES
- SENTINEL_LINE_BREAK
- pretty
- identify
- normalize
- pad
- unsupported_level
- max_unsupported
- leading_comma
- max_text_width
- comments
- dialect
- normalize_functions
- unsupported_messages
- generate
- preprocess
- unsupported
- sep
- seg
- pad_comment
- maybe_comment
- wrap
- no_identify
- normalize_func
- indent
- sql
- uncache_sql
- cache_sql
- characterset_sql
- column_parts
- column_sql
- columnposition_sql
- columndef_sql
- columnconstraint_sql
- computedcolumnconstraint_sql
- autoincrementcolumnconstraint_sql
- compresscolumnconstraint_sql
- generatedasidentitycolumnconstraint_sql
- generatedasrowcolumnconstraint_sql
- periodforsystemtimeconstraint_sql
- notnullcolumnconstraint_sql
- transformcolumnconstraint_sql
- primarykeycolumnconstraint_sql
- uniquecolumnconstraint_sql
- sequenceproperties_sql
- clone_sql
- describe_sql
- heredoc_sql
- prepend_ctes
- with_sql
- tablealias_sql
- bitstring_sql
- hexstring_sql
- bytestring_sql
- unicodestring_sql
- rawstring_sql
- datatypeparam_sql
- directory_sql
- delete_sql
- drop_sql
- except_sql
- except_op
- fetch_sql
- filter_sql
- hint_sql
- indexparameters_sql
- index_sql
- identifier_sql
- hex_sql
- lowerhex_sql
- inputoutputformat_sql
- national_sql
- properties_sql
- root_properties
- properties
- with_properties
- locate_properties
- property_name
- property_sql
- fallbackproperty_sql
- journalproperty_sql
- freespaceproperty_sql
- checksumproperty_sql
- mergeblockratioproperty_sql
- datablocksizeproperty_sql
- blockcompressionproperty_sql
- isolatedloadingproperty_sql
- partitionboundspec_sql
- partitionedofproperty_sql
- lockingproperty_sql
- withdataproperty_sql
- withsystemversioningproperty_sql
- insert_sql
- intersect_sql
- intersect_op
- introducer_sql
- kill_sql
- pseudotype_sql
- objectidentifier_sql
- onconflict_sql
- returning_sql
- rowformatdelimitedproperty_sql
- withtablehint_sql
- indextablehint_sql
- historicaldata_sql
- table_parts
- table_sql
- tablesample_sql
- pivot_sql
- version_sql
- tuple_sql
- update_sql
- values_sql
- var_sql
- into_sql
- from_sql
- group_sql
- having_sql
- connect_sql
- prior_sql
- join_sql
- lambda_sql
- lateral_op
- lateral_sql
- limit_sql
- offset_sql
- setitem_sql
- set_sql
- pragma_sql
- lock_sql
- literal_sql
- escape_str
- loaddata_sql
- null_sql
- boolean_sql
- order_sql
- withfill_sql
- cluster_sql
- distribute_sql
- sort_sql
- ordered_sql
- matchrecognizemeasure_sql
- matchrecognize_sql
- query_modifiers
- options_modifier
- queryoption_sql
- offset_limit_modifiers
- select_sql
- schema_sql
- schema_columns_sql
- star_sql
- parameter_sql
- sessionparameter_sql
- subquery_sql
- qualify_sql
- set_operations
- union_sql
- union_op
- unnest_sql
- where_sql
- window_sql
- partition_by_sql
- windowspec_sql
- withingroup_sql
- between_sql
- bracket_offset_expressions
- bracket_sql
- all_sql
- any_sql
- exists_sql
- case_sql
- constraint_sql
- nextvaluefor_sql
- extract_sql
- trim_sql
- convert_concat_args
- concat_sql
- concatws_sql
- check_sql
- foreignkey_sql
- primarykey_sql
- if_sql
- matchagainst_sql
- jsonkeyvalue_sql
- jsonpath_sql
- json_path_part
- formatjson_sql
- jsonobject_sql
- jsonobjectagg_sql
- jsonarray_sql
- jsonarrayagg_sql
- jsoncolumndef_sql
- jsonschema_sql
- jsontable_sql
- openjsoncolumndef_sql
- openjson_sql
- in_sql
- in_unnest_op
- interval_sql
- return_sql
- reference_sql
- anonymous_sql
- paren_sql
- neg_sql
- not_sql
- alias_sql
- pivotalias_sql
- aliases_sql
- atindex_sql
- attimezone_sql
- fromtimezone_sql
- add_sql
- and_sql
- or_sql
- xor_sql
- connector_sql
- bitwiseand_sql
- bitwiseleftshift_sql
- bitwisenot_sql
- bitwiseor_sql
- bitwiserightshift_sql
- bitwisexor_sql
- currentdate_sql
- collate_sql
- command_sql
- comment_sql
- mergetreettlaction_sql
- mergetreettl_sql
- transaction_sql
- commit_sql
- rollback_sql
- altercolumn_sql
- alterdiststyle_sql
- altersortkey_sql
- renametable_sql
- renamecolumn_sql
- alterset_sql
- alter_sql
- add_column_sql
- droppartition_sql
- addconstraint_sql
- distinct_sql
- ignorenulls_sql
- respectnulls_sql
- havingmax_sql
- intdiv_sql
- dpipe_sql
- div_sql
- overlaps_sql
- distance_sql
- dot_sql
- propertyeq_sql
- escape_sql
- glob_sql
- gt_sql
- gte_sql
- ilike_sql
- ilikeany_sql
- is_sql
- like_sql
- likeany_sql
- similarto_sql
- lt_sql
- lte_sql
- mod_sql
- mul_sql
- nullsafeeq_sql
- nullsafeneq_sql
- slice_sql
- sub_sql
- try_sql
- log_sql
- use_sql
- binary
- function_fallback_sql
- func
- format_args
- too_wide
- format_time
- expressions
- op_expressions
- naked_property
- tag_sql
- token_sql
- userdefinedfunction_sql
- joinhint_sql
- kwarg_sql
- when_sql
- merge_sql
- tochar_sql
- tonumber_sql
- dictproperty_sql
- dictrange_sql
- dictsubproperty_sql
- clusteredbyproperty_sql
- anyvalue_sql
- querytransform_sql
- indexconstraintoption_sql
- checkcolumnconstraint_sql
- nvl2_sql
- comprehension_sql
- columnprefix_sql
- opclass_sql
- predict_sql
- forin_sql
- refresh_sql
- operator_sql
- toarray_sql
- tsordstotime_sql
- tsordstotimestamp_sql
- tsordstodate_sql
- unixdate_sql
- lastday_sql
- dateadd_sql
- arrayany_sql
- struct_sql
- partitionrange_sql
- truncatetable_sql
- convert_sql
- copyparameter_sql
- credentials_sql
- copy_sql
- semicolon_sql
- datadeletionproperty_sql
- maskingpolicycolumnconstraint_sql
- gapfill_sql
- scope_resolution
- scoperesolution_sql
- parsejson_sql
- rand_sql
- changes_sql
- pad_sql
- summarize_sql
- explodinggenerateseries_sql
- arrayconcat_sql
- converttimezone_sql