1use std::collections::BTreeMap;
11use std::future::Future;
12use std::net::ToSocketAddrs;
13use std::path::PathBuf;
14use std::sync::LazyLock;
15use std::time::Duration;
16use std::{env, fs};
17
18use anyhow::{Context, anyhow, bail};
19use async_trait::async_trait;
20use aws_credential_types::provider::ProvideCredentials;
21use aws_types::SdkConfig;
22use futures::future::FutureExt;
23use itertools::Itertools;
24use mz_adapter::catalog::{Catalog, ConnCatalog};
25use mz_adapter::session::Session;
26use mz_build_info::BuildInfo;
27use mz_catalog::config::ClusterReplicaSizeMap;
28use mz_catalog::durable::BootstrapArgs;
29use mz_kafka_util::client::{MzClientContext, create_new_client_config_simple};
30use mz_ore::error::ErrorExt;
31use mz_ore::metrics::MetricsRegistry;
32use mz_ore::now::SYSTEM_TIME;
33use mz_ore::retry::Retry;
34use mz_ore::task;
35use mz_ore::url::SensitiveUrl;
36use mz_persist_client::cache::PersistClientCache;
37use mz_persist_client::cfg::PersistConfig;
38use mz_persist_client::rpc::PubSubClientConnection;
39use mz_persist_client::{PersistClient, PersistLocation};
40use mz_sql::catalog::EnvironmentId;
41use mz_tls_util::make_tls;
42use rand::Rng;
43use rdkafka::ClientConfig;
44use rdkafka::producer::Producer;
45use regex::{Captures, Regex};
46use semver::Version;
47use tokio_postgres::error::{DbError, SqlState};
48use tracing::info;
49use url::Url;
50
51use crate::error::PosError;
52use crate::parser::{
53 Command, PosCommand, SqlExpectedError, SqlOutput, VersionConstraint, validate_ident,
54};
55use crate::util;
56use crate::util::postgres::postgres_client;
57
58pub mod consistency;
59
60mod file;
61mod fivetran;
62mod http;
63mod kafka;
64mod mysql;
65mod nop;
66mod persist;
67mod postgres;
68mod protobuf;
69mod psql;
70mod s3;
71mod schema_registry;
72mod set;
73mod skip_end;
74mod skip_if;
75mod sleep;
76mod sql;
77mod sql_server;
78mod version_check;
79mod webhook;
80
81#[derive(Debug)]
83pub struct Config {
84 pub arg_vars: BTreeMap<String, String>,
90 pub seed: Option<u32>,
92 pub reset: bool,
95 pub temp_dir: Option<String>,
100 pub source: Option<String>,
102 pub default_timeout: Duration,
104 pub default_max_tries: usize,
106 pub initial_backoff: Duration,
110 pub backoff_factor: f64,
114 pub consistency_checks: consistency::Level,
116 pub rewrite_results: bool,
118
119 pub materialize_pgconfig: tokio_postgres::Config,
123 pub materialize_internal_pgconfig: tokio_postgres::Config,
126 pub materialize_use_https: bool,
128 pub materialize_http_port: u16,
131 pub materialize_internal_http_port: u16,
134 pub materialize_password_sql_port: u16,
137 pub materialize_sasl_sql_port: u16,
140 pub materialize_params: Vec<(String, String)>,
142 pub materialize_catalog_config: Option<CatalogConfig>,
144 pub build_info: &'static BuildInfo,
146 pub materialize_cluster_replica_sizes: ClusterReplicaSizeMap,
148
149 pub persist_consensus_url: Option<SensitiveUrl>,
152 pub persist_blob_url: Option<SensitiveUrl>,
154
155 pub kafka_addr: String,
158 pub kafka_default_partitions: usize,
160 pub kafka_opts: Vec<(String, String)>,
163 pub schema_registry_url: Url,
165 pub cert_path: Option<String>,
170 pub cert_password: Option<String>,
172 pub ccsr_username: Option<String>,
175 pub ccsr_password: Option<String>,
178
179 pub aws_config: SdkConfig,
182 pub aws_account: String,
184
185 pub fivetran_destination_url: String,
188 pub fivetran_destination_files_path: String,
190}
191
192pub struct MaterializeState {
193 catalog_config: Option<CatalogConfig>,
194
195 sql_addr: String,
196 use_https: bool,
197 http_addr: String,
198 internal_sql_addr: String,
199 internal_http_addr: String,
200 password_sql_addr: String,
201 sasl_sql_addr: String,
202 user: String,
203 pgclient: tokio_postgres::Client,
204 environment_id: EnvironmentId,
205 bootstrap_args: BootstrapArgs,
206}
207
208pub struct State {
209 arg_vars: BTreeMap<String, String>,
211 cmd_vars: BTreeMap<String, String>,
212 seed: u32,
213 temp_path: PathBuf,
214 _tempfile: Option<tempfile::TempDir>,
215 default_timeout: Duration,
216 timeout: Duration,
217 max_tries: usize,
218 initial_backoff: Duration,
219 backoff_factor: f64,
220 consistency_checks: consistency::Level,
221 consistency_checks_adhoc_skip: bool,
222 regex: Option<Regex>,
223 regex_replacement: String,
224 error_line_count: usize,
225 error_string: String,
226
227 materialize: MaterializeState,
229
230 persist_consensus_url: Option<SensitiveUrl>,
232 persist_blob_url: Option<SensitiveUrl>,
233 build_info: &'static BuildInfo,
234 persist_clients: PersistClientCache,
235
236 schema_registry_url: Url,
238 ccsr_client: mz_ccsr::Client,
239 kafka_addr: String,
240 kafka_admin: rdkafka::admin::AdminClient<MzClientContext>,
241 kafka_admin_opts: rdkafka::admin::AdminOptions,
242 kafka_config: ClientConfig,
243 kafka_default_partitions: usize,
244 kafka_producer: rdkafka::producer::FutureProducer<MzClientContext>,
245 kafka_topics: BTreeMap<String, usize>,
246
247 aws_account: String,
249 aws_config: SdkConfig,
250
251 mysql_clients: BTreeMap<String, mysql_async::Conn>,
253 postgres_clients: BTreeMap<String, tokio_postgres::Client>,
254 sql_server_clients: BTreeMap<String, mz_sql_server_util::Client>,
255
256 fivetran_destination_url: String,
258 fivetran_destination_files_path: String,
259
260 rewrite_results: bool,
262 pub rewrites: Vec<Rewrite>,
264 pub rewrite_pos_start: usize,
266 pub rewrite_pos_end: usize,
268}
269
270pub struct Rewrite {
271 pub content: String,
272 pub start: usize,
273 pub end: usize,
274}
275
276impl State {
277 pub async fn initialize_cmd_vars(&mut self) -> Result<(), anyhow::Error> {
278 self.cmd_vars
279 .insert("testdrive.kafka-addr".into(), self.kafka_addr.clone());
280 self.cmd_vars.insert(
281 "testdrive.kafka-addr-resolved".into(),
282 self.kafka_addr
283 .to_socket_addrs()
284 .ok()
285 .and_then(|mut addrs| addrs.next())
286 .map(|addr| addr.to_string())
287 .unwrap_or_else(|| "#RESOLUTION-FAILURE#".into()),
288 );
289 self.cmd_vars.insert(
290 "testdrive.schema-registry-url".into(),
291 self.schema_registry_url.to_string(),
292 );
293 self.cmd_vars
294 .insert("testdrive.seed".into(), self.seed.to_string());
295 self.cmd_vars.insert(
296 "testdrive.temp-dir".into(),
297 self.temp_path.display().to_string(),
298 );
299 self.cmd_vars
300 .insert("testdrive.aws-region".into(), self.aws_region().into());
301 self.cmd_vars
302 .insert("testdrive.aws-endpoint".into(), self.aws_endpoint().into());
303 self.cmd_vars
304 .insert("testdrive.aws-account".into(), self.aws_account.clone());
305 {
306 let aws_credentials = self
307 .aws_config
308 .credentials_provider()
309 .ok_or_else(|| anyhow!("no AWS credentials provider configured"))?
310 .provide_credentials()
311 .await
312 .context("fetching AWS credentials")?;
313 self.cmd_vars.insert(
314 "testdrive.aws-access-key-id".into(),
315 aws_credentials.access_key_id().to_owned(),
316 );
317 self.cmd_vars.insert(
318 "testdrive.aws-secret-access-key".into(),
319 aws_credentials.secret_access_key().to_owned(),
320 );
321 self.cmd_vars.insert(
322 "testdrive.aws-token".into(),
323 aws_credentials
324 .session_token()
325 .map(|token| token.to_owned())
326 .unwrap_or_else(String::new),
327 );
328 }
329 self.cmd_vars.insert(
330 "testdrive.materialize-environment-id".into(),
331 self.materialize.environment_id.to_string(),
332 );
333 self.cmd_vars.insert(
334 "testdrive.materialize-sql-addr".into(),
335 self.materialize.sql_addr.clone(),
336 );
337 self.cmd_vars.insert(
338 "testdrive.materialize-internal-sql-addr".into(),
339 self.materialize.internal_sql_addr.clone(),
340 );
341 self.cmd_vars.insert(
342 "testdrive.materialize-password-sql-addr".into(),
343 self.materialize.password_sql_addr.clone(),
344 );
345 self.cmd_vars.insert(
346 "testdrive.materialize-sasl-sql-addr".into(),
347 self.materialize.sasl_sql_addr.clone(),
348 );
349 self.cmd_vars.insert(
350 "testdrive.materialize-user".into(),
351 self.materialize.user.clone(),
352 );
353 self.cmd_vars.insert(
354 "testdrive.fivetran-destination-url".into(),
355 self.fivetran_destination_url.clone(),
356 );
357 self.cmd_vars.insert(
358 "testdrive.fivetran-destination-files-path".into(),
359 self.fivetran_destination_files_path.clone(),
360 );
361
362 for (key, value) in env::vars() {
363 self.cmd_vars.insert(format!("env.{}", key), value);
364 }
365
366 for (key, value) in &self.arg_vars {
367 validate_ident(key)?;
368 self.cmd_vars
369 .insert(format!("arg.{}", key), value.to_string());
370 }
371
372 Ok(())
373 }
374 pub async fn with_catalog_copy<F, T>(
377 &self,
378 system_parameter_defaults: BTreeMap<String, String>,
379 build_info: &'static BuildInfo,
380 bootstrap_args: &BootstrapArgs,
381 enable_expression_cache_override: Option<bool>,
382 f: F,
383 ) -> Result<Option<T>, anyhow::Error>
384 where
385 F: FnOnce(ConnCatalog) -> T,
386 {
387 async fn persist_client(
388 persist_consensus_url: SensitiveUrl,
389 persist_blob_url: SensitiveUrl,
390 persist_clients: &PersistClientCache,
391 ) -> Result<PersistClient, anyhow::Error> {
392 let persist_location = PersistLocation {
393 blob_uri: persist_blob_url,
394 consensus_uri: persist_consensus_url,
395 };
396 Ok(persist_clients.open(persist_location).await?)
397 }
398
399 if let Some(CatalogConfig {
400 persist_consensus_url,
401 persist_blob_url,
402 }) = &self.materialize.catalog_config
403 {
404 let persist_client = persist_client(
405 persist_consensus_url.clone(),
406 persist_blob_url.clone(),
407 &self.persist_clients,
408 )
409 .await?;
410 let catalog = Catalog::open_debug_read_only_persist_catalog_config(
411 persist_client,
412 SYSTEM_TIME.clone(),
413 self.materialize.environment_id.clone(),
414 system_parameter_defaults,
415 build_info,
416 bootstrap_args,
417 enable_expression_cache_override,
418 )
419 .await?;
420 let res = f(catalog.for_session(&Session::dummy()));
421 catalog.expire().await;
422 Ok(Some(res))
423 } else {
424 Ok(None)
425 }
426 }
427
428 pub fn aws_endpoint(&self) -> &str {
429 self.aws_config.endpoint_url().unwrap_or("")
430 }
431
432 pub fn aws_region(&self) -> &str {
433 self.aws_config.region().map(|r| r.as_ref()).unwrap_or("")
434 }
435
436 pub fn clear_skip_consistency_checks(&mut self) -> bool {
439 std::mem::replace(&mut self.consistency_checks_adhoc_skip, false)
440 }
441
442 pub async fn reset_materialize(&self) -> Result<(), anyhow::Error> {
443 let (inner_client, _) = postgres_client(
444 &format!(
445 "postgres://mz_system:materialize@{}",
446 self.materialize.internal_sql_addr
447 ),
448 self.default_timeout,
449 )
450 .await?;
451
452 let version = inner_client
453 .query_one("SELECT mz_version_num()", &[])
454 .await
455 .context("getting version of materialize")
456 .map(|row| row.get::<_, i32>(0))?;
457
458 let semver = inner_client
459 .query_one("SELECT right(split_part(mz_version(), ' ', 1), -1)", &[])
460 .await
461 .context("getting semver of materialize")
462 .map(|row| row.get::<_, String>(0))?
463 .parse::<semver::Version>()
464 .context("parsing semver of materialize")?;
465
466 inner_client
467 .batch_execute("ALTER SYSTEM RESET ALL")
468 .await
469 .context("resetting materialize state: ALTER SYSTEM RESET ALL")?;
470
471 {
473 let rename_version = Version::parse("0.128.0-dev.1").expect("known to be valid");
474 let enable_unsafe_functions = if semver >= rename_version {
475 "unsafe_enable_unsafe_functions"
476 } else {
477 "enable_unsafe_functions"
478 };
479 let res = inner_client
480 .batch_execute(&format!("ALTER SYSTEM SET {enable_unsafe_functions} = on"))
481 .await
482 .context("enabling dangerous functions");
483 if let Err(e) = res {
484 match e.root_cause().downcast_ref::<DbError>() {
485 Some(e) if *e.code() == SqlState::CANT_CHANGE_RUNTIME_PARAM => {
486 info!(
487 "can't enable unsafe functions because the server is safe mode; \
488 testdrive scripts will fail if they use unsafe functions",
489 );
490 }
491 _ => return Err(e),
492 }
493 }
494 }
495
496 for row in inner_client
497 .query("SHOW DATABASES", &[])
498 .await
499 .context("resetting materialize state: SHOW DATABASES")?
500 {
501 let db_name: String = row.get(0);
502 if db_name.starts_with("testdrive_no_reset_") {
503 continue;
504 }
505 let query = format!(
506 "DROP DATABASE {}",
507 postgres_protocol::escape::escape_identifier(&db_name)
508 );
509 sql::print_query(&query, None);
510 inner_client.batch_execute(&query).await.context(format!(
511 "resetting materialize state: DROP DATABASE {}",
512 db_name,
513 ))?;
514 }
515
516 let inactive_user_clusters = "
518 WITH
519 active_user_clusters AS
520 (
521 SELECT DISTINCT cluster_id, object_id
522 FROM
523 (
524 SELECT cluster_id, id FROM mz_catalog.mz_sources
525 UNION ALL SELECT cluster_id, id FROM mz_catalog.mz_sinks
526 UNION ALL
527 SELECT cluster_id, id
528 FROM mz_catalog.mz_materialized_views
529 UNION ALL
530 SELECT cluster_id, id FROM mz_catalog.mz_indexes
531 UNION ALL
532 SELECT cluster_id, id
533 FROM mz_internal.mz_subscriptions
534 )
535 AS t (cluster_id, object_id)
536 WHERE cluster_id IS NOT NULL AND object_id LIKE 'u%'
537 )
538 SELECT name
539 FROM mz_catalog.mz_clusters
540 WHERE
541 id NOT IN ( SELECT cluster_id FROM active_user_clusters ) AND id LIKE 'u%'
542 AND
543 owner_id LIKE 'u%';";
544
545 let inactive_clusters = inner_client
546 .query(inactive_user_clusters, &[])
547 .await
548 .context("resetting materialize state: inactive_user_clusters")?;
549
550 if !inactive_clusters.is_empty() {
551 println!("cleaning up user clusters from previous tests...")
552 }
553
554 for cluster_name in inactive_clusters {
555 let cluster_name: String = cluster_name.get(0);
556 if cluster_name.starts_with("testdrive_no_reset_") {
557 continue;
558 }
559 let query = format!(
560 "DROP CLUSTER {}",
561 postgres_protocol::escape::escape_identifier(&cluster_name)
562 );
563 sql::print_query(&query, None);
564 inner_client.batch_execute(&query).await.context(format!(
565 "resetting materialize state: DROP CLUSTER {}",
566 cluster_name,
567 ))?;
568 }
569
570 inner_client
571 .batch_execute("CREATE DATABASE materialize")
572 .await
573 .context("resetting materialize state: CREATE DATABASE materialize")?;
574
575 if let Ok(rows) = inner_client.query("SELECT name FROM mz_roles", &[]).await {
579 for row in rows {
580 let role_name: String = row.get(0);
581 if role_name == self.materialize.user || role_name.starts_with("mz_") {
582 continue;
583 }
584 let query = format!(
585 "DROP ROLE {}",
586 postgres_protocol::escape::escape_identifier(&role_name)
587 );
588 sql::print_query(&query, None);
589 inner_client.batch_execute(&query).await.context(format!(
590 "resetting materialize state: DROP ROLE {}",
591 role_name,
592 ))?;
593 }
594 }
595
596 inner_client
598 .batch_execute(&format!(
599 "GRANT ALL PRIVILEGES ON SYSTEM TO {}",
600 self.materialize.user
601 ))
602 .await?;
603
604 inner_client
606 .batch_execute("GRANT USAGE ON DATABASE materialize TO PUBLIC")
607 .await?;
608 inner_client
609 .batch_execute(&format!(
610 "GRANT ALL PRIVILEGES ON DATABASE materialize TO {}",
611 self.materialize.user
612 ))
613 .await?;
614 inner_client
615 .batch_execute(&format!(
616 "GRANT ALL PRIVILEGES ON SCHEMA materialize.public TO {}",
617 self.materialize.user
618 ))
619 .await?;
620
621 let cluster = match version {
622 ..=8199 => "default",
623 8200.. => "quickstart",
624 };
625 inner_client
626 .batch_execute(&format!("GRANT USAGE ON CLUSTER {cluster} TO PUBLIC"))
627 .await?;
628 inner_client
629 .batch_execute(&format!(
630 "GRANT ALL PRIVILEGES ON CLUSTER {cluster} TO {}",
631 self.materialize.user
632 ))
633 .await?;
634
635 Ok(())
636 }
637
638 pub async fn reset_kafka(&self) -> Result<(), anyhow::Error> {
640 let mut errors: Vec<anyhow::Error> = Vec::new();
641
642 let metadata = self.kafka_producer.client().fetch_metadata(
643 None,
644 Some(std::cmp::max(Duration::from_secs(1), self.default_timeout)),
645 )?;
646
647 let testdrive_topics: Vec<_> = metadata
648 .topics()
649 .iter()
650 .filter_map(|t| {
651 if t.name().starts_with("testdrive-") {
652 Some(t.name())
653 } else {
654 None
655 }
656 })
657 .collect();
658
659 if !testdrive_topics.is_empty() {
660 match self
661 .kafka_admin
662 .delete_topics(&testdrive_topics, &self.kafka_admin_opts)
663 .await
664 {
665 Ok(res) => {
666 if res.len() != testdrive_topics.len() {
667 errors.push(anyhow!(
668 "kafka topic deletion returned {} results, but exactly {} expected",
669 res.len(),
670 testdrive_topics.len()
671 ));
672 }
673 for (res, topic) in res.iter().zip_eq(testdrive_topics.iter()) {
674 match res {
675 Ok(_)
676 | Err((_, rdkafka::types::RDKafkaErrorCode::UnknownTopicOrPartition)) => {
677 ()
678 }
679 Err((_, err)) => {
680 errors.push(anyhow!("unable to delete {}: {}", topic, err));
681 }
682 }
683 }
684 }
685 Err(e) => {
686 errors.push(e.into());
687 }
688 };
689 }
690
691 match self
692 .ccsr_client
693 .list_subjects()
694 .await
695 .context("listing schema registry subjects")
696 {
697 Ok(subjects) => {
698 let testdrive_subjects: Vec<_> = subjects
699 .iter()
700 .filter(|s| s.starts_with("testdrive-"))
701 .collect();
702
703 for subject in testdrive_subjects {
704 match self.ccsr_client.delete_subject(subject).await {
705 Ok(()) | Err(mz_ccsr::DeleteError::SubjectNotFound) => (),
706 Err(e) => errors.push(e.into()),
707 }
708 }
709 }
710 Err(e) => {
711 errors.push(e);
712 }
713 }
714
715 if errors.is_empty() {
716 Ok(())
717 } else {
718 bail!(
719 "deleting Kafka topics: {} errors: {}",
720 errors.len(),
721 errors
722 .into_iter()
723 .map(|e| e.to_string_with_causes())
724 .join("\n")
725 );
726 }
727 }
728}
729
730#[derive(Debug, Clone)]
732pub struct CatalogConfig {
733 pub persist_consensus_url: SensitiveUrl,
735 pub persist_blob_url: SensitiveUrl,
737}
738
739pub enum ControlFlow {
740 Continue,
741 SkipBegin,
742 SkipEnd,
743}
744
745#[async_trait]
746pub(crate) trait Run {
747 async fn run(self, state: &mut State) -> Result<ControlFlow, PosError>;
748}
749
750#[async_trait]
751impl Run for PosCommand {
752 async fn run(self, state: &mut State) -> Result<ControlFlow, PosError> {
753 macro_rules! handle_version {
754 ($version_constraint:expr) => {
755 match $version_constraint {
756 Some(VersionConstraint { min, max }) => {
757 match version_check::run_version_check(min, max, state).await {
758 Ok(true) => return Ok(ControlFlow::Continue),
759 Ok(false) => {}
760 Err(err) => return Err(PosError::new(err, self.pos)),
761 }
762 }
763 None => {}
764 }
765 };
766 }
767
768 let wrap_err = |e| PosError::new(e, self.pos);
769 let ignore_prefix = match &self.command {
772 Command::Builtin(builtin, _) => Some(builtin.name.clone()),
773 _ => None,
774 };
775 let subst = |msg: &str, vars: &BTreeMap<String, String>| {
776 substitute_vars(msg, vars, &ignore_prefix, false).map_err(wrap_err)
777 };
778 let subst_re = |msg: &str, vars: &BTreeMap<String, String>| {
779 substitute_vars(msg, vars, &ignore_prefix, true).map_err(wrap_err)
780 };
781
782 let r = match self.command {
783 Command::Builtin(mut builtin, version_constraint) => {
784 handle_version!(version_constraint);
785 for val in builtin.args.values_mut() {
786 *val = subst(val, &state.cmd_vars)?;
787 }
788 for line in &mut builtin.input {
789 *line = subst(line, &state.cmd_vars)?;
790 }
791 match builtin.name.as_ref() {
792 "check-consistency" => consistency::run_consistency_checks(state).await,
793 "skip-consistency-checks" => {
794 consistency::skip_consistency_checks(builtin, state)
795 }
796 "check-shard-tombstone" => {
797 consistency::run_check_shard_tombstone(builtin, state).await
798 }
799 "fivetran-destination" => {
800 fivetran::run_destination_command(builtin, state).await
801 }
802 "file-append" => file::run_append(builtin, state).await,
803 "file-delete" => file::run_delete(builtin, state).await,
804 "http-request" => http::run_request(builtin, state).await,
805 "kafka-add-partitions" => kafka::run_add_partitions(builtin, state).await,
806 "kafka-create-topic" => kafka::run_create_topic(builtin, state).await,
807 "kafka-wait-topic" => kafka::run_wait_topic(builtin, state).await,
808 "kafka-delete-records" => kafka::run_delete_records(builtin, state).await,
809 "kafka-delete-topic-flaky" => kafka::run_delete_topic(builtin, state).await,
810 "kafka-ingest" => kafka::run_ingest(builtin, state).await,
811 "kafka-verify-data" => kafka::run_verify_data(builtin, state).await,
812 "kafka-verify-commit" => kafka::run_verify_commit(builtin, state).await,
813 "kafka-verify-topic" => kafka::run_verify_topic(builtin, state).await,
814 "mysql-connect" => mysql::run_connect(builtin, state).await,
815 "mysql-execute" => mysql::run_execute(builtin, state).await,
816 "nop" => nop::run_nop(),
817 "postgres-connect" => postgres::run_connect(builtin, state).await,
818 "postgres-execute" => postgres::run_execute(builtin, state).await,
819 "postgres-verify-slot" => postgres::run_verify_slot(builtin, state).await,
820 "protobuf-compile-descriptors" => {
821 protobuf::run_compile_descriptors(builtin, state).await
822 }
823 "psql-execute" => psql::run_execute(builtin, state).await,
824 "s3-verify-data" => s3::run_verify_data(builtin, state).await,
825 "s3-verify-keys" => s3::run_verify_keys(builtin, state).await,
826 "s3-file-upload" => s3::run_upload(builtin, state).await,
827 "s3-set-presigned-url" => s3::run_set_presigned_url(builtin, state).await,
828 "schema-registry-publish" => schema_registry::run_publish(builtin, state).await,
829 "schema-registry-verify" => schema_registry::run_verify(builtin, state).await,
830 "schema-registry-wait" => schema_registry::run_wait(builtin, state).await,
831 "skip-if" => skip_if::run_skip_if(builtin, state).await,
832 "skip-end" => skip_end::run_skip_end(),
833 "sql-server-connect" => sql_server::run_connect(builtin, state).await,
834 "sql-server-execute" => sql_server::run_execute(builtin, state).await,
835 "sql-server-set-from-sql" => sql_server::run_set_from_sql(builtin, state).await,
836 "persist-force-compaction" => {
837 persist::run_force_compaction(builtin, state).await
838 }
839 "random-sleep" => sleep::run_random_sleep(builtin),
840 "set-regex" => set::run_regex_set(builtin, state),
841 "unset-regex" => set::run_regex_unset(builtin, state),
842 "set-sql-timeout" => set::run_sql_timeout(builtin, state),
843 "set-max-tries" => set::run_max_tries(builtin, state),
844 "sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment" => {
845 sleep::run_sleep(builtin)
846 }
847 "set" => set::set_vars(builtin, state),
848 "set-arg-default" => set::run_set_arg_default(builtin, state),
849 "set-from-sql" => set::run_set_from_sql(builtin, state).await,
850 "set-from-file" => set::run_set_from_file(builtin, state).await,
851 "webhook-append" => webhook::run_append(builtin, state).await,
852 _ => {
853 return Err(PosError::new(
854 anyhow!("unknown built-in command {}", builtin.name),
855 self.pos,
856 ));
857 }
858 }
859 }
860 Command::Sql(mut sql, version_constraint) => {
861 handle_version!(version_constraint);
862 sql.query = subst(&sql.query, &state.cmd_vars)?;
863 if let SqlOutput::Full { expected_rows, .. } = &mut sql.expected_output {
864 for row in expected_rows {
865 for col in row {
866 *col = subst(col, &state.cmd_vars)?;
867 }
868 }
869 }
870 sql::run_sql(sql, state).await
871 }
872 Command::FailSql(mut sql, version_constraint) => {
873 handle_version!(version_constraint);
874 sql.query = subst(&sql.query, &state.cmd_vars)?;
875 sql.expected_error = match &sql.expected_error {
876 SqlExpectedError::Contains(s) => {
877 SqlExpectedError::Contains(subst(s, &state.cmd_vars)?)
878 }
879 SqlExpectedError::Exact(s) => {
880 SqlExpectedError::Exact(subst(s, &state.cmd_vars)?)
881 }
882 SqlExpectedError::Regex(s) => {
883 SqlExpectedError::Regex(subst_re(s, &state.cmd_vars)?)
884 }
885 SqlExpectedError::Timeout => SqlExpectedError::Timeout,
886 };
887 sql::run_fail_sql(sql, state).await
888 }
889 };
890
891 r.map_err(wrap_err)
892 }
893}
894
895fn substitute_vars(
897 msg: &str,
898 vars: &BTreeMap<String, String>,
899 ignore_prefix: &Option<String>,
900 regex_escape: bool,
901) -> Result<String, anyhow::Error> {
902 static RE: LazyLock<Regex> = LazyLock::new(|| Regex::new(r"\$\{([^}]+)\}").unwrap());
903 let mut err = None;
904 let out = RE.replace_all(msg, |caps: &Captures| {
905 let name = &caps[1];
906 if let Some(ignore_prefix) = &ignore_prefix {
907 if name.starts_with(format!("{}.", ignore_prefix).as_str()) {
908 return caps.get(0).unwrap().as_str().to_string();
910 }
911 }
912
913 if let Some(val) = vars.get(name) {
914 if regex_escape {
915 regex::escape(val)
916 } else {
917 val.to_string()
918 }
919 } else {
920 err = Some(anyhow!("unknown variable: {}", name));
921 "#VAR-MISSING#".to_string()
922 }
923 });
924 match err {
925 Some(err) => Err(err),
926 None => Ok(out.into_owned()),
927 }
928}
929
930pub async fn create_state(
938 config: &Config,
939) -> Result<(State, impl Future<Output = Result<(), anyhow::Error>>), anyhow::Error> {
940 let seed = config.seed.unwrap_or_else(|| rand::thread_rng().r#gen());
941
942 let (_tempfile, temp_path) = match &config.temp_dir {
943 Some(temp_dir) => {
944 fs::create_dir_all(temp_dir).context("creating temporary directory")?;
945 (None, PathBuf::from(&temp_dir))
946 }
947 _ => {
948 let tempfile_handle = tempfile::tempdir().context("creating temporary directory")?;
951 let temp_path = tempfile_handle.path().to_path_buf();
952 (Some(tempfile_handle), temp_path)
953 }
954 };
955
956 let materialize_catalog_config = config.materialize_catalog_config.clone();
957
958 let materialize_url = util::postgres::config_url(&config.materialize_pgconfig)?;
959 info!("Connecting to {}", materialize_url.as_str());
960 let (pgclient, pgconn) = Retry::default()
961 .max_duration(config.default_timeout)
962 .retry_async_canceling(|_| async move {
963 let mut pgconfig = config.materialize_pgconfig.clone();
964 pgconfig.connect_timeout(config.default_timeout);
965 let tls = make_tls(&pgconfig)?;
966 pgconfig.connect(tls).await.map_err(|e| anyhow!(e))
967 })
968 .await?;
969
970 let pgconn_task = task::spawn(|| "pgconn_task", pgconn).map(|join| {
971 join.expect("pgconn_task unexpectedly canceled")
972 .context("running SQL connection")
973 });
974
975 let materialize_state =
976 create_materialize_state(&config, materialize_catalog_config, pgclient).await?;
977
978 let schema_registry_url = config.schema_registry_url.to_owned();
979
980 let ccsr_client = {
981 let mut ccsr_config = mz_ccsr::ClientConfig::new(schema_registry_url.clone());
982
983 if let Some(cert_path) = &config.cert_path {
984 let cert = fs::read(cert_path).context("reading cert")?;
985 let pass = config.cert_password.as_deref().unwrap_or("").to_owned();
986 let ident = mz_ccsr::tls::Identity::from_pkcs12_der(cert, pass)
987 .context("reading keystore file as pkcs12")?;
988 ccsr_config = ccsr_config.identity(ident);
989 }
990
991 if let Some(ccsr_username) = &config.ccsr_username {
992 ccsr_config = ccsr_config.auth(ccsr_username.clone(), config.ccsr_password.clone());
993 }
994
995 ccsr_config.build().context("Creating CCSR client")?
996 };
997
998 let (kafka_addr, kafka_admin, kafka_admin_opts, kafka_producer, kafka_topics, kafka_config) = {
999 use rdkafka::admin::{AdminClient, AdminOptions};
1000 use rdkafka::producer::FutureProducer;
1001
1002 let mut kafka_config = create_new_client_config_simple();
1003 kafka_config.set("bootstrap.servers", &config.kafka_addr);
1004 kafka_config.set("group.id", "materialize-testdrive");
1005 kafka_config.set("auto.offset.reset", "earliest");
1006 kafka_config.set("isolation.level", "read_committed");
1007 if let Some(cert_path) = &config.cert_path {
1008 kafka_config.set("security.protocol", "ssl");
1009 kafka_config.set("ssl.keystore.location", cert_path);
1010 if let Some(cert_password) = &config.cert_password {
1011 kafka_config.set("ssl.keystore.password", cert_password);
1012 }
1013 }
1014 kafka_config.set("message.max.bytes", "15728640");
1015
1016 for (key, value) in &config.kafka_opts {
1017 kafka_config.set(key, value);
1018 }
1019
1020 let admin: AdminClient<_> = kafka_config
1021 .create_with_context(MzClientContext::default())
1022 .with_context(|| format!("opening Kafka connection: {}", config.kafka_addr))?;
1023
1024 let admin_opts = AdminOptions::new().operation_timeout(Some(config.default_timeout));
1025
1026 let producer: FutureProducer<_> = kafka_config
1027 .create_with_context(MzClientContext::default())
1028 .with_context(|| format!("opening Kafka producer connection: {}", config.kafka_addr))?;
1029
1030 let topics = BTreeMap::new();
1031
1032 (
1033 config.kafka_addr.to_owned(),
1034 admin,
1035 admin_opts,
1036 producer,
1037 topics,
1038 kafka_config,
1039 )
1040 };
1041
1042 let mut state = State {
1043 arg_vars: config.arg_vars.clone(),
1045 cmd_vars: BTreeMap::new(),
1046 seed,
1047 temp_path,
1048 _tempfile,
1049 default_timeout: config.default_timeout,
1050 timeout: config.default_timeout,
1051 max_tries: config.default_max_tries,
1052 initial_backoff: config.initial_backoff,
1053 backoff_factor: config.backoff_factor,
1054 consistency_checks: config.consistency_checks,
1055 consistency_checks_adhoc_skip: false,
1056 regex: None,
1057 regex_replacement: set::DEFAULT_REGEX_REPLACEMENT.into(),
1058 rewrite_results: config.rewrite_results,
1059 error_line_count: 0,
1060 error_string: "".to_string(),
1061
1062 materialize: materialize_state,
1064
1065 persist_consensus_url: config.persist_consensus_url.clone(),
1067 persist_blob_url: config.persist_blob_url.clone(),
1068 build_info: config.build_info,
1069 persist_clients: PersistClientCache::new(
1070 PersistConfig::new_default_configs(config.build_info, SYSTEM_TIME.clone()),
1071 &MetricsRegistry::new(),
1072 |_, _| PubSubClientConnection::noop(),
1073 ),
1074
1075 schema_registry_url,
1077 ccsr_client,
1078 kafka_addr,
1079 kafka_admin,
1080 kafka_admin_opts,
1081 kafka_config,
1082 kafka_default_partitions: config.kafka_default_partitions,
1083 kafka_producer,
1084 kafka_topics,
1085
1086 aws_account: config.aws_account.clone(),
1088 aws_config: config.aws_config.clone(),
1089
1090 mysql_clients: BTreeMap::new(),
1092 postgres_clients: BTreeMap::new(),
1093 sql_server_clients: BTreeMap::new(),
1094
1095 fivetran_destination_url: config.fivetran_destination_url.clone(),
1097 fivetran_destination_files_path: config.fivetran_destination_files_path.clone(),
1098
1099 rewrites: Vec::new(),
1100 rewrite_pos_start: 0,
1101 rewrite_pos_end: 0,
1102 };
1103 state.initialize_cmd_vars().await?;
1104 Ok((state, pgconn_task))
1105}
1106
1107async fn create_materialize_state(
1108 config: &&Config,
1109 materialize_catalog_config: Option<CatalogConfig>,
1110 pgclient: tokio_postgres::Client,
1111) -> Result<MaterializeState, anyhow::Error> {
1112 let materialize_url = util::postgres::config_url(&config.materialize_pgconfig)?;
1113 let materialize_internal_url =
1114 util::postgres::config_url(&config.materialize_internal_pgconfig)?;
1115
1116 for (key, value) in &config.materialize_params {
1117 pgclient
1118 .batch_execute(&format!("SET {key} = {value}"))
1119 .await
1120 .context("setting session parameter")?;
1121 }
1122
1123 let materialize_user = config
1124 .materialize_pgconfig
1125 .get_user()
1126 .expect("testdrive URL must contain user")
1127 .to_string();
1128
1129 let materialize_sql_addr = format!(
1130 "{}:{}",
1131 materialize_url.host_str().unwrap(),
1132 materialize_url.port().unwrap()
1133 );
1134 let materialize_http_addr = format!(
1135 "{}:{}",
1136 materialize_url.host_str().unwrap(),
1137 config.materialize_http_port
1138 );
1139 let materialize_internal_sql_addr = format!(
1140 "{}:{}",
1141 materialize_internal_url.host_str().unwrap(),
1142 materialize_internal_url.port().unwrap()
1143 );
1144 let materialize_password_sql_addr = format!(
1145 "{}:{}",
1146 materialize_url.host_str().unwrap(),
1147 config.materialize_password_sql_port
1148 );
1149 let materialize_sasl_sql_addr = format!(
1150 "{}:{}",
1151 materialize_url.host_str().unwrap(),
1152 config.materialize_sasl_sql_port
1153 );
1154 let materialize_internal_http_addr = format!(
1155 "{}:{}",
1156 materialize_internal_url.host_str().unwrap(),
1157 config.materialize_internal_http_port
1158 );
1159 let environment_id = pgclient
1160 .query_one("SELECT mz_environment_id()", &[])
1161 .await?
1162 .get::<_, String>(0)
1163 .parse()
1164 .context("parsing environment ID")?;
1165
1166 let bootstrap_args = BootstrapArgs {
1167 cluster_replica_size_map: config.materialize_cluster_replica_sizes.clone(),
1168 default_cluster_replica_size: "ABC".to_string(),
1169 default_cluster_replication_factor: 1,
1170 bootstrap_role: None,
1171 };
1172
1173 let materialize_state = MaterializeState {
1174 catalog_config: materialize_catalog_config,
1175 sql_addr: materialize_sql_addr,
1176 use_https: config.materialize_use_https,
1177 http_addr: materialize_http_addr,
1178 internal_sql_addr: materialize_internal_sql_addr,
1179 internal_http_addr: materialize_internal_http_addr,
1180 password_sql_addr: materialize_password_sql_addr,
1181 sasl_sql_addr: materialize_sasl_sql_addr,
1182 user: materialize_user,
1183 pgclient,
1184 environment_id,
1185 bootstrap_args,
1186 };
1187
1188 Ok(materialize_state)
1189}