From 6788c6cdf4a922775d8527528a43a172c6e9ad57 Mon Sep 17 00:00:00 2001 From: Dennis Felsing Date: Sun, 29 Sep 2024 14:10:10 +0000 Subject: [PATCH] Move materialize issue references to database-issues Collected the issues via git grep "https://github.com/MaterializeInc/materialize/issues/" | sed -e "s/.*issues\/\([0-9]*\).*/\1/g" | sort -n | uniq git grep "materialize#" | sed -e "s/.*materialize#\([0-9]*\).*/\1/g" | sort -n | uniq > out Then found the correct new urls via while read line; do curl --header "Authorization: Bearer $GITHUB_TOKEN" --header "X-GitHub-Api-Version: 2022-11-28" "https://api.github.com/repos/MaterializeInc/materialize/issues/$line" -s | jq -r ".url" ; done And ran sed commands to replace them. I'm not sure if we actually want this, but I prefer to have the actual link instead of one that will be redirected. --- ci/nightly/pipeline.template.yml | 34 ++-- ci/plugins/cloudtest/hooks/post-command | 2 +- .../pipeline.template.yml | 8 +- ci/test/cargo-test/mzcompose.py | 2 +- ci/test/pipeline.template.yml | 2 +- clippy.toml | 2 +- .../design/20210311_prometheus_source.md | 2 +- .../20210413_source_sink_resource_sharing.md | 2 +- .../design/20210601_build_mirrelationexpr.md | 2 +- ...10707_qgm_sql_high_level_representation.md | 22 +-- .../20210713_S3_sources_with_headers.md | 2 +- ...r_handling_surfacing_connector_failures.md | 16 +- .../design/20220109_monotonic_topk.md | 6 +- .../design/20220404_metadata_storage.md | 2 +- .../20220411_reclocking_implementation.md | 2 +- .../20220516_transactional_consistency.md | 4 +- doc/developer/design/20220718_envelopes.md | 2 +- .../design/20220819_bounded_input_reliance.md | 2 +- .../20221204_with_mutually_recursive.md | 4 +- .../design/20230209_invalid_accumulations.md | 6 +- .../20230210_storage_persist_sink_redux.md | 2 +- ...30223_stabilize_with_mutually_recursive.md | 104 ++++++------ .../design/20230306_persist_mfp_pushdown.md | 4 +- .../design/20230315_backup_restore.md | 6 +- .../20230322_metrics_since_granularity.md | 4 +- .../design/20230330_recursion_limit.md | 10 +- ...0411_envelope_upsert_order_by_timestamp.md | 2 +- .../20230411_faster_dataflow_shutdown.md | 18 +-- .../design/20230418_persist_state_pubsub.md | 4 +- .../design/20230418_subscribe_output.md | 2 +- .../design/20230419_stash_migrations.md | 10 +- .../20230421_stabilize_monotonic_select.md | 30 ++-- .../design/20230512_mir_cost_model.md | 34 ++-- .../design/20230528_managed_clusters.md | 4 +- .../design/20230531_compute_metrics.md | 12 +- .../design/20230607_shift_status_updates.md | 2 +- .../design/20230613_cloudtest_improvements.md | 2 +- .../design/20230615_webhook_source.md | 4 +- .../design/20230714_optimizer_interface.md | 2 +- .../design/20230717_alter_set_cluster.md | 2 +- ...0_single_statement_explitic_transaction.md | 2 +- .../20230728_left_join_stack_consolidation.md | 2 +- .../design/20230801_sink_conn_mgmt.md | 6 +- .../20230802_explain_running_dataflows.md | 12 +- .../design/20230806_durable_catalog_state.md | 6 +- .../design/20230810_alter_connection.md | 2 +- .../design/20230814_unbilled_replicas.md | 2 +- .../design/20230829_topk_size_hint.md | 10 +- doc/developer/design/20230903_avro_doc.md | 4 +- doc/developer/design/20231027_refresh_mvs.md | 22 +-- doc/developer/design/20231102_read_holds.md | 6 +- .../20231103_privatelink_status_table.md | 2 +- .../design/20231110_aws_connections.md | 2 +- .../20231113_optimizer_notice_catalog.md | 8 +- doc/developer/design/20231117_copy_to_s3.md | 2 +- .../20231201_catalog_migration_to_persist.md | 4 +- ...20231204_query_lifecycle_events_logging.md | 2 +- ...pute_operator_hydration_status_tracking.md | 4 +- .../design/20240108_source_metrics_2.md | 4 +- .../20240205_cluster_specific_optimization.md | 2 +- ..._reconfiguration_of_managed_clusters_v1.md | 4 +- .../design/20240609_error_handling.md | 8 +- .../20240610_unified_compute_introspection.md | 6 +- ...5_source_versioning__table_from_sources.md | 12 +- .../design/20320328_persist_columnar.md | 10 +- doc/developer/diagnostic-questions.md | 2 +- .../content/ingest-data/cdc-sql-server.md | 2 +- .../content/ingest-data/mysql/debezium.md | 2 +- .../content/ingest-data/postgres/debezium.md | 2 +- doc/user/content/integrations/_index.md | 2 +- misc/cockroach/setup_materialize.sql | 2 +- misc/dbt-materialize/CHANGELOG.md | 14 +- misc/dbt-materialize/README.md | 2 +- .../dbt/adapters/materialize/impl.py | 2 +- .../dbt/adapters/materialize/relation.py | 2 +- .../include/materialize/macros/adapters.sql | 4 +- .../tests/adapter/test_constraints.py | 4 +- .../tests/adapter/test_relation_types.py | 2 +- .../tests/adapter/test_utils.py | 2 +- .../buildkite_insights/segfaults/find.py | 2 +- .../materialize/checks/all_checks/debezium.py | 2 +- .../checks/all_checks/identifiers.py | 2 +- .../checks/all_checks/materialized_views.py | 2 +- .../checks/all_checks/multiple_partitions.py | 2 +- .../checks/all_checks/peek_cancellation.py | 2 +- .../materialize/checks/all_checks/pg_cdc.py | 6 +- .../materialize/cli/ci_annotate_errors.py | 2 +- .../feature_benchmark/benchmark_versioning.py | 4 +- .../scenarios/benchmark_main.py | 4 +- .../feature_benchmark/scenarios/subscribe.py | 6 +- .../feature_flag_consistency_ignore_filter.py | 4 +- .../materialize/mzcompose/composition.py | 2 +- .../mzcompose/services/cockroach.py | 2 +- .../mzcompose/services/materialized.py | 2 +- .../generators/query_generator.py | 2 +- ...rnal_output_inconsistency_ignore_filter.py | 28 ++-- .../types/date_time_types_provider.py | 6 +- .../validation/error_message_normalizer.py | 6 +- .../parallel_benchmark/scenarios.py | 6 +- .../materialize/parallel_workload/action.py | 38 ++--- .../custom/predefined_pg_queries.py | 2 +- .../execution/pg_dialect_adjuster.py | 2 +- .../pg_inconsistency_ignore_filter.py | 148 +++++++++--------- .../validation/pg_result_comparator.py | 6 +- misc/python/materialize/rustc_flags.py | 2 +- misc/python/materialize/sqlsmith.py | 12 +- .../data/build/build_data_storage.py | 2 +- .../feature_benchmark_result_storage.py | 4 +- .../parallel_benchmark_result_storage.py | 2 +- .../setup/views/02-build-job.sql | 2 +- .../zippy/backup_and_restore_actions.py | 2 +- .../src/catalog/builtin_table_updates.rs | 4 +- src/adapter/src/catalog/open.rs | 4 +- src/adapter/src/catalog/state.rs | 2 +- src/adapter/src/coord.rs | 2 +- src/adapter/src/coord/command_handler.rs | 6 +- src/adapter/src/coord/ddl.rs | 4 +- src/adapter/src/coord/sequencer/cluster.rs | 6 +- src/adapter/src/coord/sequencer/inner.rs | 6 +- src/adapter/src/error.rs | 2 +- src/adapter/src/util.rs | 2 +- src/avro/tests/schema.rs | 2 +- src/aws-util/src/s3_uploader.rs | 6 +- src/catalog/src/builtin.rs | 8 +- src/catalog/src/durable/upgrade.rs | 2 +- src/catalog/src/memory/objects.rs | 2 +- src/ccsr/tests/client.rs | 4 +- src/compute-client/src/controller.rs | 4 +- src/compute-client/src/controller/error.rs | 32 ++-- src/compute-client/src/controller/instance.rs | 6 +- src/compute-client/src/logging.rs | 52 +++--- src/compute-client/src/metrics.rs | 10 +- src/compute-client/src/protocol/command.rs | 12 +- src/compute-client/src/protocol/history.rs | 4 +- src/compute-client/src/protocol/response.rs | 4 +- src/compute-client/src/service.rs | 4 +- src/compute-types/src/dataflows.rs | 4 +- src/compute-types/src/explain.rs | 2 +- src/compute-types/src/plan/interpret/api.rs | 32 ++-- src/compute-types/src/plan/join.rs | 4 +- src/compute-types/src/plan/lowering.rs | 2 +- src/compute-types/src/plan/transform/api.rs | 6 +- .../plan/transform/relax_must_consolidate.rs | 2 +- src/compute-types/src/sinks.rs | 32 ++-- src/compute/src/compute_state.rs | 4 +- src/compute/src/logging/initialize.rs | 2 +- src/compute/src/render.rs | 4 +- src/compute/src/render/context.rs | 2 +- src/compute/src/render/errors.rs | 2 +- src/compute/src/render/join/linear_join.rs | 2 +- src/compute/src/render/reduce.rs | 6 +- src/compute/src/render/top_k.rs | 4 +- src/compute/src/server.rs | 2 +- src/compute/src/sink/subscribe.rs | 2 +- src/dyncfg/src/lib.rs | 2 +- src/environmentd/tests/pgwire.rs | 6 +- src/environmentd/tests/server.rs | 8 +- src/environmentd/tests/sql.rs | 16 +- src/expr/src/explain.rs | 2 +- src/expr/src/interpret.rs | 2 +- src/expr/src/relation/func.rs | 8 +- src/expr/src/scalar.rs | 2 +- src/expr/src/scalar/func.rs | 4 +- src/ore/src/retry.rs | 2 +- src/ore/src/tracing.rs | 2 +- src/persist-client/src/batch.rs | 4 +- src/persist-client/src/cli/admin.rs | 4 +- src/persist-client/src/internal/compact.rs | 2 +- src/persist-client/src/internal/machine.rs | 4 +- src/persist-client/src/internal/state.rs | 8 +- src/persist-client/src/internal/state_diff.rs | 2 +- .../src/internal/state_versions.rs | 2 +- src/persist-client/src/internal/trace.rs | 4 +- src/persist-client/src/internals_bench.rs | 2 +- src/persist-client/src/lib.rs | 2 +- src/persist-client/src/read.rs | 4 +- src/persist-client/src/rpc.rs | 2 +- src/persist-client/src/usage.rs | 8 +- src/persist/src/postgres.rs | 2 +- src/persist/src/s3.rs | 2 +- src/repr/src/explain.rs | 2 +- src/repr/src/row/encoding.rs | 4 +- src/repr/src/scalar.rs | 2 +- src/repr/tests/strconv.rs | 2 +- src/service/src/grpc.rs | 2 +- src/sql-parser/src/parser.rs | 4 +- src/sql-parser/tests/testdata/explain | 2 +- src/sql/src/ast/transform.rs | 4 +- src/sql/src/catalog.rs | 4 +- src/sql/src/names.rs | 6 +- src/sql/src/plan.rs | 2 +- src/sql/src/plan/expr.rs | 2 +- src/sql/src/plan/lowering.rs | 10 +- src/sql/src/plan/lowering/variadic_left.rs | 2 +- src/sql/src/plan/query.rs | 12 +- src/sql/src/plan/statement.rs | 4 +- src/sql/src/plan/statement/ddl.rs | 14 +- src/sql/src/plan/statement/ddl/connection.rs | 8 +- src/sql/src/plan/statement/scl.rs | 2 +- src/sql/src/plan/statement/show.rs | 4 +- src/sql/src/plan/transform_ast.rs | 2 +- src/sql/src/plan/transform_expr.rs | 2 +- src/sql/src/pure.rs | 2 +- src/sql/src/session/vars.rs | 4 +- src/sqllogictest/src/runner.rs | 2 +- src/storage-client/src/storage_collections.rs | 2 +- src/storage-controller/src/collection_mgmt.rs | 2 +- src/storage-controller/src/instance.rs | 2 +- src/storage-controller/src/lib.rs | 2 +- .../src/s3_oneshot_sink/pgcopy.rs | 2 +- src/storage-types/src/connections.rs | 2 +- src/storage/src/healthcheck.rs | 4 +- src/storage/src/render/sinks.rs | 4 +- src/storage/src/sink/kafka.rs | 2 +- src/storage/src/source/reclock.rs | 2 +- src/testdrive/src/action/kafka/ingest.rs | 2 +- src/timestamp-oracle/src/postgres_oracle.rs | 2 +- src/transform/src/analysis.rs | 2 +- src/transform/src/analysis/monotonic.rs | 2 +- src/transform/src/dataflow.rs | 6 +- src/transform/src/fold_constants.rs | 2 +- src/transform/src/fusion/join.rs | 2 +- src/transform/src/lib.rs | 18 +-- src/transform/src/literal_lifting.rs | 2 +- src/transform/src/normalize_lets.rs | 4 +- src/transform/src/predicate_pushdown.rs | 8 +- src/transform/src/reduction_pushdown.rs | 4 +- src/transform/src/typecheck.rs | 2 +- .../non_null_requirements.spec | 2 +- .../test_transforms/reduction_pushdown.spec | 4 +- src/txn-wal/src/txn_cache.rs | 4 +- src/txn-wal/src/txn_read.rs | 2 +- test/bounded-memory/mzcompose.py | 2 +- .../tests/generic/makes_progress.sql | 2 +- test/canary-load/mzcompose.py | 4 +- test/cloud-canary/mzcompose.py | 2 +- test/cloudtest/test_cluster_rehydration.py | 2 +- test/cloudtest/test_compute.py | 4 +- test/cloudtest/test_full_testdrive.py | 2 +- test/cloudtest/test_metrics.py | 2 +- test/cloudtest/test_replica_restart.py | 2 +- test/cloudtest/test_secrets.py | 4 +- test/cluster-isolation/mzcompose.py | 2 +- test/cluster/mzcompose.py | 46 +++--- test/cluster/resources/resource-limits.td | 6 +- test/debezium/postgres/02-add-primary-key.td | 2 +- test/debezium/postgres/02-drop-primary-key.td | 2 +- .../postgres/05-add-column-primary-key.td | 2 +- .../postgres/08-primary-key-extend.td | 2 +- .../postgres/08-primary-key-shrink.td | 2 +- .../postgres/10-remove-nullability.td | 2 +- test/debezium/postgres/12-change-decimal.td | 2 +- test/debezium/postgres/20-types-temporal.td | 2 +- test/debezium/sql-server/40-check-types.td | 4 +- test/kafka-resumption/mzcompose.py | 6 +- .../check-from-v0.27.0-schema-registry.td | 2 +- test/limits/mzcompose.py | 58 +++---- test/mysql-cdc-old-syntax/alter-source.td | 4 +- .../alter-table-after-source.td | 10 +- test/mysql-cdc-old-syntax/empty-table.td | 2 +- test/mysql-cdc-old-syntax/gh-10981.td | 2 +- .../mysql-cdc-old-syntax/invisible-columns.td | 2 +- test/mysql-cdc-old-syntax/mysql-cdc-ssl.td | 12 +- test/mysql-cdc-old-syntax/mysql-cdc.td | 20 +-- .../schema-restart/after-restart.td | 2 +- .../two-destination-schemas.td | 2 +- .../two-source-schemas.td | 2 +- test/mysql-cdc-old-syntax/types-enum.td | 4 +- .../mzcompose.py | 4 +- test/mysql-cdc-resumption/mzcompose.py | 4 +- test/mysql-cdc/alter-source.td | 6 +- test/mysql-cdc/alter-table-after-source.td | 10 +- test/mysql-cdc/gh-10981.td | 2 +- test/mysql-cdc/invisible-columns.td | 2 +- test/mysql-cdc/mysql-cdc-ssl.td | 12 +- test/mysql-cdc/mysql-cdc.td | 22 +-- .../mysql-cdc/schema-restart/after-restart.td | 2 +- test/mysql-cdc/two-destination-schemas.td | 2 +- test/mysql-cdc/two-source-schemas.td | 2 +- test/mysql-cdc/types-enum.td | 4 +- test/pg-cdc-old-syntax/gh-10981.td | 2 +- test/pg-cdc-old-syntax/mzcompose.py | 6 +- test/pg-cdc-old-syntax/pg-cdc-ssl.td | 2 +- test/pg-cdc-old-syntax/pg-cdc.td | 6 +- .../publication-with-publish-option.td | 2 +- .../replica-identity-default-nothing.td | 2 +- .../status/04-drop-publication.td | 2 +- test/pg-cdc/alter-source.td | 2 +- test/pg-cdc/gh-10981.td | 2 +- test/pg-cdc/mzcompose.py | 6 +- test/pg-cdc/pg-cdc-ssl.td | 2 +- test/pg-cdc/pg-cdc.td | 10 +- .../pg-cdc/publication-with-publish-option.td | 2 +- .../replica-identity-default-nothing.td | 2 +- test/pg-cdc/statistics.td | 4 +- test/pg-cdc/status/04-drop-publication.td | 2 +- test/platform-checks/mzcompose.py | 2 +- test/replica-isolation/mzcompose.py | 2 +- test/restart/mzcompose.py | 4 +- test/retain-history/mzcompose.py | 2 +- test/rqg/mzcompose.py | 2 +- test/source-sink-errors/mzcompose.py | 16 +- test/sqllogictest/aggregates.slt | 2 +- test/sqllogictest/arrays.slt | 4 +- test/sqllogictest/char.slt | 6 +- test/sqllogictest/chbench.slt | 50 +++--- test/sqllogictest/cockroach/aggregate.slt | 16 +- test/sqllogictest/cockroach/alter_table.slt | 6 +- test/sqllogictest/cockroach/apply_join.slt | 4 +- test/sqllogictest/cockroach/array.slt | 6 +- .../cockroach/builtin_function.slt | 12 +- .../sqllogictest/cockroach/collatedstring.slt | 2 +- test/sqllogictest/cockroach/computed.slt | 4 +- test/sqllogictest/cockroach/create_as.slt | 4 +- test/sqllogictest/cockroach/datetime.slt | 2 +- test/sqllogictest/cockroach/distinct_on.slt | 4 +- test/sqllogictest/cockroach/insert.slt | 2 +- test/sqllogictest/cockroach/join.slt | 2 +- test/sqllogictest/cockroach/json.slt | 2 +- test/sqllogictest/cockroach/json_builtins.slt | 2 +- test/sqllogictest/cockroach/ordinality.slt | 2 +- test/sqllogictest/cockroach/orms.slt | 2 +- test/sqllogictest/cockroach/pg_catalog.slt | 2 +- test/sqllogictest/cockroach/pgoidtype.slt | 2 +- test/sqllogictest/cockroach/prepare.slt | 4 +- .../cockroach/rename_database.slt | 2 +- test/sqllogictest/cockroach/rename_table.slt | 2 +- test/sqllogictest/cockroach/select.slt | 2 +- test/sqllogictest/cockroach/sqlsmith.slt | 8 +- test/sqllogictest/cockroach/srfs.slt | 6 +- .../cockroach/statement_source.slt | 2 +- test/sqllogictest/cockroach/subquery-opt.slt | 4 +- test/sqllogictest/cockroach/subquery.slt | 4 +- .../cockroach/subquery_correlated.slt | 6 +- test/sqllogictest/cockroach/table.slt | 2 +- test/sqllogictest/cockroach/tuple.slt | 2 +- test/sqllogictest/cockroach/update.slt | 8 +- test/sqllogictest/cockroach/upsert.slt | 4 +- test/sqllogictest/cockroach/views.slt | 2 +- test/sqllogictest/cockroach/window.slt | 8 +- test/sqllogictest/coercion.slt | 2 +- test/sqllogictest/cte.slt | 4 +- test/sqllogictest/dates-times.slt | 6 +- test/sqllogictest/encode.slt | 2 +- .../explain/broken_statements.slt | 2 +- .../explain/optimized_plan_as_text.slt | 2 +- .../optimized_plan_as_text_redacted.slt | 2 +- .../explain/physical_plan_as_text.slt | 2 +- .../physical_plan_as_text_redacted.slt | 2 +- test/sqllogictest/extract.slt | 2 +- test/sqllogictest/filter-pushdown.slt | 2 +- test/sqllogictest/float.slt | 2 +- test/sqllogictest/funcs.slt | 10 +- test/sqllogictest/github-11139.slt | 2 +- test/sqllogictest/github-11568.slt | 2 +- test/sqllogictest/github-13857.slt | 2 +- test/sqllogictest/github-16036.slt | 2 +- test/sqllogictest/github-17616.slt | 2 +- test/sqllogictest/github-17762.slt | 2 +- test/sqllogictest/github-17808.slt | 2 +- test/sqllogictest/github-18522.slt | 4 +- test/sqllogictest/github-18708.slt | 2 +- test/sqllogictest/github-19273.slt | 6 +- test/sqllogictest/github-19290.slt | 2 +- test/sqllogictest/github-19511.slt | 4 +- test/sqllogictest/github-21244.slt | 2 +- test/sqllogictest/github-21501.slt | 2 +- test/sqllogictest/github-23015.slt | 2 +- test/sqllogictest/github-24511.slt | 2 +- test/sqllogictest/github-25015.slt | 4 +- test/sqllogictest/github-25769.slt | 2 +- test/sqllogictest/github-26352.slt | 2 +- test/sqllogictest/github-26707.slt | 2 +- test/sqllogictest/github-28166.slt | 2 +- test/sqllogictest/github-28174-29110.slt | 6 +- test/sqllogictest/github-7168.slt | 2 +- test/sqllogictest/github-7472.slt | 2 +- test/sqllogictest/github-8241.slt | 4 +- test/sqllogictest/github-8713.slt | 2 +- test/sqllogictest/github-8717.slt | 2 +- test/sqllogictest/github-9027.slt | 2 +- test/sqllogictest/github-9147.slt | 2 +- test/sqllogictest/github-9504.slt | 2 +- test/sqllogictest/github-9782.slt | 2 +- test/sqllogictest/github-9931.slt | 2 +- test/sqllogictest/interval.slt | 2 +- test/sqllogictest/joins.slt | 28 ++-- test/sqllogictest/jsonb.slt | 8 +- test/sqllogictest/like.slt | 2 +- test/sqllogictest/managed_cluster.slt | 2 +- test/sqllogictest/materialized_views.slt | 4 +- test/sqllogictest/mzcompose.py | 6 +- test/sqllogictest/numeric.slt | 6 +- test/sqllogictest/order_by.slt | 6 +- .../outer_join_simplification.slt | 12 +- .../sqllogictest/postgres-incompatibility.slt | 2 +- test/sqllogictest/postgres/subselect.slt | 4 +- test/sqllogictest/regclass.slt | 2 +- test/sqllogictest/regex.slt | 6 +- test/sqllogictest/regressions.slt | 2 +- test/sqllogictest/regtype.slt | 2 +- test/sqllogictest/scoping.slt | 4 +- test/sqllogictest/string.slt | 2 +- test/sqllogictest/subquery.slt | 14 +- test/sqllogictest/table_func.slt | 16 +- test/sqllogictest/temporal.slt | 4 +- test/sqllogictest/timezone.slt | 2 +- test/sqllogictest/transactions.slt | 2 +- .../transform/aggregation_nullability.slt | 4 +- .../transform/column_knowledge.slt | 4 +- .../sqllogictest/transform/fold_constants.slt | 8 +- .../fold_vs_dataflow/1_numbers_dataflow.slt | 4 +- .../3_number_aggfns_dataflow.slt | 2 +- .../fold_vs_dataflow/4_text_dataflow.slt | 2 +- .../5_repeat_row_dataflow.slt | 2 +- .../fold_vs_dataflow/5_repeat_row_folding.slt | 2 +- test/sqllogictest/transform/join_index.slt | 4 +- test/sqllogictest/transform/lifting.slt | 2 +- .../transform/literal_constraints.slt | 12 +- .../transform/non_null_requirements.slt | 2 +- .../sqllogictest/transform/normalize_lets.slt | 20 +-- ...index_too_wide_for_literal_constraints.slt | 2 +- .../transform/predicate_pushdown.slt | 14 +- .../transform/reduction_pushdown.slt | 2 +- .../sqllogictest/transform/redundant_join.slt | 2 +- test/sqllogictest/transform/relation_cse.slt | 2 +- test/sqllogictest/transform/scalability.slt | 2 +- test/sqllogictest/transform/scalar_cse.slt | 2 +- .../transform/threshold_elision.slt | 6 +- test/sqllogictest/types.slt | 2 +- .../uniqueness_propagation_filter.slt | 2 +- test/sqllogictest/unsigned_int.slt | 2 +- test/sqllogictest/updates.slt | 4 +- test/sqllogictest/vars.slt | 2 +- test/sqllogictest/window_funcs.slt | 10 +- test/sqllogictest/with_mutually_recursive.slt | 4 +- test/sqlsmith/mzcompose.py | 2 +- .../pg-source-after-ssh-restart.td | 2 +- test/storage-usage/mzcompose.py | 2 +- test/testdrive/avro-nonnull-record.td | 2 +- test/testdrive/consolidation.td | 2 +- test/testdrive/copy-to-s3-minio.td | 2 +- test/testdrive/create-views.td | 2 +- test/testdrive/dataflow-cleanup.td | 2 +- test/testdrive/date_func.td | 2 +- test/testdrive/dependencies.td | 2 +- .../kafka-avro-debezium-transaction.td | 2 +- test/testdrive/distinct-arrangements.td | 4 +- test/testdrive/drop.td | 4 +- test/testdrive/fivetran-destination.td | 2 +- test/testdrive/github-10587.td | 2 +- test/testdrive/github-13790.td | 2 +- test/testdrive/github-15095.td | 2 +- test/testdrive/github-18608.td | 2 +- test/testdrive/github-21031.td | 2 +- test/testdrive/github-2276.td | 2 +- test/testdrive/github-3281.td | 2 +- test/testdrive/github-5668.td | 2 +- test/testdrive/github-5774.td | 2 +- test/testdrive/github-5983.td | 2 +- test/testdrive/github-5984.td | 2 +- test/testdrive/github-6305.td | 2 +- test/testdrive/github-6950.td | 2 +- test/testdrive/github-7191.td | 2 +- test/testdrive/index-source-stuck.td | 2 +- test/testdrive/kafka-avro-sinks.td | 6 +- test/testdrive/kafka-include-key-sources.td | 2 +- test/testdrive/kafka-json-sinks.td | 2 +- .../materialized-view-refresh-options.td | 4 +- test/testdrive/negative-multiplicities.td | 12 +- test/testdrive/protobuf-basic.td | 2 +- test/testdrive/quickstart.td | 2 +- test/testdrive/rename.td | 2 +- test/testdrive/render-delta-join.td | 2 +- test/testdrive/replica-targeting.td | 2 +- test/testdrive/source-tables.td | 4 +- test/testdrive/subexpression-replacement.td | 2 +- test/testdrive/subquery-scalar-errors.td | 2 +- test/testdrive/temporary.td | 2 +- test/testdrive/timestamps-debezium-kafka.td | 2 +- test/testdrive/top-k-monotonic.td | 2 +- ...transactions-timedomain-nonmaterialized.td | 4 +- test/testdrive/types.td | 2 +- test/zippy/mzcompose.py | 2 +- 484 files changed, 1297 insertions(+), 1283 deletions(-) diff --git a/ci/nightly/pipeline.template.yml b/ci/nightly/pipeline.template.yml index 350df6c6433ee..a1458f854ad0b 100644 --- a/ci/nightly/pipeline.template.yml +++ b/ci/nightly/pipeline.template.yml @@ -656,7 +656,7 @@ steps: - id: checks-parallel-drop-create-default-replica label: "Checks parallel + DROP/CREATE replica %N" depends_on: build-aarch64 - skip: "Affected by materialize#23882" + skip: "Affected by database-issues#7172" timeout_in_minutes: 180 parallelism: 2 agents: @@ -669,7 +669,7 @@ steps: - id: checks-parallel-restart-clusterd-compute label: "Checks parallel + restart compute clusterd %N" depends_on: build-aarch64 - skip: "Affected by materialize#23882" + skip: "Affected by database-issues#7172" timeout_in_minutes: 180 parallelism: 2 agents: @@ -682,7 +682,7 @@ steps: - id: checks-parallel-restart-entire-mz label: "Checks parallel + restart of the entire Mz %N" depends_on: build-aarch64 - skip: "Affected by materialize#23882" + skip: "Affected by database-issues#7172" timeout_in_minutes: 180 parallelism: 2 agents: @@ -695,7 +695,7 @@ steps: - id: checks-parallel-restart-environmentd-clusterd-storage label: "Checks parallel + restart of environmentd & storage clusterd %N" depends_on: build-aarch64 - skip: "Affected by materialize#23882" + skip: "Affected by database-issues#7172" timeout_in_minutes: 180 parallelism: 2 agents: @@ -708,7 +708,7 @@ steps: - id: checks-parallel-kill-clusterd-storage label: "Checks parallel + kill storage clusterd %N" depends_on: build-aarch64 - skip: "Affected by materialize#23882" + skip: "Affected by database-issues#7172" timeout_in_minutes: 180 parallelism: 2 agents: @@ -721,7 +721,7 @@ steps: - id: checks-parallel-restart-redpanda label: "Checks parallel + restart Redpanda & Debezium %N" depends_on: build-aarch64 - skip: "Affected by materialize#23882" + skip: "Affected by database-issues#7172" timeout_in_minutes: 180 parallelism: 2 agents: @@ -873,7 +873,7 @@ steps: label: "K8s recovery: storage on failing node" depends_on: build-aarch64 timeout_in_minutes: 60 - # TODO: materialize#25108 (k8s node recovery tests flaky) + # TODO: database-issues#7499 (k8s node recovery tests flaky) retry: automatic: - exit_status: 1 @@ -899,7 +899,7 @@ steps: label: "K8s recovery: compute on failing node" depends_on: build-aarch64 timeout_in_minutes: 60 - # TODO: materialize#25108 (k8s node recovery tests flaky) + # TODO: database-issues#7499 (k8s node recovery tests flaky) retry: automatic: - exit_status: 1 @@ -925,7 +925,7 @@ steps: label: "K8s recovery: replicated compute on failing node" depends_on: build-aarch64 timeout_in_minutes: 60 - # TODO: materialize#25108 (k8s node recovery tests flaky) + # TODO: database-issues#7499 (k8s node recovery tests flaky) retry: automatic: - exit_status: 1 @@ -951,7 +951,7 @@ steps: label: "K8s recovery: envd on failing node" depends_on: build-aarch64 timeout_in_minutes: 60 - # TODO: materialize#25108 (k8s node recovery tests flaky) + # TODO: database-issues#7499 (k8s node recovery tests flaky) retry: automatic: - exit_status: 1 @@ -1157,7 +1157,7 @@ steps: plugins: - ./ci/plugins/mzcompose: composition: sqlsmith - # TODO(def-) Increase number of joins when materialize#23451 is fixed + # TODO(def-) Increase number of joins when database-issues#7046 is fixed args: [--max-joins=1, --runtime=1500] - id: sqlsmith-explain @@ -1169,7 +1169,7 @@ steps: plugins: - ./ci/plugins/mzcompose: composition: sqlsmith - # TODO(def-) Increase number of joins when materialize#23451 is fixed + # TODO(def-) Increase number of joins when database-issues#7046 is fixed args: [--max-joins=5, --explain-only, --runtime=1500] - group: SQLancer @@ -1232,7 +1232,7 @@ steps: - ./ci/plugins/mzcompose: composition: rqg args: ["simple-aggregates", "--seed=$BUILDKITE_JOB_ID"] - skip: "flaky until materialize#27954 is fixed" + skip: "flaky until database-issues#8197 is fixed" - id: rqg-db3-joins label: "RQG dbt3-joins workload" @@ -1249,7 +1249,7 @@ steps: - id: rqg-lateral-joins label: "RQG lateral-joins workload" depends_on: build-aarch64 - skip: "flaky until materialize#25865 is fixed" + skip: "flaky until database-issues#7713 is fixed" timeout_in_minutes: 45 agents: queue: hetzner-aarch64-4cpu-8gb @@ -1283,7 +1283,7 @@ steps: - id: rqg-wmr label: "RQG WMR workload" depends_on: build-aarch64 - skip: "flaky until materialize#24904 is fixed" + skip: "flaky until database-issues#7433 is fixed" timeout_in_minutes: 45 agents: queue: hetzner-aarch64-8cpu-16gb @@ -1339,7 +1339,7 @@ steps: - id: retain-history label: "Check retain history" depends_on: build-aarch64 - skip: "materialize#24479" + skip: "database-issues#7310" timeout_in_minutes: 15 agents: queue: hetzner-aarch64-8cpu-16gb @@ -1440,7 +1440,7 @@ steps: depends_on: build-aarch64 artifact_paths: [parallel-workload-queries.log.zst] timeout_in_minutes: 90 - skip: "TODO(def-): Reenable when materialize#2392 is fixed" + skip: "TODO(def-): Reenable when database-issues#835 is fixed" agents: queue: hetzner-aarch64-16cpu-32gb plugins: diff --git a/ci/plugins/cloudtest/hooks/post-command b/ci/plugins/cloudtest/hooks/post-command index cc7d6d46c0465..0dd67303d9954 100755 --- a/ci/plugins/cloudtest/hooks/post-command +++ b/ci/plugins/cloudtest/hooks/post-command @@ -94,7 +94,7 @@ buildkite-agent artifact upload "$artifacts_str" bin/ci-builder run stable bin/ci-annotate-errors --test-cmd="$(cat test_cmd)" "${artifacts[@]}" > ci-annotate-errors.log || CI_ANNOTATE_ERRORS_RESULT=$? buildkite-agent artifact upload "ci-annotate-errors.log" -# File should not be empty, see materialize#25369 +# File should not be empty, see database-issues#7569 test -s kubectl-get-logs-previous.log if [ "$CI_ANNOTATE_ERRORS_RESULT" -ne 0 ]; then diff --git a/ci/release-qualification/pipeline.template.yml b/ci/release-qualification/pipeline.template.yml index 40b6a276688c2..bd92e9ca3a857 100644 --- a/ci/release-qualification/pipeline.template.yml +++ b/ci/release-qualification/pipeline.template.yml @@ -115,7 +115,7 @@ steps: plugins: - ./ci/plugins/mzcompose: composition: zippy - # Execution times longer than 4h are apparently not feasible at this time due to materialize#17845 + # Execution times longer than 4h are apparently not feasible at this time due to database-issues#5201 args: [--scenario=ClusterReplicas, --actions=10000, --max-execution-time=4h] - id: zippy-user-tables-large @@ -166,7 +166,7 @@ steps: plugins: - ./ci/plugins/mzcompose: composition: zippy - # TODO(def-) Increase number of actions when materialize#24250 is fixed + # TODO(def-) Increase number of actions when database-issues#7257 is fixed args: [--scenario=KafkaParallelInsert, --transaction-isolation=serializable, --actions=50000, --max-execution-time=4h] - group: Benchmarks @@ -216,7 +216,7 @@ steps: - ./ci/plugins/mzcompose: composition: sqlsmith args: [--max-joins=2, --runtime=6000] - skip: "Reenable when materialize#23451 is fixed" + skip: "Reenable when database-issues#7046 is fixed" - id: sqlsmith-explain-long label: "Longer SQLsmith explain" @@ -228,7 +228,7 @@ steps: - ./ci/plugins/mzcompose: composition: sqlsmith args: [--max-joins=15, --explain-only, --runtime=6000] - skip: "Reenable when materialize#23451 is fixed" + skip: "Reenable when database-issues#7046 is fixed" - id: test-preflight-check-rollback label: Test with preflight check and rollback diff --git a/ci/test/cargo-test/mzcompose.py b/ci/test/cargo-test/mzcompose.py index 17d148a36b4fb..de30eacc05164 100644 --- a/ci/test/cargo-test/mzcompose.py +++ b/ci/test/cargo-test/mzcompose.py @@ -64,7 +64,7 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: args = parser.parse_args() c.up("zookeeper", "kafka", "schema-registry", "postgres", "cockroach", "minio") # Heads up: this intentionally runs on the host rather than in a Docker - # image. See materialize#13010. + # image. See database-issues#3739. postgres_url = ( f"postgres://postgres:postgres@localhost:{c.default_port('postgres')}" ) diff --git a/ci/test/pipeline.template.yml b/ci/test/pipeline.template.yml index dc10b0eab1f37..06a522e6749c2 100644 --- a/ci/test/pipeline.template.yml +++ b/ci/test/pipeline.template.yml @@ -747,7 +747,7 @@ steps: composition: rtr-combined agents: queue: hetzner-aarch64-8cpu-16gb - skip: "Flakes because of materialize#29289" + skip: "Flakes because of database-issues#8489" - id: skip-version-upgrade label: "Skip Version Upgrade" diff --git a/clippy.toml b/clippy.toml index 7b31c9306d191..f9a001298bca7 100644 --- a/clippy.toml +++ b/clippy.toml @@ -84,7 +84,7 @@ disallowed-methods = [ disallowed-macros = [ { path = "proptest::prop_oneof", reason = "use `proptest::strategy::Union::new` instead" }, - { path = "log::log", reason = "use the macros provided by `tracing` instead (materialize#9992)" }, + { path = "log::log", reason = "use the macros provided by `tracing` instead (database-issues#3001)" }, { path = "tracing::instrument", reason = "use `mz_ore::instrument` instead" }, ] diff --git a/doc/developer/design/20210311_prometheus_source.md b/doc/developer/design/20210311_prometheus_source.md index b65d2e54d791c..05fd8c4bbd179 100644 --- a/doc/developer/design/20210311_prometheus_source.md +++ b/doc/developer/design/20210311_prometheus_source.md @@ -134,4 +134,4 @@ I think if we want to go down this road, we'll want something like an "is a" tab // These questions may be technical, product, or anything in-between. --> -[#5825]: https://github.com/MaterializeInc/materialize/issues/5825 +[#5825]: https://api.github.com/repos/MaterializeInc/database-issues/issues/1793 diff --git a/doc/developer/design/20210413_source_sink_resource_sharing.md b/doc/developer/design/20210413_source_sink_resource_sharing.md index b1f56badd4fa9..8ed9a3bcc20a2 100644 --- a/doc/developer/design/20210413_source_sink_resource_sharing.md +++ b/doc/developer/design/20210413_source_sink_resource_sharing.md @@ -287,5 +287,5 @@ CREATE OR REPLACE VIEW kafka_view AS SELECT col1, col2, col3 FROM kafka_src; CREATE VIEW view3 AS SELECT col3 FROM kafka_view; ``` -[#3791]: https://github.com/MaterializeInc/materialize/issues/3791 +[#3791]: https://api.github.com/repos/MaterializeInc/database-issues/issues/1182 [jettison-discussion]: https://github.com/MaterializeInc/materialize/pull/6450/files#r612735779 diff --git a/doc/developer/design/20210601_build_mirrelationexpr.md b/doc/developer/design/20210601_build_mirrelationexpr.md index 4dcd2e76bea67..0aba9e3180289 100644 --- a/doc/developer/design/20210601_build_mirrelationexpr.md +++ b/doc/developer/design/20210601_build_mirrelationexpr.md @@ -82,7 +82,7 @@ Figuring out how to more efficiently implement the syntax has been an ongoing issue. There has been some earlier design discussions on this, which are documented in the comments of [#5684]. -[#5684]: https://github.com/MaterializeInc/materialize/issues/5684 +[#5684]: https://api.github.com/repos/MaterializeInc/database-issues/issues/1750 ## Goals diff --git a/doc/developer/design/20210707_qgm_sql_high_level_representation.md b/doc/developer/design/20210707_qgm_sql_high_level_representation.md index a64a696a48486..5fbd55e92b9b6 100644 --- a/doc/developer/design/20210707_qgm_sql_high_level_representation.md +++ b/doc/developer/design/20210707_qgm_sql_high_level_representation.md @@ -32,16 +32,16 @@ and the one proposed in this document, which adds normalization at the SQL level * be easy to use, * be normalization-friendly, * allow supporting complex features such as recursion in CTEs, -* Proper support of `LATERAL` joins ([#6875](https://github.com/MaterializeInc/materialize/issues/6875)) +* Proper support of `LATERAL` joins ([#6875](https://api.github.com/repos/MaterializeInc/database-issues/issues/2143)) * Support for functional dependency analysis during name resolution -([#7647](https://github.com/MaterializeInc/materialize/issues/7647)). +([#7647](https://api.github.com/repos/MaterializeInc/database-issues/issues/2369)). Existing issues that should be addressed sooner rather than later: -* [sql: handle aggregates that refer exclusively to outer columns #3720](https://github.com/MaterializeInc/materialize/issues/3720) +* [sql: handle aggregates that refer exclusively to outer columns #3720](https://api.github.com/repos/MaterializeInc/database-issues/issues/1163) * [sql: make SQL responsible for its equality semantics #7597](https://github.com/MaterializeInc/materialize/pull/7597) -* [sql: wrong results with outer join due to incorrect name resolution #7618](https://github.com/MaterializeInc/materialize/issues/7618) -* [sql: ambiguous column name error reported for equivalent expressions in the selection list](https://github.com/MaterializeInc/materialize/issues/7549) -* [Ignore the selection list in EXISTS subqueries](https://github.com/MaterializeInc/materialize/issues/6490) +* [sql: wrong results with outer join due to incorrect name resolution #7618](https://api.github.com/repos/MaterializeInc/database-issues/issues/2364) +* [sql: ambiguous column name error reported for equivalent expressions in the selection list](https://api.github.com/repos/MaterializeInc/database-issues/issues/2340) +* [Ignore the selection list in EXISTS subqueries](https://api.github.com/repos/MaterializeInc/database-issues/issues/2020) ## Non-Goals @@ -168,7 +168,7 @@ the query or discovered via some query transformation. As shown above, there aren't many different types of operators, since QGM is meant to be a representation for query normalization. The set of operators listed above is very close to the one suggested in -[#692](https://github.com/MaterializeInc/materialize/issues/692). +[#692](https://api.github.com/repos/MaterializeInc/database-issues/issues/224). The core operator is represented by the `Select` box, which represents a whole query block (sub-block). @@ -405,7 +405,7 @@ A `SELECT` query is a grouping query if any of the following conditions is met: * the projection of the query contains any aggregate which parameters are constant, * the projection of the query contains any aggregate where all columns referenced within it come from the tables in the `FROM` clause, either directly in the projection or within a subquery - (see [#3720](https://github.com/MaterializeInc/materialize/issues/3720)) + (see [#3720](https://api.github.com/repos/MaterializeInc/database-issues/issues/1163)) That means that in order to determine whether a query is a grouping query or not, we must inspect the projection of the query first. For this reason, after having processed the `FROM` clause and the `WHERE` clause, we will @@ -502,9 +502,9 @@ the nested `Select` box would not be guaranteed and hence, it could not be merge Some normalization transformations are better/easier done with a representation at a higher level than our current `MirRelationExpr` representation. Specially those around SQL-specific concepts such as outer joins that are -lost during lowering. Several examples of this are [#6932](https://github.com/MaterializeInc/materialize/issues/6932), -[#6987](https://github.com/MaterializeInc/materialize/issues/6987) or -[#6988](https://github.com/MaterializeInc/materialize/issues/6988), but the list of unsupported cases that are +lost during lowering. Several examples of this are [#6932](https://api.github.com/repos/MaterializeInc/database-issues/issues/2155), +[#6987](https://api.github.com/repos/MaterializeInc/database-issues/issues/2174) or +[#6988](https://api.github.com/repos/MaterializeInc/database-issues/issues/2175), but the list of unsupported cases that are hard to support at the moment is much longer. ### Query decorrelation during normalization diff --git a/doc/developer/design/20210713_S3_sources_with_headers.md b/doc/developer/design/20210713_S3_sources_with_headers.md index b89813de8dd51..ee445d3c8304f 100644 --- a/doc/developer/design/20210713_S3_sources_with_headers.md +++ b/doc/developer/design/20210713_S3_sources_with_headers.md @@ -209,7 +209,7 @@ schema we will put its dataflow into an error state to prevent users from reading invalid data. Currently there is no way to recover from this state. There are a few options: -* Issue [#6605](https://github.com/MaterializeInc/materialize/issues/6605#issuecomment-880101445) +* Issue [#6605](https://api.github.com/repos/MaterializeInc/database-issues/issues/2060#issuecomment-880101445) suggests that with a full sql-accessible catalog we could use standard sql to retract S3 objects that are invalid, thereby recovering. * We could add specific syntax for S3 sources that allow retracting invalid CSV diff --git a/doc/developer/design/20210728_error_handling_surfacing_connector_failures.md b/doc/developer/design/20210728_error_handling_surfacing_connector_failures.md index 259ab6ab4af9a..b8448f55bf4a9 100644 --- a/doc/developer/design/20210728_error_handling_surfacing_connector_failures.md +++ b/doc/developer/design/20210728_error_handling_surfacing_connector_failures.md @@ -166,20 +166,20 @@ There were some, which lead to the addition of a `DEGRADED` state above. ## References - Product thoughts from Nicolle: https://docs.google.com/document/d/10RsEnpJJBN-lQKyYl08KRQ15Ckn-_gCWa5MAnZcc-ts/edit#heading=h.yvax2aldz1n9 - - Meta-issue by Eli: https://github.com/MaterializeInc/materialize/issues/7115 + - Meta-issue by Eli: https://api.github.com/repos/MaterializeInc/database-issues/issues/2208 Critical: - - https://github.com/MaterializeInc/materialize/issues/7396: sink error during startup kills materialized - - https://github.com/MaterializeInc/materialize/issues/7009: failure in S3 source is printed to log but source doesn't error out - - https://github.com/MaterializeInc/materialize/issues/6586: transient broker failure leads to error in log, but source doesn't report as errored and doesn't continue to produce data + - https://api.github.com/repos/MaterializeInc/database-issues/issues/2304: sink error during startup kills materialized + - https://api.github.com/repos/MaterializeInc/database-issues/issues/2180: failure in S3 source is printed to log but source doesn't error out + - https://api.github.com/repos/MaterializeInc/database-issues/issues/2051: transient broker failure leads to error in log, but source doesn't report as errored and doesn't continue to produce data Unknown: - - https://github.com/MaterializeInc/materialize/issues/7035: fetching in source doesn't continue after transient failure - - https://github.com/MaterializeInc/materialize/issues/6916: metadata fetch errors in log + - https://api.github.com/repos/MaterializeInc/database-issues/issues/2187: fetching in source doesn't continue after transient failure + - https://api.github.com/repos/MaterializeInc/database-issues/issues/2153: metadata fetch errors in log Related: - - https://github.com/MaterializeInc/materialize/issues/3122: "no complete timestamps" message is not useful - - related: https://github.com/MaterializeInc/materialize/issues/2917 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/1024: "no complete timestamps" message is not useful + - related: https://api.github.com/repos/MaterializeInc/database-issues/issues/978 Future Work, aka. non-goals: - https://github.com/MaterializeInc/materialize/discussions/5340: more structured user errors (error, details, hints) diff --git a/doc/developer/design/20220109_monotonic_topk.md b/doc/developer/design/20220109_monotonic_topk.md index 7d2013930094a..88f8840fcf944 100644 --- a/doc/developer/design/20220109_monotonic_topk.md +++ b/doc/developer/design/20220109_monotonic_topk.md @@ -1,13 +1,13 @@ # Monotonic TopK Rendering: Current State and Potential Improvements Created at: December 16, 2022 5:35 PM; -GitHub epic: https://github.com/MaterializeInc/materialize/issues/16698 +GitHub epic: https://api.github.com/repos/MaterializeInc/database-issues/issues/4838 ## Summary This document aims at capturing a high-level understanding of `TopK` rendering code as well as propose potential improvements in rendering monotonic top-k plans with strategies that that are more efficient when k is small, as is common in practice. ## Goals -Presently, our rendering of `TopK` plans treats three cases: `MonotonicTop1Plan`, `MonotonicTopKPlan`, and `BasicTopKPlan`. Taking a step back, there seem to be at least two aspects to consider in our rendering options: monotonicity and boundedness. Monotonicity is an input property that allows us to come up with more specialized rendering for `TopK` plans, since we do not have to account for retractions and can thus not maintain as much state. Boundedness can refer to the input or to the output sizes. It has been observed in issue [#14444](https://github.com/MaterializeInc/materialize/issues/14444) that input size boundedness can be exploited to simplify `TopK` chains. Output size boundedness is exploited in the rendering of `MonotonicTop1Plan` as well as `MonotonicTopKPlan`. However, there may be opportunities to improve the rendering even further, especially for small k, of `MonotonicTopKPlan`. +Presently, our rendering of `TopK` plans treats three cases: `MonotonicTop1Plan`, `MonotonicTopKPlan`, and `BasicTopKPlan`. Taking a step back, there seem to be at least two aspects to consider in our rendering options: monotonicity and boundedness. Monotonicity is an input property that allows us to come up with more specialized rendering for `TopK` plans, since we do not have to account for retractions and can thus not maintain as much state. Boundedness can refer to the input or to the output sizes. It has been observed in issue [#14444](https://api.github.com/repos/MaterializeInc/database-issues/issues/4127) that input size boundedness can be exploited to simplify `TopK` chains. Output size boundedness is exploited in the rendering of `MonotonicTop1Plan` as well as `MonotonicTopKPlan`. However, there may be opportunities to improve the rendering even further, especially for small k, of `MonotonicTopKPlan`. ## Non-Goals @@ -54,7 +54,7 @@ The behavior above is concretely implemented by a feedback operator, triggered b - Update (1 row) work: $O(\log n)$ - Argument: Similar to the above, but observing that we only need to propagate now 1 row through the reduction. - Note that in this strategy, memory consumption is minimal, which is a great property to have. -- However, the implementation currently suffers from excessive memory allocation: https://github.com/MaterializeInc/materialize/issues/7354. An interesting fix has been suggested in [this PR](https://github.com/MaterializeInc/materialize/pull/16782). +- However, the implementation currently suffers from excessive memory allocation: https://api.github.com/repos/MaterializeInc/database-issues/issues/2296. An interesting fix has been suggested in [this PR](https://github.com/MaterializeInc/materialize/pull/16782). ## Alternatives: Potential Improvements diff --git a/doc/developer/design/20220404_metadata_storage.md b/doc/developer/design/20220404_metadata_storage.md index f37c686058bc6..cda1f43a9e194 100644 --- a/doc/developer/design/20220404_metadata_storage.md +++ b/doc/developer/design/20220404_metadata_storage.md @@ -301,4 +301,4 @@ a per-region metadata database can be satisified in other ways: selling this product to our customers?) [`Stash`]: https://dev.materialize.com/api/rust/mz_stash/index.html -[sqlite-problems]: https://github.com/MaterializeInc/materialize/issues/11577#issuecomment-1087873791 +[sqlite-problems]: https://api.github.com/repos/MaterializeInc/database-issues/issues/3378#issuecomment-1087873791 diff --git a/doc/developer/design/20220411_reclocking_implementation.md b/doc/developer/design/20220411_reclocking_implementation.md index ab08a0734576f..b5ebbde5ae546 100644 --- a/doc/developer/design/20220411_reclocking_implementation.md +++ b/doc/developer/design/20220411_reclocking_implementation.md @@ -269,4 +269,4 @@ of `source_progress`. [`CreateSourceCommand`]: https://dev.materialize.com/api/rust/mz_dataflow_types/client/struct.CreateSourceCommand.html [`GlobalId`]: https://dev.materialize.com/api/rust/mz_expr/enum.GlobalId.html -[#11218]: https://github.com/MaterializeInc/materialize/issues/11218#issuecomment-1087849617 +[#11218]: https://api.github.com/repos/MaterializeInc/database-issues/issues/3271#issuecomment-1087849617 diff --git a/doc/developer/design/20220516_transactional_consistency.md b/doc/developer/design/20220516_transactional_consistency.md index 52d52dce0c838..30326a3264c1e 100644 --- a/doc/developer/design/20220516_transactional_consistency.md +++ b/doc/developer/design/20220516_transactional_consistency.md @@ -5,7 +5,7 @@ A description of Materialize’s consistency guarantees with respect to the operations that occur inside of Materialize. Operations include: `SELECT`, `INSERT`, `UPDATE`, and `DELETE` statements (but not `TAIL`) -This design document is in service of [Epic #11631](https://github.com/MaterializeInc/materialize/issues/11631) +This design document is in service of [Epic #11631](https://api.github.com/repos/MaterializeInc/database-issues/issues/3393) and [Correctness Property #1](https://github.com/MaterializeInc/materialize/blob/main/doc/developer/platform/ux.md#correctness) (copied below). @@ -168,7 +168,7 @@ places a bounds on how much faster the global timestamp can advance compared to NOTE: If the client had multiple writes to a single table known ahead of time, then grouping them in a single multi-statement write transaction would increase throughput. There may be some user education needed for this. -NOTE: This would also fix the problem discussed in [#12198](https://github.com/MaterializeInc/materialize/issues/12198). +NOTE: This would also fix the problem discussed in [#12198](https://api.github.com/repos/MaterializeInc/database-issues/issues/3532). ### Read Capabilities on Global Timestamp diff --git a/doc/developer/design/20220718_envelopes.md b/doc/developer/design/20220718_envelopes.md index 1a1f0189a1af3..162e7a1eb3fef 100644 --- a/doc/developer/design/20220718_envelopes.md +++ b/doc/developer/design/20220718_envelopes.md @@ -116,7 +116,7 @@ confident in, we will re-evaluate this. Note that Debezium data from Microsoft SQL Server should work fine with `DEBEZIUM UPSERT` -Solving the issue of [bounded input reliance](https://github.com/MaterializeInc/materialize/issues/13534) +Solving the issue of [bounded input reliance](https://api.github.com/repos/MaterializeInc/database-issues/issues/3875) is outside the scope of this project; until that issue is solved, we must assume that data is not being compacted away while we're busy reading it, which is somewhat problematic as we have no way to express our frontiers back upstream to diff --git a/doc/developer/design/20220819_bounded_input_reliance.md b/doc/developer/design/20220819_bounded_input_reliance.md index 22f7924e22ae4..491c3b5cbf77e 100644 --- a/doc/developer/design/20220819_bounded_input_reliance.md +++ b/doc/developer/design/20220819_bounded_input_reliance.md @@ -8,7 +8,7 @@ This means that sources should not re-read data that has already been written to This requires that Materialize _bound_ the information we need to restart a source. [source ingestion pipeline design doc]: https://github.com/MaterializeInc/materialize/pull/12109/files -[Epic task]: https://github.com/MaterializeInc/materialize/issues/13534 +[Epic task]: https://api.github.com/repos/MaterializeInc/database-issues/issues/3875 ## Jargon diff --git a/doc/developer/design/20221204_with_mutually_recursive.md b/doc/developer/design/20221204_with_mutually_recursive.md index e968fcaf854ad..45d4e6a74b453 100644 --- a/doc/developer/design/20221204_with_mutually_recursive.md +++ b/doc/developer/design/20221204_with_mutually_recursive.md @@ -7,7 +7,7 @@ // Be sure to capture the customer impact - framing this as a release note may be useful. --> -Progressively introduce support for `WITH MUTUALLY RECURSIVE` common table expressions, as demonstrated in https://github.com/MaterializeInc/materialize/issues/11176. This alternate form of `WITH` requires column names and identifiers for each binding, but allows all bindings to reference each other bindings, and introduces support for recursion and mutual recursion. +Progressively introduce support for `WITH MUTUALLY RECURSIVE` common table expressions, as demonstrated in https://api.github.com/repos/MaterializeInc/database-issues/issues/3264. This alternate form of `WITH` requires column names and identifiers for each binding, but allows all bindings to reference each other bindings, and introduces support for recursion and mutual recursion. ## Goals @@ -63,7 +63,7 @@ These are not likely to be renderable in the foreseeable future, as they cannot // If applicable, be sure to call out any new testing/validation that will be required --> -Much of this is taken from https://github.com/MaterializeInc/materialize/issues/11176. +Much of this is taken from https://api.github.com/repos/MaterializeInc/database-issues/issues/3264. We can extend SQL's `WITH` fragment with a new variant, not strictly more general, which looks like ```sql diff --git a/doc/developer/design/20230209_invalid_accumulations.md b/doc/developer/design/20230209_invalid_accumulations.md index d21bf05a1a06f..c93b98e22fac9 100644 --- a/doc/developer/design/20230209_invalid_accumulations.md +++ b/doc/developer/design/20230209_invalid_accumulations.md @@ -2,7 +2,7 @@ ## Summary -Invalid accumulations may happen when at some points in a dataflow computation, we know that a collection must be a multiset, but we observe negative multiplicities. Upon seeing invalid accumulations in aggregation queries / dataflows, Materialize may log errors or even crash. In some conditions, errors that are logged will not be reported as failures in the corresponding dataflow, making it difficult for users to take corrective actions themselves and/or notify us of erroneous sources. This design addresses making our strategy to handle invalid accumulation errors, when detected, uniform. This design work is pursued as part of epic [#17178](https://github.com/MaterializeInc/materialize/issues/17178). +Invalid accumulations may happen when at some points in a dataflow computation, we know that a collection must be a multiset, but we observe negative multiplicities. Upon seeing invalid accumulations in aggregation queries / dataflows, Materialize may log errors or even crash. In some conditions, errors that are logged will not be reported as failures in the corresponding dataflow, making it difficult for users to take corrective actions themselves and/or notify us of erroneous sources. This design addresses making our strategy to handle invalid accumulation errors, when detected, uniform. This design work is pursued as part of epic [#17178](https://api.github.com/repos/MaterializeInc/database-issues/issues/4967). ### Problem Description Differential Dataflow (DD) operates on `(data, time, diff)` tuples representing changes to `data` at `time` for a given multiplicity `diff`. The multiplicity `diff` is an integer, signaling that changes can lead to either creation or removal of `data` over time. Tuples with negative multiplicities are thus a fact of life and Materialize should be able to process them correctly. @@ -27,7 +27,7 @@ We do not aim to tackle here errors other than invalid accumulation errors in re We identify a few important error categories to be considered: -1. When we implement introspection sources, tables, and regular sources, we ensure that only multisets are given as input to the COMPUTE layer, despite their representation being in terms of changes that may include retractions. However, sometimes this assumption can be violated when subtle bugs occur (see, e.g., issue [#15930](https://github.com/MaterializeInc/materialize/issues/15930)). Additionally, users may [directly introduce invalid retractions in source data](https://materializeinc.slack.com/archives/CM7ATT65S/p1675428135441319?thread_ts=1675355385.487949&cid=CM7ATT65S), e.g., exploiting `ENVELOPE DEBEZIUM`. Therefore, a number of sanity checks are performed during reductions rendered by the COMPUTE layer to ensure that we are operating on multisets. For example, when we compute a min/max aggregation, the closure given to specialize a Differential Dataflow reduction operator cannot observe negative multiplicities if we are operating on a multiset. So checks and error reporting are introduced, e.g., in [intermediate stages](https://github.com/MaterializeInc/materialize/blob/9933639ea5393d1aaa40c1079d07ffc1cc622516/src/compute/src/render/reduce.rs#L634-L642) and [final reduction](https://github.com/MaterializeInc/materialize/blob/9933639ea5393d1aaa40c1079d07ffc1cc622516/src/compute/src/render/reduce.rs#L590-L596) of hierarchical aggregates. The strategy for error reporting is to employ a soft assertion: during development, an assertion failure will trigger a crash; in production, an error will be logged and captured in Sentry. +1. When we implement introspection sources, tables, and regular sources, we ensure that only multisets are given as input to the COMPUTE layer, despite their representation being in terms of changes that may include retractions. However, sometimes this assumption can be violated when subtle bugs occur (see, e.g., issue [#15930](https://api.github.com/repos/MaterializeInc/database-issues/issues/4587)). Additionally, users may [directly introduce invalid retractions in source data](https://materializeinc.slack.com/archives/CM7ATT65S/p1675428135441319?thread_ts=1675355385.487949&cid=CM7ATT65S), e.g., exploiting `ENVELOPE DEBEZIUM`. Therefore, a number of sanity checks are performed during reductions rendered by the COMPUTE layer to ensure that we are operating on multisets. For example, when we compute a min/max aggregation, the closure given to specialize a Differential Dataflow reduction operator cannot observe negative multiplicities if we are operating on a multiset. So checks and error reporting are introduced, e.g., in [intermediate stages](https://github.com/MaterializeInc/materialize/blob/9933639ea5393d1aaa40c1079d07ffc1cc622516/src/compute/src/render/reduce.rs#L634-L642) and [final reduction](https://github.com/MaterializeInc/materialize/blob/9933639ea5393d1aaa40c1079d07ffc1cc622516/src/compute/src/render/reduce.rs#L590-L596) of hierarchical aggregates. The strategy for error reporting is to employ a soft assertion: during development, an assertion failure will trigger a crash; in production, an error will be logged and captured in Sentry. 2. Another reason for error due to negative multiplicities is that a negative accumulated result cannot be coerced to an [unsigned type](https://github.com/MaterializeInc/materialize/blob/9933639ea5393d1aaa40c1079d07ffc1cc622516/src/compute/src/render/reduce.rs#L1344-L1346). This error is now treated as unrecoverable, leading to a `panic!` instead of the above strategy of employing a soft assertion. 3. Following a similar error reporting strategy as in category 1 above, we have a soft assertion regarding net-zero records with [non-zero accumulation in accumulable reductions](https://github.com/MaterializeInc/materialize/blob/9933639ea5393d1aaa40c1079d07ffc1cc622516/src/compute/src/render/reduce.rs#L1283-L1290). This category of error can emerge if we determine that an aggregate row has been removed, due to the corresponding tuple multiplicity accumulating to zero, but there is an inconsistency with the accumulation state, namely the aggregation function on the raw changes ends up with a non-zero result. 4. As observed in an incident, shutdown of a `persist_source` operator could trigger errors due to the sanity checks above in the COMPUTE layer. This behavior occurs because `persist_source` would not emit the entirety of a batch nor guarantee that a partially emitted batch would consolidate to a multiset. We ignore this problem in the remainder of this document, as a solution was introduced by PR [#17147](https://github.com/MaterializeInc/materialize/pull/17147). @@ -134,4 +134,4 @@ An alternative approach would be to include checks for the multiset property at ## Final Remarks -This design does not address improving reduction performance by "atomization", as proposed in issue [#8086](https://github.com/MaterializeInc/materialize/issues/8066). We are leaving this additional scope to be tackled as part of a separate work stream. +This design does not address improving reduction performance by "atomization", as proposed in issue [#8086](https://api.github.com/repos/MaterializeInc/database-issues/issues/2470). We are leaving this additional scope to be tackled as part of a separate work stream. diff --git a/doc/developer/design/20230210_storage_persist_sink_redux.md b/doc/developer/design/20230210_storage_persist_sink_redux.md index 8e7a4d9cbe8ba..ea7543b7f9f38 100644 --- a/doc/developer/design/20230210_storage_persist_sink_redux.md +++ b/doc/developer/design/20230210_storage_persist_sink_redux.md @@ -1,5 +1,5 @@ - Feature name: storage `persist_sink` upgrade -- Associated: +- Associated: # Summary [summary]: #summary diff --git a/doc/developer/design/20230223_stabilize_with_mutually_recursive.md b/doc/developer/design/20230223_stabilize_with_mutually_recursive.md index 2fc654806fcc9..e09cd709f8b3c 100644 --- a/doc/developer/design/20230223_stabilize_with_mutually_recursive.md +++ b/doc/developer/design/20230223_stabilize_with_mutually_recursive.md @@ -1,7 +1,7 @@ - Feature name: Stabilize `WITH MUTUALLY RECURSIVE` - Associated: - MaterializeInc/materialize#11176 (first iteration), - MaterializeInc/materialize#17012 (current epic). + MaterializeInc/database-issues#3264 (first iteration), + MaterializeInc/database-issues#6172 (current epic). # Summary [Summary]: #summary @@ -17,9 +17,9 @@ Adding first-class support for recursive queries will: 1. Exercise one of the key strengths of the underlying runtime (support for incremental maintenance of iterative dataflows). 1. Enable new use cases across different domains, most likely based on various forms of graph analysis (for example for social networks, fraud detection, software security). -1. Enable tractable encodings of high-level concepts such as session windows in terms of SQL (see MaterializeInc/materialize#8698). +1. Enable tractable encodings of high-level concepts such as session windows in terms of SQL (see MaterializeInc/database-issues#2664). -We should fill in the implementation gaps that were intentionally left as TODOs during MaterializeInc/materialize#11176 and bring the feature into a shape where it can be gradually rolled out behind a feature toggle and ultimately stabilized. +We should fill in the implementation gaps that were intentionally left as TODOs during MaterializeInc/database-issues#3264 and bring the feature into a shape where it can be gradually rolled out behind a feature toggle and ultimately stabilized. # Explanation [Explanation]: #explanation @@ -47,7 +47,7 @@ The original design doc laid out a plan for adding support in each of the follow 6. MIR generalization (✓ in MaterializeInc/materialize#16561) 7. MIR optimization corrections (focus of this document) 8. LIR generalization (✓ in MaterializeInc/materialize#16656, MaterializeInc/materialize#17705) -9. Rendering (✓ in MaterializeInc/materialize#16787, TODO: MaterializeInc/materialize#16800) +9. Rendering (✓ in MaterializeInc/materialize#16787, TODO: MaterializeInc/database-issues#4869) The outstanding tracks of work can be summarized as follows: @@ -93,46 +93,46 @@ The proposed implementation plan is summarized after the table. transformation | estimate | solution | tracked in ----------------------------|----------|----------|------------------------------------------------- -`canonicalize_mfp` | ✓ | trivial | MaterializeInc/materialize#18123 -`column_knowledge` | ✓ | advanced | MaterializeInc/materialize#18161 -`demand` | ✓ | basic | MaterializeInc/materialize#18162 -`filter_fusion` | ✓ | trivial | MaterializeInc/materialize#18123 (depends on type inference) +`canonicalize_mfp` | ✓ | trivial | MaterializeInc/database-issues#5317 +`column_knowledge` | ✓ | advanced | MaterializeInc/database-issues#5330 +`demand` | ✓ | basic | MaterializeInc/database-issues#5331 +`filter_fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 (depends on type inference) `fixpoint` | ✓ | trivial | MaterializeInc/materialize#16561 -`flatmap_to_map` | ✓ | trivial | MaterializeInc/materialize#18123 -`fold_constants` | ✓ | basic | MaterializeInc/materialize#18163 -`fuse_and_collapse` | ✓ | trivial | MaterializeInc/materialize#18164 -`fusion` | ✓ | trivial | MaterializeInc/materialize#18123 -`join_fusion` | ✓ | trivial | MaterializeInc/materialize#18123 +`flatmap_to_map` | ✓ | trivial | MaterializeInc/database-issues#5317 +`fold_constants` | ✓ | basic | MaterializeInc/database-issues#5332 +`fuse_and_collapse` | ✓ | trivial | MaterializeInc/database-issues#5333 +`fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 +`join_fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 `join_implementation` | ✓ | advanced | MaterializeInc/materialize#16561 -`literal_constraints` | ✓ | trivial | MaterializeInc/materialize#18123 -`literal_lifting` | ✓ | basic | MaterializeInc/materialize#18165 -`map_fusion` | ✓ | trivial | MaterializeInc/materialize#18123 -`monotonic_flag` | ✓ | advanced | MaterializeInc/materialize#18472 -`negate_fusion` | ✓ | trivial | MaterializeInc/materialize#18123 -`non_null_requirements` | ✓ | basic | MaterializeInc/materialize#18166 -`non_nullable` | ✓ | trivial | MaterializeInc/materialize#18123 (somewhat restricted) -`normalize_ops` | ✓ | trivial | MaterializeInc/materialize#18123 +`literal_constraints` | ✓ | trivial | MaterializeInc/database-issues#5317 +`literal_lifting` | ✓ | basic | MaterializeInc/database-issues#5334 +`map_fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 +`monotonic_flag` | ✓ | advanced | MaterializeInc/database-issues#5453 +`negate_fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 +`non_null_requirements` | ✓ | basic | MaterializeInc/database-issues#5335 +`non_nullable` | ✓ | trivial | MaterializeInc/database-issues#5317 (somewhat restricted) +`normalize_ops` | ✓ | trivial | MaterializeInc/database-issues#5317 `normalize_lets` | ✓ | advanced | MaterializeInc/materialize#16665 -`predicate_pushdown` | ✓ | basic | MaterializeInc/materialize#18167 -`project_fusion` | ✓ | trivial | MaterializeInc/materialize#18123 -`projection_extraction` | ✓ | trivial | MaterializeInc/materialize#18123 -`projection_lifting` | ✓ | basic | MaterializeInc/materialize#18168 -`projection_pushdown` | ✓ | basic | MaterializeInc/materialize#18169 (depends on MaterializeInc/materialize#18553) -`reduce_elision` | ✓ | basic | MaterializeInc/materialize#18170 (depends on MaterializeInc/materialize#18553) -`reduce_fusion` | ✓ | trivial | MaterializeInc/materialize#18123 -`reduction_pushdown` | ✓ | basic | MaterializeInc/materialize#18171 (depends on MaterializeInc/materialize#18553) -`redundant_join` | ✓ | basic | MaterializeInc/materialize#18172 -`relation_cse` | ✓ | basic | MaterializeInc/materialize#18173 -`semijoin_idempotence` | ✓ | basic | MaterializeInc/materialize#18174 (depends on MaterializeInc/materialize#18553) -`threshold_elision` | ✓ | basic | MaterializeInc/materialize#18175 -`topk_elision` | ✓ | trivial | MaterializeInc/materialize#18123 -`topk_fusion` | ✓ | trivial | MaterializeInc/materialize#18123 -`union` | ✓ | trivial | MaterializeInc/materialize#18123 -`union_branch_cancellation` | ✓ | trivial | MaterializeInc/materialize#18176 -`union_negate` | ✓ | trivial | MaterializeInc/materialize#18123 +`predicate_pushdown` | ✓ | basic | MaterializeInc/database-issues#5336 +`project_fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 +`projection_extraction` | ✓ | trivial | MaterializeInc/database-issues#5317 +`projection_lifting` | ✓ | basic | MaterializeInc/database-issues#5337 +`projection_pushdown` | ✓ | basic | MaterializeInc/materialize#18169 (depends on MaterializeInc/database-issues#5487) +`reduce_elision` | ✓ | basic | MaterializeInc/materialize#18170 (depends on MaterializeInc/database-issues#5487) +`reduce_fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 +`reduction_pushdown` | ✓ | basic | MaterializeInc/materialize#18171 (depends on MaterializeInc/database-issues#5487) +`redundant_join` | ✓ | basic | MaterializeInc/database-issues#5341 +`relation_cse` | ✓ | basic | MaterializeInc/database-issues#5342 +`semijoin_idempotence` | ✓ | basic | MaterializeInc/database-issues#5343 (depends on MaterializeInc/database-issues#5487) +`threshold_elision` | ✓ | basic | MaterializeInc/database-issues#5344 +`topk_elision` | ✓ | trivial | MaterializeInc/database-issues#5317 +`topk_fusion` | ✓ | trivial | MaterializeInc/database-issues#5317 +`union` | ✓ | trivial | MaterializeInc/database-issues#5317 +`union_branch_cancellation` | ✓ | trivial | MaterializeInc/database-issues#5345 +`union_negate` | ✓ | trivial | MaterializeInc/database-issues#5317 We have 36 `Transform` implementations, of which 3 are currently marked as `recursion_safe`. -All but 16 can be trivially marked as recursion safe (done in MaterializeInc/materialize#18123) because they represent local transformations that don't depend on transformation context that depends on the `Let` bindings that are currently in scope. +All but 16 can be trivially marked as recursion safe (done in MaterializeInc/database-issues#5317) because they represent local transformations that don't depend on transformation context that depends on the `Let` bindings that are currently in scope. From the remaining 16, based on an initial analysis it seems that: - 4 are relatively straight-forward to fix (size estimate `M?`), @@ -145,13 +145,13 @@ For most non-trivial transforms, we have multiple solutions at our disposal: 2. An _advanced_ solution which uses abstract interpretation based on lattice theory to propagate information through `LetRec` nodes. Transforms using this solution are marked with _solution = advanced_ in the table above. For the _basic_ transforms the _advanced_ solution is sketched in the linked issue in case we want to improve them as future work. -3. An _advanced_ solution that we will get without changes to the actual `Transform` code if we implement MaterializeInc/materialize#18174. - Those are marked with _solution = basic_ and the corresponding issue as depending on MaterializeInc/materialize#18553. +3. An _advanced_ solution that we will get without changes to the actual `Transform` code if we implement MaterializeInc/database-issues#5343. + Those are marked with _solution = basic_ and the corresponding issue as depending on MaterializeInc/database-issues#5487. ## Generalization of LIR rendering This should be mostly handled by MaterializeInc/materialize#17705. -There is also an additional feature request for an optional max recursion limit in MaterializeInc/materialize#16800 which will affect how plans are rendered. +There is also an additional feature request for an optional max recursion limit in MaterializeInc/database-issues#4869 which will affect how plans are rendered. We might have to add more tests for that (see [Testing and observability](#testing-and-observability)). # Rollout @@ -167,7 +167,7 @@ The following aspects need special attention: 1. Queries producing wrong results (discussed in [Testing and observability](testing-and-observability)). 2. Queries that do not terminate. This is tricky because some queries might be divergent because of a bad query definition (a user error) instead of an optimization or interpretation bug (a system error). - A related issue to track this is MaterializeInc/materialize#16800. The plan is to have maximum iteration limit as a safeguard. Edit: We won't have a default limit, because we now have proper dataflow cancellation between iterations. However, the user can set `ERROR AT RECURSION LIMIT 1000`, if she wants an additional guardrail. + A related issue to track this is MaterializeInc/database-issues#4869. The plan is to have maximum iteration limit as a safeguard. Edit: We won't have a default limit, because we now have proper dataflow cancellation between iterations. However, the user can set `ERROR AT RECURSION LIMIT 1000`, if she wants an additional guardrail. To validate (1), I suggest to: @@ -220,7 +220,7 @@ We will perform a bunch of end-to-end experiments (available in [the `letrec-ben 1. **Synthetic tests.** (punted as follow-up work) The best synthetic use case that we have identified so far seems to be the LDBC social network benchmark[^ldbc]. -With the scope of the dedicated epic (MaterializeInc/materialize#17591), we will select a subset of the work in order to bootstrap a testing environment that consists of +With the scope of the dedicated epic (MaterializeInc/database-issues#5110), we will select a subset of the work in order to bootstrap a testing environment that consists of (a) LDBC data + updates, and (b) several of the recursive queries defined by the benchmark. We can use the choke-point characterization of each query to figure out the most representative subset. @@ -303,7 +303,7 @@ I think that we can re-evaluate these points as part of an "end of epic" retrosp See discussion of use-case driven tests in [By test scenario]. - Can we measure / observe the amount of work / data diff that a specific change to the input introduces? See discussion of use-case driven tests in [By test scenario]. - Tracked in MaterializeInc/materialize#18022. + Tracked in MaterializeInc/database-issues#5271. # Future work [Future work]: #future-work @@ -315,7 +315,7 @@ This section can also serve as a place to dump ideas that are related but not pa If you can't think of any, please note this down. --> -We are still lacking operational observability (tracked in MaterializeInc/materialize#18022). +We are still lacking operational observability (tracked in MaterializeInc/database-issues#5271). At the very least, we can export a Prometheus metric that tracks the number of indexed or materialized views that have recursive CTEs. @@ -323,7 +323,7 @@ Once we have anonymized query logging, we can get some deper insights which woul --- -Within the scope of MaterializeInc/materialize#17012 we only provided the _basic case_ for most non-trivial transforms. +Within the scope of MaterializeInc/database-issues#6172 we only provided the _basic case_ for most non-trivial transforms. Issues marked in [the above table](#mir-transformations) with _solution = basic_ represent opportunities for improvement. --- @@ -332,16 +332,16 @@ UI/UX improvements: - In "linear chains" mode the `EXPLAIN` output of plans that have recursive queries does not work. We will need to revisit this if we ever decide to make this the default or we have people that use it on a daily basis. - Tracked in MaterializeInc/materialize#19012. + Tracked in MaterializeInc/database-issues#5631. - Similarly, the graph visualizer for dataflows that have iterative scopes might need to be fixed. - As we gain insights how people use the feature, we might want to follow-up with more focused "guidance docs" that go in depth of some common considerations and pitfalls. Currently, this is partially covered by [the final two sections of the reference docs](https://materialize.com/docs/sql/recursive-ctes/#examples), but this might not be sufficient to cover everything in the long run. - Tracked in MaterializeInc/materialize#19334. + Tracked in MaterializeInc/database-issues#5734. --- Due to time constraints benchmarking of WMR based on LDBC has been punted in favor of a more limited benchmark available in [the `letrec-bench` GitHub repository](https://github.com/MaterializeInc/letrec-bench). -The tracking epic for this is MaterializeInc/materialize#17591. +The tracking epic for this is MaterializeInc/database-issues#5110. --- @@ -368,7 +368,7 @@ select * from reach; --- -Session windows can be defined in an easier way (see MaterializeInc/materialize#8698). +Session windows can be defined in an easier way (see MaterializeInc/database-issues#2664). [@sploiselle](https://github.com/sploiselle) was kind enough to add a PR for a prototype of that function in MaterializeInc/materialize#18330. --- diff --git a/doc/developer/design/20230306_persist_mfp_pushdown.md b/doc/developer/design/20230306_persist_mfp_pushdown.md index 22b1136031023..bc0adf5d9f52f 100644 --- a/doc/developer/design/20230306_persist_mfp_pushdown.md +++ b/doc/developer/design/20230306_persist_mfp_pushdown.md @@ -1,5 +1,5 @@ - Feature name: Persist Filter Pushdown -- Associated: https://github.com/MaterializeInc/materialize/issues/12684 +- Associated: https://api.github.com/repos/MaterializeInc/database-issues/issues/3657 # Summary [summary]: #summary @@ -266,7 +266,7 @@ with minimal impact on read performance. ### Schema evaluation Understanding schemas at the persistent data level gives us a building block [to explore schema evolution at -the Persist level](https://github.com/MaterializeInc/materialize/issues/16625). +the Persist level](https://api.github.com/repos/MaterializeInc/database-issues/issues/4818). ### Internal improvements diff --git a/doc/developer/design/20230315_backup_restore.md b/doc/developer/design/20230315_backup_restore.md index 1e1e5e87cae2b..cb75235b72f4b 100644 --- a/doc/developer/design/20230315_backup_restore.md +++ b/doc/developer/design/20230315_backup_restore.md @@ -1,6 +1,6 @@ # Backup and restore - Feature name: backup/restore -- Associated: [#17605](https://github.com/MaterializeInc/materialize/issues/17605) +- Associated: [#17605](https://api.github.com/repos/MaterializeInc/database-issues/issues/5119) # The Problem @@ -19,7 +19,7 @@ Reasons one might want backup/restore, and whether they’re in scope for this d | Snapshot | A shard causes an unusual performance problem for some compute operator, and we’d like to inspect a previous state to investigate. | Nice to have. | | Operator error | An operator typos an aws CLI command, accidentally deleting blobs that are still referenced. | Yes. (Impossible to prevent an admin from deleting data entirely, but it’s good if we can make ordinary operations less risky.) | -Motivated by the above and [some other feedback](https://github.com/MaterializeInc/materialize/issues/17605), this design doc focuses on infrastructure-level backups (without no product surface area) that optimize for disaster recovery. For other possible approaches or extensions to backup/restore, see the [section on future work](#future-work). +Motivated by the above and [some other feedback](https://api.github.com/repos/MaterializeInc/database-issues/issues/5119), this design doc focuses on infrastructure-level backups (without no product surface area) that optimize for disaster recovery. For other possible approaches or extensions to backup/restore, see the [section on future work](#future-work). This means backups should be: - High frequency: at most one hour between restore points. (Point-in-time would be cool, but is not required.) @@ -194,7 +194,7 @@ It seems plausible that one could reconstruct such a state by inspecting the ful One can imagine wanting a user-facing syntax for backup and restore: for example, to make an ad-hoc snapshot of a table before making changes, or to periodically back up some critical dataset. -We expect this need to be served by future sources and sinks, [like a potential S3 integration](https://github.com/MaterializeInc/materialize/issues/17605#issuecomment-1432420387), instead of relying on any infrastructure-level backup. +We expect this need to be served by future sources and sinks, [like a potential S3 integration](https://api.github.com/repos/MaterializeInc/database-issues/issues/5119#issuecomment-1432420387), instead of relying on any infrastructure-level backup. # Appendix A: S3 costs diff --git a/doc/developer/design/20230322_metrics_since_granularity.md b/doc/developer/design/20230322_metrics_since_granularity.md index dce6184c86fe3..edc25a7b822e8 100644 --- a/doc/developer/design/20230322_metrics_since_granularity.md +++ b/doc/developer/design/20230322_metrics_since_granularity.md @@ -1,5 +1,5 @@ - Feature name: Retained metrics "since" granularity -- Associated: https://github.com/MaterializeInc/materialize/issues/17741 +- Associated: https://api.github.com/repos/MaterializeInc/database-issues/issues/5166 # Summary [summary]: #summary @@ -41,4 +41,4 @@ Slightly increases system complexity # Future work [future-work]: #future-work -We still need to complete the other tasks in https://github.com/MaterializeInc/materialize/issues/18347 . +We still need to complete the other tasks in https://api.github.com/repos/MaterializeInc/database-issues/issues/5405 . diff --git a/doc/developer/design/20230330_recursion_limit.md b/doc/developer/design/20230330_recursion_limit.md index 78f2c9e9b22c6..16e6f3bff9e0e 100644 --- a/doc/developer/design/20230330_recursion_limit.md +++ b/doc/developer/design/20230330_recursion_limit.md @@ -1,5 +1,5 @@ - Feature name: (Recursion limit in WITH MUTUALLY RECURSIVE) -- Associated: (Issues: [Stabilizing WMR (#17012)](https://github.com/MaterializeInc/materialize/issues/17012), [#16800](https://github.com/MaterializeInc/materialize/issues/16800), [#18362](https://github.com/MaterializeInc/materialize/issues/18362), [#2392](https://github.com/MaterializeInc/materialize/issues/2392)) +- Associated: (Issues: [Stabilizing WMR (#17012)](https://api.github.com/repos/MaterializeInc/database-issues/issues/6172), [#16800](https://api.github.com/repos/MaterializeInc/database-issues/issues/4869), [#18362](https://api.github.com/repos/MaterializeInc/database-issues/issues/5409), [#2392](https://api.github.com/repos/MaterializeInc/database-issues/issues/835)) # Summary [summary]: #summary @@ -14,8 +14,8 @@ The user can specify a limit using a SQL syntax extension at the level of WMR bl [motivation]: #motivation I can imagine 3 use cases, of which the 2. seems urgent to me: -1. _Stopping divergent queries ([#16800](https://github.com/MaterializeInc/materialize/issues/16800)):_ It is very easy to accidentally write a WMR query that runs forever (actually, until OOM in most cases, but that would often take a very long time). We used to have the additional problem that Ctrl+C (or dropping a materialized view or index) didn't cancel a dataflow ([#2392](https://github.com/MaterializeInc/materialize/issues/2392)), which made for quite an unpleasant user experience, as the user had to manually DROP the entire replica. However, proper dataflow cancellation for WMR queries has been [recently implemented](https://github.com/MaterializeInc/materialize/pull/18718), and therefore this use case of recursion limits is not so important anymore. -2. _Debugging state between iterations ([#18362](https://github.com/MaterializeInc/materialize/issues/18362)):_ WMR queries are not so obvious to write, so users often need to debug their queries during development. In such cases, it can be enlightening to inspect the intermediate states between iterations. If we had an option to limit the number of iterations, the user could just run the query repeatedly with successively larger limits. Multiple people expressed their wish for this feature, e.g., [here](https://materializeinc.slack.com/archives/C02CB7L4TCG/p1678266977745819?thread_ts=1678266846.169599&cid=C02CB7L4TCG). +1. _Stopping divergent queries ([#16800](https://api.github.com/repos/MaterializeInc/database-issues/issues/4869)):_ It is very easy to accidentally write a WMR query that runs forever (actually, until OOM in most cases, but that would often take a very long time). We used to have the additional problem that Ctrl+C (or dropping a materialized view or index) didn't cancel a dataflow ([#2392](https://api.github.com/repos/MaterializeInc/database-issues/issues/835)), which made for quite an unpleasant user experience, as the user had to manually DROP the entire replica. However, proper dataflow cancellation for WMR queries has been [recently implemented](https://github.com/MaterializeInc/materialize/pull/18718), and therefore this use case of recursion limits is not so important anymore. +2. _Debugging state between iterations ([#18362](https://api.github.com/repos/MaterializeInc/database-issues/issues/5409)):_ WMR queries are not so obvious to write, so users often need to debug their queries during development. In such cases, it can be enlightening to inspect the intermediate states between iterations. If we had an option to limit the number of iterations, the user could just run the query repeatedly with successively larger limits. Multiple people expressed their wish for this feature, e.g., [here](https://materializeinc.slack.com/archives/C02CB7L4TCG/p1678266977745819?thread_ts=1678266846.169599&cid=C02CB7L4TCG). 3. _Algorithms requiring a fixed number of iterations:_ Some algorithms are unintuitive to express as a loop running until a fixpoint (or reaching a fixpoint can't be ensured), and instead need a loop that runs a specific number of times. Note that in the above two use cases, we don't expect to run into the limit in production under normal operation, but in this use case a limit will be part of the logic of production queries. # Explanation @@ -135,7 +135,7 @@ One trick would be to implement an HIR rewrite to an extra binding that counts i ### Workaround: Manually setting a limit using existing SQL building blocks -[Users can manually write SQL that counts the number of iterations, and converts reaching the iteration count limit into a fixpoint.](https://github.com/MaterializeInc/materialize/issues/18362#issuecomment-1481605865) +[Users can manually write SQL that counts the number of iterations, and converts reaching the iteration count limit into a fixpoint.](https://api.github.com/repos/MaterializeInc/database-issues/issues/5409#issuecomment-1481605865) However, it would be cumbersome for users to do this every time they want to debug a query. Also note that, currently, this would have the problem for use case 3. that it would involve a cross join with a singleton collection, which would need a broadcast join to avoid grabbing everything onto one worker. @@ -161,4 +161,4 @@ The bookkeeping for different limits for different WMR blocks inside the same qu What should be the keyword? This also depends on what will the final keywords be for `WITH MUTUALLY RECURSIVE` itself. (The latest suggestion was `WITH REPEATEDLY`.) -[Do we want also a hard limit later?](https://github.com/MaterializeInc/materialize/issues/18832) If yes, what should be the default value? +[Do we want also a hard limit later?](https://api.github.com/repos/MaterializeInc/database-issues/issues/5564) If yes, what should be the default value? diff --git a/doc/developer/design/20230411_envelope_upsert_order_by_timestamp.md b/doc/developer/design/20230411_envelope_upsert_order_by_timestamp.md index 39e56a9a9b96b..ac002e2c0b51f 100644 --- a/doc/developer/design/20230411_envelope_upsert_order_by_timestamp.md +++ b/doc/developer/design/20230411_envelope_upsert_order_by_timestamp.md @@ -1,5 +1,5 @@ - Feature name: Envelope upsert with order by timestamp (limited scope) -- Associated: https://github.com/MaterializeInc/materialize/issues/16512 +- Associated: https://api.github.com/repos/MaterializeInc/database-issues/issues/4782 # Summary diff --git a/doc/developer/design/20230411_faster_dataflow_shutdown.md b/doc/developer/design/20230411_faster_dataflow_shutdown.md index 2c865f0baf2ea..75e541d30d15a 100644 --- a/doc/developer/design/20230411_faster_dataflow_shutdown.md +++ b/doc/developer/design/20230411_faster_dataflow_shutdown.md @@ -1,8 +1,8 @@ - Feature name: Faster Dataflow Shutdown - Associated: - * - * - * + * + * + * # Summary @@ -85,9 +85,9 @@ For these reasons, we are interested in completing dataflow shutdowns as quickly [`Worker::drop_dataflow`]: https://dev.materialize.com/api/rust/timely/worker/struct.Worker.html#method.drop_dataflow [timely-dataflow/#519]: https://github.com/TimelyDataflow/timely-dataflow/pull/519 [WMR]: https://github.com/MaterializeInc/materialize/blob/main/doc/developer/design/20221204_with_mutually_recursive.md -[#7577]: https://github.com/MaterializeInc/materialize/issues/7577 -[#16800]: https://github.com/MaterializeInc/materialize/issues/16800 -[#16860]: https://github.com/MaterializeInc/materialize/issues/16860 +[#7577]: https://api.github.com/repos/MaterializeInc/database-issues/issues/2350 +[#16800]: https://api.github.com/repos/MaterializeInc/database-issues/issues/4869 +[#16860]: https://api.github.com/repos/MaterializeInc/database-issues/issues/4876 # Explanation @@ -150,7 +150,7 @@ As part of [#17178], COMPUTE operators have been made much more robust to unexpe They now consistently report errors gracefully instead of crashing the process. Because of this the workaround in the persist source is not necessary anymore and we are free to revert it to the previous more efficient behavior. -[#17178]: https://github.com/MaterializeInc/materialize/issues/17178 +[#17178]: https://api.github.com/repos/MaterializeInc/database-issues/issues/4967 ## Tokenizing Join Operators @@ -169,7 +169,7 @@ It is likely that the compiler is able to lift the token check out of the loop t There might be other places in the join implementations where adding a token check improves shutdown performance. In the interest of keeping code complexity in check, we suggest to not speculatively add more token checks to the join implementations, unless we have evidence (e.g., example queries) that they significantly improve dataflow shutdown performance. The same applies to adding token checks to other operators than join. -[#7577]: https://github.com/MaterializeInc/materialize/issues/7577 +[#7577]: https://api.github.com/repos/MaterializeInc/database-issues/issues/2350 ## Handling Incomplete Time Slices @@ -261,4 +261,4 @@ These instances can only be solved by adding or improving fueling for these oper Finally, we should continue the work of stabilizing `drop_dataflow`. As pointed out above, it is a simpler and more effective approach than what this design proposes, and therefore a strictly better solution provided we can gain confidence in it. -[#8853]: https://github.com/MaterializeInc/materialize/issues/8853 +[#8853]: https://api.github.com/repos/MaterializeInc/database-issues/issues/2700 diff --git a/doc/developer/design/20230418_persist_state_pubsub.md b/doc/developer/design/20230418_persist_state_pubsub.md index 12d985ff6b0f5..ac90196f8b319 100644 --- a/doc/developer/design/20230418_persist_state_pubsub.md +++ b/doc/developer/design/20230418_persist_state_pubsub.md @@ -1,5 +1,5 @@ - Feature name: Persist State PubSub -- Associated: https://github.com/MaterializeInc/materialize/issues/18661 +- Associated: https://api.github.com/repos/MaterializeInc/database-issues/issues/5515 # Summary [summary]: #summary @@ -11,7 +11,7 @@ Complementing Persist's polling-based discovery of new state with pub-sub update tl;dr * Reduces CockroachDB read traffic to near zero, reducing current CPU usage by 20-30% (relative to total usage). - This will allow us to [scale down our clusters further](https://github.com/MaterializeInc/materialize/issues/18665). + This will allow us to [scale down our clusters further](https://api.github.com/repos/MaterializeInc/database-issues/issues/5518). * Reduces the median latency of observing new data written to a shard from ~300ms to single digit millis --- diff --git a/doc/developer/design/20230418_subscribe_output.md b/doc/developer/design/20230418_subscribe_output.md index c5870a4b6f0d9..9c3c909a4e44d 100644 --- a/doc/developer/design/20230418_subscribe_output.md +++ b/doc/developer/design/20230418_subscribe_output.md @@ -1,5 +1,5 @@ - Feature name: Subscribe outputs -- Associated: [#10593](https://github.com/MaterializeInc/materialize/issues/10593) +- Associated: [#10593](https://api.github.com/repos/MaterializeInc/database-issues/issues/3136) # Summary [summary]: #summary diff --git a/doc/developer/design/20230419_stash_migrations.md b/doc/developer/design/20230419_stash_migrations.md index fa19a4246dc1b..a9f541c3f9930 100644 --- a/doc/developer/design/20230419_stash_migrations.md +++ b/doc/developer/design/20230419_stash_migrations.md @@ -1,5 +1,5 @@ - Feature name: Improved Stash Migrations -- Associated: [#17466](https://github.com/MaterializeInc/materialize/issues/17466) +- Associated: [#17466](https://api.github.com/repos/MaterializeInc/database-issues/issues/5072) # Summary [summary]: #summary @@ -31,7 +31,7 @@ if our builds and tests are passing, then the migration won't break anything in There have been several issues caused by Stash migrations: -* [materialize#17824](https://github.com/MaterializeInc/materialize/issues/17824) +* [database-issues#5197](https://api.github.com/repos/MaterializeInc/database-issues/issues/5197) * General difficulty in writing Stash migrations, see [materialize#18719](https://github.com/MaterializeInc/materialize/pull/18719) or [materialize#17727](https://github.com/MaterializeInc/materialize/pull/17727) for examples. * [materialize#18578](https://github.com/MaterializeInc/materialize/pull/18578) * Fixed previous migrations that failed to remove old types from the Stash. @@ -339,12 +339,12 @@ We can test this change in the following ways: `environmentd` after this change has been applied, assert we're able to migrate successfully. 2. As mentioned above, use the `stash-debug` tool's `upgrade-check` command to make sure customer environments would be able to successfully upgrade. -3. Track the number of rows, and the size of rows, that exist in the Stash [materialize#18871](https://github.com/MaterializeInc/materialize/issues/18871). +3. Track the number of rows, and the size of rows, that exist in the Stash [database-issues#5575](https://api.github.com/repos/MaterializeInc/database-issues/issues/5575). If we don't see any change in these numbers, and we don't see any errors otherwise (e.g. in Sentry), that's a strong indication that the migration is working as expected. 4. Add more cases to the existing [upgrade tests](https://github.com/MaterializeInc/materialize/blob/main/test/upgrade/mzcompose.py) to cover any features we might be missing. -5. (Future) Re-introduce the builtin-migration tests [materialize#17236](https://github.com/MaterializeInc/materialize/issues/17236) +5. (Future) Re-introduce the builtin-migration tests [database-issues#4994](https://api.github.com/repos/MaterializeInc/database-issues/issues/4994) that start Materialize at a previous version of the repo, and then restarts it with the current commit. 6. (Future) In a Rust test, use the snapshotted Stash types to generate "old" version of the Stash @@ -423,7 +423,7 @@ engineering cost. [unresolved-questions]: #unresolved-questions * How important do we believe human readability of the Stash is? -* ~~Does the Stash need to be JSON for billing purposes? This [issue](https://github.com/MaterializeInc/materialize/issues/14264) indicates it might, but this [comment](https://github.com/MaterializeInc/materialize/issues/14264#issuecomment-1218250985) indicates it does not.~~ +* ~~Does the Stash need to be JSON for billing purposes? This [issue](https://api.github.com/repos/MaterializeInc/database-issues/issues/4076) indicates it might, but this [comment](https://api.github.com/repos/MaterializeInc/database-issues/issues/4076#issuecomment-1218250985) indicates it does not.~~ * **Answer:** No. `environmentd` and `stash-debug` are the only two things that directly depend on the Stash. If application code wants to access the Stash, it needs to do it through `environmentd`. diff --git a/doc/developer/design/20230421_stabilize_monotonic_select.md b/doc/developer/design/20230421_stabilize_monotonic_select.md index 3bdc8b0bbb0af..ebdbc0766cc45 100644 --- a/doc/developer/design/20230421_stabilize_monotonic_select.md +++ b/doc/developer/design/20230421_stabilize_monotonic_select.md @@ -1,15 +1,15 @@ - Feature name: Monotonic One-Shot `SELECT`s - Associated: - MaterializeInc/materialize#14442 (epic), + MaterializeInc/database-issues#4125 (epic), MaterializeInc/materialize#18546 (MVP PR), - MaterializeInc/materialize#18089 (required issue), - MaterializeInc/materialize#18734 (required issue), - MaterializeInc/materialize#17358 (required issue), + MaterializeInc/database-issues#5301 (required issue), + MaterializeInc/database-issues#5543 (required issue), + MaterializeInc/database-issues#5030 (required issue), # Summary [summary]: #summary -With PR MaterializeInc/materialize#18546, it became possible to enable a feature flag to allow for execution of one-shot `SELECT` queries with plans utilizing, whenever possible, monotonic operators instead of their non-monotonic variants. This feature has a potentially high payoff by reducing query processing time and memory demands of one-shot `SELECT`s that are not supported by indexes or materialized views. However, a few issues remain to be addressed before we can stabilize the feature, the main of which are: (a) Extending the syntax of `EXPLAIN` to produce one-shot as well as indexed / materialized view plans (MaterializeInc/materialize#18089); (b) Managing the impact in our tests, to keep coverage high (MaterializeInc/materialize#18734); (c) Refining the planning of monotonic one-shot `SELECT`s to avoid consolidation when possible (MaterializeInc/materialize#18732) or to otherwise ameliorate its execution (MaterializeInc/materialize#18890). Point (c) can be seen as the remaining work to completely exploit the almost monotonicity of one-shot `SELECT`s (MaterializeInc/materialize#17358). +With PR MaterializeInc/materialize#18546, it became possible to enable a feature flag to allow for execution of one-shot `SELECT` queries with plans utilizing, whenever possible, monotonic operators instead of their non-monotonic variants. This feature has a potentially high payoff by reducing query processing time and memory demands of one-shot `SELECT`s that are not supported by indexes or materialized views. However, a few issues remain to be addressed before we can stabilize the feature, the main of which are: (a) Extending the syntax of `EXPLAIN` to produce one-shot as well as indexed / materialized view plans (MaterializeInc/database-issues#5301); (b) Managing the impact in our tests, to keep coverage high (MaterializeInc/database-issues#5543); (c) Refining the planning of monotonic one-shot `SELECT`s to avoid consolidation when possible (MaterializeInc/database-issues#5542) or to otherwise ameliorate its execution (MaterializeInc/database-issues#5582). Point (c) can be seen as the remaining work to completely exploit the almost monotonicity of one-shot `SELECT`s (MaterializeInc/database-issues#5030). # Motivation [motivation]: #motivation @@ -18,7 +18,7 @@ Interactive queries that cannot exploit a fast path are processed in Materialize Thus, some one-shot `SELECT` queries could be accelerated by use of monotonic operators, exploiting the observation that single-time dataflows operate on logically monotonic input. The latter can be performed safely by selectively adding consolidation to monotonic plan variants, when necessary, to coerce logical into physical monotonicity. In many cases, the resulting queries can require less memory to process as well as execute in much less query processing time. -An MVP version of this approach was introduced in PR MaterializeInc/materialize#18546. In an [initial evaluation](https://docs.google.com/spreadsheets/d/1xiGX6gq8JWFgPXc3fOIeQMO8QtWTMtini4M0xtjomG4/edit#gid=0), it was shown that speedups are achieved in a variety of queries. In some queries, speedups can reach up to an order of magnitude. In only one out of 10 queries in the initial evaluation was a slowdown observed, and only when that query was executed with parallelism (for details of why, see issue MaterializeInc/materialize#18890). +An MVP version of this approach was introduced in PR MaterializeInc/materialize#18546. In an [initial evaluation](https://docs.google.com/spreadsheets/d/1xiGX6gq8JWFgPXc3fOIeQMO8QtWTMtini4M0xtjomG4/edit#gid=0), it was shown that speedups are achieved in a variety of queries. In some queries, speedups can reach up to an order of magnitude. In only one out of 10 queries in the initial evaluation was a slowdown observed, and only when that query was executed with parallelism (for details of why, see issue MaterializeInc/database-issues#5582). The MVP PR gated the execution of monotonic one-shot `SELECT`s behind the feature flag `enable_monotonic_oneshot_selects`, turned off by default. The goal of this design is to outline the necessary steps to stabilize this feature and retire the feature flag. @@ -62,15 +62,15 @@ When physical monotonicity originating from monotonic sources is recognized in M In the following, we consider the main outstanding issues to stabilize monotonic one-shot `SELECT`s and propose strategies to address each in turn. -## Revisiting the Output of `EXPLAIN` (MaterializeInc/materialize#18089) +## Revisiting the Output of `EXPLAIN` (MaterializeInc/database-issues#5301) A monotonic one-shot `SELECT` has a physical plan that differs from the one for the same `SELECT` in an indexed or materialized view. We therefore propose that syntax be provided to enable `EXPLAIN`, and in particular `EXPLAIN PHYSICAL PLAN`, to either produce one-shot `SELECT` plans or plans for other optimization paths. -Issue MaterializeInc/materialize#18089 captures the discussion and design of the syntax for `EXPLAIN` to reflect different optimization paths. We defer the proposal of a concrete syntax to the design originating from that issue. +Issue MaterializeInc/database-issues#5301 captures the discussion and design of the syntax for `EXPLAIN` to reflect different optimization paths. We defer the proposal of a concrete syntax to the design originating from that issue. Given an adequate syntax, the optimization path for one-shot `SELECT`s in `EXPLAIN` should then include a setting of both `as_of` and `until` prior to invoking `finalize_dataflow`. It is critical that `until` be set to `as_of + 1`, assuming both attributes refer to a single-timestamp, one-dimensional frontier. This proper setting of `as_of` and `until` was already implemented in MaterializeInc/materialize#19223. -## Running Tests in CI with Both Monotonic and Non-Monotonic Plan Variants (MaterializeInc/materialize#18734) +## Running Tests in CI with Both Monotonic and Non-Monotonic Plan Variants (MaterializeInc/database-issues#5543) Reusing `sqllogictest` corpora allows us to get a high coverage of SQL constructs. So it is ideal that we be able to continue reusing corpora designed for other DBMS with low effort, even after the feature flag `enable_monotonic_oneshot_selects` is retired. At the same time, other DBMS may not introduce the same differentiation between one-shot `SELECT` plans and plans for indexed or materialized views as in Materialize. Therefore, their corpora would start to fundamentallly lack coverage of the plans in Materialize that are specialized to view maintenance, once the feature of monotonic one-shot `SELECT`s is enabled. @@ -100,7 +100,7 @@ A final alternative to be considered is a refinement of the proposal above in wh There are two strategies that we propose to lower the expense of monotonic operators introduced with forced consolidation during single-time LIR `Plan` refinement. -### Improve `refine_single_time_dataflow` LIR-based Refinement (MaterializeInc/materialize#18732) +### Improve `refine_single_time_dataflow` LIR-based Refinement (MaterializeInc/database-issues#5542) It is not always the case the `must_consolidate` must be set for every operator coverted to monotonic in a one-shot `SELECT`. For example, in a single-time dataflow, some operators may produce physically monotonic output regardless of whether their input was even logically monotonic, e.g., reductions. Furthermore, some operators will convert single-time, logically monotonic input into physically monotonic input, e.g., operators creating an arrangement. So if the input to a monotonic operator with forced consolidation came from a reduction in a non-recursive context, then this operator's `must_consolidate` flag could instead be toggled to `false`. @@ -124,7 +124,7 @@ Instead of breaking down the single-time LIR-based refinement step into two sub- We also considered an iterative alternative for `refine_single_time_consolidation` that could be implemented as a series of iterative child traversals starting from each monotonic `Plan` node replaced in the `refine_single_time_operator_selection` sub-action. However, that alternative has the potential for poor runtime complexity in the presence of nesting of monotonic `Plan::Reduce` nodes or if the expressions defined in `Plan::Let` nodes need to be revisited many times. These issues can be ameliorated by pruning the iterative traversal in monotonicity enforcers (`Plan::Reduce` being one of them, and thus addressing the nesting issue) and by not expading `Plan:Let` expressions. Unfortunately, the latter results in imprecision in the analysis, wherein some opportunities to turn off consolidation would be lost. -### Improve `Exchange` in Forced Consolidation (MaterializeInc/materialize#18890) +### Improve `Exchange` in Forced Consolidation (MaterializeInc/database-issues#5582) Among the work items to stabilize monotonic one-shot `SELECT`s, this is the least controversial, since it emerges from experimental observation of the cost of forced consolidation in a subset of simple interactive queries, especifically computing `MIN`/`MAX` over an entire collection. After investigation, it was determined that the expense of forced consolidation manifested primarily in a multi-worker setting, and it was due to load imbalance among workers. PR MaterializeInc/materialize#19372 goes into more depth, determining that the problem was caused by low entropy in DD's default `FnvHasher`. The PR then fixes the problem by changing the hash function used for the `Exchange` in forced consolidation to `AHash`. While `AHash` does not provide theoretical guarantees (in contrast to twisted tabulation hashing), it strikes an adequate trade-off in ecosystem support, development agility, and empirically observed hashing quality and speed. A second PR MaterializeInc/materialize#19848 ensured that our usage of `AHash` was deterministic across workers, even in multi-process settings. @@ -133,12 +133,12 @@ Among the work items to stabilize monotonic one-shot `SELECT`s, this is the leas As mentioned previously, the feature is now gated behind the flag `enable_monotonic_oneshot_selects`. The proposals in this document assume that we would like to eventually retire this feature flag. Based on this observation, the following lifecycle is proposed: -* Once MaterializeInc/materialize#18734 is tackled, we can let the feature be continuously tested in CI and nightly during at least 2 weeks. -* Given the work in MaterializeInc/materialize#19223, we can enable the flag in staging once initial validation through testing succeeds. Ideally, we should let people experiment with the feature in staging for 1-2 weeks. Note that a full solution to MaterializeInc/materialize#18089 would be ideal to eliminate confusion regarding `EXPLAIN` behavior and make the feature more understandable to customers in production. -* We consider solutions to MaterializeInc/materialize#18732 and MaterializeInc/materialize#18890 as highly desirable work items that bring about performance improvements for this feature. We can consider these improvements, however, optional wrt. enablement of the feature in staging. It would be ideal to tackle at a minimum MaterializeInc/materialize#18890 before enabling the feature in production, as we observed potential regressions in simple plans (e.g., for `MIN`/`MAX` over an entire collection) that would be ameliorated by a fix. If any of these fixes lands after the observation period for the previous two bullet points, we propose an additional 1 week of observation in staging and CI for derisking. +* Once MaterializeInc/database-issues#5543 is tackled, we can let the feature be continuously tested in CI and nightly during at least 2 weeks. +* Given the work in MaterializeInc/materialize#19223, we can enable the flag in staging once initial validation through testing succeeds. Ideally, we should let people experiment with the feature in staging for 1-2 weeks. Note that a full solution to MaterializeInc/database-issues#5301 would be ideal to eliminate confusion regarding `EXPLAIN` behavior and make the feature more understandable to customers in production. +* We consider solutions to MaterializeInc/database-issues#5542 and MaterializeInc/database-issues#5582 as highly desirable work items that bring about performance improvements for this feature. We can consider these improvements, however, optional wrt. enablement of the feature in staging. It would be ideal to tackle at a minimum MaterializeInc/database-issues#5582 before enabling the feature in production, as we observed potential regressions in simple plans (e.g., for `MIN`/`MAX` over an entire collection) that would be ameliorated by a fix. If any of these fixes lands after the observation period for the previous two bullet points, we propose an additional 1 week of observation in staging and CI for derisking. * Once the feature is enabled in production, we can monitor for 1-2 weeks, and then plan the retirement of the feature flag in the next release cycle. -The relatively short observation periods above are introduced based on the judgment that this feature is of medium risk. We already render monotonic operators in production at the present time when Materialize is faced with queries that operate on monotonic sources. Additionally, we have a small set of `testdrive` tests that are focused on monotonic operators. However, with this feature, usage of monotonic operators will increase substantially. So we believe that increases in test diversity (MaterializeInc/materialize#18734) and observability (MaterializeInc/materialize#18089) are pre-conditions to move forward safely. +The relatively short observation periods above are introduced based on the judgment that this feature is of medium risk. We already render monotonic operators in production at the present time when Materialize is faced with queries that operate on monotonic sources. Additionally, we have a small set of `testdrive` tests that are focused on monotonic operators. However, with this feature, usage of monotonic operators will increase substantially. So we believe that increases in test diversity (MaterializeInc/database-issues#5543) and observability (MaterializeInc/database-issues#5301) are pre-conditions to move forward safely. # Unresolved questions [unresolved-questions]: #unresolved-questions diff --git a/doc/developer/design/20230512_mir_cost_model.md b/doc/developer/design/20230512_mir_cost_model.md index 4539c85f9a9b7..a2d0a06deae89 100644 --- a/doc/developer/design/20230512_mir_cost_model.md +++ b/doc/developer/design/20230512_mir_cost_model.md @@ -1,13 +1,13 @@ - Feature name: MIR cost model - Associated: - * https://github.com/MaterializeInc/materialize/issues/17883 - * https://github.com/MaterializeInc/materialize/issues/16511 - * https://github.com/MaterializeInc/materialize/issues/15244 - * https://github.com/MaterializeInc/materialize/issues/13140 - * https://github.com/MaterializeInc/materialize/issues/12069 - * https://github.com/MaterializeInc/materialize/issues/9157 - * https://github.com/MaterializeInc/materialize/issues/14663 - * https://github.com/MaterializeInc/materialize/issues/13046 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/5216 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/4781 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/4364 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/3779 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/3511 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/2804 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/4188 + * https://api.github.com/repos/MaterializeInc/database-issues/issues/3754 Authors: Michael Greenberg Created: May 5, 2023 10:25 AM @@ -38,15 +38,15 @@ Our cost model should be *executable* and *accurate:* ## Goals - Reify what we learn from various benchmarks/experiments. - - https://github.com/MaterializeInc/materialize/issues/17883 - - https://github.com/MaterializeInc/materialize/issues/16511 - - https://github.com/MaterializeInc/materialize/issues/15244 - - https://github.com/MaterializeInc/materialize/issues/13140 - - https://github.com/MaterializeInc/materialize/issues/12069 - - https://github.com/MaterializeInc/materialize/issues/9157 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/5216 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/4781 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/4364 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/3779 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/3511 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/2804 - Offer visibility/predicability. - - https://github.com/MaterializeInc/materialize/issues/14663 - - https://github.com/MaterializeInc/materialize/issues/13046 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/4188 + - https://api.github.com/repos/MaterializeInc/database-issues/issues/3754 - Open opportunities for principled/well-informed choices in the optimizer. ## Non-goals @@ -184,4 +184,4 @@ We've identified the following early candidate clients for the cost model: 5. **Broadcast selection.** Using cardinality information, choose cheap broadcasts. https://materializeinc.slack.com/archives/C02PPB50ZHS/p1685614805876319?thread_ts=1685552977.381309&cid=C02PPB50ZHS -6. **OR <-> UNION decomposition.** It is possible to decompose a disjunctive `WHERE` clause into a `UNION`, and this is _sometimes_ worthwhile. https://github.com/MaterializeInc/materialize/issues/4229#issuecomment-1571767817 +6. **OR <-> UNION decomposition.** It is possible to decompose a disjunctive `WHERE` clause into a `UNION`, and this is _sometimes_ worthwhile. https://api.github.com/repos/MaterializeInc/database-issues/issues/1312#issuecomment-1571767817 diff --git a/doc/developer/design/20230528_managed_clusters.md b/doc/developer/design/20230528_managed_clusters.md index 03a0e44869b9c..8ca8756b9eb60 100644 --- a/doc/developer/design/20230528_managed_clusters.md +++ b/doc/developer/design/20230528_managed_clusters.md @@ -1,6 +1,6 @@ # Managed clusters -Epic: [#19547](https://github.com/MaterializeInc/materialize/issues/19547) +Epic: [#19547](https://api.github.com/repos/MaterializeInc/database-issues/issues/5808) ## Context @@ -361,6 +361,6 @@ We could wait to improve the API until we've built [dynamic cluster scheduling]. [cluster replica design]: https://github.com/MaterializeInc/materialize/blob/d7101d4b952f9eb4f1185fdcceacd64c4d151de5/doc/developer/design/20220413_cluster_replica.md#future-work -[dynamic cluster scheduling]: https://github.com/MaterializeInc/materialize/issues/13870 +[dynamic cluster scheduling]: https://api.github.com/repos/MaterializeInc/database-issues/issues/3968 [terraform provider]: https://github.com/MaterializeInc/terraform-provider-materialize/issues/145 [web console]: https://materializeinc.slack.com/archives/CU7ELJ6E9/p1683568811596419?thread_ts=1683565139.938549&cid=CU7ELJ6E9 diff --git a/doc/developer/design/20230531_compute_metrics.md b/doc/developer/design/20230531_compute_metrics.md index e03ef5149d5cb..debbb32b22511 100644 --- a/doc/developer/design/20230531_compute_metrics.md +++ b/doc/developer/design/20230531_compute_metrics.md @@ -1,9 +1,9 @@ - Feature name: Operational metrics for Compute - Associated: - * [#18745: Improve Compute metrics dashboard](https://github.com/MaterializeInc/materialize/issues/18745) - * [#16026: Prometheus metrics for the Compute Controller](https://github.com/MaterializeInc/materialize/issues/16026) - * [#16951: Add replica connection metrics](https://github.com/MaterializeInc/materialize/issues/16951) - * [#17064: Add replica fully hydrated metric](https://github.com/MaterializeInc/materialize/issues/17064) + * [#18745: Improve Compute metrics dashboard](https://api.github.com/repos/MaterializeInc/database-issues/issues/5547) + * [#16026: Prometheus metrics for the Compute Controller](https://api.github.com/repos/MaterializeInc/database-issues/issues/4621) + * [#16951: Add replica connection metrics](https://api.github.com/repos/MaterializeInc/database-issues/issues/4895) + * [#17064: Add replica fully hydrated metric](https://api.github.com/repos/MaterializeInc/database-issues/issues/4934) # Summary [summary]: #summary @@ -56,7 +56,7 @@ The Compute components in scope are: We consider the following goals useful but outside the scope of this design: * Providing analytics data to inform product decisions. -* Providing observability for the optimizer. This work is tracked in . +* Providing observability for the optimizer. This work is tracked in . # Explanation [explanation]: #explanation @@ -410,6 +410,6 @@ If it turns out that the additional queries made by the prometheus-exporter plac Once this design is implemented, Compute observability will be solid but likely not perfect. Future work will consist of identifying a) new valuable metrics we should also implement and b) implemented metrics that should be removed because they don't provide enough value. -A large part of a) will consist of adding metrics for optimizer operation, as tracked by . +A large part of a) will consist of adding metrics for optimizer operation, as tracked by . Implementing a reliable and hydration signal for replicas and dataflows is left as future work as well. diff --git a/doc/developer/design/20230607_shift_status_updates.md b/doc/developer/design/20230607_shift_status_updates.md index d56f9c4e2fd44..5fcb46383abf7 100644 --- a/doc/developer/design/20230607_shift_status_updates.md +++ b/doc/developer/design/20230607_shift_status_updates.md @@ -14,7 +14,7 @@ ## Associated -- [#20036](https://github.com/MaterializeInc/materialize/issues/20036) +- [#20036](https://api.github.com/repos/MaterializeInc/database-issues/issues/5986) - Accomplishing 'subsource demuxing' using the concept of _subsources_ in-use today. - The goals of https://github.com/MaterializeInc/materialize/issues/24843 will be + The goals of https://api.github.com/repos/MaterializeInc/database-issues/issues/7417 will be accomplished during the process of replacing _subsources_ with _tables_ (see solution proposal below). @@ -562,7 +562,7 @@ problems outlined above: have to be implemented for both top-level sources and subsources). 3. Implement demuxing of upstream tables using existing subsource model - (https://github.com/MaterializeInc/materialize/issues/24843). This would be an + (https://api.github.com/repos/MaterializeInc/database-issues/issues/7417). This would be an easier effort to implement, but rather than doing this work in the short term for it to be later ripped out, it is preferable to implement the interface that solves both this need and the other problems outlined above. diff --git a/doc/developer/design/20320328_persist_columnar.md b/doc/developer/design/20320328_persist_columnar.md index 7d45bdc9859f1..953f4fae84641 100644 --- a/doc/developer/design/20320328_persist_columnar.md +++ b/doc/developer/design/20320328_persist_columnar.md @@ -1,9 +1,9 @@ # Self Describing Persist Batches - Associated: - - [write Schema-ified data blobs #24830](https://github.com/MaterializeInc/materialize/issues/24830) - - [persist: schema evolution](https://github.com/MaterializeInc/materialize/issues/16625) - - [Table support for push sources](https://github.com/MaterializeInc/materialize/issues/22836) + - [write Schema-ified data blobs #24830](https://api.github.com/repos/MaterializeInc/database-issues/issues/7411) + - [persist: schema evolution](https://api.github.com/repos/MaterializeInc/database-issues/issues/4818) + - [Table support for push sources](https://api.github.com/repos/MaterializeInc/database-issues/issues/6896) - [[dnm] columnar: Write columnar encodings as part of a Batch](https://github.com/MaterializeInc/materialize/pull/26120) - [[dnm] columnar: Array support](https://github.com/MaterializeInc/materialize/pull/25848) @@ -55,7 +55,7 @@ column named `b`. * User defined sort order of data, i.e. `PARTITION BY`. * Only fetch the columns from a shard that are needed, i.e. projection pushdown. - * Make `persist_source` faster [#25901](https://github.com/MaterializeInc/materialize/issues/25901). + * Make `persist_source` faster [#25901](https://api.github.com/repos/MaterializeInc/database-issues/issues/7726). ## Out of Scope @@ -831,7 +831,7 @@ be relatively inefficient, but none of these scenarios should happen too often. > **Note:** In the current state of the world we'll need to support compacting `Codec` and Arrow data forever since we can't guarantee all batches written with `Codec` have been compacted yet. The ability to make this guarantee is -something we could add to [Compaction 2.0](https://github.com/MaterializeInc/materialize/issues/16607). +something we could add to [Compaction 2.0](https://api.github.com/repos/MaterializeInc/database-issues/issues/4809). > **Note:** While technically out-of-scope for this design doc, we can implement efficient consolidation for case 3 (table evolution) without decoding diff --git a/doc/developer/diagnostic-questions.md b/doc/developer/diagnostic-questions.md index ec72fd199b641..9042692307683 100644 --- a/doc/developer/diagnostic-questions.md +++ b/doc/developer/diagnostic-questions.md @@ -59,7 +59,7 @@ What is the user's goal? What are the column types? - Are two numbers being joined on actually the same type? Join plans can be messed up by implicit casting from int4 (resp. float4) to int8 (resp. float8) - See [#4171](https://github.com/MaterializeInc/materialize/issues/4171) + See [#4171](https://api.github.com/repos/MaterializeInc/database-issues/issues/1291) Is the plan sane? (Ask for raw/decorrelated/optimized plans) - Can an expensive operator be pushed down? diff --git a/doc/user/content/ingest-data/cdc-sql-server.md b/doc/user/content/ingest-data/cdc-sql-server.md index a7b674171c538..94a6a879ece90 100644 --- a/doc/user/content/ingest-data/cdc-sql-server.md +++ b/doc/user/content/ingest-data/cdc-sql-server.md @@ -173,7 +173,7 @@ cluster, use the `IN CLUSTER` clause. Debezium provides [transaction metadata](https://debezium.io/documentation/reference/connectors/sqlserver.html#sqlserver-transaction-metadata) that can be used to preserve transactional boundaries downstream. Work is in progress to utilize this topic to support transaction-aware processing in -[Materialize #7537](https://github.com/MaterializeInc/materialize/issues/7537)! +[Materialize #7537](https://api.github.com/repos/MaterializeInc/database-issues/issues/2337)! ### Create a materialized view diff --git a/doc/user/content/ingest-data/mysql/debezium.md b/doc/user/content/ingest-data/mysql/debezium.md index a8665f88cac04..1eac1897e5db6 100644 --- a/doc/user/content/ingest-data/mysql/debezium.md +++ b/doc/user/content/ingest-data/mysql/debezium.md @@ -209,7 +209,7 @@ cluster, use the `IN CLUSTER` clause. Debezium provides [transaction metadata](https://debezium.io/documentation/reference/connectors/mysql.html#mysql-transaction-metadata) that can be used to preserve transactional boundaries downstream. We are working -on using this topic to support transaction-aware processing in [Materialize #7537](https://github.com/MaterializeInc/materialize/issues/7537)! +on using this topic to support transaction-aware processing in [Materialize #7537](https://api.github.com/repos/MaterializeInc/database-issues/issues/2337)! ### Create a materialized view diff --git a/doc/user/content/ingest-data/postgres/debezium.md b/doc/user/content/ingest-data/postgres/debezium.md index a9a64dd122cf3..b543a56ac7cbf 100644 --- a/doc/user/content/ingest-data/postgres/debezium.md +++ b/doc/user/content/ingest-data/postgres/debezium.md @@ -352,7 +352,7 @@ This allows you to replicate tables with `REPLICA IDENTITY DEFAULT`, `INDEX`, or Debezium provides [transaction metadata](https://debezium.io/documentation/reference/connectors/postgresql.html#postgresql-transaction-metadata) that can be used to preserve transactional boundaries downstream. We are working on using this topic to support transaction-aware processing in -[Materialize #7537](https://github.com/MaterializeInc/materialize/issues/7537)! +[Materialize #7537](https://api.github.com/repos/MaterializeInc/database-issues/issues/2337)! ## D. Create a materialized view diff --git a/doc/user/content/integrations/_index.md b/doc/user/content/integrations/_index.md index 41e10ca25dcc2..a736f4e9a1429 100644 --- a/doc/user/content/integrations/_index.md +++ b/doc/user/content/integrations/_index.md @@ -11,7 +11,7 @@ aliases: - /guides/ --- -[//]: # "TODO(morsapaes) Once materialize#8396 lands, link the page here" +[//]: # "TODO(morsapaes) Once database-issues#2562 lands, link the page here" Materialize is **wire-compatible** with PostgreSQL, which means it integrates with many SQL clients and other tools in the data ecosystem that support diff --git a/misc/cockroach/setup_materialize.sql b/misc/cockroach/setup_materialize.sql index b372c6f2dd118..5d9f5ee1e569f 100644 --- a/misc/cockroach/setup_materialize.sql +++ b/misc/cockroach/setup_materialize.sql @@ -10,7 +10,7 @@ -- Sets up a CockroachDB cluster for use by Materialize. -- See: https://github.com/cockroachdb/cockroach/issues/93892 --- See: https://github.com/MaterializeInc/materialize/issues/16726 +-- See: https://api.github.com/repos/MaterializeInc/database-issues/issues/4843 SET CLUSTER SETTING sql.stats.forecasts.enabled = false; SET CLUSTER SETTING storage.max_sync_duration.fatal.enabled = false; diff --git a/misc/dbt-materialize/CHANGELOG.md b/misc/dbt-materialize/CHANGELOG.md index b4544d7c0124b..ab72bfbaf0411 100644 --- a/misc/dbt-materialize/CHANGELOG.md +++ b/misc/dbt-materialize/CHANGELOG.md @@ -39,7 +39,7 @@ * Produce an error message when attempting to use the [`grants` configuration](https://docs.getdbt.com/reference/resource-configs/grants), which is not supported in the adapter. This configuration will be supported in - the future (see [#20244](https://github.com/MaterializeInc/materialize/issues/20244)). + the future (see [#20244](https://api.github.com/repos/MaterializeInc/database-issues/issues/6073)). * Stop hardcoding `quickstart` as the default cluster to fall back to when no cluster is specified. When no cluster is specified, either in `profiles.yml` @@ -112,7 +112,7 @@ ## 1.7.7 - 2024-04-19 * Tweak [`deploy_permission_validation`](https://github.com/MaterializeInc/materialize/blob/main/misc/dbt-materialize/dbt/include/materialize/macros/deploy/deploy_permission_validation.sql) - macro to work around [#26738](https://github.com/MaterializeInc/materialize/issues/26738). + macro to work around [#26738](https://api.github.com/repos/MaterializeInc/database-issues/issues/7902). ## 1.7.6 - 2024-04-18 @@ -217,7 +217,7 @@ ## 1.7.1 - 2023-12-14 * Remove the dependency of data contracts pre-flight checks on the existence of - the pre-installed `default` cluster. Fixes [#23600](https://github.com/MaterializeInc/materialize/issues/23600). + the pre-installed `default` cluster. Fixes [#23600](https://api.github.com/repos/MaterializeInc/database-issues/issues/7091). * Work around [dbt-core #8353](https://github.com/dbt-labs/dbt-core/issues/8353) while a permanent fix doesn't land in dbt Core to unblock users using UUID @@ -358,7 +358,7 @@ supported** in this release (see [dbt-core #7213](https://github.com/dbt-labs/dbt-core/discussions/7213#discussioncomment-5903205)). * Fix a bug in the `materialize__list_relations_without_caching` macro which - could cause the adapter to break for multi-output sources ([#20483](https://github.com/MaterializeInc/materialize/issues/20483)). + could cause the adapter to break for multi-output sources ([#20483](https://api.github.com/repos/MaterializeInc/database-issues/issues/6162)). * Expose `owner` in the dbt documentation, now that Materialize supports [role-based access control (RBAC)](https://materialize.com/docs/manage/access-control/). @@ -549,7 +549,7 @@ dbt adapters, which simply execute all hooks outside of a transaction regardless of their configured `transaction` behavior. -[#7675]: https://github.com/MaterializeInc/materialize/issues/7675 +[#7675]: https://api.github.com/repos/MaterializeInc/database-issues/issues/2375 ## 1.0.1.post1 - 2022-02-14 @@ -592,7 +592,7 @@ * Add three new custom materialization types: `source`, `index`, and `sink`. These replace the aforementioned macros that were removed in this release. -[#7810]: https://github.com/MaterializeInc/materialize/issues/7810 +[#7810]: https://api.github.com/repos/MaterializeInc/database-issues/issues/2404 ## 0.20.1 - 2021-08-12 @@ -629,7 +629,7 @@ incorrectly determined ([#6063]). This most notably caused information about model columns to be missing in the documentation generated by `dbt docs`. -[#6063]: https://github.com/MaterializeInc/materialize/issues/6063 +[#6063]: https://api.github.com/repos/MaterializeInc/database-issues/issues/1871 ## 0.18.1 - 2021-02-25 diff --git a/misc/dbt-materialize/README.md b/misc/dbt-materialize/README.md index 39a843fd1263e..c8b7498339e4e 100644 --- a/misc/dbt-materialize/README.md +++ b/misc/dbt-materialize/README.md @@ -128,7 +128,7 @@ Not supported. Support is not planned for the near term. A huge thank you to [Josh Wills](https://github.com/jwills), who created the original version of this adapter. 🤠 -[#5266]: https://github.com/MaterializeInc/materialize/issues/5266 +[#5266]: https://api.github.com/repos/MaterializeInc/database-issues/issues/1623 [dbt]: https://www.getdbt.com/ [index]: https://materialize.com/docs/sql/create-index/ [Materialize]: http://materialize.com diff --git a/misc/dbt-materialize/dbt/adapters/materialize/impl.py b/misc/dbt-materialize/dbt/adapters/materialize/impl.py index 816a11a628ea4..735e7e7336a64 100644 --- a/misc/dbt-materialize/dbt/adapters/materialize/impl.py +++ b/misc/dbt-materialize/dbt/adapters/materialize/impl.py @@ -113,7 +113,7 @@ class MaterializeAdapter(PostgresAdapter, SQLAdapter): # NOTE(morsapaes): Materialize supports enforcing the NOT NULL constraint # for materialized views (via the ASSERT NOT NULL clause) and tables. As a - # reminder, tables are modeled as static materialized views (see materialize#5266). + # reminder, tables are modeled as static materialized views (see database-issues#1623). CONSTRAINT_SUPPORT = { ConstraintType.check: ConstraintSupport.NOT_SUPPORTED, ConstraintType.not_null: ConstraintSupport.ENFORCED, diff --git a/misc/dbt-materialize/dbt/adapters/materialize/relation.py b/misc/dbt-materialize/dbt/adapters/materialize/relation.py index afd382ebd8566..08842ded09ad6 100644 --- a/misc/dbt-materialize/dbt/adapters/materialize/relation.py +++ b/misc/dbt-materialize/dbt/adapters/materialize/relation.py @@ -49,7 +49,7 @@ class MaterializeRelation(PostgresRelation): # Materialize does not have a 63-character limit for relation names, unlike # PostgreSQL (see dbt-core #2727). Instead, we set 255 as the maximum - # identifier length (see materialize#20931). + # identifier length (see database-issues#6303). def relation_max_name_length(self): return 255 diff --git a/misc/dbt-materialize/dbt/include/materialize/macros/adapters.sql b/misc/dbt-materialize/dbt/include/materialize/macros/adapters.sql index 90e6ed8ff6138..88e753214bfc9 100644 --- a/misc/dbt-materialize/dbt/include/materialize/macros/adapters.sql +++ b/misc/dbt-materialize/dbt/include/materialize/macros/adapters.sql @@ -218,7 +218,7 @@ -- Materialize does not support running multiple COMMENT ON commands in a -- transaction, so we work around that by forcing a transaction per comment -- instead - -- See: https://github.com/MaterializeInc/materialize/issues/22379 + -- See: https://api.github.com/repos/MaterializeInc/database-issues/issues/6759 {% for column_name in column_dict if (column_name in existing_columns) %} {% set comment = column_dict[column_name]['description'] %} {% set quote = column_dict[column_name]['quote'] %} @@ -290,7 +290,7 @@ join mz_databases d on s.database_id = d.id and d.name = '{{ schema_relation.database }}' where o.type in ('table', 'source', 'view', 'materialized-view', 'index', 'sink') -- Exclude subsources and progress subsources, which aren't relevant in this - -- context and can bork the adapter (see materialize#20483) + -- context and can bork the adapter (see database-issues#6162) and coalesce(so.type, '') not in ('subsource', 'progress') {% endcall %} {{ return(load_result('list_relations_without_caching').table) }} diff --git a/misc/dbt-materialize/tests/adapter/test_constraints.py b/misc/dbt-materialize/tests/adapter/test_constraints.py index 1aeea2a96dfdf..a488659518948 100644 --- a/misc/dbt-materialize/tests/adapter/test_constraints.py +++ b/misc/dbt-materialize/tests/adapter/test_constraints.py @@ -157,7 +157,7 @@ def models(self): # In the absence of the pre-installed `quickstart` cluster, Materialize should # not error if data contracts are enforced. - # See materialize#23600: https://github.com/MaterializeInc/materialize/issues/23600 + # See database-issues#7091: https://api.github.com/repos/MaterializeInc/database-issues/issues/7091 def test_materialize_drop_quickstart(self, project): project.run_sql("DROP CLUSTER quickstart CASCADE") @@ -176,6 +176,6 @@ def models(self): # Pseudo-types in Materialize cannot be cast using the cast() function, so we # special-handle their NULL casting for contract validation. - # See materialize#17870: https://github.com/MaterializeInc/materialize/issues/17870 + # See database-issues#5211: https://api.github.com/repos/MaterializeInc/database-issues/issues/5211 def test_pseudo_types(self, project): run_dbt(["run", "--models", "contract_pseudo_types"], expect_pass=True) diff --git a/misc/dbt-materialize/tests/adapter/test_relation_types.py b/misc/dbt-materialize/tests/adapter/test_relation_types.py index a539cfd1970ee..8e1e958040f27 100644 --- a/misc/dbt-materialize/tests/adapter/test_relation_types.py +++ b/misc/dbt-materialize/tests/adapter/test_relation_types.py @@ -58,7 +58,7 @@ def docs(self, project): ("model.test.test_source", "source"), # ("model.test.test_subsource", "subsource"), # NOTE(dehume): Tables are materialized as materialized views - # https://github.com/MaterializeInc/materialize/issues/5266 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/1623 ("model.test.test_table", "materialized_view"), ("model.test.test_view", "view"), ], diff --git a/misc/dbt-materialize/tests/adapter/test_utils.py b/misc/dbt-materialize/tests/adapter/test_utils.py index 4417d4655bd97..f027cb4f73f02 100644 --- a/misc/dbt-materialize/tests/adapter/test_utils.py +++ b/misc/dbt-materialize/tests/adapter/test_utils.py @@ -104,7 +104,7 @@ class TestCast(BaseCast): # The `cast_bool_to_text` macro works as expected, but we must alter the test case # because set operation type conversions do not work properly. -# See https://github.com/MaterializeInc/materialize/issues/3331 +# See https://api.github.com/repos/MaterializeInc/database-issues/issues/1065 class TestCastBoolToText(BaseCastBoolToText): @pytest.fixture(scope="class") def models(self): diff --git a/misc/python/materialize/buildkite_insights/segfaults/find.py b/misc/python/materialize/buildkite_insights/segfaults/find.py index e353dd49fba7c..f7335c06444a4 100755 --- a/misc/python/materialize/buildkite_insights/segfaults/find.py +++ b/misc/python/materialize/buildkite_insights/segfaults/find.py @@ -15,7 +15,7 @@ def main() -> None: - # Used to find recent instances of https://github.com/MaterializeInc/materialize/issues/24644 + # Used to find recent instances of https://api.github.com/repos/MaterializeInc/database-issues/issues/7338 # 2 weeks ~ 2000 builds data = builds_api.get_builds_of_all_pipelines(max_fetches=20, branch=None) diff --git a/misc/python/materialize/checks/all_checks/debezium.py b/misc/python/materialize/checks/all_checks/debezium.py index 9c92d52b864c0..b0a81fcdb1498 100644 --- a/misc/python/materialize/checks/all_checks/debezium.py +++ b/misc/python/materialize/checks/all_checks/debezium.py @@ -52,7 +52,7 @@ def initialize(self) -> Testdrive: $ kafka-wait-topic topic=postgres.public.debezium_table - # UPSERT is requred due to https://github.com/MaterializeInc/materialize/issues/14211 + # UPSERT is requred due to https://api.github.com/repos/MaterializeInc/database-issues/issues/4064 > CREATE SOURCE debezium_source1 FROM KAFKA CONNECTION kafka_conn (TOPIC 'postgres.public.debezium_table') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn diff --git a/misc/python/materialize/checks/all_checks/identifiers.py b/misc/python/materialize/checks/all_checks/identifiers.py index a8100f93859b9..b7d4ff1451729 100644 --- a/misc/python/materialize/checks/all_checks/identifiers.py +++ b/misc/python/materialize/checks/all_checks/identifiers.py @@ -83,7 +83,7 @@ def __init__(self, base_version: MzVersion, rng: Random | None) -> None: } # ERROR: invalid input syntax for type bytea: invalid escape sequence self.ident["secret_value"] = "secret_value" - # https://github.com/MaterializeInc/materialize/issues/22535 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/6813 self.ident["source"] = "source" super().__init__(base_version, rng) diff --git a/misc/python/materialize/checks/all_checks/materialized_views.py b/misc/python/materialize/checks/all_checks/materialized_views.py index cfb83de59b45f..9a5c198ef9071 100644 --- a/misc/python/materialize/checks/all_checks/materialized_views.py +++ b/misc/python/materialize/checks/all_checks/materialized_views.py @@ -26,7 +26,7 @@ def initialize(self) -> Testdrive: """ + ( """ - # Regression test for materialize#27167. + # Regression test for database-issues#8032. > CREATE MATERIALIZED VIEW zero_arity AS SELECT; """ if self.base_version >= MzVersion.parse_mz("v0.101.0") diff --git a/misc/python/materialize/checks/all_checks/multiple_partitions.py b/misc/python/materialize/checks/all_checks/multiple_partitions.py index 466e8bbf8a941..16c9e46fb5eac 100644 --- a/misc/python/materialize/checks/all_checks/multiple_partitions.py +++ b/misc/python/materialize/checks/all_checks/multiple_partitions.py @@ -120,7 +120,7 @@ def validate(self) -> Testdrive: > SELECT SUM(p.offset) FROM multiple_partitions_source_progress p; 420 - # TODO: Reenable when materialize#25803 is fixed + # TODO: Reenable when database-issues#7695 is fixed # > SELECT status FROM mz_internal.mz_source_statuses WHERE name = 'multiple_partitions_source'; # running diff --git a/misc/python/materialize/checks/all_checks/peek_cancellation.py b/misc/python/materialize/checks/all_checks/peek_cancellation.py index 0554108b0011d..5686c9d5eecf5 100644 --- a/misc/python/materialize/checks/all_checks/peek_cancellation.py +++ b/misc/python/materialize/checks/all_checks/peek_cancellation.py @@ -12,7 +12,7 @@ from materialize.checks.checks import Check, disabled -@disabled("due to materialize#20743") +@disabled("due to database-issues#6249") class PeekCancellation(Check): def initialize(self) -> Testdrive: return Testdrive( diff --git a/misc/python/materialize/checks/all_checks/pg_cdc.py b/misc/python/materialize/checks/all_checks/pg_cdc.py index c82ea403b0cfa..5be926ae824ac 100644 --- a/misc/python/materialize/checks/all_checks/pg_cdc.py +++ b/misc/python/materialize/checks/all_checks/pg_cdc.py @@ -121,11 +121,11 @@ def manipulate(self) -> list[Testdrive]: """ + ( f""" - # Wait until Pg snapshot is complete in order to avoid materialize#18940 + # Wait until Pg snapshot is complete in order to avoid database-issues#5601 > SELECT COUNT(*) > 0 FROM postgres_source_tableA{self.suffix} true - # Wait until Pg snapshot is complete in order to avoid materialize#18940 + # Wait until Pg snapshot is complete in order to avoid database-issues#5601 > SELECT COUNT(*) > 0 FROM postgres_source_tableB{self.suffix} true """ @@ -171,7 +171,7 @@ def manipulate(self) -> list[Testdrive]: """ + ( f""" - # Wait until Pg snapshot is complete in order to avoid materialize#18940 + # Wait until Pg snapshot is complete in order to avoid database-issues#5601 > SELECT COUNT(*) > 0 FROM postgres_source_tableB{self.suffix} true > SELECT COUNT(*) > 0 FROM postgres_source_tableC{self.suffix} diff --git a/misc/python/materialize/cli/ci_annotate_errors.py b/misc/python/materialize/cli/ci_annotate_errors.py index f23b539a4a0c2..af9ef6b0c8e68 100644 --- a/misc/python/materialize/cli/ci_annotate_errors.py +++ b/misc/python/materialize/cli/ci_annotate_errors.py @@ -722,7 +722,7 @@ def _collect_errors_in_logs(data: Any, log_file_name: str) -> list[ErrorLog]: for match in ERROR_RE.finditer(data): if IGNORE_RE.search(match.group(0)): continue - # environmentd segfaults during normal shutdown in coverage builds, see materialize#20016 + # environmentd segfaults during normal shutdown in coverage builds, see database-issues#5980 # Ignoring this in regular ways would still be quite spammy. if ( b"environmentd" in match.group(0) diff --git a/misc/python/materialize/feature_benchmark/benchmark_versioning.py b/misc/python/materialize/feature_benchmark/benchmark_versioning.py index f7842d348e026..78d9da2d30b6a 100644 --- a/misc/python/materialize/feature_benchmark/benchmark_versioning.py +++ b/misc/python/materialize/feature_benchmark/benchmark_versioning.py @@ -23,11 +23,11 @@ # Consider increasing the scenario's class #version() if changes are expected to impact results! SHA256_BY_SCENARIO_FILE: dict[str, str] = { - "benchmark_main.py": "1d61feccbf7178b6b430bbcbec00f70e008339d6f64f9cd9ea6208c13c1f5851", + "benchmark_main.py": "02e5a841429af21d6fb44da33c929bb64bac850bf26d5ddf223df62cbc6db99a", "concurrency.py": "2e9c149c136b83b3853abc923a1adbdaf55a998ab4557712f8424c8b16f2adb1", "customer.py": "d1e72837a342c3ebf1f4a32ec583b1b78a78644cdba495030a6df45ebbffe703", "optbench.py": "314c7578fc84d8aaaeb838e2df90acd917b5f2c09fe07559ff1ace1af9964def", "scale.py": "c4c8749d166e4df34e0b0e92220434fdb508c5c2ac56eb80c07043be0048dded", "skew.py": "bf60802205fc51ebf94fb008bbdb6b2ccce3c9ed88a6188fa7f090f2c84b120f", - "subscribe.py": "510cf9037308936f1bb02a1c2e6345a90bcbda89f701653518d46d1fb56a2328", + "subscribe.py": "66b6ba61daed10a0e78291f6251e62dcb41f206228028bc0cbd0d738ad76252b", } diff --git a/misc/python/materialize/feature_benchmark/scenarios/benchmark_main.py b/misc/python/materialize/feature_benchmark/scenarios/benchmark_main.py index 0ad988dae8816..3dcebb24c37fa 100644 --- a/misc/python/materialize/feature_benchmark/scenarios/benchmark_main.py +++ b/misc/python/materialize/feature_benchmark/scenarios/benchmark_main.py @@ -385,7 +385,7 @@ def benchmark(self) -> MeasurementSource: class UpdateMultiNoIndex(DML): - """Measure the time it takes to perform multiple updates over the same records in a non-indexed table. GitHub Issue materialize#11071""" + """Measure the time it takes to perform multiple updates over the same records in a non-indexed table. GitHub Issue database-issues#3233""" def before(self) -> Action: # Due to exterme variability in the results, we have no option but to drop and re-create @@ -1610,7 +1610,7 @@ class MySqlInitialLoad(MySqlCdc): """Measure the time it takes to read 1M existing records from MySQL when creating a materialized source""" - FIXED_SCALE = True # TODO: Remove when materialize#25323 is fixed + FIXED_SCALE = True # TODO: Remove when database-issues#7556 is fixed def shared(self) -> Action: return TdAction( diff --git a/misc/python/materialize/feature_benchmark/scenarios/subscribe.py b/misc/python/materialize/feature_benchmark/scenarios/subscribe.py index 678cc08caae1a..1a0d1a6386046 100644 --- a/misc/python/materialize/feature_benchmark/scenarios/subscribe.py +++ b/misc/python/materialize/feature_benchmark/scenarios/subscribe.py @@ -17,7 +17,9 @@ class SubscribeParallel(Scenario): """Feature benchmarks related to SUBSCRIBE""" - SCALE = 2 # So 100 concurrent SUBSCRIBEs by default, limited by materialize#18261 + SCALE = ( + 2 # So 100 concurrent SUBSCRIBEs by default, limited by database-issues#5376 + ) FIXED_SCALE = True def benchmark(self) -> MeasurementSource: @@ -29,7 +31,7 @@ def benchmark(self) -> MeasurementSource: f""" $ postgres-connect name=conn{i} url=postgres://materialize:materialize@${{testdrive.materialize-sql-addr}} $ postgres-execute connection=conn{i} - # STRICT SERIALIZABLE is affected by materialize#18353 + # STRICT SERIALIZABLE is affected by database-issues#5407 START TRANSACTION ISOLATION LEVEL SERIALIZABLE; DECLARE c{i} CURSOR FOR SUBSCRIBE s1 """ diff --git a/misc/python/materialize/feature_flag_consistency/ignore_filter/feature_flag_consistency_ignore_filter.py b/misc/python/materialize/feature_flag_consistency/ignore_filter/feature_flag_consistency_ignore_filter.py index 07a5f72ab0787..669028b5c662d 100644 --- a/misc/python/materialize/feature_flag_consistency/ignore_filter/feature_flag_consistency_ignore_filter.py +++ b/misc/python/materialize/feature_flag_consistency/ignore_filter/feature_flag_consistency_ignore_filter.py @@ -58,7 +58,7 @@ def _shall_ignore_success_mismatch( if query_template.uses_join() and ( query_template.has_where_condition() or query_template.has_row_selection() ): - return YesIgnore("materialize#17189: evaluation order") + return YesIgnore("database-issues#4972: evaluation order") return super()._shall_ignore_success_mismatch( error, query_template, contains_aggregation @@ -73,7 +73,7 @@ def _shall_ignore_error_mismatch( if query_template.uses_join() and ( query_template.has_where_condition() or query_template.has_row_selection() ): - return YesIgnore("materialize#17189: evaluation order") + return YesIgnore("database-issues#4972: evaluation order") return super()._shall_ignore_error_mismatch( error, query_template, contains_aggregation diff --git a/misc/python/materialize/mzcompose/composition.py b/misc/python/materialize/mzcompose/composition.py index 08679591ec8d2..ceab31e8fb974 100644 --- a/misc/python/materialize/mzcompose/composition.py +++ b/misc/python/materialize/mzcompose/composition.py @@ -922,7 +922,7 @@ def validate_sources_sinks_clusters(self) -> str | None: ) exclusion_clause = f"name NOT IN ({excluded_items})" - # starting sources are currently expected if no new data is produced, see materialize#21980 + # starting sources are currently expected if no new data is produced, see database-issues#6605 results = self.sql_query( f""" SELECT name, status, error, details diff --git a/misc/python/materialize/mzcompose/services/cockroach.py b/misc/python/materialize/mzcompose/services/cockroach.py index 6d1cb8ebb30c8..9c16d0669fd42 100644 --- a/misc/python/materialize/mzcompose/services/cockroach.py +++ b/misc/python/materialize/mzcompose/services/cockroach.py @@ -32,7 +32,7 @@ def __init__( setup_materialize: bool = True, in_memory: bool = False, healthcheck: ServiceHealthcheck | None = None, - # Workaround for materialize#19809, should be "no" otherwise + # Workaround for database-issues#5898, should be "no" otherwise restart: str = "on-failure:5", ): volumes = [] diff --git a/misc/python/materialize/mzcompose/services/materialized.py b/misc/python/materialize/mzcompose/services/materialized.py index 5080132e8482d..5af077e2aa3e7 100644 --- a/misc/python/materialize/mzcompose/services/materialized.py +++ b/misc/python/materialize/mzcompose/services/materialized.py @@ -164,7 +164,7 @@ def __init__( ) ) command += [ - # Issue materialize#15858 prevents the habitual use of large introspection + # Issue database-issues#4562 prevents the habitual use of large introspection # clusters, so we leave the builtin cluster replica size as is. # f"--bootstrap-builtin-cluster-replica-size={self.default_replica_size}", f"--bootstrap-default-cluster-replica-size={self.default_replica_size}", diff --git a/misc/python/materialize/output_consistency/generators/query_generator.py b/misc/python/materialize/output_consistency/generators/query_generator.py index 8b000116998fc..190d62a7917b4 100644 --- a/misc/python/materialize/output_consistency/generators/query_generator.py +++ b/misc/python/materialize/output_consistency/generators/query_generator.py @@ -232,7 +232,7 @@ def _create_multi_column_queries( if self.randomized_picker.random_boolean( probability.NO_SOURCE_MINIMIZATION ): - # do not minimize sources to catch errors like materialize#29110 + # do not minimize sources to catch errors like database-issues#8463 pass else: # remove sources that are not used by any (remaining) expression diff --git a/misc/python/materialize/output_consistency/ignore_filter/internal_output_inconsistency_ignore_filter.py b/misc/python/materialize/output_consistency/ignore_filter/internal_output_inconsistency_ignore_filter.py index 6a55d1526591c..37254640f9203 100644 --- a/misc/python/materialize/output_consistency/ignore_filter/internal_output_inconsistency_ignore_filter.py +++ b/misc/python/materialize/output_consistency/ignore_filter/internal_output_inconsistency_ignore_filter.py @@ -83,7 +83,7 @@ def _matches_problematic_operation_or_function_invocation( isinstance(arg_type_spec, NumericReturnTypeSpec) and not arg_type_spec.only_integer ): - return YesIgnore("materialize#15186") + return YesIgnore("database-issues#4341") return NoIgnore() @@ -103,13 +103,13 @@ def _matches_problematic_function_invocation( "var_pop", }: if ExpressionCharacteristics.MAX_VALUE in all_involved_characteristics: - return YesIgnore("materialize#15186") + return YesIgnore("database-issues#4341") if ( ExpressionCharacteristics.DECIMAL in all_involved_characteristics and ExpressionCharacteristics.TINY_VALUE in all_involved_characteristics ): - return YesIgnore("materialize#15186") + return YesIgnore("database-issues#4341") return NoIgnore() @@ -146,7 +146,7 @@ def _shall_ignore_success_mismatch( if dfr_fails_but_ctf_succeeds and self._uses_shortcut_optimization( query_template.select_expressions, contains_aggregation ): - return YesIgnore("materialize#19662") + return YesIgnore("database-issues#5850") if ( dfr_fails_but_ctf_succeeds @@ -155,13 +155,13 @@ def _shall_ignore_success_mismatch( [query_template.where_expression], contains_aggregation ) ): - return YesIgnore("materialize#17189") + return YesIgnore("database-issues#4972") if dfr_succeeds_but_ctf_fails or dfr_fails_but_ctf_succeeds: if query_template.has_where_condition(): # An evaluation strategy may touch further rows than the selected subset and thereby run into evaluation # errors (while the other uses another order). - return YesIgnore("materialize#17189") + return YesIgnore("database-issues#4972") if ( query_template.has_where_condition() @@ -170,10 +170,10 @@ def _shall_ignore_success_mismatch( # Where expression, or row filter, or join constraint are set. They might be evaluated in a different # order. Furthermore, constant folding may detect that the join constraint cannot be satisfied without # evaluating it (which will fail). - return YesIgnore("materialize#17189: evaluation order") + return YesIgnore("database-issues#4972: evaluation order") if self._uses_eager_evaluation(query_template): - return YesIgnore("materialize#17189") + return YesIgnore("database-issues#4972") if dfr_succeeds_but_ctf_fails: assert isinstance(ctf_outcome, QueryFailure) @@ -188,7 +188,9 @@ def _shall_ignore_success_mismatch( True, ) ): - return YesIgnore("materialize#28136: jsonb_object_agg with NULL as key") + return YesIgnore( + "database-issues#8246: jsonb_object_agg with NULL as key" + ) return NoIgnore() @@ -215,20 +217,20 @@ def _shall_ignore_error_mismatch( if self._uses_shortcut_optimization( query_template.select_expressions, contains_aggregation ): - return YesIgnore("materialize#17189: evaluation order") + return YesIgnore("database-issues#4972: evaluation order") if self._uses_eager_evaluation(query_template): - return YesIgnore("materialize#17189: evaluation order") + return YesIgnore("database-issues#4972: evaluation order") if query_template.has_where_condition() or query_template.uses_join(): # The error message may depend on the evaluation order of the where expression or join constraint. - return YesIgnore("materialize#17189: evaluation order") + return YesIgnore("database-issues#4972: evaluation order") if ( ExpressionCharacteristics.INFINITY in all_characteristics and ExpressionCharacteristics.MAX_VALUE in all_characteristics ): - return YesIgnore("materialize#17189: evaluation order") + return YesIgnore("database-issues#4972: evaluation order") if query_template.matches_any_expression( is_table_function, diff --git a/misc/python/materialize/output_consistency/input_data/types/date_time_types_provider.py b/misc/python/materialize/output_consistency/input_data/types/date_time_types_provider.py index 1d553d1375ab5..50e4fc73046bc 100644 --- a/misc/python/materialize/output_consistency/input_data/types/date_time_types_provider.py +++ b/misc/python/materialize/output_consistency/input_data/types/date_time_types_provider.py @@ -62,7 +62,7 @@ def resolve_return_type_spec( DATE_TYPE = DateTimeDataType( DATE_TYPE_IDENTIFIER, "DATE", - # BC, AD not working, see: https://github.com/MaterializeInc/materialize/issues/19637 + # BC, AD not working, see: https://api.github.com/repos/MaterializeInc/database-issues/issues/5843 "0001-01-01", "99999-12-31", [ @@ -85,7 +85,7 @@ def resolve_return_type_spec( TIMESTAMP_TYPE = DateTimeDataType( TIMESTAMP_TYPE_IDENTIFIER, "TIMESTAMP", - # BC, AD not working, see: https://github.com/MaterializeInc/materialize/issues/19637 + # BC, AD not working, see: https://api.github.com/repos/MaterializeInc/database-issues/issues/5843 "0001-01-01 00:00:00", "99999-12-31 23:59:59", [ @@ -104,7 +104,7 @@ def resolve_return_type_spec( TIMESTAMPTZ_TYPE = DateTimeDataType( TIMESTAMPTZ_TYPE_IDENTIFIER, "TIMESTAMPTZ", - # BC, AD not working, see: https://github.com/MaterializeInc/materialize/issues/19637 + # BC, AD not working, see: https://api.github.com/repos/MaterializeInc/database-issues/issues/5843 "0001-01-01 00:00:00", "99999-12-31 23:59:59", further_values=[("2023-06-01 11:22:33.44444", set())], diff --git a/misc/python/materialize/output_consistency/validation/error_message_normalizer.py b/misc/python/materialize/output_consistency/validation/error_message_normalizer.py index 6249930c7d107..e964b9ea14009 100644 --- a/misc/python/materialize/output_consistency/validation/error_message_normalizer.py +++ b/misc/python/materialize/output_consistency/validation/error_message_normalizer.py @@ -28,7 +28,7 @@ def normalize(self, error_message: str) -> str: normalized_message = normalized_message.replace("Evaluation error: ", "") # This will replace ln, log, and log10 mentions with log - # see https://github.com/MaterializeInc/materialize/issues/19815 + # see https://api.github.com/repos/MaterializeInc/database-issues/issues/5902 normalized_message = re.sub( "(?<=function )(ln|log|log10)(?= is not defined for zero)", "log", @@ -45,10 +45,10 @@ def normalize(self, error_message: str) -> str: or (re.search(r"operator does not exist: .*?record", normalized_message)) or "CAST does not support casting from record" in normalized_message ): - # tracked with https://github.com/MaterializeInc/materialize/issues/28129 + # tracked with https://api.github.com/repos/MaterializeInc/database-issues/issues/8243 normalized_message = normalized_message.replace("?", "") - # strip error message details (see materialize#29661, materialize#19822, materialize#23497) + # strip error message details (see materialize#29661, materialize#19822, database-issues#7061) normalized_message = re.sub(r" \(.*?\.\)", "", normalized_message) return normalized_message diff --git a/misc/python/materialize/parallel_benchmark/scenarios.py b/misc/python/materialize/parallel_benchmark/scenarios.py index 68167e3a35ceb..3f12a819b217b 100644 --- a/misc/python/materialize/parallel_benchmark/scenarios.py +++ b/misc/python/materialize/parallel_benchmark/scenarios.py @@ -338,7 +338,7 @@ def __init__(self, c: Composition, conn_infos: dict[str, PgConnInfo]): class FlagUpdate(Scenario): - """Reproduces materialize#29235""" + """Reproduces database-issues#8480""" def __init__(self, c: Composition, conn_infos: dict[str, PgConnInfo]): self.init( @@ -367,7 +367,7 @@ def __init__(self, c: Composition, conn_infos: dict[str, PgConnInfo]): ), ], guarantees={ - # TODO(def-): Lower when materialize#29235 is fixed to prevent regressions + # TODO(def-): Lower when database-issues#8480 is fixed to prevent regressions "SELECT 1 (reuse connection)": {"avg": 5, "max": 500, "slope": 0.1}, }, ) @@ -485,7 +485,7 @@ def __init__(self, c: Composition, conn_infos: dict[str, PgConnInfo]): ), ], conn_pool_size=100, - # TODO(def-): Bump per_second and add guarantees when materialize#29371 is fixed + # TODO(def-): Bump per_second and add guarantees when database-issues#8510 is fixed ) diff --git a/misc/python/materialize/parallel_workload/action.py b/misc/python/materialize/parallel_workload/action.py index 51ed2ee585963..b2b4203c8cef5 100644 --- a/misc/python/materialize/parallel_workload/action.py +++ b/misc/python/materialize/parallel_workload/action.py @@ -139,12 +139,12 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: "query could not complete", "cached plan must not change result type", "violates not-null constraint", - "unknown catalog item", # Expected, see materialize#20381 + "unknown catalog item", # Expected, see database-issues#6124 "was concurrently dropped", # role was dropped "unknown cluster", # cluster was dropped "unknown schema", # schema was dropped "the transaction's active cluster has been dropped", # cluster was dropped - "was removed", # dependency was removed, started with moving optimization off main thread, see materialize#24367 + "was removed", # dependency was removed, started with moving optimization off main thread, see database-issues#7285 "real-time source dropped before ingesting the upstream system's visible frontier", # Expected, see https://buildkite.com/materialize/nightly/builds/9399#0191be17-1f4c-4321-9b51-edc4b08b71c5 "object state changed while transaction was in progress", ] @@ -186,7 +186,7 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: ] ) if exe.db.scenario in (Scenario.Kill, Scenario.ZeroDowntimeDeploy): - # Expected, see materialize#20465 + # Expected, see database-issues#6156 result.extend(["unknown catalog item", "unknown schema"]) if exe.db.scenario == Scenario.Rename: result.extend(["unknown schema", "ambiguous reference to schema name"]) @@ -200,7 +200,7 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: result = super().errors_to_ignore(exe) result.extend( [ - "is not of expected type", # TODO(def-) Remove when materialize#26549 is fixed + "is not of expected type", # TODO(def-) Remove when database-issues#7857 is fixed ] ) if exe.db.complexity == Complexity.DDL: @@ -215,7 +215,7 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: def run(self, exe: Executor) -> bool: obj = self.rng.choice(exe.db.db_objects()) # Unsupported via this API - # See https://github.com/MaterializeInc/materialize/issues/20474 + # See https://api.github.com/repos/MaterializeInc/database-issues/issues/6159 ( exe.rollback(http=Http.NO) if self.rng.choice([True, False]) @@ -767,7 +767,7 @@ def run(self, exe: Executor) -> bool: class AlterKafkaSinkFromAction(Action): def run(self, exe: Executor) -> bool: if exe.db.scenario in (Scenario.Kill, Scenario.ZeroDowntimeDeploy): - # Does not work reliably with kills, see materialize#28870 + # Does not work reliably with kills, see database-issues#8421 return False with exe.db.lock: if not exe.db.kafka_sinks: @@ -791,7 +791,7 @@ def run(self, exe: Executor) -> bool: ] else: # multi column formats require at least as many columns as before - # columns also have to be of the same type, see materialize#28726 + # columns also have to be of the same type, see database-issues#8385 objs = [ o for o in exe.db.db_objects_without_views() @@ -1040,7 +1040,7 @@ def __init__( def run(self, exe: Executor) -> bool: flag_name = self.rng.choice(list(self.flags_with_values.keys())) - # TODO: Remove when materialize#28576 is fixed + # TODO: Remove when database-issues#8352 is fixed if exe.db.scenario == Scenario.ZeroDowntimeDeploy and flag_name.startswith( "persist_use_critical_since_" ): @@ -1181,7 +1181,7 @@ def run(self, exe: Executor) -> bool: try: exe.execute(query, http=Http.RANDOM) except QueryError as e: - # expected, see materialize#20465 + # expected, see database-issues#6156 if ( exe.db.scenario not in (Scenario.Kill, Scenario.ZeroDowntimeDeploy) or "unknown role" not in e.msg @@ -1233,7 +1233,7 @@ def run(self, exe: Executor) -> bool: try: exe.execute(query, http=Http.RANDOM) except QueryError as e: - # expected, see materialize#20465 + # expected, see database-issues#6156 if ( exe.db.scenario not in (Scenario.Kill, Scenario.ZeroDowntimeDeploy) or "unknown cluster" not in e.msg @@ -1368,7 +1368,7 @@ def run(self, exe: Executor) -> bool: try: exe.execute(query, http=Http.RANDOM) except QueryError as e: - # expected, see materialize#20465 + # expected, see database-issues#6156 if ( exe.db.scenario not in (Scenario.Kill, Scenario.ZeroDowntimeDeploy) or "has no CLUSTER REPLICA named" not in e.msg @@ -1397,7 +1397,7 @@ def run(self, exe: Executor) -> bool: try: exe.execute(query, http=Http.RANDOM) except QueryError as e: - # expected, see materialize#20465 + # expected, see database-issues#6156 if ( exe.db.scenario not in (Scenario.Kill, Scenario.ZeroDowntimeDeploy) or "unknown role" not in e.msg @@ -1426,7 +1426,7 @@ def run(self, exe: Executor) -> bool: try: exe.execute(query, http=Http.RANDOM) except QueryError as e: - # expected, see materialize#20465 + # expected, see database-issues#6156 if ( exe.db.scenario not in (Scenario.Kill, Scenario.ZeroDowntimeDeploy) or "unknown role" not in e.msg @@ -1578,7 +1578,7 @@ def run(self, exe: Executor) -> bool: extra_info=f"Canceling {worker}", http=Http.RANDOM, ) - # Sleep less often to work around materialize#22228 / materialize#2392 + # Sleep less often to work around materialize#22228 / database-issues#835 time.sleep(self.rng.uniform(1, 10)) return True @@ -1860,7 +1860,7 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: return result def run(self, exe: Executor) -> bool: - # TODO: Reenable when materialize#22770 is fixed + # TODO: Reenable when database-issues#6881 is fixed if exe.db.scenario == Scenario.BackupRestore: return False @@ -1932,7 +1932,7 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: return result def run(self, exe: Executor) -> bool: - # TODO: Reenable when materialize#22770 is fixed + # TODO: Reenable when database-issues#6881 is fixed if exe.db.scenario == Scenario.BackupRestore: return False @@ -2112,7 +2112,7 @@ def run(self, exe: Executor) -> bool: ): raise except QueryError as e: - # expected, see materialize#20465 + # expected, see database-issues#6156 if exe.db.scenario not in ( Scenario.Kill, Scenario.ZeroDowntimeDeploy, @@ -2226,7 +2226,7 @@ def __init__( (DropKafkaSinkAction, 4), (CreateKafkaSourceAction, 4), (DropKafkaSourceAction, 4), - # TODO: Reenable when materialize#28108 is fixed + # TODO: Reenable when database-issues#8237 is fixed # (CreateMySqlSourceAction, 4), # (DropMySqlSourceAction, 4), (CreatePostgresSourceAction, 4), @@ -2244,7 +2244,7 @@ def __init__( (RenameSinkAction, 10), (SwapSchemaAction, 10), (FlipFlagsAction, 2), - # TODO: Reenable when materialize#28962 is fixed + # TODO: Reenable when database-issues#8445 is fixed # (AlterKafkaSinkFromAction, 8), # (TransactionIsolationAction, 1), ], diff --git a/misc/python/materialize/postgres_consistency/custom/predefined_pg_queries.py b/misc/python/materialize/postgres_consistency/custom/predefined_pg_queries.py index 1bacc30b361e0..901e91c23f24a 100644 --- a/misc/python/materialize/postgres_consistency/custom/predefined_pg_queries.py +++ b/misc/python/materialize/postgres_consistency/custom/predefined_pg_queries.py @@ -95,7 +95,7 @@ def create_pg_timezone_names_query() -> QueryTemplate: ], ) - # TODO materialize#26521: time zones differ + # TODO database-issues#7851: time zones differ excluded_timezones = [ # abbrev, utc_offset, is_dst differ (as of 2024-04-08) "America/Godthab", diff --git a/misc/python/materialize/postgres_consistency/execution/pg_dialect_adjuster.py b/misc/python/materialize/postgres_consistency/execution/pg_dialect_adjuster.py index 89dd9905edf34..7287fede743fd 100644 --- a/misc/python/materialize/postgres_consistency/execution/pg_dialect_adjuster.py +++ b/misc/python/materialize/postgres_consistency/execution/pg_dialect_adjuster.py @@ -42,7 +42,7 @@ def adjust_value( if type_name.startswith("INT") and string_value.startswith("-"): # wrap negative numbers in parentheses - # see: https://github.com/MaterializeInc/materialize/issues/21993 + # see: https://api.github.com/repos/MaterializeInc/database-issues/issues/6611 string_value = f"({string_value})" return string_value diff --git a/misc/python/materialize/postgres_consistency/ignore_filter/pg_inconsistency_ignore_filter.py b/misc/python/materialize/postgres_consistency/ignore_filter/pg_inconsistency_ignore_filter.py index ab8a16b35ad52..6edd9c8589149 100644 --- a/misc/python/materialize/postgres_consistency/ignore_filter/pg_inconsistency_ignore_filter.py +++ b/misc/python/materialize/postgres_consistency/ignore_filter/pg_inconsistency_ignore_filter.py @@ -157,7 +157,7 @@ def _matches_problematic_operation_or_function_invocation( all_involved_characteristics: set[ExpressionCharacteristics], ) -> IgnoreVerdict: if matches_float_comparison(expression): - return YesIgnore("materialize#22022: real with decimal comparison") + return YesIgnore("database-issues#6630: real with decimal comparison") if ( operation.is_tagged(TAG_JSONB_TO_TEXT) @@ -169,13 +169,13 @@ def _matches_problematic_operation_or_function_invocation( ), True, ): - return YesIgnore("Consequence of materialize#23571") + return YesIgnore("Consequence of database-issues#7085") if operation.is_tagged(TAG_CASTING) and expression.matches( partial(matches_fun_by_name, function_name_in_lower_case="to_char"), True, ): - return YesIgnore("materialize#25228: date format that cannot be parsed") + return YesIgnore("database-issues#7529: date format that cannot be parsed") if ( expression.matches( @@ -191,7 +191,7 @@ def _matches_problematic_operation_or_function_invocation( and ExpressionCharacteristics.STRING_WITH_SPECIAL_SPACE_CHARS in all_involved_characteristics ): - return YesIgnore("materialize#27253: bpchar and char trim newline") + return YesIgnore("database-issues#8061: bpchar and char trim newline") if operation.is_tagged(TAG_JSONB_AGGREGATION) and expression.matches( partial( @@ -204,7 +204,7 @@ def _matches_problematic_operation_or_function_invocation( True, ): return YesIgnore( - "materialize#28193: bpchar in jsonb aggregation without spaces" + "database-issues#8267: bpchar in jsonb aggregation without spaces" ) if expression.matches( @@ -231,42 +231,42 @@ def _matches_problematic_function_invocation( if db_function.function_name_in_lower_case == "date_trunc": precision = expression.args[0] if isinstance(precision, EnumConstant) and precision.value == "second": - return YesIgnore("materialize#22017: date_trunc with seconds") + return YesIgnore("database-issues#6628: date_trunc with seconds") if isinstance(precision, EnumConstant) and precision.value == "quarter": return YesIgnore( - "materialize#21996: date_trunc does not support quarter" + "database-issues#6614: date_trunc does not support quarter" ) if db_function.function_name_in_lower_case == "lpad" and expression.args[ 1 ].has_any_characteristic({ExpressionCharacteristics.NEGATIVE}): - return YesIgnore("materialize#21997: lpad with negative") + return YesIgnore("database-issues#6615: lpad with negative") if db_function.function_name_in_lower_case in {"min", "max"}: return_type_category = expression.args[0].resolve_return_type_category() if return_type_category == DataTypeCategory.STRING: return YesIgnore( - "materialize#22002: ordering on text different (min/max)" + "database-issues#6620: ordering on text different (min/max)" ) if return_type_category == DataTypeCategory.JSONB: return YesIgnore( - "materialize#26309: ordering on JSON different (min/max)" + "database-issues#7812: ordering on JSON different (min/max)" ) if return_type_category == DataTypeCategory.ARRAY: return YesIgnore( - "materialize#27457: ordering on array different (min/max)" + "database-issues#8108: ordering on array different (min/max)" ) if db_function.function_name_in_lower_case == "replace": # replace is not working properly with empty text; however, it is not possible to reliably determine if an # expression is an empty text, we therefore need to exclude the function completely - return YesIgnore("materialize#22001: replace") + return YesIgnore("database-issues#6619: replace") if db_function.function_name_in_lower_case == "regexp_replace": if expression.args[2].has_any_characteristic( {ExpressionCharacteristics.STRING_WITH_BACKSLASH_CHAR} ): - return YesIgnore("materialize#23605: regexp with backslash") + return YesIgnore("database-issues#7095: regexp with backslash") if expression.count_args() == 4: regex_flag = expression.args[3] assert isinstance(regex_flag, EnumConstant) @@ -290,7 +290,9 @@ def _matches_problematic_function_invocation( ].has_any_characteristic( {ExpressionCharacteristics.STRING_WITH_SPECIAL_SPACE_CHARS} ): - return YesIgnore("materialize#25937 (base64 decode with new line and tab)") + return YesIgnore( + "database-issues#7733 (base64 decode with new line and tab)" + ) if ( db_function.function_name_in_lower_case == "coalesce" @@ -306,7 +308,7 @@ def _matches_problematic_function_invocation( ) ): # do not explicitly require the TEXT type to be included because it can appear by applying || to two char values - return YesIgnore("materialize#27278: bpchar and char with coalesce") + return YesIgnore("database-issues#8067: bpchar and char with coalesce") if db_function.function_name_in_lower_case == "row": if expression.matches( @@ -320,7 +322,7 @@ def _matches_problematic_function_invocation( True, ): return YesIgnore( - "materialize#28130 / materialize#28131: record type with array or ranges" + "materialize#28130 / database-issues#8245: record type with array or ranges" ) if expression.matches( @@ -330,7 +332,7 @@ def _matches_problematic_function_invocation( ), True, ): - return YesIgnore("materialize#28392: record type with bytea") + return YesIgnore("database-issues#8314: record type with bytea") if expression.matches( partial( @@ -340,7 +342,7 @@ def _matches_problematic_function_invocation( True, ): return YesIgnore( - "Consequence of materialize#25723: decimal 0s are not shown" + "Consequence of database-issues#7675: decimal 0s are not shown" ) if expression.matches( @@ -361,7 +363,7 @@ def _matches_problematic_function_invocation( True, ): return YesIgnore( - "Consequence of materialize#25723: decimal 0s are not shown" + "Consequence of database-issues#7675: decimal 0s are not shown" ) return NoIgnore() @@ -381,7 +383,7 @@ def _matches_problematic_operation_invocation( ) and expression.args[0].has_any_characteristic( {ExpressionCharacteristics.NULL} ): - return YesIgnore("materialize#22014: timestamp precision with null") + return YesIgnore("database-issues#6625: timestamp precision with null") if ( db_operation.pattern in {"$ + $", "$ - $"} @@ -390,13 +392,13 @@ def _matches_problematic_operation_invocation( and db_operation.params[1].get_declared_type_category() == DataTypeCategory.DATE_TIME ): - return YesIgnore("materialize#24578: different representation") + return YesIgnore("database-issues#7325: different representation") if db_operation.pattern == "$ % $" and ( ExpressionCharacteristics.MAX_VALUE in all_involved_characteristics or ExpressionCharacteristics.TINY_VALUE in all_involved_characteristics ): - return YesIgnore("materialize#27634: modulo with large / tiny values") + return YesIgnore("database-issues#8134: modulo with large / tiny values") if db_operation.is_tagged(TAG_REGEX): return YesIgnore( @@ -415,7 +417,7 @@ def _matches_problematic_operation_invocation( == DataTypeCategory.NUMERIC ): return YesIgnore( - "materialize#26323: imprecise comparison between REAL and DECIMAL" + "database-issues#7815: imprecise comparison between REAL and DECIMAL" ) if ( @@ -423,7 +425,9 @@ def _matches_problematic_operation_invocation( and db_operation.params[0].get_declared_type_category() == DataTypeCategory.JSONB ): - return YesIgnore("materialize#23578: empty result in || with simple values") + return YesIgnore( + "database-issues#7087: empty result in || with simple values" + ) if db_operation.pattern in [ "position($ IN $)", @@ -434,7 +438,7 @@ def _matches_problematic_operation_invocation( ), True, ): - return YesIgnore("Consequence of materialize#23571") + return YesIgnore("Consequence of database-issues#7085") if db_operation.is_tagged(TAG_CASTING): casting_target = expression.args[1] @@ -453,21 +457,21 @@ def _matches_problematic_operation_invocation( return_type_category_2, }: return YesIgnore( - "materialize#22002: ordering on text different (<, <=, ...)" + "database-issues#6620: ordering on text different (<, <=, ...)" ) if DataTypeCategory.JSONB in { return_type_category_1, return_type_category_2, }: return YesIgnore( - "materialize#26309: ordering on JSON different (<, <=, ...)" + "database-issues#7812: ordering on JSON different (<, <=, ...)" ) if DataTypeCategory.ARRAY in { return_type_category_1, return_type_category_2, }: return YesIgnore( - "materialize#27457: ordering on array different (<, <=, ...)" + "database-issues#8108: ordering on array different (<, <=, ...)" ) if db_operation.is_tagged(TAG_CASTING) and expression.matches( @@ -480,7 +484,7 @@ def _matches_problematic_operation_invocation( ), True, ): - return YesIgnore("materialize#27282: casting bpchar or char") + return YesIgnore("database-issues#8068: casting bpchar or char") return NoIgnore() @@ -532,7 +536,7 @@ def _shall_ignore_success_mismatch( True, ): return YesIgnore( - "Deviations in floating point functions may cause an invalid JSONB (e.g., materialize#29350 producing NaN in mz)" + "Deviations in floating point functions may cause an invalid JSONB (e.g., database-issues#8505 producing NaN in mz)" ) return super()._shall_ignore_success_mismatch( @@ -563,10 +567,10 @@ def _shall_ignore_pg_failure_where_mz_succeeds( return YesIgnore("accepted") if " out of range" in pg_error_msg: - return YesIgnore("materialize#22265") + return YesIgnore("database-issues#6717") if "value overflows numeric format" in pg_error_msg: - return YesIgnore("materialize#21994") + return YesIgnore("database-issues#6612") if _error_message_is_about_zero_or_value_ranges(pg_error_msg): return YesIgnore("Caused by a different precision") @@ -591,7 +595,7 @@ def _shall_ignore_pg_failure_where_mz_succeeds( if 'invalid input syntax for type time: ""' in pg_error_msg: return YesIgnore( - "materialize#24736: different handling of empty time string" + "database-issues#7367: different handling of empty time string" ) if ( @@ -635,7 +639,7 @@ def _shall_ignore_pg_failure_where_mz_succeeds( True, ) ): - return YesIgnore("materialize#28136: jsonb_object_agg with NULL as key") + return YesIgnore("database-issues#8246: jsonb_object_agg with NULL as key") if ( re.search("function pg_size_pretty(.*?) is not unique", pg_error_msg) @@ -653,7 +657,9 @@ def _shall_ignore_pg_failure_where_mz_succeeds( True, ) ): - return YesIgnore("materialize#28141: jsonb_object_agg with non-scalar key") + return YesIgnore( + "database-issues#8249: jsonb_object_agg with non-scalar key" + ) if query_template.matches_any_expression( partial( @@ -718,7 +724,7 @@ def matches_round_function(expression: Expression) -> bool: "value out of range: overflow" in mz_error_msg and query_template.matches_any_expression(matches_round_function, True) ): - return YesIgnore("materialize#22028: round overflow") + return YesIgnore("database-issues#6634: round overflow") if ( "value out of range: overflow" in mz_error_msg @@ -727,50 +733,50 @@ def matches_round_function(expression: Expression) -> bool: or 'inf" real out of range' in mz_error_msg or 'inf" double precision out of range' in mz_error_msg ): - return YesIgnore("materialize#21994: overflow") + return YesIgnore("database-issues#6612: overflow") if ( "value out of range: underflow" in mz_error_msg or '"-inf" real out of range' in mz_error_msg ): - return YesIgnore("materialize#21995: underflow") + return YesIgnore("database-issues#6613: underflow") if ( "precision for type timestamp or timestamptz must be between 0 and 6" in mz_error_msg ): - return YesIgnore("materialize#22020: unsupported timestamp precision") + return YesIgnore("database-issues#6629: unsupported timestamp precision") if "array_agg on arrays not yet supported" in mz_error_msg: - return YesIgnore("materialize#28384: array_agg on arrays") + return YesIgnore("database-issues#8310: array_agg on arrays") if "field position must be greater than zero" in mz_error_msg: - return YesIgnore("materialize#22023: split_part") + return YesIgnore("database-issues#6631: split_part") if "timestamp out of range" in mz_error_msg: - return YesIgnore("materialize#22264") + return YesIgnore("database-issues#6716") if "bigint out of range" in mz_error_msg: # when a large decimal number or NaN is used as an array index - return YesIgnore("materialize#28145") + return YesIgnore("database-issues#8252") if "invalid regular expression: regex parse error" in mz_error_msg: - return YesIgnore("materialize#22956") + return YesIgnore("database-issues#6921") if "invalid regular expression flag" in mz_error_msg: - return YesIgnore("materialize#22958") + return YesIgnore("database-issues#6923") if "unit 'invalid_value_123' not recognized" in mz_error_msg: - return YesIgnore("materialize#22957") + return YesIgnore("database-issues#6922") if "invalid time zone" in mz_error_msg: - return YesIgnore("materialize#22984") + return YesIgnore("database-issues#6927") if _error_message_is_about_zero_or_value_ranges(mz_error_msg): return YesIgnore("Caused by a different precision") if query_template.limit == 0: - return YesIgnore("materialize#17189: LIMIT 0 does not swallow errors") + return YesIgnore("database-issues#4972: LIMIT 0 does not swallow errors") if ( query_template.matches_any_expression( @@ -783,18 +789,18 @@ def matches_round_function(expression: Expression) -> bool: return YesIgnore("regtype of postgres can be cast") if "array_agg on character not yet supported" in mz_error_msg: - return YesIgnore("materialize#27252: array_agg on character") + return YesIgnore("database-issues#8060: array_agg on character") if "array subscript does not support slices" in mz_error_msg: return YesIgnore("array subscript does not support slices") if "|| does not support implicitly casting" in mz_error_msg: return YesIgnore( - "materialize#28024: no implicit casting from ...[] to ...[]" + "database-issues#8219: no implicit casting from ...[] to ...[]" ) if "cannot reference pseudo type pg_catalog.record" in mz_error_msg: - return YesIgnore("materialize#17870: cannot reference pg_catalog.record") + return YesIgnore("database-issues#5211: cannot reference pg_catalog.record") if query_template.matches_any_expression( partial(is_operation_tagged, tag=TAG_ARRAY_INDEX_OPERATION), @@ -812,16 +818,16 @@ def matches_round_function(expression: Expression) -> bool: True, ) ): - return YesIgnore("materialize#28169: JSONB with large number") + return YesIgnore("database-issues#8258: JSONB with large number") if ( "function casting double precision to numeric is only defined for finite arguments" in mz_error_msg ): - return YesIgnore("materialize#28240: infinity to decimal") + return YesIgnore("database-issues#8281: infinity to decimal") if "invalid regular expression flag: n" in mz_error_msg: - return YesIgnore("materialize#28805: regex n flag") + return YesIgnore("database-issues#9725: regex n flag") if "aggregate functions are not allowed in table function" in mz_error_msg: return YesIgnore( @@ -851,7 +857,7 @@ def matches_round_function(expression: Expression) -> bool: if query_template.uses_join(): # more generic catch return YesIgnore( - "materialize#29347: eager evaluation of select expression in mz" + "database-issues#8504: eager evaluation of select expression in mz" ) if ( @@ -915,7 +921,7 @@ def matches_nullif(expression: Expression) -> bool: ), True, ): - return YesIgnore("materialize#22003: aggregation function") + return YesIgnore("database-issues#6621: aggregation function") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -942,7 +948,7 @@ def matches_nullif(expression: Expression) -> bool: partial(matches_fun_by_name, function_name_in_lower_case="mod"), True, ): - return YesIgnore("materialize#22005: mod") + return YesIgnore("database-issues#6623: mod") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -953,17 +959,17 @@ def matches_nullif(expression: Expression) -> bool: ), True, ): - return YesIgnore("materialize#23586") + return YesIgnore("database-issues#7089") if query_template.matches_specific_select_or_filter_expression( col_index, matches_math_op_with_large_or_tiny_val, True ): - return YesIgnore("materialize#22266: arithmetic funs with large value") + return YesIgnore("database-issues#6718: arithmetic funs with large value") if query_template.matches_specific_select_or_filter_expression( col_index, matches_nullif, True ): - return YesIgnore("materialize#22267: nullif") + return YesIgnore("database-issues#6719: nullif") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -1004,7 +1010,7 @@ def matches_nullif(expression: Expression) -> bool: and ExpressionCharacteristics.STRING_WITH_ESZETT in all_involved_characteristics ): - return YesIgnore("materialize#26846: eszett in upper") + return YesIgnore("database-issues#7938: eszett in upper") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -1034,7 +1040,7 @@ def matches_nullif(expression: Expression) -> bool: ), True, ): - return YesIgnore("materialize#28137: different date string in JSONB") + return YesIgnore("database-issues#8247: different date string in JSONB") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -1044,13 +1050,13 @@ def matches_nullif(expression: Expression) -> bool: ), True, ): - return YesIgnore("materialize#28143: non-quoted numbers") + return YesIgnore("database-issues#8251: non-quoted numbers") if ( ExpressionCharacteristics.DATE_WITH_SHORT_YEAR in all_involved_characteristics ): - return YesIgnore("materialize#28284: short date format") + return YesIgnore("database-issues#8289: short date format") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -1064,7 +1070,9 @@ def matches_nullif(expression: Expression) -> bool: or ExpressionCharacteristics.COLLECTION_EMPTY in all_involved_characteristics ): - return YesIgnore("materialize#28300: ALL and ANY with NULL or empty array") + return YesIgnore( + "database-issues#8293: ALL and ANY with NULL or empty array" + ) if query_template.matches_specific_select_or_filter_expression( col_index, @@ -1089,7 +1097,7 @@ def matches_nullif(expression: Expression) -> bool: ), True, ): - return YesIgnore("materialize#29143: unnest on array uses wrong order") + return YesIgnore("database-issues#8466: unnest on array uses wrong order") return NoIgnore() @@ -1113,7 +1121,7 @@ def _shall_ignore_pg_typeof_content_mismatch( ), True, ): - return YesIgnore("materialize#28801: return type of floor and ceil") + return YesIgnore("database-issues#9721: return type of floor and ceil") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -1121,7 +1129,7 @@ def _shall_ignore_pg_typeof_content_mismatch( True, ): return YesIgnore( - "materialize#27150: array_agg(pg_typeof(...)) in pg flattens result" + "database-issues#8028: array_agg(pg_typeof(...)) in pg flattens result" ) if query_template.matches_specific_select_or_filter_expression( @@ -1164,7 +1172,7 @@ def _shall_ignore_pg_typeof_content_mismatch( if query_template.matches_specific_select_or_filter_expression( col_index, partial(is_operation_tagged, tag=TAG_BASIC_ARITHMETIC_OP), True ): - return YesIgnore("materialize#28852: numeric return type inconsistency") + return YesIgnore("database-issues#8417: numeric return type inconsistency") if query_template.matches_specific_select_or_filter_expression( col_index, @@ -1205,7 +1213,7 @@ def _shall_ignore_content_type_mismatch( pg_error = details_by_strategy_key[EvaluationStrategyKey.POSTGRES] if mz_error.value == float and pg_error.value == int: - return YesIgnore("materialize#26306: float instead of int returned") + return YesIgnore("database-issues#7811: float instead of int returned") return self._shall_ignore_content_mismatch( error, diff --git a/misc/python/materialize/postgres_consistency/validation/pg_result_comparator.py b/misc/python/materialize/postgres_consistency/validation/pg_result_comparator.py index c2425e19e3812..4d17fbd707ac7 100644 --- a/misc/python/materialize/postgres_consistency/validation/pg_result_comparator.py +++ b/misc/python/materialize/postgres_consistency/validation/pg_result_comparator.py @@ -130,7 +130,7 @@ def is_str_equal(self, value1: str, value2: str, is_tolerant: bool) -> bool: or JSON_OBJECT_PATTERN.search(value2) ): # This is a rather eager pattern to also match concatenated strings. - # tracked with materialize#23571 + # tracked with database-issues#7085 value1 = value1.replace(", ", ",").replace(": ", ":") value2 = value2.replace(", ", ",").replace(": ", ":") @@ -177,7 +177,7 @@ def is_timestamp_equal(self, value1: str, value2: str) -> bool: return value1 == value2 def _normalize_jsonb_timestamp(self, value: str) -> str: - # this is due to materialize#28137 + # this is due to database-issues#8247 pattern_for_date = r"\d+-\d+-\d+" pattern_for_time = r"\d+:\d+:\d+[+-]\d+" @@ -206,7 +206,7 @@ def ignore_order_when_comparing_collection(self, expression: Expression) -> bool partial(is_operation_tagged, tag=TAG_JSONB_OBJECT_GENERATION), True, ): - # this is because of materialize#28192 + # this is because of database-issues#8266 return True return False diff --git a/misc/python/materialize/rustc_flags.py b/misc/python/materialize/rustc_flags.py index 1340bc0cbbbe1..6353f83d3b68a 100644 --- a/misc/python/materialize/rustc_flags.py +++ b/misc/python/materialize/rustc_flags.py @@ -58,7 +58,7 @@ def __str__(self) -> str: "-Zsanitizer=address", "-Cllvm-args=-asan-use-after-scope", "-Cllvm-args=-asan-use-after-return=always", - # "-Cllvm-args=-asan-stack=false", # Remove when materialize#25017 is fixed + # "-Cllvm-args=-asan-stack=false", # Remove when database-issues#7468 is fixed "-Cdebug-assertions=on", "-Clink-arg=-fuse-ld=lld", # access beyond end of merged section "-Clinker=clang++", diff --git a/misc/python/materialize/sqlsmith.py b/misc/python/materialize/sqlsmith.py index d86f66bb7c57f..0a27c984222b7 100644 --- a/misc/python/materialize/sqlsmith.py +++ b/misc/python/materialize/sqlsmith.py @@ -25,7 +25,7 @@ "function list_prepend(", # insufficient type system, parameter types have to match "function list_cat(", # insufficient type system, parameter types have to match "does not support implicitly casting from", - "aggregate functions that refer exclusively to outer columns not yet supported", # https://github.com/MaterializeInc/materialize/issues/3720 + "aggregate functions that refer exclusively to outer columns not yet supported", # https://api.github.com/repos/MaterializeInc/database-issues/issues/1163 "range lower bound must be less than or equal to range upper bound", "violates not-null constraint", "division by zero", @@ -68,7 +68,7 @@ "field position must be greater than zero", "array_fill on ", # Not yet supported "must not be null", # Expected with array_fill, array_position - "' not recognized", # Expected, see https://github.com/MaterializeInc/materialize/issues/17981 + "' not recognized", # Expected, see https://api.github.com/repos/MaterializeInc/database-issues/issues/5253 "must appear in the GROUP BY clause or be used in an aggregate function", "Expected joined table, found", # Should fix for multi table join "Expected ON, or USING after JOIN, found", # Should fix for multi table join @@ -79,9 +79,9 @@ "expected expression, but found reserved keyword", # Should fix, but only happens rarely with subqueries "Expected right parenthesis, found left parenthesis", # Should fix, but only happens rarely with cast+coalesce "invalid selection: operation may only refer to user-defined tables", # Seems expected when using catalog tables - "Unsupported temporal predicate", # Expected, see https://github.com/MaterializeInc/materialize/issues/18048 - "OneShot plan has temporal constraints", # Expected, see https://github.com/MaterializeInc/materialize/issues/18048 - "internal error: cannot evaluate unmaterializable function", # Currently expected, see https://github.com/MaterializeInc/materialize/issues/14290 + "Unsupported temporal predicate", # Expected, see https://api.github.com/repos/MaterializeInc/database-issues/issues/5288 + "OneShot plan has temporal constraints", # Expected, see https://api.github.com/repos/MaterializeInc/database-issues/issues/5288 + "internal error: cannot evaluate unmaterializable function", # Currently expected, see https://api.github.com/repos/MaterializeInc/database-issues/issues/4083 "string is not a valid identifier:", # Expected in parse_ident & quote_ident "invalid datepart", "pg_cancel_backend in this position not yet supported", @@ -115,7 +115,7 @@ "arrays must not contain null values", # aclexplode, mz_aclexplode "OVER clause not allowed on", # window functions "cannot reference pseudo type", - "window functions are not allowed in table function arguments", # TODO: Remove when materialize#20979 is implemented + "window functions are not allowed in table function arguments", # TODO: Remove when database-issues#6317 is implemented "window functions are not allowed in OR argument", # wrong error message "window functions are not allowed in AND argument", # wrong error message "window functions are not allowed in aggregate function", diff --git a/misc/python/materialize/test_analytics/data/build/build_data_storage.py b/misc/python/materialize/test_analytics/data/build/build_data_storage.py index 4feaa49618e9e..4b773fe56ff18 100644 --- a/misc/python/materialize/test_analytics/data/build/build_data_storage.py +++ b/misc/python/materialize/test_analytics/data/build/build_data_storage.py @@ -84,7 +84,7 @@ def add_build_job( job_id = buildkite.get_var(BuildkiteEnvVar.BUILDKITE_JOB_ID) step_id = buildkite.get_var(BuildkiteEnvVar.BUILDKITE_STEP_ID) step_key = buildkite.get_var(BuildkiteEnvVar.BUILDKITE_STEP_KEY) - # TODO: remove NULL casting when materialize#27429 is resolved + # TODO: remove NULL casting when database-issues#8100 is resolved shard_index = buildkite.get_var( BuildkiteEnvVar.BUILDKITE_PARALLEL_JOB, "NULL::INT" ) diff --git a/misc/python/materialize/test_analytics/data/feature_benchmark/feature_benchmark_result_storage.py b/misc/python/materialize/test_analytics/data/feature_benchmark/feature_benchmark_result_storage.py index 201b46695c236..c4db97e537533 100644 --- a/misc/python/materialize/test_analytics/data/feature_benchmark/feature_benchmark_result_storage.py +++ b/misc/python/materialize/test_analytics/data/feature_benchmark/feature_benchmark_result_storage.py @@ -41,7 +41,7 @@ def add_result( sql_statements = [] for result_entry in results: - # TODO: remove NULL castings when materialize#27429 is resolved + # TODO: remove NULL castings when database-issues#8100 is resolved sql_statements.append( f""" INSERT INTO feature_benchmark_result @@ -95,7 +95,7 @@ def add_discarded_entries( sql_statements = [] for discarded_entry in discarded_entries: - # TODO: remove NULL castings when materialize#27429 is resolved + # TODO: remove NULL castings when database-issues#8100 is resolved # Do not store framework version, scenario version, and scale. If needed, they can be retrieved from the # result entries. diff --git a/misc/python/materialize/test_analytics/data/parallel_benchmark/parallel_benchmark_result_storage.py b/misc/python/materialize/test_analytics/data/parallel_benchmark/parallel_benchmark_result_storage.py index 7720cd86071a9..dfa25633736d5 100644 --- a/misc/python/materialize/test_analytics/data/parallel_benchmark/parallel_benchmark_result_storage.py +++ b/misc/python/materialize/test_analytics/data/parallel_benchmark/parallel_benchmark_result_storage.py @@ -50,7 +50,7 @@ def add_result( sql_statements = [] for result_entry in results: - # TODO: remove NULL castings when materialize#27429 is resolved + # TODO: remove NULL castings when database-issues#8100 is resolved sql_statements.append( f""" INSERT INTO parallel_benchmark_result diff --git a/misc/python/materialize/test_analytics/setup/views/02-build-job.sql b/misc/python/materialize/test_analytics/setup/views/02-build-job.sql index f2bfa81e12d2c..3fa0e6707cc40 100644 --- a/misc/python/materialize/test_analytics/setup/views/02-build-job.sql +++ b/misc/python/materialize/test_analytics/setup/views/02-build-job.sql @@ -38,7 +38,7 @@ SELECT bj.shard_index, count(*) AS count_all, sum(CASE WHEN bj.success THEN 1 ELSE 0 END) AS count_successful, - -- avg(bj.end_time - bj.start_time) AS mean_duration_on_success, -- TODO materialize#28559: avg(interval) not supported + -- avg(bj.end_time - bj.start_time) AS mean_duration_on_success, -- TODO database-issues#8346: avg(interval) not supported avg(bj.retry_count) AS mean_retry_count FROM build b INNER JOIN build_job bj diff --git a/misc/python/materialize/zippy/backup_and_restore_actions.py b/misc/python/materialize/zippy/backup_and_restore_actions.py index 31231996cd74a..8c547ec0a7376 100644 --- a/misc/python/materialize/zippy/backup_and_restore_actions.py +++ b/misc/python/materialize/zippy/backup_and_restore_actions.py @@ -21,7 +21,7 @@ def requires(cls) -> set[type[Capability]]: return {MzIsRunning, CockroachIsRunning} def run(self, c: Composition, state: State) -> None: - # Required because of materialize#22762 + # Required because of database-issues#6880 c.kill("storaged") c.backup_crdb() diff --git a/src/adapter/src/catalog/builtin_table_updates.rs b/src/adapter/src/catalog/builtin_table_updates.rs index 5bf842a82b787..628d31a47ff98 100644 --- a/src/adapter/src/catalog/builtin_table_updates.rs +++ b/src/adapter/src/catalog/builtin_table_updates.rs @@ -1734,7 +1734,7 @@ impl CatalogState { id: &*MZ_AGGREGATES, row: Row::pack_slice(&[ Datum::UInt32(func_impl_details.oid), - // TODO(materialize#3326): Support ordered-set aggregate functions. + // TODO(database-issues#1064): Support ordered-set aggregate functions. Datum::String("n"), Datum::Int16(0), ]), @@ -2114,7 +2114,7 @@ impl CatalogState { }; let column_pos_datum = match column_pos { Some(pos) => { - // TODO(parkmycar): https://github.com/MaterializeInc/materialize/issues/22246. + // TODO(parkmycar): https://api.github.com/repos/MaterializeInc/database-issues/issues/6711. let pos = i32::try_from(pos).expect("we constrain this value in the planning layer"); Datum::Int32(pos) diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 6ba52e1a8be53..0abc6dc949a83 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -700,7 +700,7 @@ impl Catalog { CatalogItem::Sink(_) => { // Sinks don't have any external objects to drop--however, // this would change if we add a collections for sinks - // materialize#17672. + // database-issues#5148. } CatalogItem::View(_) => { // Views don't have any external objects to drop. @@ -772,7 +772,7 @@ impl Catalog { error!( "user sink {full_name} will be recreated as part of a builtin migration which \ can result in duplicate data being emitted. This is a known issue, \ - https://github.com/MaterializeInc/materialize/issues/18767. Please inform the \ + https://api.github.com/repos/MaterializeInc/database-issues/issues/5553. Please inform the \ customer that their sink may produce duplicate data." ) } diff --git a/src/adapter/src/catalog/state.rs b/src/adapter/src/catalog/state.rs index 0b69196d29eca..abd92c6f55266 100644 --- a/src/adapter/src/catalog/state.rs +++ b/src/adapter/src/catalog/state.rs @@ -1612,7 +1612,7 @@ impl CatalogState { // Some relations that have previously lived in the `mz_internal` schema have been moved to // `mz_catalog_unstable` or `mz_introspection`. To simplify the transition for users, we // automatically let uses of the old schema resolve to the new ones as well. - // TODO(materialize#27831) remove this after sufficient time has passed + // TODO(database-issues#8173) remove this after sufficient time has passed let mz_internal_schema = SchemaSpecifier::Id(self.get_mz_internal_schema_id()); if schemas.iter().any(|(_, spec)| *spec == mz_internal_schema) { for schema_id in [ diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index 353c1622a2a0d..6a1887b85ad77 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -1868,7 +1868,7 @@ impl Coordinator { let mut privatelink_connections = BTreeMap::new(); for entry in &entries { - // TODO(materialize#26794): we should move this invariant into `CatalogEntry`. + // TODO(database-issues#7922): we should move this invariant into `CatalogEntry`. mz_ore::soft_assert_or_log!( // We only expect user objects to objects obey this invariant. // System objects, for instance, can depend on other system diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index 42fae029a3023..3dfbd16c1247a 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -1079,10 +1079,10 @@ impl Coordinator { // If we didn't do this, then there would be a danger of missing the first refresh, // which might cause the materialized view to be unreadable for hours. This might // be what was happening here: - // https://github.com/MaterializeInc/materialize/issues/24288#issuecomment-1931856361 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/7265#issuecomment-1931856361 // // In the long term, it would be good to actually block the MV creation statement - // until `least_valid_read`. https://github.com/MaterializeInc/materialize/issues/25127 + // until `least_valid_read`. https://api.github.com/repos/MaterializeInc/database-issues/issues/7504 // Without blocking, we have the problem that a REFRESH AT CREATION is not linearized // with the CREATE MATERIALIZED VIEW statement, in the sense that a query from the MV // after its creation might see input changes that happened after the CRATE MATERIALIZED @@ -1212,7 +1212,7 @@ impl Coordinator { // If the session doesn't exist in `active_conns`, then this method will panic later on. // Instead we explicitly panic here while dumping the entire Coord to the logs to help // debug. This panic is very infrequent so we want as much information as possible. - // See https://github.com/MaterializeInc/materialize/issues/18996. + // See https://api.github.com/repos/MaterializeInc/database-issues/issues/5627. panic!("unknown connection: {conn_id:?}\n\n{self:?}") } diff --git a/src/adapter/src/coord/ddl.rs b/src/adapter/src/coord/ddl.rs index cc7e825b4f207..d60365a9de49f 100644 --- a/src/adapter/src/coord/ddl.rs +++ b/src/adapter/src/coord/ddl.rs @@ -667,9 +667,9 @@ impl Coordinator { // up external resources (PostgreSQL replication slots and secrets), // so we perform that cleanup in a background task. // - // TODO(materialize#14551): This is inherently best effort. An ill-timed crash + // TODO(database-issues#4154): This is inherently best effort. An ill-timed crash // means we'll never clean these resources up. Safer cleanup for non-Materialize resources. - // See + // See task::spawn(|| "drop_replication_slots_and_secrets", { let ssh_tunnel_manager = self.connection_context().ssh_tunnel_manager.clone(); let secrets_controller = Arc::clone(&self.secrets_controller); diff --git a/src/adapter/src/coord/sequencer/cluster.rs b/src/adapter/src/coord/sequencer/cluster.rs index e0b9d3c9ec1fd..541c41c2c3283 100644 --- a/src/adapter/src/coord/sequencer/cluster.rs +++ b/src/adapter/src/coord/sequencer/cluster.rs @@ -134,7 +134,7 @@ impl Coordinator { // Eagerly validate the `max_replicas_per_cluster` limit. // `catalog_transact` will do this validation too, but allocating // replica IDs is expensive enough that we need to do this validation - // before allocating replica IDs. See materialize#20195. + // before allocating replica IDs. See database-issues#6046. self.validate_resource_limit( 0, i64::from(replication_factor), @@ -266,7 +266,7 @@ impl Coordinator { // Eagerly validate the `max_replicas_per_cluster` limit. // `catalog_transact` will do this validation too, but allocating // replica IDs is expensive enough that we need to do this validation - // before allocating replica IDs. See materialize#20195. + // before allocating replica IDs. See database-issues#6046. self.validate_resource_limit( 0, i64::try_from(replicas.len()).unwrap_or(i64::MAX), @@ -622,7 +622,7 @@ impl Coordinator { // Eagerly validate the `max_replicas_per_cluster` limit. // `catalog_transact` will do this validation too, but allocating // replica IDs is expensive enough that we need to do this validation - // before allocating replica IDs. See materialize#20195. + // before allocating replica IDs. See database-issues#6046. if new_replication_factor > replication_factor { self.validate_resource_limit( usize::cast_from(*replication_factor), diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index c415cf14504b4..d1ff54dd8581c 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -941,7 +941,7 @@ impl Coordinator { // may also be trying to use `register_ts` for a different // purpose. // - // See materialize#28216. + // See database-issues#8273. coord .catalog .confirm_leadership() @@ -1872,7 +1872,7 @@ impl Coordinator { let vars = session.vars(); // Emit a warning when deprecated variables are used. - // TODO(materialize#27285) remove this after sufficient time has passed + // TODO(database-issues#8069) remove this after sufficient time has passed if name == vars::OLD_AUTO_ROUTE_CATALOG_QUERIES { session.add_notice(AdapterNotice::AutoRouteIntrospectionQueriesUsage); } else if name == vars::CLUSTER.name() @@ -3075,7 +3075,7 @@ impl Coordinator { session_var.visible(session.user(), Some(catalog.system_vars()))?; // Emit a warning when deprecated variables are used. - // TODO(materialize#27285) remove this after sufficient time has passed + // TODO(database-issues#8069) remove this after sufficient time has passed if variable.name() == vars::OLD_AUTO_ROUTE_CATALOG_QUERIES { notices.push(AdapterNotice::AutoRouteIntrospectionQueriesUsage); } else if let PlannedRoleVariable::Set { diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index 39c99accbc065..07d07e64abd54 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -45,7 +45,7 @@ pub enum AdapterError { up_to: mz_repr::Timestamp, }, /// Attempted to use a potentially ambiguous column reference expression with a system table. - // We don't allow this until https://github.com/MaterializeInc/materialize/issues/16650 is + // We don't allow this until https://api.github.com/repos/MaterializeInc/database-issues/issues/4824 is // resolved because it prevents us from adding columns to system tables. AmbiguousSystemColumnReference, /// An error occurred in a catalog operation. diff --git a/src/adapter/src/util.rs b/src/adapter/src/util.rs index 764a2ce472b02..b71dba454d2d4 100644 --- a/src/adapter/src/util.rs +++ b/src/adapter/src/util.rs @@ -472,7 +472,7 @@ pub fn verify_datum_desc( // Verify the first row is of the expected type. This is often good enough to // find problems. // - // Notably it failed to find materialize#6304 when "FETCH 2" was used in a test, instead + // Notably it failed to find database-issues#1946 when "FETCH 2" was used in a test, instead // we had to use "FETCH 1" twice. let Some(row) = rows.peek() else { diff --git a/src/avro/tests/schema.rs b/src/avro/tests/schema.rs index 36ac524725fdd..a99c3fa3dac46 100644 --- a/src/avro/tests/schema.rs +++ b/src/avro/tests/schema.rs @@ -465,7 +465,7 @@ static IGNORED_LOGICAL_TYPES: LazyLock> = LazyLock::n r#"{"type": "int", "logicalType": "timestamp-micros"}"#, Value::Int(1010), ), - // UUID logical type - materialize#3577 + // UUID logical type - database-issues#1120 // (r#"{"type": "string", "logicalType": "uuid"}"#, Value::String("string".into())), ] }); diff --git a/src/aws-util/src/s3_uploader.rs b/src/aws-util/src/s3_uploader.rs index 594b33cefe866..10d81d1bd1e30 100644 --- a/src/aws-util/src/s3_uploader.rs +++ b/src/aws-util/src/s3_uploader.rs @@ -392,7 +392,7 @@ mod tests { } #[mz_ore::test(tokio::test(flavor = "multi_thread"))] - #[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18898 + #[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5586 #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `TLS_method` on OS `linux` async fn multi_part_upload_success() -> Result<(), S3MultiPartUploadError> { let sdk_config = defaults().load().await; @@ -443,7 +443,7 @@ mod tests { } #[mz_ore::test(tokio::test(flavor = "multi_thread"))] - #[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18898 + #[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5586 #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `TLS_method` on OS `linux` async fn multi_part_upload_buffer() -> Result<(), S3MultiPartUploadError> { let sdk_config = defaults().load().await; @@ -501,7 +501,7 @@ mod tests { } #[mz_ore::test(tokio::test(flavor = "multi_thread"))] - #[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18898 + #[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5586 #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `TLS_method` on OS `linux` async fn multi_part_upload_error() -> Result<(), S3MultiPartUploadError> { let sdk_config = defaults().load().await; diff --git a/src/catalog/src/builtin.rs b/src/catalog/src/builtin.rs index 424d9305b02dc..2929113c74d70 100644 --- a/src/catalog/src/builtin.rs +++ b/src/catalog/src/builtin.rs @@ -3137,7 +3137,7 @@ SELECT name, mz_sources.type, occurred_at AS last_status_change_at, - -- TODO(parkmycar): Report status of webhook source once materialize#20036 is closed. + -- TODO(parkmycar): Report status of webhook source once database-issues#5986 is closed. CASE WHEN mz_sources.type = 'webhook' OR @@ -4124,7 +4124,7 @@ SELECT -- MZ doesn't use TOAST tables so reltoastrelid is filled with 0 0::pg_catalog.oid AS reltoastrelid, EXISTS (SELECT id, oid, name, on_id, cluster_id FROM mz_catalog.mz_indexes where mz_indexes.on_id = class_objects.id) AS relhasindex, - -- MZ doesn't have unlogged tables and because of (https://github.com/MaterializeInc/materialize/issues/8805) + -- MZ doesn't have unlogged tables and because of (https://api.github.com/repos/MaterializeInc/database-issues/issues/2689) -- temporary objects don't show up here, so relpersistence is filled with 'p' for permanent. -- TODO(jkosh44): update this column when issue is resolved. 'p'::pg_catalog.\"char\" AS relpersistence, @@ -5072,7 +5072,7 @@ HAVING pg_catalog.sum(count) != 0", pub static MZ_COMPUTE_ERROR_COUNTS_RAW_UNIFIED: LazyLock = LazyLock::new(|| BuiltinSource { - // TODO(materialize#27831): Rename this source to `mz_compute_error_counts_raw`. Currently this causes a + // TODO(database-issues#8173): Rename this source to `mz_compute_error_counts_raw`. Currently this causes a // naming conflict because the resolver stumbles over the source with the same name in // `mz_introspection` due to the automatic schema translation. name: "mz_compute_error_counts_raw_unified", @@ -7205,7 +7205,7 @@ sources AS ( JOIN mz_catalog.mz_cluster_replicas r ON (r.cluster_id = s.cluster_id) ), --- We don't yet report sink hydration status (materialize#28459), so we do a best effort attempt here and +-- We don't yet report sink hydration status (database-issues#8331), so we do a best effort attempt here and -- define a sink as hydrated when it's both "running" and has a frontier greater than the minimum. -- There is likely still a possibility of FPs. sinks AS ( diff --git a/src/catalog/src/durable/upgrade.rs b/src/catalog/src/durable/upgrade.rs index eab0b488844df..6f4652b87ce37 100644 --- a/src/catalog/src/durable/upgrade.rs +++ b/src/catalog/src/durable/upgrade.rs @@ -92,7 +92,7 @@ macro_rules! objects { fn from(value: StateUpdateKind) -> Self { let kind = value.kind.expect("kind should be set"); // TODO: This requires that the json->proto->json roundtrips - // exactly, see materialize#23908. + // exactly, see database-issues#7179. StateUpdateKindJson::from_serde(&kind) } } diff --git a/src/catalog/src/memory/objects.rs b/src/catalog/src/memory/objects.rs index 7b4d1385849b6..a55d61205a4c7 100644 --- a/src/catalog/src/memory/objects.rs +++ b/src/catalog/src/memory/objects.rs @@ -490,7 +490,7 @@ pub struct CatalogEntry { pub item: CatalogItem, #[serde(skip)] pub referenced_by: Vec, - // TODO(materialize#26794)––this should have an invariant tied to it that all + // TODO(database-issues#7922)––this should have an invariant tied to it that all // dependents (i.e. entries in this field) have IDs greater than this // entry's ID. #[serde(skip)] diff --git a/src/ccsr/tests/client.rs b/src/ccsr/tests/client.rs index e806ed9484f32..82bc923d59214 100644 --- a/src/ccsr/tests/client.rs +++ b/src/ccsr/tests/client.rs @@ -27,7 +27,7 @@ pub static SCHEMA_REGISTRY_URL: LazyLock = }); #[mz_ore::test(tokio::test)] -#[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18900 +#[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5588 #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_method` on OS `linux` async fn test_client() -> Result<(), anyhow::Error> { let client = mz_ccsr::ClientConfig::new(SCHEMA_REGISTRY_URL.clone()).build()?; @@ -254,7 +254,7 @@ async fn test_client_subject_and_references() -> Result<(), anyhow::Error> { #[mz_ore::test(tokio::test)] #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_method` on OS `linux` -#[ignore] // TODO: Reenable when materialize#22557 is fixed +#[ignore] // TODO: Reenable when database-issues#6818 is fixed async fn test_client_errors() -> Result<(), anyhow::Error> { let invalid_schema_registry_url: reqwest::Url = "data::text/plain,Info".parse().unwrap(); match mz_ccsr::ClientConfig::new(invalid_schema_registry_url).build() { diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 47f866099eb5b..c5d10de2e6de7 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -123,7 +123,7 @@ pub enum ComputeControllerResponse { /// Replica configuration #[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq)] pub struct ComputeReplicaConfig { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub logging: ComputeReplicaLogging, } @@ -281,7 +281,7 @@ impl ComputeController { } } - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn instance_exists(&self, id: ComputeInstanceId) -> bool { self.instances.contains_key(&id) } diff --git a/src/compute-client/src/controller/error.rs b/src/compute-client/src/controller/error.rs index 6ddb82c6257a3..38861ca92d09c 100644 --- a/src/compute-client/src/controller/error.rs +++ b/src/compute-client/src/controller/error.rs @@ -51,10 +51,10 @@ pub struct HydrationCheckBadTarget(pub Vec); /// Errors arising during compute collection lookup. #[derive(Error, Debug)] pub enum CollectionLookupError { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("instance does not exist: {0}")] InstanceMissing(ComputeInstanceId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("collection does not exist: {0}")] CollectionMissing(GlobalId), } @@ -74,10 +74,10 @@ impl From for CollectionLookupError { /// Errors arising during compute replica creation. #[derive(Error, Debug)] pub enum ReplicaCreationError { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("instance does not exist: {0}")] InstanceMissing(ComputeInstanceId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("replica exists already: {0}")] ReplicaExists(ReplicaId), } @@ -91,10 +91,10 @@ impl From for ReplicaCreationError { /// Errors arising during compute replica removal. #[derive(Error, Debug)] pub enum ReplicaDropError { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("instance does not exist: {0}")] InstanceMissing(ComputeInstanceId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("replica does not exist: {0}")] ReplicaMissing(ReplicaId), } @@ -157,16 +157,16 @@ impl From for DataflowCreationError { /// Errors arising during peek processing. #[derive(Error, Debug)] pub enum PeekError { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("instance does not exist: {0}")] InstanceMissing(ComputeInstanceId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("collection does not exist: {0}")] CollectionMissing(GlobalId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("replica does not exist: {0}")] ReplicaMissing(ReplicaId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("peek timestamp is not beyond the since of collection: {0}")] SinceViolation(GlobalId), } @@ -195,10 +195,10 @@ impl From for PeekError { /// Errors arising during collection updates. #[derive(Error, Debug)] pub enum CollectionUpdateError { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("instance does not exist: {0}")] InstanceMissing(ComputeInstanceId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("collection does not exist: {0}")] CollectionMissing(GlobalId), } @@ -218,13 +218,13 @@ impl From for CollectionUpdateError { /// Errors arising during collection read policy assignment. #[derive(Error, Debug)] pub enum ReadPolicyError { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("instance does not exist: {0}")] InstanceMissing(ComputeInstanceId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("collection does not exist: {0}")] CollectionMissing(GlobalId), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("collection is write-only: {0}")] WriteOnlyCollection(GlobalId), } @@ -244,7 +244,7 @@ impl From for ReadPolicyError { /// Errors arising during orphan removal. #[derive(Error, Debug)] pub enum RemoveOrphansError { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. #[error("orchestrator error: {0}")] OrchestratorError(anyhow::Error), } diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 890e151b89973..76fb46d7956b0 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -219,7 +219,7 @@ pub(super) struct Instance { /// /// The entry for a peek is only removed once all replicas have responded to the peek. This is /// currently required to ensure all replicas have stopped reading from the peeked collection's - /// inputs before we allow them to compact. materialize#16641 tracks changing this so we only have to wait + /// inputs before we allow them to compact. database-issues#4822 tracks changing this so we only have to wait /// for the first peek response. peeks: BTreeMap>, /// Currently in-progress subscribes. @@ -1686,7 +1686,7 @@ where }; // NOTE: We need to send the `CancelPeek` command _before_ we release the peek's read hold - // (by dropping it), to avoid the edge case that caused materialize#16615. + // (by dropping it), to avoid the edge case that caused database-issues#4812. self.send(ComputeCommand::CancelPeek { uuid }); drop(peek); @@ -1856,7 +1856,7 @@ where // For subscribes we downgrade all replica frontiers based on write frontiers. This should // be fine because the input and output frontier of a subscribe track its write frontier. - // TODO(materialize#16274): report subscribe frontiers through `Frontiers` responses + // TODO(database-issues#4701): report subscribe frontiers through `Frontiers` responses replica_collection.update_write_frontier(write_frontier.clone()); replica_collection.update_input_frontier(write_frontier.clone()); replica_collection.update_output_frontier(write_frontier.clone()); diff --git a/src/compute-client/src/logging.rs b/src/compute-client/src/logging.rs index 28f53e66e464a..d9c77f7faea77 100644 --- a/src/compute-client/src/logging.rs +++ b/src/compute-client/src/logging.rs @@ -26,7 +26,7 @@ include!(concat!(env!("OUT_DIR"), "/mz_compute_client.logging.rs")); // // Ideally we'd want to instead signal disabled logging by leaving `index_logs` // empty. Unfortunately, we have to always provide `index_logs`, because we must -// install the logging dataflows even on replicas that have logging disabled. See materialize#15799. +// install the logging dataflows even on replicas that have logging disabled. See database-issues#4545. #[derive(Debug, Default, Clone, PartialEq, Serialize, Deserialize)] pub struct LoggingConfig { /// The logging interval @@ -100,16 +100,16 @@ impl ProtoMapEntry for ProtoIndexLog { } } -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. #[derive( Arbitrary, Hash, Eq, PartialEq, Ord, PartialOrd, Debug, Clone, Copy, Serialize, Deserialize, )] pub enum LogVariant { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Timely(TimelyLog), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Differential(DifferentialLog), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Compute(ComputeLog), } @@ -154,32 +154,32 @@ impl RustType for LogVariant { } } -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. #[derive( Arbitrary, Hash, Eq, Ord, PartialEq, PartialOrd, Debug, Clone, Copy, Serialize, Deserialize, )] pub enum TimelyLog { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Operates, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Channels, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Elapsed, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Histogram, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Addresses, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Parks, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. MessagesSent, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. MessagesReceived, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Reachability, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. BatchesSent, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. BatchesReceived, } @@ -222,24 +222,24 @@ impl RustType for TimelyLog { } } -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. #[derive( Arbitrary, Hash, Eq, Ord, PartialEq, PartialOrd, Debug, Clone, Copy, Serialize, Deserialize, )] pub enum DifferentialLog { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. ArrangementBatches, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. ArrangementRecords, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Sharing, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. BatcherRecords, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. BatcherSize, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. BatcherCapacity, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. BatcherAllocations, } @@ -360,7 +360,7 @@ impl LogVariant { .unwrap_or_else(|| (0..arity).collect()) } - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn desc(&self) -> RelationDesc { match self { LogVariant::Timely(TimelyLog::Operates) => RelationDesc::builder() diff --git a/src/compute-client/src/metrics.rs b/src/compute-client/src/metrics.rs index 4675b8496a384..4bce8cbc6f068 100644 --- a/src/compute-client/src/metrics.rs +++ b/src/compute-client/src/metrics.rs @@ -33,7 +33,7 @@ use crate::protocol::response::{PeekResponse, ProtoComputeResponse}; type Counter = DeleteOnDropCounter<'static, AtomicF64, Vec>; type IntCounter = DeleteOnDropCounter<'static, AtomicU64, Vec>; type Gauge = DeleteOnDropGauge<'static, AtomicF64, Vec>; -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. pub type UIntGauge = DeleteOnDropGauge<'static, AtomicU64, Vec>; type Histogram = DeleteOnDropHistogram<'static, Vec>; @@ -267,7 +267,7 @@ pub struct InstanceMetrics { } impl InstanceMetrics { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn for_replica(&self, replica_id: ReplicaId) -> ReplicaMetrics { let labels = vec![self.instance_id.to_string(), replica_id.to_string()]; let extended_labels = |extra: &str| { @@ -332,7 +332,7 @@ impl InstanceMetrics { } } - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn for_history(&self) -> HistoryMetrics { let labels = vec![self.instance_id.to_string()]; let command_counts = CommandMetrics::build(|typ| { @@ -530,7 +530,7 @@ pub struct CommandMetrics { } impl CommandMetrics { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn build(build_metric: F) -> Self where F: Fn(&str) -> M, @@ -564,7 +564,7 @@ impl CommandMetrics { f(&self.cancel_peek); } - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn for_command(&self, command: &ComputeCommand) -> &M { use ComputeCommand::*; diff --git a/src/compute-client/src/protocol/command.rs b/src/compute-client/src/protocol/command.rs index 1bfecccf2cdf0..30701fcdc3d79 100644 --- a/src/compute-client/src/protocol/command.rs +++ b/src/compute-client/src/protocol/command.rs @@ -60,9 +60,9 @@ pub enum ComputeCommand { /// use the `epoch` to ensure that their individual processes agree on which protocol iteration /// they are in. CreateTimely { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. config: TimelyConfig, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. epoch: ClusterStartupEpoch, }, @@ -210,11 +210,11 @@ pub enum ComputeCommand { /// same collection. ([#16271]) /// /// [`Frontiers`]: super::response::ComputeResponse::Frontiers - /// [#16271]: https://github.com/MaterializeInc/materialize/issues/16271 + /// [#16271]: https://api.github.com/repos/MaterializeInc/database-issues/issues/4699 AllowCompaction { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. id: GlobalId, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. frontier: Antichain, }, @@ -368,7 +368,7 @@ impl Arbitrary for ComputeCommand { /// for anything in this struct. #[derive(Clone, Debug, PartialEq, Serialize, Deserialize, Arbitrary)] pub struct InstanceConfig { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub logging: LoggingConfig, } diff --git a/src/compute-client/src/protocol/history.rs b/src/compute-client/src/protocol/history.rs index d9d09b71c50bf..758e2f508fe27 100644 --- a/src/compute-client/src/protocol/history.rs +++ b/src/compute-client/src/protocol/history.rs @@ -21,7 +21,7 @@ use timely::PartialOrder; use crate::metrics::HistoryMetrics; use crate::protocol::command::{ComputeCommand, ComputeParameters}; -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. #[derive(Debug)] pub struct ComputeCommandHistory { /// The number of commands at the last time we compacted the history. @@ -41,7 +41,7 @@ where M: Borrow, T: timely::progress::Timestamp, { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn new(metrics: HistoryMetrics) -> Self { metrics.reset(); diff --git a/src/compute-client/src/protocol/response.rs b/src/compute-client/src/protocol/response.rs index f515f6574fc96..c5b0f215dfb72 100644 --- a/src/compute-client/src/protocol/response.rs +++ b/src/compute-client/src/protocol/response.rs @@ -66,8 +66,8 @@ pub enum ComputeResponse { /// [`AllowCompaction` command]: super::command::ComputeCommand::AllowCompaction /// [`CreateDataflow` command]: super::command::ComputeCommand::CreateDataflow /// [`CreateInstance` command]: super::command::ComputeCommand::CreateInstance - /// [#16271]: https://github.com/MaterializeInc/materialize/issues/16271 - /// [#16274]: https://github.com/MaterializeInc/materialize/issues/16274 + /// [#16271]: https://api.github.com/repos/MaterializeInc/database-issues/issues/4699 + /// [#16274]: https://api.github.com/repos/MaterializeInc/database-issues/issues/4701 Frontiers(GlobalId, FrontiersResponse), /// `PeekResponse` reports the result of a previous [`Peek` command]. The peek is identified by diff --git a/src/compute-client/src/service.rs b/src/compute-client/src/service.rs index 795c6998c21ea..db9dc2019f00c 100644 --- a/src/compute-client/src/service.rs +++ b/src/compute-client/src/service.rs @@ -65,7 +65,7 @@ impl GenericClient, ComputeResponse> for Box; #[async_trait] diff --git a/src/compute-types/src/dataflows.rs b/src/compute-types/src/dataflows.rs index 7e9ae76d7fcef..85723e8f8907d 100644 --- a/src/compute-types/src/dataflows.rs +++ b/src/compute-types/src/dataflows.rs @@ -820,9 +820,9 @@ pub struct IndexImport { /// An association of a global identifier to an expression. #[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] pub struct BuildDesc

{ - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub id: GlobalId, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub plan: P, } diff --git a/src/compute-types/src/explain.rs b/src/compute-types/src/explain.rs index a8d3169a1eef7..d840276bee251 100644 --- a/src/compute-types/src/explain.rs +++ b/src/compute-types/src/explain.rs @@ -151,7 +151,7 @@ impl<'a> DataflowDescription { } } -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. pub fn export_ids_for(dd: &DataflowDescription) -> BTreeMap { let mut map = BTreeMap::::default(); diff --git a/src/compute-types/src/plan/interpret/api.rs b/src/compute-types/src/plan/interpret/api.rs index 63411bcdb83b3..2f8f4e946f669 100644 --- a/src/compute-types/src/plan/interpret/api.rs +++ b/src/compute-types/src/plan/interpret/api.rs @@ -46,19 +46,19 @@ use crate::plan::{AvailableCollections, GetPlan, Plan}; /// /// [tagless final encoding]: /// -/// TODO(materialize#24943): align this with the `Plan` structure +/// TODO(database-issues#7446): align this with the `Plan` structure pub trait Interpreter { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. type Domain: Debug + Sized; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn constant( &self, ctx: &Context, rows: &Result, EvalError>, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn get( &self, ctx: &Context, @@ -67,7 +67,7 @@ pub trait Interpreter { plan: &GetPlan, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn mfp( &self, ctx: &Context, @@ -76,7 +76,7 @@ pub trait Interpreter { input_key_val: &Option<(Vec, Option)>, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn flat_map( &self, ctx: &Context, @@ -87,7 +87,7 @@ pub trait Interpreter { input_key: &Option>, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn join( &self, ctx: &Context, @@ -95,7 +95,7 @@ pub trait Interpreter { plan: &JoinPlan, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn reduce( &self, ctx: &Context, @@ -106,7 +106,7 @@ pub trait Interpreter { mfp_after: &MapFilterProject, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn top_k( &self, ctx: &Context, @@ -114,10 +114,10 @@ pub trait Interpreter { top_k_plan: &TopKPlan, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn negate(&self, ctx: &Context, input: Self::Domain) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn threshold( &self, ctx: &Context, @@ -125,7 +125,7 @@ pub trait Interpreter { threshold_plan: &ThresholdPlan, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn union( &self, ctx: &Context, @@ -133,7 +133,7 @@ pub trait Interpreter { consolidate_output: bool, ) -> Self::Domain; - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn arrange_by( &self, ctx: &Context, @@ -153,7 +153,7 @@ pub struct InterpreterContext { pub is_rec: bool, } -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. pub type Context = InterpreterContext; impl Default for InterpreterContext { @@ -225,7 +225,7 @@ where I: Interpreter, I::Domain: BoundedLattice + Clone, { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn new(interpreter: I) -> Self { Self { interpret: interpreter, @@ -500,7 +500,7 @@ where I::Domain: BoundedLattice + Clone, A: FnMut(&mut Plan, &I::Domain, &[I::Domain]), { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn new(interpreter: I, action: A) -> Self { Self { interpret: interpreter, diff --git a/src/compute-types/src/plan/join.rs b/src/compute-types/src/plan/join.rs index 607b9231af813..7f95169a32adb 100644 --- a/src/compute-types/src/plan/join.rs +++ b/src/compute-types/src/plan/join.rs @@ -84,9 +84,9 @@ impl RustType for JoinPlan { /// this with a Rust closure (glorious battle was waged, but ultimately lost). #[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq, Ord, PartialOrd)] pub struct JoinClosure { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub ready_equivalences: Vec>, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub before: mz_expr::SafeMfpPlan, } diff --git a/src/compute-types/src/plan/lowering.rs b/src/compute-types/src/plan/lowering.rs index bd58900857a02..ed973056068f5 100644 --- a/src/compute-types/src/plan/lowering.rs +++ b/src/compute-types/src/plan/lowering.rs @@ -554,7 +554,7 @@ impl Context { // Extract temporal predicates as joins cannot currently absorb them. let (plan, missing) = match implementation { IndexedFilter(_coll_id, _idx_id, key, _val) => { - // Start with the constant input. (This used to be important before materialize#14059 + // Start with the constant input. (This used to be important before database-issues#4016 // was fixed.) let start: usize = 1; let order = vec![(0usize, key.clone(), None)]; diff --git a/src/compute-types/src/plan/transform/api.rs b/src/compute-types/src/plan/transform/api.rs index ed996f0ae0923..30bba3f5eecf4 100644 --- a/src/compute-types/src/plan/transform/api.rs +++ b/src/compute-types/src/plan/transform/api.rs @@ -21,13 +21,13 @@ use crate::plan::Plan; /// as an immutable reference. #[derive(Debug)] pub struct TransformConfig { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub monotonic_ids: BTreeSet, } /// A transform for [crate::plan::Plan] nodes. pub trait Transform { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. fn name(&self) -> &'static str; /// Transform a [Plan] using the given [TransformConfig]. @@ -58,7 +58,7 @@ pub trait Transform { ) -> Result<(), RecursionLimitError>; } -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. pub trait BottomUpTransform { /// A type representing analysis information to be associated with each /// sub-term and exposed to the transformation action callback. diff --git a/src/compute-types/src/plan/transform/relax_must_consolidate.rs b/src/compute-types/src/plan/transform/relax_must_consolidate.rs index c727fd9d9f034..8138235997b7a 100644 --- a/src/compute-types/src/plan/transform/relax_must_consolidate.rs +++ b/src/compute-types/src/plan/transform/relax_must_consolidate.rs @@ -27,7 +27,7 @@ pub struct RelaxMustConsolidate { } impl RelaxMustConsolidate { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub fn new() -> Self { RelaxMustConsolidate { _phantom: Default::default(), diff --git a/src/compute-types/src/sinks.rs b/src/compute-types/src/sinks.rs index 672ae49ee80c7..e2330139e28a3 100644 --- a/src/compute-types/src/sinks.rs +++ b/src/compute-types/src/sinks.rs @@ -25,19 +25,19 @@ include!(concat!(env!("OUT_DIR"), "/mz_compute_types.sinks.rs")); /// A sink for updates to a relational collection. #[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)] pub struct ComputeSinkDesc { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub from: GlobalId, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub from_desc: RelationDesc, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub connection: ComputeSinkConnection, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub with_snapshot: bool, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub up_to: Antichain, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub non_null_assertions: Vec, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub refresh_schedule: Option, } @@ -112,14 +112,14 @@ impl RustType for ComputeSinkDesc { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Subscribe(SubscribeSinkConnection), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. Persist(PersistSinkConnection), - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. /// /// TODO(ct): This also writes to persist, but with different behavior /// (conflict resolution, only at input times, etc). It might be time to @@ -183,7 +183,7 @@ impl RustType for ComputeSinkConnection for CopyToS3OneshotSinkConnect } } -/// TODO(materialize#25239): Add documentation. +/// TODO(database-issues#7533): Add documentation. #[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] pub struct PersistSinkConnection { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub value_desc: RelationDesc, - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. pub storage_metadata: S, } @@ -260,7 +260,7 @@ impl RustType for PersistSinkConnection { - /// TODO(materialize#25239): Add documentation. + /// TODO(database-issues#7533): Add documentation. // // TODO(ct): This can be removed once we render the "input" sources without // the hack. diff --git a/src/compute/src/compute_state.rs b/src/compute/src/compute_state.rs index 8811d8f330586..40437f511ab75 100644 --- a/src/compute/src/compute_state.rs +++ b/src/compute/src/compute_state.rs @@ -627,7 +627,7 @@ impl<'a, A: Allocate + 'static> ActiveComputeState<'a, A> { for (&id, collection) in self.compute_state.collections.iter_mut() { // The compute protocol does not allow `Frontiers` responses for subscribe and copy-to - // collections (materialize#16274). + // collections (database-issues#4701). if collection.is_subscribe_or_copy { continue; } @@ -1480,7 +1480,7 @@ pub struct CollectionState { /// /// The compute protocol does not allow `Frontiers` responses for subscribe and copy-to /// collections, so we need to be able to recognize them. This is something we would like to - /// change in the future (materialize#16274). + /// change in the future (database-issues#4701). pub is_subscribe_or_copy: bool, /// The collection's initial as-of frontier. /// diff --git a/src/compute/src/logging/initialize.rs b/src/compute/src/logging/initialize.rs index ee61163318fa3..9f8ea349b4c7a 100644 --- a/src/compute/src/logging/initialize.rs +++ b/src/compute/src/logging/initialize.rs @@ -142,7 +142,7 @@ impl LoggingContext<'_, A> { // TODO(vmarcos): If we introduce introspection sources that would match // type specialization for keys, we'd need to ensure that type specialized - // variants reach the map below (issue materialize#22398). + // variants reach the map below (issue database-issues#6763). collections .into_iter() .map(|(log, collection)| { diff --git a/src/compute/src/render.rs b/src/compute/src/render.rs index 86fadd2105f54..f1d5bdf84b7bb 100644 --- a/src/compute/src/render.rs +++ b/src/compute/src/render.rs @@ -266,7 +266,7 @@ pub fn build_compute_dataflow( // For the moment, assert that it is either trivial or `None`. assert!(mfp.map(|x| x.is_identity()).unwrap_or(true)); - // To avoid a memory spike during arrangement hydration (materialize#21165), need to + // To avoid a memory spike during arrangement hydration (database-issues#6368), need to // ensure that the first frontier we report into the dataflow is beyond the // `as_of`. if let Some(as_of) = dataflow.as_of.clone() { @@ -1322,7 +1322,7 @@ where /// Suppress progress messages for times before the given `as_of`. /// /// This operator exists specifically to work around a memory spike we'd otherwise see when -/// hydrating arrangements (materialize#21165). The memory spike happens because when the `arrange_core` +/// hydrating arrangements (database-issues#6368). The memory spike happens because when the `arrange_core` /// operator observes a frontier advancement without data it inserts an empty batch into the spine. /// When it later inserts the snapshot batch into the spine, an empty batch is already there and /// the spine initiates a merge of these batches, which requires allocating a new batch the size of diff --git a/src/compute/src/render/context.rs b/src/compute/src/render/context.rs index 3e8be584c94b1..8bffc0ff427f6 100644 --- a/src/compute/src/render/context.rs +++ b/src/compute/src/render/context.rs @@ -920,7 +920,7 @@ where // In the case that we weren't going to apply the `key_val` optimization, // this path results in a slightly smaller and faster // dataflow graph, and is intended to fix - // https://github.com/MaterializeInc/materialize/issues/10507 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/3111 let has_key_val = if let Some((_key, Some(_val))) = &key_val { true } else { diff --git a/src/compute/src/render/errors.rs b/src/compute/src/render/errors.rs index b949f06461a73..887015ef677ed 100644 --- a/src/compute/src/render/errors.rs +++ b/src/compute/src/render/errors.rs @@ -105,7 +105,7 @@ impl ErrorLogger { /// contain some sensitive customer data. We include the `message` to make it possible to match /// the breadcrumbs to their associated error in Sentry. /// - // TODO(materialize#18214): Rethink or justify our error logging strategy. + // TODO(database-issues#5362): Rethink or justify our error logging strategy. pub fn log(&self, message: &'static str, details: &str) { if !self.token.in_shutdown() { self.log_always(message, details); diff --git a/src/compute/src/render/join/linear_join.rs b/src/compute/src/render/join/linear_join.rs index fa5e7396360a2..11bd344839e6c 100644 --- a/src/compute/src/render/join/linear_join.rs +++ b/src/compute/src/render/join/linear_join.rs @@ -381,7 +381,7 @@ where errors.push(errs); - // TODO(vmarcos): We should implement further arrangement specialization here (materialize#22104). + // TODO(vmarcos): We should implement further arrangement specialization here (database-issues#6659). // By knowing how types propagate through joins we could specialize intermediate // arrangements as well, either in values or eventually in keys. let arranged = keyed.mz_arrange::>("JoinStage"); diff --git a/src/compute/src/render/reduce.rs b/src/compute/src/render/reduce.rs index df8aa6ec53fad..39aa425f8700e 100644 --- a/src/compute/src/render/reduce.rs +++ b/src/compute/src/render/reduce.rs @@ -205,7 +205,7 @@ where S: Scope, { // TODO(vmarcos): Arrangement specialization here could eventually be extended to keys, - // not only values (materialize#22103). + // not only values (database-issues#6658). let arrangement = match plan { // If we have no aggregations or just a single type of reduction, we // can go ahead and render them directly. @@ -409,7 +409,7 @@ where // If we did not have enough values to stitch together, then we do not // generate an output row. Not outputting here corresponds to the semantics - // of an equi-join on the key, similarly to the proposal in PR materialize#17013. + // of an equi-join on the key, similarly to the proposal in PR database-issues#6173. // // Note that we also do not want to have anything left over to stich. If we // do, then we also have an error, reported elsewhere, and would violate @@ -1923,7 +1923,7 @@ fn finalize_accum<'a>(aggr_func: &'a AggregateFunc, accum: &'a Accum, total: Dif | (AggregateFunc::SumUInt32, Accum::SimpleNumber { accum, .. }) => { if !accum.is_negative() { // Our semantics of overflow are not clearly articulated wrt. - // unsigned vs. signed types (materialize#17758). We adopt an + // unsigned vs. signed types (database-issues#5172). We adopt an // unsigned wrapping behavior to match what we do above for // signed types. // TODO(vmarcos): remove potentially dangerous usage of `as`. diff --git a/src/compute/src/render/top_k.rs b/src/compute/src/render/top_k.rs index 330a7951dc4db..c7ffc61548bc9 100644 --- a/src/compute/src/render/top_k.rs +++ b/src/compute/src/render/top_k.rs @@ -68,7 +68,7 @@ where // is an opportunity to do so for every group key instead if the error handling is // integrated with: 1. The intra-timestamp thinning step in monotonic top-k, e.g., by // adding an error output there; 2. The validating reduction on basic top-k - // (materialize#23687). + // (database-issues#7108). let limit_err = match &top_k_plan { TopKPlan::MonotonicTop1(MonotonicTop1Plan { .. }) => None, TopKPlan::MonotonicTopK(MonotonicTopKPlan { limit, .. }) => Some(limit), @@ -499,7 +499,7 @@ where output.push((accum.row.clone(), 1)); }, ); - // TODO(materialize#7331): Here we discard the arranged output. + // TODO(database-issues#2288): Here we discard the arranged output. (result.as_collection(|_k, v| v.into_owned()), errs) } } diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index e03a7fdef4e6f..48e7b6397fcb1 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -805,7 +805,7 @@ impl<'w, A: Allocate + 'static> Worker<'w, A> { // distinct TVCs, so the controller doesn't expect any historical consistency from // these collections when it reconnects to a replica. // - // TODO(materialize#27730): Consider resolving this with controller-side reconciliation instead. + // TODO(database-issues#8152): Consider resolving this with controller-side reconciliation instead. if let Some(config) = old_instance_config { for id in config.logging.index_logs.values() { let trace = compute_state diff --git a/src/compute/src/sink/subscribe.rs b/src/compute/src/sink/subscribe.rs index b7377426e45bf..a2002ea58721e 100644 --- a/src/compute/src/sink/subscribe.rs +++ b/src/compute/src/sink/subscribe.rs @@ -180,7 +180,7 @@ struct SubscribeProtocol { /// /// As soon as a subscribe has encountered an error, it is poisoned: It will only return the /// same error in subsequent batches, until it is dropped. The subscribe protocol currently - /// does not support retracting errors (materialize#17781). + /// does not support retracting errors (database-issues#5182). pub poison: Option, } diff --git a/src/dyncfg/src/lib.rs b/src/dyncfg/src/lib.rs index 792e93541c824..f055b334ce144 100644 --- a/src/dyncfg/src/lib.rs +++ b/src/dyncfg/src/lib.rs @@ -795,7 +795,7 @@ mod tests { #[mz_ore::test] fn config_updates_extend() { - // Regression test for materialize#26196. + // Regression test for database-issues#7793. // // Construct two ConfigUpdates with overlapping, but not identical, sets // of configs. Combine them and assert that the expected number of diff --git a/src/environmentd/tests/pgwire.rs b/src/environmentd/tests/pgwire.rs index 2fb2965f1e2e9..8e6fe2fbb5ecb 100644 --- a/src/environmentd/tests/pgwire.rs +++ b/src/environmentd/tests/pgwire.rs @@ -49,7 +49,7 @@ fn test_bind_params() { .unwrap() .get::<_, bool>(0)); - // Just ensure it does not panic (see materialize#2498). + // Just ensure it does not panic (see database-issues#871). client .query("EXPLAIN PLAN FOR SELECT $1::int", &[&42_i32]) .unwrap(); @@ -112,7 +112,7 @@ fn test_bind_params() { .query_one("CREATE VIEW v AS SELECT $3", &[]) .unwrap_db_error(); // TODO(benesch): this should be `UNDEFINED_PARAMETER`, but blocked - // on materialize#3147. + // on database-issues#1031. assert_eq!(err.message(), "there is no parameter $3"); assert_eq!(err.code(), &SqlState::INTERNAL_ERROR); @@ -376,7 +376,7 @@ fn test_simple_query_no_hang() { let server = test_util::TestHarness::default().start_blocking(); let mut client = server.connect(postgres::NoTls).unwrap(); assert_err!(client.simple_query("asdfjkl;")); - // This will hang if materialize#2880 is not fixed. + // This will hang if database-issues#972 is not fixed. assert_ok!(client.simple_query("SELECT 1")); } diff --git a/src/environmentd/tests/server.rs b/src/environmentd/tests/server.rs index 3879a7939a8f3..4d4ef5b5ce406 100644 --- a/src/environmentd/tests/server.rs +++ b/src/environmentd/tests/server.rs @@ -200,7 +200,7 @@ fn setup_statement_logging( // Test that we log various kinds of statement whose execution terminates in the coordinator. #[mz_ore::test] -#[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/21598 +#[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/6487 fn test_statement_logging_immediate() { let (server, mut client) = setup_statement_logging(1.0, 1.0); let successful_immediates: &[&str] = &[ @@ -1295,7 +1295,7 @@ fn test_storage_usage_updates_between_restarts() { } #[mz_ore::test] -#[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18896 +#[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5584 fn test_storage_usage_doesnt_update_between_restarts() { let data_dir = tempfile::tempdir().unwrap(); let storage_usage_collection_interval = Duration::from_secs(10); @@ -1624,7 +1624,7 @@ fn test_default_cluster_sizes() { } #[mz_ore::test] -#[ignore] // TODO: Reenable when materialize#22998 is fixed +#[ignore] // TODO: Reenable when database-issues#6931 is fixed fn test_max_request_size() { let statement = "SELECT $1::text"; let statement_size = statement.bytes().count(); @@ -3137,7 +3137,7 @@ fn test_github_20262() { } // Test that the server properly handles cancellation requests of read-then-write queries. -// See materialize#20404. +// See database-issues#6134. #[mz_ore::test] #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `epoll_wait` on OS `linux` fn test_cancel_read_then_write() { diff --git a/src/environmentd/tests/sql.rs b/src/environmentd/tests/sql.rs index 6b5df6ef89ed8..7dcecfbebba2b 100644 --- a/src/environmentd/tests/sql.rs +++ b/src/environmentd/tests/sql.rs @@ -599,7 +599,7 @@ fn test_subscribe_basic() { // Aggressively compact the data in the index, then subscribe an unmaterialized // view derived from the index. This previously selected an invalid - // `AS OF` timestamp (materialize#5391). + // `AS OF` timestamp (database-issues#1666). client_writes .batch_execute("ALTER TABLE t SET (RETAIN HISTORY = FOR '1s')") .unwrap(); @@ -756,7 +756,7 @@ fn test_subscribe_progress() { } // Verifies that subscribing to non-nullable columns with progress information -// turns them into nullable columns. See materialize#6304. +// turns them into nullable columns. See database-issues#1946. #[mz_ore::test] fn test_subscribe_progress_non_nullable_columns() { let server = test_util::TestHarness::default().start_blocking(); @@ -966,7 +966,7 @@ fn test_subscribe_fetch_timeout() { // Make a third cursor. Fetch should return immediately if there are enough // rows, even with a really long timeout. // - // Regression test for materialize#6307 + // Regression test for database-issues#1949 client .batch_execute( "COMMIT; BEGIN; @@ -1378,8 +1378,8 @@ fn test_transactional_explain_timestamps() { // Feel free to modify this test if that product requirement changes, // but please at least keep _something_ that tests that custom compaction windows are working. #[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 1))] -#[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18934 -#[ignore] // TODO: Reenable when materialize#29299 is fixed +#[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5600 +#[ignore] // TODO: Reenable when database-issues#8491 is fixed async fn test_utilization_hold() { const THIRTY_DAYS_MS: u64 = 30 * 24 * 60 * 60 * 1000; // `mz_catalog_server` tests indexes, `quickstart` tests tables. @@ -1624,7 +1624,7 @@ fn test_github_12951() { } #[mz_ore::test] -// Tests github issue materialize#13100 +// Tests github issue database-issues#3761 fn test_subscribe_outlive_cluster() { let server = test_util::TestHarness::default().start_blocking(); @@ -3359,7 +3359,7 @@ async fn test_explain_as_of() { // Test that RETAIN HISTORY results in the since and upper being separated by the specified amount. #[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 1))] -#[ignore] // TODO: Reenable when materialize#24957 is fixed +#[ignore] // TODO: Reenable when database-issues#7450 is fixed async fn test_retain_history() { let server = test_util::TestHarness::default().start().await; let client = server.connect().await.unwrap(); @@ -3655,7 +3655,7 @@ async fn test_explain_timestamp_blocking() { #[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 1))] #[cfg_attr(miri, ignore)] // too slow async fn test_explain_timestamp_on_const_with_temporal() { - // Regression test for https://github.com/MaterializeInc/materialize/issues/25841 + // Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/7705 let server = test_util::TestHarness::default().start().await; let client = server.connect().await.unwrap(); diff --git a/src/expr/src/explain.rs b/src/expr/src/explain.rs index bb3002b5f06b4..31066898397dd 100644 --- a/src/expr/src/explain.rs +++ b/src/expr/src/explain.rs @@ -222,7 +222,7 @@ pub fn enforce_linear_chains(expr: &mut MirRelationExpr) -> Result<(), ExplainEr if expr.is_recursive() { // `linear_chains` is not implemented for WMR, see - // https://github.com/MaterializeInc/materialize/issues/19012 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/5631 return Err(LinearChainsPlusRecursive); } diff --git a/src/expr/src/interpret.rs b/src/expr/src/interpret.rs index 2adb32509ce65..c702e46652881 100644 --- a/src/expr/src/interpret.rs +++ b/src/expr/src/interpret.rs @@ -1430,7 +1430,7 @@ mod tests { #[mz_ore::test] fn test_mfp() { - // Regression test for https://github.com/MaterializeInc/materialize/issues/19338 + // Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5736 use MirScalarExpr::*; let mfp = MapFilterProject { diff --git a/src/expr/src/relation/func.rs b/src/expr/src/relation/func.rs index eddedd2f48302..13ef6888d7d8a 100644 --- a/src/expr/src/relation/func.rs +++ b/src/expr/src/relation/func.rs @@ -1589,13 +1589,13 @@ where | (Rows, OffsetPreceding(..), UnboundedFollowing) | (Rows, OffsetFollowing(..), UnboundedFollowing) => { // Unsupported. Bail in the planner. - // https://github.com/MaterializeInc/materialize/issues/22268 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6720 unreachable!() } (Range, _, _) => { // Unsupported. // The planner doesn't allow Range frame mode for now (except for the default - // frame), see https://github.com/MaterializeInc/materialize/issues/21934 + // frame), see https://api.github.com/repos/MaterializeInc/database-issues/issues/6585 // Note that it would be easy to handle (Range, CurrentRow, UnboundedFollowing): // it would be similar to (Rows, CurrentRow, UnboundedFollowing), but would call // groups_between_unbounded_preceding_current_row. @@ -1604,7 +1604,7 @@ where (Groups, _, _) => { // Unsupported. // The planner doesn't allow Groups frame mode for now, see - // https://github.com/MaterializeInc/materialize/issues/21940 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6588 unreachable!() } } @@ -3298,7 +3298,7 @@ impl AnalyzedRegex { index: i as u32, name: name.map(String::from), // TODO -- we can do better. - // https://github.com/MaterializeInc/materialize/issues/1685 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/612 nullable: true, }) .collect(); diff --git a/src/expr/src/scalar.rs b/src/expr/src/scalar.rs index 55e3a64be4dc4..8964303f82206 100644 --- a/src/expr/src/scalar.rs +++ b/src/expr/src/scalar.rs @@ -1220,7 +1220,7 @@ impl MirScalarExpr { let (limit, flags) = regexp_replace_parse_flags(flags); // Defer errors until evaluation instead of eagerly returning them here - // to match the error behavior of the dynamic function (part of materialize#17189). + // to match the error behavior of the dynamic function (part of database-issues#4972). let regex = match func::build_regex(pattern, &flags) { Ok(regex) => Ok((regex, limit)), Err(err) => Err(err), diff --git a/src/expr/src/scalar/func.rs b/src/expr/src/scalar/func.rs index a1d26dbf561bc..c7906521aacc4 100644 --- a/src/expr/src/scalar/func.rs +++ b/src/expr/src/scalar/func.rs @@ -483,7 +483,7 @@ fn convert_from<'a>(a: Datum<'a>, b: Datum<'a>) -> Result, EvalError> // [3]: https://github.com/lifthrasiir/rust-encoding/blob/4e79c35ab6a351881a86dbff565c4db0085cc113/src/label.rs let encoding_name = b.unwrap_str().to_lowercase().replace('_', "-"); - // Supporting other encodings is tracked by materialize#2282. + // Supporting other encodings is tracked by database-issues#797. if encoding_from_whatwg_label(&encoding_name).map(|e| e.name()) != Some("utf-8") { return Err(EvalError::InvalidEncodingName(encoding_name)); } @@ -8003,7 +8003,7 @@ impl VariadicFunc { .nullable(true), ListCreate { elem_type } => { // commented out to work around - // https://github.com/MaterializeInc/materialize/issues/8963 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/2730 // soft_assert!( // input_types.iter().all(|t| t.scalar_type.base_eq(elem_type)), // "{}", format!("Args to ListCreate should have types that are compatible with the elem_type.\nArgs:{:#?}\nelem_type:{:#?}", input_types, elem_type) diff --git a/src/ore/src/retry.rs b/src/ore/src/retry.rs index 689421571f778..8fada7316ac24 100644 --- a/src/ore/src/retry.rs +++ b/src/ore/src/retry.rs @@ -749,7 +749,7 @@ mod tests { #[crate::test(tokio::test)] #[cfg_attr(miri, ignore)] // unsupported operation: cannot write to event - #[ignore] // TODO: Reenable when materialize#24933 is fixed + #[ignore] // TODO: Reenable when database-issues#7441 is fixed async fn test_retry_async_fail_max_duration() { let mut states = vec![]; let res = Retry::default() diff --git a/src/ore/src/tracing.rs b/src/ore/src/tracing.rs index 9dfea18db6170..68ab42b2f0292 100644 --- a/src/ore/src/tracing.rs +++ b/src/ore/src/tracing.rs @@ -572,7 +572,7 @@ where // that the sentry layer would specifically communicating that it wants to see // everything. This bug appears to be related to the presence of a `reload::Layer` // _around a filter, not a layer_, and guswynn is tracking investigating it here: - // . Because we don't + // . Because we don't // enable a reload-able filter in CI/locally, but DO in production (the otel layer), it // was once possible to trigger and rely on the fast path in CI, but not notice that it // was disabled in production. diff --git a/src/persist-client/src/batch.rs b/src/persist-client/src/batch.rs index df9aa0b3a3db3..be6dcef3419ff 100644 --- a/src/persist-client/src/batch.rs +++ b/src/persist-client/src/batch.rs @@ -310,7 +310,7 @@ where /// to be _not possible_ for partially ordered times. It is believed that we /// could fix this by collecting different metadata in batch creation (e.g. /// the join of or an antichain of the original contained timestamps), but - /// the experience of materialize#26384 has shaken our confidence in our own abilities + /// the experience of database-issues#7825 has shaken our confidence in our own abilities /// to reason about partially ordered times and anyway all the initial uses /// have totally ordered times. pub fn rewrite_ts( @@ -510,7 +510,7 @@ pub(crate) struct UntrimmableColumns { impl UntrimmableColumns { pub(crate) fn should_retain(&self, name: &str) -> bool { // TODO: see if there's a better way to match different formats than lowercasing - // https://github.com/MaterializeInc/materialize/issues/21353#issue-1863623805 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6421#issue-1863623805 let name_lower = name.to_lowercase(); for s in &self.equals { if *s == name_lower { diff --git a/src/persist-client/src/cli/admin.rs b/src/persist-client/src/cli/admin.rs index 59745425adfca..3de6e69d1de5f 100644 --- a/src/persist-client/src/cli/admin.rs +++ b/src/persist-client/src/cli/admin.rs @@ -702,7 +702,7 @@ pub async fn dangerous_force_compaction_and_break_pushdown( write: &WriteHandle, fuel: impl Fn() -> usize, wait: impl Fn() -> Duration, - // TODO: Here to make the test semi-work until we fix materialize#29459. + // TODO: Here to make the test semi-work until we fix database-issues#8530. max_attempts: Option, ) where K: Debug + Codec, @@ -830,7 +830,7 @@ mod tests { .await; let batches_after = machine.applier.all_batches().len(); println!("{}: got {} batches after", num_batches, batches_after); - // TODO: Enable this after materialize#29459 is fixed. + // TODO: Enable this after database-issues#8530 is fixed. // assert!(batches_after < 2, "{} vs {}", num_batches, batches_after); } } diff --git a/src/persist-client/src/internal/compact.rs b/src/persist-client/src/internal/compact.rs index ebd82e510d2fe..b70586ca6691e 100644 --- a/src/persist-client/src/internal/compact.rs +++ b/src/persist-client/src/internal/compact.rs @@ -690,7 +690,7 @@ where ); // Duplicating a large codepath here during the migration. - // TODO(materialize#23942): dedup once the migration is complete. + // TODO(database-issues#7188): dedup once the migration is complete. if cfg.batch.expected_order == RunOrder::Structured { // If we're not writing down the record metadata, we must always use the old compaction // order. (Since that's the default when the metadata's not present.) diff --git a/src/persist-client/src/internal/machine.rs b/src/persist-client/src/internal/machine.rs index 13df1edd8f5e5..4159e21b70f68 100644 --- a/src/persist-client/src/internal/machine.rs +++ b/src/persist-client/src/internal/machine.rs @@ -365,7 +365,7 @@ where .try_into() .expect("reasonable duration"); // SUBTLE: Retries of compare_and_append with Indeterminate errors are - // tricky (more discussion of this in materialize#12797): + // tricky (more discussion of this in database-issues#3680): // // - (1) We compare_and_append and get an Indeterminate error back from // CRDB/Consensus. This means we don't know if it committed or not. @@ -2519,7 +2519,7 @@ pub mod tests { ); } - // A regression test for materialize#14719, where a bug in gc led to an incremental + // A regression test for database-issues#4206, where a bug in gc led to an incremental // state invariant being violated which resulted in gc being permanently // wedged for the shard. #[mz_persist_proc::test(tokio::test(flavor = "multi_thread"))] diff --git a/src/persist-client/src/internal/state.rs b/src/persist-client/src/internal/state.rs index 71c69e9f2dcf4..813b7debe6907 100644 --- a/src/persist-client/src/internal/state.rs +++ b/src/persist-client/src/internal/state.rs @@ -1430,7 +1430,7 @@ where let existed = self.leased_readers.remove(reader_id).is_some(); if existed { - // TODO(materialize#22789): Re-enable this + // TODO(database-issues#6885): Re-enable this // // Temporarily disabling this because we think it might be the cause // of the remap since bug. Specifically, a clusterd process has a @@ -1463,7 +1463,7 @@ where let existed = self.critical_readers.remove(reader_id).is_some(); if existed { - // TODO(materialize#22789): Re-enable this + // TODO(database-issues#6885): Re-enable this // // Temporarily disabling this because we think it might be the cause // of the remap since bug. Specifically, a clusterd process has a @@ -2609,7 +2609,7 @@ pub(crate) mod tests { state.collections.expire_leased_reader(&reader2), Continue(true) ); - // TODO(materialize#22789): expiry temporarily doesn't advance since + // TODO(database-issues#6885): expiry temporarily doesn't advance since // Switch this assertion back when we re-enable this. // // assert_eq!(state.collections.trace.since(), &Antichain::from_elem(10)); @@ -2620,7 +2620,7 @@ pub(crate) mod tests { state.collections.expire_leased_reader(&reader3), Continue(true) ); - // TODO(materialize#22789): expiry temporarily doesn't advance since + // TODO(database-issues#6885): expiry temporarily doesn't advance since // Switch this assertion back when we re-enable this. // // assert_eq!(state.collections.trace.since(), &Antichain::from_elem(10)); diff --git a/src/persist-client/src/internal/state_diff.rs b/src/persist-client/src/internal/state_diff.rs index c6bb7f0709bd5..c49af7e5521bb 100644 --- a/src/persist-client/src/internal/state_diff.rs +++ b/src/persist-client/src/internal/state_diff.rs @@ -1383,7 +1383,7 @@ mod tests { // Regression test for the apply_diffs_spine special case that sniffs out an // insert, applies it, and then lets the remaining diffs (if any) fall - // through to the rest of the code. See materialize#15493. + // through to the rest of the code. See database-issues#4431. #[mz_ore::test] fn regression_15493_sniff_insert() { fn hb(lower: u64, upper: u64, len: usize) -> HollowBatch { diff --git a/src/persist-client/src/internal/state_versions.rs b/src/persist-client/src/internal/state_versions.rs index a7da6f82bd4fe..57017f1063b16 100644 --- a/src/persist-client/src/internal/state_versions.rs +++ b/src/persist-client/src/internal/state_versions.rs @@ -1220,7 +1220,7 @@ mod tests { use super::*; - /// Regression test for (part of) materialize#17752, where an interrupted + /// Regression test for (part of) database-issues#5170, where an interrupted /// `bin/environmentd --reset` resulted in panic in persist usage code. #[mz_persist_proc::test(tokio::test)] #[cfg_attr(miri, ignore)] // unsupported operation: returning ready events from epoll_wait is not yet implemented diff --git a/src/persist-client/src/internal/trace.rs b/src/persist-client/src/internal/trace.rs index d591f5a20490a..2b4a32940e30d 100644 --- a/src/persist-client/src/internal/trace.rs +++ b/src/persist-client/src/internal/trace.rs @@ -514,7 +514,7 @@ impl Trace { } /// The same as [Self::push_batch] but without the `FueledMergeReq`s, which - /// account for a surprising amount of cpu in prod. materialize#18368 + /// account for a surprising amount of cpu in prod. database-issues#5411 pub(crate) fn push_batch_no_merge_reqs(&mut self, batch: HollowBatch) { self.spine.insert(batch, &mut SpineLog::Disabled); } @@ -1172,7 +1172,7 @@ impl Spine { // Otherwise, we'll need to introduce fake updates to move merges // along. - // TODO: Tracked in materialize#29459. Once we've finished deleting the legacy + // TODO: Tracked in database-issues#8530. Once we've finished deleting the legacy // spine-diff code, push an empty batch here to move things along. return false; } diff --git a/src/persist-client/src/internals_bench.rs b/src/persist-client/src/internals_bench.rs index c4af4b85b8cbb..180fb6d369c4e 100644 --- a/src/persist-client/src/internals_bench.rs +++ b/src/persist-client/src/internals_bench.rs @@ -32,7 +32,7 @@ pub fn trace_push_batch_one_iter(num_batches: usize) { // A single non-empty batch followed by a large number of empty batches // and no compaction. This is a particularly problematic workload for // our fork of Spine which came up during deserialization of State in - // materialize#17214. + // database-issues#4985. // // Other, much better handled, workloads include all empty or all // non-empty. diff --git a/src/persist-client/src/lib.rs b/src/persist-client/src/lib.rs index 2ed63eec4aeb0..bd30a334de8e1 100644 --- a/src/persist-client/src/lib.rs +++ b/src/persist-client/src/lib.rs @@ -1770,7 +1770,7 @@ mod tests { ); } - // Regression test for materialize#12131. Snapshot with as_of >= upper would + // Regression test for database-issues#3523. Snapshot with as_of >= upper would // immediately return the data currently available instead of waiting for // upper to advance past as_of. #[mz_persist_proc::test(tokio::test)] diff --git a/src/persist-client/src/read.rs b/src/persist-client/src/read.rs index 1f183b25046b9..25fae94c357a4 100644 --- a/src/persist-client/src/read.rs +++ b/src/persist-client/src/read.rs @@ -605,7 +605,7 @@ where .downgrade_since(&self.reader_id, outstanding_seqno, new_since, heartbeat_ts) .await; - // Debugging for materialize#15937. + // Debugging for database-issues#4590. if let Some(outstanding_seqno) = outstanding_seqno { let seqnos_held = _seqno.0.saturating_sub(outstanding_seqno.0); // We get just over 1 seqno-per-second on average for a shard in @@ -1336,7 +1336,7 @@ mod tests { // Verifies the semantics of `SeqNo` leases + checks dropping `LeasedBatchPart` semantics. #[mz_persist_proc::test(tokio::test)] - #[cfg_attr(miri, ignore)] // https://github.com/MaterializeInc/materialize/issues/19983 + #[cfg_attr(miri, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5964 async fn seqno_leases(dyncfgs: ConfigUpdates) { let mut data = vec![]; for i in 0..20 { diff --git a/src/persist-client/src/rpc.rs b/src/persist-client/src/rpc.rs index 8215d04a5f9dc..7a4bcca1414bc 100644 --- a/src/persist-client/src/rpc.rs +++ b/src/persist-client/src/rpc.rs @@ -213,7 +213,7 @@ impl GrpcPubSubClient { // Once enabled, the PubSub server cannot be disabled or otherwise // reconfigured. So we wait for at least one configuration sync to // complete. This gives `environmentd` at least one chance to update - // PubSub configuration parameters. See materialize#23869 for details. + // PubSub configuration parameters. See database-issues#7168 for details. config.persist_cfg.configs_synced_once().await; let mut is_first_connection_attempt = true; diff --git a/src/persist-client/src/usage.rs b/src/persist-client/src/usage.rs index 7777670019ec1..62070516e13cd 100644 --- a/src/persist-client/src/usage.rs +++ b/src/persist-client/src/usage.rs @@ -82,7 +82,7 @@ pub struct ShardUsageAudit { /// SeqNo pub not_leaked_not_referenced_bytes: u64, /// 2b: Data that is eligible for reclamation by a (future) leaked blob - /// cleanup task (materialize#17322). + /// cleanup task (database-issues#5018). /// /// Possible causes: /// - A batch or rollup written by a process which crashed (or was rolled) @@ -102,7 +102,7 @@ impl ShardUsageAudit { } /// 2a: Data that would not be reclaimed by a (future) leaked blob - /// cleanup task (materialize#17322). + /// cleanup task (database-issues#5018). pub fn not_leaked_bytes(&self) -> u64 { self.referenced_bytes() + self.not_leaked_not_referenced_bytes } @@ -1227,7 +1227,7 @@ mod tests { .run("1 0/1 0/1 0/1 0/1"); } - /// A regression test for (part of) materialize#17752, which led to seeing the "blob + /// A regression test for (part of) database-issues#5170, which led to seeing the "blob /// inputs should be cumulative" should be cumulative panic in /// staging/canary. #[mz_ore::test] @@ -1244,7 +1244,7 @@ mod tests { .run("3 0/3 0/3 3/0 0/0"); } - /// Regression test for (part of) materialize#17752, where an interrupted + /// Regression test for (part of) database-issues#5170, where an interrupted /// `bin/environmentd --reset` resulted in panic in persist usage code. /// /// This also tests a (hypothesized) race that's possible in prod where an diff --git a/src/persist/src/postgres.rs b/src/persist/src/postgres.rs index 384bdf82f0163..eee37991d8480 100644 --- a/src/persist/src/postgres.rs +++ b/src/persist/src/postgres.rs @@ -194,7 +194,7 @@ impl PostgresConsensus { // these tombstones accumulate, scanning over the table will take increasingly and // prohibitively long. // - // See: https://github.com/MaterializeInc/materialize/issues/13975 + // See: https://api.github.com/repos/MaterializeInc/database-issues/issues/4001 // See: https://www.cockroachlabs.com/docs/stable/configure-zone.html#variables match client .batch_execute(&format!( diff --git a/src/persist/src/s3.rs b/src/persist/src/s3.rs index b45d70c082a00..96f2e5c2dc31c 100644 --- a/src/persist/src/s3.rs +++ b/src/persist/src/s3.rs @@ -1029,7 +1029,7 @@ mod tests { use super::*; #[mz_ore::test(tokio::test(flavor = "multi_thread"))] - #[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18898 + #[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5586 #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `TLS_method` on OS `linux` async fn s3_blob() -> Result<(), ExternalError> { let config = match S3BlobConfig::new_for_test().await? { diff --git a/src/repr/src/explain.rs b/src/repr/src/explain.rs index 5188253887d67..c209cca4bec5a 100644 --- a/src/repr/src/explain.rs +++ b/src/repr/src/explain.rs @@ -243,7 +243,7 @@ pub enum Explainee { Index(GlobalId), /// An object that will be served using a dataflow. /// - /// This variant is deprecated and will be removed in materialize#18089. + /// This variant is deprecated and will be removed in database-issues#5301. Dataflow(GlobalId), /// The object to be explained is a one-off query and may or may not be /// served using a dataflow. diff --git a/src/repr/src/row/encoding.rs b/src/repr/src/row/encoding.rs index a9b9a00d9f7c9..1d676ad83fce4 100644 --- a/src/repr/src/row/encoding.rs +++ b/src/repr/src/row/encoding.rs @@ -354,7 +354,7 @@ impl TryFrom<&ProtoRow> for Row { fn try_from(x: &ProtoRow) -> Result { // TODO: Try to pre-size this. - // see https://github.com/MaterializeInc/materialize/issues/12631 + // see https://api.github.com/repos/MaterializeInc/database-issues/issues/3640 let mut row = Row::default(); let mut packer = row.packer(); for d in x.datums.iter() { @@ -372,7 +372,7 @@ impl RustType for Row { fn from_proto(proto: ProtoRow) -> Result { // TODO: Try to pre-size this. - // see https://github.com/MaterializeInc/materialize/issues/12631 + // see https://api.github.com/repos/MaterializeInc/database-issues/issues/3640 let mut row = Row::default(); let mut packer = row.packer(); for d in proto.datums.iter() { diff --git a/src/repr/src/scalar.rs b/src/repr/src/scalar.rs index f4dbcf9f9a59f..a9611690ffa8b 100644 --- a/src/repr/src/scalar.rs +++ b/src/repr/src/scalar.rs @@ -3556,7 +3556,7 @@ impl ScalarType { ScalarType::Array(elem) => Ok(elem.array_of_self_elem_type()?), - // https://github.com/MaterializeInc/materialize/issues/7613 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/2360 t @ (ScalarType::Char { .. } // not sensible to put in arrays | ScalarType::Map { .. } diff --git a/src/repr/tests/strconv.rs b/src/repr/tests/strconv.rs index 04cb96190de30..41679837cc2c4 100644 --- a/src/repr/tests/strconv.rs +++ b/src/repr/tests/strconv.rs @@ -109,7 +109,7 @@ fn test_parse_time() { ); run_test_parse_time("01:02", NaiveTime::from_hms_opt(1, 2, 0).unwrap()); - // Regression for materialize#6272. + // Regression for database-issues#1933. run_test_parse_time( "9::60", NaiveTime::from_hms_nano_opt(9, 0, 59, 1_000_000_000).unwrap(), diff --git a/src/service/src/grpc.rs b/src/service/src/grpc.rs index 62786ad999d18..ddaeeb4ebeb51 100644 --- a/src/service/src/grpc.rs +++ b/src/service/src/grpc.rs @@ -339,7 +339,7 @@ where // TODO(benesch): rather than blindly dropping the existing cancellation // token, we should check epochs, and only drop the existing connection // if it is at a lower epoch. - // See: https://github.com/MaterializeInc/materialize/issues/13377 + // See: https://api.github.com/repos/MaterializeInc/database-issues/issues/3840 let (cancel_tx, mut cancel_rx) = oneshot::channel(); *self.state.cancel_tx.lock().await = cancel_tx; diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index c5c0ddd744d33..144d62076375f 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -544,7 +544,7 @@ impl<'a> Parser<'a> { // "date". // // Note: the maybe! block here does swallow valid parsing errors - // See for more details + // See for more details maybe!(self.maybe_parse(|parser| { let data_type = parser.parse_data_type()?; if data_type.to_string().as_str() == "interval" { @@ -763,7 +763,7 @@ impl<'a> Parser<'a> { let over = if self.peek_keyword(OVER) || self.peek_keyword(IGNORE) || self.peek_keyword(RESPECT) { // TBD: support window names (`OVER mywin`) in place of inline specification - // https://github.com/MaterializeInc/materialize/issues/19755 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/5882 let ignore_nulls = self.parse_keywords(&[IGNORE, NULLS]); let respect_nulls = self.parse_keywords(&[RESPECT, NULLS]); diff --git a/src/sql-parser/tests/testdata/explain b/src/sql-parser/tests/testdata/explain index 89a9ea0629fb8..295568931c86e 100644 --- a/src/sql-parser/tests/testdata/explain +++ b/src/sql-parser/tests/testdata/explain @@ -158,7 +158,7 @@ EXPLAIN OPTIMIZED PLAN AS TEXT FOR WITH a AS (SELECT 1) SELECT * FROM a => ExplainPlan(ExplainPlanStatement { stage: Some(GlobalPlan), with_options: [], format: Some(Text), explainee: Select(SelectStatement { query: Query { ctes: Simple([Cte { alias: TableAlias { name: Ident("a"), columns: [], strict: false }, id: (), query: Query { ctes: Simple([]), body: Select(Select { distinct: None, projection: [Expr { expr: Value(Number("1")), alias: None }], from: [], selection: None, group_by: [], having: None, options: [] }), order_by: [], limit: None, offset: None } }]), body: Select(Select { distinct: None, projection: [Wildcard], from: [TableWithJoins { relation: Table { name: Name(UnresolvedItemName([Ident("a")])), alias: None }, joins: [] }], selection: None, group_by: [], having: None, options: [] }), order_by: [], limit: None, offset: None }, as_of: None }, false) }) -# regression test for materialize#16029 +# regression test for database-issues#4624 parse-statement EXPLAIN WITH a AS (SELECT 1) SELECT * FROM a ---- diff --git a/src/sql/src/ast/transform.rs b/src/sql/src/ast/transform.rs index e8af8980cdad2..0a460c3cafcf4 100644 --- a/src/sql/src/ast/transform.rs +++ b/src/sql/src/ast/transform.rs @@ -155,7 +155,7 @@ pub fn create_stmt_rename(create_stmt: &mut Statement, to_item_name: String | Statement::CreateWebhookSource(CreateWebhookSourceStatement { name, .. }) => { // The last name in an ItemName is the item name. The item name // does not have a fixed index. - // TODO: https://github.com/MaterializeInc/materialize/issues/5591 + // TODO: https://api.github.com/repos/MaterializeInc/database-issues/issues/1721 let item_name_len = name.0.len() - 1; name.0[item_name_len] = Ident::new_unchecked(to_item_name); } @@ -185,7 +185,7 @@ pub fn create_stmt_rename_refs( if item_name.0 == from_item.0 { // The last name in an ItemName is the item name. The item name // does not have a fixed index. - // TODO: https://github.com/MaterializeInc/materialize/issues/5591 + // TODO: https://api.github.com/repos/MaterializeInc/database-issues/issues/1721 let item_name_len = item_name.0.len() - 1; item_name.0[item_name_len] = Ident::new_unchecked(to_item_name.clone()); } diff --git a/src/sql/src/catalog.rs b/src/sql/src/catalog.rs index 91ef7f0084dc8..468356ad8fc51 100644 --- a/src/sql/src/catalog.rs +++ b/src/sql/src/catalog.rs @@ -362,7 +362,7 @@ pub struct CatalogConfig { /// A random integer associated with this instance of the catalog. /// /// NOTE(benesch): this is only necessary for producing unique Kafka sink - /// topics. Perhaps we can remove this when materialize#2915 is complete. + /// topics. Perhaps we can remove this when database-issues#977 is complete. pub nonce: u64, /// A persistent ID associated with the environment. pub environment_id: EnvironmentId, @@ -693,7 +693,7 @@ impl CatalogItemType { /// /// We don't presently construct types that mirror relational objects, /// though we likely will need to in the future for full PostgreSQL - /// compatibility (see materialize#23789). For now, we use this method to + /// compatibility (see database-issues#7142). For now, we use this method to /// prevent creating types and relational objects that have the same name, so /// that it is a backwards compatible change in the future to introduce a /// type named after each relational object in the system. diff --git a/src/sql/src/names.rs b/src/sql/src/names.rs index 7019dc314b068..e65c7c1d3ebd3 100644 --- a/src/sql/src/names.rs +++ b/src/sql/src/names.rs @@ -322,7 +322,7 @@ impl From> for ResolvedDatabaseSpecifier { /* * TODO(jkosh44) It's possible that in order to fix - * https://github.com/MaterializeInc/materialize/issues/8805 we will need to assign temporary + * https://api.github.com/repos/MaterializeInc/database-issues/issues/2689 we will need to assign temporary * schemas unique Ids. If/when that happens we can remove this enum and refer to all schemas by * their Id. */ @@ -1203,7 +1203,7 @@ pub enum CommentObjectId { /// relations. // NOTE(benesch,sploiselle): The fact that some names are looked up in multiple // namespaces is a bit dubious, and stems from the fact that we don't -// automatically create types for relations (see materialize#23789). It's possible that we +// automatically create types for relations (see database-issues#7142). It's possible that we // don't allow names to be looked up in multiple namespaces (i.e., this becomes // `enum ItemResolutionNamespace`), but it's also possible that the design of // the `DOC ON TYPE` option means we're forever stuck with this complexity. @@ -2058,7 +2058,7 @@ impl<'a> Fold for NameResolver<'a> { // In `DOC ON TYPE ...`, the type can refer to either a type or // a relation. // - // It's possible this will get simpler once materialize#23789 is fixed. See + // It's possible this will get simpler once database-issues#7142 is fixed. See // the comment on `ItemResolutionConfig` for details. ItemResolutionConfig { functions: false, diff --git a/src/sql/src/plan.rs b/src/sql/src/plan.rs index fce723d743455..ddf297570fa03 100644 --- a/src/sql/src/plan.rs +++ b/src/sql/src/plan.rs @@ -1326,7 +1326,7 @@ pub struct CommentPlan { pub object_id: CommentObjectId, /// A sub-component of the object that this comment is associated with, e.g. a column. /// - /// TODO(parkmycar): . + /// TODO(parkmycar): . pub sub_component: Option, /// The comment itself. If `None` that indicates we should clear the existing comment. pub comment: Option, diff --git a/src/sql/src/plan/expr.rs b/src/sql/src/plan/expr.rs index 6bd3333679f9b..316fd15cd13b1 100644 --- a/src/sql/src/plan/expr.rs +++ b/src/sql/src/plan/expr.rs @@ -218,7 +218,7 @@ pub enum HirScalarExpr { /// * 0 rows, return NULL /// * 1 row, return the value of that row /// * >1 rows, the sql spec says we should throw an error but we can't - /// (see ) + /// (see ) /// so instead we return all the rows. /// If there are multiple `Select` expressions in a single SQL query, the result is that we take the product of all of them. /// This is counter to the spec, but is consistent with eg postgres' treatment of multiple set-returning-functions diff --git a/src/sql/src/plan/lowering.rs b/src/sql/src/plan/lowering.rs index b344eea580fb4..ba09f597491e3 100644 --- a/src/sql/src/plan/lowering.rs +++ b/src/sql/src/plan/lowering.rs @@ -132,9 +132,9 @@ struct CteDesc { #[derive(Debug)] pub struct Config { - /// Enable outer join lowering implemented in materialize#22343. + /// Enable outer join lowering implemented in database-issues#6747. pub enable_new_outer_join_lowering: bool, - /// Enable outer join lowering implemented in materialize#25340. + /// Enable outer join lowering implemented in database-issues#7561. pub enable_variadic_left_join_lowering: bool, /// Enable the extra null filter implemented in materialize#28018. pub enable_outer_join_null_filter: bool, @@ -1375,7 +1375,7 @@ impl HirScalarExpr { // ORDER BY columns. // - The currently always captures the entire original row. This should // improve when we make `ProjectionPushdown` smarter, see - // https://github.com/MaterializeInc/materialize/issues/17522 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/5090 // // TODO: // We should probably introduce some dedicated Datum constructor functions instead of `row` @@ -1994,11 +1994,11 @@ fn attempt_outer_equijoin( id_gen: &mut mz_ore::id_gen::IdGen, context: &Context, ) -> Result, PlanError> { - // TODO(materialize#22581): In theory, we can be smarter and also handle `on` + // TODO(database-issues#6827): In theory, we can be smarter and also handle `on` // predicates that reference subqueries as long as these subqueries don't // reference `left` and `right` at the same time. // - // TODO(materialize#22582): This code can be improved as follows: + // TODO(database-issues#6828): This code can be improved as follows: // // 1. Move the `canonicalize_predicates(...)` call to `applied_to`. // 2. Use the canonicalized `on` predicate in the non-equijoin based diff --git a/src/sql/src/plan/lowering/variadic_left.rs b/src/sql/src/plan/lowering/variadic_left.rs index 6c594a3c3e6a9..029b2e45659b1 100644 --- a/src/sql/src/plan/lowering/variadic_left.rs +++ b/src/sql/src/plan/lowering/variadic_left.rs @@ -420,7 +420,7 @@ fn decompose_equations(predicate: &MirScalarExpr) -> Option> // Ensure that every rhs column c2 appears only once. Otherwise, we have at // least two lhs columns c1 and c1' that are rendered equal by the same c2 // column. The VOJ lowering will then produce a plan that will incorrectly - // push down a local filter c1 = c1' to the lhs (see materialize#26707). + // push down a local filter c1 = c1' to the lhs (see database-issues#7892). if equations.iter().duplicates_by(|(_, c)| c).next().is_some() { return None; } diff --git a/src/sql/src/plan/query.rs b/src/sql/src/plan/query.rs index 0c012978d6d6c..ec98f689db35b 100644 --- a/src/sql/src/plan/query.rs +++ b/src/sql/src/plan/query.rs @@ -3192,7 +3192,7 @@ fn plan_table_alias(mut scope: Scope, alias: Option<&TableAlias>) -> Result( let (ignore_nulls, order_by_exprs, col_orders, window_frame, partition_by) = plan_window_function_common(ecx, &f.name, &f.over)?; - // https://github.com/MaterializeInc/materialize/issues/22268 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6720 match (&window_frame.start_bound, &window_frame.end_bound) { ( mz_expr::WindowFrameBound::UnboundedPreceding, @@ -5167,7 +5167,7 @@ fn plan_function<'a>( } if ignore_nulls { - // https://github.com/MaterializeInc/materialize/issues/22272 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6722 // If we ever add support for ignore_nulls for a window aggregate, then don't // forget to also update HIR EXPLAIN. bail_unsupported!(IGNORE_NULLS_ERROR_MSG); @@ -5176,7 +5176,7 @@ fn plan_function<'a>( let aggregate_expr = plan_aggregate_common(ecx, f)?; if aggregate_expr.distinct { - // https://github.com/MaterializeInc/materialize/issues/22015 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6626 bail_unsupported!("DISTINCT in window aggregates"); } @@ -5590,7 +5590,7 @@ fn plan_window_frame( } // RANGE is only supported in the default frame - // https://github.com/MaterializeInc/materialize/issues/21934 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6585 if units == mz_expr::WindowFrameUnits::Range && (start_bound != UnboundedPreceding || end_bound != CurrentRow) { diff --git a/src/sql/src/plan/statement.rs b/src/sql/src/plan/statement.rs index 71845502c2bde..84fd5808e18e7 100644 --- a/src/sql/src/plan/statement.rs +++ b/src/sql/src/plan/statement.rs @@ -258,7 +258,7 @@ pub fn describe( /// [`crate::pure::purify_create_materialized_view_options`]. /// /// TODO: sinks do not currently obey this rule, which is a bug -/// +/// /// /// The returned plan is tied to the state of the provided catalog. If the state /// of the catalog changes after planning, the validity of the plan is not @@ -771,7 +771,7 @@ impl<'a> StatementContext<'a> { // it. This should be safe enough because our types are wide enough // to support the maximum possible precision. // - // See: https://github.com/MaterializeInc/materialize/issues/10837 + // See: https://api.github.com/repos/MaterializeInc/database-issues/issues/3179 match &mut ty { mz_pgrepr::Type::Interval { constraints } => *constraints = None, mz_pgrepr::Type::Time { precision } => *precision = None, diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index c441820f57b35..d987f9cb0a563 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -947,7 +947,7 @@ pub fn plan_create_source( // Apply user-specified key constraint if let Some(KeyConstraint::PrimaryKeyNotEnforced { columns }) = key_constraint.clone() { // Don't remove this without addressing - // https://github.com/MaterializeInc/materialize/issues/15272. + // https://api.github.com/repos/MaterializeInc/database-issues/issues/4371. scx.require_feature_flag(&vars::ENABLE_PRIMARY_KEY_NOT_ENFORCED)?; let key_columns = columns @@ -2320,9 +2320,9 @@ pub fn plan_view( } = query::plan_root_query(scx, query.clone(), QueryLifetime::View)?; // We get back a trivial finishing, because `plan_root_query` applies the given finishing. // Note: Earlier, we were thinking to maybe persist the finishing information with the view - // here to help with materialize#724. However, in the meantime, there might be a better - // approach to solve materialize#724: - // https://github.com/MaterializeInc/materialize/issues/724#issuecomment-1688293709 + // here to help with database-issues#236. However, in the meantime, there might be a better + // approach to solve database-issues#236: + // https://api.github.com/repos/MaterializeInc/database-issues/issues/236#issuecomment-1688293709 assert!(finishing.is_trivial(expr.arity())); expr.bind_parameters(params)?; @@ -3198,7 +3198,7 @@ fn key_constraint_err(desc: &RelationDesc, user_keys: &[ColumnName]) -> PlanErro } /// Creating this by hand instead of using generate_extracted_config! macro -/// because the macro doesn't support parameterized enums. See +/// because the macro doesn't support parameterized enums. See #[derive(Debug, Default, PartialEq, Clone)] pub struct CsrConfigOptionExtracted { seen: ::std::collections::BTreeSet>, @@ -5251,7 +5251,7 @@ fn plan_retain_history( // A zero duration has already been converted to `None` by `OptionalDuration` (and means // disable compaction), and should never occur here. Furthermore, some things actually do // break when this is set to real zero: - // https://github.com/MaterializeInc/materialize/issues/13221. + // https://api.github.com/repos/MaterializeInc/database-issues/issues/3798. Some(Duration::ZERO) => Err(PlanError::InvalidOptionValue { option_name: "RETAIN HISTORY".to_string(), err: Box::new(PlanError::Unstructured( @@ -6795,7 +6795,7 @@ pub fn plan_comment( // store a `usize` which would be a `Uint8`. We guard against a safe conversion here because // it's the easiest place to raise an error. // - // TODO(parkmycar): https://github.com/MaterializeInc/materialize/issues/22246. + // TODO(parkmycar): https://api.github.com/repos/MaterializeInc/database-issues/issues/6711. if let Some(p) = column_pos { i32::try_from(p).map_err(|_| PlanError::TooManyColumns { max_num_columns: MAX_NUM_COLUMNS, diff --git a/src/sql/src/plan/statement/ddl/connection.rs b/src/sql/src/plan/statement/ddl/connection.rs index df63506bc4ea4..6af487f037a4e 100644 --- a/src/sql/src/plan/statement/ddl/connection.rs +++ b/src/sql/src/plan/statement/ddl/connection.rs @@ -337,7 +337,7 @@ impl ConnectionOptionExtracted { password: self.password.map(|secret| secret.into()), }); - // TODO we should move to self.port being unsupported if aws_privatelink is some, see + // TODO we should move to self.port being unsupported if aws_privatelink is some, see if let Some(privatelink) = self.aws_privatelink.as_ref() { if privatelink.port.is_some() { sql_bail!("invalid CONNECTION: PORT in AWS PRIVATELINK is only supported for kafka") @@ -380,7 +380,7 @@ impl ConnectionOptionExtracted { Some(m) => sql_bail!("invalid CONNECTION: unknown SSL MODE {}", m.quoted()), }; - // TODO we should move to self.port being unsupported if aws_privatelink is some, see + // TODO we should move to self.port being unsupported if aws_privatelink is some, see if let Some(privatelink) = self.aws_privatelink.as_ref() { if privatelink.port.is_some() { sql_bail!("invalid CONNECTION: PORT in AWS PRIVATELINK is only supported for kafka") @@ -470,7 +470,7 @@ impl ConnectionOptionExtracted { Some(m) => sql_bail!("invalid CONNECTION: unknown SSL MODE {}", m.quoted()), }; - // TODO we should move to self.port being unsupported if aws_privatelink is some, see + // TODO we should move to self.port being unsupported if aws_privatelink is some, see if let Some(privatelink) = self.aws_privatelink.as_ref() { if privatelink.port.is_some() { sql_bail!("invalid CONNECTION: PORT in AWS PRIVATELINK is only supported for kafka") @@ -719,7 +719,7 @@ fn plan_kafka_security( // Materialize previously rejected with an error of "SASL // mechanism must be uppercase." This was deeply frustarting for // users who were not familiar with identifier case folding - // rules. See materialize#22205. + // rules. See database-issues#6693. mechanism: mechanism.to_uppercase(), username: username.clone(), password: Some((*password).into()), diff --git a/src/sql/src/plan/statement/scl.rs b/src/sql/src/plan/statement/scl.rs index 01ad0a851f764..60e16d5a387bb 100644 --- a/src/sql/src/plan/statement/scl.rs +++ b/src/sql/src/plan/statement/scl.rs @@ -284,7 +284,7 @@ pub fn describe_execute( // means if the statement is now invalid due to an object having been dropped, // describe is unable to notice that. This is currently an existing problem // with prepared statements over pgwire as well, so we can leave this for now. - // See materialize#8397. + // See database-issues#2563. Ok(plan_execute_desc(scx, stmt)?.0.clone()) } diff --git a/src/sql/src/plan/statement/show.rs b/src/sql/src/plan/statement/show.rs index 2f4981590933b..f0fefe8b49f0f 100644 --- a/src/sql/src/plan/statement/show.rs +++ b/src/sql/src/plan/statement/show.rs @@ -430,7 +430,7 @@ fn show_subsources<'a>( query_filter.push(format!("subsources.schema_id = '{schema_spec}'")); } - // TODO(materialize#28430): this looks in both directions for subsources as long as + // TODO(database-issues#8322): this looks in both directions for subsources as long as // progress collections still exist let query = format!( "SELECT DISTINCT @@ -1082,7 +1082,7 @@ fn humanize_sql_for_show_create( } CreateSourceConnection::LoadGenerator { .. } if !curr_references.is_empty() => { // Load generator sources with any references only support - // `FOR ALL TABLES`. However, this would change if materialize#26765 + // `FOR ALL TABLES`. However, this would change if database-issues#7911 // landed. curr_references.clear(); stmt.external_references = Some(ExternalReferences::All); diff --git a/src/sql/src/plan/transform_ast.rs b/src/sql/src/plan/transform_ast.rs index 5ac9669c68cd0..3955256c1a763 100644 --- a/src/sql/src/plan/transform_ast.rs +++ b/src/sql/src/plan/transform_ast.rs @@ -215,7 +215,7 @@ impl<'a> FuncRewriter<'a> { // is known to accumulate problematic amounts of error. The numerically // stable variants, the most well-known of which is Welford's, are // however difficult to implement inside of Differential Dataflow, as - // they do not obviously support retractions efficiently (materialize#1240). + // they do not obviously support retractions efficiently (database-issues#436). // // The code below converts var_samp(x) into // diff --git a/src/sql/src/plan/transform_expr.rs b/src/sql/src/plan/transform_expr.rs index e8e2f50d82f58..2e6ca9ce4bd87 100644 --- a/src/sql/src/plan/transform_expr.rs +++ b/src/sql/src/plan/transform_expr.rs @@ -299,7 +299,7 @@ fn column_type( /// For now, we fuse only value window function calls (`WindowExprType::Value`). /// TODO: We should consider fusing the other types later: /// - `WindowExprType::Aggregate`: Would be great to also fuse these soon, see e.g. -/// +/// /// - `WindowExprType::Scalar`: probably won't need to fuse these for a long time.) /// /// For now, we can fuse value window function calls where the diff --git a/src/sql/src/pure.rs b/src/sql/src/pure.rs index c0a147995e7f3..9577ed5be822a 100644 --- a/src/sql/src/pure.rs +++ b/src/sql/src/pure.rs @@ -1925,7 +1925,7 @@ pub fn generate_subsource_statements( subsource_stmts } PurifiedExportDetails::Kafka { .. } => { - // TODO: as part of materialize#28430, Kafka sources will begin + // TODO: as part of database-issues#8322, Kafka sources will begin // producing data––we'll need to understand the schema // of the output here. assert!( diff --git a/src/sql/src/session/vars.rs b/src/sql/src/session/vars.rs index e2e1869d2f788..27c3ded0a6355 100644 --- a/src/sql/src/session/vars.rs +++ b/src/sql/src/session/vars.rs @@ -836,7 +836,7 @@ impl SessionVars { } } -// TODO(materialize#27285) remove together with `compat_translate` +// TODO(database-issues#8069) remove together with `compat_translate` pub const OLD_CATALOG_SERVER_CLUSTER: &str = "mz_introspection"; pub const OLD_AUTO_ROUTE_CATALOG_QUERIES: &str = "auto_route_introspection_queries"; @@ -846,7 +846,7 @@ pub const OLD_AUTO_ROUTE_CATALOG_QUERIES: &str = "auto_route_introspection_queri /// This method was introduced to gracefully handle the rename of the `mz_introspection` cluster to /// `mz_cluster_server`. The plan is to remove it once all users have migrated to the new name. The /// debug logs will be helpful for checking this in production. -// TODO(materialize#27285) remove this after sufficient time has passed +// TODO(database-issues#8069) remove this after sufficient time has passed fn compat_translate<'a, 'b>(name: &'a str, input: VarInput<'b>) -> (&'a str, VarInput<'b>) { if name == CLUSTER.name() { if let Ok(value) = CLUSTER.parse(input) { diff --git a/src/sqllogictest/src/runner.rs b/src/sqllogictest/src/runner.rs index af1dcdb207681..cacbd5a1a5ede 100644 --- a/src/sqllogictest/src/runner.rs +++ b/src/sqllogictest/src/runner.rs @@ -1356,7 +1356,7 @@ impl<'a> RunnerInner<'a> { match output { Ok(_) => { if self.auto_transactions && !*in_transaction { - // No ISOLATION LEVEL SERIALIZABLE because of materialize#18136 + // No ISOLATION LEVEL SERIALIZABLE because of database-issues#5323 self.client.execute("BEGIN", &[]).await?; *in_transaction = true; } diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index 34407a348f414..d7ac4aa9fb455 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -1441,7 +1441,7 @@ where let (id, description, metadata) = data?; // should be replaced with real introspection - // (https://github.com/MaterializeInc/materialize/issues/14266) + // (https://api.github.com/repos/MaterializeInc/database-issues/issues/4078) // but for now, it's helpful to have this mapping written down // somewhere debug!( diff --git a/src/storage-controller/src/collection_mgmt.rs b/src/storage-controller/src/collection_mgmt.rs index a5b26e1dc2795..5e27cdcd33d9f 100644 --- a/src/storage-controller/src/collection_mgmt.rs +++ b/src/storage-controller/src/collection_mgmt.rs @@ -1275,7 +1275,7 @@ async fn prepare_append_only_introspection_collection( // NOTE(aljoscha): We never remove from these // collections. Someone, at some point needs to // think about that! Issue: - // https://github.com/MaterializeInc/materialize/issues/25696 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/7666 } // TODO(jkosh44) Handle these here instead of in the controller. diff --git a/src/storage-controller/src/instance.rs b/src/storage-controller/src/instance.rs index 4224b4ce27935..3e2417da4e1fa 100644 --- a/src/storage-controller/src/instance.rs +++ b/src/storage-controller/src/instance.rs @@ -38,7 +38,7 @@ use crate::history::CommandHistory; /// /// Encapsulates communication with replicas in this instance, and their rehydration. /// -/// Note that storage objects (sources and sinks) don't currently support replication (materialize#17418). +/// Note that storage objects (sources and sinks) don't currently support replication (database-issues#5051). /// An instance can have muliple replicas connected, but only if it has no storage objects /// installed. Attempting to install storage objects on multi-replica instances, or attempting to /// add more than one replica to instances that have storage objects installed, is illegal and will diff --git a/src/storage-controller/src/lib.rs b/src/storage-controller/src/lib.rs index fe3e9fe0db304..e7b8f4928d438 100644 --- a/src/storage-controller/src/lib.rs +++ b/src/storage-controller/src/lib.rs @@ -514,7 +514,7 @@ where async move { let (id, description, metadata) = data?; - // should be replaced with real introspection (https://github.com/MaterializeInc/materialize/issues/14266) + // should be replaced with real introspection (https://api.github.com/repos/MaterializeInc/database-issues/issues/4078) // but for now, it's helpful to have this mapping written down somewhere debug!( "mapping GlobalId={} to remap shard ({:?}), data shard ({}), status shard ({:?})", diff --git a/src/storage-operators/src/s3_oneshot_sink/pgcopy.rs b/src/storage-operators/src/s3_oneshot_sink/pgcopy.rs index ad4b38cf86138..889e917a7853b 100644 --- a/src/storage-operators/src/s3_oneshot_sink/pgcopy.rs +++ b/src/storage-operators/src/s3_oneshot_sink/pgcopy.rs @@ -207,7 +207,7 @@ mod tests { } #[mz_ore::test(tokio::test(flavor = "multi_thread"))] - #[cfg_attr(coverage, ignore)] // https://github.com/MaterializeInc/materialize/issues/18898 + #[cfg_attr(coverage, ignore)] // https://api.github.com/repos/MaterializeInc/database-issues/issues/5586 #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `TLS_method` on OS `linux` async fn test_multiple_files() -> Result<(), anyhow::Error> { let sdk_config = mz_aws_util::defaults().load().await; diff --git a/src/storage-types/src/connections.rs b/src/storage-types/src/connections.rs index d8a5a95751a25..77fd353a5afe5 100644 --- a/src/storage-types/src/connections.rs +++ b/src/storage-types/src/connections.rs @@ -855,7 +855,7 @@ impl KafkaConnection { // Don't drop the consumer until after we've drained the errors // channel. Dropping the consumer can introduce spurious errors. - // See materialize#24901. + // See database-issues#7432. drop(consumer); match main_err { diff --git a/src/storage/src/healthcheck.rs b/src/storage/src/healthcheck.rs index ea8614f6d20d9..bcb33b588c675 100644 --- a/src/storage/src/healthcheck.rs +++ b/src/storage/src/healthcheck.rs @@ -115,7 +115,7 @@ impl PerWorkerHealthStatus { // HealthStatusUpdate::Ceased is currently unused, so just // treat it as if it were a normal error. // - // TODO: redesign ceased status materialize#25768 + // TODO: redesign ceased status database-issues#7687 HealthStatusUpdate::Ceased { error } => { if Some(error) > namespaced_errors.get(ns).as_deref() { namespaced_errors.insert(*ns, error.to_string()); @@ -566,7 +566,7 @@ impl HealthStatusUpdate { } } - // TODO: redesign ceased status materialize#25768 + // TODO: redesign ceased status database-issues#7687 // Generates a ceasing [`HealthStatusUpdate`] with `update`. // pub(crate) fn ceasing(error: String) -> Self { // HealthStatusUpdate::Ceased { error } diff --git a/src/storage/src/render/sinks.rs b/src/storage/src/render/sinks.rs index 06f0a6e02672f..93195eb7f3762 100644 --- a/src/storage/src/render/sinks.rs +++ b/src/storage/src/render/sinks.rs @@ -143,7 +143,7 @@ where // Group messages by key at each timestamp. // // Allow access to `arrange_named` because we cannot access Mz's wrapper - // from here. TODO(materialize#17413): Revisit with cluster unification. + // from here. TODO(database-issues#5046): Revisit with cluster unification. #[allow(clippy::disallowed_methods)] let mut collection = combine_at_timestamp(collection.arrange_named::>("Arrange Sink")); @@ -165,7 +165,7 @@ where // // TODO: put the sink in a user-visible errored state instead of // only logging internally. See: - // https://github.com/MaterializeInc/materialize/issues/17549. + // https://api.github.com/repos/MaterializeInc/database-issues/issues/5099. if !key_is_synthetic && vs.len() > 1 { // We rate limit how often we emit this warning to avoid // flooding logs. diff --git a/src/storage/src/sink/kafka.rs b/src/storage/src/sink/kafka.rs index cd257d904f0b1..4eb7a49c21905 100644 --- a/src/storage/src/sink/kafka.rs +++ b/src/storage/src/sink/kafka.rs @@ -434,7 +434,7 @@ impl TransactionalProducer { // internal use, so we silently drop any such user-specified // headers. While this behavior is documented, it'd be a nicer UX to // send a warning or error somewhere. Unfortunately sinks don't have - // anywhere user-visible to send errors. See materialize#17672. + // anywhere user-visible to send errors. See database-issues#5148. if header.key.starts_with("materialize-") { continue; } diff --git a/src/storage/src/source/reclock.rs b/src/storage/src/source/reclock.rs index a9e0f29199d4c..bc35a57be0a6d 100644 --- a/src/storage/src/source/reclock.rs +++ b/src/storage/src/source/reclock.rs @@ -1738,7 +1738,7 @@ mod tests { } // Regression test for - // https://github.com/MaterializeInc/materialize/issues/14740. + // https://api.github.com/repos/MaterializeInc/database-issues/issues/4216. #[mz_ore::test(tokio::test(start_paused = true))] #[cfg_attr(miri, ignore)] // error: unsupported operation: can't call foreign function `decNumberFromInt32` on OS `linux` async fn test_since_hold() { diff --git a/src/testdrive/src/action/kafka/ingest.rs b/src/testdrive/src/action/kafka/ingest.rs index edfc1cfdb43d3..e8557f4db6cf8 100644 --- a/src/testdrive/src/action/kafka/ingest.rs +++ b/src/testdrive/src/action/kafka/ingest.rs @@ -123,7 +123,7 @@ impl Transcoder { .context("parsing protobuf JSON")?; let mut out = vec![]; if *confluent_wire_format { - // See: https://github.com/MaterializeInc/materialize/issues/9250 + // See: https://api.github.com/repos/MaterializeInc/database-issues/issues/2837 // The first byte is a magic byte (0) that indicates the Confluent // serialization format version, and the next four bytes are a // 32-bit schema ID, which we default to something fun. diff --git a/src/timestamp-oracle/src/postgres_oracle.rs b/src/timestamp-oracle/src/postgres_oracle.rs index abce07e356158..cbcd6893a3aae 100644 --- a/src/timestamp-oracle/src/postgres_oracle.rs +++ b/src/timestamp-oracle/src/postgres_oracle.rs @@ -437,7 +437,7 @@ where // accumulate, scanning over the table will take increasingly and // prohibitively long. // - // See: https://github.com/MaterializeInc/materialize/issues/13975 + // See: https://api.github.com/repos/MaterializeInc/database-issues/issues/4001 // See: https://www.cockroachlabs.com/docs/stable/configure-zone.html#variables client .batch_execute(&format!( diff --git a/src/transform/src/analysis.rs b/src/transform/src/analysis.rs index 29e4aff71628a..bfb64fac08507 100644 --- a/src/transform/src/analysis.rs +++ b/src/transform/src/analysis.rs @@ -796,7 +796,7 @@ mod non_negative { /// These rules are .. somewhat arbitrary, and likely reflect observed opportunities. For example, /// while we do relate `distinct(filter(A)) <= distinct(A)`, we do not relate `distinct(A) <= A`. /// Further thoughts about the class of optimizations, and whether there should be more or fewer, - /// can be found here: . + /// can be found here: . fn is_superset_of(mut lhs: &MirRelationExpr, mut rhs: &MirRelationExpr) -> bool { // This implementation is iterative. // Before converting this implementation to recursive (e.g. to improve its accuracy) diff --git a/src/transform/src/analysis/monotonic.rs b/src/transform/src/analysis/monotonic.rs index 8b93140ad0963..61381a563eec9 100644 --- a/src/transform/src/analysis/monotonic.rs +++ b/src/transform/src/analysis/monotonic.rs @@ -101,7 +101,7 @@ impl Analysis for Monotonic { MirRelationExpr::Constant { rows: Ok(rows), .. } => { rows.iter().all(|(_, diff)| diff > &0) } - // TODO: materialize#28499 (Investigate if constant expressions with error rows can be marked monotonic) + // TODO: database-issues#8337 (Investigate if constant expressions with error rows can be marked monotonic) MirRelationExpr::Constant { rows: Err(_), .. } => false, } } diff --git a/src/transform/src/dataflow.rs b/src/transform/src/dataflow.rs index 9c7fca181110f..ea6748b43e475 100644 --- a/src/transform/src/dataflow.rs +++ b/src/transform/src/dataflow.rs @@ -769,7 +769,7 @@ id: {}, key: {:?}", /// - Some indexes might be less skewed than others. (Although, picking a unique key tries to /// capture this already.) /// - Some indexes might have an error, while others don't. -/// +/// /// - Some indexes might have more extra data in their keys (because of being on more complicated /// expressions than just column references), which won't be used in a full scan. fn choose_index( @@ -896,7 +896,7 @@ impl<'a> CollectIndexRequests<'a> { } JoinImplementation::DeltaQuery(..) => { // For Delta joins, the first input is special, see - // https://github.com/MaterializeInc/materialize/issues/6789 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/2115 this.collect_index_reqs_inner( &mut inputs[0], &IndexUsageContext::from_usage_type(IndexUsageType::DeltaJoin(DeltaJoinIndexUsageType::Unknown)), @@ -1025,7 +1025,7 @@ impl<'a> CollectIndexRequests<'a> { // top of the Let, or when the 2 uses each have an `ArrangeBy`. In both cases, // we'll add only 1 full scan, which would be wrong in the latter case. However, // the latter case can't currently happen until we do - // https://github.com/MaterializeInc/materialize/issues/21145 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/6363 // Also note that currently we are deduplicating index usage types when // printing index usages in EXPLAIN. if let Some((idx_id, key)) = pick_index_for_full_scan(global_id) { diff --git a/src/transform/src/fold_constants.rs b/src/transform/src/fold_constants.rs index db1dba39bfec2..7f43724867db9 100644 --- a/src/transform/src/fold_constants.rs +++ b/src/transform/src/fold_constants.rs @@ -75,7 +75,7 @@ impl FoldConstants { MirRelationExpr::Get { .. } => {} MirRelationExpr::Let { .. } | MirRelationExpr::LetRec { .. } => { // Constant propagation through bindings is currently handled by in NormalizeLets. - // Maybe we should move it / replicate it here (see materialize#18180 for context)? + // Maybe we should move it / replicate it here (see database-issues#5346 for context)? } MirRelationExpr::Reduce { input, diff --git a/src/transform/src/fusion/join.rs b/src/transform/src/fusion/join.rs index f098f9e07c8f7..0bf635e28122a 100644 --- a/src/transform/src/fusion/join.rs +++ b/src/transform/src/fusion/join.rs @@ -62,7 +62,7 @@ impl crate::Transform for Join { // and CanonicalizeMfp in order to re-construct an equi-Join which would // be de-constructed as a Filter + CrossJoin by the action application. // - // TODO(materialize#25918): This is a temporary solution which fixes the "Product + // TODO(database-issues#7728): This is a temporary solution which fixes the "Product // limits" issue observed in a failed Nightly run when the PR was first // tested (https://buildkite.com/materialize/nightly/builds/6670). We // should re-evaluate if we need this ad-hoc re-normalization step when diff --git a/src/transform/src/lib.rs b/src/transform/src/lib.rs index a4c7f166902af..4a5b72ccc200a 100644 --- a/src/transform/src/lib.rs +++ b/src/transform/src/lib.rs @@ -356,7 +356,7 @@ impl Transform for Fixpoint { // done now, but it would be great to eventually find a way to prevent // these loops from happening in the first place. We have several // relevant issues, see - // https://github.com/MaterializeInc/materialize/issues/27954#issuecomment-2200172227 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/8197#issuecomment-2200172227 mz_repr::explain::trace_plan(relation); soft_panic_or_log!( "Fixpoint `{}` detected a loop of length {} after {} iterations", @@ -397,7 +397,7 @@ impl Transform for Fixpoint { // This is not catastrophic, because we can just say we are done now, // but it would be great to eventually find a way to prevent these loops from // happening in the first place. We have several relevant issues, see - // https://github.com/MaterializeInc/materialize/issues/27954#issuecomment-2200172227 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/8197#issuecomment-2200172227 mz_repr::explain::trace_plan(relation); soft_panic_or_log!( "Fixpoint {} failed to reach a fixed point, or cycle of length at most {}", @@ -470,7 +470,7 @@ impl Default for FuseAndCollapse { Self { // TODO: The relative orders of the transforms have not been // determined except where there are comments. - // TODO (materialize#6542): All the transforms here except for `ProjectionLifting` + // TODO (database-issues#2036): All the transforms here except for `ProjectionLifting` // and `RedundantJoin` can be implemented as free functions. transforms: vec![ Box::new(canonicalization::ProjectionExtraction), @@ -591,7 +591,7 @@ impl Optimizer { // Lifts the information `!isnull(col)` Box::new(nonnullable::NonNullable), // Lifts the information `col = literal` - // TODO (materialize#6613): this also tries to lift `!isnull(col)` but + // TODO (database-issues#2062): this also tries to lift `!isnull(col)` but // less well than the previous transform. Eliminate // redundancy between the two transforms. Box::new(column_knowledge::ColumnKnowledge::default()), @@ -657,11 +657,11 @@ impl Optimizer { // run in the logical optimizer). // - Not running ColumnKnowledge in the same fixpoint loop with JoinImplementation // is slightly hurting our plans. However, I'd say we should fix these problems by - // making ColumnKnowledge (and/or JoinImplementation) smarter (materialize#18051), rather than + // making ColumnKnowledge (and/or JoinImplementation) smarter (database-issues#5289), rather than // having them in the same fixpoint loop. If they would be in the same fixpoint loop, - // then we either run the risk of ColumnKnowledge invalidating a join plan (materialize#17993), + // then we either run the risk of ColumnKnowledge invalidating a join plan (database-issues#5260), // or we would have to run JoinImplementation an unbounded number of times, which is - // also not good materialize#16076. + // also not good database-issues#4639. // (The same is true for FoldConstants, Demand, and LiteralLifting to a lesser // extent.) // @@ -708,7 +708,7 @@ impl Optimizer { Box::new(threshold_elision::ThresholdElision), // We need this to ensure that `CollectIndexRequests` gets a normalized plan. // (For example, `FoldConstants` can break the normalized form by removing all - // references to a Let, see https://github.com/MaterializeInc/materialize/issues/21175) + // references to a Let, see https://api.github.com/repos/MaterializeInc/database-issues/issues/6371) Box::new(normalize_lets::NormalizeLets::new(false)), Box::new(typecheck::Typecheck::new(ctx.typecheck()).disallow_new_globals()), ]; @@ -824,7 +824,7 @@ impl Optimizer { Err(e) => { // Without this, the dropping of `relation` (which happens automatically when // returning from this function) might run into a stack overflow, see - // https://github.com/MaterializeInc/materialize/issues/14141 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/4043 relation.destroy_carefully(); error!("Optimizer::optimize(): {}", e); Err(e) diff --git a/src/transform/src/literal_lifting.rs b/src/transform/src/literal_lifting.rs index ca2f7a2c0ea09..d1e71c87b350d 100644 --- a/src/transform/src/literal_lifting.rs +++ b/src/transform/src/literal_lifting.rs @@ -124,7 +124,7 @@ impl LiteralLifting { /// In theory, all literals could be treated in the same way if this method /// returned both a list of literals and a projection vector, making the /// caller have to deal with the reshuffling. - /// (see ) + /// (see ) /// pub fn action( &self, diff --git a/src/transform/src/normalize_lets.rs b/src/transform/src/normalize_lets.rs index c7c44c17870cc..2fa4fadb065cf 100644 --- a/src/transform/src/normalize_lets.rs +++ b/src/transform/src/normalize_lets.rs @@ -726,7 +726,7 @@ mod inlining { /// there was no recursion in it. /// /// The case of `Constant` binding is handled here (as opposed to - /// `FoldConstants`) in a somewhat limited manner (see materialize#18180). Although a + /// `FoldConstants`) in a somewhat limited manner (see database-issues#5346). Although a /// bit weird, constants should also not be inlined into prior bindings as /// this does change the behavior from one where the collection is initially /// empty to one where it is always the constant. @@ -805,7 +805,7 @@ mod inlining { match stripped_value { // TODO: One could imagine CSEing multiple occurrences of a global Get // to make us read from Persist only once. - // See + // See MirRelationExpr::Get { .. } | MirRelationExpr::Constant { .. } => true, _ => false, } diff --git a/src/transform/src/predicate_pushdown.rs b/src/transform/src/predicate_pushdown.rs index 33ee407f6787c..e0d66a9e07443 100644 --- a/src/transform/src/predicate_pushdown.rs +++ b/src/transform/src/predicate_pushdown.rs @@ -24,7 +24,7 @@ //! or if we are not certain that the input is non-empty (e.g. join). //! Note that this is not addressing the problem in its full generality, because this problem can //! occur with any function call that might error (although much more rarely than with literal -//! errors). See +//! errors). See //! //! ```rust //! use mz_expr::{BinaryFunc, MirRelationExpr, MirScalarExpr}; @@ -462,7 +462,7 @@ impl PredicatePushdown { // hard to appropriately reflect the negation in the error stream: // - If we don't negate, then errors that should cancel out will not // cancel out. For example, see - // https://github.com/MaterializeInc/materialize/issues/19179 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/5691 // - If we negate, then unrelated errors might cancel out. E.g., there // might be a division-by-0 in both inputs to an EXCEPT ALL, but // on different input data. These shouldn't cancel out. @@ -523,7 +523,7 @@ impl PredicatePushdown { body, } => { // Note: This could be extended to be able to do a little more pushdowns, see - // https://github.com/MaterializeInc/materialize/issues/18167#issuecomment-1477588262 + // https://api.github.com/repos/MaterializeInc/database-issues/issues/5336#issuecomment-1477588262 // Pre-compute which Ids are used across iterations let ids_used_across_iterations = MirRelationExpr::recursive_ids(ids, values); @@ -868,7 +868,7 @@ impl PredicatePushdown { // this problem in general (as we can't just not push anything that might // error), but we decided to fix the specific problem instance involving // `error_if_null`, because it was very painful: - // + // } else { let mut localized = predicate.clone(); if input_mapper.try_localize_to_input_with_bound_expr( diff --git a/src/transform/src/reduction_pushdown.rs b/src/transform/src/reduction_pushdown.rs index 095245056d235..ce6bd3ec13d2b 100644 --- a/src/transform/src/reduction_pushdown.rs +++ b/src/transform/src/reduction_pushdown.rs @@ -300,14 +300,14 @@ fn try_push_reduce_through_join( { if !agg.distinct { // TODO: support non-distinct aggs. - // For more details, see https://github.com/MaterializeInc/materialize/issues/9604 + // For more details, see https://api.github.com/repos/MaterializeInc/database-issues/issues/2924 return None; } new_projection.push((component, new_reduces[component].arity())); new_reduces[component].add_aggregate(agg.clone()); } else { // TODO: support multi- and zero- component aggs - // For more details, see https://github.com/MaterializeInc/materialize/issues/9604 + // For more details, see https://api.github.com/repos/MaterializeInc/database-issues/issues/2924 return None; } } diff --git a/src/transform/src/typecheck.rs b/src/transform/src/typecheck.rs index 48c5eea229329..7a6e615fd430a 100644 --- a/src/transform/src/typecheck.rs +++ b/src/transform/src/typecheck.rs @@ -1151,7 +1151,7 @@ macro_rules! type_error { ($severity:expr, $($arg:tt)+) => {{ if $severity { ::tracing::warn!($($arg)+); - ::tracing::error!("type error in MIR optimization (details in warning; see 'Type error omnibus' issue materialize#19101 )"); + ::tracing::error!("type error in MIR optimization (details in warning; see 'Type error omnibus' issue database-issues#5663 )"); } else { ::tracing::debug!($($arg)+); } diff --git a/src/transform/tests/test_transforms/non_null_requirements.spec b/src/transform/tests/test_transforms/non_null_requirements.spec index 4bd11b780f87c..1b628debf7bfc 100644 --- a/src/transform/tests/test_transforms/non_null_requirements.spec +++ b/src/transform/tests/test_transforms/non_null_requirements.spec @@ -103,7 +103,7 @@ With Constant -# Regression test for materialize#5520 +# Regression test for database-issues#1700 apply pipeline=non_null_requirements Filter #0 = #3 FlatMap generate_series(#1) diff --git a/src/transform/tests/test_transforms/reduction_pushdown.spec b/src/transform/tests/test_transforms/reduction_pushdown.spec index fae6e54fbda60..408f5e3df2221 100644 --- a/src/transform/tests/test_transforms/reduction_pushdown.spec +++ b/src/transform/tests/test_transforms/reduction_pushdown.spec @@ -27,10 +27,10 @@ DefSource name=y keys=[[#0]] Source defined as t1 -# Regression test for materialize#25015. +# Regression test for database-issues#7467. # # The Join has a condition that is a local predicate -# and was lost prior to materialize#25015. +# and was lost prior to database-issues#7467. apply pipeline=reduction_pushdown Distinct project=[#1] Join on=((#1 + #1) = #0) diff --git a/src/txn-wal/src/txn_cache.rs b/src/txn-wal/src/txn_cache.rs index 4336d91407ca7..f968a11e2cabe 100644 --- a/src/txn-wal/src/txn_cache.rs +++ b/src/txn-wal/src/txn_cache.rs @@ -1264,7 +1264,7 @@ mod tests { // ts 11 (forget) // Revisit when - // https://github.com/MaterializeInc/materialize/issues/25992 is fixed, + // https://api.github.com/repos/MaterializeInc/database-issues/issues/7746 is fixed, // it's unclear how to encode the register timestamp in a forget. c.push_register(d0, 11, -1, 11); testcase(&mut c, 11, d0, ds(None, 11, 12), ReadDataTo(12)); @@ -1286,7 +1286,7 @@ mod tests { // ts 16 (forgotten, registered at preceding ts) // Revisit when - // https://github.com/MaterializeInc/materialize/issues/25992 is fixed, + // https://api.github.com/repos/MaterializeInc/database-issues/issues/7746 is fixed, // it's unclear how to encode the register timestamp in a forget. c.push_register(d0, 16, -1, 16); testcase(&mut c, 16, d0, ds(None, 16, 17), ReadDataTo(17)); diff --git a/src/txn-wal/src/txn_read.rs b/src/txn-wal/src/txn_read.rs index cee108b37d73c..19723a8fefe00 100644 --- a/src/txn-wal/src/txn_read.rs +++ b/src/txn-wal/src/txn_read.rs @@ -104,7 +104,7 @@ impl DataSnapshot { // TODO(jkosh44) We should not be writing to unregistered shards, but // we haven't checked to see if this was registered at `self.empty_to`. - // See https://github.com/MaterializeInc/materialize/issues/27088. + // See https://api.github.com/repos/MaterializeInc/database-issues/issues/8022. while data_upper < self.empty_to { // It would be very bad if we accidentally filled any times <= // latest_write with empty updates, so defensively assert on each diff --git a/test/bounded-memory/mzcompose.py b/test/bounded-memory/mzcompose.py index 113d250f11509..954df3d1b166c 100644 --- a/test/bounded-memory/mzcompose.py +++ b/test/bounded-memory/mzcompose.py @@ -28,7 +28,7 @@ from materialize.mzcompose.services.testdrive import Testdrive # Those defaults have been carefully chosen to avoid known OOMs -# such as materialize#15093 and materialize#15044 while hopefully catching any further +# such as materialize#15093 and database-issues#4297 while hopefully catching any further # regressions in memory usage PAD_LEN = 1024 STRING_PAD = "x" * PAD_LEN diff --git a/test/canary-environment/tests/generic/makes_progress.sql b/test/canary-environment/tests/generic/makes_progress.sql index 55fefcd92dbe8..8544a0157c9e3 100644 --- a/test/canary-environment/tests/generic/makes_progress.sql +++ b/test/canary-environment/tests/generic/makes_progress.sql @@ -14,7 +14,7 @@ SET CLUSTER = qa_canary_environment_compute; BEGIN; DECLARE c1 CURSOR FOR SUBSCRIBE ( SELECT * FROM {{ model }} ) WITH (SNAPSHOT = FALSE); - -- TODO: Switch timeout back to 60s after materialize#22061 is fixed + -- TODO: Switch timeout back to 60s after database-issues#6647 is fixed FETCH 1 c1 WITH (timeout='300s'); {% endset %} diff --git a/test/canary-load/mzcompose.py b/test/canary-load/mzcompose.py index 8471b50773d1f..39fcf70644bf4 100644 --- a/test/canary-load/mzcompose.py +++ b/test/canary-load/mzcompose.py @@ -123,7 +123,7 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: except FailedTestExecutionError as e: assert len(e.errors) > 0, "Exception contains no errors" for error in e.errors: - # TODO(def-): Remove when materialize#22576 is fixed + # TODO(def-): Remove when database-issues#6825 is fixed if "Non-positive multiplicity in DistinctBy" in error.message: continue print( @@ -133,7 +133,7 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: failures.append(error) except CommandFailureCausedUIError as e: msg = (e.stdout or "") + (e.stderr or "") - # TODO(def-): Remove when materialize#22576 is fixed + # TODO(def-): Remove when database-issues#6825 is fixed if "Non-positive multiplicity in DistinctBy" in msg: continue print(f"Test failure occurred ({msg}), collecting it, and continuing.") diff --git a/test/cloud-canary/mzcompose.py b/test/cloud-canary/mzcompose.py index a5cd3a9c3a354..ba381914f6f54 100644 --- a/test/cloud-canary/mzcompose.py +++ b/test/cloud-canary/mzcompose.py @@ -261,7 +261,7 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: try: c.run("mz", "region", "enable", "--version", VERSION) except UIError: - # Work around https://github.com/MaterializeInc/materialize/issues/17219 + # Work around https://api.github.com/repos/MaterializeInc/database-issues/issues/4989 pass time.sleep(10) diff --git a/test/cloudtest/test_cluster_rehydration.py b/test/cloudtest/test_cluster_rehydration.py index 6b665562f9801..9e68d17e314bf 100644 --- a/test/cloudtest/test_cluster_rehydration.py +++ b/test/cloudtest/test_cluster_rehydration.py @@ -13,7 +13,7 @@ # Test that a crashed (and restarted) cluster replica handles rehydration -# correctly by not recreating dropped sources. Tests materialize#18102. +# correctly by not recreating dropped sources. Tests database-issues#5308. def test_create_drop_source(mz: MaterializeApplication) -> None: mz.testdrive.run( input=dedent( diff --git a/test/cloudtest/test_compute.py b/test/cloudtest/test_compute.py index 57a75eae1fb4f..f488fa2270f76 100644 --- a/test/cloudtest/test_compute.py +++ b/test/cloudtest/test_compute.py @@ -52,7 +52,7 @@ def test_cluster_sizing(mz: MaterializeApplication) -> None: "failpoint", ["", "after_catalog_drop_replica=panic", "after_sequencer_drop_replica=panic"], ) -@pytest.mark.skip(reason="Failpoints mess up the Mz instance materialize#18000") +@pytest.mark.skip(reason="Failpoints mess up the Mz instance database-issues#5263") def test_cluster_shutdown(mz: MaterializeApplication, failpoint: str) -> None: """Test that dropping a cluster or replica causes the associated clusterds to shut down.""" @@ -163,7 +163,7 @@ def test_disk_label(mz: MaterializeApplication) -> None: ) -@pytest.mark.skip(reason="Keeps flaking, see materialize#28327") +@pytest.mark.skip(reason="Keeps flaking, see database-issues#8299") def test_cluster_replica_sizes(mz: MaterializeApplication) -> None: """Test that --cluster-replica-sizes mapping is respected""" # Some time for existing cluster drops to complete so we don't try to spin them up again diff --git a/test/cloudtest/test_full_testdrive.py b/test/cloudtest/test_full_testdrive.py index a9c30ba522c3c..451e623862c2a 100644 --- a/test/cloudtest/test_full_testdrive.py +++ b/test/cloudtest/test_full_testdrive.py @@ -26,7 +26,7 @@ def test_full_testdrive(mz: MaterializeApplication) -> None: matching_files = glob.glob(f"testdrive/{args.file_pattern}", root_dir="test") - # TODO: materialize#26392 (test requires fivetran running in cloudtest) + # TODO: database-issues#7827 (test requires fivetran running in cloudtest) matching_files.remove("testdrive/fivetran-destination.td") sharded_files = buildkite.shard_list(matching_files, lambda file: file) diff --git a/test/cloudtest/test_metrics.py b/test/cloudtest/test_metrics.py index 1d2b6491ccae7..7914483de9a7c 100644 --- a/test/cloudtest/test_metrics.py +++ b/test/cloudtest/test_metrics.py @@ -14,7 +14,7 @@ from materialize.cloudtest.app.materialize_application import MaterializeApplication -@pytest.mark.skip(reason="Fails occasionally, see materialize#29171") +@pytest.mark.skip(reason="Fails occasionally, see database-issues#8472") def test_replica_metrics(mz: MaterializeApplication) -> None: mz.testdrive.run( input=dedent( diff --git a/test/cloudtest/test_replica_restart.py b/test/cloudtest/test_replica_restart.py index eefb1dee554e3..341baefbcd772 100644 --- a/test/cloudtest/test_replica_restart.py +++ b/test/cloudtest/test_replica_restart.py @@ -49,7 +49,7 @@ def assert_notice(conn: Connection, contains: bytes) -> None: # Test that an OOMing cluster replica generates expected entries in # `mz_cluster_replica_statuses` -@pytest.mark.skip(reason="Now fails after a Buildkite upgrade materialize#20948") +@pytest.mark.skip(reason="Now fails after a Buildkite upgrade database-issues#6307") def test_oom_clusterd(mz: MaterializeApplication) -> None: def verify_cluster_oomed() -> None: with mz.environmentd.sql_cursor(autocommit=False) as cur: diff --git a/test/cloudtest/test_secrets.py b/test/cloudtest/test_secrets.py index 2783c9a918f13..8695d5a0bb97c 100644 --- a/test/cloudtest/test_secrets.py +++ b/test/cloudtest/test_secrets.py @@ -62,7 +62,7 @@ def test_secrets(mz: MaterializeApplication) -> None: # Tests that secrets deleted from the catalog but not from k8s are cleaned up on # envd startup. -@pytest.mark.skip(reason="Failpoints mess up the Mz intance materialize#18000") +@pytest.mark.skip(reason="Failpoints mess up the Mz intance database-issues#5263") def test_orphaned_secrets(mz: MaterializeApplication) -> None: # Use two separate failpoints. One that crashes after modifying the catalog # (drop_secrets), and one that fails during bootstrap (orphan_secrets) so @@ -95,7 +95,7 @@ def test_orphaned_secrets(mz: MaterializeApplication) -> None: wait(condition="delete", resource=f"secret/{secret}") -@pytest.mark.skip(reason="Flaky, see materialize#29072") +@pytest.mark.skip(reason="Flaky, see database-issues#8456") def test_missing_secret(mz: MaterializeApplication) -> None: """Test that Mz does not panic if a secret goes missing from K8s""" mz.testdrive.run( diff --git a/test/cluster-isolation/mzcompose.py b/test/cluster-isolation/mzcompose.py index 83b11d8eb5287..3c3e7db18ba61 100644 --- a/test/cluster-isolation/mzcompose.py +++ b/test/cluster-isolation/mzcompose.py @@ -213,7 +213,7 @@ def validate(c: Composition) -> None: > SELECT * FROM source1 A -# TODO: This should be made reliable without sleeping, materialize#25479 +# TODO: This should be made reliable without sleeping, database-issues#7611 $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=5s # Sinks diff --git a/test/cluster/mzcompose.py b/test/cluster/mzcompose.py index d27a80ccc4645..138f744864a7d 100644 --- a/test/cluster/mzcompose.py +++ b/test/cluster/mzcompose.py @@ -255,7 +255,7 @@ def workflow_test_github_15531(c: Composition) -> None: """ Test that compute command history does not leak peek commands. - Regression test for https://github.com/MaterializeInc/materialize/issues/15531. + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4443. """ c.down(destroy_volumes=True) @@ -429,7 +429,7 @@ def workflow_test_github_15535(c: Composition) -> None: """ Test that compute reconciliation does not produce empty frontiers. - Regression test for https://github.com/MaterializeInc/materialize/issues/15535. + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4444. """ c.down(destroy_volumes=True) @@ -490,7 +490,7 @@ def workflow_test_github_15799(c: Composition) -> None: Test that querying introspection sources on a replica does not crash other replicas in the same cluster that have introspection disabled. - Regression test for https://github.com/MaterializeInc/materialize/issues/15799. + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4545. """ c.down(destroy_volumes=True) @@ -541,7 +541,7 @@ def workflow_test_github_15930(c: Composition) -> None: Test that triggering reconciliation does not wedge the mz_compute_frontiers_per_worker introspection source. - Regression test for https://github.com/MaterializeInc/materialize/issues/15930. + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4587. """ c.down(destroy_volumes=True) @@ -639,7 +639,7 @@ def workflow_test_github_15496(c: Composition) -> None: Test that a reduce collation over a source with an invalid accumulation does not panic, but rather logs errors, when soft assertions are turned off. - Regression test for https://github.com/MaterializeInc/materialize/issues/15496. + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4433. """ c.down(destroy_volumes=True) @@ -718,7 +718,7 @@ def workflow_test_github_17177(c: Composition) -> None: emits errors to the logs when soft assertions are turned off, but also produces a clean query-level error. - Regression test for https://github.com/MaterializeInc/materialize/issues/17177. + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4966. """ c.down(destroy_volumes=True) @@ -797,7 +797,7 @@ def workflow_test_github_17510(c: Composition) -> None: with invalid accumulations due to too many retractions in a source. Additionally, we verify that in these cases, an adequate error message is written to the logs. - Regression test for https://github.com/MaterializeInc/materialize/issues/17510. + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5087. """ c.down(destroy_volumes=True) @@ -881,7 +881,7 @@ def workflow_test_github_17510(c: Composition) -> None: ! SELECT * FROM constant_sums; contains:constant folding encountered reduce on collection with non-positive multiplicities - # The following statement verifies that the behavior introduced in PR materialize#16852 + # The following statement verifies that the behavior introduced in PR database-issues#6122 # is now rectified, i.e., instead of wrapping to a negative number, we produce # an error upon seeing invalid multiplicities. ! SELECT SUM(data8) FROM data; @@ -935,7 +935,7 @@ def workflow_test_github_17510(c: Composition) -> None: > INSERT INTO base VALUES (1, 1, 1, 1), (1, 1, 1, 1), (1, 1, 1, 1); # Constant-folding behavior matches for now the rendered behavior - # wrt. wraparound; this can be revisited as part of materialize#17758. + # wrt. wraparound; this can be revisited as part of database-issues#5172. > SELECT * FROM constant_wrapped_sums; 1 1 18446744073709551617 @@ -962,7 +962,7 @@ def workflow_test_github_17509(c: Composition) -> None: a clean error when an arrangement hierarchy is built, in addition to logging an error, when soft assertions are turned off. - This is a partial regression test for https://github.com/MaterializeInc/materialize/issues/17509. + This is a partial regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5086. The checks here are extended by opting into a smaller group size with a query hint (e.g., OPTIONS (AGGREGATE INPUT GROUP SIZE = 1)) in workflow test-github-15496. This scenario was initially not covered, but eventually got supported as well. @@ -1057,7 +1057,7 @@ def workflow_test_github_19610(c: Composition) -> None: We introduce data that results in a multiset and compute min/max. In a monotonic one-shot evaluation strategy, we must consolidate and subsequently assert monotonicity. - This is a regression test for https://github.com/MaterializeInc/materialize/issues/19610, where + This is a regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5831, where we observed a performance regression caused by a correctness issue. Here, we validate that the underlying correctness issue has been fixed. """ @@ -1411,7 +1411,7 @@ def workflow_pg_snapshot_resumption(c: Composition) -> None: c.run_testdrive_files("pg-snapshot-resumption/03-ensure-source-down.td") # Temporarily disabled because it is timing out. - # https://github.com/MaterializeInc/materialize/issues/14533 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/4145 # # clusterd should crash # c.run_testdrive_files("pg-snapshot-resumption/04-while-clusterd-down.td") @@ -1871,7 +1871,7 @@ def fetch_reconciliation_metrics(process: str) -> tuple[int, int]: ) # Give the dataflows some time to make progress and get compacted. - # This is done to trigger the bug described in materialize#17594. + # This is done to trigger the bug described in database-issues#5113. time.sleep(10) # Restart environmentd to trigger a reconciliation. @@ -1903,7 +1903,7 @@ def workflow_test_compute_reconciliation_replace(c: Composition) -> None: Test that compute reconciliation replaces changed dataflows, as well as dataflows transitively depending on them. - Regression test for materialize#28961. + Regression test for database-issues#8444. """ c.down(destroy_volumes=True) @@ -2080,7 +2080,7 @@ def workflow_test_drop_during_reconciliation(c: Composition) -> None: """ Test that dropping storage and compute objects during reconciliation works. - Regression test for materialize#28784. + Regression test for database-issues#8399. """ c.down(destroy_volumes=True) @@ -2286,7 +2286,7 @@ def workflow_test_mv_source_sink(c: Composition) -> None: """ Test that compute materialized view's "since" timestamp is at least as large as source table's "since" timestamp. - Regression test for https://github.com/MaterializeInc/materialize/issues/19151 + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5676 """ c.down(destroy_volumes=True) @@ -2353,7 +2353,7 @@ def workflow_test_clusterd_death_detection(c: Composition) -> None: """ Test that environmentd notices when a clusterd becomes disconnected. - Regression test for https://github.com/MaterializeInc/materialize/issues/20299 + Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6095 """ c.down(destroy_volumes=True) @@ -3047,7 +3047,7 @@ def check_workload_class(expected: str | None): def workflow_test_concurrent_connections(c: Composition) -> None: """ Run many concurrent connections, measure their p50 and p99 latency, make - sure materialize#21782 does not regress. + sure database-issues#6537 does not regress. """ num_conns = 2000 p50_limit = 10.0 @@ -3284,7 +3284,7 @@ def worker(c: Composition, worker_index: int) -> None: for thread in threads: thread.start() - # this is because of materialize#22038 + # this is because of database-issues#6639 time.sleep(0.2) for thread in threads: @@ -3324,7 +3324,7 @@ def workflow_test_github_cloud_7998( def workflow_test_github_23246(c: Composition, parser: WorkflowArgumentParser) -> None: - """Regression test for materialize#23246.""" + """Regression test for database-issues#7000.""" c.down(destroy_volumes=True) @@ -3593,9 +3593,9 @@ def workflow_test_refresh_mv_warmup( ) -> None: """ Test REFRESH materialized view warmup behavior after envd restarts: - 1. Regression test for https://github.com/MaterializeInc/materialize/issues/25380 + 1. Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/7574 If an MV is past its last refresh, it shouldn't get rehydrated after a restart. - 2. Regression test for https://github.com/MaterializeInc/materialize/issues/25279 + 2. Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/7543 Bootstrapping should select an `as_of` for an MV dataflow in a way that allows it to warm up before its next refresh. """ @@ -4149,7 +4149,7 @@ def check_introspection(): def workflow_test_github_26215(c: Composition, parser: WorkflowArgumentParser) -> None: - """Regression test for materialize#26215.""" + """Regression test for database-issues#7798.""" c.down(destroy_volumes=True) diff --git a/test/cluster/resources/resource-limits.td b/test/cluster/resources/resource-limits.td index 92adfd0771dbf..afd9562c5c53e 100644 --- a/test/cluster/resources/resource-limits.td +++ b/test/cluster/resources/resource-limits.td @@ -322,7 +322,7 @@ ALTER SYSTEM SET max_tables = 2 > CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source'); > CREATE SOURCE mz_source2 FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source'); -# TODO: materialize#29556 (source limit not applied) +# TODO: database-issues#8556 (source limit not applied) # ! CREATE SOURCE mz_source3 FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source'); # contains:creating source would violate max_sources limit (desired: 3, limit: 2, current: 0) @@ -362,7 +362,7 @@ INSERT INTO t4 VALUES (4); CREATE PUBLICATION mz_source2 FOR TABLE t4; # Show that we're at limit -# TODO: materialize#29556 (source limit not applied) +# TODO: database-issues#8556 (source limit not applied) # ! CREATE SOURCE mz_source4 # FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source2'); # contains:creating source would violate max_sources limit (desired: 4, limit: 3, current: 3) @@ -373,7 +373,7 @@ CREATE PUBLICATION mz_source2 FOR TABLE t4; > CREATE SOURCE mz_source2 FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source2'); -# TODO: materialize#29556 (source limit not applied) +# TODO: database-issues#8556 (source limit not applied) # ! CREATE SOURCE mz_source5 # FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source'); # contains:creating source would violate max_sources limit (desired: 4, limit: 3, current: 3) diff --git a/test/debezium/postgres/02-add-primary-key.td b/test/debezium/postgres/02-add-primary-key.td index e21c25f3a2247..7783cadaa658d 100644 --- a/test/debezium/postgres/02-add-primary-key.td +++ b/test/debezium/postgres/02-add-primary-key.td @@ -11,7 +11,7 @@ # Test adding a primary key over an existing column # -# TODO: Reenable when materialize#6570 is fixed +# TODO: Reenable when database-issues#2047 is fixed $ skip-if SELECT true diff --git a/test/debezium/postgres/02-drop-primary-key.td b/test/debezium/postgres/02-drop-primary-key.td index 0a16a29d18a80..7dba52910155a 100644 --- a/test/debezium/postgres/02-drop-primary-key.td +++ b/test/debezium/postgres/02-drop-primary-key.td @@ -11,7 +11,7 @@ # Test that dropping the primary key is handled correctly # -# TODO: Reenable when materialize#6521 is fixed +# TODO: Reenable when database-issues#2025 is fixed $ skip-if SELECT true diff --git a/test/debezium/postgres/05-add-column-primary-key.td b/test/debezium/postgres/05-add-column-primary-key.td index be89b2b58ccb1..1bdc97d3e68fa 100644 --- a/test/debezium/postgres/05-add-column-primary-key.td +++ b/test/debezium/postgres/05-add-column-primary-key.td @@ -12,7 +12,7 @@ # Currently rejected by the schema registry on the Debezium side. Replication stops # -# TODO: Reenable when materialize#6570 is fixed +# TODO: Reenable when database-issues#2047 is fixed $ skip-if SELECT true diff --git a/test/debezium/postgres/08-primary-key-extend.td b/test/debezium/postgres/08-primary-key-extend.td index 5365e70b152b3..5107745613f6d 100644 --- a/test/debezium/postgres/08-primary-key-extend.td +++ b/test/debezium/postgres/08-primary-key-extend.td @@ -12,7 +12,7 @@ # Currently, this throws an error on the Debezium side and replication stops # -# TODO: Reenable when materialize#6570 is fixed +# TODO: Reenable when database-issues#2047 is fixed $ skip-if SELECT true diff --git a/test/debezium/postgres/08-primary-key-shrink.td b/test/debezium/postgres/08-primary-key-shrink.td index db73e577fcf48..544b6652bda9c 100644 --- a/test/debezium/postgres/08-primary-key-shrink.td +++ b/test/debezium/postgres/08-primary-key-shrink.td @@ -12,7 +12,7 @@ # Currently, this throws an error on the Debezium side and replication stops # -# TODO: Reenable when materialize#6570 is fixed +# TODO: Reenable when database-issues#2047 is fixed $ skip-if SELECT true diff --git a/test/debezium/postgres/10-remove-nullability.td b/test/debezium/postgres/10-remove-nullability.td index 235184511927c..04604395935fa 100644 --- a/test/debezium/postgres/10-remove-nullability.td +++ b/test/debezium/postgres/10-remove-nullability.td @@ -11,7 +11,7 @@ # Change the definition of a column to be NOT NULL # -# TODO: Reenable when materialize#6570 is fixed +# TODO: Reenable when database-issues#2047 is fixed $ skip-if SELECT true diff --git a/test/debezium/postgres/12-change-decimal.td b/test/debezium/postgres/12-change-decimal.td index b63f62c1cf9a2..fe53b21657261 100644 --- a/test/debezium/postgres/12-change-decimal.td +++ b/test/debezium/postgres/12-change-decimal.td @@ -9,7 +9,7 @@ # # Changing the definition of a DECIMAL column results in a source error -# see discussion in https://github.com/MaterializeInc/materialize/issues/6536 +# see discussion in https://api.github.com/repos/MaterializeInc/database-issues/issues/2032 # $ postgres-execute connection=postgres://postgres:postgres@postgres diff --git a/test/debezium/postgres/20-types-temporal.td b/test/debezium/postgres/20-types-temporal.td index 705b60b131e6b..ee33bb1e27b83 100644 --- a/test/debezium/postgres/20-types-temporal.td +++ b/test/debezium/postgres/20-types-temporal.td @@ -11,7 +11,7 @@ # Make sure that temporal types are properly replicated, including sub-second precision # -# TODO: Reenable when materialize#6535 is fixed +# TODO: Reenable when database-issues#2031 is fixed $ skip-if SELECT true diff --git a/test/debezium/sql-server/40-check-types.td b/test/debezium/sql-server/40-check-types.td index 75b7f9e8e4554..0b017d93e8f45 100644 --- a/test/debezium/sql-server/40-check-types.td +++ b/test/debezium/sql-server/40-check-types.td @@ -24,8 +24,8 @@ $ schema-registry-wait topic=sql-server.test.dbo.types_table ENVELOPE DEBEZIUM; # -# DATETIMEOFFSET is replicated as TEXT, see https://github.com/MaterializeInc/materialize/issues/8017 -# DATETIME2 is replicated as BIGINT, see https://github.com/MaterializeInc/materialize/issues/8041 +# DATETIMEOFFSET is replicated as TEXT, see https://api.github.com/repos/MaterializeInc/database-issues/issues/2453 +# DATETIME2 is replicated as BIGINT, see https://api.github.com/repos/MaterializeInc/database-issues/issues/2458 # SMALLDATETIME does not store seconds or fractional seconds > SELECT * FROM types_table diff --git a/test/kafka-resumption/mzcompose.py b/test/kafka-resumption/mzcompose.py index d15659abaf109..b9879ebfbb5e3 100644 --- a/test/kafka-resumption/mzcompose.py +++ b/test/kafka-resumption/mzcompose.py @@ -140,7 +140,7 @@ def workflow_source_resumption(c: Composition, parser: WorkflowArgumentParser) - c.run_testdrive_files("source-resumption/setup.td") c.run_testdrive_files("source-resumption/verify.td") - # Disabled due to https://github.com/MaterializeInc/materialize/issues/20819 + # Disabled due to https://api.github.com/repos/MaterializeInc/database-issues/issues/6271 # assert ( # find_source_resume_upper( # c, @@ -160,7 +160,7 @@ def workflow_source_resumption(c: Composition, parser: WorkflowArgumentParser) - # the first clusterd instance ingested 3 messages, so our # upper is at the 4th offset (0-indexed) - # Disabled due to https://github.com/MaterializeInc/materialize/issues/20819 + # Disabled due to https://api.github.com/repos/MaterializeInc/database-issues/issues/6271 # assert ( # find_source_resume_upper( # c, @@ -188,7 +188,7 @@ def find_source_resume_upper(c: Composition, partition_id: str) -> int | None: def workflow_sink_queue_full(c: Composition, parser: WorkflowArgumentParser) -> None: - """Similar to the sink-networking workflow, but with 11 million rows (more then the 11 million defined as queue.buffering.max.messages) and only creating the sink after these rows are ingested into Mz. Triggers materialize#24936""" + """Similar to the sink-networking workflow, but with 11 million rows (more then the 11 million defined as queue.buffering.max.messages) and only creating the sink after these rows are ingested into Mz. Triggers database-issues#7442""" args = parse_args(parser) seed = random.getrandbits(16) c.up(*(["materialized", "toxiproxy"] + get_kafka_services(args.redpanda))) diff --git a/test/legacy-upgrade/check-from-v0.27.0-schema-registry.td b/test/legacy-upgrade/check-from-v0.27.0-schema-registry.td index 5af313b2b4483..3940339883d45 100644 --- a/test/legacy-upgrade/check-from-v0.27.0-schema-registry.td +++ b/test/legacy-upgrade/check-from-v0.27.0-schema-registry.td @@ -13,7 +13,7 @@ $ skip-if SELECT mz_version_num() >= 4300; -# This next query will fail if a bug like materialize#10234 occurs again, where a migration +# This next query will fail if a bug like database-issues#3053 occurs again, where a migration # injects a `WITH` option into the wrong spot. > SHOW CREATE SOURCE data diff --git a/test/limits/mzcompose.py b/test/limits/mzcompose.py index 6053d54263c0e..7103faedd7578 100644 --- a/test/limits/mzcompose.py +++ b/test/limits/mzcompose.py @@ -103,7 +103,7 @@ def body(cls) -> None: class Tables(Generator): - COUNT = 90 # https://github.com/MaterializeInc/materialize/issues/12773 and https://github.com/MaterializeInc/materialize/issues/26404 + COUNT = 90 # https://github.com/MaterializeInc/materialize/issues/12773 and https://api.github.com/repos/MaterializeInc/database-issues/issues/7830 @classmethod def body(cls) -> None: @@ -595,7 +595,9 @@ def body(cls) -> None: class TablesCommaJoinNoCondition(Generator): - COUNT = 100 # https://github.com/MaterializeInc/materialize/issues/12806 + COUNT = ( + 100 # https://api.github.com/repos/MaterializeInc/database-issues/issues/3682 + ) @classmethod def body(cls) -> None: @@ -663,7 +665,7 @@ def body(cls) -> None: class SubqueriesScalarSelectListWithCondition(Generator): COUNT = min( Generator.COUNT, 100 - ) # https://github.com/MaterializeInc/materialize/issues/8598 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2626 @classmethod def body(cls) -> None: @@ -680,7 +682,7 @@ def body(cls) -> None: class SubqueriesScalarWhereClauseAnd(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8598 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2626 @classmethod def body(cls) -> None: @@ -696,7 +698,7 @@ def body(cls) -> None: class SubqueriesExistWhereClause(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8598 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2626 @classmethod def body(cls) -> None: @@ -712,7 +714,7 @@ def body(cls) -> None: class SubqueriesInWhereClauseCorrelated(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/20557 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/6189 @classmethod def body(cls) -> None: @@ -729,7 +731,7 @@ def body(cls) -> None: class SubqueriesInWhereClauseUncorrelated(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/20557 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/6189 @classmethod def body(cls) -> None: @@ -745,7 +747,7 @@ def body(cls) -> None: class SubqueriesWhereClauseOr(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8602 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2630 @classmethod def body(cls) -> None: @@ -776,7 +778,7 @@ def body(cls) -> None: class ViewsNested(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8598 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2626 @classmethod def body(cls) -> None: @@ -796,7 +798,7 @@ def body(cls) -> None: class ViewsMaterializedNested(Generator): COUNT = min( Generator.COUNT, 25 - ) # https://github.com/MaterializeInc/materialize/issues/13840 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/3958 @classmethod def body(cls) -> None: @@ -825,7 +827,7 @@ def body(cls) -> None: class CTEs(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8600 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2628 @classmethod def body(cls) -> None: @@ -842,7 +844,7 @@ def body(cls) -> None: class NestedCTEsIndependent(Generator): COUNT = min( Generator.COUNT, 9 - ) # https://github.com/MaterializeInc/materialize/issues/8600 and https://github.com/MaterializeInc/materialize/issues/26404 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2628 and https://api.github.com/repos/MaterializeInc/database-issues/issues/7830 @classmethod def body(cls) -> None: @@ -862,7 +864,7 @@ def body(cls) -> None: class NestedCTEsChained(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8601 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2629 @classmethod def body(cls) -> None: @@ -881,7 +883,7 @@ def body(cls) -> None: class DerivedTables(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8602 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2630 @classmethod def body(cls) -> None: @@ -898,7 +900,7 @@ def body(cls) -> None: class Lateral(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8603 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2631 @classmethod def body(cls) -> None: @@ -914,7 +916,7 @@ def body(cls) -> None: class SelectExpression(Generator): # Stack exhaustion with COUNT=1000 due to unprotected path: - # https://github.com/MaterializeInc/materialize/issues/10496 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/3107 COUNT = min(Generator.COUNT, 500) @classmethod @@ -936,7 +938,7 @@ def body(cls) -> None: class WhereExpression(Generator): # Stack exhaustion with COUNT=1000 due to unprotected path: - # https://github.com/MaterializeInc/materialize/issues/10496 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/3107 COUNT = min(Generator.COUNT, 500) @classmethod @@ -953,7 +955,7 @@ def body(cls) -> None: class WhereConditionAnd(Generator): - # Stack overflow, see https://github.com/MaterializeInc/materialize/issues/19327 + # Stack overflow, see https://api.github.com/repos/MaterializeInc/database-issues/issues/5731 COUNT = min(Generator.COUNT, 500) @classmethod @@ -970,7 +972,7 @@ def body(cls) -> None: class WhereConditionAndSameColumn(Generator): - # Stack overflow, see https://github.com/MaterializeInc/materialize/issues/19327 + # Stack overflow, see https://api.github.com/repos/MaterializeInc/database-issues/issues/5731 COUNT = min(Generator.COUNT, 500) @classmethod @@ -985,7 +987,7 @@ def body(cls) -> None: class WhereConditionOr(Generator): - # Stack overflow, see https://github.com/MaterializeInc/materialize/issues/19327 + # Stack overflow, see https://api.github.com/repos/MaterializeInc/database-issues/issues/5731 COUNT = min(Generator.COUNT, 500) @classmethod @@ -1002,7 +1004,7 @@ def body(cls) -> None: class WhereConditionOrSameColumn(Generator): - # Stack overflow, see https://github.com/MaterializeInc/materialize/issues/19327 + # Stack overflow, see https://api.github.com/repos/MaterializeInc/database-issues/issues/5731 COUNT = min(Generator.COUNT, 500) @classmethod @@ -1075,7 +1077,7 @@ def body(cls) -> None: class AggregateExpression(Generator): # Stack exhaustion with COUNT=1000 due to unprotected path: - # https://github.com/MaterializeInc/materialize/issues/10496 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/3107 COUNT = min(Generator.COUNT, 500) @classmethod @@ -1113,7 +1115,7 @@ def body(cls) -> None: class Unions(Generator): COUNT = min( Generator.COUNT, 10 - ) # https://github.com/MaterializeInc/materialize/issues/8600 + ) # https://api.github.com/repos/MaterializeInc/database-issues/issues/2628 @classmethod def body(cls) -> None: @@ -1150,8 +1152,8 @@ def body(cls) -> None: class CaseWhen(Generator): # Originally this was working with 1000, but after moving lowering and # decorrelation from the `plan_~` to the `sequence_~` method we had to - # reduce it a bit in order to avoid overflowing the stack. See materialize#24076 - # and materialize#24820 for the latest occurrences of this. + # reduce it a bit in order to avoid overflowing the stack. See database-issues#7216 + # and database-issues#7407 for the latest occurrences of this. COUNT = 600 @classmethod @@ -1244,9 +1246,9 @@ def body(cls) -> None: class FilterSubqueries(Generator): """ - Regression test for materialize#20557. + Regression test for database-issues#6189. - Without the materialize#20557 fix in materialize#20702 this will cause `environmend` to OOM + Without the database-issues#6189 fix in materialize#20702 this will cause `environmend` to OOM because of excessive memory allocations in the `RedundantJoin` transform. """ @@ -1518,7 +1520,7 @@ def body(cls) -> None: class WebhookSources(Generator): - COUNT = 100 # TODO: Remove when materialize#29358 is fixed + COUNT = 100 # TODO: Remove when database-issues#8508 is fixed @classmethod def body(cls) -> None: diff --git a/test/mysql-cdc-old-syntax/alter-source.td b/test/mysql-cdc-old-syntax/alter-source.td index 7b8b049cebe69..a38900c591307 100644 --- a/test/mysql-cdc-old-syntax/alter-source.td +++ b/test/mysql-cdc-old-syntax/alter-source.td @@ -44,7 +44,7 @@ $ mysql-execute name=mysql CREATE TABLE table_a (pk INTEGER PRIMARY KEY, f2 TEXT); INSERT INTO table_a VALUES (9, 'nine'); -# TODO: materialize#27585 (expected table was dropped) +# TODO: database-issues#8127 (expected table was dropped) $ skip-if SELECT true @@ -59,7 +59,7 @@ contains:Invalid MySQL table reference: table_a ! ALTER SOURCE mz_source ADD SUBSOURCE public.table_a; contains:catalog item 'table_a' already exists -# TODO: materialize#27585 (expected table was dropped) +# TODO: database-issues#8127 (expected table was dropped) $ skip-end > DROP SOURCE mz_source CASCADE; diff --git a/test/mysql-cdc-old-syntax/alter-table-after-source.td b/test/mysql-cdc-old-syntax/alter-table-after-source.td index 679879469a21b..d931ce1e0032d 100644 --- a/test/mysql-cdc-old-syntax/alter-table-after-source.td +++ b/test/mysql-cdc-old-syntax/alter-table-after-source.td @@ -10,7 +10,7 @@ $ set-sql-timeout duration=1s $ set-max-tries max-tries=20 -# TODO(def-) Reenable when materialize#26733 is fixed +# TODO(def-) Reenable when database-issues#7900 is fixed $ skip-if SELECT true @@ -146,7 +146,7 @@ $ mysql-execute name=mysql ALTER TABLE alter_column_type_2 MODIFY f1 int; INSERT INTO alter_column_type_2 VALUES (2048); -# TODO(def-): Reenable when materialize#25660 is fixed +# TODO(def-): Reenable when database-issues#7655 is fixed # ! select * from alter_column_type_2; # contains:incompatible schema change @@ -182,7 +182,7 @@ INSERT INTO alter_add_nullability VALUES (1); 1 1 -# TODO: materialize#25040 (changes to columns) +# TODO: database-issues#7475 (changes to columns) # ? EXPLAIN SELECT * FROM alter_add_nullability WHERE f1 IS NULL; # Explained Query (fast path): # Constant @@ -338,7 +338,7 @@ INSERT INTO alter_table_rename_column (f1, f2) VALUES ('f1_renamed', 'f2_renamed contains:incompatible schema change -# TODO: materialize#25660 (column drop & recreate detection) +# TODO: database-issues#7655 (column drop & recreate detection) # Change column attnum # > SELECT * from alter_table_change_attnum; @@ -402,7 +402,7 @@ contains: table was truncated $ mysql-execute name=mysql DROP TABLE drop_table; -# TODO: redesign ceased status materialize#25768 +# TODO: redesign ceased status database-issues#7687 # > SELECT status FROM mz_internal.mz_source_statuses WHERE name = 'drop_table'; # ceased diff --git a/test/mysql-cdc-old-syntax/empty-table.td b/test/mysql-cdc-old-syntax/empty-table.td index 48879f593f184..65d78911175c9 100644 --- a/test/mysql-cdc-old-syntax/empty-table.td +++ b/test/mysql-cdc-old-syntax/empty-table.td @@ -39,7 +39,7 @@ CREATE TABLE empty_table (f1 BOOLEAN); > SELECT COUNT(*) FROM empty_table; 0 -# TODO: materialize#25774 (subsource remains starting) +# TODO: database-issues#7690 (subsource remains starting) > SELECT name, type, status FROM mz_internal.mz_source_statuses WHERE name in ('empty_table', 'mz_source'); empty_table subsource starting mz_source mysql running diff --git a/test/mysql-cdc-old-syntax/gh-10981.td b/test/mysql-cdc-old-syntax/gh-10981.td index c15f02f68aa4a..baf1e2e1e185d 100644 --- a/test/mysql-cdc-old-syntax/gh-10981.td +++ b/test/mysql-cdc-old-syntax/gh-10981.td @@ -12,7 +12,7 @@ $ set-max-tries max-tries=20 # -# Test for issue materialize#10981 +# Test for issue database-issues#3215 # > CREATE SECRET mysqlpass AS '${arg.mysql-root-password}' diff --git a/test/mysql-cdc-old-syntax/invisible-columns.td b/test/mysql-cdc-old-syntax/invisible-columns.td index b2fa31837f492..fee51943aceb4 100644 --- a/test/mysql-cdc-old-syntax/invisible-columns.td +++ b/test/mysql-cdc-old-syntax/invisible-columns.td @@ -33,7 +33,7 @@ CREATE TABLE t1 (f1 INT, f2 INT INVISIBLE, f3 INT INVISIBLE); INSERT INTO t1 (f1, f2, f3) VALUES (10, 20, 30); INSERT INTO t1 VALUES (11); -# TODO: materialize#26152 (invisible columns not supported) +# TODO: database-issues#7782 (invisible columns not supported) # > CREATE SOURCE mz_source # FROM MYSQL CONNECTION mysql_conn # FOR ALL TABLES; diff --git a/test/mysql-cdc-old-syntax/mysql-cdc-ssl.td b/test/mysql-cdc-old-syntax/mysql-cdc-ssl.td index caf07dfea134e..5ab3d0ffda6e8 100644 --- a/test/mysql-cdc-old-syntax/mysql-cdc-ssl.td +++ b/test/mysql-cdc-old-syntax/mysql-cdc-ssl.td @@ -126,7 +126,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: hostssl, client: verify-ca => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -161,7 +161,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: hostssl, client: verify-identity => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -174,7 +174,7 @@ DELETE FROM numbers WHERE number = 2; # contains:self signed certificate in certificate chain # > DROP CONNECTION mysql_conn; -# TODO: materialize#25676 +# TODO: database-issues#7660 # # server: hostssl, client: verify-identity => OK # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -218,7 +218,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: hostnossl, client: verify-ca => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -277,7 +277,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: certuser, client: verify-ca (wrong cert) => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -316,7 +316,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: certuser, client: verify-identity => OK # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, diff --git a/test/mysql-cdc-old-syntax/mysql-cdc.td b/test/mysql-cdc-old-syntax/mysql-cdc.td index ef2674ecbd9a4..ef5fa06602296 100644 --- a/test/mysql-cdc-old-syntax/mysql-cdc.td +++ b/test/mysql-cdc-old-syntax/mysql-cdc.td @@ -13,7 +13,7 @@ $ set-max-tries max-tries=20 > CREATE SECRET mysqlpass AS '${arg.mysql-root-password}' -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # ! CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, # USER root, @@ -190,7 +190,7 @@ ALTER SYSTEM SET mysql_source_snapshot_lock_wait_timeout = 3600 PASSWORD SECRET mysqlpass ) contains:failed to lookup address information -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # contains:error connecting to server: failed to lookup address information: Name or service not known: failed to lookup address ! CREATE CONNECTION no_such_port TO MYSQL ( @@ -200,16 +200,16 @@ contains:failed to lookup address information PASSWORD SECRET mysqlpass ) contains:Connection refused -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # contains:error connecting to server: Connection refused -# TODO: materialize#25413 (error not stable) +# TODO: database-issues#7587 (error not stable) # ! CREATE CONNECTION no_such_user TO MYSQL ( # HOST mysql, # USER no_such_user, # PASSWORD SECRET mysqlpass # ) -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # contains:password authentication failed for user "no_such_user" > CREATE SECRET badpass AS 'badpass' @@ -628,7 +628,7 @@ DELETE FROM mixED_CAse; 0 # Reference exists in two schemas, so is not unambiguous -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE SOURCE enum_source # IN CLUSTER cdc_cluster # FROM MYSQL CONNECTION mysql_conn ( @@ -640,7 +640,7 @@ DELETE FROM mixED_CAse; # ); # contains: invalid TEXT COLUMNS option value: table another_enum_table is ambiguous, consider specifying the schema -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE SOURCE enum_source # IN CLUSTER cdc_cluster # FROM MYSQL CONNECTION mysql_conn ( @@ -649,7 +649,7 @@ DELETE FROM mixED_CAse; # FOR TABLES (pk_table); # contains: invalid TEXT COLUMNS option value: column name 'foo' must have at least a table qualification -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE SOURCE enum_source # IN CLUSTER cdc_cluster # FROM MYSQL CONNECTION mysql_conn ( @@ -658,7 +658,7 @@ DELETE FROM mixED_CAse; # FOR TABLES (pk_table); # contains: invalid TEXT COLUMNS option value: qualified name did not have between 1 and 3 components: foo.bar.qux.qax -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE SOURCE enum_source # IN CLUSTER cdc_cluster # FROM MYSQL CONNECTION mysql_conn ( @@ -667,7 +667,7 @@ DELETE FROM mixED_CAse; # FOR TABLES (enum_table); # contains: invalid TEXT COLUMNS option value: unexpected multiple references to postgres.public.enum_table.a -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # utf8_table is not part of mz_source_narrow publication # ! CREATE SOURCE enum_source # IN CLUSTER cdc_cluster diff --git a/test/mysql-cdc-old-syntax/schema-restart/after-restart.td b/test/mysql-cdc-old-syntax/schema-restart/after-restart.td index 0b3b64783b747..2fb7e9185cbea 100644 --- a/test/mysql-cdc-old-syntax/schema-restart/after-restart.td +++ b/test/mysql-cdc-old-syntax/schema-restart/after-restart.td @@ -14,7 +14,7 @@ name = 'schema_test' AND status = 'running'; true -# TODO: redesign ceased status materialize#25768 +# TODO: redesign ceased status database-issues#7687 # # dummy subsource is put into error state # > SELECT true # FROM mz_internal.mz_source_statuses diff --git a/test/mysql-cdc-old-syntax/two-destination-schemas.td b/test/mysql-cdc-old-syntax/two-destination-schemas.td index f3af87b5e6bf2..efa2494d3c946 100644 --- a/test/mysql-cdc-old-syntax/two-destination-schemas.td +++ b/test/mysql-cdc-old-syntax/two-destination-schemas.td @@ -46,7 +46,7 @@ INSERT INTO t2 VALUES (2); > CREATE SCHEMA schema1; > CREATE SCHEMA schema2; -# TODO: materialize#24797 (schema must be specified even when no collisions exist): remove public prefix +# TODO: database-issues#7397 (schema must be specified even when no collisions exist): remove public prefix > CREATE SOURCE mz_source FROM MYSQL CONNECTION mysql_conn FOR TABLES (public.t1 AS schema1.t1, public.t2 AS schema2.t1); diff --git a/test/mysql-cdc-old-syntax/two-source-schemas.td b/test/mysql-cdc-old-syntax/two-source-schemas.td index 3f1feca025efc..d8378ac3b5cda 100644 --- a/test/mysql-cdc-old-syntax/two-source-schemas.td +++ b/test/mysql-cdc-old-syntax/two-source-schemas.td @@ -46,7 +46,7 @@ INSERT INTO schema2.t1 SELECT * FROM schema2.t1; FOR ALL TABLES; contains:multiple subsources would be named t1 -# TODO: materialize#24797 +# TODO: database-issues#7397 # ! CREATE SOURCE mz_source # FROM MYSQL CONNECTION mysql_conn # FOR TABLES (t1); diff --git a/test/mysql-cdc-old-syntax/types-enum.td b/test/mysql-cdc-old-syntax/types-enum.td index 2f05295af9c3f..f049eed8937ac 100644 --- a/test/mysql-cdc-old-syntax/types-enum.td +++ b/test/mysql-cdc-old-syntax/types-enum.td @@ -32,7 +32,7 @@ USE public; CREATE TABLE enum_type (f1 ENUM ('val1', 'val2'), f2 TEXT); INSERT INTO enum_type VALUES ('val1', 'val1'), ('val2', 'val2'); -# TODO: materialize#25881 (enum unsupported) +# TODO: database-issues#7719 (enum unsupported) ! CREATE SOURCE mz_source FROM MYSQL CONNECTION mysql_conn FOR ALL TABLES; @@ -77,7 +77,7 @@ $ mysql-execute name=mysql ALTER TABLE enum_type CHANGE f1 f1 ENUM ('val2', 'val1', 'val3', 'val4'); INSERT INTO enum_type VALUES ('val1', 'val1'); -# TODO: materialize#26128 (expected incompatible schema change) +# TODO: database-issues#7776 (expected incompatible schema change) $ skip-if SELECT true diff --git a/test/mysql-cdc-resumption-old-syntax/mzcompose.py b/test/mysql-cdc-resumption-old-syntax/mzcompose.py index 54af7f294b69a..b87fd5282ed19 100644 --- a/test/mysql-cdc-resumption-old-syntax/mzcompose.py +++ b/test/mysql-cdc-resumption-old-syntax/mzcompose.py @@ -66,7 +66,7 @@ def workflow_default(c: Composition) -> None: if name == "default": continue - # TODO(def-): Reenable when materialize#26127 is fixed + # TODO(def-): Reenable when database-issues#7775 is fixed if name in ("bin-log-manipulations", "short-bin-log-retention"): continue @@ -606,7 +606,7 @@ def backup_restore_mysql(c: Composition) -> None: run_testdrive_files(c, "verify-mysql-select.td") - # TODO: materialize#25760: one of the two following commands must succeed + # TODO: database-issues#7683: one of the two following commands must succeed # run_testdrive_files(c, "verify-rows-after-restore-t1.td") # run_testdrive_files(c, "verify-source-failed.td") diff --git a/test/mysql-cdc-resumption/mzcompose.py b/test/mysql-cdc-resumption/mzcompose.py index eb4d5f2a9113b..5d4cfc5e065e6 100644 --- a/test/mysql-cdc-resumption/mzcompose.py +++ b/test/mysql-cdc-resumption/mzcompose.py @@ -66,7 +66,7 @@ def workflow_default(c: Composition) -> None: if name == "default": continue - # TODO(def-): Reenable when materialize#26127 is fixed + # TODO(def-): Reenable when database-issues#7775 is fixed if name in ("bin-log-manipulations", "short-bin-log-retention"): continue @@ -606,7 +606,7 @@ def backup_restore_mysql(c: Composition) -> None: run_testdrive_files(c, "verify-mysql-select.td") - # TODO: materialize#25760: one of the two following commands must succeed + # TODO: database-issues#7683: one of the two following commands must succeed # run_testdrive_files(c, "verify-rows-after-restore-t1.td") # run_testdrive_files(c, "verify-source-failed.td") diff --git a/test/mysql-cdc/alter-source.td b/test/mysql-cdc/alter-source.td index 2874ece27fc41..fe806907998fb 100644 --- a/test/mysql-cdc/alter-source.td +++ b/test/mysql-cdc/alter-source.td @@ -44,7 +44,7 @@ $ mysql-execute name=mysql CREATE TABLE table_a (pk INTEGER PRIMARY KEY, f2 TEXT); INSERT INTO table_a VALUES (9, 'nine'); -# TODO: materialize#27585 (expected table was dropped) +# TODO: database-issues#8127 (expected table was dropped) $ skip-if SELECT true @@ -55,7 +55,7 @@ contains:table was dropped ! CREATE TABLE table_a FROM SOURCE mz_source (REFERENCE public.table_a); contains:catalog item 'table_a' already exists -# TODO: materialize#27585 (expected table was dropped) +# TODO: database-issues#8127 (expected table was dropped) $ skip-end > DROP SOURCE mz_source CASCADE; @@ -272,7 +272,7 @@ INSERT INTO table_a VALUES (3); ! SELECT * FROM table_a; contains:incompatible schema change -# TODO: source status support with materialize#29373 +# TODO: source status support with database-issues#8511 # > SELECT error ~~ '%incompatible schema change%' FROM mz_internal.mz_source_statuses WHERE name = 'table_a'; # true # diff --git a/test/mysql-cdc/alter-table-after-source.td b/test/mysql-cdc/alter-table-after-source.td index d8fd4f0f4e132..60eb19ab7e5d4 100644 --- a/test/mysql-cdc/alter-table-after-source.td +++ b/test/mysql-cdc/alter-table-after-source.td @@ -10,7 +10,7 @@ $ set-sql-timeout duration=1s $ set-max-tries max-tries=20 -# TODO(def-) Reenable when materialize#26733 is fixed +# TODO(def-) Reenable when database-issues#7900 is fixed $ skip-if SELECT true @@ -165,7 +165,7 @@ $ mysql-execute name=mysql ALTER TABLE alter_column_type_2 MODIFY f1 int; INSERT INTO alter_column_type_2 VALUES (2048); -# TODO(def-): Reenable when materialize#25660 is fixed +# TODO(def-): Reenable when database-issues#7655 is fixed # ! select * from alter_column_type_2; # contains:incompatible schema change @@ -201,7 +201,7 @@ INSERT INTO alter_add_nullability VALUES (1); 1 1 -# TODO: materialize#25040 (changes to columns) +# TODO: database-issues#7475 (changes to columns) # ? EXPLAIN SELECT * FROM alter_add_nullability WHERE f1 IS NULL; # Explained Query (fast path): # Constant @@ -357,7 +357,7 @@ INSERT INTO alter_table_rename_column (f1, f2) VALUES ('f1_renamed', 'f2_renamed contains:incompatible schema change -# TODO: materialize#25660 (column drop & recreate detection) +# TODO: database-issues#7655 (column drop & recreate detection) # Change column attnum # > SELECT * from alter_table_change_attnum; @@ -421,7 +421,7 @@ contains: table was truncated $ mysql-execute name=mysql DROP TABLE drop_table; -# TODO: redesign ceased status materialize#25768 +# TODO: redesign ceased status database-issues#7687 # > SELECT status FROM mz_internal.mz_source_statuses WHERE name = 'drop_table'; # ceased diff --git a/test/mysql-cdc/gh-10981.td b/test/mysql-cdc/gh-10981.td index 52ca87d8b275a..bf666b2c825e3 100644 --- a/test/mysql-cdc/gh-10981.td +++ b/test/mysql-cdc/gh-10981.td @@ -12,7 +12,7 @@ $ set-max-tries max-tries=20 # -# Test for issue materialize#10981 +# Test for issue database-issues#3215 # > CREATE SECRET mysqlpass AS '${arg.mysql-root-password}' diff --git a/test/mysql-cdc/invisible-columns.td b/test/mysql-cdc/invisible-columns.td index a81a9bbc66a65..c4be9483ac9db 100644 --- a/test/mysql-cdc/invisible-columns.td +++ b/test/mysql-cdc/invisible-columns.td @@ -33,7 +33,7 @@ CREATE TABLE t1 (f1 INT, f2 INT INVISIBLE, f3 INT INVISIBLE); INSERT INTO t1 (f1, f2, f3) VALUES (10, 20, 30); INSERT INTO t1 VALUES (11); -# TODO: materialize#26152 (invisible columns not supported) +# TODO: database-issues#7782 (invisible columns not supported) # > CREATE SOURCE mz_source FROM MYSQL CONNECTION mysql_conn; # > CREATE TABLE t1 FROM SOURCE mz_source (REFERENCE public.t1); # diff --git a/test/mysql-cdc/mysql-cdc-ssl.td b/test/mysql-cdc/mysql-cdc-ssl.td index c89333c02fa87..29c098670c665 100644 --- a/test/mysql-cdc/mysql-cdc-ssl.td +++ b/test/mysql-cdc/mysql-cdc-ssl.td @@ -126,7 +126,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: hostssl, client: verify-ca => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -160,7 +160,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: hostssl, client: verify-identity => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -172,7 +172,7 @@ DELETE FROM numbers WHERE number = 2; # contains:self signed certificate in certificate chain # > DROP CONNECTION mysql_conn; -# TODO: materialize#25676 +# TODO: database-issues#7660 # # server: hostssl, client: verify-identity => OK # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -215,7 +215,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: hostnossl, client: verify-ca => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -273,7 +273,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: certuser, client: verify-ca (wrong cert) => ERROR # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, @@ -312,7 +312,7 @@ INSERT INTO numbers VALUES (2, true, 'two'); $ mysql-execute name=mysql DELETE FROM numbers WHERE number = 2; -# TODO: materialize#25676 (error not handled properly) +# TODO: database-issues#7660 (error not handled properly) # # server: certuser, client: verify-identity => OK # > CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, diff --git a/test/mysql-cdc/mysql-cdc.td b/test/mysql-cdc/mysql-cdc.td index dd1ffaeb90492..76087a77b7194 100644 --- a/test/mysql-cdc/mysql-cdc.td +++ b/test/mysql-cdc/mysql-cdc.td @@ -13,7 +13,7 @@ $ set-max-tries max-tries=20 > CREATE SECRET mysqlpass AS '${arg.mysql-root-password}' -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # ! CREATE CONNECTION mysql_conn TO MYSQL ( # HOST mysql, # USER root, @@ -194,7 +194,7 @@ ALTER SYSTEM SET mysql_source_snapshot_lock_wait_timeout = 3600 PASSWORD SECRET mysqlpass ) contains:failed to lookup address information -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # contains:error connecting to server: failed to lookup address information: Name or service not known: failed to lookup address ! CREATE CONNECTION no_such_port TO MYSQL ( @@ -204,16 +204,16 @@ contains:failed to lookup address information PASSWORD SECRET mysqlpass ) contains:Connection refused -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # contains:error connecting to server: Connection refused -# TODO: materialize#25413 (error not stable) +# TODO: database-issues#7587 (error not stable) # ! CREATE CONNECTION no_such_user TO MYSQL ( # HOST mysql, # USER no_such_user, # PASSWORD SECRET mysqlpass # ) -# TODO: materialize#25076 (misleading error message) +# TODO: database-issues#7490 (misleading error message) # contains:password authentication failed for user "no_such_user" > CREATE SECRET badpass AS 'badpass' @@ -557,7 +557,7 @@ true 2 two_two 5 five -# TODO: materialize#29373 (introspection tables) +# TODO: database-issues#8511 (introspection tables) # > SELECT status = 'running' FROM mz_internal.mz_source_statuses WHERE name = 'large_cluster_source_pk_table' and type = 'table'; # true @@ -625,35 +625,35 @@ DELETE FROM mixED_CAse; FROM MYSQL CONNECTION mysql_conn; # Reference exists in two schemas, so is not unambiguous -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE TABLE conflict_enum # FROM SOURCE enum_source # REFERENCE (conflict_schema.another_enum_table) # WITH (TEXT COLUMNS = [another_enum_table."колона"]); # contains: invalid TEXT COLUMNS option value: table another_enum_table is ambiguous, consider specifying the schema -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE TABLE pk_table # FROM SOURCE enum_source # REFERENCE (pk_table) # WITH (TEXT COLUMNS = [foo]); # contains: invalid TEXT COLUMNS option value: column name 'foo' must have at least a table qualification -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE TABLE pk_table # FROM SOURCE enum_source # REFERENCE (pk_table) # WITH (TEXT COLUMNS = [foo.bar.qux.qax.foo]); # contains: invalid TEXT COLUMNS option value: qualified name did not have between 1 and 3 components: foo.bar.qux.qax -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # ! CREATE TABLE enum_table # FROM SOURCE enum_source # REFERENCE (enum_table) # WITH (TEXT COLUMNS = [enum_table.a, enum_table.a]); # contains: invalid TEXT COLUMNS option value: unexpected multiple references to postgres.public.enum_table.a -# TODO: materialize#24797 (explicit schema definition required) +# TODO: database-issues#7397 (explicit schema definition required) # utf8_table is not part of mz_source_narrow publication # ! CREATE TABLE enum_table # FROM SOURCE enum_source diff --git a/test/mysql-cdc/schema-restart/after-restart.td b/test/mysql-cdc/schema-restart/after-restart.td index 0b3b64783b747..2fb7e9185cbea 100644 --- a/test/mysql-cdc/schema-restart/after-restart.td +++ b/test/mysql-cdc/schema-restart/after-restart.td @@ -14,7 +14,7 @@ name = 'schema_test' AND status = 'running'; true -# TODO: redesign ceased status materialize#25768 +# TODO: redesign ceased status database-issues#7687 # # dummy subsource is put into error state # > SELECT true # FROM mz_internal.mz_source_statuses diff --git a/test/mysql-cdc/two-destination-schemas.td b/test/mysql-cdc/two-destination-schemas.td index 3d5f83cba8bc9..5b1a94ca9ce68 100644 --- a/test/mysql-cdc/two-destination-schemas.td +++ b/test/mysql-cdc/two-destination-schemas.td @@ -46,7 +46,7 @@ INSERT INTO t2 VALUES (2); > CREATE SCHEMA schema1; > CREATE SCHEMA schema2; -# TODO: materialize#24797 (schema must be specified even when no collisions exist): remove public prefix +# TODO: database-issues#7397 (schema must be specified even when no collisions exist): remove public prefix > CREATE SOURCE mz_source FROM MYSQL CONNECTION mysql_conn; > CREATE TABLE schema1.t1 FROM SOURCE mz_source (REFERENCE public.t1); > CREATE TABLE schema2.t1 FROM SOURCE mz_source (REFERENCE public.t2); diff --git a/test/mysql-cdc/two-source-schemas.td b/test/mysql-cdc/two-source-schemas.td index 11e9d5e06157c..7fb00fe97376c 100644 --- a/test/mysql-cdc/two-source-schemas.td +++ b/test/mysql-cdc/two-source-schemas.td @@ -48,7 +48,7 @@ INSERT INTO schema2.t1 SELECT * FROM schema2.t1; contains:catalog item 't1' already exists -# TODO: materialize#24797 +# TODO: database-issues#7397 # CREATE TABLE t1 FROM SOURCE mz_source (REFERENCE t1); # contains:table t1 is ambiguous, consider specifying the schema diff --git a/test/mysql-cdc/types-enum.td b/test/mysql-cdc/types-enum.td index 23f7133e582f6..af475050de7cb 100644 --- a/test/mysql-cdc/types-enum.td +++ b/test/mysql-cdc/types-enum.td @@ -34,7 +34,7 @@ INSERT INTO enum_type VALUES ('val1', 'val1'), ('val2', 'val2'); > CREATE SOURCE mz_source FROM MYSQL CONNECTION mysql_conn; -# TODO: materialize#25881 (enum unsupported) +# TODO: database-issues#7719 (enum unsupported) ! CREATE TABLE enum_type FROM SOURCE mz_source (REFERENCE public.enum_type); contains:referenced tables use unsupported types @@ -73,7 +73,7 @@ $ mysql-execute name=mysql ALTER TABLE enum_type CHANGE f1 f1 ENUM ('val2', 'val1', 'val3', 'val4'); INSERT INTO enum_type VALUES ('val1', 'val1'); -# TODO: materialize#26128 (expected incompatible schema change) +# TODO: database-issues#7776 (expected incompatible schema change) $ skip-if SELECT true diff --git a/test/pg-cdc-old-syntax/gh-10981.td b/test/pg-cdc-old-syntax/gh-10981.td index f8848de0efba0..0494fd76bb48c 100644 --- a/test/pg-cdc-old-syntax/gh-10981.td +++ b/test/pg-cdc-old-syntax/gh-10981.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # -# Test for issue materialize#10981 +# Test for issue database-issues#3215 # > CREATE SECRET pgpass AS 'postgres' diff --git a/test/pg-cdc-old-syntax/mzcompose.py b/test/pg-cdc-old-syntax/mzcompose.py index 8e7d37f723893..0e83052b6a0c8 100644 --- a/test/pg-cdc-old-syntax/mzcompose.py +++ b/test/pg-cdc-old-syntax/mzcompose.py @@ -114,7 +114,7 @@ def get_targeted_pg_version(parser: WorkflowArgumentParser) -> str | None: return pg_version -# TODO: redesign ceased status materialize#25768 +# TODO: redesign ceased status database-issues#7687 # Test that how subsource statuses work across a variety of scenarios # def workflow_statuses(c: Composition, parser: WorkflowArgumentParser) -> None: # c.up("materialized", "postgres", "toxiproxy") @@ -337,11 +337,11 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: if name == "default": continue - # TODO: Flaky, reenable when materialize#25479 is fixed + # TODO: Flaky, reenable when database-issues#7611 is fixed if name == "statuses": continue - # TODO: Flaky, reenable when materialize#28989 is fixed + # TODO: Flaky, reenable when database-issues#8447 is fixed if name == "silent-connection-drop": continue diff --git a/test/pg-cdc-old-syntax/pg-cdc-ssl.td b/test/pg-cdc-old-syntax/pg-cdc-ssl.td index 40123f2bda4ce..3ba1d0692c596 100644 --- a/test/pg-cdc-old-syntax/pg-cdc-ssl.td +++ b/test/pg-cdc-old-syntax/pg-cdc-ssl.td @@ -18,7 +18,7 @@ # Check out https://www.postgresql.org/docs/13/auth-pg-hba-conf.html # for more details. -# TODO: Reenable when materialize#14037 is fixed +# TODO: Reenable when database-issues#4009 is fixed $ skip-if SELECT true diff --git a/test/pg-cdc-old-syntax/pg-cdc.td b/test/pg-cdc-old-syntax/pg-cdc.td index caa0bea5a2531..cc1e5f38dc8cb 100644 --- a/test/pg-cdc-old-syntax/pg-cdc.td +++ b/test/pg-cdc-old-syntax/pg-cdc.td @@ -152,7 +152,7 @@ CREATE PUBLICATION another_publication FOR TABLE another_schema.another_table; # # Test that slots created for replication sources are deleted on DROP -# TODO: enable once we land materialize#24586 +# TODO: enable once we land database-issues#7327 # $ postgres-verify-slot connection=postgres://postgres:postgres@postgres slot=materialize_% active=false # Sneak in a test for pg_source_snapshot_statement_timeout, pg_source_wal_sender_timeout @@ -166,7 +166,7 @@ ALTER SYSTEM SET pg_source_wal_sender_timeout = 0; FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FOR TABLES ("pk_table"); -# TODO: enable once we land materialize#24586 +# TODO: enable once we land database-issues#7327 # $ postgres-verify-slot connection=postgres://postgres:postgres@postgres slot=materialize_% active=true > SHOW SUBSOURCES ON test_slot_source @@ -175,7 +175,7 @@ test_slot_source_progress progress > DROP SOURCE test_slot_source CASCADE -# TODO: enable once we land materialize#24586 +# TODO: enable once we land database-issues#7327 # $ postgres-verify-slot connection=postgres://postgres:postgres@postgres slot=materialize_% active=false $ postgres-execute connection=mz_system diff --git a/test/pg-cdc-old-syntax/publication-with-publish-option.td b/test/pg-cdc-old-syntax/publication-with-publish-option.td index be2fc8c0f0afe..6fc32aa199899 100644 --- a/test/pg-cdc-old-syntax/publication-with-publish-option.td +++ b/test/pg-cdc-old-syntax/publication-with-publish-option.td @@ -52,7 +52,7 @@ CREATE PUBLICATION mz_source_delete FOR TABLE t1 WITH ( publish = 'delete') ; # events post-snapshot. # # Currently there is no good way to do this other than sleeping. -# tracks +# tracks # making this better $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=10s > SELECT COUNT(*) FROM mz_source_insert; diff --git a/test/pg-cdc-old-syntax/replica-identity-default-nothing.td b/test/pg-cdc-old-syntax/replica-identity-default-nothing.td index aa4ad2d7b217e..9eaac699ac5a4 100644 --- a/test/pg-cdc-old-syntax/replica-identity-default-nothing.td +++ b/test/pg-cdc-old-syntax/replica-identity-default-nothing.td @@ -14,7 +14,7 @@ # IDENTITY DEFAULT is the same as USING INDEX (t1_pkey) # -# TODO: Reenable when materialize#14802 is fixed +# TODO: Reenable when database-issues#4231 is fixed $ skip-if SELECT true diff --git a/test/pg-cdc-old-syntax/status/04-drop-publication.td b/test/pg-cdc-old-syntax/status/04-drop-publication.td index 3e2884138206f..0760f078a6ed9 100644 --- a/test/pg-cdc-old-syntax/status/04-drop-publication.td +++ b/test/pg-cdc-old-syntax/status/04-drop-publication.td @@ -23,7 +23,7 @@ true > SELECT error ILIKE '%publication "mz_source" does not exist' FROM mz_internal.mz_source_statuses WHERE name = 't'; true -# TODO: This should be made reliable without sleeping, materialize#25479 +# TODO: This should be made reliable without sleeping, database-issues#7611 $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s ! SELECT * FROM t; diff --git a/test/pg-cdc/alter-source.td b/test/pg-cdc/alter-source.td index c9762916abf37..710380be0e754 100644 --- a/test/pg-cdc/alter-source.td +++ b/test/pg-cdc/alter-source.td @@ -290,7 +290,7 @@ INSERT INTO table_a VALUES (3); ! SELECT * FROM table_a; contains:incompatible schema change -# TODO: materialize#29373 (introspection tables) +# TODO: database-issues#8511 (introspection tables) # > SELECT error ~~ '%incompatible schema change%' FROM mz_internal.mz_source_statuses WHERE name = 'table_a' and type = 'table'; # true diff --git a/test/pg-cdc/gh-10981.td b/test/pg-cdc/gh-10981.td index a0f9e43316391..9d0bff46df433 100644 --- a/test/pg-cdc/gh-10981.td +++ b/test/pg-cdc/gh-10981.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # -# Test for issue materialize#10981 +# Test for issue database-issues#3215 # > CREATE SECRET pgpass AS 'postgres' diff --git a/test/pg-cdc/mzcompose.py b/test/pg-cdc/mzcompose.py index 4e12d3d8f22d9..9cc9fb5483f83 100644 --- a/test/pg-cdc/mzcompose.py +++ b/test/pg-cdc/mzcompose.py @@ -114,7 +114,7 @@ def get_targeted_pg_version(parser: WorkflowArgumentParser) -> str | None: return pg_version -# TODO: redesign ceased status materialize#25768 +# TODO: redesign ceased status database-issues#7687 # Test that how subsource statuses work across a variety of scenarios # def workflow_statuses(c: Composition, parser: WorkflowArgumentParser) -> None: # c.up("materialized", "postgres", "toxiproxy") @@ -337,11 +337,11 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: if name == "default": continue - # TODO: Flaky, reenable when materialize#25479 is fixed + # TODO: Flaky, reenable when database-issues#7611 is fixed if name == "statuses": continue - # TODO: Flaky, reenable when materialize#28989 is fixed + # TODO: Flaky, reenable when database-issues#8447 is fixed if name == "silent-connection-drop": continue diff --git a/test/pg-cdc/pg-cdc-ssl.td b/test/pg-cdc/pg-cdc-ssl.td index 985441f814b43..edb94af2be150 100644 --- a/test/pg-cdc/pg-cdc-ssl.td +++ b/test/pg-cdc/pg-cdc-ssl.td @@ -18,7 +18,7 @@ # Check out https://www.postgresql.org/docs/13/auth-pg-hba-conf.html # for more details. -# TODO: Reenable when materialize#14037 is fixed +# TODO: Reenable when database-issues#4009 is fixed $ skip-if SELECT true diff --git a/test/pg-cdc/pg-cdc.td b/test/pg-cdc/pg-cdc.td index a0e3ec90de62e..23b64482690c0 100644 --- a/test/pg-cdc/pg-cdc.td +++ b/test/pg-cdc/pg-cdc.td @@ -152,7 +152,7 @@ CREATE PUBLICATION another_publication FOR TABLE another_schema.another_table; # # Test that slots created for replication sources are deleted on DROP -# TODO: enable once we land materialize#24586 +# TODO: enable once we land database-issues#7327 # $ postgres-verify-slot connection=postgres://postgres:postgres@postgres slot=materialize_% active=false # Sneak in a test for pg_source_snapshot_statement_timeout, pg_source_wal_sender_timeout @@ -167,7 +167,7 @@ ALTER SYSTEM SET pg_source_wal_sender_timeout = 0; > CREATE TABLE pk_table FROM SOURCE "test_slot_source" (REFERENCE pk_table); -# TODO: enable once we land materialize#24586 +# TODO: enable once we land database-issues#7327 # $ postgres-verify-slot connection=postgres://postgres:postgres@postgres slot=materialize_% active=true > SHOW SUBSOURCES ON test_slot_source @@ -175,7 +175,7 @@ test_slot_source_progress progress > DROP SOURCE test_slot_source CASCADE -# TODO: enable once we land materialize#24586 +# TODO: enable once we land database-issues#7327 # $ postgres-verify-slot connection=postgres://postgres:postgres@postgres slot=materialize_% active=false $ postgres-execute connection=mz_system @@ -337,7 +337,7 @@ public trailing_space_nopk public "\"\"\"literal_quotes\"\"\"" conflict_schema conflict_table -# TODO: materialize#29373 (introspection tables) +# TODO: database-issues#8511 (introspection tables) # Ensure all ingestion export subsources have an ID greater than the primary source ID > SELECT bool_and(primary_source_id < subsource_id) FROM @@ -593,7 +593,7 @@ true 2 two_two 5 five -# TODO: materialize#29373 (introspection tables) +# TODO: database-issues#8511 (introspection tables) # > SELECT status = 'running' FROM mz_internal.mz_source_statuses WHERE name = 'large_cluster_source_pk_table' AND type = 'table'; # true diff --git a/test/pg-cdc/publication-with-publish-option.td b/test/pg-cdc/publication-with-publish-option.td index 4eaa71f69e8ab..6b6834e8db132 100644 --- a/test/pg-cdc/publication-with-publish-option.td +++ b/test/pg-cdc/publication-with-publish-option.td @@ -52,7 +52,7 @@ CREATE PUBLICATION mz_source_delete FOR TABLE t1 WITH ( publish = 'delete') ; # events post-snapshot. # # Currently there is no good way to do this other than sleeping. -# tracks +# tracks # making this better $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=10s > SELECT COUNT(*) FROM mz_source_insert; diff --git a/test/pg-cdc/replica-identity-default-nothing.td b/test/pg-cdc/replica-identity-default-nothing.td index 3b440849934e0..1a940364242eb 100644 --- a/test/pg-cdc/replica-identity-default-nothing.td +++ b/test/pg-cdc/replica-identity-default-nothing.td @@ -14,7 +14,7 @@ # IDENTITY DEFAULT is the same as USING INDEX (t1_pkey) # -# TODO: Reenable when materialize#14802 is fixed +# TODO: Reenable when database-issues#4231 is fixed $ skip-if SELECT true diff --git a/test/pg-cdc/statistics.td b/test/pg-cdc/statistics.td index a67aa5c196e0c..868d7118813c1 100644 --- a/test/pg-cdc/statistics.td +++ b/test/pg-cdc/statistics.td @@ -72,7 +72,7 @@ SELECT $ postgres-execute connection=postgres://postgres:postgres@postgres INSERT INTO t1 VALUES ('three'); -# TODO: materialize#29373 (introspection tables) +# TODO: database-issues#8511 (introspection tables) # > SELECT # s.name, # SUM(u.offset_committed) > ${previous-offset-committed}, @@ -137,7 +137,7 @@ true ORDER BY s.name mz_source 1 1 -# TODO: materialize#29373 (introspection tables) +# TODO: database-issues#8511 (introspection tables) # # Ensure subsource stats show up, and then are removed when we drop subsources. # > SELECT # t.name, diff --git a/test/pg-cdc/status/04-drop-publication.td b/test/pg-cdc/status/04-drop-publication.td index 3e2884138206f..0760f078a6ed9 100644 --- a/test/pg-cdc/status/04-drop-publication.td +++ b/test/pg-cdc/status/04-drop-publication.td @@ -23,7 +23,7 @@ true > SELECT error ILIKE '%publication "mz_source" does not exist' FROM mz_internal.mz_source_statuses WHERE name = 't'; true -# TODO: This should be made reliable without sleeping, materialize#25479 +# TODO: This should be made reliable without sleeping, database-issues#7611 $ sleep-is-probably-flaky-i-have-justified-my-need-with-a-comment duration=2s ! SELECT * FROM t; diff --git a/test/platform-checks/mzcompose.py b/test/platform-checks/mzcompose.py index f8e587ae3681a..d518db4034033 100644 --- a/test/platform-checks/mzcompose.py +++ b/test/platform-checks/mzcompose.py @@ -48,7 +48,7 @@ TestCerts(), Cockroach( setup_materialize=True, - # Workaround for materialize#19810 + # Workaround for database-issues#5899 restart="on-failure:5", ), Minio(setup_materialize=True, additional_directories=["copytos3"]), diff --git a/test/replica-isolation/mzcompose.py b/test/replica-isolation/mzcompose.py index 881f8f495c712..d9b2a2a377a1d 100644 --- a/test/replica-isolation/mzcompose.py +++ b/test/replica-isolation/mzcompose.py @@ -453,7 +453,7 @@ def run_test(c: Composition, disruption: Disruption, id: int) -> None: # frontier forward. If the targeted replica is crashing, the write # frontier cannot advance and thus the read frontier cannot either. # - # TODO(materialize#27399): Fix this by installing targeted subscribes only on the + # TODO(database-issues#8091): Fix this by installing targeted subscribes only on the # targeted replica. c.sql( "ALTER SYSTEM SET enable_introspection_subscribes = false;", diff --git a/test/restart/mzcompose.py b/test/restart/mzcompose.py index c4a0b7b3bd4ac..c24cd0a197a57 100644 --- a/test/restart/mzcompose.py +++ b/test/restart/mzcompose.py @@ -213,7 +213,7 @@ def workflow_audit_log(c: Composition) -> None: raise Exception("audit logs emtpy or not equal after restart") -# Test for GitHub issue materialize#13726 +# Test for GitHub issue database-issues#3923 def workflow_timelines(c: Composition) -> None: for _ in range(3): c.up("zookeeper", "kafka", "schema-registry", "materialized") @@ -250,7 +250,7 @@ def workflow_stash(c: Composition) -> None: cursor.execute("CREATE TABLE b (i INT)") # No implicit restart as sanity check here, will panic: - # https://github.com/MaterializeInc/materialize/issues/20510 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/6168 c.down(sanity_restart_mz=False) diff --git a/test/retain-history/mzcompose.py b/test/retain-history/mzcompose.py index 8407bcb053e98..f9ba3ae67c973 100644 --- a/test/retain-history/mzcompose.py +++ b/test/retain-history/mzcompose.py @@ -31,7 +31,7 @@ def workflow_default(c: Composition) -> None: run_test_with_mv_on_table_with_altered_retention(c) run_test_with_mv_on_counter_source(c) run_test_with_counter_source(c) - # TODO: materialize#24479 needs to be fixed + # TODO: database-issues#7310 needs to be fixed # run_test_gh_24479(c) run_test_with_index(c) run_test_consistency(c) diff --git a/test/rqg/mzcompose.py b/test/rqg/mzcompose.py index c9f2ba2f20cb6..26a07f6a1b38c 100644 --- a/test/rqg/mzcompose.py +++ b/test/rqg/mzcompose.py @@ -147,7 +147,7 @@ def dataset_files(self) -> list[str]: reference_implementation=None, validator="QueryProperties,RepeatableRead", ), - # Added as part of MaterializeInc/materialize#25340. + # Added as part of MaterializeInc/database-issues#7561. Workload( name="left-join-stacks", dataset=Dataset.STAR_SCHEMA, diff --git a/test/source-sink-errors/mzcompose.py b/test/source-sink-errors/mzcompose.py index 5ae6a80bd413b..2f0027940cf3d 100644 --- a/test/source-sink-errors/mzcompose.py +++ b/test/source-sink-errors/mzcompose.py @@ -194,7 +194,7 @@ def populate(self, c: Composition) -> None: ) FORMAT BYTES ENVELOPE NONE - # WITH ( REMOTE 'clusterd:2100' ) https://github.com/MaterializeInc/materialize/issues/16582 + # WITH ( REMOTE 'clusterd:2100' ) https://api.github.com/repos/MaterializeInc/database-issues/issues/4800 # Ensure the source makes _real_ progress before we disrupt it. This also # ensures the sink makes progress, which is required to hit certain stalls. @@ -206,7 +206,7 @@ def populate(self, c: Composition) -> None: INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-sink-topic-${testdrive.seed}') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM - # WITH ( REMOTE 'clusterd:2100' ) https://github.com/MaterializeInc/materialize/issues/16582 + # WITH ( REMOTE 'clusterd:2100' ) https://api.github.com/repos/MaterializeInc/database-issues/issues/4800 $ kafka-verify-topic sink=materialize.public.sink1 """ @@ -303,13 +303,13 @@ def populate(self, c: Composition) -> None: FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-source-topic-${testdrive.seed}') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE NONE - # WITH ( REMOTE 'clusterd:2100' ) https://github.com/MaterializeInc/materialize/issues/16582 + # WITH ( REMOTE 'clusterd:2100' ) https://api.github.com/repos/MaterializeInc/database-issues/issues/4800 > CREATE SINK sink1 FROM source1 INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-sink-topic-${testdrive.seed}') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM - # WITH ( REMOTE 'clusterd:2100' ) https://github.com/MaterializeInc/materialize/issues/16582 + # WITH ( REMOTE 'clusterd:2100' ) https://api.github.com/repos/MaterializeInc/database-issues/issues/4800 $ kafka-verify-data format=avro sink=materialize.public.sink1 sort-messages=true {"before": null, "after": {"row":{"f1": "A"}}} @@ -435,7 +435,7 @@ def assert_recovery(self, c: Composition) -> None: $ postgres-execute connection=postgres://postgres:postgres@postgres INSERT INTO source1 VALUES (3); - # TODO: materialize#29373 (introspection tables) + # TODO: database-issues#8511 (introspection tables) # > SELECT status, error # FROM mz_internal.mz_source_statuses # WHERE name = 'source1' @@ -499,7 +499,7 @@ def assert_recovery(self, c: Composition) -> None: expected_error="BrokerTransportFailure|Resolve|Broker transport failure|Timed out", fixage=lambda c, _: c.up("redpanda"), ), - # https://github.com/MaterializeInc/materialize/issues/16582 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/4800 # KafkaDisruption( # name="kill-redpanda-clusterd", # breakage=lambda c, _: c.kill("redpanda", "clusterd"), @@ -527,14 +527,14 @@ def assert_recovery(self, c: Composition) -> None: # Can't recover when publication state is deleted. fixage=None, ), - # TODO: materialize#29373 (introspection tables) + # TODO: database-issues#8511 (introspection tables) # PgDisruption( # name="alter-postgres", # breakage=lambda c, _: alter_pg_table(c), # expected_error="source table source1 with oid .+ has been altered", # fixage=None, # ), - # TODO: materialize#29373 (introspection tables) + # TODO: database-issues#8511 (introspection tables) # PgDisruption( # name="unsupported-postgres", # breakage=lambda c, _: unsupported_pg_table(c), diff --git a/test/sqllogictest/aggregates.slt b/test/sqllogictest/aggregates.slt index cb4fd1db5654e..a966843971df1 100644 --- a/test/sqllogictest/aggregates.slt +++ b/test/sqllogictest/aggregates.slt @@ -364,7 +364,7 @@ SELECT v, variance(k) FILTER (WHERE k > 5), stddev(k) FILTER (WHERE k > 5) FROM NULL NULL NULL -# Multiple tests related to distinctness of aggregates on constants (issue materialize#2535) +# Multiple tests related to distinctness of aggregates on constants (issue database-issues#887) query I rowsort select count(distinct column1) from (values (1)) _; ---- diff --git a/test/sqllogictest/arrays.slt b/test/sqllogictest/arrays.slt index 32f10ebfa0ecf..4b1a15e33e9ec 100644 --- a/test/sqllogictest/arrays.slt +++ b/test/sqllogictest/arrays.slt @@ -911,7 +911,7 @@ SELECT ARRAY[ARRAY[customer.first_name], ARRAY[customer.zip], ARRAY[customer.id: {{alice},{10003},{1}} {{charlie},{11217},{3}} -# Regression for materialize#18144 +# Regression for database-issues#5326 query error array_in not yet supported SELECT array_in('', 0, 0) ---- @@ -1393,7 +1393,7 @@ SELECT ARRAY[2,7] @> ARRAY[1,7,4,2,6] AS is_contained_by ---- false -# verify fix for issue materialize#28035 +# verify fix for issue database-issues#8223 query T SELECT '{}'::TEXT[][] @> '{{a, b}, {a, A}}'::TEXT[][]; ---- diff --git a/test/sqllogictest/char.slt b/test/sqllogictest/char.slt index ea81fb97de19f..85ea73c5f937b 100644 --- a/test/sqllogictest/char.slt +++ b/test/sqllogictest/char.slt @@ -16,7 +16,7 @@ # 2.0 license, a copy of which can be found in the LICENSE file at the # root of this repository. -# Fixes materialize#17871 +# Fixes database-issues#5212 query T SELECT 'a'::character = 'a'::"char"; @@ -26,7 +26,7 @@ true query error coalesce could not convert type character to "char" SELECT pg_typeof(coalesce('1'::"char", '1'::char)); -# Fixes materialize#17807 +# Fixes database-issues#5191 query T SELECT 'a'::"char"::char; @@ -88,7 +88,7 @@ SELECT pg_typeof('abc'::text::"char"); ---- "char" -# Fixes materialize#17899 +# Fixes database-issues#5222 query error db error: ERROR: coalesce could not convert type "char" to character SELECT COALESCE('a'::char, 'a'::"char"); diff --git a/test/sqllogictest/chbench.slt b/test/sqllogictest/chbench.slt index 8ef04f31e74c3..ac13e112cfeae 100644 --- a/test/sqllogictest/chbench.slt +++ b/test/sqllogictest/chbench.slt @@ -28,7 +28,7 @@ CREATE TABLE warehouse ( statement ok CREATE TABLE district ( - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 d_id integer, d_w_id integer, d_name char(10), @@ -48,9 +48,9 @@ CREATE INDEX fk_district_warehouse ON district (d_w_id ASC) statement ok CREATE TABLE customer ( - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 c_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 c_d_id integer, c_w_id integer, c_first char(16), @@ -68,9 +68,9 @@ CREATE TABLE customer ( c_discount decimal(4, 4), c_balance decimal(12, 2), c_ytd_payment decimal(12, 2), - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 c_payment_cnt integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 c_delivery_cnt integer, c_data text, c_n_nationkey integer, @@ -85,12 +85,12 @@ CREATE INDEX fk_customer_nation ON customer(c_n_nationkey ASC) statement ok CREATE TABLE history ( - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 h_c_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 h_c_d_id integer, h_c_w_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 h_d_id integer, h_w_id integer, h_date date, @@ -107,7 +107,7 @@ CREATE INDEX fk_history_district ON history (h_w_id ASC, h_d_id ASC) statement ok CREATE TABLE neworder ( no_o_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 no_d_id integer, no_w_id integer, PRIMARY KEY (no_w_id, no_d_id, no_o_id) @@ -116,17 +116,17 @@ CREATE TABLE neworder ( statement ok CREATE TABLE "order" ( o_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 o_d_id integer, o_w_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 o_c_id integer, o_entry_d date, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 o_carrier_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 o_ol_cnt integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 o_all_local integer, PRIMARY KEY (o_w_id, o_d_id, o_id) ) @@ -137,15 +137,15 @@ CREATE INDEX fk_order_customer ON order (o_w_id ASC, o_d_id ASC, o_c_id ASC) statement ok CREATE TABLE orderline ( ol_o_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 ol_d_id integer, ol_w_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 ol_number integer, ol_i_id integer, ol_supply_w_id integer, ol_delivery_d date, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 ol_quantity integer, ol_amount decimal(6, 2), ol_dist_info char(24), @@ -164,7 +164,7 @@ CREATE INDEX fk_orderline_item ON orderline (ol_i_id ASC) statement ok CREATE TABLE item ( i_id integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 i_im_id integer, i_name char(24), i_price decimal(5, 2), @@ -189,9 +189,9 @@ CREATE TABLE stock ( s_dist_09 char(24), s_dist_10 char(24), s_ytd integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 s_order_cnt integer, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 s_remote_cnt integer, s_data char(50), s_su_suppkey integer NOT NULL, @@ -209,10 +209,10 @@ CREATE INDEX fk_stock_supplier ON stock (s_su_suppkey ASC) statement ok CREATE TABLE nation ( - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 n_nationkey integer NOT NULL, n_name char(25) NOT NULL, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 n_regionkey integer NOT NULL, n_comment char(152) NOT NULL, PRIMARY KEY (n_nationkey) @@ -223,11 +223,11 @@ CREATE INDEX fk_nation_regionkey ON nation (n_regionkey ASC) statement ok CREATE TABLE supplier ( - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 su_suppkey integer NOT NULL, su_name char(25) NOT NULL, su_address char(40) NOT NULL, - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 su_nationkey integer NOT NULL, su_phone char(15) NOT NULL, su_acctbal decimal(12, 2) NOT NULL, @@ -240,7 +240,7 @@ CREATE INDEX fk_supplier_nationkey ON supplier (su_nationkey ASC) statement ok CREATE TABLE region ( - -- should be smallint, see materialize#4171 + -- should be smallint, see database-issues#1291 r_regionkey integer NOT NULL, r_name char(55) NOT NULL, r_comment char(152) NOT NULL, diff --git a/test/sqllogictest/cockroach/aggregate.slt b/test/sqllogictest/cockroach/aggregate.slt index 5c47e70525bd6..11970e9c8f436 100644 --- a/test/sqllogictest/cockroach/aggregate.slt +++ b/test/sqllogictest/cockroach/aggregate.slt @@ -216,7 +216,7 @@ SELECT 3 FROM kv HAVING TRUE query error column "kv.k" must appear in the GROUP BY clause or be used in an aggregate function SELECT count(*), k FROM kv -# materialize#3164 +# database-issues#1036 # query error unsupported comparison operator: < # SELECT count(*) FROM kv GROUP BY s < 5 @@ -531,7 +531,7 @@ SELECT length(s), count(DISTINCT k), count(DISTINCT v), count(DISTINCT (v)) FROM 1 5 2 2 NULL 1 0 0 -# materialize#1195 +# database-issues#414 # query I # SELECT count((k, v)) FROM kv # ---- @@ -772,7 +772,7 @@ SELECT max(y) FROM xyz WHERE x = 7 ---- NULL -# materialize#1195 +# database-issues#414 # query I # SELECT min(x) FROM xyz WHERE (y, z) = (2, 3.0) # ---- @@ -1007,7 +1007,7 @@ CREATE TABLE xy(x TEXT, y TEXT); statement ok INSERT INTO xy(x, y) VALUES ('a', 'b'), ('c', 'd') -# materialize#1195 +# database-issues#414 # query T rowsort # SELECT (b, a) FROM ab GROUP BY (b, a) # ---- @@ -1499,7 +1499,7 @@ FROM ( GROUP BY e.company_id ORDER BY e.company_id; -# This will differ from PG until we close https://github.com/MaterializeInc/materialize/issues/2415 +# This will differ from PG until we close https://api.github.com/repos/MaterializeInc/database-issues/issues/843 query TT SELECT e.company_id, string_agg(e.employee, ', ') FROM ( @@ -1522,7 +1522,7 @@ FROM ( GROUP BY e.company_id ORDER BY e.company_id; -# This will differ from PG until we close https://github.com/MaterializeInc/materialize/issues/2415 +# This will differ from PG until we close https://api.github.com/repos/MaterializeInc/database-issues/issues/843 query TT SELECT e.company_id, string_agg(e.employee, NULL) FROM ( @@ -1556,7 +1556,7 @@ SELECT string_agg('foo', CAST ((SELECT NULL) AS BYTEA)) OVER (); query error table functions are not allowed in aggregate function calls SELECT array_agg(generate_series(1, 2)) -# Regression test for materialize#31882. +# Regression test for cockroach#31882. statement ok CREATE TABLE uvw (u INT, v INT, w INT) @@ -1570,7 +1570,7 @@ SELECT u, v, array_agg(w) AS s FROM (SELECT * FROM uvw ORDER BY w) GROUP BY u, v 1 2 {3,3} 3 2 {1,3} -# Regression test for materialize#36433: don't panic with count_agg if a post-render produces an error. +# Regression test for cockroach#36433: don't panic with count_agg if a post-render produces an error. query error lpad SELECT count(*)::TEXT||lpad('foo', 23984729388383834723984) FROM (VALUES(1)); diff --git a/test/sqllogictest/cockroach/alter_table.slt b/test/sqllogictest/cockroach/alter_table.slt index cd2a65caa2774..63b787d04f6e9 100644 --- a/test/sqllogictest/cockroach/alter_table.slt +++ b/test/sqllogictest/cockroach/alter_table.slt @@ -440,7 +440,7 @@ ALTER TABLE t ADD d INT UNIQUE, ADD e INT UNIQUE, ADD f INT statement error pq: validation of CHECK "g = h" failed on row:.* g=3.* h=2 ALTER TABLE t ADD g INT DEFAULT 3, ADD h INT DEFAULT 2 CHECK (g = h) -# Multiple unique columns can be added, followed by other commands (materialize#35011) +# Multiple unique columns can be added, followed by other commands (cockroach#35011) statement ok ALTER TABLE t ADD COLUMN u INT UNIQUE, ADD COLUMN v INT UNIQUE, ADD CONSTRAINT ck CHECK (a > 0); @@ -720,7 +720,7 @@ statement error pgcode 42809 "privsview" is not a table ALTER TABLE privsview UNSPLIT AT VALUES (42) # Verify that impure defaults are evaluated separately on each row -# (materialize#14352) +# (database-issues#4105) statement ok CREATE TABLE impure (x INT); INSERT INTO impure(x) VALUES (1), (2), (3); @@ -902,7 +902,7 @@ statement ok ALTER TABLE vehicles DROP COLUMN mycol; # check that adding a reference on a column still being backfilled fails. -# fix through materialize#32917 +# fix through cockroach#32917 statement ok CREATE TABLE t32917 (a INT PRIMARY KEY) diff --git a/test/sqllogictest/cockroach/apply_join.slt b/test/sqllogictest/cockroach/apply_join.slt index 8e3fe7bffa5aa..21f94d21854c8 100644 --- a/test/sqllogictest/cockroach/apply_join.slt +++ b/test/sqllogictest/cockroach/apply_join.slt @@ -55,7 +55,7 @@ FROM 4 -# Regression test for materialize#36197: 0-col applyjoin RHS doesn't panic +# Regression test for cockroach#36197: 0-col applyjoin RHS doesn't panic statement ok CREATE TABLE table9 ( @@ -135,7 +135,7 @@ LIMIT ---- true -# Regression test for materialize#37454: untyped null produced at top level. +# Regression test for cockroach#37454: untyped null produced at top level. statement ok CREATE TABLE x (a INT) diff --git a/test/sqllogictest/cockroach/array.slt b/test/sqllogictest/cockroach/array.slt index b5393ef355437..6123500e98461 100644 --- a/test/sqllogictest/cockroach/array.slt +++ b/test/sqllogictest/cockroach/array.slt @@ -465,7 +465,7 @@ statement error column b is of type int\[\] and thus is not indexable CREATE TABLE a (b INT[] UNIQUE) -# Regression test for materialize#18745 +# Regression test for database-issues#5547 statement ok CREATE TABLE ident (x INT) @@ -1238,13 +1238,13 @@ CREATE TABLE v (y INT[]) statement ok INSERT INTO v VALUES (ARRAY[1, 2]) -# Regression test for materialize#30191. Ensure ArrayFlatten returns correct type. +# Regression test for cockroach#30191. Ensure ArrayFlatten returns correct type. query T SELECT * FROM v WHERE y = ARRAY(SELECT x FROM u ORDER BY x); ---- {1,2} -# Regression test for materialize#34439. Ensure that empty arrays are interned correctly. +# Regression test for cockroach#34439. Ensure that empty arrays are interned correctly. query B SELECT ARRAY[''] = ARRAY[] FROM (VALUES (1)) WHERE ARRAY[B''] != ARRAY[] ---- diff --git a/test/sqllogictest/cockroach/builtin_function.slt b/test/sqllogictest/cockroach/builtin_function.slt index aa388338dabfc..76dbace7aabb9 100644 --- a/test/sqllogictest/cockroach/builtin_function.slt +++ b/test/sqllogictest/cockroach/builtin_function.slt @@ -290,7 +290,7 @@ empty statement error pq: repeat\(\): requested length too large SELECT repeat('s', 9223372036854775807) -# Regression for materialize#19035. +# Regression for database-issues#5638. statement error pq: repeat\(\): requested length too large SELECT repeat('1234567890'::string, 6978072892806141784::int) @@ -1232,7 +1232,7 @@ SELECT regexp_replace('abc', 'c', 'a\', 'w') ---- aba\ -# materialize#19046 +# database-issues#5644 query T SELECT regexp_replace('ReRe','R(e)','1\\1','g'); ---- @@ -1435,21 +1435,21 @@ SELECT strpos(version(), 'CockroachDB') query error pq: cos\(\): cannot use "\*" in this context SELECT cos(*) FROM system.namespace -# Don't panic with invalid names (materialize#8045) +# Don't panic with invalid names (database-issues#2461) query error cannot use "nonexistent.\*" without a FROM clause SELECT TRIM(TRAILING nonexistent.*[1]) query error rtrim\(\): cannot subscript type tuple SELECT TRIM(TRAILING foo.*[1]) FROM (VALUES (1)) AS foo(x) -# Don't panic with invalid names (materialize#8044) +# Don't panic with invalid names (database-issues#2460) query error cannot use "nonexistent.\*" without a FROM clause SELECT OVERLAY(nonexistent.* PLACING 'string' FROM 'string') query error unknown signature SELECT OVERLAY(foo.* PLACING 'string' FROM 'string') FROM (VALUES (1)) AS foo(x) -# Don't panic with invalid names (materialize#8023) +# Don't panic with invalid names (database-issues#2455) query error cannot use "nonexistent.\*" without a FROM clause SELECT nonexistent.* IS NOT TRUE @@ -1762,7 +1762,7 @@ timestamptz bytes int[] -# TODO(def-): Reenable after materialize#21961 is fixed +# TODO(def-): Reenable after database-issues#6599 is fixed # query T # VALUES (format_type('anyelement'::regtype, -1)), # (format_type('bit'::regtype, -1)), diff --git a/test/sqllogictest/cockroach/collatedstring.slt b/test/sqllogictest/cockroach/collatedstring.slt index 3e05ff7e9fa1a..340c1865a395f 100644 --- a/test/sqllogictest/cockroach/collatedstring.slt +++ b/test/sqllogictest/cockroach/collatedstring.slt @@ -327,7 +327,7 @@ SELECT a FROM foo ---- NULL -# Regression test for materialize#24449 +# Regression test for database-issues#7300 statement ok INSERT INTO foo VALUES ('aBcD' COLLATE en_u_ks_level2) diff --git a/test/sqllogictest/cockroach/computed.slt b/test/sqllogictest/cockroach/computed.slt index 20b497ce432c6..0ce066fd3a724 100644 --- a/test/sqllogictest/cockroach/computed.slt +++ b/test/sqllogictest/cockroach/computed.slt @@ -373,7 +373,7 @@ CREATE TABLE y ( r INT AS (1) STORED REFERENCES x ) -# Regression test for materialize#36036. +# Regression test for cockroach#36036. statement ok CREATE TABLE tt (i INT8 AS (1) STORED) @@ -779,7 +779,7 @@ CREATE TABLE x ( query error value type decimal doesn't match type int of column "a" INSERT INTO x VALUES(1.4) -# Regression test for materialize#34901: verify that builtins can be used in computed +# Regression test for cockroach#34901: verify that builtins can be used in computed # column expressions without a "memory budget exceeded" error while backfilling statement ok CREATE TABLE t34901 (x STRING) diff --git a/test/sqllogictest/cockroach/create_as.slt b/test/sqllogictest/cockroach/create_as.slt index 3ba808334e59a..e873c5650e3d0 100644 --- a/test/sqllogictest/cockroach/create_as.slt +++ b/test/sqllogictest/cockroach/create_as.slt @@ -117,7 +117,7 @@ CREATE TABLE foo2 (x) AS (VALUES(generate_series(1,3))) statement error pq: value type unknown cannot be used for table columns CREATE TABLE foo2 (x) AS (VALUES(NULL)) -# Check nulls are handled properly (materialize#13921) +# Check nulls are handled properly (database-issues#3988) query I CREATE TABLE foo3 (x) AS VALUES (1), (NULL); SELECT * FROM foo3 ORDER BY x ---- @@ -130,7 +130,7 @@ CREATE TABLE foo4 (x) AS SELECT EXISTS(SELECT * FROM foo3 WHERE x IS NULL); SELE ---- true -# Regression test for materialize#36930. +# Regression test for cockroach#36930. statement ok CREATE TABLE bar AS SELECT 1 AS a, 2 AS b, count(*) AS c FROM foo diff --git a/test/sqllogictest/cockroach/datetime.slt b/test/sqllogictest/cockroach/datetime.slt index 9628cb54c5fdc..49b09884cb951 100644 --- a/test/sqllogictest/cockroach/datetime.slt +++ b/test/sqllogictest/cockroach/datetime.slt @@ -1284,7 +1284,7 @@ SELECT INTERVAL '1-2 3 4:5:6' YEAR 1 year -# Test regression, materialize#20464. When a built-in returns a datum that does not +# Test regression, database-issues#6155. When a built-in returns a datum that does not # match the function signature, distSQL will panic on table scans. statement ok diff --git a/test/sqllogictest/cockroach/distinct_on.slt b/test/sqllogictest/cockroach/distinct_on.slt index 2a60c6c9574fd..ab92e9c3d79bf 100644 --- a/test/sqllogictest/cockroach/distinct_on.slt +++ b/test/sqllogictest/cockroach/distinct_on.slt @@ -232,7 +232,7 @@ SELECT DISTINCT ON (x) y, z, x FROM xyz ORDER BY x ASC, z DESC, y DESC 2 3 2 5 6 4 -# Regression test for materialize#35437: Discard extra ordering columns after performing +# Regression test for cockroach#35437: Discard extra ordering columns after performing # DISTINCT operation. query T SELECT (SELECT DISTINCT ON (a) a FROM abc ORDER BY a, b||'foo') || 'bar'; @@ -397,7 +397,7 @@ SELECT DISTINCT ON (x, y, z) pk1 FROM (SELECT * FROM xyz WHERE x >= 2) ORDER BY 6 7 -# Regression tests for materialize#34112: distinct on constant column. +# Regression tests for cockroach#34112: distinct on constant column. query II SELECT DISTINCT ON (x) x, y FROM xyz WHERE x = 1 ORDER BY x, y ---- diff --git a/test/sqllogictest/cockroach/insert.slt b/test/sqllogictest/cockroach/insert.slt index 277879936be1c..58b0583a74ee8 100644 --- a/test/sqllogictest/cockroach/insert.slt +++ b/test/sqllogictest/cockroach/insert.slt @@ -710,7 +710,7 @@ statement ok COMMIT # ------------------------------------------------------------------------------ -# Regression for materialize#35364. +# Regression for cockroach#35364. # ------------------------------------------------------------------------------ subtest regression_35364 diff --git a/test/sqllogictest/cockroach/join.slt b/test/sqllogictest/cockroach/join.slt index aa2a7420f9888..6d16a6db4ad9e 100644 --- a/test/sqllogictest/cockroach/join.slt +++ b/test/sqllogictest/cockroach/join.slt @@ -1011,7 +1011,7 @@ SELECT * FROM l RIGHT OUTER JOIN r USING (a) WHERE a = 4 4 -# Regression tests for mixed-type equality columns (materialize#22514). +# Regression tests for mixed-type equality columns (database-issues#6807). statement ok CREATE TABLE foo ( a INT, diff --git a/test/sqllogictest/cockroach/json.slt b/test/sqllogictest/cockroach/json.slt index 09fa167de2a3d..e955fbdda2bbb 100644 --- a/test/sqllogictest/cockroach/json.slt +++ b/test/sqllogictest/cockroach/json.slt @@ -716,7 +716,7 @@ SELECT '{"b":[],"c":{"a":"b"}}'::JSONB - array['a']; ---- {"b":[],"c":{"a":"b"}} -# Regression test for materialize#34756. +# Regression test for cockroach#34756. query T SELECT '{"b":[],"c":{"a":"b"}}'::JSONB - array['foo',NULL] ---- diff --git a/test/sqllogictest/cockroach/json_builtins.slt b/test/sqllogictest/cockroach/json_builtins.slt index d1913b5e71be9..c13f3dec04797 100644 --- a/test/sqllogictest/cockroach/json_builtins.slt +++ b/test/sqllogictest/cockroach/json_builtins.slt @@ -608,7 +608,7 @@ SELECT jsonb_build_array(1,'1'::JSON,1.2,NULL,ARRAY['x','y']) ---- [1,1,1.2,null,["x","y"]] -# Regression for materialize#37318 +# Regression for cockroach#37318 query T SELECT jsonb_build_array('+Inf'::FLOAT8,'NaN'::FLOAT8)::STRING::JSONB ---- diff --git a/test/sqllogictest/cockroach/ordinality.slt b/test/sqllogictest/cockroach/ordinality.slt index 4f84ba41e38bf..47c7f4d59335a 100644 --- a/test/sqllogictest/cockroach/ordinality.slt +++ b/test/sqllogictest/cockroach/ordinality.slt @@ -106,6 +106,6 @@ true true true -# Regression test for materialize#33659 +# Regression test for cockroach#33659 statement ok TABLE [SHOW ZONE CONFIGURATIONS] WITH ORDINALITY diff --git a/test/sqllogictest/cockroach/orms.slt b/test/sqllogictest/cockroach/orms.slt index 038bd9c263679..7bbad06ae1f5e 100644 --- a/test/sqllogictest/cockroach/orms.slt +++ b/test/sqllogictest/cockroach/orms.slt @@ -45,7 +45,7 @@ rowid bigint unique_rowid() true 20 -1 # materialize#12115 -# Skipped until materialize#25148 is solved +# Skipped until database-issues#7510 is solved #query TT #SELECT t.typname enum_name, array_agg(e.enumlabel ORDER BY enumsortorder) enum_value # FROM pg_type t diff --git a/test/sqllogictest/cockroach/pg_catalog.slt b/test/sqllogictest/cockroach/pg_catalog.slt index e5022f5f55b52..0bf27eb8c8c86 100644 --- a/test/sqllogictest/cockroach/pg_catalog.slt +++ b/test/sqllogictest/cockroach/pg_catalog.slt @@ -94,7 +94,7 @@ DROP DATABASE pg_catalog # the following query checks that the planDataSource instantiated from # a virtual table in the FROM clause is properly deallocated even when -# query preparation causes an error. materialize#9853 +# query preparation causes an error. database-issues#2980 query error unknown function SELECT * FROM pg_catalog.pg_class c WHERE nonexistent_function() diff --git a/test/sqllogictest/cockroach/pgoidtype.slt b/test/sqllogictest/cockroach/pgoidtype.slt index f9fdcb46b2788..af6c72c04368d 100644 --- a/test/sqllogictest/cockroach/pgoidtype.slt +++ b/test/sqllogictest/cockroach/pgoidtype.slt @@ -326,7 +326,7 @@ SELECT crdb_internal.create_regtype(10, 'foo')::oid, crdb_internal.create_regcla ---- 10 10 10 10 10 -# Regression test for materialize#32422: ensure that VALUES nodes properly retain special +# Regression test for cockroach#32422: ensure that VALUES nodes properly retain special # OID properties. query OO diff --git a/test/sqllogictest/cockroach/prepare.slt b/test/sqllogictest/cockroach/prepare.slt index f052d9103310f..d179b1a046514 100644 --- a/test/sqllogictest/cockroach/prepare.slt +++ b/test/sqllogictest/cockroach/prepare.slt @@ -126,7 +126,7 @@ EXECUTE a(1, 1, 1) query error wrong number of parameters for prepared statement \"a\": expected 2, got 0 EXECUTE a -# Regression test for materialize#36153. +# Regression test for cockroach#36153. statement error unknown signature: array_length\(int, int\) PREPARE fail AS SELECT array_length($1, 1) @@ -233,7 +233,7 @@ EXECUTE y statement DEALLOCATE ALL -# Regression test for materialize#16062 +# Regression test for database-issues#4634 statement CREATE TABLE IF NOT EXISTS f (v INT) diff --git a/test/sqllogictest/cockroach/rename_database.slt b/test/sqllogictest/cockroach/rename_database.slt index 383fb32046d38..a4823ad51adc2 100644 --- a/test/sqllogictest/cockroach/rename_database.slt +++ b/test/sqllogictest/cockroach/rename_database.slt @@ -183,7 +183,7 @@ EXPLAIN ALTER DATABASE v RENAME TO x ---- rename database · · -# Verify that the EXPLAIN above does not actually rename the database (materialize#30543) +# Verify that the EXPLAIN above does not actually rename the database (cockroach#30543) query T colnames SHOW DATABASES ---- diff --git a/test/sqllogictest/cockroach/rename_table.slt b/test/sqllogictest/cockroach/rename_table.slt index e5a5fdbdd8510..a252829a2a725 100644 --- a/test/sqllogictest/cockroach/rename_table.slt +++ b/test/sqllogictest/cockroach/rename_table.slt @@ -297,7 +297,7 @@ EXPLAIN ALTER TABLE kv RENAME TO kv2 ---- rename table · · -# Verify that the EXPLAIN above does not actually rename the table (materialize#30543) +# Verify that the EXPLAIN above does not actually rename the table (cockroach#30543) query T SHOW TABLES ---- diff --git a/test/sqllogictest/cockroach/select.slt b/test/sqllogictest/cockroach/select.slt index 8e48ceb27f9b8..241d073f2dc1b 100644 --- a/test/sqllogictest/cockroach/select.slt +++ b/test/sqllogictest/cockroach/select.slt @@ -172,7 +172,7 @@ SELECT kv.* FROM kv ---- a NULL -# Regression tests for materialize#24169 +# Regression tests for database-issues#7241 query TT SELECT test.kv.* FROM kv ---- diff --git a/test/sqllogictest/cockroach/sqlsmith.slt b/test/sqllogictest/cockroach/sqlsmith.slt index 1aa1865e19268..0d3143f525c01 100644 --- a/test/sqllogictest/cockroach/sqlsmith.slt +++ b/test/sqllogictest/cockroach/sqlsmith.slt @@ -41,7 +41,7 @@ CREATE TABLE a (a INT PRIMARY KEY); statement ok SELECT true FROM (SELECT ref_1.a AS c0 FROM crdb_internal.cluster_queries AS ref_0 JOIN a AS ref_1 ON (ref_0.node_id = ref_1.a) WHERE (SELECT a from a limit 1 offset 1) is null); -# Regression: materialize#34437 (union all could produce panic in distsql planning) +# Regression: cockroach#34437 (union all could produce panic in distsql planning) statement ok CREATE TABLE table8 (col1 TIME, col2 BYTEA, col4 OID, col6 NAME, col9 TIMESTAMP, PRIMARY KEY (col1)); @@ -54,7 +54,7 @@ INSERT INTO table8 (col1, col2, col4, col6) VALUES ('19:06:18.321589', NULL, NULL, NULL) UNION ALL (SELECT NULL, NULL, NULL, NULL FROM table5 AS tab_8); -# Regression: materialize#36441 (raw indexed var can't be type checked) +# Regression: cockroach#36441 (raw indexed var can't be type checked) query TO WITH with_20394 (col_162526) @@ -94,14 +94,14 @@ ORDER BY 1d6eaf81-8a2c-43c5-a495-a3b102917ab1 3697877132 d2d225e2-e9be-4420-a645-d1b8f577511c 3697877132 -# Regression: materialize#36830 (can't run wrapped window node) +# Regression: cockroach#36830 (can't run wrapped window node) statement ok CREATE TABLE table9 (a INT8); statement ok INSERT INTO table9 SELECT lag(a) OVER (PARTITION BY a) FROM table9; -# Regression: materialize#36607 (can't serialize or type-check arrays of NULL properly) +# Regression: cockroach#36607 (can't serialize or type-check arrays of NULL properly) query TTTT WITH with_194015 (col_1548014) diff --git a/test/sqllogictest/cockroach/srfs.slt b/test/sqllogictest/cockroach/srfs.slt index d0bccd68b2a66..1ca1627f4aacb 100644 --- a/test/sqllogictest/cockroach/srfs.slt +++ b/test/sqllogictest/cockroach/srfs.slt @@ -397,7 +397,7 @@ SELECT a.*, b.*, c.* FROM generate_series(1,1) a, unnest(ARRAY[1]) b, pg_get_key ---- a b word catcode catdesc -# Regression for materialize#36501: the column from a single-column SRF should not be +# Regression for cockroach#36501: the column from a single-column SRF should not be # renamed because of a higher-level table alias. query I colnames SELECT * FROM (SELECT * FROM generate_series(1, 2)) AS a @@ -427,7 +427,7 @@ ON uq.filter_id2 = ab.filter_id ---- 1 1 -# Beware of multi-valued SRFs in render position (materialize#19149) +# Beware of multi-valued SRFs in render position (database-issues#5675) query TTT colnames SELECT 'a' AS a, pg_get_keywords(), 'c' AS c LIMIT 1 ---- @@ -1046,7 +1046,7 @@ subtest liquibase_example_query # # Test from materialize#24713 (Liquibase) listed on materialize#16971. # # TODO(knz) Needs support for pg_get_indexdef with 3 arguments, -# # see materialize#26629. +# # see database-issues#7870. # query TTTBTTIITTTTT # SELECT NULL AS table_cat, # n.nspname AS table_schem, diff --git a/test/sqllogictest/cockroach/statement_source.slt b/test/sqllogictest/cockroach/statement_source.slt index 32c8fdaa919ed..ab55a985425d5 100644 --- a/test/sqllogictest/cockroach/statement_source.slt +++ b/test/sqllogictest/cockroach/statement_source.slt @@ -95,7 +95,7 @@ SELECT * FROM a ORDER BY b -2 3 7 8 -# Regression for materialize#30936: ensure that wrapped planNodes with non-needed columns work ok +# Regression for cockroach#30936: ensure that wrapped planNodes with non-needed columns work ok statement ok CREATE TABLE b (a int, b int) diff --git a/test/sqllogictest/cockroach/subquery-opt.slt b/test/sqllogictest/cockroach/subquery-opt.slt index 615ada96ac7cd..bebe2f3c19ed5 100644 --- a/test/sqllogictest/cockroach/subquery-opt.slt +++ b/test/sqllogictest/cockroach/subquery-opt.slt @@ -19,7 +19,7 @@ mode cockroach -# Regression test for materialize#37263. This test is broken in the heuristic planner +# Regression test for cockroach#37263. This test is broken in the heuristic planner # because it does not correctly type check subqueries. query B SELECT 3::decimal IN (SELECT 1) @@ -34,7 +34,7 @@ SELECT 1 IN (SELECT '1'); ---- true -# Regression test for materialize#14554. +# Regression test for database-issues#4155. query ITIIIII SELECT t.oid, t.typname, t.typsend, t.typreceive, t.typoutput, t.typinput, t.typelem FROM pg_type AS t diff --git a/test/sqllogictest/cockroach/subquery.slt b/test/sqllogictest/cockroach/subquery.slt index 3e03ffa2f2b4d..632fb7c37054a 100644 --- a/test/sqllogictest/cockroach/subquery.slt +++ b/test/sqllogictest/cockroach/subquery.slt @@ -261,7 +261,7 @@ UPDATE xyz SET z = (SELECT (10, 11)) WHERE x = 7 statement error [subquery must return 2 columns, found 1 | number of columns (2) does not match number of values (1)] UPDATE xyz SET (y, z) = (SELECT (11, 12)) WHERE x = 7 -#regression, see materialize#6852 +#regression, see database-issues#2135 #statement ok #UPDATE xyz SET (y, z) = (SELECT 11, 12) WHERE x = 7 # @@ -492,7 +492,7 @@ query I SELECT * FROM test2 WHERE 0 = CASE WHEN true THEN (SELECT a FROM test LIMIT 1) ELSE 10 END ---- -# Regression test for materialize#28335. +# Regression test for database-issues#8301. query I SELECT (SELECT ARRAY(SELECT 1))[1] ---- diff --git a/test/sqllogictest/cockroach/subquery_correlated.slt b/test/sqllogictest/cockroach/subquery_correlated.slt index aa59f581bd37f..c18a631726fc6 100644 --- a/test/sqllogictest/cockroach/subquery_correlated.slt +++ b/test/sqllogictest/cockroach/subquery_correlated.slt @@ -1023,7 +1023,7 @@ ON c.c_id=o.c_id AND EXISTS(SELECT * FROM o WHERE o.c_id=c.c_id AND ship IS NULL # 5 {} # 6 {90} -# Regression for issue materialize#24676: missing support for correlated subquery in JSON +# Regression for issue database-issues#7343: missing support for correlated subquery in JSON # operator. statement ok CREATE TABLE groups( @@ -1037,7 +1037,7 @@ INSERT INTO groups(id, data) VALUES(1, '{"name": "Group 1", "members": [{"name": statement ok INSERT INTO groups(id, data) VALUES(2, '{"name": "Group 2", "members": [{"name": "admin2", "type": "USER"}]}') -# materialize#1546 +# database-issues#544 # query TT # SELECT # g.data->>'name' AS group_name, @@ -1119,7 +1119,7 @@ INSERT INTO stuff(id, date, user_id) VALUES (5, '2007-06-15'::DATE, 3) statement ok INSERT INTO stuff(id, date, user_id) VALUES (6, '2007-03-15'::DATE, 3) -# materialize#2804 +# database-issues#949 # query ITITI # SELECT # users.id AS users_id, diff --git a/test/sqllogictest/cockroach/table.slt b/test/sqllogictest/cockroach/table.slt index 204ec06107b3e..fa1e10a4f0ae2 100644 --- a/test/sqllogictest/cockroach/table.slt +++ b/test/sqllogictest/cockroach/table.slt @@ -473,7 +473,7 @@ ALTER TABLE smtng.something ADD COLUMN IF NOT EXISTS MODEL_ID INT statement ok ALTER TABLE smtng.something ADD COLUMN IF NOT EXISTS NAME STRING -# Regression test for materialize#13725 +# Regression test for database-issues#3922 statement ok CREATE TABLE test.empty () diff --git a/test/sqllogictest/cockroach/tuple.slt b/test/sqllogictest/cockroach/tuple.slt index f4059def48cd3..dcd69ddb87660 100644 --- a/test/sqllogictest/cockroach/tuple.slt +++ b/test/sqllogictest/cockroach/tuple.slt @@ -270,7 +270,7 @@ k v statement ok DROP TABLE kv -# Issue materialize#12295 +# Issue database-issues#3567 query B colnames SELECT 'foo' IN (x, 'aaa') AS r FROM (SELECT 'foo' AS x FROM tb) diff --git a/test/sqllogictest/cockroach/update.slt b/test/sqllogictest/cockroach/update.slt index 7060706797058..482032e673b44 100644 --- a/test/sqllogictest/cockroach/update.slt +++ b/test/sqllogictest/cockroach/update.slt @@ -522,7 +522,7 @@ SELECT * FROM mutation m n o p 2 1 10 11 -#regression test for materialize#32477 +#regression test for cockroach#32477 subtest reject_special_funcs_inset statement ok @@ -537,7 +537,7 @@ UPDATE t32477 SET x = rank() OVER () statement error generator functions are not allowed in UPDATE SET UPDATE t32477 SET x = generate_series(1,2) -#regression test for materialize#32054 +#regression test for cockroach#32054 subtest empty_update_subquery statement ok @@ -555,7 +555,7 @@ SELECT * FROM t32054 NULL NULL # ------------------------------------------------------------------------------ -# Regression for materialize#35364. +# Regression for cockroach#35364. # ------------------------------------------------------------------------------ subtest regression_35364 @@ -574,7 +574,7 @@ SELECT x FROM t35364 1 # ------------------------------------------------------------------------------ -# Regression for materialize#35970. +# Regression for cockroach#35970. # ------------------------------------------------------------------------------ statement ok CREATE TABLE table35970 ( diff --git a/test/sqllogictest/cockroach/upsert.slt b/test/sqllogictest/cockroach/upsert.slt index 6fce5f8477354..b990c18c76902 100644 --- a/test/sqllogictest/cockroach/upsert.slt +++ b/test/sqllogictest/cockroach/upsert.slt @@ -892,7 +892,7 @@ statement ok DROP TABLE test35040 # ------------------------------------------------------------------------------ -# Regression for materialize#35364. +# Regression for cockroach#35364. # ------------------------------------------------------------------------------ subtest regression_35364 @@ -958,7 +958,7 @@ SELECT * FROM t35364 3 4 # ------------------------------------------------------------------------------ -# Regression for materialize#35970. +# Regression for cockroach#35970. # ------------------------------------------------------------------------------ statement ok CREATE TABLE table35970 ( diff --git a/test/sqllogictest/cockroach/views.slt b/test/sqllogictest/cockroach/views.slt index 5bebd3b3dbf4a..96d3f94761be7 100644 --- a/test/sqllogictest/cockroach/views.slt +++ b/test/sqllogictest/cockroach/views.slt @@ -286,7 +286,7 @@ SELECT sum ("QuotedCaps15951". a) FROM "QuotedCaps15951" GROUP BY b ORDER BY b 3 6 -# Regression tests for materialize#23833 +# Regression tests for database-issues#7154 statement ok CREATE VIEW w AS WITH a AS (SELECT 1 AS x) SELECT x FROM a diff --git a/test/sqllogictest/cockroach/window.slt b/test/sqllogictest/cockroach/window.slt index 75884f99c90cb..a769a0c702b30 100644 --- a/test/sqllogictest/cockroach/window.slt +++ b/test/sqllogictest/cockroach/window.slt @@ -2996,7 +2996,7 @@ Tablet iPad 700.00 NULL 700.00 Tablet Kindle Fire 150.00 NULL 150.00 Tablet Samsung 200.00 NULL 200.00 -# Test for materialize#32702 +# Test for cockroach#32702 statement ok CREATE TABLE x (a INT) @@ -3011,7 +3011,7 @@ SELECT a, json_agg(a) OVER (ORDER BY a) FROM x ORDER BY a 2 [1, 2] 3 [1, 2, 3] -# Test for materialize#35267 +# Test for cockroach#35267 query I SELECT row_number() OVER (PARTITION BY s) @@ -3022,7 +3022,7 @@ FROM 1 1 -# Tests for materialize#32050 +# Tests for cockroach#32050 statement error window function calls cannot be nested SELECT sum(a) OVER (PARTITION BY count(a) OVER ()) FROM x @@ -3073,7 +3073,7 @@ SELECT string_agg('foo', s) OVER () FROM (SELECT * FROM kv LIMIT 1) ---- foo -# Regression test for materialize#37201. +# Regression test for cockroach#37201. query I SELECT jsonb_agg(a) OVER (ORDER BY a GROUPS BETWEEN 5 FOLLOWING AND UNBOUNDED FOLLOWING) FROM x ---- diff --git a/test/sqllogictest/coercion.slt b/test/sqllogictest/coercion.slt index c48378f4be20c..bb06dc876f87c 100644 --- a/test/sqllogictest/coercion.slt +++ b/test/sqllogictest/coercion.slt @@ -10,6 +10,6 @@ statement ok CREATE TABLE decimal_zero (f1 DECIMAL) -# Regression test for https://github.com/MaterializeInc/materialize/issues/7360 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2297 statement ok INSERT INTO decimal_zero VALUES ('0.0'), (0) diff --git a/test/sqllogictest/cte.slt b/test/sqllogictest/cte.slt index ef3c9430afbb5..14c91523a837b 100644 --- a/test/sqllogictest/cte.slt +++ b/test/sqllogictest/cte.slt @@ -156,7 +156,7 @@ WHERE x IN ( ---- 1 1 -# Same query, but inside a view. Regression test for materialize#5092. +# Same query, but inside a view. Regression test for database-issues#1582. statement ok CREATE MATERIALIZED VIEW v AS SELECT * FROM squares @@ -243,7 +243,7 @@ WITH outermost(x) AS ( ) SELECT * FROM outermost ORDER BY 1; -# test materialize#23629 +# test database-issues#7102 query I colnames WITH count AS (VALUES (9)) SELECT count(*) FROM count; ---- diff --git a/test/sqllogictest/dates-times.slt b/test/sqllogictest/dates-times.slt index c49f4f8c76b36..9af627a028ff2 100644 --- a/test/sqllogictest/dates-times.slt +++ b/test/sqllogictest/dates-times.slt @@ -1345,7 +1345,7 @@ SELECT '"?!?2024-02-13 17:01:58.37848+ 0 0'::timestamp with time zone; query error invalid input syntax for type timestamp with time zone: Cannot parse timezone offset \+ \? 00: "\\"\?!\?2024\-02\-13 17:01:58\.37848\+ \? 00" SELECT '"?!?2024-02-13 17:01:58.37848+ ? 00'::timestamp with time zone; -# Regression for materialize#6272. These match postgres. +# Regression for database-issues#1933. These match postgres. query TTT select '9::60'::time, '9:59:60'::time, '9::59.999999'::time ---- @@ -1449,7 +1449,7 @@ SELECT to_timestamp(-0.1::float8); ---- 1969-12-31 23:59:59.9+00 -# Regression test for https://github.com/MaterializeInc/materialize/issues/20077 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6002 query T SELECT now() + null; ---- @@ -1661,7 +1661,7 @@ query error precision for type timestamp or timestamptz must be between 0 and 6 SELECT timestamptz(-1) '95143-12-31 23:59:59.123456789+06'; # Note: we regressed the error messages for the following -# See +# See query error Expected end of statement SELECT timestamp(10000000000000000000) '95143-12-31 23:59:59.123456789+06'; diff --git a/test/sqllogictest/encode.slt b/test/sqllogictest/encode.slt index 52a39f95d1d7f..af6e93a9d6dbf 100644 --- a/test/sqllogictest/encode.slt +++ b/test/sqllogictest/encode.slt @@ -100,7 +100,7 @@ NULL NULL query error invalid input syntax for type bytea SELECT decode('\9', 'escape') -# checks https://github.com/MaterializeInc/materialize/issues/11369 +# checks https://api.github.com/repos/MaterializeInc/database-issues/issues/3311 query T SELECT encode('se', 'base64') ---- diff --git a/test/sqllogictest/explain/broken_statements.slt b/test/sqllogictest/explain/broken_statements.slt index e7d40bb846dbc..fbee7f1cc3182 100644 --- a/test/sqllogictest/explain/broken_statements.slt +++ b/test/sqllogictest/explain/broken_statements.slt @@ -90,7 +90,7 @@ EXPLAIN OPTIMIZER TRACE FOR BROKEN CREATE MATERIALIZED VIEW mv AS SELECT mz_unsafe.mz_panic('forced optimizer panic'); -# Regression test for materialize#24398 +# Regression test for database-issues#7289 statement error internal error: stage `optimize/global` not present EXPLAIN OPTIMIZED PLAN FOR BROKEN CREATE MATERIALIZED VIEW mv AS diff --git a/test/sqllogictest/explain/optimized_plan_as_text.slt b/test/sqllogictest/explain/optimized_plan_as_text.slt index acef6c20adfb6..879af6203f45e 100644 --- a/test/sqllogictest/explain/optimized_plan_as_text.slt +++ b/test/sqllogictest/explain/optimized_plan_as_text.slt @@ -1034,7 +1034,7 @@ WITH MUTUALLY RECURSIVE bar (a int) as (SELECT a FROM foo) SELECT * FROM bar; -# Regression test for materialize#19148: support mz_now() on select from indexed table +# Regression test for database-issues#5674: support mz_now() on select from indexed table # --- simple conn=mz_system,user=mz_system diff --git a/test/sqllogictest/explain/optimized_plan_as_text_redacted.slt b/test/sqllogictest/explain/optimized_plan_as_text_redacted.slt index bb125c28932a6..1196e23f44419 100644 --- a/test/sqllogictest/explain/optimized_plan_as_text_redacted.slt +++ b/test/sqllogictest/explain/optimized_plan_as_text_redacted.slt @@ -513,7 +513,7 @@ WITH MUTUALLY RECURSIVE bar (a int) as (SELECT a FROM foo) SELECT * FROM bar; -# Regression test for materialize#19148: support mz_now() on select from indexed table +# Regression test for database-issues#5674: support mz_now() on select from indexed table # --- simple conn=mz_system,user=mz_system diff --git a/test/sqllogictest/explain/physical_plan_as_text.slt b/test/sqllogictest/explain/physical_plan_as_text.slt index ffc2a184c637d..f3750d812cdaa 100644 --- a/test/sqllogictest/explain/physical_plan_as_text.slt +++ b/test/sqllogictest/explain/physical_plan_as_text.slt @@ -1336,7 +1336,7 @@ Target cluster: quickstart EOF # Test `LetRec` printing, with and without RECURSION LIMIT -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN PHYSICAL PLAN AS TEXT FOR SELECT * FROM ( diff --git a/test/sqllogictest/explain/physical_plan_as_text_redacted.slt b/test/sqllogictest/explain/physical_plan_as_text_redacted.slt index aa0dc87f3cf79..583950f96ffee 100644 --- a/test/sqllogictest/explain/physical_plan_as_text_redacted.slt +++ b/test/sqllogictest/explain/physical_plan_as_text_redacted.slt @@ -1321,7 +1321,7 @@ Target cluster: quickstart EOF # Test `LetRec` printing, with and without RECURSION LIMIT -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN PHYSICAL PLAN WITH(redacted) AS TEXT FOR SELECT * FROM ( diff --git a/test/sqllogictest/extract.slt b/test/sqllogictest/extract.slt index d49035957c1dd..c816eb5ddce2e 100644 --- a/test/sqllogictest/extract.slt +++ b/test/sqllogictest/extract.slt @@ -28,7 +28,7 @@ asdfjkl asdf NULL jkl NULL jkl # TODO(brennan): test that the regex columns have the correct nullability, once -# they actually do (materialize#1685). +# they actually do (database-issues#612). query TTT colnames SELECT data.*, csv.* FROM data, csv_extract(2, data.input) csv diff --git a/test/sqllogictest/filter-pushdown.slt b/test/sqllogictest/filter-pushdown.slt index 99b663765d1f3..bd1b5d5d4375b 100644 --- a/test/sqllogictest/filter-pushdown.slt +++ b/test/sqllogictest/filter-pushdown.slt @@ -248,7 +248,7 @@ Target cluster: quickstart EOF -# Regression tests for https://github.com/MaterializeInc/materialize/issues/22040 +# Regression tests for https://api.github.com/repos/MaterializeInc/database-issues/issues/6640 statement ok create table t(x int, t timestamp); diff --git a/test/sqllogictest/float.slt b/test/sqllogictest/float.slt index b297519574c32..123aa492b52d2 100644 --- a/test/sqllogictest/float.slt +++ b/test/sqllogictest/float.slt @@ -152,7 +152,7 @@ SELECT SUM(f1) FROM t1 ---- -282409603651671152154661355520.000 -# This *should* be zero, known issue https://github.com/MaterializeInc/materialize/issues/15186 +# This *should* be zero, known issue https://api.github.com/repos/MaterializeInc/database-issues/issues/4341 query T SELECT MIN(f1+f2)-SUM(f1) FROM t1 ---- diff --git a/test/sqllogictest/funcs.slt b/test/sqllogictest/funcs.slt index 6c6b9f4cbf5f4..7cdc5794304b0 100644 --- a/test/sqllogictest/funcs.slt +++ b/test/sqllogictest/funcs.slt @@ -481,7 +481,7 @@ SELECT coalesce('abcde',null::char(1),'abc'); ---- abcde -# TODO(materialize#11469) +# TODO(database-issues#3339) query error coalesce could not convert type record SELECT coalesce(row(1, 2), row(3), row(4, 5)); @@ -568,7 +568,7 @@ SELECT greatest(row(row(2, 4), 5), row(row(0, 10), 10), row(row(4, 3), 4)); ---- ("(4,3)",4) -# TODO(materialize#11469) +# TODO(database-issues#3339) query error greatest could not convert type record SELECT greatest(row(1, 2), row(3), row(4, 5)); @@ -629,7 +629,7 @@ SELECT least(row(row(2, 4), 5), row(row(0, 10), 10), row(row(4, 3), 4)); ---- ("(0,10)",10) -# TODO(materialize#11469) +# TODO(database-issues#3339) query error least could not convert type record SELECT least(row(1, 2), row(3), row(4, 5)); @@ -644,7 +644,7 @@ SELECT least(row(1), row(1)::custom_composite) query error least types integer and text cannot be matched SELECT least(1::int, 2::text) -# Tests issue materialize#2355, that type information for Maps are correctly constructed +# Tests issue database-issues#827, that type information for Maps are correctly constructed # before being passed to expressions for optimization. statement ok CREATE VIEW bytes AS SELECT null::bytea AS data @@ -991,7 +991,7 @@ NULL query error could not determine polymorphic type because input has type unknown SELECT array_length(NULL, 1); -# Test strange collapsing behavior of nested empty arrays. See materialize#5545. +# Test strange collapsing behavior of nested empty arrays. See database-issues#1709. query TIII SELECT arr, array_lower(arr, 1), array_upper(arr, 1), array_length(arr, 1) FROM diff --git a/test/sqllogictest/github-11139.slt b/test/sqllogictest/github-11139.slt index 5370dfecfdcd9..81bbaacb2305b 100644 --- a/test/sqllogictest/github-11139.slt +++ b/test/sqllogictest/github-11139.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/11139 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/3256 mode cockroach diff --git a/test/sqllogictest/github-11568.slt b/test/sqllogictest/github-11568.slt index 4923afdaa25eb..c147f7d9b4aa1 100644 --- a/test/sqllogictest/github-11568.slt +++ b/test/sqllogictest/github-11568.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/11568. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/3374. mode cockroach diff --git a/test/sqllogictest/github-13857.slt b/test/sqllogictest/github-13857.slt index addd3ad9f3b8d..9375161fc5bc7 100644 --- a/test/sqllogictest/github-13857.slt +++ b/test/sqllogictest/github-13857.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/13857 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/3963 simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_rbac_checks TO false; diff --git a/test/sqllogictest/github-16036.slt b/test/sqllogictest/github-16036.slt index 3109ecff25b0b..c983ec4d301de 100644 --- a/test/sqllogictest/github-16036.slt +++ b/test/sqllogictest/github-16036.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/16036 +# Regression test for https://api.github.com/repos/MaterializeInc/materialize/pulls/16036 mode cockroach statement ok diff --git a/test/sqllogictest/github-17616.slt b/test/sqllogictest/github-17616.slt index 49fe79717326f..0d3e4f1dfd85b 100644 --- a/test/sqllogictest/github-17616.slt +++ b/test/sqllogictest/github-17616.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/17616 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5126 mode cockroach # reset diff --git a/test/sqllogictest/github-17762.slt b/test/sqllogictest/github-17762.slt index 059921ed41a98..6e4181e6f2f9d 100644 --- a/test/sqllogictest/github-17762.slt +++ b/test/sqllogictest/github-17762.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/17762 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5174 mode cockroach query T diff --git a/test/sqllogictest/github-17808.slt b/test/sqllogictest/github-17808.slt index c3b2abe86620d..2ecd67a066b9f 100644 --- a/test/sqllogictest/github-17808.slt +++ b/test/sqllogictest/github-17808.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for issues corrected by https://github.com/MaterializeInc/materialize/issues/17808. +# Regression test for issues corrected by https://api.github.com/repos/MaterializeInc/materialize/issues/17808. # The following query should not frustrate the system, although it did prior to # the associated PR being merged. It should produce an empty result set. diff --git a/test/sqllogictest/github-18522.slt b/test/sqllogictest/github-18522.slt index 55c6db39907c2..bd863d926d767 100644 --- a/test/sqllogictest/github-18522.slt +++ b/test/sqllogictest/github-18522.slt @@ -7,9 +7,9 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for issues corrected by https://github.com/MaterializeInc/materialize/issues/18522. +# Regression test for issues corrected by https://api.github.com/repos/MaterializeInc/database-issues/issues/5474. -# Regression test for materialize#18522. +# Regression test for database-issues#5474. statement ok CREATE TABLE t0(c0 VARCHAR(203) ); diff --git a/test/sqllogictest/github-18708.slt b/test/sqllogictest/github-18708.slt index 735180daca7a1..46f9a5b6b8942 100644 --- a/test/sqllogictest/github-18708.slt +++ b/test/sqllogictest/github-18708.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#18708. +# Regression test for database-issues#5536. statement ok CREATE TABLE t0(c0 FLOAT DEFAULT (-1.77794611E9)); diff --git a/test/sqllogictest/github-19273.slt b/test/sqllogictest/github-19273.slt index a0569a27525a5..1ec44e3363dff 100644 --- a/test/sqllogictest/github-19273.slt +++ b/test/sqllogictest/github-19273.slt @@ -7,9 +7,9 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#19273 and materialize#19283. +# Regression test for database-issues#5717 and database-issues#5722. -# The query from materialize#19273. +# The query from database-issues#5717. statement ok CREATE SOURCE tpch @@ -40,7 +40,7 @@ from where subq_0."c5" > subq_0."c5" limit 21; -# The query from materialize#19283. +# The query from database-issues#5722. statement ok CREATE TABLE t (a int, b int); diff --git a/test/sqllogictest/github-19290.slt b/test/sqllogictest/github-19290.slt index cd447721491d7..e370b46b4324f 100644 --- a/test/sqllogictest/github-19290.slt +++ b/test/sqllogictest/github-19290.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#19290. +# Regression test for database-issues#5723. statement ok CREATE SOURCE tpch diff --git a/test/sqllogictest/github-19511.slt b/test/sqllogictest/github-19511.slt index bfb0373664bb7..ca696c841295d 100644 --- a/test/sqllogictest/github-19511.slt +++ b/test/sqllogictest/github-19511.slt @@ -10,7 +10,7 @@ mode cockroach # The following is a regression test for cases 3), 4), and 6) -# of https://github.com/MaterializeInc/materialize/issues/19511, +# of https://api.github.com/repos/MaterializeInc/database-issues/issues/5797, # where aggregate functions would type-promote to floating point, # but could promote to numeric. @@ -31,7 +31,7 @@ CREATE OR REPLACE VIEW v_19511_case_3 (row_index, decimal_39_8_val) # Note that we get an error in the evaluation of this case, while # PostgreSQL successfully computes a result. This is due to the # precision of our numeric types vs. the algorithm used to calculate -# standard deviation (https://github.com/MaterializeInc/materialize/issues/1240). +# standard deviation (https://api.github.com/repos/MaterializeInc/database-issues/issues/436). query error SELECT stddev_samp(decimal_39_8_val) diff --git a/test/sqllogictest/github-21244.slt b/test/sqllogictest/github-21244.slt index 5e3d5b6db2862..2e958b3d74ed3 100644 --- a/test/sqllogictest/github-21244.slt +++ b/test/sqllogictest/github-21244.slt @@ -10,7 +10,7 @@ mode cockroach # Regression test for -# https://github.com/MaterializeInc/materialize/issues/21244. +# https://api.github.com/repos/MaterializeInc/database-issues/issues/6388. statement ok CREATE TABLE r(x bool); diff --git a/test/sqllogictest/github-21501.slt b/test/sqllogictest/github-21501.slt index 15ae700a6c775..dfbd2260c2667 100644 --- a/test/sqllogictest/github-21501.slt +++ b/test/sqllogictest/github-21501.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#21501. +# Regression test for database-issues#6464. # Note that this test employs the old syntax with the EXPECTED GROUP SIZE because the behavior # tested is exactly the backwards-compatible one of assigning the hinted value to all operators # in the same query block. diff --git a/test/sqllogictest/github-23015.slt b/test/sqllogictest/github-23015.slt index 5fe354fc39749..8b2c0ee954c2b 100644 --- a/test/sqllogictest/github-23015.slt +++ b/test/sqllogictest/github-23015.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#23015. +# Regression test for database-issues#6935. query TT select distinct 'postgres', current_user() from mz_internal.mz_cluster_replica_metrics; diff --git a/test/sqllogictest/github-24511.slt b/test/sqllogictest/github-24511.slt index 79fbb623b4e9d..4dd863da5b350 100644 --- a/test/sqllogictest/github-24511.slt +++ b/test/sqllogictest/github-24511.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/24511 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/7318 # This query involves a 1-input join. `JoinImplementation` or other join code shouldn't choke on this. diff --git a/test/sqllogictest/github-25015.slt b/test/sqllogictest/github-25015.slt index 673d2b3a66bf5..2662848148c0f 100644 --- a/test/sqllogictest/github-25015.slt +++ b/test/sqllogictest/github-25015.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#25015. +# Regression test for database-issues#7467. # The schema is a simplified version of # https://github.com/MaterializeInc/RQG/blob/main/conf/mz/simple.sql @@ -24,7 +24,7 @@ CREATE TABLE t2 (f1 INT, f2 INT NOT NULL); statement ok INSERT INTO t2 VALUES (NULL, 0); -# Prior to the bugfix for materialize#25015 this query was (wrongly) returning a single +# Prior to the bugfix for database-issues#7467 this query was (wrongly) returning a single # row. query II SELECT DISTINCT diff --git a/test/sqllogictest/github-25769.slt b/test/sqllogictest/github-25769.slt index d4ed4827024b0..222fd815c3118 100644 --- a/test/sqllogictest/github-25769.slt +++ b/test/sqllogictest/github-25769.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#25769. +# Regression test for database-issues#7688. simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_cardinality_estimates = true diff --git a/test/sqllogictest/github-26352.slt b/test/sqllogictest/github-26352.slt index 2d86e75f3b76f..fc8dddc1f41cd 100644 --- a/test/sqllogictest/github-26352.slt +++ b/test/sqllogictest/github-26352.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#26352. +# Regression test for database-issues#7821. simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_variadic_left_join_lowering TO true; diff --git a/test/sqllogictest/github-26707.slt b/test/sqllogictest/github-26707.slt index 2cc784c399f67..819448699b63c 100644 --- a/test/sqllogictest/github-26707.slt +++ b/test/sqllogictest/github-26707.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#26707. +# Regression test for database-issues#7892. simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_variadic_left_join_lowering TO true; diff --git a/test/sqllogictest/github-28166.slt b/test/sqllogictest/github-28166.slt index fbcbfb303c4d8..5d98f56c0dc70 100644 --- a/test/sqllogictest/github-28166.slt +++ b/test/sqllogictest/github-28166.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#28166. +# Regression test for database-issues#8257. statement ok CREATE CLUSTER test REPLICAS( diff --git a/test/sqllogictest/github-28174-29110.slt b/test/sqllogictest/github-28174-29110.slt index 108a12d565e25..e4f2dbc68eaf6 100644 --- a/test/sqllogictest/github-28174-29110.slt +++ b/test/sqllogictest/github-28174-29110.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for materialize#28174 and materialize#29110. +# Regression test for database-issues#8261 and database-issues#8463. # The setup is based on https://github.com/MaterializeInc/RQG/blob/main/conf/mz/simple.sql @@ -127,7 +127,7 @@ CREATE MATERIALIZED VIEW pk2 AS SELECT DISTINCT ON (f1) f1 , f2 FROM t2 WHERE f1 statement ok CREATE MATERIALIZED VIEW pk3 AS SELECT DISTINCT ON (f1) f1 , f2 FROM t3 WHERE f1 IS NOT NULL AND f2 IS NOT NULL; -# materialize#28174 +# database-issues#8261 query RRRIR SELECT (a1.f1) AS c1, (a2.f1) AS c2, (a1.f2) AS c3, (count(a1.f2)) AS agg1, (max(a1.f1 + a1.f1)) AS agg2 FROM @@ -280,7 +280,7 @@ NULL 0 NULL -# materialize#29110 +# database-issues#8463 query RR SELECT a1.f1 AS c3, diff --git a/test/sqllogictest/github-7168.slt b/test/sqllogictest/github-7168.slt index f66561547150f..6815d58c05473 100644 --- a/test/sqllogictest/github-7168.slt +++ b/test/sqllogictest/github-7168.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/7168. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2235. # # The shape of this table would reliably trigger memory unsafety. The datums # were exactly the right length to corrupt the stack given the bug. It's diff --git a/test/sqllogictest/github-7472.slt b/test/sqllogictest/github-7472.slt index f2b554079b21e..c40efb5c05541 100644 --- a/test/sqllogictest/github-7472.slt +++ b/test/sqllogictest/github-7472.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/7472. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2327. statement ok CREATE TABLE t2 (f1 DOUBLE PRECISION, f2 DOUBLE PRECISION NOT NULL); diff --git a/test/sqllogictest/github-8241.slt b/test/sqllogictest/github-8241.slt index b4055c8ad4a2d..78675b7f7e5d3 100644 --- a/test/sqllogictest/github-8241.slt +++ b/test/sqllogictest/github-8241.slt @@ -7,8 +7,8 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/8241 -# See https://github.com/MaterializeInc/materialize/issues/8318 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2514 +# See https://api.github.com/repos/MaterializeInc/database-issues/issues/2535 # Verify that an index added after a transaction has started complains. diff --git a/test/sqllogictest/github-8713.slt b/test/sqllogictest/github-8713.slt index 07416759b904f..6a3d7c25898bc 100644 --- a/test/sqllogictest/github-8713.slt +++ b/test/sqllogictest/github-8713.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/8713 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2666 mode cockroach statement ok diff --git a/test/sqllogictest/github-8717.slt b/test/sqllogictest/github-8717.slt index e420ba88e8525..70f393a9b644b 100644 --- a/test/sqllogictest/github-8717.slt +++ b/test/sqllogictest/github-8717.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/8717 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2668 statement ok CREATE VIEW v1 (f1) AS (VALUES (1), (1)); diff --git a/test/sqllogictest/github-9027.slt b/test/sqllogictest/github-9027.slt index f1460a1a4e666..638ecc7663514 100644 --- a/test/sqllogictest/github-9027.slt +++ b/test/sqllogictest/github-9027.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/9027 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2746 mode cockroach statement ok diff --git a/test/sqllogictest/github-9147.slt b/test/sqllogictest/github-9147.slt index 5bdaef3543218..ac28e481d58db 100644 --- a/test/sqllogictest/github-9147.slt +++ b/test/sqllogictest/github-9147.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/9147 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2799 mode cockroach statement ok diff --git a/test/sqllogictest/github-9504.slt b/test/sqllogictest/github-9504.slt index bae070ba9c5c7..382c45f2c6945 100644 --- a/test/sqllogictest/github-9504.slt +++ b/test/sqllogictest/github-9504.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/9504 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2906 statement ok CREATE TABLE tableA(happened_at TIMESTAMP, json_col JSON, offs NUMERIC, id INTEGER); diff --git a/test/sqllogictest/github-9782.slt b/test/sqllogictest/github-9782.slt index 96d15f837eb35..73bf620146e19 100644 --- a/test/sqllogictest/github-9782.slt +++ b/test/sqllogictest/github-9782.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/9782. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2969. statement ok CREATE TABLE table_f1 (f1 INTEGER); diff --git a/test/sqllogictest/github-9931.slt b/test/sqllogictest/github-9931.slt index 2afff097c4b64..d4a3ae285935c 100644 --- a/test/sqllogictest/github-9931.slt +++ b/test/sqllogictest/github-9931.slt @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/9931. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2996. mode cockroach diff --git a/test/sqllogictest/interval.slt b/test/sqllogictest/interval.slt index b3c732e56c87e..c79a72808f516 100644 --- a/test/sqllogictest/interval.slt +++ b/test/sqllogictest/interval.slt @@ -1340,7 +1340,7 @@ SELECT EXTRACT(epoch FROM INTERVAL '-2147483648 months -2147483648 days -2147483 -5840726025531599.999999 # Interval comparison is done on a field by field basis, without combining the fields. This is due to a limitation in -# Materialize (See: https://github.com/MaterializeInc/materialize/issues/10566) +# Materialize (See: https://api.github.com/repos/MaterializeInc/database-issues/issues/3130) query T SELECT INTERVAL '1 month' > INTERVAL '9999999 days'; diff --git a/test/sqllogictest/joins.slt b/test/sqllogictest/joins.slt index e665086f25bbf..f301ac607baae 100644 --- a/test/sqllogictest/joins.slt +++ b/test/sqllogictest/joins.slt @@ -34,7 +34,7 @@ SELECT * FROM l LEFT JOIN r ON l.la = r.ra 3 l3 3 r3 # This test may look the same as the last, but listing out the columns -# explicitly checks for regressions of materialize#1314. +# explicitly checks for regressions of database-issues#472. query ITIT rowsort SELECT l.la, l.lb, r.ra, r.rb FROM l LEFT JOIN r ON l.la = r.ra ---- @@ -50,7 +50,7 @@ NULL NULL 4 r4 3 l3 3 r3 # This test may look the same as the last, but listing out the columns -# explicitly checks for regressions of materialize#1314. +# explicitly checks for regressions of database-issues#472. query ITIT rowsort SELECT l.la, l.lb, r.ra, r.rb FROM l RIGHT JOIN r ON l.la = r.ra ---- @@ -59,21 +59,21 @@ NULL NULL 4 r4 3 l3 3 r3 # Test that columns detected to be equivalent retain the names that the user -# asks for. Protects against regression of materialize#1217. +# asks for. Protects against regression of database-issues#429. query II colnames,rowsort SELECT ra, r.ra FROM l JOIN r ON l.la = r.ra LIMIT 0 ---- ra ra # Test that columns detected to be equivalent retain the names that the user -# asks for. Protects against regression of materialize#1217. +# asks for. Protects against regression of database-issues#429. query ITIT colnames,rowsort SELECT * FROM l JOIN r ON l.la = r.ra LIMIT 0 ---- la lb ra rb # Test that columns detected to be equivalent do not interfere with qualified -# wildcard expansion. Protects against regression of materialize#4477. +# wildcard expansion. Protects against regression of database-issues#1389. query ITIT colnames SELECT l.*, r.* FROM l JOIN r ON l.la = r.ra LIMIT 0 ---- @@ -212,7 +212,7 @@ Target cluster: quickstart EOF # Test that multiplicities in outer joins are preserved. -# Fixed in materialize#2724. +# Fixed in database-issues#931. statement ok CREATE TABLE l2 (la int, lb text) @@ -242,7 +242,7 @@ NULL NULL 4 r4 3 l3 3 r3 -# Test that outer joins work correctly in correlated subqueries (materialize#3047) +# Test that outer joins work correctly in correlated subqueries (database-issues#1007) query IT rowsort SELECT * FROM l WHERE EXISTS (SELECT * from l as l2 LEFT JOIN r ON l.la = r.ra) @@ -265,7 +265,7 @@ SELECT * FROM r WHERE EXISTS (SELECT * from l RIGHT JOIN r as r2 ON l.la = r.ra) 1 r1 3 r3 -# Regression test for materialize#3426. +# Regression test for database-issues#1089. query III colnames SELECT * FROM (SELECT 1 AS baz) t1 @@ -304,7 +304,7 @@ SELECT * FROM 1 NULL NULL 1 NULL NULL -# Regression test for materialize#3914. The gist is that we want to exercise the +# Regression test for database-issues#1218. The gist is that we want to exercise the # redundant join optimization on a join whose inputs contain projections. This # turns out to be extremely hard to accomplish because the projection lifting # optimization is very good at eliminating all but the top-level projection. @@ -484,7 +484,7 @@ SELECT * FROM l INNER JOIN r ON mod(l.la, 2) = mod(r.ra, 2) 3 l3 3 r3 # Test that when both keys are expressions, the join is not planned as a cross -# join. Protects against regression of materialize#4170. +# join. Protects against regression of database-issues#1290. query T multiline EXPLAIN WITH(arity, join implementations) SELECT * FROM l INNER JOIN r ON mod(l.la, 2) = mod(r.ra, 2) ---- @@ -508,7 +508,7 @@ Target cluster: quickstart EOF -# Regression test for materialize#4361, in which inter-view predicate pushdown would cause +# Regression test for database-issues#1355, in which inter-view predicate pushdown would cause # misoptimizations. Specifically, a predicate that was applied to only one use # of a view could be incorrectly applied to all uses of that view. # @@ -705,7 +705,7 @@ l1 r1 l3 r3 l4 r5 -# Regression test for https://github.com/MaterializeInc/materialize/issues/7618 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2364 statement ok CREATE TABLE t1 (f1 int, f2 int); @@ -1144,7 +1144,7 @@ SELECT * FROM l NATURAL JOIN l AS a LIMIT 0 ---- la lb -# Regression test for https://github.com/MaterializeInc/materialize/issues/17897 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5220 # This query fails if JoinImplementation is not in the same fixpoint loop with LiteralLifting. statement ok @@ -1270,7 +1270,7 @@ INSERT INTO error_test VALUES (0) query error division by zero SELECT t1.a / t2.a FROM error_test t1, error_test t2 -# Regression test for https://github.com/MaterializeInc/materialize/issues/20060 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5998 query select from diff --git a/test/sqllogictest/jsonb.slt b/test/sqllogictest/jsonb.slt index 5a430ec531fbf..3c504f0dc77bc 100644 --- a/test/sqllogictest/jsonb.slt +++ b/test/sqllogictest/jsonb.slt @@ -31,7 +31,7 @@ SELECT '{"a": 1, "a": 2, "a": 3}'::jsonb ---- {"a":3} -# Regression test for materialize#5919, in which 64-bit integers could not be represented +# Regression test for database-issues#1824, in which 64-bit integers could not be represented # in the jsonb type. query T SELECT '1614259308542846100'::jsonb @@ -932,7 +932,7 @@ SELECT '[1,"1",1.0]'::JSONB - '1' # ---- # {"b":[],"c":{"a":"b"}} -# # Regression test for materialize#34756. +# # Regression test for cockroach#34756. # query T # SELECT '{"b":[],"c":{"a":"b"}}'::JSONB - array['foo',NULL] # ---- @@ -1480,7 +1480,7 @@ SELECT jsonb_build_array('\x0001'::BYTEA) # ---- # [1,1,1.2,null,["x","y"]] -# # Regression for materialize#37318 +# # Regression for cockroach#37318 # query T # SELECT jsonb_build_array('+Inf'::FLOAT,'-Inf'::FLOAT,'NaN'::FLOAT)::STRING::JSONB # ---- @@ -2109,7 +2109,7 @@ SELECT jsonb_object_agg(a, a) FILTER (WHERE a IS NOT NULL) FROM t1 ---- {"1":1,"2":2,"3":3} -# Null casts. Protects against materialize#7183. +# Null casts. Protects against database-issues#2244. query TTTTTTT SELECT NULL::jsonb::text, diff --git a/test/sqllogictest/like.slt b/test/sqllogictest/like.slt index d5d0a5a5c1b41..923d031f02766 100644 --- a/test/sqllogictest/like.slt +++ b/test/sqllogictest/like.slt @@ -143,7 +143,7 @@ SELECT s FROM t WHERE s ILIKE like_pat; ABC abc -# Regression test for https://github.com/MaterializeInc/materialize/issues/26177 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/7790 statement ok CREATE TABLE t2 (text_val TEXT); diff --git a/test/sqllogictest/managed_cluster.slt b/test/sqllogictest/managed_cluster.slt index 7506c1a076aa8..972981f93d91c 100644 --- a/test/sqllogictest/managed_cluster.slt +++ b/test/sqllogictest/managed_cluster.slt @@ -411,7 +411,7 @@ DROP CLUSTER foo CASCADE statement error db error: ERROR: unknown cluster replica size invalid_size CREATE CLUSTER foo SIZE invalid_size, REPLICATION FACTOR 0 -# Test for materialize#20195 . Without the fix, this query will hang indefinitely +# Test for database-issues#6046 . Without the fix, this query will hang indefinitely statement error db error: ERROR: creating cluster replica would violate max_replicas_per_cluster limit \(desired: 9999999, limit: 5, current: 0\) CREATE CLUSTER foo SIZE '1', replication factor 9999999; diff --git a/test/sqllogictest/materialized_views.slt b/test/sqllogictest/materialized_views.slt index 5032479f11a8e..5f6f87cad2454 100644 --- a/test/sqllogictest/materialized_views.slt +++ b/test/sqllogictest/materialized_views.slt @@ -226,7 +226,7 @@ SELECT * FROM v # Test: a view on a materialized view that optimizes to the empty set # still prevents the underlying view from being dropped. -# See: https://github.com/MaterializeInc/materialize/issues/20315 +# See: https://api.github.com/repos/MaterializeInc/database-issues/issues/6101 statement ok CREATE VIEW v AS SELECT 1 AS c @@ -1005,7 +1005,7 @@ NULL -1 -2 30 30 30 100 100 100 -# Regression test for https://github.com/MaterializeInc/materialize/issues/24288 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/7265 # The sleep makes _optimization_ take a few seconds, so we need to grab read holds in purification right away after # choosing a timestamp for mz_now. diff --git a/test/sqllogictest/mzcompose.py b/test/sqllogictest/mzcompose.py index 4c32efbc2fc4d..f5666a9596b9e 100644 --- a/test/sqllogictest/mzcompose.py +++ b/test/sqllogictest/mzcompose.py @@ -581,7 +581,7 @@ def compileSlowSltConfig() -> SltRunConfig: } tests_without_views_and_replica = { # errors: - # https://github.com/MaterializeInc/materialize/issues/20534 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/6181 "test/sqllogictest/list.slt", # transactions: "test/sqllogictest/github-11568.slt", @@ -599,9 +599,9 @@ def compileSlowSltConfig() -> SltRunConfig: "test/sqllogictest/cluster.slt", # different indexes auto-created "test/sqllogictest/object_ownership.slt", - # https://github.com/MaterializeInc/materialize/issues/20110 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/6011 "test/sqllogictest/interval.slt", - # https://github.com/MaterializeInc/materialize/issues/20110 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/6011 "test/sqllogictest/operator.slt", # specific replica size tested: "test/sqllogictest/managed_cluster.slt", diff --git a/test/sqllogictest/numeric.slt b/test/sqllogictest/numeric.slt index 0444af97d6cae..b512a628d20e2 100644 --- a/test/sqllogictest/numeric.slt +++ b/test/sqllogictest/numeric.slt @@ -857,7 +857,7 @@ SELECT (999999999999999999999999999999999999.123 + 1::numeric)::numeric(39,3); # Successive operations -## regression test for materialize#6858 +## regression test for database-issues#2137 query R SELECT -0.0 - 1::numeric * '-0.0 '::numeric; ---- @@ -2178,7 +2178,7 @@ select min(a) from (select a from unnest(ARRAY[]::numeric[]) a); NULL # Distinct, except -# regression tests for materialize#6877 materialize#6912 +# regression tests for database-issues#2145 database-issues#2152 query R SELECT DISTINCT column1 FROM ( @@ -2203,7 +2203,7 @@ SELECT column1 FROM ( ) ---- -# regression test for https://github.com/MaterializeInc/materialize/issues/7361 +# regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2298 statement ok CREATE TABLE A (f1 DECIMAL); diff --git a/test/sqllogictest/order_by.slt b/test/sqllogictest/order_by.slt index ce9565d790657..34e83ecbc77c4 100644 --- a/test/sqllogictest/order_by.slt +++ b/test/sqllogictest/order_by.slt @@ -685,7 +685,7 @@ SELECT * FROM bazv statement ok CREATE VIEW fizzorderview AS SELECT a, b FROM fizz ORDER BY a DESC, b -# TODO: materialize#724 take out the rowsort and rearrange results +# TODO: database-issues#236 take out the rowsort and rearrange results # when ORDER BY's persist past the view creation query IT rowsort SELECT * FROM fizzorderview @@ -868,7 +868,7 @@ SELECT * FROM fizzlimitoffsetview2 21758 fourteen 12345 three -# NULL should sort last by default. See materialize#5982. +# NULL should sort last by default. See database-issues#1844. # NULL (default, default) query I SELECT * FROM (VALUES (NULL), (1)) ORDER BY column1 @@ -994,7 +994,7 @@ SELECT row_number() OVER (ORDER BY x NULLS LAST), x FROM t; 4 NULL ## TopK removal when it's completely covered by the finishing. -## See https://github.com/MaterializeInc/materialize/issues/8194 +## See https://api.github.com/repos/MaterializeInc/database-issues/issues/2498 statement ok DROP TABLE t; diff --git a/test/sqllogictest/outer_join_simplification.slt b/test/sqllogictest/outer_join_simplification.slt index 5aead93a7d67d..1c72c693e532d 100644 --- a/test/sqllogictest/outer_join_simplification.slt +++ b/test/sqllogictest/outer_join_simplification.slt @@ -397,7 +397,7 @@ EOF # Here, SemijoinIdempotence relies on the keys introduced by the `select distinct on` being propagated through the # recursive Get. Note that in this test even if SemijoinIdempotence wouldn't work, RedundantJoin would step in. (But # see later a similar situation, but with a LEFT JOIN, where RedundantJoin wouldn't be able to eliminate a join.) -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline explain with(keys) with mutually recursive @@ -426,7 +426,7 @@ EOF # Similar to the previous test, but the recursive Get is at the other input. This means that the input that should have # a known key is the static one, for which key inference works fine. # The resulting plan should have 1 join with 2 inputs. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline explain with mutually recursive @@ -452,7 +452,7 @@ EOF # Manually written idempotent semijoin. # Similar to the previous test, but the CTE from inside c0 is manually lifted to the enclosing LetRec. # The resulting plan should have 1 join with 2 inputs. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline explain with mutually recursive @@ -477,7 +477,7 @@ EOF # Similar to the previous test, but instead of using bar directly, we add an extra cte (bar2), which will recursively # refer to `c0`. `keys` will refer to `bar2` instead of `c0`. # The resulting plan should have 1 join with 2 inputs. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline explain with mutually recursive @@ -508,7 +508,7 @@ EOF # changes meaning when `bar2` is being assigned after `key`. To correctly handle this situation, we need the expirations # in SemijoinIdempotence. To demonstrate this, if we comment out the two `do_expirations` lines, SemijoinIdempotence # incorrectly transforms this plan. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline explain with(keys) with mutually recursive @@ -538,7 +538,7 @@ EOF # Another negative test. Similar to the first test with `bar2`, but both `keys` and `bar2` are being referenced from the # body, which means that SemijoinIdempotence can't kick in, as it would eliminate the intermediate Join's result, which # is now being referenced from the body. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline explain with mutually recursive diff --git a/test/sqllogictest/postgres-incompatibility.slt b/test/sqllogictest/postgres-incompatibility.slt index 580ddd69b89a6..0bb720a811c1c 100644 --- a/test/sqllogictest/postgres-incompatibility.slt +++ b/test/sqllogictest/postgres-incompatibility.slt @@ -58,7 +58,7 @@ SELECT 17 / - COALESCE ( - + 60, - + AVG ( DISTINCT 54 ) * COUNT ( * ), + 0 ) * -1256.75 0 # these return null in postgres -# https://github.com/MaterializeInc/materialize/issues/2418 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/845 query error division by zero SELECT ALL 56 * 97 - SUM ( - 51 ) + NULLIF ( + - 36, + + CAST ( NULL AS INTEGER ) ) * + - 91 / - 0 * - 45 * - 10 * - CAST ( NULL AS INTEGER ) * - - 85 diff --git a/test/sqllogictest/postgres/subselect.slt b/test/sqllogictest/postgres/subselect.slt index 9fc4114cc3c01..4a24234c51ab4 100644 --- a/test/sqllogictest/postgres/subselect.slt +++ b/test/sqllogictest/postgres/subselect.slt @@ -519,7 +519,7 @@ min 0 # -- -# -- Test case for bug materialize#4290: bogus calculation of subplan param sets +# -- Test case for bug database-issues#1339: bogus calculation of subplan param sets # -- # create temp table ta (id int primary key, val int); # insert into ta values(1,1); @@ -724,7 +724,7 @@ select exists(select * from nocolumns) exists false -# Check behavior with a SubPlan in VALUES (bug materialize#14924) +# Check behavior with a SubPlan in VALUES (bug database-issues#4272) query I rowsort select val.x from generate_series(1,10) as s(i), diff --git a/test/sqllogictest/regclass.slt b/test/sqllogictest/regclass.slt index f1498b141dd98..84a67095de2c9 100644 --- a/test/sqllogictest/regclass.slt +++ b/test/sqllogictest/regclass.slt @@ -241,7 +241,7 @@ SELECT 'mz_tables'::regclass::oid = (SELECT oid FROM mz_objects WHERE name = 'mz true # This shouldn't be an error but seems to be impacted by -# some evaluation order issue akin to materialize#17189 +# some evaluation order issue akin to database-issues#4972 # # Regression for 9194 # query B # select 'mz_tables'::regclass::oid::text::regclass = (SELECT oid FROM mz_objects WHERE name = 'mz_tables') diff --git a/test/sqllogictest/regex.slt b/test/sqllogictest/regex.slt index 844c8552e92a9..1a46579a593e8 100644 --- a/test/sqllogictest/regex.slt +++ b/test/sqllogictest/regex.slt @@ -22,7 +22,7 @@ asdf asdf NULL asdfjkl asdf NULL jkl NULL jkl -# TODO - Test that the columns have the correct nullability, once they actually do (materialize#1685) +# TODO - Test that the columns have the correct nullability, once they actually do (database-issues#612) # Standard regex matching. query TTT @@ -486,7 +486,7 @@ ORDER BY 1 {hello,world} {hello,world} -# TODO: Correct ordering when materialize#16464 is fixed. +# TODO: Correct ordering when database-issues#4764 is fixed. query T SELECT foo FROM regexp_split_to_table('the quick brown fox jumps over the lazy dog', '\s+') AS foo ORDER BY 1 ---- @@ -500,7 +500,7 @@ quick the the -# TODO: Correct ordering when materialize#16464 is fixed. +# TODO: Correct ordering when database-issues#4764 is fixed. query T SELECT foo FROM regexp_split_to_table('the quick brown fox', '\s*') AS foo ORDER BY 1 ---- diff --git a/test/sqllogictest/regressions.slt b/test/sqllogictest/regressions.slt index e50b5b543b799..d0ed8f41d2cd5 100644 --- a/test/sqllogictest/regressions.slt +++ b/test/sqllogictest/regressions.slt @@ -21,6 +21,6 @@ INSERT INTO t (a, b, c) VALUES (0, 'test', 1) statement ok INSERT INTO t (a, b, c) VALUES (3, 'test', 1) -# Regression test for materialize#5003 +# Regression test for database-issues#1555 statement error Evaluation error: division by zero SELECT recip - 1 FROM (SELECT b, 1/a as recip, max(c) from t GROUP BY b, 1/a) diff --git a/test/sqllogictest/regtype.slt b/test/sqllogictest/regtype.slt index 5f9a0a4594753..f8d336c220c55 100644 --- a/test/sqllogictest/regtype.slt +++ b/test/sqllogictest/regtype.slt @@ -88,7 +88,7 @@ NULL # Regression for materialize issue 9194 # This shouldn't be an error but seems to be impacted by -# some evaluation order issue akin to materialize#17189 +# some evaluation order issue akin to database-issues#4972 # TODO: this has an optimization bug. query error select 'date'::regtype::oid::text::regtype diff --git a/test/sqllogictest/scoping.slt b/test/sqllogictest/scoping.slt index fa6a78be2bdce..b2082aee5d552 100644 --- a/test/sqllogictest/scoping.slt +++ b/test/sqllogictest/scoping.slt @@ -30,7 +30,7 @@ query error column "t1.a" must appear in the GROUP BY clause or be used in an ag SELECT t1.a FROM t1 JOIN (t2 JOIN t3 ON t2.a = t3.a) ON t1.a = t2.a GROUP BY t3.a # Ensure that reflexive equality expressions do not cause the specified column -# to become unnameable. See materialize#5765. +# to become unnameable. See database-issues#1778. query I SELECT t1.a FROM t1 JOIN t2 ON t1.a = t1.a GROUP BY t1.a ---- @@ -45,7 +45,7 @@ query I SELECT t2.a FROM t1 NATURAL JOIN t2 ---- -# Regression tests for materialize#16920. +# Regression tests for database-issues#4887. query I SELECT t.a FROM (t1 NATURAL JOIN t2) t ---- diff --git a/test/sqllogictest/string.slt b/test/sqllogictest/string.slt index 42d92a507bc5a..d9743cfdb7618 100644 --- a/test/sqllogictest/string.slt +++ b/test/sqllogictest/string.slt @@ -1521,7 +1521,7 @@ SELECT CONCAT('a'::text, 'b'::char(3), 'c'); ---- ab c -# Regression for https://github.com/MaterializeInc/materialize/issues/18095 +# Regression for https://api.github.com/repos/MaterializeInc/database-issues/issues/5304 statement ok CREATE TABLE gh18095_0(c0 VARCHAR(55)) diff --git a/test/sqllogictest/subquery.slt b/test/sqllogictest/subquery.slt index f731d3f5c6a3f..4fd56233e297d 100644 --- a/test/sqllogictest/subquery.slt +++ b/test/sqllogictest/subquery.slt @@ -322,7 +322,7 @@ Target cluster: quickstart EOF -# Regression test for materialize#1158 +# Regression test for database-issues#396 # The following subquery currently generates a plan with a map with # 4 scalars that refer to other scalars in the map. If query planning optimizes away # this particular case, replace with another query that generates such a plan @@ -377,7 +377,7 @@ SELECT age, ascii_num * 2 as result FROM ( 103 210 103 236 -# regression test for https://github.com/MaterializeInc/materialize/issues/1708 +# regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/621 statement ok CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col3 INTEGER) @@ -635,14 +635,14 @@ from ys 1 -1 2 0.5 -# Regression test for materialize#3924, in which quantified comparision simplification was +# Regression test for database-issues#1221, in which quantified comparision simplification was # not correctly handling LATERAL joins. query II SELECT * FROM (VALUES (1)), LATERAL (SELECT * FROM (SELECT column1) WHERE true) ---- 1 1 -# Regression test for materialize#3933, in which the query planner mishandled the outer +# Regression test for database-issues#1222, in which the query planner mishandled the outer # scope of a nested LATERAL join. query IIII SELECT * FROM @@ -651,7 +651,7 @@ SELECT * FROM ---- 1 1 1 1 -# Regression test for materialize#4157, in which quantified expression simplification +# Regression test for database-issues#1287, in which quantified expression simplification # failed to handle map expressions which depended upon a column introduced by # an earlier expression in the same map node. query I @@ -663,7 +663,7 @@ FROM (SELECT 1 col1) query error aggregate functions that refer exclusively to outer columns not yet supported SELECT (SELECT count(likes.likee)) FROM likes -# Regression test for materialize#7121, where the multiplicity of the outer query was not +# Regression test for database-issues#2209, where the multiplicity of the outer query was not # preserved when an EXISTS subquery only involves constants/mpf/flatmaps. query I SELECT x FROM xs WHERE EXISTS (SELECT y FROM (SELECT 1 as y) WHERE x = y) @@ -671,7 +671,7 @@ SELECT x FROM xs WHERE EXISTS (SELECT y FROM (SELECT 1 as y) WHERE x = y) 1 1 -# Regression test for materialize#9921, in which the presence of a SELECT subquery inside +# Regression test for database-issues#2994, in which the presence of a SELECT subquery inside # of another subexpression produced a degenerate join that was not elided before # join implementation planning. query I diff --git a/test/sqllogictest/table_func.slt b/test/sqllogictest/table_func.slt index 3a1a40ad5c0d4..fea7c755d0662 100644 --- a/test/sqllogictest/table_func.slt +++ b/test/sqllogictest/table_func.slt @@ -175,7 +175,7 @@ SELECT * FROM x x1, x x2, generate_series(x1.a, x2.a) WHERE x1.b = x2.b 2 3 2 3 2 3 4 3 4 3 -# Regression test for materialize#3877: a table function as the first FROM item inside of +# Regression test for database-issues#1206: a table function as the first FROM item inside of # a LATERAL subquery should not miscount outer scope depth. query II SELECT x.a, generate_series FROM x, LATERAL (SELECT * FROM generate_series(1, x.a)) @@ -187,7 +187,7 @@ SELECT x.a, generate_series FROM x, LATERAL (SELECT * FROM generate_series(1, x. 3 2 3 3 -# Regression test for materialize#5520: crash when a filter references an output column of +# Regression test for database-issues#1700: crash when a filter references an output column of # a table function query IIIII SELECT * FROM x x1, x x2, generate_series(x1.a, x2.b) AS x3(b) WHERE x1.b = x2.b AND x1.a = x3.b @@ -311,7 +311,7 @@ Target cluster: quickstart EOF -# Regression test for materialize#5520: crash when a filter references an output column of +# Regression test for database-issues#1700: crash when a filter references an output column of # a table function around a join query T multiline EXPLAIN WITH(arity, join implementations) SELECT * FROM x x1, x x2, generate_series(x1.a, x2.b) AS x3(b) WHERE x1.b = x2.b AND x1.a = x3.b @@ -1375,7 +1375,7 @@ select generate_subscripts(ARRAY[ARRAY[1,2,3], ARRAY[4,5,6]], 2); query error could not determine polymorphic type because input has type unknown SELECT generate_subscripts(NULL, 1) -# Regression test for materialize#9653: LATERAL column references in ROWS FROM +# Regression test for database-issues#2939: LATERAL column references in ROWS FROM query III SELECT * FROM generate_series(1, 3) as foo(a), ROWS FROM (generate_series(foo.a, foo.a + 2), generate_series(foo.a, foo.a + 1)) order by 1, 2, 3; @@ -1403,11 +1403,11 @@ SELECT * FROM generate_series(1, 3) as foo(a), ROWS FROM (generate_series(foo.a, 3 4 4 NULL 3 5 NULL NULL -# Regression test for materialize#9657 +# Regression test for database-issues#2941 statement ok create view bar as select * from y, rows from (generate_series(1, 2), jsonb_array_elements(y.a)); -# Regression for materialize#10363 +# Regression for database-issues#3078 query IT WITH a(x) AS (SELECT 'a') SELECT generate_series(1, 2), * FROM a ORDER BY generate_series ---- @@ -1416,7 +1416,7 @@ WITH a(x) AS (SELECT 'a') SELECT generate_series(1, 2), * FROM a ORDER BY genera # Test optimization for single table function in scalar position # Plan should be just a simple FlatMap + Project -# Regression test for materialize#10801 +# Regression test for database-issues#3173 query T multiline EXPLAIN WITH(arity, join implementations) SELECT jsonb_object_keys(a) FROM y; ---- @@ -1473,6 +1473,6 @@ SELECT generate_series FROM generate_series(1, 3) OVER (ORDER BY generate_series query error Expected right parenthesis, found number "1" SELECT generate_series FROM generate_series(DISTINCT 1, 3); -# Regression for https://github.com/MaterializeInc/materialize/issues/20533 +# Regression for https://api.github.com/repos/MaterializeInc/database-issues/issues/6180 query error db error: ERROR: regexp_extract must specify at least one capture group select regexp_extract('aaa', 'a') diff --git a/test/sqllogictest/temporal.slt b/test/sqllogictest/temporal.slt index d6be94136a24d..11bfe422cb73e 100644 --- a/test/sqllogictest/temporal.slt +++ b/test/sqllogictest/temporal.slt @@ -72,7 +72,7 @@ query error step mz_timestamp overflow select * from valid_max AS OF 0 # -# Regression test for materialize#6635 +# Regression test for database-issues#2066 # statement ok CREATE TABLE events ( @@ -219,7 +219,7 @@ select mz_now() < '3000-01-01'; ---- true -# Regression test for https://github.com/MaterializeInc/materialize/issues/25339 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/7560 query B with v as (select mz_now() < '3000-01-01') select * from v; ---- diff --git a/test/sqllogictest/timezone.slt b/test/sqllogictest/timezone.slt index 4548a90f56267..d1fa0e95cfbdf 100644 --- a/test/sqllogictest/timezone.slt +++ b/test/sqllogictest/timezone.slt @@ -140,7 +140,7 @@ SELECT TIMESTAMPTZ '2020-11-01 01:00:00 America/New_York' query error timestamp out of range SELECT pg_catalog.timezone(-INTERVAL '1' MINUTE, TIMESTAMP '95143-12-31 23:59:59' + INTERVAL '167 MILLENNIUM') -# Regression for materialize#20514 +# Regression for database-issues#6170 query error timestamp out of range SELECT pg_catalog.timezone('JAPAN', TIMESTAMPTZ '95143-12-31 23:59:59+06' + INTERVAL '167 MILLENNIUM') diff --git a/test/sqllogictest/transactions.slt b/test/sqllogictest/transactions.slt index 243b283a4bff8..c7456cffb546d 100644 --- a/test/sqllogictest/transactions.slt +++ b/test/sqllogictest/transactions.slt @@ -170,7 +170,7 @@ COMMIT ---- COMPLETE 0 -# Regression for materialize#5725 +# Regression for database-issues#1768 statement ok CREATE TABLE t5727 (i INT) diff --git a/test/sqllogictest/transform/aggregation_nullability.slt b/test/sqllogictest/transform/aggregation_nullability.slt index cb699198ec2a4..c82c8de0870ad 100644 --- a/test/sqllogictest/transform/aggregation_nullability.slt +++ b/test/sqllogictest/transform/aggregation_nullability.slt @@ -18,7 +18,7 @@ CREATE TABLE t2(f1 int, f2 int) statement ok INSERT INTO t1 VALUES (1, 2) -# regression test for materialize#7049: a filter on COUNT aggregation must not trigger the outer to inner join conversion +# regression test for database-issues#2192: a filter on COUNT aggregation must not trigger the outer to inner join conversion query T multiline EXPLAIN WITH(arity, join implementations) select t1.f1, count(t2.f1) from t1 LEFT JOIN t2 ON t1.f1 = t2.f1 group by t1.f1 having count(t2.f1) >= 0; ---- @@ -782,7 +782,7 @@ Target cluster: quickstart EOF -# regression test for materialize#7047 +# regression test for database-issues#2190 statement ok drop table t1 diff --git a/test/sqllogictest/transform/column_knowledge.slt b/test/sqllogictest/transform/column_knowledge.slt index 5cd8a8ff0ff5e..e63d4395c84a6 100644 --- a/test/sqllogictest/transform/column_knowledge.slt +++ b/test/sqllogictest/transform/column_knowledge.slt @@ -598,7 +598,7 @@ EOF statement ok CREATE TABLE json_table(data JSONB); -# Include map prefix in the `column_types` passed to `column_knowledge::optimize` (https://github.com/MaterializeInc/materialize/issues/15570) +# Include map prefix in the `column_types` passed to `column_knowledge::optimize` (https://api.github.com/repos/MaterializeInc/database-issues/issues/4460) query T multiline EXPLAIN WITH(arity, types) SELECT COALESCE(field, '') FROM (SELECT data->>'field' AS field FROM json_table); @@ -656,7 +656,7 @@ EOF # Same as the previous query, but with RECURSION LIMIT 1. # As of materialize#27389 this does not give up after the recursion limit, # as doing so is not necessary for equivalence propagation. -# Issue materialize#28308 tracks tests that may not be serving their intended +# Issue database-issues#8294 tracks tests that may not be serving their intended # purpose; perhaps we can remove the test if we remove the # column_knowledge transform. query T multiline diff --git a/test/sqllogictest/transform/fold_constants.slt b/test/sqllogictest/transform/fold_constants.slt index e88f96126204d..809675b179cbc 100644 --- a/test/sqllogictest/transform/fold_constants.slt +++ b/test/sqllogictest/transform/fold_constants.slt @@ -19,7 +19,7 @@ CREATE TABLE edges(src INTEGER NOT NULL, dst INTEGER NOT NULL); # If we make it so that reductions on constants # can be computed without writing out # the entire input constant in unary, we can get rid of this test: -# See https://github.com/MaterializeInc/materialize/issues/12963 . +# See https://api.github.com/repos/MaterializeInc/database-issues/issues/3723 . query T multiline EXPLAIN WITH(arity, join implementations) SELECT count(*) FROM billion; @@ -45,7 +45,7 @@ Target cluster: quickstart EOF -# Regression test for https://github.com/MaterializeInc/materialize/issues/16195 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4672 statement ok CREATE TABLE t1 (f1 int, f2 int); @@ -239,7 +239,7 @@ FROM t ---- 1 a b -# Regression test for materialize#8963 +# Regression test for database-issues#2730 query II SELECT row_number() OVER (), row_number() OVER () ---- @@ -252,7 +252,7 @@ SELECT row_number() OVER (), row_number() OVER () from t 1 1 2 2 -# Regression for materialize#9749 +# Regression for database-issues#2962 query error window functions are not allowed in ON WITH t (x) AS (VALUES ('a'), ('b'), ('c')) SELECT * FROM t AS v JOIN t ON row_number() over () > 1; diff --git a/test/sqllogictest/transform/fold_vs_dataflow/1_numbers_dataflow.slt b/test/sqllogictest/transform/fold_vs_dataflow/1_numbers_dataflow.slt index f327e3bd1963a..f68aac14b0419 100644 --- a/test/sqllogictest/transform/fold_vs_dataflow/1_numbers_dataflow.slt +++ b/test/sqllogictest/transform/fold_vs_dataflow/1_numbers_dataflow.slt @@ -22,7 +22,7 @@ statement ok INSERT INTO t_using_dataflow_rendering VALUES (1e-39::real, 1e-39::double, 1e-39::numeric); # DIFF TO CONSTANT FOLDING (SELECT on types [REAL, DOUBLE])! -# to be addressed with https://github.com/MaterializeInc/materialize/issues/15186 +# to be addressed with https://api.github.com/repos/MaterializeInc/database-issues/issues/4341 query RRR SELECT * FROM t_using_dataflow_rendering UNION ALL @@ -55,7 +55,7 @@ statement ok INSERT INTO t_using_dataflow_rendering VALUES (1e38::real, 1e38::double, 1e38::numeric); # DIFF TO CONSTANT FOLDING (SELECT on types [REAL, DOUBLE])! -# to be addressed with https://github.com/MaterializeInc/materialize/issues/15186 +# to be addressed with https://api.github.com/repos/MaterializeInc/database-issues/issues/4341 query RRR SELECT * FROM t_using_dataflow_rendering UNION ALL diff --git a/test/sqllogictest/transform/fold_vs_dataflow/3_number_aggfns_dataflow.slt b/test/sqllogictest/transform/fold_vs_dataflow/3_number_aggfns_dataflow.slt index aba1efb6c7de8..5080a5175b04c 100644 --- a/test/sqllogictest/transform/fold_vs_dataflow/3_number_aggfns_dataflow.slt +++ b/test/sqllogictest/transform/fold_vs_dataflow/3_number_aggfns_dataflow.slt @@ -24,7 +24,7 @@ INSERT INTO t_using_dataflow_rendering VALUES (2, 2, 2, 32767::INT2, 1::INT2); # DIFF TO CONSTANT FOLDING ([SUM, AVG] with types [REAL, DOUBLE])! -# to be addressed with https://github.com/MaterializeInc/materialize/issues/15186 +# to be addressed with https://api.github.com/repos/MaterializeInc/database-issues/issues/4341 query IIIIIIIIIIIIIIIIIIIIIIII SELECT SUM(real1), SUM(double1), SUM(numeric1), diff --git a/test/sqllogictest/transform/fold_vs_dataflow/4_text_dataflow.slt b/test/sqllogictest/transform/fold_vs_dataflow/4_text_dataflow.slt index f8b99a7328836..288c8addfefc7 100644 --- a/test/sqllogictest/transform/fold_vs_dataflow/4_text_dataflow.slt +++ b/test/sqllogictest/transform/fold_vs_dataflow/4_text_dataflow.slt @@ -27,7 +27,7 @@ statement ok INSERT INTO t_operator VALUES ('%bCd%'); # DIFF TO CONSTANT FOLDING (ILIKE on TEXT)! -# to be addressed with https://github.com/MaterializeInc/materialize/issues/18494 +# to be addressed with https://api.github.com/repos/MaterializeInc/database-issues/issues/5462 query T SELECT * FROM t_data WHERE f1 ILIKE (SELECT op_val FROM t_operator) diff --git a/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_dataflow.slt b/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_dataflow.slt index 05e8516e1e84e..7dd98b0a11287 100644 --- a/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_dataflow.slt +++ b/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_dataflow.slt @@ -31,7 +31,7 @@ statement ok CREATE MATERIALIZED VIEW mv_using_dataflow_rendering AS SELECT data FROM t_base, repeat_row(diff); -# see also: https://github.com/MaterializeInc/materialize/issues/17758 +# see also: https://api.github.com/repos/MaterializeInc/database-issues/issues/5172 # query error # SELECT SUM(data) # FROM mv_using_dataflow_rendering; diff --git a/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_folding.slt b/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_folding.slt index 0fe6c088ec7ba..38a41bcb34016 100644 --- a/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_folding.slt +++ b/test/sqllogictest/transform/fold_vs_dataflow/5_repeat_row_folding.slt @@ -22,7 +22,7 @@ INSERT INTO t_base VALUES (1, 9223372036854775807); statement ok INSERT INTO t_base VALUES (1, 9223372036854775807); -# see also: https://github.com/MaterializeInc/materialize/issues/17758 +# see also: https://api.github.com/repos/MaterializeInc/database-issues/issues/5172 # statement error # CREATE VIEW v_using_constant_folding AS # SELECT data FROM ( diff --git a/test/sqllogictest/transform/join_index.slt b/test/sqllogictest/transform/join_index.slt index e244072dbd24c..ec8b59853c5b1 100644 --- a/test/sqllogictest/transform/join_index.slt +++ b/test/sqllogictest/transform/join_index.slt @@ -245,7 +245,7 @@ statement ok CREATE INDEX bar_idx2 on bar(-a); # Test that when join planning uses indexes on expressions. -# Protects against regression of materialize#4170. +# Protects against regression of database-issues#1290. query T multiline EXPLAIN WITH(arity, join implementations) select foo.b, bar.b @@ -403,7 +403,7 @@ and bar.b - foo.b = foo.a / bar.a statement ok DROP INDEX baz_idx -# materialize#8002: it would be nice if this join used the indexes on bar(a+4) +# database-issues#2449: it would be nice if this join used the indexes on bar(a+4) query T multiline EXPLAIN WITH(arity, join implementations) diff --git a/test/sqllogictest/transform/lifting.slt b/test/sqllogictest/transform/lifting.slt index b5903d7079899..f6bc3711a2710 100644 --- a/test/sqllogictest/transform/lifting.slt +++ b/test/sqllogictest/transform/lifting.slt @@ -735,7 +735,7 @@ Target cluster: quickstart EOF -# regression tests for materialize#6943 +# regression tests for database-issues#2159 query T multiline explain with(arity, join implementations) select min(1/x) from (select a as y, 0 as x from t); ---- diff --git a/test/sqllogictest/transform/literal_constraints.slt b/test/sqllogictest/transform/literal_constraints.slt index 6d0864e620773..1a9eb12edc686 100644 --- a/test/sqllogictest/transform/literal_constraints.slt +++ b/test/sqllogictest/transform/literal_constraints.slt @@ -24,7 +24,7 @@ statement ok INSERT INTO t1 VALUES (0, 'nnn'), (1, 'l1'), (1, 'a'), (2, 'l2'), (3, 'l3'), (1234, 'xxx'), (3456, 'yyy'), (12345, 'zzz'), (null, 'nnn') # A very large IN list shouldn't cause a stack overflow or other issue in the optimizer, see -# https://github.com/MaterializeInc/materialize/issues/6247 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/1924 query T rowsort SELECT b FROM t1 @@ -455,7 +455,7 @@ WHERE a+a IN (-1, 1, 2, 3, 6, 7, 9) # A non-trivial expression in a literal constraint, and the same expression also appears in the output. # Exercises the second part of `inline_literal_constraints`. -# https://github.com/MaterializeInc/materialize/issues/13151#issuecomment-1192763523 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/3783#issuecomment-1192763523 query T multiline EXPLAIN WITH(arity, join implementations) SELECT a+a FROM t1 @@ -510,7 +510,7 @@ WHERE (a,b) IN ((1, 4*a), (2, 5*a), (3, a+20)) 1 4 7 3 23 33 -# Regression test for https://github.com/MaterializeInc/materialize/issues/14532 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4144 # Copied from test/sqllogictest/sqlite/test/index/orderby_nosort/100/slt_good_3.test statement ok @@ -528,7 +528,7 @@ SELECT pk FROM tab0 WHERE (((col4 >= 660.98) AND ((col4 IN (724.71,445.29,441.2, 0 1 -# Regression test for https://github.com/MaterializeInc/materialize/issues/14548 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4153 # Also tests the situation when `constraints_to_residual_sets` has multiple elements in the sets. statement ok @@ -1126,7 +1126,7 @@ Target cluster: quickstart EOF -# Regression test for https://github.com/MaterializeInc/materialize/issues/15696 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4506 # CREATE INDEX should call reduce on index expressions. statement ok @@ -1149,7 +1149,7 @@ Target cluster: quickstart EOF -# Regression test for https://github.com/MaterializeInc/materialize/issues/18410 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5424 # We should resist the temptation to do an index lookup in the following test; `preserves_uniqueness` should be checked # not just on the inverse cast, but on the original cast as well. diff --git a/test/sqllogictest/transform/non_null_requirements.slt b/test/sqllogictest/transform/non_null_requirements.slt index 1433068036c76..14b26f1e25b9e 100644 --- a/test/sqllogictest/transform/non_null_requirements.slt +++ b/test/sqllogictest/transform/non_null_requirements.slt @@ -22,7 +22,7 @@ CREATE TABLE t1 (f1 INTEGER, f2 INTEGER); # Simplify non-nullable trees with multiple non-recursive bindings defined under # a single`WITH MUTUALLY RECURSIVE` block # As of materialize#27389 this no longer tests what it used to test. -# Issue materialize#28308 tracks tests that may not be serving their intended +# Issue database-issues#8294 tracks tests that may not be serving their intended # purpose; perhaps we can remove the test if we remove the # non_null_requirements transform. query T multiline diff --git a/test/sqllogictest/transform/normalize_lets.slt b/test/sqllogictest/transform/normalize_lets.slt index 501eb818de34f..8bf59c90416d2 100644 --- a/test/sqllogictest/transform/normalize_lets.slt +++ b/test/sqllogictest/transform/normalize_lets.slt @@ -36,7 +36,7 @@ Target cluster: mz_catalog_server EOF ## Test a nested recursive query. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN WITH MUTUALLY RECURSIVE foo (a int8) AS ( @@ -55,9 +55,9 @@ Target cluster: mz_catalog_server EOF -## Test inlining at an inner nesting level. (materialize#18889) +## Test inlining at an inner nesting level. (database-issues#5581) ## `bar` is used only in `baz`, so it should be inlined. The inner WMR should have only one cte. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN WITH MUTUALLY RECURSIVE foo (a int8) AS ( @@ -120,7 +120,7 @@ Target cluster: quickstart EOF ## Test consolidation of not-really nested recursive query. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN WITH MUTUALLY RECURSIVE foo (a int8) AS ( @@ -140,7 +140,7 @@ Target cluster: mz_catalog_server EOF ## Test consolidation of independent recursive query blocks. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN SELECT * FROM ( WITH MUTUALLY RECURSIVE @@ -321,7 +321,7 @@ Target cluster: mz_catalog_server EOF ## RECURSION LIMIT -- consolidation of independent recursive query blocks with different RECURSION LIMIT -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN SELECT * FROM ( @@ -345,7 +345,7 @@ EOF ## RECURSION LIMIT -- consolidation of independent recursive query blocks with equal RECURSION LIMIT. ## We want to see RECURSION LIMIT printed at the block level rather than on each cte. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN SELECT * FROM ( @@ -369,7 +369,7 @@ EOF ## ITERATION RECURSION -- consolidation of not-really nested recursive query. ## Here, the ITERATION RECURSION of the inner WMR is irrelevant, because it's not recursive. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN WITH MUTUALLY RECURSIVE (RECURSION LIMIT 9) foo (a int8) AS ( @@ -389,7 +389,7 @@ Target cluster: mz_catalog_server EOF # ITERATION RECURSION -- a nested recursive query. -# With materialize#27389 this stopped testing a thing; see issue materialize#28308. +# With materialize#27389 this stopped testing a thing; see issue database-issues#8294. query T multiline EXPLAIN WITH MUTUALLY RECURSIVE (RECURSION LIMIT 17) foo (a int8) AS ( @@ -483,7 +483,7 @@ Target cluster: quickstart EOF ## Make sure that we run `NormalizeLets` just before `CollectIndexRequests`. -## Regression test for https://github.com/MaterializeInc/materialize/issues/21175 +## Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6371 statement ok CREATE TABLE t (a int2, b int4, c int8, d uint2, e uint4, f uint8, g text); diff --git a/test/sqllogictest/transform/notice/index_too_wide_for_literal_constraints.slt b/test/sqllogictest/transform/notice/index_too_wide_for_literal_constraints.slt index 5492291a48823..662b687d0f416 100644 --- a/test/sqllogictest/transform/notice/index_too_wide_for_literal_constraints.slt +++ b/test/sqllogictest/transform/notice/index_too_wide_for_literal_constraints.slt @@ -405,7 +405,7 @@ Notices: EOF -# Regression test for materialize#24301 +# Regression test for database-issues#7268 # -------------------------- statement ok diff --git a/test/sqllogictest/transform/predicate_pushdown.slt b/test/sqllogictest/transform/predicate_pushdown.slt index 073efadc6545d..4f1302f10aa5d 100644 --- a/test/sqllogictest/transform/predicate_pushdown.slt +++ b/test/sqllogictest/transform/predicate_pushdown.slt @@ -186,7 +186,7 @@ Target cluster: quickstart EOF -# materialize#7684 avoidable cross joins +# database-issues#2377 avoidable cross joins query T multiline EXPLAIN WITH(arity, join implementations) select * from t1, t2 where t1.f1 = t2.f1 + 1 or (t1.f1 is null and t2.f1 is null); @@ -387,7 +387,7 @@ Target cluster: quickstart EOF -# Regression tests for https://github.com/MaterializeInc/materialize/issues/16128 +# Regression tests for https://api.github.com/repos/MaterializeInc/database-issues/issues/4659 statement ok CREATE TABLE tt1 (f1 DOUBLE PRECISION, f2 DOUBLE PRECISION NOT NULL); @@ -450,7 +450,7 @@ Target cluster: mz_catalog_server EOF -# Regression tests for materialize#17515 +# Regression tests for database-issues#5089 # Push predicates through Threshold and TopK # Test Threshold. @@ -516,7 +516,7 @@ Target cluster: quickstart EOF -# One more for https://github.com/MaterializeInc/materialize/issues/16128 +# One more for https://api.github.com/repos/MaterializeInc/database-issues/issues/4659 # https://github.com/MaterializeInc/materialize/pull/16147#issuecomment-1322042176 simple conn=mz_system,user=mz_system @@ -925,7 +925,7 @@ Target cluster: quickstart EOF -# Regression test for https://github.com/MaterializeInc/materialize/issues/19179 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5691 statement ok with v1 as ( @@ -954,7 +954,7 @@ with select * from v1 WHERE credits_per_hour > credits_per_hour; -# Regression test for https://github.com/MaterializeInc/materialize/issues/20199 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6049 statement ok CREATE ROLE r2; @@ -1009,7 +1009,7 @@ EOF statement ok DROP ROLE r2; -# Regression test for https://github.com/MaterializeInc/materialize/issues/22257 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6714 # The transitive closure computation in `inline_if_not_too_big` has to do more than one step here. For this, later map # expressions should mention only the previous map expression, but not earlier ones. Also, there has to be at least # 3 map expressions, so that adding the support of the last one doesn't immediately cover all of the map expressions. diff --git a/test/sqllogictest/transform/reduction_pushdown.slt b/test/sqllogictest/transform/reduction_pushdown.slt index 62db19a0fe08b..71e2b28f077de 100644 --- a/test/sqllogictest/transform/reduction_pushdown.slt +++ b/test/sqllogictest/transform/reduction_pushdown.slt @@ -88,7 +88,7 @@ Target cluster: quickstart EOF -## Regression test for https://github.com/MaterializeInc/materialize/issues/27702 +## Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/8146 statement ok CREATE TABLE t1 (f1 DOUBLE PRECISION, f2 DOUBLE PRECISION NOT NULL); diff --git a/test/sqllogictest/transform/redundant_join.slt b/test/sqllogictest/transform/redundant_join.slt index ced426dea7569..5d6c77d22e624 100644 --- a/test/sqllogictest/transform/redundant_join.slt +++ b/test/sqllogictest/transform/redundant_join.slt @@ -86,7 +86,7 @@ EOF # Same query, but selecting from t1 instead of t2. The added `- IS NOT NULL` # filters prevent redundant join elimination at first. To be able to eliminate # the join we need to factor out the filter on top of t1 behind a common -# binding (fixed by materialize#18173). +# binding (fixed by database-issues#5342). query T multiline EXPLAIN WITH(arity, join implementations) WITH MUTUALLY RECURSIVE diff --git a/test/sqllogictest/transform/relation_cse.slt b/test/sqllogictest/transform/relation_cse.slt index ed1abfe47ecce..2f9d37d5b8082 100644 --- a/test/sqllogictest/transform/relation_cse.slt +++ b/test/sqllogictest/transform/relation_cse.slt @@ -1547,7 +1547,7 @@ EOF # (1) a Filter (#1 > 7) over l0 appears twice. # (2) l2 is equivalent to l5. # (3) l1 is not equivalent (although structurally equal) to l4. -# With materialize#27389 this stopped testing what it says it tests; see issue materialize#28308. +# With materialize#27389 this stopped testing what it says it tests; see issue database-issues#8294. query T multiline EXPLAIN WITH(arity, join implementations) WITH MUTUALLY RECURSIVE diff --git a/test/sqllogictest/transform/scalability.slt b/test/sqllogictest/transform/scalability.slt index 7f705e0477bf8..c90f56d0b7628 100644 --- a/test/sqllogictest/transform/scalability.slt +++ b/test/sqllogictest/transform/scalability.slt @@ -18,7 +18,7 @@ create table events(origin_id int, category_id int, payload jsonb, timestamp_col statement ok create table origins(id int, category_id int); -# Regression test for materialize#8387 +# Regression test for database-issues#2558 # Ignoring the plan on purpose. We just want to check we can handle the query. statement ok EXPLAIN WITH(arity, join implementations) diff --git a/test/sqllogictest/transform/scalar_cse.slt b/test/sqllogictest/transform/scalar_cse.slt index 6af66b5671856..a11714ca829e6 100644 --- a/test/sqllogictest/transform/scalar_cse.slt +++ b/test/sqllogictest/transform/scalar_cse.slt @@ -118,7 +118,7 @@ Target cluster: quickstart EOF -# Regression test for https://github.com/MaterializeInc/materialize/issues/21178 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6372 statement ok CREATE TABLE t(x int); diff --git a/test/sqllogictest/transform/threshold_elision.slt b/test/sqllogictest/transform/threshold_elision.slt index c267a9230dd09..ae53607c6821a 100644 --- a/test/sqllogictest/transform/threshold_elision.slt +++ b/test/sqllogictest/transform/threshold_elision.slt @@ -409,10 +409,10 @@ Target cluster: quickstart EOF # WMR -# https://github.com/MaterializeInc/materialize/issues/18175 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/5344 # WMR -- Threshold in the loop-invariant part -# The "Very basic" implementation from materialize#18175 should handle this. +# The "Very basic" implementation from database-issues#5344 should handle this. query T multiline EXPLAIN OPTIMIZED PLAN WITH(non negative) FOR WITH MUTUALLY RECURSIVE @@ -458,7 +458,7 @@ EOF # WMR -- Threshold inside the cycle -- Two Thresholds, the second one should be easy to eliminate, because the result of # the first one is obviously non-negative. -# The "Basic" implementation from materialize#18175 should handle this. +# The "Basic" implementation from database-issues#5344 should handle this. query T multiline EXPLAIN OPTIMIZED PLAN WITH(non negative) FOR WITH MUTUALLY RECURSIVE diff --git a/test/sqllogictest/types.slt b/test/sqllogictest/types.slt index 49edebd9f9a47..f219c1a6aac95 100644 --- a/test/sqllogictest/types.slt +++ b/test/sqllogictest/types.slt @@ -507,7 +507,7 @@ statement error view "materialize.public.int4" already exists CREATE TYPE int4 AS (a int) # creating relations with the same name as an existing type is not allowed -# (see materialize#23789)... +# (see database-issues#7142)... statement ok CREATE TYPE rectype AS (a int) diff --git a/test/sqllogictest/uniqueness_propagation_filter.slt b/test/sqllogictest/uniqueness_propagation_filter.slt index 0865ca3d09500..73c170c0f8978 100644 --- a/test/sqllogictest/uniqueness_propagation_filter.slt +++ b/test/sqllogictest/uniqueness_propagation_filter.slt @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # Regression test for -# bug: https://github.com/MaterializeInc/materialize/issues/9397 +# bug: https://api.github.com/repos/MaterializeInc/database-issues/issues/2874 # fix: https://github.com/MaterializeInc/materialize/pull/9398 simple conn=mz_system,user=mz_system diff --git a/test/sqllogictest/unsigned_int.slt b/test/sqllogictest/unsigned_int.slt index 68e0c02a751a5..b441ac8cac795 100644 --- a/test/sqllogictest/unsigned_int.slt +++ b/test/sqllogictest/unsigned_int.slt @@ -1334,7 +1334,7 @@ SELECT STDDEV(a) FROM t8 ---- 4611686017711549096.944 -# Avoid overflow (known issue https://github.com/MaterializeInc/materialize/issues/15186) +# Avoid overflow (known issue https://api.github.com/repos/MaterializeInc/database-issues/issues/4341) statement ok DROP TABLE t8 diff --git a/test/sqllogictest/updates.slt b/test/sqllogictest/updates.slt index 555c3f58e264d..f9f4146db9d29 100644 --- a/test/sqllogictest/updates.slt +++ b/test/sqllogictest/updates.slt @@ -135,7 +135,7 @@ SELECT * FROM t1 statement error db error: ERROR: column "t1\.x" does not exist UPDATE t1 AS m SET x = 5 WHERE t1.x < 10 -# Regression for materialize#24612 +# Regression for database-issues#7334 simple BEGIN; SELECT 1; @@ -178,7 +178,7 @@ statement ok ROLLBACK # Verify that max_query_result_size doesn't affect read part of RTW queries. -# See materialize#27428 +# See database-issues#8099 statement ok SET max_query_result_size = '8B'; diff --git a/test/sqllogictest/vars.slt b/test/sqllogictest/vars.slt index 1c83a73e25f7a..4b4a742f09f75 100644 --- a/test/sqllogictest/vars.slt +++ b/test/sqllogictest/vars.slt @@ -434,7 +434,7 @@ SELECT generate_series(1, 2) query error db error: ERROR: result exceeds max size of 100 B SELECT generate_series(1, 51) -# Regression for materialize#22724 +# Regression for database-issues#6866 # Ensure duplicate rows don't overcount bytes in the presence of LIMIT. query T SELECT x FROM (VALUES ('{"row": 1}')) AS a (x), generate_series(1, 50000) LIMIT 1 diff --git a/test/sqllogictest/window_funcs.slt b/test/sqllogictest/window_funcs.slt index 4977b691a3743..f973070e55b91 100644 --- a/test/sqllogictest/window_funcs.slt +++ b/test/sqllogictest/window_funcs.slt @@ -119,7 +119,7 @@ FROM t ---- 1 a b -# Regression test for materialize#8963 +# Regression test for database-issues#2730 query II SELECT row_number() OVER (), row_number() OVER () ---- @@ -179,7 +179,7 @@ Target cluster: quickstart EOF # -# Regression test for materialize#9077 +# Regression test for database-issues#2760 # statement ok @@ -404,7 +404,7 @@ CREATE TABLE t(x string); statement ok INSERT INTO t VALUES ('a'), ('b'), ('c'); -# Regression for materialize#9749 +# Regression for database-issues#2962 query error window functions are not allowed in ON SELECT * FROM t AS v JOIN t ON row_number() over () > 1; @@ -5497,7 +5497,7 @@ ORDER BY a; 3 three 2 # Window function inside CASE WHEN -# Regression test for https://github.com/MaterializeInc/materialize/issues/20746 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6250 statement ok CREATE TABLE bools(cond bool, x int); @@ -6233,7 +6233,7 @@ NULL 3 NULL {9,3,11} {6,18,22} 18 NULL 9 NULL {9,11} {18,22} 22 NULL 11 NULL {11} {22} NULL -# https://github.com/MaterializeInc/materialize/issues/22015 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/6626 query error db error: ERROR: DISTINCT in window aggregates not yet supported SELECT x-y, diff --git a/test/sqllogictest/with_mutually_recursive.slt b/test/sqllogictest/with_mutually_recursive.slt index 24a37afc22068..37edb304c438a 100644 --- a/test/sqllogictest/with_mutually_recursive.slt +++ b/test/sqllogictest/with_mutually_recursive.slt @@ -337,7 +337,7 @@ GROUP BY round; ---- 0 10 -## Regression test for https://github.com/MaterializeInc/materialize/issues/18759 +## Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5550 ## Test a WMR query with a delta join. query III WITH MUTUALLY RECURSIVE @@ -363,7 +363,7 @@ SELECT * FROM c1; ---- 0 0 0 -## Regression test for https://github.com/MaterializeInc/materialize/issues/18949 +## Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/5606 ## Test the situation when a WMR cte has an inner WMR whose body ends with an arrangement. query I WITH MUTUALLY RECURSIVE diff --git a/test/sqlsmith/mzcompose.py b/test/sqlsmith/mzcompose.py index c362d9cf87390..f8ff2ae8edc9a 100644 --- a/test/sqlsmith/mzcompose.py +++ b/test/sqlsmith/mzcompose.py @@ -85,7 +85,7 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: parser.add_argument("--num-sqlsmith", default=len(MZ_SERVERS), type=int) # parser.add_argument("--queries", default=10000, type=int) parser.add_argument("--runtime", default=600, type=int) - # https://github.com/MaterializeInc/materialize/issues/2392 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/835 parser.add_argument("--max-joins", default=2, type=int) parser.add_argument("--explain-only", action="store_true") parser.add_argument("--exclude-catalog", default=False, type=bool) diff --git a/test/ssh-connection/pg-source-after-ssh-restart.td b/test/ssh-connection/pg-source-after-ssh-restart.td index 4469ff05ee74e..d8a081ae180bc 100644 --- a/test/ssh-connection/pg-source-after-ssh-restart.td +++ b/test/ssh-connection/pg-source-after-ssh-restart.td @@ -22,7 +22,7 @@ INSERT INTO t1 VALUES (3); 3 -# TODO: materialize#29373 (introspection tables) +# TODO: database-issues#8511 (introspection tables) # > SELECT name, type, error # FROM mz_internal.mz_source_statuses # WHERE diff --git a/test/storage-usage/mzcompose.py b/test/storage-usage/mzcompose.py index 5c4f0d93616f2..34a3823393f7c 100644 --- a/test/storage-usage/mzcompose.py +++ b/test/storage-usage/mzcompose.py @@ -102,7 +102,7 @@ class DatabaseObject: expected_size=1024 * 1024, ), # Deleted/updated rows should be garbage-collected - # https://github.com/MaterializeInc/materialize/issues/15093 + # https://api.github.com/repos/MaterializeInc/database-issues/issues/4313 # DatabaseObject( # name="table_delete", # testdrive=dedent( diff --git a/test/testdrive/avro-nonnull-record.td b/test/testdrive/avro-nonnull-record.td index cd3afd1b91162..87d4dd4597789 100644 --- a/test/testdrive/avro-nonnull-record.td +++ b/test/testdrive/avro-nonnull-record.td @@ -9,7 +9,7 @@ $ set-arg-default single-replica-cluster=quickstart -# Regression test for materialize#7170, in which non-nullable fields of nullable records +# Regression test for database-issues#2237, in which non-nullable fields of nullable records # were not correctly handled. $ set writer-schema={ diff --git a/test/testdrive/consolidation.td b/test/testdrive/consolidation.td index eb4c519b2cdd5..7e3d6ed96060b 100644 --- a/test/testdrive/consolidation.td +++ b/test/testdrive/consolidation.td @@ -170,7 +170,7 @@ $ kafka-verify-data headers=materialize-timestamp format=avro sink=materialize.p # 6 {"before": null, "after": {"row": {"num": 5}}} # Validate that `SUBSCRIBE` is similarly consolidated. -# This protects against regression of materialize#5421. +# This protects against regression of database-issues#1675. > BEGIN > DECLARE cur CURSOR FOR SUBSCRIBE nums AS OF 3 diff --git a/test/testdrive/copy-to-s3-minio.td b/test/testdrive/copy-to-s3-minio.td index 8e3a70d319a8d..671a9284a3c20 100644 --- a/test/testdrive/copy-to-s3-minio.td +++ b/test/testdrive/copy-to-s3-minio.td @@ -182,7 +182,7 @@ array;int4;jsonb;timestamp # Copy a large amount of data in the background and check to see that the INCOMPLETE # sentinel object is written during the copy -# TODO(materialize#26963): Enable this test once it is more reliable +# TODO(database-issues#7984): Enable this test once it is more reliable # $ postgres-execute background=true connection=postgres://materialize:materialize@${testdrive.materialize-sql-addr} # COPY (SELECT * FROM generate_series(1, 50000000)) TO 's3://copytos3/test/5' WITH (AWS CONNECTION = aws_conn, MAX FILE SIZE = "100MB", FORMAT = 'csv'); diff --git a/test/testdrive/create-views.td b/test/testdrive/create-views.td index 03fe3b36f0d48..31daa30a0d40a 100644 --- a/test/testdrive/create-views.td +++ b/test/testdrive/create-views.td @@ -54,7 +54,7 @@ contains:column reference "a" is ambiguous > SELECT * FROM t_v_alias 2 1 -# Regression for materialize#9376 +# Regression for database-issues#2869 ! CREATE VIEW gh9376 AS SELECT 1, 2; contains:column "?column?" specified more than once diff --git a/test/testdrive/dataflow-cleanup.td b/test/testdrive/dataflow-cleanup.td index e5ffe2f13893b..89ca2fbf0b42b 100644 --- a/test/testdrive/dataflow-cleanup.td +++ b/test/testdrive/dataflow-cleanup.td @@ -78,7 +78,7 @@ false > SELECT count(*) > 0 FROM mz_introspection.mz_dataflow_operators false -# Regression test for https://github.com/MaterializeInc/materialize/issues/16326 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4712 > CREATE CLUSTER lgtpch_cluster_cleanup SIZE '1'; diff --git a/test/testdrive/date_func.td b/test/testdrive/date_func.td index 5608509557a0e..9df5ced526b48 100644 --- a/test/testdrive/date_func.td +++ b/test/testdrive/date_func.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # -# Argument to EXTRACT not checked for correctness, NULL returned if invalid materialize#5521 +# Argument to EXTRACT not checked for correctness, NULL returned if invalid database-issues#1701 # expr: propagate errors when reducing date_{part,trunc} materialize#5700 # diff --git a/test/testdrive/dependencies.td b/test/testdrive/dependencies.td index 057b157643bf2..59e94e8ed2bd8 100644 --- a/test/testdrive/dependencies.td +++ b/test/testdrive/dependencies.td @@ -338,7 +338,7 @@ contains:unknown catalog item 'j2' > DROP SOURCE s5 CASCADE; -# https://github.com/MaterializeInc/materialize/issues/5577 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/1714 > CREATE TYPE int4_list AS LIST (ELEMENT TYPE = int4) > CREATE VIEW v1 AS SELECT CAST('{2}' AS int4_list) diff --git a/test/testdrive/disabled/kafka-avro-debezium-transaction.td b/test/testdrive/disabled/kafka-avro-debezium-transaction.td index 2a72c0d875f5d..ecba44cfc648a 100644 --- a/test/testdrive/disabled/kafka-avro-debezium-transaction.td +++ b/test/testdrive/disabled/kafka-avro-debezium-transaction.td @@ -12,7 +12,7 @@ $ set-arg-default single-replica-cluster=quickstart # Test support for Avro sources without using the Confluent Schema Registry. # This test is broken. -# See: https://github.com/MaterializeInc/materialize/issues/13629 +# See: https://api.github.com/repos/MaterializeInc/database-issues/issues/3892 $ set schema={ "type": "record", diff --git a/test/testdrive/distinct-arrangements.td b/test/testdrive/distinct-arrangements.td index 58ee77991418e..159fadcb5d3da 100644 --- a/test/testdrive/distinct-arrangements.td +++ b/test/testdrive/distinct-arrangements.td @@ -11,7 +11,7 @@ $ set-arg-default default-replica-size=1 # When updating this file, make sure that the number of arrangements doesn't # increase unexpectedly. This is to prevent issues like this: -# https://github.com/MaterializeInc/materialize/issues/20179 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/6038 # Introspection subscribes add noise to the introspection sources, so disable them. $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} @@ -114,7 +114,7 @@ DistinctByErrorCheck > DROP TABLE t3 CASCADE # from materialized_views.slt -# Disabled because of https://github.com/MaterializeInc/materialize/issues/20188 +# Disabled because of https://api.github.com/repos/MaterializeInc/database-issues/issues/6044 #> CREATE MATERIALIZED VIEW mat_clusters AS SELECT name FROM (SHOW CLUSTERS); #> CREATE MATERIALIZED VIEW mat_connections AS SELECT name, type FROM (SHOW CONNECTIONS); #> CREATE MATERIALIZED VIEW mat_databases AS SELECT name FROM (SHOW DATABASES); diff --git a/test/testdrive/drop.td b/test/testdrive/drop.td index 74467cdf238d4..e9eb95e0dd6f7 100644 --- a/test/testdrive/drop.td +++ b/test/testdrive/drop.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # Test DROP ... CASCADE statements with multiple items -# From https://github.com/MaterializeInc/materialize/issues/5316 +# From https://api.github.com/repos/MaterializeInc/database-issues/issues/1647 > CREATE TABLE t1 (f1 INTEGER) > CREATE TABLE t2 (f2 INTEGER) > CREATE MATERIALIZED VIEW v1 AS SELECT * FROM t1, t2 @@ -16,7 +16,7 @@ # Test CREATE OR REPLACE statement that attempts to depend # on the object that is being replaced -# https://github.com/MaterializeInc/materialize/issues/5272 +# https://api.github.com/repos/MaterializeInc/database-issues/issues/1626 > CREATE VIEW v2 AS SELECT 1 ! CREATE OR REPLACE VIEW v2 AS SELECT * FROM v2 contains:cannot replace view materialize.public.v2: depended upon by new materialize.public.v2 definition diff --git a/test/testdrive/fivetran-destination.td b/test/testdrive/fivetran-destination.td index cf559b730fab4..63441859806d0 100644 --- a/test/testdrive/fivetran-destination.td +++ b/test/testdrive/fivetran-destination.td @@ -123,7 +123,7 @@ $ file-append container=fivetran path=d.csv compression=gzip header=a,b,c,d repe # Note: Both 'c.csv' and 'd.csv' are individually under the 'max_copy_from_size', but together # they exceed it. We want to make sure the write_batch still succeeds in this case. -# TODO(def-) Reenable when materialize#28687 is fixed +# TODO(def-) Reenable when database-issues#8373 is fixed $ skip-if SELECT true diff --git a/test/testdrive/github-10587.td b/test/testdrive/github-10587.td index 275970ec88b06..521fb64e88ebb 100644 --- a/test/testdrive/github-10587.td +++ b/test/testdrive/github-10587.td @@ -12,7 +12,7 @@ $ set-arg-default default-storage-size=1 $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET max_clusters = 20 -# Regression test for https://github.com/MaterializeInc/materialize/issues/10587 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/3135 # # This file uses the old create source syntax. diff --git a/test/testdrive/github-13790.td b/test/testdrive/github-13790.td index 24869ca3f9697..427f1030d6ba4 100644 --- a/test/testdrive/github-13790.td +++ b/test/testdrive/github-13790.td @@ -9,7 +9,7 @@ $ set-arg-default default-replica-size=1 -# Regression test for https://github.com/MaterializeInc/materialize/issues/13790 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/3955 # The contents of the introspection tables depend on the replica size $ skip-if diff --git a/test/testdrive/github-15095.td b/test/testdrive/github-15095.td index 3136f637bd012..6d48e4b9ad324 100644 --- a/test/testdrive/github-15095.td +++ b/test/testdrive/github-15095.td @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/15095 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/4315 > CREATE CLUSTER c REPLICAS (r (SIZE '2-2')) diff --git a/test/testdrive/github-18608.td b/test/testdrive/github-18608.td index 8455aa9ed5993..4b6ee4b07b623 100644 --- a/test/testdrive/github-18608.td +++ b/test/testdrive/github-18608.td @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test from materialize#18608, where earlier errors were erroneously +# Regression test from database-issues#5502, where earlier errors were erroneously # discarded by TopK's error handling. > CREATE TABLE t0(c0 FLOAT); diff --git a/test/testdrive/github-21031.td b/test/testdrive/github-21031.td index 8293ecdb0f159..31636b060fac1 100644 --- a/test/testdrive/github-21031.td +++ b/test/testdrive/github-21031.td @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/21031. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/6335. # # This test confirms that subscribes that advance to the empty frontier are # fully cleaned up. diff --git a/test/testdrive/github-2276.td b/test/testdrive/github-2276.td index 54b349b296dd0..360980fd38fcf 100644 --- a/test/testdrive/github-2276.td +++ b/test/testdrive/github-2276.td @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/2276. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/795. # # Verifies that a deep stack of dependencies can be properly created. diff --git a/test/testdrive/github-3281.td b/test/testdrive/github-3281.td index ed46830386a9e..b62bfac0556af 100644 --- a/test/testdrive/github-3281.td +++ b/test/testdrive/github-3281.td @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/3281. +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/1056. # # Verifies that the precedence of `->>`, `::`, and `CAST` are not mishandled # when rendering a SQL statement. diff --git a/test/testdrive/github-5668.td b/test/testdrive/github-5668.td index 28f464197ee0b..9f8bddbd07164 100644 --- a/test/testdrive/github-5668.td +++ b/test/testdrive/github-5668.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # For Debezium versions <1.5, Materialize had no way of determining the complete order -# of messages coming from a Postgres source. (Details in materialize#5668.) We fixed this upstream +# of messages coming from a Postgres source. (Details in database-issues#1744.) We fixed this upstream # in Debezium (details in DBZ-2911) by adding a new "sequence" field to the source # metadata. This test uses that new sequence field to show that we can now handle messages # that seem to go backwards in time (by LSN). diff --git a/test/testdrive/github-5774.td b/test/testdrive/github-5774.td index c4677fa5c9d86..33856913bf419 100644 --- a/test/testdrive/github-5774.td +++ b/test/testdrive/github-5774.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # -# Regression test for https://github.com/MaterializeInc/materialize/issues/5774 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/1780 # # wrong result with inner join # diff --git a/test/testdrive/github-5983.td b/test/testdrive/github-5983.td index 136af484968da..cb34620b7eb18 100644 --- a/test/testdrive/github-5983.td +++ b/test/testdrive/github-5983.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # -# Regression test for https://github.com/MaterializeInc/materialize/issues/5983 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/1845 # # panic with self-join # diff --git a/test/testdrive/github-5984.td b/test/testdrive/github-5984.td index 9c8bcaa18eda8..14dad3dc5763b 100644 --- a/test/testdrive/github-5984.td +++ b/test/testdrive/github-5984.td @@ -8,7 +8,7 @@ # by the Apache License, Version 2.0. # -# Regression test for https://github.com/MaterializeInc/materialize/issues/5984 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/1846 # # wrong result with ORDER BY and NULLs in scalar subquery # diff --git a/test/testdrive/github-6305.td b/test/testdrive/github-6305.td index 2c5c695616ba7..acbe77b96295d 100644 --- a/test/testdrive/github-6305.td +++ b/test/testdrive/github-6305.td @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/6305 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/1947 > CREATE TABLE t1 (i1 INT8); diff --git a/test/testdrive/github-6950.td b/test/testdrive/github-6950.td index 670507a6be748..8141fa66f4960 100644 --- a/test/testdrive/github-6950.td +++ b/test/testdrive/github-6950.td @@ -7,7 +7,7 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -# Regression test for https://github.com/MaterializeInc/materialize/issues/6950 +# Regression test for https://api.github.com/repos/MaterializeInc/database-issues/issues/2164 > CREATE SCHEMA some_schema diff --git a/test/testdrive/github-7191.td b/test/testdrive/github-7191.td index 741a16f23170d..134ac200e0b5b 100644 --- a/test/testdrive/github-7191.td +++ b/test/testdrive/github-7191.td @@ -9,7 +9,7 @@ # The presence of this query in a view tests that the transformation of # "pow" -> "power" is correctly planned on reboot when testdrive is run with -# --validate-data-dir. This protects against regression of materialize#7191. +# --validate-data-dir. This protects against regression of database-issues#2251. > CREATE VIEW v AS SELECT pow(2, 4) > SELECT * FROM v 16 diff --git a/test/testdrive/index-source-stuck.td b/test/testdrive/index-source-stuck.td index 33133cb63d113..5e96448d7a9d3 100644 --- a/test/testdrive/index-source-stuck.td +++ b/test/testdrive/index-source-stuck.td @@ -15,7 +15,7 @@ > CREATE MATERIALIZED VIEW mv IN CLUSTER cluster2 AS SELECT * FROM src; > SET cluster = cluster2; -# Prevent us from getting stuck, see materialize#28328 +# Prevent us from getting stuck, see database-issues#8300 > SELECT min(counter) FROM src 1 > SELECT min(counter) FROM mv diff --git a/test/testdrive/kafka-avro-sinks.td b/test/testdrive/kafka-avro-sinks.td index 3d29635dfcb2a..e9e2716db1f26 100644 --- a/test/testdrive/kafka-avro-sinks.td +++ b/test/testdrive/kafka-avro-sinks.td @@ -51,7 +51,7 @@ $ kafka-verify-data format=avro sink=materialize.public.interval_data_sink sort- {"before": null, "after": {"row": {"interval": [1, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0]}}} {"before": null, "after": {"row": {"interval": [255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255]}}} -# See materialize#9723 +# See database-issues#2957 #> CREATE MATERIALIZED VIEW unnamed_cols AS SELECT 1, 2 AS b, 3; # #> CREATE SINK unnamed_cols_sink @@ -65,7 +65,7 @@ $ kafka-verify-data format=avro sink=materialize.public.interval_data_sink sort- # Test that invented field names do not clash with named columns. -# See materialize#9723 +# See database-issues#2957 #> CREATE MATERIALIZED VIEW clashing_cols AS SELECT 1, 2 AS column1, 3 as b, 4 as b2, 5 as b3; # #> CREATE SINK clashing_cols_sink @@ -121,7 +121,7 @@ $ kafka-verify-data format=avro sink=materialize.public.time_data_sink sort-mess > CREATE MATERIALIZED VIEW json_data (a, b) AS VALUES ('{"a":1, "b":2}'::jsonb, 2) -# Sinks with JSON columns should not crash - see https://github.com/MaterializeInc/materialize/issues/4722 +# Sinks with JSON columns should not crash - see https://api.github.com/repos/MaterializeInc/database-issues/issues/1477 > CREATE CLUSTER json_data_sink_cluster SIZE '${arg.default-storage-size}'; > CREATE SINK json_data_sink IN CLUSTER json_data_sink_cluster diff --git a/test/testdrive/kafka-include-key-sources.td b/test/testdrive/kafka-include-key-sources.td index f62ef2c2ac7c7..6c5a001ab6f9f 100644 --- a/test/testdrive/kafka-include-key-sources.td +++ b/test/testdrive/kafka-include-key-sources.td @@ -333,7 +333,7 @@ key id1 id2 measurement (1,2) 1 2 10 # -# Regression test for materialize#20981 +# Regression test for database-issues#6319 # # For UPSERT sources with INCLUDE KEY, we expect the queries to diff --git a/test/testdrive/kafka-json-sinks.td b/test/testdrive/kafka-json-sinks.td index 01f5ba6238f74..453b4fc7e8497 100644 --- a/test/testdrive/kafka-json-sinks.td +++ b/test/testdrive/kafka-json-sinks.td @@ -165,7 +165,7 @@ contains:column "a" specified more than once > CREATE TYPE int4_map_map AS MAP (KEY TYPE = text, VALUE TYPE = int4_map); -# We do this dance here to work around materialize#25282 +# We do this dance here to work around database-issues#7544 $ set-from-sql var=int4_id SELECT id FROM mz_objects WHERE name = '_int4'; $ set-from-sql var=text_id diff --git a/test/testdrive/materialized-view-refresh-options.td b/test/testdrive/materialized-view-refresh-options.td index aeca811da3a90..9aa7c0a1fb1f8 100644 --- a/test/testdrive/materialized-view-refresh-options.td +++ b/test/testdrive/materialized-view-refresh-options.td @@ -269,7 +269,7 @@ ALTER SYSTEM SET enable_unstable_dependencies = true 3 # Test that MVs that advance to the empty frontier do not retain read holds on -# their inputs. Regression test for materialize#24469. +# their inputs. Regression test for database-issues#7308. > CREATE TABLE t4 (x int) > CREATE MATERIALIZED VIEW mv7 WITH (REFRESH AT CREATION) AS SELECT * FROM t4 @@ -623,7 +623,7 @@ true WHERE mv.id = mvr.materialized_view_id AND name = 'mv1'; 8000 -# Negative test for `mz_hydration_statuses`, a regression test for materialize#25518. +# Negative test for `mz_hydration_statuses`, a regression test for database-issues#7621. # # It's hard to observe the bug before the first hydration, so we make the first # hydration quick to just get it over with, and then we make the next hydration diff --git a/test/testdrive/negative-multiplicities.td b/test/testdrive/negative-multiplicities.td index d08a12878567d..162b65aa4381e 100644 --- a/test/testdrive/negative-multiplicities.td +++ b/test/testdrive/negative-multiplicities.td @@ -33,7 +33,7 @@ contains:Invalid data in source, saw retractions (1) for row that does not exist ! SELECT * FROM data contains:Invalid data in source, saw retractions (2) for row that does not exist: [Int64(1), UInt32(1)] -# regression scenario per materialize#17963, with non-monotonic rendering +# regression scenario per database-issues#5246, with non-monotonic rendering > CREATE VIEW topk AS SELECT grp.id, count(t.data) AS top_2_count, (SELECT COUNT(d.data) FROM data d WHERE d.data % 2 = grp.id) AS total_count @@ -46,19 +46,19 @@ contains:Invalid data in source, saw retractions (2) for row that does not exist ! SELECT * from topk; contains:Negative multiplicities in TopK -# regression scenario per materialize#17908 +# regression scenario per database-issues#5224 ! SELECT DISTINCT data FROM data; contains:Non-positive multiplicity in DistinctBy -# regression scenario per materialize#18211 +# regression scenario per database-issues#5359 ! SELECT list_agg(DISTINCT data)[1] FROM data; contains:Non-positive accumulation in ReduceInaccumulable DISTINCT -# regression scenario per materialize#18212 +# regression scenario per database-issues#5360 ! SELECT list_agg(data)[1] FROM data; contains:Non-positive accumulation in ReduceInaccumulable -# regression scenario per materialize#17509, with non-monotonic rendering +# regression scenario per database-issues#5086, with non-monotonic rendering > CREATE VIEW max_data AS SELECT MAX(data) FROM data; @@ -89,7 +89,7 @@ contains:on-positive accumulation # Window aggregations # These are currently commented out, because window functions currently don't check their inputs for negative -# multiplicities, see https://github.com/MaterializeInc/materialize/issues/29624 +# multiplicities, see https://api.github.com/repos/MaterializeInc/database-issues/issues/8568 #! SELECT list_agg(data) OVER ()[1] FROM data; #contains:Non-positive accumulation in ReduceInaccumulable diff --git a/test/testdrive/protobuf-basic.td b/test/testdrive/protobuf-basic.td index 222184cc8efb3..f12aa65798720 100644 --- a/test/testdrive/protobuf-basic.td +++ b/test/testdrive/protobuf-basic.td @@ -66,7 +66,7 @@ $ kafka-ingest topic=basic format=protobuf descriptor-file=basic.pb message=Basi {"bool": true, "int32": 1, "int64": 2, "sint32": -1, "sint64": -2, "sfixed32": -3, "sfixed64": -4, "uint32": 3, "uint64": 4, "fixed32": 5, "fixed64": 6, "float": 1.2, "double": 3.2, "bytes": "YWFh", "string": "bbb", "enum": "ENUM1", "message": {"bool": true, "int32": 1, "int64": 2, "sint32": -1, "sint64": -2, "sfixed32": -3, "sfixed64": -4, "uint32": 3, "uint64": 4, "fixed32": 5, "fixed64": 6, "float": 1.2, "double": 3.2, "bytes": "YWFh", "string": "bbb", "enum": "ENUM1"}} {} # Test the case where the nested message is explicitly set to all default -# values. This previously crashed Materialize (see materialize#8930). +# values. This previously crashed Materialize (see database-issues#2723). {"message": {}} > CREATE CONNECTION kafka_conn diff --git a/test/testdrive/quickstart.td b/test/testdrive/quickstart.td index 667f66809c0d7..dcf5f9b285bba 100644 --- a/test/testdrive/quickstart.td +++ b/test/testdrive/quickstart.td @@ -10,7 +10,7 @@ # This test verifies the Quickstart page works: https://materialize.com/docs/get-started/quickstart/ # Uses shared compute+storage cluster -# TODO: Reenable when materialize#29696 is fixed +# TODO: Reenable when database-issues#8581 is fixed $ skip-if SELECT true diff --git a/test/testdrive/rename.td b/test/testdrive/rename.td index d30d47c8ae7b4..5528e5c632434 100644 --- a/test/testdrive/rename.td +++ b/test/testdrive/rename.td @@ -829,7 +829,7 @@ renamed_j # Test that after renaming a materialized object it is possible to create # another object with the original name. This used to fail because the index -# on the original object is not renamed. See materialize#5096. +# on the original object is not renamed. See database-issues#1585. > CREATE TABLE t_orig () > ALTER TABLE t_orig RENAME TO t_dontcare > CREATE TABLE t_orig () diff --git a/test/testdrive/render-delta-join.td b/test/testdrive/render-delta-join.td index a8db0d1bc5acd..2665710493195 100644 --- a/test/testdrive/render-delta-join.td +++ b/test/testdrive/render-delta-join.td @@ -25,7 +25,7 @@ count > CREATE VIEW delta_join (t1_f1, t1_f2, t2_f1, t2_f2) AS SELECT * FROM t1, t2 WHERE t1.f1 = t2.f1; # This should cause a delta join to be executed. -# But this test is currently broken: https://github.com/MaterializeInc/materialize/issues/16478 +# But this test is currently broken: https://api.github.com/repos/MaterializeInc/database-issues/issues/4772 > SELECT COUNT(*) FROM delta_join; count ---- diff --git a/test/testdrive/replica-targeting.td b/test/testdrive/replica-targeting.td index aa1f65fc41f60..694b4f143198d 100644 --- a/test/testdrive/replica-targeting.td +++ b/test/testdrive/replica-targeting.td @@ -75,7 +75,7 @@ $ set-regex match=\d{13} replacement= > COMMIT # Test that replica-targeted subscribes work when the subscribed collection -# advances to the empty frontier. Regression test for materialize#24981. +# advances to the empty frontier. Regression test for database-issues#8711. > DROP CLUSTER test CASCADE > CREATE CLUSTER test SIZE '4-4', REPLICATION FACTOR 4 diff --git a/test/testdrive/source-tables.td b/test/testdrive/source-tables.td index ad59f0aab6e3a..df8ba9b7d2615 100644 --- a/test/testdrive/source-tables.td +++ b/test/testdrive/source-tables.td @@ -10,7 +10,7 @@ $ set-arg-default default-replica-size=1 $ set-arg-default single-replica-cluster=quickstart -# TODO(def-) Remove when materialize#29397 and materialize#29435 are fixed +# TODO(def-) Remove when materialize#29397 and database-issues#8526 are fixed $ skip-if SELECT true @@ -280,7 +280,7 @@ var1 4444 12 ! SELECT * FROM mysql_table_3; contains:unknown catalog item 'mysql_table_3' -# TODO(def-) Remove when materialize#29397 and materialize#29435 are fixed +# TODO(def-) Remove when materialize#29397 and database-issues#8526 are fixed $ skip-end # diff --git a/test/testdrive/subexpression-replacement.td b/test/testdrive/subexpression-replacement.td index 84772f56f56df..3bb6692ddc64b 100644 --- a/test/testdrive/subexpression-replacement.td +++ b/test/testdrive/subexpression-replacement.td @@ -190,7 +190,7 @@ Used Indexes: Target cluster: quickstart -# TODO (https://github.com/MaterializeInc/materialize/issues/6262): Avoid simplifying mz_sleep. +# TODO (https://api.github.com/repos/MaterializeInc/database-issues/issues/1929): Avoid simplifying mz_sleep. ? EXPLAIN WITH(no notices) SELECT * FROM t1 WHERE mz_unsafe.mz_sleep(col_not_null) > mz_unsafe.mz_sleep(col_not_null) diff --git a/test/testdrive/subquery-scalar-errors.td b/test/testdrive/subquery-scalar-errors.td index b33ba8450b0e6..33ab89ae4c22f 100644 --- a/test/testdrive/subquery-scalar-errors.td +++ b/test/testdrive/subquery-scalar-errors.td @@ -13,7 +13,7 @@ # # Some of the queries that fail in Materialize will succeed in Postgresql # -# See https://github.com/MaterializeInc/materialize/issues/5670 for a discussion on the remaining +# See https://api.github.com/repos/MaterializeInc/database-issues/issues/1745 for a discussion on the remaining # differences between the two databases # diff --git a/test/testdrive/temporary.td b/test/testdrive/temporary.td index 4c02e54c63476..7dee6b8b441d4 100644 --- a/test/testdrive/temporary.td +++ b/test/testdrive/temporary.td @@ -107,7 +107,7 @@ contains:table "mz_temp.temp_t" already exists > SHOW INDEXES ON temp_t -# Blocked on https://github.com/MaterializeInc/materialize/issues/3105. +# Blocked on https://api.github.com/repos/MaterializeInc/database-issues/issues/1017. # # > CREATE TEMPORARY DEFAULT INDEX ON temp_t # diff --git a/test/testdrive/timestamps-debezium-kafka.td b/test/testdrive/timestamps-debezium-kafka.td index df780dcb4b6ab..eaa55afdac70d 100644 --- a/test/testdrive/timestamps-debezium-kafka.td +++ b/test/testdrive/timestamps-debezium-kafka.td @@ -122,7 +122,7 @@ $ kafka-ingest format=avro topic=bar schema=${schema} # Verify that we don't see any data yet. We can't use `set-sql-timeout` here # because the SELECT is blocked in the coordinator. For the same reason we -# can't use `DECLARE c CURSOR FOR SELECT ...` (see materialize#10763). +# can't use `DECLARE c CURSOR FOR SELECT ...` (see database-issues#3168). > BEGIN diff --git a/test/testdrive/top-k-monotonic.td b/test/testdrive/top-k-monotonic.td index 76dcce256b548..e97a53de9c510 100644 --- a/test/testdrive/top-k-monotonic.td +++ b/test/testdrive/top-k-monotonic.td @@ -164,7 +164,7 @@ contains:division by zero > DROP VIEW v_monotonic; -# The following tests repeat the scenario in materialize#18445. +# The following tests repeat the scenario in database-issues#5442. $ set other-non-dbz-schema={ "type": "record", "name": "cpx", diff --git a/test/testdrive/transactions-timedomain-nonmaterialized.td b/test/testdrive/transactions-timedomain-nonmaterialized.td index 7be53258b093a..10dd63ccc9fca 100644 --- a/test/testdrive/transactions-timedomain-nonmaterialized.td +++ b/test/testdrive/transactions-timedomain-nonmaterialized.td @@ -118,7 +118,7 @@ $ kafka-ingest topic=static format=bytes > COMMIT -# Regression for materialize#8657 +# Regression for database-issues#2647 # Ensure that views referencing other schemas are transitively included. Here, # pg_catalog is generally a view over mz_catalog. > BEGIN @@ -126,7 +126,7 @@ $ kafka-ingest topic=static format=bytes > SELECT pg_catalog.format_type(a.atttypid, a.atttypmod) FROM pg_catalog.pg_attribute a LIMIT 0; > COMMIT -# Regression for materialize#8942 +# Regression for database-issues#2727 # Ensure that non-materialized, transitive views are not included. Here, # unindexed should not be included in the timedomain. > CREATE MATERIALIZED VIEW v_materialized AS SELECT count(*) FROM unindexed diff --git a/test/testdrive/types.td b/test/testdrive/types.td index c27fda2e84f33..76c09ed79be46 100644 --- a/test/testdrive/types.td +++ b/test/testdrive/types.td @@ -229,7 +229,7 @@ other_record_c "" > CREATE TABLE varchar_array_list_t (a varchar_array_list_c); > INSERT INTO varchar_array_list_t VALUES (LIST[ARRAY['a'::varchar,'b'::varchar]]); -# Enable this when closing materialize#7613 +# Enable this when closing database-issues#2360 # > CREATE TYPE char_array_list_c AS LIST (ELEMENT TYPE = _char); # > CREATE TABLE char_array_list_t (a char_array_list_c); # > INSERT INTO char_array_list_t VALUES (LIST[ARRAY['a'::char,'b'::char]]); diff --git a/test/zippy/mzcompose.py b/test/zippy/mzcompose.py index 7e9a65952b8b5..e839ef12d3d28 100644 --- a/test/zippy/mzcompose.py +++ b/test/zippy/mzcompose.py @@ -152,7 +152,7 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: with c.override( Cockroach( image=f"cockroachdb/cockroach:{args.cockroach_tag}", - # Workaround for materialize#19276 + # Workaround for database-issues#5719 restart="on-failure:5", setup_materialize=True, ),