mzcompose.py 30 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719
  1. # Copyright Materialize, Inc. and contributors. All rights reserved.
  2. #
  3. # Use of this software is governed by the Business Source License
  4. # included in the LICENSE file at the root of this repository.
  5. #
  6. # As of the Change Date specified in that file, in accordance with
  7. # the Business Source License, use of this software will be governed
  8. # by the Apache License, Version 2.0.
  9. """
  10. Run SQL tests using an instance of Mz that is embedded in the sqllogic binary
  11. itself. Good for basic SQL tests, but can't interact with sources like
  12. MySQL/Kafka, see Testdrive for that.
  13. """
  14. from __future__ import annotations
  15. import argparse
  16. import os
  17. import threading
  18. from argparse import Namespace
  19. from concurrent.futures import ThreadPoolExecutor, as_completed
  20. from pathlib import Path
  21. from queue import Queue
  22. from materialize import buildkite, ci_util, file_util
  23. from materialize.mzcompose.composition import Composition, WorkflowArgumentParser
  24. from materialize.mzcompose.services.mz import Mz
  25. from materialize.mzcompose.services.postgres import CockroachOrPostgresMetadata
  26. from materialize.mzcompose.services.sql_logic_test import SqlLogicTest
  27. from materialize.ui import CommandFailureCausedUIError
  28. NUM_SLTS = 8
  29. SLTS = [f"slt_{i+1}" for i in range(NUM_SLTS)]
  30. SERVICES = [CockroachOrPostgresMetadata(), Mz(app_password="")] + [
  31. SqlLogicTest(name=slt) for slt in SLTS
  32. ]
  33. COCKROACH_DEFAULT_PORT = 26257
  34. def workflow_default(c: Composition) -> None:
  35. def process(name: str) -> None:
  36. if name == "default":
  37. return
  38. with c.test_case(name):
  39. c.workflow(name)
  40. c.test_parts(list(c.workflows.keys()), process)
  41. def workflow_fast_tests(c: Composition, parser: WorkflowArgumentParser) -> None:
  42. """Run fast SQL logic tests"""
  43. run_sqllogictest(c, parser, compileFastSltConfig())
  44. def workflow_slow_tests(c: Composition, parser: WorkflowArgumentParser) -> None:
  45. """Run slow SQL logic tests"""
  46. run_sqllogictest(
  47. c,
  48. parser,
  49. compileSlowSltConfig(),
  50. )
  51. def workflow_selection(c: Composition, parser: WorkflowArgumentParser) -> None:
  52. """Run specific SQL logic tests using pattern"""
  53. parser.add_argument(
  54. "--pattern",
  55. type=str,
  56. action="append",
  57. help="Example: test/sqllogictest/map.slt",
  58. )
  59. parser.add_argument(
  60. "--auto-index-selects",
  61. default=False,
  62. type=bool,
  63. action=argparse.BooleanOptionalAction,
  64. )
  65. run_sqllogictest(
  66. c,
  67. parser,
  68. InputBasedSltRunConfig(),
  69. )
  70. def run_sqllogictest(
  71. c: Composition, parser: WorkflowArgumentParser, run_config: SltRunConfig
  72. ) -> None:
  73. parser.add_argument("--replica-size", default=2, type=int)
  74. parser.add_argument("--replicas", default=1, type=int)
  75. args = parser.parse_args()
  76. c.up(c.metadata_store())
  77. work_queue = Queue()
  78. stop_event = threading.Event()
  79. for step in run_config.steps:
  80. step.configure(args)
  81. sharded_files: list[str] = sorted(
  82. buildkite.shard_list(step.file_list, lambda file: file)
  83. )
  84. for file in sharded_files:
  85. work_queue.put((step, file))
  86. # Hacky way to make sure we have downloaded the image
  87. c.up({"name": "slt_1", "persistent": True})
  88. def worker(container_name: str):
  89. exception: Exception | None = None
  90. while True:
  91. if stop_event.is_set():
  92. return
  93. try:
  94. step, file = work_queue.get_nowait()
  95. except Exception:
  96. break # Queue is empty
  97. if "singlereplica_" in file and args.replicas > 1:
  98. continue
  99. junit_report_path = ci_util.junit_report_filename(
  100. f"{c.name}-{file.replace('.', '_').replace('/', '_')}"
  101. )
  102. cmd = step.to_command(
  103. container_name,
  104. file,
  105. args.replicas,
  106. args.replica_size,
  107. junit_report_path,
  108. c.metadata_store(),
  109. )
  110. try:
  111. c.run(container_name, *cmd, capture=True, capture_stderr=True)
  112. # Uploading successful junit files wastes time and contains no useful information
  113. os.remove(junit_report_path)
  114. except CommandFailureCausedUIError as e:
  115. print(f"+++ {file} failed, STDOUT:\n{e.stdout}\nSTDERR:\n{e.stderr}")
  116. exception = e
  117. finally:
  118. work_queue.task_done()
  119. if exception:
  120. raise exception
  121. with ThreadPoolExecutor(max_workers=len(SLTS)) as executor:
  122. futures = [executor.submit(worker, container_name) for container_name in SLTS]
  123. try:
  124. for future in as_completed(futures):
  125. future.result()
  126. except Exception:
  127. stop_event.set()
  128. raise
  129. class SltRunConfig:
  130. def __init__(self):
  131. self.steps: list[SltRunStepConfig] = []
  132. class SltRunStepConfig:
  133. def __init__(self, file_set: set[str], flags: list[str]):
  134. file_list = list(file_set)
  135. file_list.sort()
  136. self.file_list = file_list
  137. self.flags = flags
  138. def configure(self, args: Namespace) -> None:
  139. pass
  140. def to_command(
  141. self,
  142. container_name: str,
  143. file: str,
  144. replicas: int,
  145. replica_size: int,
  146. junit_report_path: Path,
  147. metadata_store: str,
  148. metadata_store_port: int = COCKROACH_DEFAULT_PORT,
  149. ) -> list[str]:
  150. sqllogictest_config = [
  151. f"--junit-report={junit_report_path}",
  152. f"--postgres-url=postgres://root@{metadata_store}:{metadata_store_port}",
  153. f"--prefix={container_name}",
  154. f"--replica-size={replica_size}",
  155. f"--replicas={replicas}",
  156. ]
  157. command = [
  158. "sqllogictest",
  159. "-v",
  160. *self.flags,
  161. *sqllogictest_config,
  162. file,
  163. ]
  164. return command
  165. class InputBasedSltRunStepConfig(SltRunStepConfig):
  166. def __init__(self) -> None:
  167. super().__init__(file_set=set(), flags=[])
  168. def configure(self, args: Namespace) -> None:
  169. if args.pattern is not None:
  170. file_list = list(file_util.resolve_paths_with_wildcard(args.pattern))
  171. file_list.sort()
  172. if len(file_list) == 0:
  173. raise RuntimeError(f"Pattern {args.pattern} matched no files!")
  174. print(f"Matched files are: {file_list}")
  175. self.file_list = file_list
  176. if args.auto_index_selects:
  177. self.flags.append("--auto-index-selects")
  178. class InputBasedSltRunConfig(SltRunConfig):
  179. def __init__(self):
  180. super().__init__()
  181. self.steps.append(InputBasedSltRunStepConfig())
  182. class DefaultSltRunStepConfig(SltRunStepConfig):
  183. def __init__(self, file_set: set[str]):
  184. super().__init__(
  185. file_set,
  186. flags=[],
  187. )
  188. def compileFastSltConfig() -> SltRunConfig:
  189. tests = {
  190. "test/sqllogictest/*.slt",
  191. "test/sqllogictest/attributes/*.slt",
  192. "test/sqllogictest/explain/*.slt",
  193. "test/sqllogictest/introspection/*.slt",
  194. "test/sqllogictest/autogenerated/*.slt",
  195. "test/sqllogictest/transform/*.slt",
  196. "test/sqllogictest/cockroach/aggregate.slt",
  197. "test/sqllogictest/cockroach/distinct_on.slt",
  198. "test/sqllogictest/cockroach/subquery_correlated.slt",
  199. "test/sqllogictest/transform/notice/*.slt",
  200. "test/sqllogictest/advent-of-code/2023/*.slt",
  201. }
  202. # Too slow
  203. tests_exclude = {
  204. "test/sqllogictest/default_privileges.slt",
  205. "test/sqllogictest/distinct_arrangements.slt",
  206. "test/sqllogictest/privilege_grants.slt",
  207. "test/sqllogictest/introspection/singlereplica_attribution_sources.slt",
  208. }
  209. tests_without_views = {
  210. "test/sqllogictest/alter.slt",
  211. "test/sqllogictest/ambiguous_rename.slt",
  212. "test/sqllogictest/arithmetic.slt",
  213. "test/sqllogictest/array_fill.slt",
  214. "test/sqllogictest/arrays.slt",
  215. "test/sqllogictest/as_of.slt",
  216. "test/sqllogictest/singlereplica_audit_log.slt",
  217. "test/sqllogictest/boolean.slt",
  218. "test/sqllogictest/bytea.slt",
  219. "test/sqllogictest/cast.slt",
  220. "test/sqllogictest/char.slt",
  221. "test/sqllogictest/chbench.slt",
  222. "test/sqllogictest/chr.slt",
  223. "test/sqllogictest/cluster.slt",
  224. "test/sqllogictest/coercion.slt",
  225. "test/sqllogictest/collate.slt",
  226. "test/sqllogictest/comparison.slt",
  227. "test/sqllogictest/cte.slt",
  228. "test/sqllogictest/cte_lowering.slt",
  229. "test/sqllogictest/current_database.slt",
  230. "test/sqllogictest/cursor.slt",
  231. "test/sqllogictest/datediff.slt",
  232. "test/sqllogictest/dates-times.slt",
  233. "test/sqllogictest/degenerate.slt",
  234. "test/sqllogictest/disambiguate_columns.slt",
  235. "test/sqllogictest/distinct_from.slt",
  236. "test/sqllogictest/distinct_on.slt",
  237. "test/sqllogictest/encode.slt",
  238. "test/sqllogictest/errors.slt",
  239. "test/sqllogictest/extract.slt",
  240. "test/sqllogictest/filter-pushdown.slt",
  241. "test/sqllogictest/float.slt",
  242. "test/sqllogictest/funcs.slt",
  243. "test/sqllogictest/github-14116.slt",
  244. "test/sqllogictest/github-16036.slt",
  245. "test/sqllogictest/github-17808.slt",
  246. "test/sqllogictest/github-2235.slt",
  247. "test/sqllogictest/github-2327.slt",
  248. "test/sqllogictest/github-2514.slt",
  249. "test/sqllogictest/github-2666.slt",
  250. "test/sqllogictest/github-2668.slt",
  251. "test/sqllogictest/github-2746.slt",
  252. "test/sqllogictest/github-2799.slt",
  253. "test/sqllogictest/github-2906.slt",
  254. "test/sqllogictest/github-2969.slt",
  255. "test/sqllogictest/github-2996.slt",
  256. "test/sqllogictest/github-3256.slt",
  257. "test/sqllogictest/github-3374.slt",
  258. "test/sqllogictest/github-3963.slt",
  259. "test/sqllogictest/github-4336.slt",
  260. "test/sqllogictest/github-5126.slt",
  261. "test/sqllogictest/github-5174.slt",
  262. "test/sqllogictest/github-5474.slt",
  263. "test/sqllogictest/github-5536.slt",
  264. "test/sqllogictest/github-5717.slt",
  265. "test/sqllogictest/github-7585.slt",
  266. "test/sqllogictest/github-31878.slt",
  267. "test/sqllogictest/id.slt",
  268. "test/sqllogictest/id_reuse.slt",
  269. "test/sqllogictest/information_schema_columns.slt",
  270. "test/sqllogictest/information_schema_tables.slt",
  271. "test/sqllogictest/int2vector.slt",
  272. "test/sqllogictest/interval.slt",
  273. "test/sqllogictest/joins.slt",
  274. "test/sqllogictest/jsonb.slt",
  275. "test/sqllogictest/keys.slt",
  276. "test/sqllogictest/like.slt",
  277. "test/sqllogictest/list.slt",
  278. "test/sqllogictest/list_subquery.slt",
  279. "test/sqllogictest/managed_cluster.slt",
  280. "test/sqllogictest/map.slt",
  281. "test/sqllogictest/materialized_views.slt",
  282. "test/sqllogictest/mz_catalog_server_index_accounting.slt",
  283. "test/sqllogictest/mztimestamp.slt",
  284. "test/sqllogictest/name_resolution.slt",
  285. "test/sqllogictest/not-null-propagation.slt",
  286. "test/sqllogictest/numeric.slt",
  287. "test/sqllogictest/object_ownership.slt",
  288. "test/sqllogictest/oid.slt",
  289. "test/sqllogictest/operator.slt",
  290. "test/sqllogictest/outer_join.slt",
  291. "test/sqllogictest/outer_join_simplification.slt",
  292. "test/sqllogictest/parse_ident.slt",
  293. "test/sqllogictest/pg_catalog_attribute.slt",
  294. "test/sqllogictest/pg_catalog_class.slt",
  295. "test/sqllogictest/pg_catalog_matviews.slt",
  296. "test/sqllogictest/pg_catalog_namespace.slt",
  297. "test/sqllogictest/pg_catalog_proc.slt",
  298. "test/sqllogictest/pg_catalog_roles.slt",
  299. "test/sqllogictest/pg_catalog_tablespace.slt",
  300. "test/sqllogictest/pg_catalog_views.slt",
  301. "test/sqllogictest/pg_get_constraintdef.slt",
  302. "test/sqllogictest/pg_get_indexdef.slt",
  303. "test/sqllogictest/pg_get_viewdef.slt",
  304. "test/sqllogictest/pgcli.slt",
  305. "test/sqllogictest/pgcrypto.slt",
  306. "test/sqllogictest/postgres-incompatibility.slt",
  307. "test/sqllogictest/pretty.slt",
  308. "test/sqllogictest/privilege_checks.slt",
  309. "test/sqllogictest/privileges_pg.slt",
  310. "test/sqllogictest/quote_ident.slt",
  311. "test/sqllogictest/quoting.slt",
  312. "test/sqllogictest/range.slt",
  313. "test/sqllogictest/rbac_enabled.slt",
  314. "test/sqllogictest/record.slt",
  315. "test/sqllogictest/recursion_limit.slt",
  316. "test/sqllogictest/recursive_type_unioning.slt",
  317. "test/sqllogictest/regclass.slt",
  318. "test/sqllogictest/regproc.slt",
  319. "test/sqllogictest/regressions.slt",
  320. "test/sqllogictest/regtype.slt",
  321. "test/sqllogictest/returning.slt",
  322. "test/sqllogictest/role.slt",
  323. "test/sqllogictest/role_create.slt",
  324. "test/sqllogictest/role_membership.slt",
  325. "test/sqllogictest/scalar-func-table-position.slt",
  326. "test/sqllogictest/scalar_subqueries_select_list.slt",
  327. "test/sqllogictest/schemas.slt",
  328. "test/sqllogictest/scoping.slt",
  329. "test/sqllogictest/secret.slt",
  330. "test/sqllogictest/select_all_group_by.slt",
  331. "test/sqllogictest/session-window-wmr.slt",
  332. "test/sqllogictest/show_create_system_objects.slt",
  333. "test/sqllogictest/slt.slt",
  334. "test/sqllogictest/source_sizing.slt",
  335. "test/sqllogictest/string.slt",
  336. "test/sqllogictest/subquery.slt",
  337. "test/sqllogictest/subscribe_error.slt",
  338. "test/sqllogictest/subscribe_outputs.slt",
  339. "test/sqllogictest/subsource.slt",
  340. "test/sqllogictest/system-cluster.slt",
  341. "test/sqllogictest/table_func.slt",
  342. "test/sqllogictest/temporal.slt",
  343. "test/sqllogictest/timedomain.slt",
  344. "test/sqllogictest/timestamp.slt",
  345. "test/sqllogictest/timestamptz.slt",
  346. "test/sqllogictest/timezone.slt",
  347. "test/sqllogictest/tpch_create_index.slt",
  348. "test/sqllogictest/tpch_create_materialized_view.slt",
  349. "test/sqllogictest/tpch_select.slt",
  350. "test/sqllogictest/transactions.slt",
  351. "test/sqllogictest/type-promotion.slt",
  352. "test/sqllogictest/typeof.slt",
  353. "test/sqllogictest/types.slt",
  354. "test/sqllogictest/uniqueness_propagation_filter.slt",
  355. "test/sqllogictest/unstable.slt",
  356. "test/sqllogictest/updates.slt",
  357. "test/sqllogictest/uuid.slt",
  358. "test/sqllogictest/vars.slt",
  359. "test/sqllogictest/web-console.slt",
  360. "test/sqllogictest/window_funcs.slt",
  361. "test/sqllogictest/attributes/mir_arity.slt",
  362. "test/sqllogictest/attributes/mir_column_types.slt",
  363. "test/sqllogictest/attributes/mir_unique_keys.slt",
  364. "test/sqllogictest/explain/bad_explain_statements.slt",
  365. "test/sqllogictest/explain/decorrelated_plan_as_json.slt",
  366. "test/sqllogictest/explain/decorrelated_plan_as_text.slt",
  367. "test/sqllogictest/explain/optimized_plan_as_json.slt",
  368. "test/sqllogictest/explain/optimized_plan_as_text.slt",
  369. "test/sqllogictest/explain/physical_plan_as_json.slt",
  370. "test/sqllogictest/explain/physical_plan_as_text.slt",
  371. "test/sqllogictest/explain/raw_plan_as_json.slt",
  372. "test/sqllogictest/explain/raw_plan_as_text.slt",
  373. "test/sqllogictest/autogenerated/all_parts_essential.slt",
  374. "test/sqllogictest/autogenerated/char-varchar-comparisons.slt",
  375. "test/sqllogictest/transform/aggregation_nullability.slt",
  376. "test/sqllogictest/transform/column_knowledge.slt",
  377. "test/sqllogictest/transform/dataflow.slt",
  378. "test/sqllogictest/transform/demand.slt",
  379. "test/sqllogictest/transform/filter_index.slt",
  380. "test/sqllogictest/transform/fold_constants.slt",
  381. "test/sqllogictest/transform/is_null_propagation.slt",
  382. "test/sqllogictest/transform/join_fusion.slt",
  383. "test/sqllogictest/transform/join_index.slt",
  384. "test/sqllogictest/transform/literal_constraints.slt",
  385. "test/sqllogictest/transform/literal_lifting.slt",
  386. "test/sqllogictest/transform/monotonic.slt",
  387. "test/sqllogictest/transform/non_null_requirements.slt",
  388. "test/sqllogictest/transform/normalize_lets.slt",
  389. "test/sqllogictest/transform/predicate_pushdown.slt",
  390. "test/sqllogictest/transform/predicate_reduction.slt",
  391. "test/sqllogictest/transform/projection_lifting.slt",
  392. "test/sqllogictest/transform/reduce_elision.slt",
  393. "test/sqllogictest/transform/reduce_fusion.slt",
  394. "test/sqllogictest/transform/reduction_pushdown.slt",
  395. "test/sqllogictest/transform/redundant_join.slt",
  396. "test/sqllogictest/transform/relation_cse.slt",
  397. "test/sqllogictest/transform/scalability.slt",
  398. "test/sqllogictest/transform/scalar_cse.slt",
  399. "test/sqllogictest/transform/threshold_elision.slt",
  400. "test/sqllogictest/transform/topk.slt",
  401. "test/sqllogictest/transform/union.slt",
  402. "test/sqllogictest/transform/union_cancel.slt",
  403. # "test/sqllogictest/sqlite/test/evidence/in1.test",
  404. "test/sqllogictest/sqlite/test/evidence/in2.test",
  405. "test/sqllogictest/sqlite/test/evidence/slt_lang_aggfunc.test",
  406. "test/sqllogictest/sqlite/test/evidence/slt_lang_createtrigger.test",
  407. "test/sqllogictest/sqlite/test/evidence/slt_lang_createview.test",
  408. "test/sqllogictest/sqlite/test/evidence/slt_lang_dropindex.test",
  409. "test/sqllogictest/sqlite/test/evidence/slt_lang_droptable.test",
  410. "test/sqllogictest/sqlite/test/evidence/slt_lang_droptrigger.test",
  411. "test/sqllogictest/sqlite/test/evidence/slt_lang_dropview.test",
  412. "test/sqllogictest/sqlite/test/evidence/slt_lang_reindex.test",
  413. "test/sqllogictest/sqlite/test/evidence/slt_lang_replace.test",
  414. "test/sqllogictest/sqlite/test/evidence/slt_lang_update.test",
  415. # TODO(benesch): renable a fast subset of the following when performance is restored.
  416. # "test/sqllogictest/sqlite/test/index/between/1/slt_good_0.test",
  417. # "test/sqllogictest/sqlite/test/index/commute/10/slt_good_0.test",
  418. # "test/sqllogictest/sqlite/test/index/delete/1/slt_good_0.test",
  419. # "test/sqllogictest/sqlite/test/index/in/10/slt_good_0.test",
  420. # "test/sqllogictest/sqlite/test/index/orderby_nosort/10/slt_good_0.test",
  421. # "test/sqllogictest/sqlite/test/index/orderby/10/slt_good_0.test",
  422. # "test/sqllogictest/sqlite/test/index/random/10/slt_good_0.test",
  423. # "test/sqllogictest/sqlite/test/index/view/10/slt_good_0.test",
  424. # "test/sqllogictest/sqlite/test/random/aggregates/slt_good_0.test",
  425. # "test/sqllogictest/sqlite/test/random/expr/slt_good_0.test",
  426. # "test/sqllogictest/sqlite/test/random/groupby/slt_good_0.test",
  427. # "test/sqllogictest/sqlite/test/random/select/slt_good_0.test",
  428. # "test/sqllogictest/sqlite/test/select1.test",
  429. # "test/sqllogictest/sqlite/test/select2.test",
  430. # "test/sqllogictest/sqlite/test/select3.test",
  431. # "test/sqllogictest/sqlite/test/select4.test",
  432. # "test/sqllogictest/sqlite/test/select5.test",
  433. "test/sqllogictest/cockroach/alias_types.slt",
  434. "test/sqllogictest/cockroach/alter_column_type.slt",
  435. "test/sqllogictest/cockroach/alter_table.slt",
  436. "test/sqllogictest/cockroach/apply_join.slt",
  437. "test/sqllogictest/cockroach/array.slt",
  438. "test/sqllogictest/cockroach/as_of.slt",
  439. "test/sqllogictest/cockroach/bit.slt",
  440. # "test/sqllogictest/cockroach/builtin_function.slt",
  441. "test/sqllogictest/cockroach/bytes.slt",
  442. "test/sqllogictest/cockroach/case_sensitive_names.slt",
  443. # "test/sqllogictest/cockroach/collatedstring_constraint.slt",
  444. "test/sqllogictest/cockroach/collatedstring_index1.slt",
  445. "test/sqllogictest/cockroach/collatedstring_index2.slt",
  446. "test/sqllogictest/cockroach/collatedstring_normalization.slt",
  447. "test/sqllogictest/cockroach/collatedstring_nullinindex.slt",
  448. "test/sqllogictest/cockroach/collatedstring_uniqueindex1.slt",
  449. "test/sqllogictest/cockroach/collatedstring_uniqueindex2.slt",
  450. "test/sqllogictest/cockroach/collatedstring.slt",
  451. "test/sqllogictest/cockroach/computed.slt",
  452. # "test/sqllogictest/cockroach/conditional.slt",
  453. "test/sqllogictest/cockroach/create_as.slt",
  454. "test/sqllogictest/cockroach/custom_escape_character.slt",
  455. "test/sqllogictest/cockroach/database.slt",
  456. # "test/sqllogictest/cockroach/datetime.slt",
  457. # "test/sqllogictest/cockroach/decimal.slt",
  458. "test/sqllogictest/cockroach/delete.slt",
  459. "test/sqllogictest/cockroach/discard.slt",
  460. "test/sqllogictest/cockroach/drop_database.slt",
  461. "test/sqllogictest/cockroach/drop_table.slt",
  462. "test/sqllogictest/cockroach/drop_user.slt",
  463. "test/sqllogictest/cockroach/drop_view.slt",
  464. "test/sqllogictest/cockroach/errors.slt",
  465. # "test/sqllogictest/cockroach/exec_hash_join.slt",
  466. # "test/sqllogictest/cockroach/exec_merge_join.slt",
  467. "test/sqllogictest/cockroach/exec_window.slt",
  468. "test/sqllogictest/cockroach/extract.slt",
  469. # "test/sqllogictest/cockroach/float.slt",
  470. "test/sqllogictest/cockroach/inet.slt",
  471. "test/sqllogictest/cockroach/information_schema.slt",
  472. "test/sqllogictest/cockroach/insert.slt",
  473. "test/sqllogictest/cockroach/int_size.slt",
  474. "test/sqllogictest/cockroach/join.slt",
  475. # "test/sqllogictest/cockroach/json_builtins.slt",
  476. # "test/sqllogictest/cockroach/json.slt",
  477. "test/sqllogictest/cockroach/like.slt",
  478. "test/sqllogictest/cockroach/limit.slt",
  479. # "test/sqllogictest/cockroach/lookup_join.slt",
  480. "test/sqllogictest/cockroach/namespace.slt",
  481. # "test/sqllogictest/cockroach/no_primary_key.slt",
  482. "test/sqllogictest/cockroach/order_by.slt",
  483. "test/sqllogictest/cockroach/ordinal_references.slt",
  484. "test/sqllogictest/cockroach/ordinality.slt",
  485. "test/sqllogictest/cockroach/orms-opt.slt",
  486. "test/sqllogictest/cockroach/orms.slt",
  487. "test/sqllogictest/cockroach/pg_catalog.slt",
  488. "test/sqllogictest/cockroach/pgoidtype.slt",
  489. # "test/sqllogictest/cockroach/postgres_jsonb.slt",
  490. # "test/sqllogictest/cockroach/postgresjoin.slt",
  491. "test/sqllogictest/cockroach/prepare.slt",
  492. "test/sqllogictest/cockroach/rename_column.slt",
  493. "test/sqllogictest/cockroach/rename_constraint.slt",
  494. "test/sqllogictest/cockroach/rename_database.slt",
  495. "test/sqllogictest/cockroach/rename_table.slt",
  496. "test/sqllogictest/cockroach/rename_view.slt",
  497. "test/sqllogictest/cockroach/returning.slt",
  498. "test/sqllogictest/cockroach/rows_from.slt",
  499. "test/sqllogictest/cockroach/scale.slt",
  500. "test/sqllogictest/cockroach/select_index_flags.slt",
  501. # "test/sqllogictest/cockroach/select_index_span_ranges.slt",
  502. # "test/sqllogictest/cockroach/select_index.slt",
  503. "test/sqllogictest/cockroach/select_search_path.slt",
  504. "test/sqllogictest/cockroach/select_table_alias.slt",
  505. # "test/sqllogictest/cockroach/select.slt",
  506. "test/sqllogictest/cockroach/shift.slt",
  507. # "test/sqllogictest/cockroach/sqlsmith.slt",
  508. "test/sqllogictest/cockroach/srfs.slt",
  509. # "test/sqllogictest/cockroach/statement_source.slt",
  510. # "test/sqllogictest/cockroach/suboperators.slt",
  511. # "test/sqllogictest/cockroach/subquery-opt.slt",
  512. "test/sqllogictest/cockroach/subquery.slt",
  513. "test/sqllogictest/cockroach/table.slt",
  514. # "test/sqllogictest/cockroach/target_names.slt",
  515. # "test/sqllogictest/cockroach/time.slt",
  516. "test/sqllogictest/cockroach/timestamp.slt",
  517. "test/sqllogictest/cockroach/truncate.slt",
  518. "test/sqllogictest/cockroach/tuple.slt",
  519. # "test/sqllogictest/cockroach/typing.slt",
  520. # "test/sqllogictest/cockroach/union-opt.slt",
  521. "test/sqllogictest/cockroach/union.slt",
  522. "test/sqllogictest/cockroach/update.slt",
  523. "test/sqllogictest/cockroach/upsert.slt",
  524. "test/sqllogictest/cockroach/uuid.slt",
  525. "test/sqllogictest/cockroach/values.slt",
  526. # "test/sqllogictest/cockroach/views.slt",
  527. "test/sqllogictest/cockroach/where.slt",
  528. "test/sqllogictest/cockroach/window.slt",
  529. "test/sqllogictest/cockroach/with.slt",
  530. # "test/sqllogictest/cockroach/zero.slt",
  531. "test/sqllogictest/postgres/float4.slt",
  532. "test/sqllogictest/postgres/float8.slt",
  533. "test/sqllogictest/postgres/join-lateral.slt",
  534. "test/sqllogictest/postgres/regex.slt",
  535. "test/sqllogictest/postgres/subselect.slt",
  536. "test/sqllogictest/postgres/pgcrypto/*.slt",
  537. "test/sqllogictest/introspection/cluster_log_compaction.slt",
  538. }
  539. tests = file_util.resolve_paths_with_wildcard(tests)
  540. tests_without_views = file_util.resolve_paths_with_wildcard(tests_without_views)
  541. tests_with_views = tests - tests_without_views - tests_exclude
  542. config = SltRunConfig()
  543. config.steps.append(DefaultSltRunStepConfig(tests_without_views))
  544. config.steps.append(SltRunStepConfig(tests_with_views, ["--auto-index-selects"]))
  545. return config
  546. def compileSlowSltConfig() -> SltRunConfig:
  547. config = SltRunConfig()
  548. # All CockroachDB and PostgreSQL SLTs can be run with --auto-index-selects, but require --no-fail
  549. cockroach_and_pg_tests = {
  550. "test/sqllogictest/cockroach/*.slt",
  551. "test/sqllogictest/postgres/*.slt",
  552. "test/sqllogictest/postgres/pgcrypto/*.slt",
  553. }
  554. cockroach_and_pg_tests = file_util.resolve_paths_with_wildcard(
  555. cockroach_and_pg_tests
  556. )
  557. config.steps.append(
  558. SltRunStepConfig(cockroach_and_pg_tests, ["--auto-index-selects", "--no-fail"])
  559. )
  560. tests = {
  561. "test/sqllogictest/*.slt",
  562. "test/sqllogictest/attributes/*.slt",
  563. "test/sqllogictest/introspection/*.slt",
  564. "test/sqllogictest/explain/*.slt",
  565. "test/sqllogictest/transform/*.slt",
  566. "test/sqllogictest/transform/fold_vs_dataflow/*.slt",
  567. "test/sqllogictest/transform/notice/*.slt",
  568. "test/sqllogictest/special/*",
  569. }
  570. tests_without_views_and_replica = {
  571. # errors:
  572. # https://github.com/MaterializeInc/database-issues/issues/6181
  573. "test/sqllogictest/list.slt",
  574. # transactions:
  575. "test/sqllogictest/distinct_arrangements.slt",
  576. "test/sqllogictest/github-3374.slt",
  577. "test/sqllogictest/introspection/cluster_log_compaction.slt",
  578. "test/sqllogictest/introspection/singlereplica_attribution_sources.slt",
  579. "test/sqllogictest/timedomain.slt",
  580. "test/sqllogictest/transactions.slt",
  581. # depends on unmaterializable functions
  582. "test/sqllogictest/regclass.slt",
  583. "test/sqllogictest/regproc.slt",
  584. "test/sqllogictest/regtype.slt",
  585. # different outputs:
  586. # seems expected for audit log to be different
  587. "test/sqllogictest/singlereplica_audit_log.slt",
  588. # different indexes auto-created
  589. "test/sqllogictest/cluster.slt",
  590. # different indexes auto-created
  591. "test/sqllogictest/object_ownership.slt",
  592. # https://github.com/MaterializeInc/database-issues/issues/6011
  593. "test/sqllogictest/interval.slt",
  594. # https://github.com/MaterializeInc/database-issues/issues/6011
  595. "test/sqllogictest/operator.slt",
  596. # specific replica size tested:
  597. "test/sqllogictest/managed_cluster.slt",
  598. "test/sqllogictest/web-console.slt",
  599. "test/sqllogictest/show_clusters.slt",
  600. }
  601. tests_no_auto_index_selects = {
  602. # pg_typeof contains public schema name in views
  603. "test/sqllogictest/cast.slt",
  604. "test/sqllogictest/map.slt",
  605. # pg_typeof contains public schema name in views
  606. "test/sqllogictest/typeof.slt",
  607. # https://github.com/MaterializeInc/database-issues/issues/9513#issuecomment-3128051157
  608. "test/sqllogictest/temporal.slt",
  609. }
  610. tests = file_util.resolve_paths_with_wildcard(tests)
  611. tests_without_views_and_replica = file_util.resolve_paths_with_wildcard(
  612. tests_without_views_and_replica
  613. )
  614. tests_no_auto_index_selects = file_util.resolve_paths_with_wildcard(
  615. tests_no_auto_index_selects
  616. )
  617. tests_with_views_or_replica = (
  618. tests - tests_without_views_and_replica - tests_no_auto_index_selects
  619. )
  620. config.steps.append(
  621. SltRunStepConfig(tests_with_views_or_replica, ["--auto-index-selects"])
  622. )
  623. config.steps.append(
  624. SltRunStepConfig(
  625. (tests_without_views_and_replica | tests_no_auto_index_selects), []
  626. )
  627. )
  628. # Due to performance issues (see below), we pick two selected SLTs from
  629. # the SQLite corpus that we can reasonably run with --auto-index-selects
  630. # and that include min/max query patterns. Note that none of the SLTs in
  631. # the corpus we presently use from SQLite contain top-k patterns.
  632. tests_with_views = {
  633. "test/sqllogictest/sqlite/test/index/random/1000/slt_good_0.test",
  634. "test/sqllogictest/sqlite/test/random/aggregates/slt_good_129.test",
  635. }
  636. tests = set(file_util.get_recursive_file_list("test/sqllogictest/sqlite/test"))
  637. tests_without_views = tests - tests_with_views
  638. # Run selected tests with --auto-index-selects
  639. config.steps.append(
  640. SltRunStepConfig(
  641. tests_with_views,
  642. ["--auto-index-selects", "--enable-table-keys"],
  643. )
  644. )
  645. # Too slow to run with --auto-index-selects, can't run together with
  646. # --auto-transactions, no differences seen in previous run. We might want to
  647. # revisit and see if we can periodically test them, even if it takes 2-3 days
  648. # for the run to finish.
  649. config.steps.append(
  650. SltRunStepConfig(
  651. tests_without_views,
  652. ["--auto-transactions", "--auto-index-tables", "--enable-table-keys"],
  653. )
  654. )
  655. return config