diff --git a/pkg/ccl/logictestccl/testdata/logic_test/triggers b/pkg/ccl/logictestccl/testdata/logic_test/triggers index 81cf63edf438..45ef1031ebf2 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/triggers +++ b/pkg/ccl/logictestccl/testdata/logic_test/triggers @@ -2230,24 +2230,28 @@ statement ok INSERT INTO parent VALUES (1), (2), (3); INSERT INTO child VALUES (1, 1), (2, 2), (3, 2), (4, 3); -# TODO(#132971): We should fire the BEFORE trigger on child. query T noticetrace UPDATE parent SET k = k * 10 WHERE k = 2 OR k = 3; ---- NOTICE: BEFORE UPDATE ON parent: (2) -> (20) NOTICE: BEFORE UPDATE ON parent: (3) -> (30) +NOTICE: BEFORE UPDATE ON child: (2,2) -> (2,20) +NOTICE: BEFORE UPDATE ON child: (3,2) -> (3,20) +NOTICE: BEFORE UPDATE ON child: (4,3) -> (4,30) NOTICE: AFTER UPDATE ON parent: (2) -> (20) NOTICE: AFTER UPDATE ON parent: (3) -> (30) NOTICE: AFTER UPDATE ON child: (2,2) -> (2,20) NOTICE: AFTER UPDATE ON child: (3,2) -> (3,20) NOTICE: AFTER UPDATE ON child: (4,3) -> (4,30) -# TODO(#132971): We should fire the BEFORE trigger on child. query T noticetrace DELETE FROM parent WHERE k = 20 OR k = 30; ---- NOTICE: BEFORE DELETE ON parent: (20) -> NOTICE: BEFORE DELETE ON parent: (30) -> +NOTICE: BEFORE DELETE ON child: (2,20) -> +NOTICE: BEFORE DELETE ON child: (3,20) -> +NOTICE: BEFORE DELETE ON child: (4,30) -> NOTICE: AFTER DELETE ON parent: (20) -> NOTICE: AFTER DELETE ON parent: (30) -> NOTICE: AFTER DELETE ON child: (2,20) -> @@ -2286,31 +2290,35 @@ $$; statement ok CREATE TRIGGER foo BEFORE INSERT ON ab FOR EACH ROW EXECUTE FUNCTION h(); -# TODO(#132971): We should fire the BEFORE trigger on child. query T noticetrace INSERT INTO ab VALUES (2, 20), (3, 30); ---- NOTICE: update parent.k to 20 in trigger where k = 2 NOTICE: BEFORE UPDATE ON parent: (2) -> (20) +NOTICE: BEFORE UPDATE ON child: (2,2) -> (2,20) +NOTICE: BEFORE UPDATE ON child: (3,2) -> (3,20) NOTICE: AFTER UPDATE ON parent: (2) -> (20) NOTICE: AFTER UPDATE ON child: (2,2) -> (2,20) NOTICE: AFTER UPDATE ON child: (3,2) -> (3,20) NOTICE: update parent.k to 30 in trigger where k = 3 NOTICE: BEFORE UPDATE ON parent: (3) -> (30) +NOTICE: BEFORE UPDATE ON child: (4,3) -> (4,30) NOTICE: AFTER UPDATE ON parent: (3) -> (30) NOTICE: AFTER UPDATE ON child: (4,3) -> (4,30) -# TODO(#132971): We should fire the BEFORE trigger on child. query T noticetrace INSERT INTO ab VALUES (20, NULL), (30, NULL); ---- NOTICE: delete from parent in trigger where k = 20 NOTICE: BEFORE DELETE ON parent: (20) -> +NOTICE: BEFORE DELETE ON child: (2,20) -> +NOTICE: BEFORE DELETE ON child: (3,20) -> NOTICE: AFTER DELETE ON parent: (20) -> NOTICE: AFTER DELETE ON child: (2,20) -> NOTICE: AFTER DELETE ON child: (3,20) -> NOTICE: delete from parent in trigger where k = 30 NOTICE: BEFORE DELETE ON parent: (30) -> +NOTICE: BEFORE DELETE ON child: (4,30) -> NOTICE: AFTER DELETE ON parent: (30) -> NOTICE: AFTER DELETE ON child: (4,30) -> @@ -2330,31 +2338,35 @@ subtest after_trigger_fires_cascades statement ok CREATE TRIGGER foo AFTER INSERT OR DELETE ON ab FOR EACH ROW EXECUTE FUNCTION h(); -# TODO(#132971): We should fire the BEFORE trigger on child. query T noticetrace INSERT INTO ab VALUES (2, 20), (3, 30); ---- NOTICE: update parent.k to 20 in trigger where k = 2 NOTICE: BEFORE UPDATE ON parent: (2) -> (20) +NOTICE: BEFORE UPDATE ON child: (2,2) -> (2,20) +NOTICE: BEFORE UPDATE ON child: (3,2) -> (3,20) NOTICE: AFTER UPDATE ON parent: (2) -> (20) NOTICE: AFTER UPDATE ON child: (2,2) -> (2,20) NOTICE: AFTER UPDATE ON child: (3,2) -> (3,20) NOTICE: update parent.k to 30 in trigger where k = 3 NOTICE: BEFORE UPDATE ON parent: (3) -> (30) +NOTICE: BEFORE UPDATE ON child: (4,3) -> (4,30) NOTICE: AFTER UPDATE ON parent: (3) -> (30) NOTICE: AFTER UPDATE ON child: (4,3) -> (4,30) -# TODO(#132971): We should fire the BEFORE trigger on child. query T noticetrace INSERT INTO ab VALUES (20, NULL), (30, NULL); ---- NOTICE: delete from parent in trigger where k = 20 NOTICE: BEFORE DELETE ON parent: (20) -> +NOTICE: BEFORE DELETE ON child: (2,20) -> +NOTICE: BEFORE DELETE ON child: (3,20) -> NOTICE: AFTER DELETE ON parent: (20) -> NOTICE: AFTER DELETE ON child: (2,20) -> NOTICE: AFTER DELETE ON child: (3,20) -> NOTICE: delete from parent in trigger where k = 30 NOTICE: BEFORE DELETE ON parent: (30) -> +NOTICE: BEFORE DELETE ON child: (4,30) -> NOTICE: AFTER DELETE ON parent: (30) -> NOTICE: AFTER DELETE ON child: (4,30) -> @@ -2415,6 +2427,8 @@ query T noticetrace CALL wrap_parent_update(2, NULL); ---- NOTICE: BEFORE DELETE ON parent: (2) -> +NOTICE: BEFORE DELETE ON child: (2,2) -> +NOTICE: BEFORE DELETE ON child: (3,2) -> NOTICE: FK violation updating parent from 2 to # Try updating a referenced row in the parent table. @@ -2422,6 +2436,8 @@ query T noticetrace CALL wrap_parent_update(2, 10); ---- NOTICE: BEFORE UPDATE ON parent: (2) -> (10) +NOTICE: BEFORE UPDATE ON child: (2,2) -> (2,10) +NOTICE: BEFORE UPDATE ON child: (3,2) -> (3,10) NOTICE: FK violation updating parent from 2 to 10 # Try inserting a row with no reference into the child2 table. @@ -2434,10 +2450,248 @@ NOTICE: FK violation inserting (10, 10) into child2 statement ok DROP PROCEDURE wrap_parent_update; DROP PROCEDURE wrap_child2_insert; +DROP TABLE child2; + +# Unless "unsafe_allow_triggers_modifying_cascades" is set, triggers are not +# allowed to modify or filter rows the mutation for a cascade. +subtest triggers_modify_fk_cascades + +statement ok +CREATE FUNCTION h() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + BEGIN + NEW.k := (NEW).k + 100; + OLD.k := (OLD).k + 100; + RETURN COALESCE(NEW, OLD); + END +$$; + +statement ok +CREATE FUNCTION filter() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ + BEGIN + RETURN NULL; + END +$$; + +statement ok +DELETE FROM child WHERE True; +DELETE FROM parent WHERE True; +INSERT INTO parent VALUES (1), (2), (3); +INSERT INTO child VALUES (1, 1), (2, 2), (3, 2), (4, 3); + +statement ok +CREATE TRIGGER mod BEFORE INSERT OR UPDATE OR DELETE ON child FOR EACH ROW EXECUTE FUNCTION h(); + +statement error pgcode 27000 pq: trigger mod attempted to modify or filter a row in a cascade operation: \(1,11\) +UPDATE parent SET k = k + 10 WHERE k < 3; + +statement error pgcode 27000 pq: trigger mod attempted to modify or filter a row in a cascade operation: \(1,1\) +DELETE FROM parent WHERE k < 3; + +statement ok +SET unsafe_allow_triggers_modifying_cascades = true; + +statement ok +UPDATE parent SET k = k + 10 WHERE k < 3; + +statement ok +DELETE FROM parent WHERE k < 3; + +statement ok +RESET unsafe_allow_triggers_modifying_cascades; + +statement ok +DROP TRIGGER mod ON child; + +statement ok +DELETE FROM child WHERE True; +DELETE FROM parent WHERE True; +INSERT INTO parent VALUES (1), (2), (3); +INSERT INTO child VALUES (1, 1), (2, 2), (3, 2), (4, 3); + +statement ok +CREATE TRIGGER filter BEFORE INSERT OR UPDATE OR DELETE ON child FOR EACH ROW EXECUTE FUNCTION filter(); + +statement error pgcode 27000 pq: trigger filter attempted to modify or filter a row in a cascade operation: \(1,11\) +UPDATE parent SET k = k + 10 WHERE k < 3; + +statement error pgcode 27000 pq: trigger filter attempted to modify or filter a row in a cascade operation: \(1,1\) +DELETE FROM parent WHERE k < 3; + +statement ok +SET unsafe_allow_triggers_modifying_cascades = true; + +statement ok +UPDATE parent SET k = k + 10 WHERE k < 3; + +statement ok +DELETE FROM parent WHERE k < 3; + +statement ok +RESET unsafe_allow_triggers_modifying_cascades; + +statement ok +DROP TRIGGER filter ON child; + +statement ok +DELETE FROM child WHERE True; +DELETE FROM parent WHERE True; +INSERT INTO parent VALUES (1), (2), (3); +INSERT INTO child VALUES (1, 1), (2, 2), (3, 2), (4, 3); + +# Modifications to mutated rows made by BEFORE triggers are visible to cascades. +subtest before_trigger_modifies_fk_cascades + +statement ok +CREATE TRIGGER mod BEFORE INSERT OR UPDATE OR DELETE ON parent FOR EACH ROW EXECUTE FUNCTION h(); + +query T noticetrace +UPDATE parent SET k = k + 10 WHERE k < 3; +---- +NOTICE: BEFORE UPDATE ON parent: (1) -> (11) +NOTICE: BEFORE UPDATE ON parent: (2) -> (12) +NOTICE: BEFORE UPDATE ON child: (1,1) -> (1,111) +NOTICE: BEFORE UPDATE ON child: (2,2) -> (2,112) +NOTICE: BEFORE UPDATE ON child: (3,2) -> (3,112) +NOTICE: AFTER UPDATE ON parent: (1) -> (111) +NOTICE: AFTER UPDATE ON parent: (2) -> (112) +NOTICE: AFTER UPDATE ON child: (1,1) -> (1,111) +NOTICE: AFTER UPDATE ON child: (2,2) -> (2,112) +NOTICE: AFTER UPDATE ON child: (3,2) -> (3,112) + +query II rowsort +SELECT * FROM child; +---- +1 111 +2 112 +3 112 +4 3 + +query T noticetrace +DELETE FROM parent WHERE k > 3; +---- +NOTICE: BEFORE DELETE ON parent: (111) -> +NOTICE: BEFORE DELETE ON parent: (112) -> +NOTICE: BEFORE DELETE ON child: (1,111) -> +NOTICE: BEFORE DELETE ON child: (2,112) -> +NOTICE: BEFORE DELETE ON child: (3,112) -> +NOTICE: AFTER DELETE ON parent: (111) -> +NOTICE: AFTER DELETE ON parent: (112) -> +NOTICE: AFTER DELETE ON child: (1,111) -> +NOTICE: AFTER DELETE ON child: (2,112) -> +NOTICE: AFTER DELETE ON child: (3,112) -> + +query II rowsort +SELECT * FROM child; +---- +4 3 + +statement ok +DROP TRIGGER mod ON parent; + +subtest cascade_diamond + +# Create a diamond cascade structure. +statement ok +DROP TABLE child; +DELETE FROM parent WHERE True; + +statement ok +CREATE TABLE child (k INT PRIMARY KEY, v INT UNIQUE NOT NULL REFERENCES parent(k) ON UPDATE CASCADE ON DELETE CASCADE); +CREATE TABLE child2 (k INT PRIMARY KEY, v INT UNIQUE NOT NULL REFERENCES parent(k) ON UPDATE CASCADE ON DELETE CASCADE); + +statement ok +CREATE TRIGGER foo BEFORE INSERT OR UPDATE OR DELETE ON child FOR EACH ROW EXECUTE FUNCTION g(); + +statement ok +CREATE TRIGGER bar AFTER INSERT OR UPDATE OR DELETE ON child FOR EACH ROW EXECUTE FUNCTION g(); + +statement ok +CREATE TRIGGER foo BEFORE INSERT OR UPDATE OR DELETE ON child2 FOR EACH ROW EXECUTE FUNCTION g(); + +statement ok +CREATE TRIGGER bar AFTER INSERT OR UPDATE OR DELETE ON child2 FOR EACH ROW EXECUTE FUNCTION g(); + +statement ok +CREATE TABLE grandchild ( + k INT PRIMARY KEY, + v INT REFERENCES child(v) ON UPDATE CASCADE ON DELETE CASCADE, + v2 INT REFERENCES child2(v) ON UPDATE CASCADE ON DELETE CASCADE +); + +statement ok +CREATE TRIGGER foo BEFORE INSERT OR UPDATE OR DELETE ON grandchild FOR EACH ROW EXECUTE FUNCTION g(); + +statement ok +CREATE TRIGGER bar AFTER INSERT OR UPDATE OR DELETE ON grandchild FOR EACH ROW EXECUTE FUNCTION g(); + +statement ok +INSERT INTO parent VALUES (1), (2), (3); +INSERT INTO child VALUES (1, 1), (2, 2), (3, 3); +INSERT INTO child2 VALUES (1, 1), (2, 2), (3, 3); +INSERT INTO grandchild VALUES (1, 1, 1), (2, 2, 2), (3, 2, 2), (4, 3, 3); + +# Update the parent table, which should cascade to the children and grandchild. +# Note that both child tables cascade to the grandchild. +# +# Regression test for #133784 and #133792. +query T noticetrace +UPDATE parent SET k = k + 10 WHERE k < 3; +---- +NOTICE: BEFORE UPDATE ON parent: (1) -> (11) +NOTICE: BEFORE UPDATE ON parent: (2) -> (12) +NOTICE: BEFORE UPDATE ON child: (1,1) -> (1,11) +NOTICE: BEFORE UPDATE ON child: (2,2) -> (2,12) +NOTICE: BEFORE UPDATE ON child2: (1,1) -> (1,11) +NOTICE: BEFORE UPDATE ON child2: (2,2) -> (2,12) +NOTICE: AFTER UPDATE ON parent: (1) -> (11) +NOTICE: AFTER UPDATE ON parent: (2) -> (12) +NOTICE: AFTER UPDATE ON child: (1,1) -> (1,11) +NOTICE: AFTER UPDATE ON child: (2,2) -> (2,12) +NOTICE: AFTER UPDATE ON child2: (1,1) -> (1,11) +NOTICE: AFTER UPDATE ON child2: (2,2) -> (2,12) +NOTICE: BEFORE UPDATE ON grandchild: (1,1,1) -> (1,11,1) +NOTICE: BEFORE UPDATE ON grandchild: (2,2,2) -> (2,12,2) +NOTICE: BEFORE UPDATE ON grandchild: (3,2,2) -> (3,12,2) +NOTICE: BEFORE UPDATE ON grandchild: (1,11,1) -> (1,11,11) +NOTICE: BEFORE UPDATE ON grandchild: (2,12,2) -> (2,12,12) +NOTICE: BEFORE UPDATE ON grandchild: (3,12,2) -> (3,12,12) +NOTICE: AFTER UPDATE ON grandchild: (1,1,1) -> (1,11,1) +NOTICE: AFTER UPDATE ON grandchild: (2,2,2) -> (2,12,2) +NOTICE: AFTER UPDATE ON grandchild: (3,2,2) -> (3,12,2) +NOTICE: AFTER UPDATE ON grandchild: (1,11,1) -> (1,11,11) +NOTICE: AFTER UPDATE ON grandchild: (2,12,2) -> (2,12,12) +NOTICE: AFTER UPDATE ON grandchild: (3,12,2) -> (3,12,12) + +query II rowsort +SELECT * FROM child; +---- +1 11 +2 12 +3 3 + +query II rowsort +SELECT * FROM child2; +---- +1 11 +2 12 +3 3 + +query III rowsort +SELECT * FROM grandchild; +---- +1 11 11 +2 12 12 +3 12 12 +4 3 3 + +statement ok +DROP TABLE grandchild; DROP TABLE child; DROP TABLE child2; DROP TABLE parent; DROP FUNCTION g; +DROP FUNCTION h; +DROP FUNCTION filter; # ============================================================================== # Test order of execution. diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel index 2546431d325d..d591d17bd376 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel @@ -14,6 +14,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/mixedversion", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/cmd/roachprod/grafana", "//pkg/cmd/roachtest/cluster", "//pkg/cmd/roachtest/option", @@ -54,6 +55,7 @@ go_test( embed = [":mixedversion"], embedsrcs = ["testdata/test_releases.yaml"], deps = [ + "//pkg/clusterversion", "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/registry", "//pkg/cmd/roachtest/roachtestutil", diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index 849bf842263a..aa4ab0a46382 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -76,6 +76,7 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" @@ -481,10 +482,32 @@ func WithTag(tag string) CustomOption { } } -// minSupportedSkipVersionUpgrade is the minimum version after which -// "skip version" upgrades are supported (i.e., upgrading two major -// releases in a single upgrade). -var minSupportedSkipVersionUpgrade = clusterupgrade.MustParseVersion("v24.1.0") +// supportsSkipUpgradeTo returns true if the given version supports skipping the +// previous major version during upgrade. For example, 24.3 supports upgrade +// directly from 24.1, but 25.1 only supports upgrade from 24.3. +func supportsSkipUpgradeTo(v *clusterupgrade.Version) bool { + major, minor := v.Version.Major(), v.Version.Minor() + + // Special case for the current release series. This is useful to keep the + // test correct when we bump the minimum supported version separately from + // the current version. + if r := clusterversion.Latest.ReleaseSeries(); int(r.Major) == major && int(r.Minor) == minor { + return len(clusterversion.SupportedPreviousReleases()) > 1 + } + + switch { + case major < 24: + return false + case major == 24: + // v24.2 supported skip upgrades experimentally. + // v24.3 is the first version which officially supports the skip upgrade. + return minor == 2 || minor == 3 + default: + // The current plan for 2025+ is for .1 and .3 to be skippable innovation + // releases. + return minor == 2 || minor == 4 + } +} func defaultTestOptions() testOptions { return testOptions{ @@ -835,11 +858,9 @@ func (t *Test) choosePreviousReleases() ([]*clusterupgrade.Version, error) { return nil, nil } - // If skip-version upgrades are not enabled, the only possible - // predecessor is the immediate predecessor release. If the - // predecessor doesn't support skip versions, then its predecessor - // won't either. Don't attempt to find it. - if !skipVersions || !pred.AtLeast(minSupportedSkipVersionUpgrade) { + // If skip-version upgrades are not enabled or v does not support them, the + // only possible predecessor is the immediate predecessor release. + if !skipVersions || !supportsSkipUpgradeTo(v) { return []*clusterupgrade.Version{pred}, nil } @@ -848,26 +869,18 @@ func (t *Test) choosePreviousReleases() ([]*clusterupgrade.Version, error) { return nil, err } - if predPred.AtLeast(minSupportedSkipVersionUpgrade) { - // If the predecessor's predecessor supports skip-version - // upgrades and we haven't performed a skip-version upgrade yet, - // do it. This logic makes sure that, when skip-version upgrades - // are enabled, it happens when upgrading to the current - // release, which is the most important upgrade to be tested on - // any release branch. - if numSkips == 0 { - numSkips++ - return []*clusterupgrade.Version{predPred}, nil - } - - // If we already performed a skip-version upgrade on this test - // plan, we can choose to do another one or not. - return []*clusterupgrade.Version{pred, predPred}, nil + // If we haven't performed a skip-version upgrade yet, do it. This logic + // makes sure that, when skip-version upgrades are enabled, it happens + // when upgrading to the current release, which is the most important + // upgrade to be tested on any release branch. + if numSkips == 0 { + numSkips++ + return []*clusterupgrade.Version{predPred}, nil } - // If the predecessor is too old and does not support skip-version - // upgrades, it's the only possible predecessor. - return []*clusterupgrade.Version{pred}, nil + // If we already performed a skip-version upgrade on this test + // plan, we can choose to do another one or not. + return []*clusterupgrade.Version{pred, predPred}, nil } currentVersion := clusterupgrade.CurrentVersion() diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go index 028de870e2fd..603bda0e31c7 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion_test.go @@ -11,6 +11,7 @@ import ( "math/rand" "testing" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade" @@ -368,3 +369,23 @@ func withTestBuildVersion(v string) func() { clusterupgrade.TestBuildVersion = testBuildVersion return func() { clusterupgrade.TestBuildVersion = nil } } + +func TestSupportsSkipUpgradeTo(t *testing.T) { + expect := func(verStr string, expected bool) { + t.Helper() + v := clusterupgrade.MustParseVersion(verStr) + if r := clusterversion.Latest.ReleaseSeries(); int(r.Major) == v.Major() && int(r.Minor) == v.Minor() { + // We have to special case the current series, to allow for bumping the + // min supported version separately from the current version. + expected = len(clusterversion.SupportedPreviousReleases()) > 1 + } + require.Equal(t, expected, supportsSkipUpgradeTo(v)) + } + for _, v := range []string{"v24.3.0", "v24.3.0-beta.1", "v25.2.1", "v25.2.0-rc.1"} { + expect(v, true) + } + + for _, v := range []string{"v25.1.0", "v25.1.0-beta.1", "v25.3.1", "v25.3.0-rc.1"} { + expect(v, false) + } +} diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index b7a63755de55..9341810d832f 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -2204,7 +2204,7 @@ func (dsp *DistSQLPlanner) PlanAndRunPostQueries( // TODO(yuzefovich): the planObserver logic in // planAndRunChecksInParallel will need to be adjusted when we switch to // using the DistSQL spec factory. - for i := cascadesIdx; i < len(plan.checkPlans); i++ { + for i := checksIdx; i < len(plan.checkPlans); i++ { if plan.checkPlans[i].plan.isPhysicalPlan() { runParallelChecks = false break @@ -2212,7 +2212,8 @@ func (dsp *DistSQLPlanner) PlanAndRunPostQueries( } } if runParallelChecks { - if err := dsp.planAndRunChecksInParallel(ctx, plan.checkPlans, planner, evalCtxFactory, recv); err != nil { + checksToRun := plan.checkPlans[checksIdx:] + if err := dsp.planAndRunChecksInParallel(ctx, checksToRun, planner, evalCtxFactory, recv); err != nil { recv.SetError(err) return false } @@ -2245,7 +2246,8 @@ func (dsp *DistSQLPlanner) PlanAndRunPostQueries( } // Finally, run triggers. - for ; triggersIdx < len(plan.triggers); triggersIdx++ { + numTriggers := len(plan.triggers) + for ; triggersIdx < numTriggers; triggersIdx++ { trigger := &plan.triggers[triggersIdx] hasBuffer, numBufferedRows := checkPostQueryBuffer(plan.triggers[triggersIdx]) if hasBuffer && numBufferedRows == 0 { diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index b1ab34b38265..6e93aaac016e 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -3892,6 +3892,10 @@ func (m *sessionDataMutator) SetBypassPCRReaderCatalogAOST(val bool) { m.data.BypassPCRReaderCatalogAOST = val } +func (m *sessionDataMutator) SetUnsafeAllowTriggersModifyingCascades(val bool) { + m.data.UnsafeAllowTriggersModifyingCascades = val +} + // Utility functions related to scrubbing sensitive information on SQL Stats. // quantizeCounts ensures that the Count field in the diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 3766846c7256..1832550d15ec 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -6394,6 +6394,7 @@ transaction_timeout 0 troubleshooting_mode off unbounded_parallel_scans off unconstrained_non_covering_index_scan_enabled off +unsafe_allow_triggers_modifying_cascades off variable_inequality_lookup_join_enabled on xmloption content diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index c1e43386fbaa..5d5112274d7b 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -3039,6 +3039,7 @@ transaction_timeout 0 N troubleshooting_mode off NULL NULL NULL string unbounded_parallel_scans off NULL NULL NULL string unconstrained_non_covering_index_scan_enabled off NULL NULL NULL string +unsafe_allow_triggers_modifying_cascades off NULL NULL NULL string use_declarative_schema_changer on NULL NULL NULL string variable_inequality_lookup_join_enabled on NULL NULL NULL string vectorize on NULL NULL NULL string @@ -3233,6 +3234,7 @@ transaction_timeout 0 N troubleshooting_mode off NULL user NULL off off unbounded_parallel_scans off NULL user NULL off off unconstrained_non_covering_index_scan_enabled off NULL user NULL off off +unsafe_allow_triggers_modifying_cascades off NULL user NULL off off use_declarative_schema_changer on NULL user NULL on on variable_inequality_lookup_join_enabled on NULL user NULL on on vectorize on NULL user NULL on on @@ -3427,6 +3429,7 @@ transaction_timeout NULL NULL NULL troubleshooting_mode NULL NULL NULL NULL NULL unbounded_parallel_scans NULL NULL NULL NULL NULL unconstrained_non_covering_index_scan_enabled NULL NULL NULL NULL NULL +unsafe_allow_triggers_modifying_cascades NULL NULL NULL NULL NULL use_declarative_schema_changer NULL NULL NULL NULL NULL variable_inequality_lookup_join_enabled NULL NULL NULL NULL NULL vectorize NULL NULL NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 22b2a45059ae..fba33c30c95a 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -205,6 +205,7 @@ transaction_timeout 0 troubleshooting_mode off unbounded_parallel_scans off unconstrained_non_covering_index_scan_enabled off +unsafe_allow_triggers_modifying_cascades off use_declarative_schema_changer on variable_inequality_lookup_join_enabled on vectorize on diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index d4f777f2bc8d..abb76e3553a0 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -197,6 +197,7 @@ type Memo struct { usePolymorphicParameterFix bool useConditionalHoistFix bool pushLimitIntoProjectFilteredScan bool + unsafeAllowTriggersModifyingCascades bool // txnIsoLevel is the isolation level under which the plan was created. This // affects the planning of some locking operations, so it must be included in @@ -285,6 +286,7 @@ func (m *Memo) Init(ctx context.Context, evalCtx *eval.Context) { usePolymorphicParameterFix: evalCtx.SessionData().OptimizerUsePolymorphicParameterFix, useConditionalHoistFix: evalCtx.SessionData().OptimizerUseConditionalHoistFix, pushLimitIntoProjectFilteredScan: evalCtx.SessionData().OptimizerPushLimitIntoProjectFilteredScan, + unsafeAllowTriggersModifyingCascades: evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades, txnIsoLevel: evalCtx.TxnIsoLevel, } m.metadata.Init() @@ -451,6 +453,7 @@ func (m *Memo) IsStale( m.usePolymorphicParameterFix != evalCtx.SessionData().OptimizerUsePolymorphicParameterFix || m.useConditionalHoistFix != evalCtx.SessionData().OptimizerUseConditionalHoistFix || m.pushLimitIntoProjectFilteredScan != evalCtx.SessionData().OptimizerPushLimitIntoProjectFilteredScan || + m.unsafeAllowTriggersModifyingCascades != evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades || m.txnIsoLevel != evalCtx.TxnIsoLevel { return true, nil } diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go index 4a4c0cd6c64c..10fc22645a74 100644 --- a/pkg/sql/opt/memo/memo_test.go +++ b/pkg/sql/opt/memo/memo_test.go @@ -521,6 +521,12 @@ func TestMemoIsStale(t *testing.T) { evalCtx.SessionData().OptimizerPushLimitIntoProjectFilteredScan = false notStale() + // Stale unsafe_allow_triggers_modifying_cascades. + evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades = true + stale() + evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades = false + notStale() + // User no longer has access to view. catalog.View(tree.NewTableNameWithSchema("t", catconstants.PublicSchemaName, "abcview")).Revoked = true _, err = o.Memo().IsStale(ctx, &evalCtx, catalog) diff --git a/pkg/sql/opt/optbuilder/delete.go b/pkg/sql/opt/optbuilder/delete.go index cc6dd4756592..3880461e6558 100644 --- a/pkg/sql/opt/optbuilder/delete.go +++ b/pkg/sql/opt/optbuilder/delete.go @@ -89,7 +89,7 @@ func (b *Builder) buildDelete(del *tree.Delete, inScope *scope) (outScope *scope mb.buildInputForDelete(inScope, del.Table, del.Where, del.Using, del.Limit, del.OrderBy) // Project row-level BEFORE triggers for DELETE. - mb.buildRowLevelBeforeTriggers(tree.TriggerEventDelete) + mb.buildRowLevelBeforeTriggers(tree.TriggerEventDelete, false /* cascade */) // Build the final delete statement, including any returned expressions. if resultsNeeded(del.Returning) { diff --git a/pkg/sql/opt/optbuilder/fk_cascade.go b/pkg/sql/opt/optbuilder/fk_cascade.go index 87d3006827b3..7a922a726f1a 100644 --- a/pkg/sql/opt/optbuilder/fk_cascade.go +++ b/pkg/sql/opt/optbuilder/fk_cascade.go @@ -112,6 +112,10 @@ func (cb *onDeleteCascadeBuilder) Build( // Set list of columns that will be fetched by the input expression. mb.setFetchColIDs(mb.outScope.cols) + + // Cascades can fire triggers on the child table. + mb.buildRowLevelBeforeTriggers(tree.TriggerEventDelete, true /* cascade */) + mb.buildDelete(nil /* returning */) return mb.outScope.expr }) @@ -354,6 +358,10 @@ func (cb *onDeleteFastCascadeBuilder) Build( // Set list of columns that will be fetched by the input expression. mb.setFetchColIDs(mb.outScope.cols) + + // Cascades can fire triggers on the child table. + mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert, true /* cascade */) + mb.buildDelete(nil /* returning */) return mb.outScope.expr }) @@ -487,6 +495,9 @@ func (cb *onDeleteSetBuilder) Build( } mb.addUpdateCols(updateExprs) + // Cascades can fire triggers on the child table. + mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate, true /* cascade */) + // TODO(radu): consider plumbing a flag to prevent building the FK check // against the parent we are cascading from. Need to investigate in which // cases this is safe (e.g. other cascades could have messed with the parent @@ -729,6 +740,9 @@ func (cb *onUpdateCascadeBuilder) Build( } mb.addUpdateCols(updateExprs) + // Cascades can fire triggers on the child table. + mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate, true /* cascade */) + mb.buildUpdate(nil /* returning */) return mb.outScope.expr }) diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index b55304cdc81a..d57f3908dce7 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -295,7 +295,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope // Case 1: Simple INSERT statement. case ins.OnConflict == nil: // Project row-level BEFORE triggers for INSERT. - mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert) + mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert, false /* cascade */) // Build the final insert statement, including any returned expressions. mb.buildInsert(returning) @@ -304,7 +304,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope case ins.OnConflict.DoNothing: // Project row-level BEFORE triggers for INSERT. - mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert) + mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert, false /* cascade */) // Wrap the input in one ANTI JOIN per UNIQUE index, and filter out rows // that have conflicts. See the buildInputForDoNothing comment for more @@ -333,7 +333,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope // NOTE: we avoid building INSERT triggers until after buildInputForUpsert // so that there are no buffering operators between INSERT and UPDATE // triggers. This helps preserve Postgres compatibility - if mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert) { + if mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert, false /* cascade */) { // INSERT triggers are able to modify the row being inserted, so we need // to recompute the upsert columns. mb.setUpsertCols(nil /* insertCols */) @@ -344,10 +344,10 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope mb.addSynthesizedColsForUpdate() // Project row-level BEFORE triggers for UPDATE. - mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate) + mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate, false /* cascade */) } else { // Project row-level BEFORE triggers for INSERT. - if mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert) { + if mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert, false /* cascade */) { // INSERT triggers are able to modify the row being inserted, so we need // to recompute the upsert columns. mb.setUpsertCols(nil /* insertCols */) @@ -364,7 +364,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope canaryCol := mb.buildInputForUpsert(inScope, ins.Table, ins.OnConflict) // Project row-level BEFORE triggers for INSERT. - mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert) + mb.buildRowLevelBeforeTriggers(tree.TriggerEventInsert, false /* cascade */) // Add a filter from the WHERE clause if one exists. This must happen after // the INSERT triggers are added, since BEFORE INSERT triggers are called @@ -380,7 +380,7 @@ func (b *Builder) buildInsert(ins *tree.Insert, inScope *scope) (outScope *scope mb.addUpdateCols(ins.OnConflict.Exprs) // Project row-level BEFORE triggers for UPDATE. - mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate) + mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate, false /* cascade */) // Build the final upsert statement, including any returned expressions. mb.buildUpsert(returning) diff --git a/pkg/sql/opt/optbuilder/plpgsql.go b/pkg/sql/opt/optbuilder/plpgsql.go index 8e1a3ee097d8..29f7b7536af9 100644 --- a/pkg/sql/opt/optbuilder/plpgsql.go +++ b/pkg/sql/opt/optbuilder/plpgsql.go @@ -1166,7 +1166,7 @@ func (b *plpgsqlBuilder) handleIntForLoop( raiseErrArgs := make([]memo.ScalarListExpr, 0, 4) const severity, detail, hint = "ERROR", "", "" addCheck := func(message, code string, checkCond opt.ScalarExpr) { - raiseErrArgs = append(raiseErrArgs, b.makeConstRaiseArgs(severity, message, detail, hint, code)) + raiseErrArgs = append(raiseErrArgs, b.ob.makeConstRaiseArgs(severity, message, detail, hint, code)) branches = append(branches, checkCond) } addNullCheck := func(context string, varRef tree.Expr) { @@ -1488,31 +1488,12 @@ func (b *plpgsqlBuilder) buildInto(stmtScope *scope, target []ast.Variable) *sco func (b *plpgsqlBuilder) buildPLpgSQLRaise(inScope *scope, args memo.ScalarListExpr) *scope { raiseColName := scopeColName("").WithMetadataName(b.makeIdentifier("stmt_raise")) raiseScope := inScope.push() - fn := b.makePLpgSQLRaiseFn(args) + fn := b.ob.makePLpgSQLRaiseFn(args) b.ob.synthesizeColumn(raiseScope, raiseColName, types.Int, nil /* expr */, fn) b.ob.constructProjectForScope(inScope, raiseScope) return raiseScope } -// makePLpgSQLRaiseFn builds a call to the crdb_internal.plpgsql_raise builtin -// function, which implements the notice-sending behavior of RAISE statements. -func (b *plpgsqlBuilder) makePLpgSQLRaiseFn(args memo.ScalarListExpr) opt.ScalarExpr { - const raiseFnName = "crdb_internal.plpgsql_raise" - fnProps, overloads := builtinsregistry.GetBuiltinProperties(raiseFnName) - if len(overloads) != 1 { - panic(errors.AssertionFailedf("expected one overload for %s", raiseFnName)) - } - return b.ob.factory.ConstructFunction( - args, - &memo.FunctionPrivate{ - Name: raiseFnName, - Typ: types.Int, - Properties: fnProps, - Overload: &overloads[0], - }, - ) -} - // getRaiseArgs validates the options attached to the given PLpgSQL RAISE // statement and returns the arguments to be used for a call to the // crdb_internal.plpgsql_raise builtin function. @@ -1601,23 +1582,6 @@ func (b *plpgsqlBuilder) getRaiseArgs(s *scope, raise *ast.Raise) memo.ScalarLis return args } -// makeConstRaiseArgs builds the arguments for a crdb_internal.plpgsql_raise -// function call. -func (b *plpgsqlBuilder) makeConstRaiseArgs( - severity, message, detail, hint, code string, -) memo.ScalarListExpr { - makeConstStr := func(str string) opt.ScalarExpr { - return b.ob.factory.ConstructConstVal(tree.NewDString(str), types.String) - } - return memo.ScalarListExpr{ - makeConstStr(severity), - makeConstStr(message), - makeConstStr(detail), - makeConstStr(hint), - makeConstStr(code), - } -} - // A PLpgSQL RAISE statement can specify a format string, where supplied // expressions replace instances of '%' in the string. A literal '%' character // is specified by doubling it: '%%'. The formatting arguments can be arbitrary @@ -1818,7 +1782,7 @@ func (b *plpgsqlBuilder) handleEndOfFunction(inScope *scope) *scope { // throws an end-of-function error, as well as a typed RETURN NULL to ensure // that type-checking works out. func (b *plpgsqlBuilder) buildEndOfFunctionRaise(con *continuation) { - args := b.makeConstRaiseArgs( + args := b.ob.makeConstRaiseArgs( "ERROR", /* severity */ "control reached end of function without RETURN", /* message */ "", /* detail */ @@ -1858,14 +1822,14 @@ func (b *plpgsqlBuilder) addOneRowCheck(s *scope) { s.expr = b.ob.factory.ConstructScalarGroupBy(s.expr, aggs, &memo.GroupingPrivate{}) // Add a runtime check for the row count. - tooFewRowsArgs := b.makeConstRaiseArgs( + tooFewRowsArgs := b.ob.makeConstRaiseArgs( "ERROR", /* severity */ "query returned no rows", /* message */ "", /* detail */ "", /* hint */ pgcode.NoDataFound.String(), /* code */ ) - tooManyRowsArgs := b.makeConstRaiseArgs( + tooManyRowsArgs := b.ob.makeConstRaiseArgs( "ERROR", /* severity */ "query returned more than one row", /* message */ "", /* detail */ @@ -1894,7 +1858,7 @@ func (b *plpgsqlBuilder) addRuntimeCheck( originalCols := s.colSet() caseWhens := make(memo.ScalarListExpr, len(branches)) for i := range branches { - raiseErrFn := b.makePLpgSQLRaiseFn(raiseErrArgs[i]) + raiseErrFn := b.ob.makePLpgSQLRaiseFn(raiseErrArgs[i]) caseWhens[i] = b.ob.factory.ConstructWhen(branches[i], raiseErrFn) } caseExpr := b.ob.factory.ConstructCase( diff --git a/pkg/sql/opt/optbuilder/testdata/trigger b/pkg/sql/opt/optbuilder/testdata/trigger index 0678354a1e8f..bb804805f587 100644 --- a/pkg/sql/opt/optbuilder/testdata/trigger +++ b/pkg/sql/opt/optbuilder/testdata/trigger @@ -2,6 +2,10 @@ exec-ddl CREATE TABLE xy (x INT PRIMARY KEY, y INT); ---- +exec-ddl +CREATE TABLE child (k INT PRIMARY KEY, x INT REFERENCES xy(x) ON UPDATE CASCADE ON DELETE CASCADE); +---- + exec-ddl CREATE FUNCTION f() RETURNS TRIGGER LANGUAGE PLpgSQL AS $$ BEGIN @@ -18,6 +22,10 @@ exec-ddl CREATE TRIGGER tr BEFORE INSERT OR UPDATE OR DELETE ON xy FOR EACH ROW EXECUTE FUNCTION f(); ---- +exec-ddl +CREATE TRIGGER tr_child BEFORE INSERT OR UPDATE OR DELETE ON child FOR EACH ROW EXECUTE FUNCTION f(); +---- + norm format=(hide-all,show-columns) INSERT INTO xy VALUES (1, 2); ---- @@ -47,211 +55,489 @@ insert xy ├── (f:21).x [as=x_new:22] └── (f:21).y [as=y_new:23] -norm format=(hide-all,show-columns) +build-post-queries format=(hide-all,show-columns) UPDATE xy SET y = 3 WHERE x = 1; ---- -update xy - ├── columns: - ├── fetch columns: x:5 y:6 - ├── update-mapping: - │ ├── x_new:26 => x:1 - │ └── y_new:27 => y:2 - └── project - ├── columns: x_new:26 y_new:27 x:5 y:6 - ├── barrier - │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 f:25 - │ └── select - │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 f:25 - │ ├── project - │ │ ├── columns: f:25 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 - │ │ ├── barrier - │ │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 - │ │ │ └── project - │ │ │ ├── columns: new:11 old:10 y_new:9 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 - │ │ │ ├── select - │ │ │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 - │ │ │ │ ├── scan xy - │ │ │ │ │ └── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 - │ │ │ │ └── filters - │ │ │ │ └── x:5 = 1 - │ │ │ └── projections - │ │ │ ├── ((x:5, 3) AS x, y) [as=new:11] - │ │ │ ├── ((x:5, y:6) AS x, y) [as=old:10] - │ │ │ └── 3 [as=y_new:9] - │ │ └── projections - │ │ └── f(new:11, old:10, 'tr', 'BEFORE', 'ROW', 'UPDATE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:25] - │ └── filters - │ └── f:25 IS DISTINCT FROM NULL - └── projections - ├── (f:25).x [as=x_new:26] - └── (f:25).y [as=y_new:27] +root + ├── update xy + │ ├── columns: + │ ├── fetch columns: x:5 y:6 + │ ├── update-mapping: + │ │ ├── x_new:26 => x:1 + │ │ └── y_new:27 => y:2 + │ ├── input binding: &1 + │ ├── cascades + │ │ └── child_x_fkey + │ └── project + │ ├── columns: x_new:26 y_new:27 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 f:25 + │ ├── barrier + │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 f:25 + │ │ └── select + │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 f:25 + │ │ ├── project + │ │ │ ├── columns: f:25 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 + │ │ │ ├── barrier + │ │ │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 new:11 + │ │ │ │ └── project + │ │ │ │ ├── columns: new:11 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 old:10 + │ │ │ │ ├── project + │ │ │ │ │ ├── columns: old:10 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 y_new:9 + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: y_new:9 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 + │ │ │ │ │ │ │ ├── scan xy + │ │ │ │ │ │ │ │ └── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ └── x:5 = 1 + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── 3 [as=y_new:9] + │ │ │ │ │ └── projections + │ │ │ │ │ └── ((x:5, y:6) AS x, y) [as=old:10] + │ │ │ │ └── projections + │ │ │ │ └── ((x:5, y_new:9) AS x, y) [as=new:11] + │ │ │ └── projections + │ │ │ └── f(new:11, old:10, 'tr', 'BEFORE', 'ROW', 'UPDATE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:25] + │ │ └── filters + │ │ └── f:25 IS DISTINCT FROM NULL + │ └── projections + │ ├── (f:25).x [as=x_new:26] + │ └── (f:25).y [as=y_new:27] + └── cascade + └── update child + ├── columns: + ├── fetch columns: k:32 child.x:33 + ├── update-mapping: + │ └── x_new:37 => child.x:29 + ├── input binding: &2 + ├── barrier + │ ├── columns: k:32 child.x:33 x_old:36 x_new:37 old:38 new:39 f:53 "check-rows":54 + │ └── select + │ ├── columns: k:32 child.x:33 x_old:36 x_new:37 old:38 new:39 f:53 "check-rows":54 + │ ├── barrier + │ │ ├── columns: k:32 child.x:33 x_old:36 x_new:37 old:38 new:39 f:53 "check-rows":54 + │ │ └── project + │ │ ├── columns: "check-rows":54 k:32 child.x:33 x_old:36 x_new:37 old:38 new:39 f:53 + │ │ ├── project + │ │ │ ├── columns: f:53 k:32 child.x:33 x_old:36 x_new:37 old:38 new:39 + │ │ │ ├── barrier + │ │ │ │ ├── columns: k:32 child.x:33 x_old:36 x_new:37 old:38 new:39 + │ │ │ │ └── project + │ │ │ │ ├── columns: new:39 k:32 child.x:33 x_old:36 x_new:37 old:38 + │ │ │ │ ├── project + │ │ │ │ │ ├── columns: old:38 k:32 child.x:33 x_old:36 x_new:37 + │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ ├── columns: k:32 child.x:33 x_old:36 x_new:37 + │ │ │ │ │ │ ├── scan child + │ │ │ │ │ │ │ └── columns: k:32 child.x:33 + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: x_old:36 x_new:37 + │ │ │ │ │ │ │ ├── with-scan &1 + │ │ │ │ │ │ │ │ ├── columns: x_old:36 x_new:37 + │ │ │ │ │ │ │ │ └── mapping: + │ │ │ │ │ │ │ │ ├── xy.x:5 => x_old:36 + │ │ │ │ │ │ │ │ └── x_new:26 => x_new:37 + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ └── x_old:36 IS DISTINCT FROM x_new:37 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── child.x:33 = x_old:36 + │ │ │ │ │ └── projections + │ │ │ │ │ └── ((k:32, child.x:33) AS k, x) [as=old:38] + │ │ │ │ └── projections + │ │ │ │ └── ((k:32, x_new:37) AS k, x) [as=new:39] + │ │ │ └── projections + │ │ │ └── f(new:39, old:38, 'tr_child', 'BEFORE', 'ROW', 'UPDATE', 54, 'child', 'child', 'public', 0, ARRAY[]) [as=f:53] + │ │ └── projections + │ │ └── CASE WHEN f:53 IS DISTINCT FROM new:39 THEN crdb_internal.plpgsql_raise('ERROR', 'trigger tr_child attempted to modify or filter a row in a cascade operation: ' || new:39::STRING, e'changing the rows updated or deleted by a foreign-key cascade\n can cause constraint violations, and therefore is not allowed', e'to enable this behavior (with risk of constraint violation), set\nthe session variable \'unsafe_allow_triggers_modifying_cascades\' to true', '27000') ELSE CAST(NULL AS INT8) END [as="check-rows":54] + │ └── filters + │ └── f:53 IS DISTINCT FROM NULL + └── f-k-checks + └── f-k-checks-item: child(x) -> xy(x) + └── anti-join (hash) + ├── columns: x:55 + ├── select + │ ├── columns: x:55 + │ ├── with-scan &2 + │ │ ├── columns: x:55 + │ │ └── mapping: + │ │ └── x_new:37 => x:55 + │ └── filters + │ └── x:55 IS NOT NULL + ├── scan xy + │ └── columns: xy.x:56 + └── filters + └── x:55 = xy.x:56 -norm format=(hide-all,show-columns) +build-post-queries format=(hide-all,show-columns) DELETE FROM xy WHERE x = 1; ---- -delete xy - ├── columns: - ├── fetch columns: x:5 - └── barrier - ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 f:23 - └── select - ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 f:23 - ├── project - │ ├── columns: f:23 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 - │ ├── barrier - │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 - │ │ └── project - │ │ ├── columns: old:9 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 - │ │ ├── select - │ │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 - │ │ │ ├── scan xy - │ │ │ │ └── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 - │ │ │ └── filters - │ │ │ └── x:5 = 1 - │ │ └── projections - │ │ └── ((x:5, y:6) AS x, y) [as=old:9] - │ └── projections - │ └── f(NULL, old:9, 'tr', 'BEFORE', 'ROW', 'DELETE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:23] - └── filters - └── f:23 IS DISTINCT FROM NULL +root + ├── delete xy + │ ├── columns: + │ ├── fetch columns: x:5 y:6 + │ ├── input binding: &1 + │ ├── cascades + │ │ └── child_x_fkey + │ └── barrier + │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 f:23 + │ └── select + │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 f:23 + │ ├── project + │ │ ├── columns: f:23 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 + │ │ ├── barrier + │ │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 old:9 + │ │ │ └── project + │ │ │ ├── columns: old:9 x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 + │ │ │ ├── select + │ │ │ │ ├── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 + │ │ │ │ ├── scan xy + │ │ │ │ │ └── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 + │ │ │ │ └── filters + │ │ │ │ └── x:5 = 1 + │ │ │ └── projections + │ │ │ └── ((x:5, y:6) AS x, y) [as=old:9] + │ │ └── projections + │ │ └── f(NULL, old:9, 'tr', 'BEFORE', 'ROW', 'DELETE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:23] + │ └── filters + │ └── f:23 IS DISTINCT FROM NULL + └── cascade + └── delete child + ├── columns: + ├── fetch columns: k:28 child.x:29 + └── barrier + ├── columns: k:28 child.x:29 old:33 f:47 "check-rows":48 + └── select + ├── columns: k:28 child.x:29 old:33 f:47 "check-rows":48 + ├── barrier + │ ├── columns: k:28 child.x:29 old:33 f:47 "check-rows":48 + │ └── project + │ ├── columns: "check-rows":48 k:28 child.x:29 old:33 f:47 + │ ├── project + │ │ ├── columns: f:47 k:28 child.x:29 old:33 + │ │ ├── barrier + │ │ │ ├── columns: k:28 child.x:29 old:33 + │ │ │ └── project + │ │ │ ├── columns: old:33 k:28 child.x:29 + │ │ │ ├── semi-join (hash) + │ │ │ │ ├── columns: k:28 child.x:29 + │ │ │ │ ├── scan child + │ │ │ │ │ └── columns: k:28 child.x:29 + │ │ │ │ ├── with-scan &1 + │ │ │ │ │ ├── columns: x:32 + │ │ │ │ │ └── mapping: + │ │ │ │ │ └── xy.x:5 => x:32 + │ │ │ │ └── filters + │ │ │ │ └── child.x:29 = x:32 + │ │ │ └── projections + │ │ │ └── ((k:28, child.x:29) AS k, x) [as=old:33] + │ │ └── projections + │ │ └── f(NULL, old:33, 'tr_child', 'BEFORE', 'ROW', 'DELETE', 54, 'child', 'child', 'public', 0, ARRAY[]) [as=f:47] + │ └── projections + │ └── CASE WHEN f:47 IS DISTINCT FROM old:33 THEN crdb_internal.plpgsql_raise('ERROR', 'trigger tr_child attempted to modify or filter a row in a cascade operation: ' || old:33::STRING, e'changing the rows updated or deleted by a foreign-key cascade\n can cause constraint violations, and therefore is not allowed', e'to enable this behavior (with risk of constraint violation), set\nthe session variable \'unsafe_allow_triggers_modifying_cascades\' to true', '27000') ELSE CAST(NULL AS INT8) END [as="check-rows":48] + └── filters + └── f:47 IS DISTINCT FROM NULL -norm format=(hide-all,show-columns) +build-post-queries format=(hide-all,show-columns) UPSERT INTO xy VALUES (1, 2); ---- -upsert xy - ├── arbiter indexes: xy_pkey - ├── columns: - ├── canary column: x:7 - ├── fetch columns: x:7 y:8 - ├── insert-mapping: - │ ├── x_new:26 => x:1 - │ └── y_new:27 => y:2 - ├── update-mapping: - │ ├── upsert_x:46 => x:1 - │ └── upsert_y:47 => y:2 - └── project - ├── columns: upsert_x:46 upsert_y:47 x:7 y:8 x_new:26 y_new:27 - ├── barrier - │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 f:43 - │ └── select - │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 f:43 - │ ├── project - │ │ ├── columns: f:43 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 - │ │ ├── barrier - │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 - │ │ │ └── project - │ │ │ ├── columns: new:29 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 - │ │ │ ├── project - │ │ │ │ ├── columns: old:28 x_new:26 y_new:27 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 - │ │ │ │ ├── barrier - │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 - │ │ │ │ │ └── select - │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 - │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: f:25 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 - │ │ │ │ │ │ ├── barrier - │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 - │ │ │ │ │ │ │ └── project - │ │ │ │ │ │ │ ├── columns: new:11 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ │ ├── left-join (cross) - │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 - │ │ │ │ │ │ │ │ │ └── (1, 2) - │ │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ │ ├── columns: x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ │ │ │ │ └── columns: x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── x:7 = 1 - │ │ │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── ((column1:5, column2:6) AS x, y) [as=new:11] - │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── f(new:11, NULL, 'tr', 'BEFORE', 'ROW', 'INSERT', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:25] - │ │ │ │ │ └── filters - │ │ │ │ │ └── f:25 IS DISTINCT FROM NULL - │ │ │ │ └── projections - │ │ │ │ ├── ((x:7, y:8) AS x, y) [as=old:28] - │ │ │ │ ├── (f:25).x [as=x_new:26] - │ │ │ │ └── (f:25).y [as=y_new:27] - │ │ │ └── projections - │ │ │ └── ((x:7, y_new:27) AS x, y) [as=new:29] - │ │ └── projections - │ │ └── CASE WHEN x:7 IS NOT NULL THEN f(new:29, old:28, 'tr', 'BEFORE', 'ROW', 'UPDATE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) ELSE new:29 END [as=f:43] - │ └── filters - │ └── f:43 IS DISTINCT FROM NULL - └── projections - ├── CASE WHEN x:7 IS NULL THEN x_new:26 ELSE (f:43).x END [as=upsert_x:46] - └── CASE WHEN x:7 IS NULL THEN y_new:27 ELSE (f:43).y END [as=upsert_y:47] +root + ├── upsert xy + │ ├── arbiter indexes: xy_pkey + │ ├── columns: + │ ├── canary column: x:7 + │ ├── fetch columns: x:7 y:8 + │ ├── insert-mapping: + │ │ ├── x_new:26 => x:1 + │ │ └── y_new:27 => y:2 + │ ├── update-mapping: + │ │ ├── upsert_x:46 => x:1 + │ │ └── upsert_y:47 => y:2 + │ ├── input binding: &1 + │ ├── cascades + │ │ └── child_x_fkey + │ └── project + │ ├── columns: upsert_x:46 upsert_y:47 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 f:43 x_new:44 y_new:45 + │ ├── project + │ │ ├── columns: x_new:44 y_new:45 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 f:43 + │ │ ├── barrier + │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 f:43 + │ │ │ └── select + │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 f:43 + │ │ │ ├── project + │ │ │ │ ├── columns: f:43 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 + │ │ │ │ ├── barrier + │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 new:29 + │ │ │ │ │ └── project + │ │ │ │ │ ├── columns: new:29 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 old:28 + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: old:28 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 + │ │ │ │ │ │ ├── project + │ │ │ │ │ │ │ ├── columns: x_new:26 y_new:27 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 + │ │ │ │ │ │ │ ├── barrier + │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 + │ │ │ │ │ │ │ │ └── select + │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 + │ │ │ │ │ │ │ │ ├── project + │ │ │ │ │ │ │ │ │ ├── columns: f:25 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 + │ │ │ │ │ │ │ │ │ ├── barrier + │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 + │ │ │ │ │ │ │ │ │ │ └── project + │ │ │ │ │ │ │ │ │ │ ├── columns: new:11 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + │ │ │ │ │ │ │ │ │ │ ├── left-join (hash) + │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + │ │ │ │ │ │ │ │ │ │ │ ├── ensure-upsert-distinct-on + │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 + │ │ │ │ │ │ │ │ │ │ │ │ ├── grouping columns: column1:5 + │ │ │ │ │ │ │ │ │ │ │ │ ├── values + │ │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 + │ │ │ │ │ │ │ │ │ │ │ │ │ └── (1, 2) + │ │ │ │ │ │ │ │ │ │ │ │ └── aggregations + │ │ │ │ │ │ │ │ │ │ │ │ └── first-agg [as=column2:6] + │ │ │ │ │ │ │ │ │ │ │ │ └── column2:6 + │ │ │ │ │ │ │ │ │ │ │ ├── scan xy + │ │ │ │ │ │ │ │ │ │ │ │ └── columns: x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + │ │ │ │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ │ │ │ └── column1:5 = x:7 + │ │ │ │ │ │ │ │ │ │ └── projections + │ │ │ │ │ │ │ │ │ │ └── ((column1:5, column2:6) AS x, y) [as=new:11] + │ │ │ │ │ │ │ │ │ └── projections + │ │ │ │ │ │ │ │ │ └── f(new:11, NULL, 'tr', 'BEFORE', 'ROW', 'INSERT', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:25] + │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ └── f:25 IS DISTINCT FROM NULL + │ │ │ │ │ │ │ └── projections + │ │ │ │ │ │ │ ├── (f:25).x [as=x_new:26] + │ │ │ │ │ │ │ └── (f:25).y [as=y_new:27] + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── ((x:7, y:8) AS x, y) [as=old:28] + │ │ │ │ │ └── projections + │ │ │ │ │ └── ((x:7, y_new:27) AS x, y) [as=new:29] + │ │ │ │ └── projections + │ │ │ │ └── CASE WHEN x:7 IS NOT NULL THEN f(new:29, old:28, 'tr', 'BEFORE', 'ROW', 'UPDATE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) ELSE new:29 END [as=f:43] + │ │ │ └── filters + │ │ │ └── f:43 IS DISTINCT FROM NULL + │ │ └── projections + │ │ ├── (f:43).x [as=x_new:44] + │ │ └── (f:43).y [as=y_new:45] + │ └── projections + │ ├── CASE WHEN x:7 IS NULL THEN x_new:26 ELSE x_new:44 END [as=upsert_x:46] + │ └── CASE WHEN x:7 IS NULL THEN y_new:27 ELSE y_new:45 END [as=upsert_y:47] + └── cascade + └── update child + ├── columns: + ├── fetch columns: k:52 child.x:53 + ├── update-mapping: + │ └── x_new:57 => child.x:49 + ├── input binding: &2 + ├── barrier + │ ├── columns: k:52 child.x:53 x_old:56 x_new:57 old:58 new:59 f:73 "check-rows":74 + │ └── select + │ ├── columns: k:52 child.x:53 x_old:56 x_new:57 old:58 new:59 f:73 "check-rows":74 + │ ├── barrier + │ │ ├── columns: k:52 child.x:53 x_old:56 x_new:57 old:58 new:59 f:73 "check-rows":74 + │ │ └── project + │ │ ├── columns: "check-rows":74 k:52 child.x:53 x_old:56 x_new:57 old:58 new:59 f:73 + │ │ ├── project + │ │ │ ├── columns: f:73 k:52 child.x:53 x_old:56 x_new:57 old:58 new:59 + │ │ │ ├── barrier + │ │ │ │ ├── columns: k:52 child.x:53 x_old:56 x_new:57 old:58 new:59 + │ │ │ │ └── project + │ │ │ │ ├── columns: new:59 k:52 child.x:53 x_old:56 x_new:57 old:58 + │ │ │ │ ├── project + │ │ │ │ │ ├── columns: old:58 k:52 child.x:53 x_old:56 x_new:57 + │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ ├── columns: k:52 child.x:53 x_old:56 x_new:57 + │ │ │ │ │ │ ├── scan child + │ │ │ │ │ │ │ └── columns: k:52 child.x:53 + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: x_old:56 x_new:57 + │ │ │ │ │ │ │ ├── with-scan &1 + │ │ │ │ │ │ │ │ ├── columns: x_old:56 x_new:57 + │ │ │ │ │ │ │ │ └── mapping: + │ │ │ │ │ │ │ │ ├── xy.x:7 => x_old:56 + │ │ │ │ │ │ │ │ └── upsert_x:46 => x_new:57 + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ └── x_old:56 IS DISTINCT FROM x_new:57 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── child.x:53 = x_old:56 + │ │ │ │ │ └── projections + │ │ │ │ │ └── ((k:52, child.x:53) AS k, x) [as=old:58] + │ │ │ │ └── projections + │ │ │ │ └── ((k:52, x_new:57) AS k, x) [as=new:59] + │ │ │ └── projections + │ │ │ └── f(new:59, old:58, 'tr_child', 'BEFORE', 'ROW', 'UPDATE', 54, 'child', 'child', 'public', 0, ARRAY[]) [as=f:73] + │ │ └── projections + │ │ └── CASE WHEN f:73 IS DISTINCT FROM new:59 THEN crdb_internal.plpgsql_raise('ERROR', 'trigger tr_child attempted to modify or filter a row in a cascade operation: ' || new:59::STRING, e'changing the rows updated or deleted by a foreign-key cascade\n can cause constraint violations, and therefore is not allowed', e'to enable this behavior (with risk of constraint violation), set\nthe session variable \'unsafe_allow_triggers_modifying_cascades\' to true', '27000') ELSE CAST(NULL AS INT8) END [as="check-rows":74] + │ └── filters + │ └── f:73 IS DISTINCT FROM NULL + └── f-k-checks + └── f-k-checks-item: child(x) -> xy(x) + └── anti-join (hash) + ├── columns: x:75 + ├── select + │ ├── columns: x:75 + │ ├── with-scan &2 + │ │ ├── columns: x:75 + │ │ └── mapping: + │ │ └── x_new:57 => x:75 + │ └── filters + │ └── x:75 IS NOT NULL + ├── scan xy + │ └── columns: xy.x:76 + └── filters + └── x:75 = xy.x:76 -norm format=(hide-all,show-columns) +build-post-queries format=(hide-all,show-columns) INSERT INTO xy VALUES (1, 2) ON CONFLICT (x) DO UPDATE SET y = 3; ---- -upsert xy - ├── arbiter indexes: xy_pkey - ├── columns: - ├── canary column: x:7 - ├── fetch columns: x:7 y:8 - ├── insert-mapping: - │ ├── x_new:26 => x:1 - │ └── y_new:27 => y:2 - ├── update-mapping: - │ ├── upsert_x:47 => x:1 - │ └── upsert_y:48 => y:2 - └── project - ├── columns: upsert_x:47 upsert_y:48 x:7 y:8 x_new:26 y_new:27 - ├── barrier - │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 f:44 - │ └── select - │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 f:44 - │ ├── project - │ │ ├── columns: f:44 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 - │ │ ├── barrier - │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 - │ │ │ └── project - │ │ │ ├── columns: new:30 old:29 y_new:28 x_new:26 y_new:27 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 - │ │ │ ├── barrier - │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 - │ │ │ │ └── select - │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 - │ │ │ │ ├── project - │ │ │ │ │ ├── columns: f:25 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 - │ │ │ │ │ ├── barrier - │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 - │ │ │ │ │ │ └── project - │ │ │ │ │ │ ├── columns: new:11 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ ├── left-join (cross) - │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 - │ │ │ │ │ │ │ │ └── (1, 2) - │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ │ │ │ └── columns: x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 - │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── x:7 = 1 - │ │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── ((column1:5, column2:6) AS x, y) [as=new:11] - │ │ │ │ │ └── projections - │ │ │ │ │ └── f(new:11, NULL, 'tr', 'BEFORE', 'ROW', 'INSERT', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:25] - │ │ │ │ └── filters - │ │ │ │ └── f:25 IS DISTINCT FROM NULL - │ │ │ └── projections - │ │ │ ├── ((x:7, 3) AS x, y) [as=new:30] - │ │ │ ├── ((x:7, y:8) AS x, y) [as=old:29] - │ │ │ ├── 3 [as=y_new:28] - │ │ │ ├── (f:25).x [as=x_new:26] - │ │ │ └── (f:25).y [as=y_new:27] - │ │ └── projections - │ │ └── CASE WHEN x:7 IS NOT NULL THEN f(new:30, old:29, 'tr', 'BEFORE', 'ROW', 'UPDATE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) ELSE new:30 END [as=f:44] - │ └── filters - │ └── f:44 IS DISTINCT FROM NULL - └── projections - ├── CASE WHEN x:7 IS NULL THEN x_new:26 ELSE (f:44).x END [as=upsert_x:47] - └── CASE WHEN x:7 IS NULL THEN y_new:27 ELSE (f:44).y END [as=upsert_y:48] +root + ├── upsert xy + │ ├── arbiter indexes: xy_pkey + │ ├── columns: + │ ├── canary column: x:7 + │ ├── fetch columns: x:7 y:8 + │ ├── insert-mapping: + │ │ ├── x_new:26 => x:1 + │ │ └── y_new:27 => y:2 + │ ├── update-mapping: + │ │ ├── upsert_x:47 => x:1 + │ │ └── upsert_y:48 => y:2 + │ ├── input binding: &1 + │ ├── cascades + │ │ └── child_x_fkey + │ └── project + │ ├── columns: upsert_x:47 upsert_y:48 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 f:44 x_new:45 y_new:46 + │ ├── project + │ │ ├── columns: x_new:45 y_new:46 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 f:44 + │ │ ├── barrier + │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 f:44 + │ │ │ └── select + │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 f:44 + │ │ │ ├── project + │ │ │ │ ├── columns: f:44 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 + │ │ │ │ ├── barrier + │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 new:30 + │ │ │ │ │ └── project + │ │ │ │ │ ├── columns: new:30 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 old:29 + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: old:29 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 y_new:28 + │ │ │ │ │ │ ├── project + │ │ │ │ │ │ │ ├── columns: y_new:28 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 x_new:26 y_new:27 + │ │ │ │ │ │ │ ├── project + │ │ │ │ │ │ │ │ ├── columns: x_new:26 y_new:27 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 + │ │ │ │ │ │ │ │ ├── barrier + │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 + │ │ │ │ │ │ │ │ │ └── select + │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 f:25 + │ │ │ │ │ │ │ │ │ ├── project + │ │ │ │ │ │ │ │ │ │ ├── columns: f:25 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 + │ │ │ │ │ │ │ │ │ │ ├── barrier + │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 new:11 + │ │ │ │ │ │ │ │ │ │ │ └── project + │ │ │ │ │ │ │ │ │ │ │ ├── columns: new:11 column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + │ │ │ │ │ │ │ │ │ │ │ ├── left-join (hash) + │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + │ │ │ │ │ │ │ │ │ │ │ │ ├── ensure-upsert-distinct-on + │ │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 + │ │ │ │ │ │ │ │ │ │ │ │ │ ├── grouping columns: column1:5 + │ │ │ │ │ │ │ │ │ │ │ │ │ ├── values + │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5 column2:6 + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └── (1, 2) + │ │ │ │ │ │ │ │ │ │ │ │ │ └── aggregations + │ │ │ │ │ │ │ │ │ │ │ │ │ └── first-agg [as=column2:6] + │ │ │ │ │ │ │ │ │ │ │ │ │ └── column2:6 + │ │ │ │ │ │ │ │ │ │ │ │ ├── scan xy + │ │ │ │ │ │ │ │ │ │ │ │ │ └── columns: x:7 y:8 crdb_internal_mvcc_timestamp:9 tableoid:10 + │ │ │ │ │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ │ │ │ │ └── column1:5 = x:7 + │ │ │ │ │ │ │ │ │ │ │ └── projections + │ │ │ │ │ │ │ │ │ │ │ └── ((column1:5, column2:6) AS x, y) [as=new:11] + │ │ │ │ │ │ │ │ │ │ └── projections + │ │ │ │ │ │ │ │ │ │ └── f(new:11, NULL, 'tr', 'BEFORE', 'ROW', 'INSERT', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:25] + │ │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ │ └── f:25 IS DISTINCT FROM NULL + │ │ │ │ │ │ │ │ └── projections + │ │ │ │ │ │ │ │ ├── (f:25).x [as=x_new:26] + │ │ │ │ │ │ │ │ └── (f:25).y [as=y_new:27] + │ │ │ │ │ │ │ └── projections + │ │ │ │ │ │ │ └── 3 [as=y_new:28] + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── ((x:7, y:8) AS x, y) [as=old:29] + │ │ │ │ │ └── projections + │ │ │ │ │ └── ((x:7, y_new:28) AS x, y) [as=new:30] + │ │ │ │ └── projections + │ │ │ │ └── CASE WHEN x:7 IS NOT NULL THEN f(new:30, old:29, 'tr', 'BEFORE', 'ROW', 'UPDATE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) ELSE new:30 END [as=f:44] + │ │ │ └── filters + │ │ │ └── f:44 IS DISTINCT FROM NULL + │ │ └── projections + │ │ ├── (f:44).x [as=x_new:45] + │ │ └── (f:44).y [as=y_new:46] + │ └── projections + │ ├── CASE WHEN x:7 IS NULL THEN x_new:26 ELSE x_new:45 END [as=upsert_x:47] + │ └── CASE WHEN x:7 IS NULL THEN y_new:27 ELSE y_new:46 END [as=upsert_y:48] + └── cascade + └── update child + ├── columns: + ├── fetch columns: k:53 child.x:54 + ├── update-mapping: + │ └── x_new:58 => child.x:50 + ├── input binding: &2 + ├── barrier + │ ├── columns: k:53 child.x:54 x_old:57 x_new:58 old:59 new:60 f:74 "check-rows":75 + │ └── select + │ ├── columns: k:53 child.x:54 x_old:57 x_new:58 old:59 new:60 f:74 "check-rows":75 + │ ├── barrier + │ │ ├── columns: k:53 child.x:54 x_old:57 x_new:58 old:59 new:60 f:74 "check-rows":75 + │ │ └── project + │ │ ├── columns: "check-rows":75 k:53 child.x:54 x_old:57 x_new:58 old:59 new:60 f:74 + │ │ ├── project + │ │ │ ├── columns: f:74 k:53 child.x:54 x_old:57 x_new:58 old:59 new:60 + │ │ │ ├── barrier + │ │ │ │ ├── columns: k:53 child.x:54 x_old:57 x_new:58 old:59 new:60 + │ │ │ │ └── project + │ │ │ │ ├── columns: new:60 k:53 child.x:54 x_old:57 x_new:58 old:59 + │ │ │ │ ├── project + │ │ │ │ │ ├── columns: old:59 k:53 child.x:54 x_old:57 x_new:58 + │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ ├── columns: k:53 child.x:54 x_old:57 x_new:58 + │ │ │ │ │ │ ├── scan child + │ │ │ │ │ │ │ └── columns: k:53 child.x:54 + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: x_old:57 x_new:58 + │ │ │ │ │ │ │ ├── with-scan &1 + │ │ │ │ │ │ │ │ ├── columns: x_old:57 x_new:58 + │ │ │ │ │ │ │ │ └── mapping: + │ │ │ │ │ │ │ │ ├── xy.x:7 => x_old:57 + │ │ │ │ │ │ │ │ └── upsert_x:47 => x_new:58 + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ └── x_old:57 IS DISTINCT FROM x_new:58 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── child.x:54 = x_old:57 + │ │ │ │ │ └── projections + │ │ │ │ │ └── ((k:53, child.x:54) AS k, x) [as=old:59] + │ │ │ │ └── projections + │ │ │ │ └── ((k:53, x_new:58) AS k, x) [as=new:60] + │ │ │ └── projections + │ │ │ └── f(new:60, old:59, 'tr_child', 'BEFORE', 'ROW', 'UPDATE', 54, 'child', 'child', 'public', 0, ARRAY[]) [as=f:74] + │ │ └── projections + │ │ └── CASE WHEN f:74 IS DISTINCT FROM new:60 THEN crdb_internal.plpgsql_raise('ERROR', 'trigger tr_child attempted to modify or filter a row in a cascade operation: ' || new:60::STRING, e'changing the rows updated or deleted by a foreign-key cascade\n can cause constraint violations, and therefore is not allowed', e'to enable this behavior (with risk of constraint violation), set\nthe session variable \'unsafe_allow_triggers_modifying_cascades\' to true', '27000') ELSE CAST(NULL AS INT8) END [as="check-rows":75] + │ └── filters + │ └── f:74 IS DISTINCT FROM NULL + └── f-k-checks + └── f-k-checks-item: child(x) -> xy(x) + └── anti-join (hash) + ├── columns: x:76 + ├── select + │ ├── columns: x:76 + │ ├── with-scan &2 + │ │ ├── columns: x:76 + │ │ └── mapping: + │ │ └── x_new:58 => x:76 + │ └── filters + │ └── x:76 IS NOT NULL + ├── scan xy + │ └── columns: xy.x:77 + └── filters + └── x:76 = xy.x:77 # ------------------------------------------------------------------------------ # Row-level AFTER triggers. @@ -354,6 +640,8 @@ root │ ├── columns: │ ├── fetch columns: x:5 y:6 │ ├── input binding: &1 + │ ├── cascades + │ │ └── child_x_fkey │ ├── after-triggers │ │ └── tr │ └── select @@ -362,26 +650,59 @@ root │ │ └── columns: x:5 y:6 crdb_internal_mvcc_timestamp:7 tableoid:8 │ └── filters │ └── x:5 = 1 + ├── cascade + │ └── delete child + │ ├── columns: + │ ├── fetch columns: k:13 child.x:14 + │ └── barrier + │ ├── columns: k:13 child.x:14 new:17 f:31 "check-rows":32 + │ └── select + │ ├── columns: k:13 child.x:14 new:17 f:31 "check-rows":32 + │ ├── barrier + │ │ ├── columns: k:13 child.x:14 new:17 f:31 "check-rows":32 + │ │ └── project + │ │ ├── columns: "check-rows":32 k:13 child.x:14 new:17 f:31 + │ │ ├── project + │ │ │ ├── columns: f:31 k:13 child.x:14 new:17 + │ │ │ ├── barrier + │ │ │ │ ├── columns: k:13 child.x:14 new:17 + │ │ │ │ └── project + │ │ │ │ ├── columns: new:17 k:13 child.x:14 + │ │ │ │ ├── select + │ │ │ │ │ ├── columns: k:13 child.x:14 + │ │ │ │ │ ├── scan child + │ │ │ │ │ │ └── columns: k:13 child.x:14 + │ │ │ │ │ └── filters + │ │ │ │ │ ├── child.x:14 = 1 + │ │ │ │ │ └── child.x:14 IS DISTINCT FROM CAST(NULL AS INT8) + │ │ │ │ └── projections + │ │ │ │ └── ((k:13, child.x:14) AS k, x) [as=new:17] + │ │ │ └── projections + │ │ │ └── f(new:17, NULL, 'tr_child', 'BEFORE', 'ROW', 'INSERT', 54, 'child', 'child', 'public', 0, ARRAY[]) [as=f:31] + │ │ └── projections + │ │ └── CASE WHEN f:31 IS DISTINCT FROM new:17 THEN crdb_internal.plpgsql_raise('ERROR', 'trigger tr_child attempted to modify or filter a row in a cascade operation: ' || new:17::STRING, e'changing the rows updated or deleted by a foreign-key cascade\n can cause constraint violations, and therefore is not allowed', e'to enable this behavior (with risk of constraint violation), set\nthe session variable \'unsafe_allow_triggers_modifying_cascades\' to true', '27000') ELSE CAST(NULL AS INT8) END [as="check-rows":32] + │ └── filters + │ └── f:31 IS DISTINCT FROM NULL └── after-triggers └── barrier - ├── columns: x_old:9 y_old:10 old:11 new:12 f:26 + ├── columns: x_old:33 y_old:34 old:35 new:36 f:50 └── project - ├── columns: f:26 x_old:9 y_old:10 old:11 new:12 + ├── columns: f:50 x_old:33 y_old:34 old:35 new:36 ├── project - │ ├── columns: new:12 x_old:9 y_old:10 old:11 + │ ├── columns: new:36 x_old:33 y_old:34 old:35 │ ├── project - │ │ ├── columns: old:11 x_old:9 y_old:10 + │ │ ├── columns: old:35 x_old:33 y_old:34 │ │ ├── with-scan &1 - │ │ │ ├── columns: x_old:9 y_old:10 + │ │ │ ├── columns: x_old:33 y_old:34 │ │ │ └── mapping: - │ │ │ ├── x:5 => x_old:9 - │ │ │ └── y:6 => y_old:10 + │ │ │ ├── xy.x:5 => x_old:33 + │ │ │ └── y:6 => y_old:34 │ │ └── projections - │ │ └── ((x_old:9, y_old:10) AS x, y) [as=old:11] + │ │ └── ((x_old:33, y_old:34) AS x, y) [as=old:35] │ └── projections - │ └── NULL [as=new:12] + │ └── NULL [as=new:36] └── projections - └── f(new:12, old:11, 'tr', 'AFTER', 'ROW', 'DELETE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:26] + └── f(new:36, old:35, 'tr', 'AFTER', 'ROW', 'DELETE', 53, 'xy', 'xy', 'public', 0, ARRAY[]) [as=f:50] build-post-queries format=(hide-all,show-columns) UPSERT INTO xy VALUES (1, 2); @@ -621,7 +942,7 @@ insert t133329 │ │ │ │ └── projections │ │ │ │ └── ((column1, column2) AS k, a) │ │ │ └── projections - │ │ │ └── f(new, NULL, 'tr', 'BEFORE', 'ROW', 'INSERT', 55, 't133329', 't133329', 'public', 0, ARRAY[]) + │ │ │ └── f(new, NULL, 'tr', 'BEFORE', 'ROW', 'INSERT', 56, 't133329', 't133329', 'public', 0, ARRAY[]) │ │ └── filters │ │ └── f IS DISTINCT FROM NULL │ └── projections diff --git a/pkg/sql/opt/optbuilder/trigger.go b/pkg/sql/opt/optbuilder/trigger.go index 21742bd47fdf..fda07828087f 100644 --- a/pkg/sql/opt/optbuilder/trigger.go +++ b/pkg/sql/opt/optbuilder/trigger.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical" "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" plpgsql "github.com/cockroachdb/cockroach/pkg/sql/plpgsql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -35,7 +36,9 @@ import ( // buildRowLevelBeforeTriggers builds any applicable row-level BEFORE triggers // based on the event type. It returns true if triggers were built, and false // otherwise. -func (mb *mutationBuilder) buildRowLevelBeforeTriggers(eventType tree.TriggerEventType) bool { +func (mb *mutationBuilder) buildRowLevelBeforeTriggers( + eventType tree.TriggerEventType, cascade bool, +) bool { var eventsToMatch tree.TriggerEventTypeSet eventsToMatch.Add(eventType) triggers := mb.getRowLevelTriggers(tree.TriggerActionTimeBefore, eventsToMatch) @@ -75,6 +78,7 @@ func (mb *mutationBuilder) buildRowLevelBeforeTriggers(eventType tree.TriggerEve // Build each trigger function invocation in order, applying optimization // barriers to ensure correct evaluation order. f := mb.b.factory + canModifyRows := true for i := range triggers { trigger := triggers[i] triggerScope.expr = f.ConstructBarrier(triggerScope.expr) @@ -110,6 +114,15 @@ func (mb *mutationBuilder) buildRowLevelBeforeTriggers(eventType tree.TriggerEve // Finally, project a column that invokes the trigger function. triggerFnColID := mb.b.projectColWithMetadataName(triggerScope, def.Name, tableTyp, triggerFn) + // Don't allow the trigger to modify or filter the row if the mutation is + // for a cascade, unless the session variable is set to allow it. + if cascade && !mb.b.evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades { + mb.ensureNoRowsModifiedByTrigger( + triggerScope, triggers[i].Name(), eventType, triggerFnColID, oldColID, newColID, + ) + canModifyRows = false + } + // BEFORE triggers can return a NULL value to indicate that the row should // be skipped. mb.applyFilterFromTrigger(triggerScope, triggerFnColID) @@ -126,7 +139,18 @@ func (mb *mutationBuilder) buildRowLevelBeforeTriggers(eventType tree.TriggerEve // INSERT and UPDATE triggers can modify the row to be inserted or updated // via the return value of the trigger function. if eventType == tree.TriggerEventInsert || eventType == tree.TriggerEventUpdate { - mb.applyChangesFromTriggers(triggerScope, eventType, tableTyp, visibleColOrds, newColID) + // If the trigger cannot modify rows, avoid changing the mutation columns. + // This is necessary to avoid adding extra checks during cascades, which + // could cause spurious constraint-violation errors. + // + // For example, in a "diamond" cascade pattern, an update to one table + // cascades to two others, which both cascade to a single grandchild table. + // Once both cascades complete, the database is in a consistent state. If a + // spurious check runs in between the two cascades, it could observe a + // constraint violation. + if canModifyRows { + mb.applyChangesFromTriggers(triggerScope, eventType, tableTyp, visibleColOrds, newColID) + } } mb.outScope = triggerScope return true @@ -140,33 +164,28 @@ func (mb *mutationBuilder) buildOldAndNewCols( triggerScope *scope, eventType tree.TriggerEventType, tableTyp *types.T, visibleColOrds []int, ) (oldColID, newColID opt.ColumnID) { f := mb.b.factory - makeTuple := func(colIDs opt.OptionalColList, name string) opt.ColumnID { + makeTuple := func(colIDs, backupColIDs opt.OptionalColList, name string) opt.ColumnID { elems := make(memo.ScalarListExpr, 0, len(visibleColOrds)) for _, i := range visibleColOrds { - if colIDs[i] == 0 { + col := colIDs[i] + if col == 0 && backupColIDs != nil { + col = backupColIDs[i] + } + if col == 0 { panic(errors.AssertionFailedf("missing column for trigger")) } - elems = append(elems, f.ConstructVariable(colIDs[i])) + elems = append(elems, f.ConstructVariable(col)) } tup := f.ConstructTuple(elems, tableTyp) return mb.b.projectColWithMetadataName(triggerScope, name, tableTyp, tup) } if eventType == tree.TriggerEventUpdate || eventType == tree.TriggerEventDelete { - oldColID = makeTuple(mb.fetchColIDs, triggerColOld) + oldColID = makeTuple(mb.fetchColIDs, nil /* backupColIDs */, triggerColOld) } if eventType == tree.TriggerEventInsert { - newColID = makeTuple(mb.insertColIDs, triggerColNew) + newColID = makeTuple(mb.insertColIDs, mb.fetchColIDs, triggerColNew) } else if eventType == tree.TriggerEventUpdate { - // Build a colIDs slice using updateColIDs, filling in the missing columns - // (which are not being updated) with the old column values. - colIDs := make(opt.OptionalColList, len(mb.updateColIDs)) - copy(colIDs, mb.updateColIDs) - for i, colID := range colIDs { - if colID == 0 { - colIDs[i] = mb.fetchColIDs[i] - } - } - newColID = makeTuple(colIDs, triggerColNew) + newColID = makeTuple(mb.updateColIDs, mb.fetchColIDs, triggerColNew) } return oldColID, newColID } @@ -273,6 +292,58 @@ func (mb *mutationBuilder) applyChangesFromTriggers( triggerScope.expr = f.ConstructProject(triggerScope.expr, projections, passThroughCols) } +// ensureNoRowsModifiedByTrigger adds a runtime check to the scope that ensures +// that the trigger function does not modify or filter any rows. This is +// necessary for cascading operations, where modifications made by the trigger +// function could cause constraint violations. +func (mb *mutationBuilder) ensureNoRowsModifiedByTrigger( + triggerScope *scope, + triggerName tree.Name, + eventType tree.TriggerEventType, + triggerFnColID opt.ColumnID, + oldColID, newColID opt.ColumnID, +) { + makeConstStr := func(str string) opt.ScalarExpr { + return mb.b.factory.ConstructConstVal(tree.NewDString(str), types.String) + } + expectedColID := newColID + if eventType == tree.TriggerEventDelete { + expectedColID = oldColID + } + // Construct a call to crdb_internal.plpgsql_raise with the error message. + severity := makeConstStr("ERROR") + message := mb.b.factory.ConstructConcat( + makeConstStr(fmt.Sprintf( + "trigger %s attempted to modify or filter a row in a cascade operation: ", triggerName)), + mb.b.factory.ConstructCast(mb.b.factory.ConstructVariable(expectedColID), types.String), + ) + detail := makeConstStr("changing the rows updated or deleted by a foreign-key cascade\n" + + " can cause constraint violations, and therefore is not allowed") + hint := makeConstStr("to enable this behavior (with risk of constraint violation), set\n" + + "the session variable 'unsafe_allow_triggers_modifying_cascades' to true") + code := makeConstStr(pgcode.TriggeredDataChangeViolation.String()) + raiseFn := mb.b.makePLpgSQLRaiseFn(memo.ScalarListExpr{severity, message, detail, hint, code}) + + // Build a CASE statement to raise the error if the trigger function modified + // the row. Add a barrier to ensure the check isn't removed or re-ordered with + // a filter. + // + // TODO(#133787): consider relaxing this check, for example, to ignore updates + // to non-FK columns. + f := mb.b.factory + check := f.ConstructCase(memo.TrueSingleton, + memo.ScalarListExpr{ + f.ConstructWhen( + f.ConstructIsNot(f.ConstructVariable(triggerFnColID), f.ConstructVariable(expectedColID)), + raiseFn, + ), + }, + f.ConstructNull(types.Int), + ) + mb.b.projectColWithMetadataName(triggerScope, "check-rows", types.Int, check) + triggerScope.expr = f.ConstructBarrier(triggerScope.expr) +} + // ============================================================================ // Row-level AFTER triggers // ============================================================================ diff --git a/pkg/sql/opt/optbuilder/update.go b/pkg/sql/opt/optbuilder/update.go index 081ba8b681a7..23c2fa73119b 100644 --- a/pkg/sql/opt/optbuilder/update.go +++ b/pkg/sql/opt/optbuilder/update.go @@ -108,7 +108,7 @@ func (b *Builder) buildUpdate(upd *tree.Update, inScope *scope) (outScope *scope mb.addUpdateCols(upd.Exprs) // Project row-level BEFORE triggers for UPDATE. - mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate) + mb.buildRowLevelBeforeTriggers(tree.TriggerEventUpdate, false /* cascade */) // Build the final update statement, including any returned expressions. if resultsNeeded(upd.Returning) { diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index af56d6c16ed5..e53024fdc0b7 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" @@ -811,3 +812,39 @@ func (b *Builder) projectColWithMetadataName( s.expr = b.factory.ConstructProject(s.expr, proj, passThroughCols) return col.id } + +// makeConstRaiseArgs builds the arguments for a crdb_internal.plpgsql_raise +// function call. +func (b *Builder) makeConstRaiseArgs( + severity, message, detail, hint, code string, +) memo.ScalarListExpr { + makeConstStr := func(str string) opt.ScalarExpr { + return b.factory.ConstructConstVal(tree.NewDString(str), types.String) + } + return memo.ScalarListExpr{ + makeConstStr(severity), + makeConstStr(message), + makeConstStr(detail), + makeConstStr(hint), + makeConstStr(code), + } +} + +// makePLpgSQLRaiseFn builds a call to the crdb_internal.plpgsql_raise builtin +// function, which implements the notice-sending behavior of RAISE statements. +func (b *Builder) makePLpgSQLRaiseFn(args memo.ScalarListExpr) opt.ScalarExpr { + const raiseFnName = "crdb_internal.plpgsql_raise" + fnProps, overloads := builtinsregistry.GetBuiltinProperties(raiseFnName) + if len(overloads) != 1 { + panic(errors.AssertionFailedf("expected one overload for %s", raiseFnName)) + } + return b.factory.ConstructFunction( + args, + &memo.FunctionPrivate{ + Name: raiseFnName, + Typ: types.Int, + Properties: fnProps, + Overload: &overloads[0], + }, + ) +} diff --git a/pkg/sql/opt/testutils/opttester/opt_tester.go b/pkg/sql/opt/testutils/opttester/opt_tester.go index 72c819b56de7..f232805da0e9 100644 --- a/pkg/sql/opt/testutils/opttester/opt_tester.go +++ b/pkg/sql/opt/testutils/opttester/opt_tester.go @@ -23,6 +23,7 @@ import ( "sort" "strconv" "strings" + "sync" "testing" "text/tabwriter" "time" @@ -591,8 +592,23 @@ func (ot *OptTester) RunCommand(tb testing.TB, d *datadriven.TestData) string { if ot.Flags.MaxStackBytes > 0 { originalMaxStack := debug.SetMaxStack(ot.Flags.MaxStackBytes) defer debug.SetMaxStack(originalMaxStack) + // Spawn a separate goroutine. A fresh stack makes tests using this + // setting more reliable. + var wg sync.WaitGroup + wg.Add(1) + var res string + go func() { + defer wg.Done() + res = ot.runCommandInternal(tb, d) + }() + wg.Wait() + return res + } else { + return ot.runCommandInternal(tb, d) } +} +func (ot *OptTester) runCommandInternal(tb testing.TB, d *datadriven.TestData) string { switch d.Cmd { case "exec-ddl": testCatalog, ok := ot.catalog.(*testcat.Catalog) diff --git a/pkg/sql/opt/xform/testdata/rules/select b/pkg/sql/opt/xform/testdata/rules/select index 4e5e57bee44c..4b1a2c6bfc73 100644 --- a/pkg/sql/opt/xform/testdata/rules/select +++ b/pkg/sql/opt/xform/testdata/rules/select @@ -2820,7 +2820,7 @@ CREATE TABLE t132669 ( # unnecessary recursion to trigger a stack overflow without having to make the # `IN` list below huge - triggering a stack overflow with Go's default max stack # size requires a list of ~1.6 million elements. -opt max-stack=125KB format=hide-all +opt max-stack=100kB skip-race format=hide-all SELECT * FROM t132669 WHERE a IN ( 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, diff --git a/pkg/sql/sessiondatapb/local_only_session_data.proto b/pkg/sql/sessiondatapb/local_only_session_data.proto index e94c25c465c3..789c87a466f5 100644 --- a/pkg/sql/sessiondatapb/local_only_session_data.proto +++ b/pkg/sql/sessiondatapb/local_only_session_data.proto @@ -551,6 +551,11 @@ message LocalOnlySessionData { // BypassPCRReaderCatalogAOST disables the AOST used by all user queries on // the PCR reader catalog. bool bypass_pcr_reader_catalog_aost = 141 [(gogoproto.customname) = "BypassPCRReaderCatalogAOST"]; + // UnsafeAllowTriggersModifyingCascades, when true, allows row-level BEFORE + // triggers to modify or filter rows that are being updated or deleted as + // part of a cascading foreign key action. This is unsafe because it can + // lead to constraint violations. + bool unsafe_allow_triggers_modifying_cascades = 142; /////////////////////////////////////////////////////////////////////////// // WARNING: consider whether a session parameter you're adding needs to // diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 7df5054f5505..fe88d9dd197f 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -3491,6 +3491,7 @@ var varGen = map[string]sessionVar{ }, GlobalDefault: globalTrue, }, + // CockroachDB extension. `bypass_pcr_reader_catalog_aost`: { Set: func(_ context.Context, m sessionDataMutator, s string) error { @@ -3506,6 +3507,23 @@ var varGen = map[string]sessionVar{ }, GlobalDefault: globalFalse, }, + + // CockroachDB extension. + `unsafe_allow_triggers_modifying_cascades`: { + GetStringVal: makePostgresBoolGetStringValFn(`unsafe_allow_triggers_modifying_cascades`), + Set: func(_ context.Context, m sessionDataMutator, s string) error { + b, err := paramparse.ParseBoolVar("unsafe_allow_triggers_modifying_cascades", s) + if err != nil { + return err + } + m.SetUnsafeAllowTriggersModifyingCascades(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) { + return formatBoolAsPostgresSetting(evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades), nil + }, + GlobalDefault: globalFalse, + }, } func ReplicationModeFromString(s string) (sessiondatapb.ReplicationMode, error) { diff --git a/pkg/ui/workspaces/cluster-ui/src/api/databaseDetailsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/databaseDetailsApi.ts deleted file mode 100644 index 1448bfc514db..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/api/databaseDetailsApi.ts +++ /dev/null @@ -1,613 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import moment from "moment-timezone"; - -import { IndexUsageStatistic, recommendDropUnusedIndex } from "../insights"; -import { getLogger, indexUnusedDuration, maybeError } from "../util"; - -import { Format, Identifier, QualifiedIdentifier } from "./safesql"; -import { - combineQueryErrors, - createSqlExecutionRequest, - executeInternalSql, - formatApiResult, - isMaxSizeError, - SqlApiQueryResponse, - SqlApiResponse, - SqlExecutionErrorMessage, - SqlExecutionRequest, - sqlResultsAreEmpty, - SqlExecutionResponse, - SqlStatement, - SqlTxnResult, - txnResultIsEmpty, -} from "./sqlApi"; -import { fromHexString, withTimeout } from "./util"; - -const { ZoneConfig } = cockroach.config.zonepb; -const { ZoneConfigurationLevel } = cockroach.server.serverpb; -type ZoneConfigType = cockroach.config.zonepb.ZoneConfig; -type ZoneConfigLevelType = cockroach.server.serverpb.ZoneConfigurationLevel; - -export type DatabaseDetailsReqParams = { - database: string; - csIndexUnusedDuration: string; -}; - -export type DatabaseDetailsSpanStatsReqParams = { - database: string; -}; - -export type DatabaseDetailsResponse = { - idResp: SqlApiQueryResponse; - grantsResp: SqlApiQueryResponse; - tablesResp: SqlApiQueryResponse; - zoneConfigResp: SqlApiQueryResponse; - stats?: DatabaseDetailsStats; - error?: SqlExecutionErrorMessage; -}; - -export type DatabaseDetailsSpanStatsResponse = { - spanStats: SqlApiQueryResponse; - error?: SqlExecutionErrorMessage; -}; - -function newDatabaseDetailsResponse(): DatabaseDetailsResponse { - return { - idResp: { database_id: "" }, - grantsResp: { grants: [] }, - tablesResp: { tables: [] }, - zoneConfigResp: { - zone_config: new ZoneConfig({ - inherited_constraints: true, - inherited_lease_preferences: true, - }), - zone_config_level: ZoneConfigurationLevel.CLUSTER, - }, - stats: { - replicaData: { - storeIDs: [], - }, - indexStats: { num_index_recommendations: 0 }, - }, - }; -} - -function newDatabaseDetailsSpanStatsResponse(): DatabaseDetailsSpanStatsResponse { - return { - spanStats: { - approximate_disk_bytes: 0, - live_bytes: 0, - total_bytes: 0, - }, - error: undefined, - }; -} - -// Database ID -type DatabaseIdRow = { - database_id: string; -}; - -const getDatabaseId: DatabaseDetailsQuery = { - createStmt: dbName => { - return { - sql: `SELECT crdb_internal.get_database_id($1) as database_id`, - arguments: [dbName], - }; - }, - addToDatabaseDetail: ( - txnResult: SqlTxnResult, - resp: DatabaseDetailsResponse, - ) => { - // Check that txn_result.rows[0].database_id is not null - if (!txnResultIsEmpty(txnResult) && txnResult.rows[0].database_id) { - resp.idResp.database_id = txnResult.rows[0].database_id; - } - if (txnResult.error) { - resp.idResp.error = txnResult.error; - } - }, - handleMaxSizeError: (_dbName, _response, _dbDetail) => { - return Promise.resolve(false); - }, -}; - -// Database Grants -type DatabaseGrantsResponse = { - grants: DatabaseGrantsRow[]; -}; - -type DatabaseGrantsRow = { - user: string; - privileges: string[]; -}; - -const getDatabaseGrantsQuery: DatabaseDetailsQuery = { - createStmt: dbName => { - return { - sql: Format( - `SELECT grantee as user, array_agg(privilege_type) as privileges - FROM %1.crdb_internal.cluster_database_privileges - WHERE database_name = $1 group by grantee`, - [new Identifier(dbName)], - ), - arguments: [dbName], - }; - }, - addToDatabaseDetail: ( - txnResult: SqlTxnResult, - resp: DatabaseDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - resp.grantsResp.grants = txnResult.rows; - if (txnResult.error) { - resp.grantsResp.error = txnResult.error; - } - } - }, - handleMaxSizeError: (_dbName, _response, _dbDetail) => { - return Promise.resolve(false); - }, -}; - -export type TableNameParts = { - // Raw unquoted, unescaped schema name. - schema: string; - // Raw unquoted, unescaped table name. - table: string; - - // qualifiedNameWithSchemaAndTable is the qualifed - // table name containing escaped, quoted schema and - // table name parts. - qualifiedNameWithSchemaAndTable: string; -}; - -// Database Tables -export type DatabaseTablesResponse = { - tables: TableNameParts[]; -}; - -type DatabaseTablesRow = { - table_schema: string; - table_name: string; -}; - -const getDatabaseTablesQuery: DatabaseDetailsQuery = { - createStmt: dbName => { - return { - sql: Format( - `SELECT table_schema, table_name - FROM %1.information_schema.tables - WHERE table_type NOT IN ('SYSTEM VIEW', 'VIEW') - ORDER BY table_name`, - [new Identifier(dbName)], - ), - }; - }, - addToDatabaseDetail: ( - txnResult: SqlTxnResult, - resp: DatabaseDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - if (!resp.tablesResp.tables) { - resp.tablesResp.tables = []; - } - - txnResult.rows.forEach(row => { - const escTableName = new QualifiedIdentifier([ - row.table_schema, - row.table_name, - ]).sqlString(); - resp.tablesResp.tables.push({ - schema: row.table_schema, - table: row.table_name, - qualifiedNameWithSchemaAndTable: escTableName, - }); - }); - } - if (txnResult.error) { - resp.tablesResp.error = txnResult.error; - } - }, - handleMaxSizeError: async ( - dbName: string, - response: SqlTxnResult, - dbDetail: DatabaseDetailsResponse, - ) => { - // Reset the error. It pulls all the names. - dbDetail.tablesResp.error = null; - let isMaxSize = true; - do { - const offsetQuery = { - sql: Format( - `SELECT table_schema, table_name - FROM %1.information_schema.tables - WHERE table_type NOT IN ('SYSTEM VIEW', 'VIEW') - ORDER BY table_name offset %2`, - [new Identifier(dbName), dbDetail.tablesResp.tables.length], - ), - }; - const req = createSqlExecutionRequest(dbName, [offsetQuery]); - const result = await executeInternalSql(req); - if (result?.execution?.txn_results?.length ?? 0 > 0) { - response = result.execution.txn_results[0]; - if (result.execution.txn_results[0].rows?.length ?? 0 > 0) { - getDatabaseTablesQuery.addToDatabaseDetail( - result.execution.txn_results[0], - dbDetail, - ); - } - } - - isMaxSize = isMaxSizeError(result.error?.message); - if (!isMaxSize) { - dbDetail.error = result.error; - } - } while (isMaxSize); - - return true; - }, -}; - -// Database Zone Config -type DatabaseZoneConfigResponse = { - zone_config: ZoneConfigType; - zone_config_level: ZoneConfigLevelType; -}; - -type DatabaseZoneConfigRow = { - zone_config_hex_string: string; -}; - -const getDatabaseZoneConfig: DatabaseDetailsQuery = { - createStmt: dbName => { - return { - sql: `SELECT - encode( - crdb_internal.get_zone_config( - (SELECT crdb_internal.get_database_id($1)) - ), - 'hex' - ) as zone_config_hex_string`, - arguments: [dbName], - }; - }, - addToDatabaseDetail: ( - txnResult: SqlTxnResult, - resp: DatabaseDetailsResponse, - ) => { - if ( - !txnResultIsEmpty(txnResult) && - // Check that txn_result.rows[0].zone_config_hex_string is not null - // and not empty. - txnResult.rows[0].zone_config_hex_string && - txnResult.rows[0].zone_config_hex_string.length !== 0 - ) { - const zoneConfigHexString = txnResult.rows[0].zone_config_hex_string; - // Try to decode the zone config bytes response. - try { - // Parse the bytes from the hex string. - const zoneConfigBytes = fromHexString(zoneConfigHexString); - // Decode the bytes using ZoneConfig protobuf. - resp.zoneConfigResp.zone_config = ZoneConfig.decode( - new Uint8Array(zoneConfigBytes), - ); - resp.zoneConfigResp.zone_config_level = ZoneConfigurationLevel.DATABASE; - } catch (e) { - getLogger().error( - `Database Details API - encountered an error decoding zone config string: ${zoneConfigHexString}`, - /* additional context */ undefined, - e, - ); - // Catch and assign the error if we encounter one decoding. - resp.zoneConfigResp.error = maybeError(e); - resp.zoneConfigResp.zone_config_level = ZoneConfigurationLevel.UNKNOWN; - } - } - if (txnResult.error) { - resp.zoneConfigResp.error = txnResult.error; - } - }, - handleMaxSizeError: (_dbName, _response, _dbDetail) => { - return Promise.resolve(false); - }, -}; - -// Database Stats -type DatabaseDetailsStats = { - replicaData: { - storeIDs: number[]; - error?: Error; - }; - indexStats: SqlApiQueryResponse; -}; - -export type DatabaseSpanStatsRow = { - approximate_disk_bytes: number; - live_bytes: number; - total_bytes: number; -}; - -function formatSpanStatsExecutionResult( - res: SqlExecutionResponse, -): DatabaseDetailsSpanStatsResponse { - const out = newDatabaseDetailsSpanStatsResponse(); - - if (res.execution.txn_results.length === 0) { - return out; - } - - const txnResult = res.execution.txn_results[0]; - - if (txnResult && txnResult.error) { - // Copy the SQLExecutionError and the SqlTransactionResult error. - out.error = res.error; - out.spanStats.error = txnResult.error; - } - if (txnResultIsEmpty(txnResult)) { - return out; - } - if (txnResult.rows.length === 1) { - const row = txnResult.rows[0]; - out.spanStats.approximate_disk_bytes = row.approximate_disk_bytes; - out.spanStats.live_bytes = row.live_bytes; - out.spanStats.total_bytes = row.total_bytes; - } else { - out.spanStats.error = new Error( - `DatabaseDetails - Span Stats, expected 1 row, got ${txnResult.rows.length}`, - ); - } - return out; -} - -type DatabaseReplicasRegionsRow = { - store_ids: number[]; -}; - -const getDatabaseReplicasAndRegions: DatabaseDetailsQuery = - { - createStmt: dbName => { - // This query is meant to retrieve the per-database set of store ids. - return { - sql: Format( - ` - SELECT array_agg(DISTINCT unnested_store_ids) AS store_ids - FROM [SHOW RANGES FROM DATABASE %1], unnest(replicas) AS unnested_store_ids -`, - [new Identifier(dbName)], - ), - }; - }, - addToDatabaseDetail: ( - txnResult: SqlTxnResult, - resp: DatabaseDetailsResponse, - ) => { - if (txnResult.error) { - resp.stats.replicaData.error = txnResult.error; - // We don't expect to have any rows for this query on error. - return; - } - if (!txnResultIsEmpty(txnResult)) { - resp.stats.replicaData.storeIDs = txnResult?.rows[0]?.store_ids ?? []; - } - }, - handleMaxSizeError: (_dbName, _response, _dbDetail) => { - return Promise.resolve(false); - }, - }; - -type DatabaseIndexUsageStatsResponse = { - num_index_recommendations?: number; -}; - -const getDatabaseIndexUsageStats: DatabaseDetailsQuery = { - createStmt: (dbName: string, csIndexUnusedDuration: string) => { - csIndexUnusedDuration = csIndexUnusedDuration ?? indexUnusedDuration; - return { - sql: Format( - `SELECT * FROM (SELECT - ti.created_at, - us.last_read, - us.total_reads, - '${csIndexUnusedDuration}' as unused_threshold, - '${csIndexUnusedDuration}'::interval as interval_threshold, - now() - COALESCE(us.last_read AT TIME ZONE 'UTC', COALESCE(ti.created_at, '0001-01-01')) as unused_interval - FROM %1.crdb_internal.index_usage_statistics AS us - JOIN %1.crdb_internal.table_indexes AS ti ON (us.index_id = ti.index_id AND us.table_id = ti.descriptor_id) - WHERE $1 != 'system' AND ti.is_unique IS false) - WHERE unused_interval > interval_threshold - ORDER BY total_reads DESC;`, - [new Identifier(dbName)], - ), - arguments: [dbName], - }; - }, - addToDatabaseDetail: ( - txnResult: SqlTxnResult, - resp: DatabaseDetailsResponse, - ) => { - txnResult.rows?.forEach(row => { - const rec = recommendDropUnusedIndex(row); - if (rec.recommend) { - resp.stats.indexStats.num_index_recommendations += 1; - } - }); - if (txnResult.error) { - resp.stats.indexStats.error = txnResult.error; - } - }, - handleMaxSizeError: (_dbName, _response, _dbDetail) => { - return Promise.resolve(false); - }, -}; - -export type DatabaseDetailsRow = - | DatabaseIdRow - | DatabaseGrantsRow - | DatabaseTablesRow - | DatabaseZoneConfigRow - | DatabaseSpanStatsRow - | DatabaseReplicasRegionsRow - | IndexUsageStatistic; - -type DatabaseDetailsQuery = { - createStmt: (dbName: string, csIndexUnusedDuration: string) => SqlStatement; - addToDatabaseDetail: ( - response: SqlTxnResult, - dbDetail: DatabaseDetailsResponse, - ) => void; - handleMaxSizeError: ( - dbName: string, - response: SqlTxnResult, - dbDetail: DatabaseDetailsResponse, - ) => Promise; -}; - -const databaseDetailQueries: DatabaseDetailsQuery[] = [ - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getDatabaseId, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getDatabaseGrantsQuery, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getDatabaseTablesQuery, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getDatabaseReplicasAndRegions, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getDatabaseIndexUsageStats, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getDatabaseZoneConfig, -]; - -export function createDatabaseDetailsReq( - params: DatabaseDetailsReqParams, -): SqlExecutionRequest { - return { - ...createSqlExecutionRequest( - params.database, - databaseDetailQueries.map(query => - query.createStmt(params.database, params.csIndexUnusedDuration), - ), - ), - separate_txns: true, - }; -} - -export function createDatabaseDetailsSpanStatsReq( - params: DatabaseDetailsSpanStatsReqParams, -): SqlExecutionRequest { - const statement = { - sql: `SELECT - sum(approximate_disk_bytes) as approximate_disk_bytes, - sum(live_bytes) as live_bytes, - sum(total_bytes) as total_bytes - FROM crdb_internal.tenant_span_stats((SELECT crdb_internal.get_database_id($1)))`, - arguments: [params.database], - }; - return createSqlExecutionRequest(params.database, [statement]); -} - -export async function getDatabaseDetailsSpanStats( - params: DatabaseDetailsSpanStatsReqParams, -) { - const req: SqlExecutionRequest = createDatabaseDetailsSpanStatsReq(params); - const sqlResp = await executeInternalSql(req); - const res = formatSpanStatsExecutionResult(sqlResp); - return formatApiResult( - res, - res.error, - "retrieving database span stats", - false, - ); -} - -export async function getDatabaseDetails( - params: DatabaseDetailsReqParams, - timeout?: moment.Duration, -): Promise> { - return withTimeout(fetchDatabaseDetails(params), timeout); -} - -async function fetchDatabaseDetails( - params: DatabaseDetailsReqParams, -): Promise> { - const detailsResponse: DatabaseDetailsResponse = newDatabaseDetailsResponse(); - const req: SqlExecutionRequest = createDatabaseDetailsReq(params); - const resp = await executeInternalSql(req); - const errs: Error[] = []; - resp.execution.txn_results.forEach(txnResult => { - if (txnResult.error) { - errs.push(txnResult.error); - } - const query: DatabaseDetailsQuery = - databaseDetailQueries[txnResult.statement - 1]; - query.addToDatabaseDetail(txnResult, detailsResponse); - }); - if (resp.error) { - if (isMaxSizeError(resp.error.message)) { - return fetchSeparatelyDatabaseDetails(params); - } - detailsResponse.error = resp.error; - } - - detailsResponse.error = combineQueryErrors(errs, detailsResponse.error); - return formatApiResult( - detailsResponse, - detailsResponse.error, - `retrieving database details information for database '${params.database}'`, - false, - ); -} - -async function fetchSeparatelyDatabaseDetails( - params: DatabaseDetailsReqParams, -): Promise> { - const detailsResponse: DatabaseDetailsResponse = newDatabaseDetailsResponse(); - const errs: Error[] = []; - for (const databaseDetailQuery of databaseDetailQueries) { - const req = createSqlExecutionRequest(params.database, [ - databaseDetailQuery.createStmt( - params.database, - params.csIndexUnusedDuration, - ), - ]); - const resp = await executeInternalSql(req); - if (sqlResultsAreEmpty(resp)) { - continue; - } - const txnResult = resp.execution.txn_results[0]; - if (txnResult.error) { - errs.push(txnResult.error); - } - databaseDetailQuery.addToDatabaseDetail(txnResult, detailsResponse); - - if (resp.error) { - const handleFailure = await databaseDetailQuery.handleMaxSizeError( - params.database, - txnResult, - detailsResponse, - ); - if (!handleFailure) { - detailsResponse.error = resp.error; - } - } - } - - detailsResponse.error = combineQueryErrors(errs, detailsResponse.error); - return formatApiResult( - detailsResponse, - detailsResponse.error, - `retrieving database details information for database '${params.database}'`, - false, - ); -} diff --git a/pkg/ui/workspaces/cluster-ui/src/api/databases/tableIndexesApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/databases/tableIndexesApi.ts index 6aa720704c1c..4c898f40c4d4 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/databases/tableIndexesApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/databases/tableIndexesApi.ts @@ -15,14 +15,16 @@ import { QualifiedIdentifier } from "../safesql"; import ResetIndexUsageStatsResponse = cockroach.server.serverpb.ResetIndexUsageStatsResponse; -type IndexRecType = cockroach.sql.IndexRecommendation.RecommendationType; - type GetTableIndexesRequest = { dbName: string; schemaName: string; tableName: string; }; +type IndexRecType = cockroach.sql.IndexRecommendation.RecommendationType; +export const IndexRecTypeEnum = + cockroach.sql.IndexRecommendation.RecommendationType; + export type IndexRecommendation = { id: number; type: IndexRecType; diff --git a/pkg/ui/workspaces/cluster-ui/src/api/index.ts b/pkg/ui/workspaces/cluster-ui/src/api/index.ts index 34780a74a0aa..7d22a32bd59c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/index.ts @@ -18,8 +18,6 @@ export * from "./sqlApi"; export * from "./tracezApi"; export * from "./databasesApi"; export * from "./eventsApi"; -export * from "./databaseDetailsApi"; -export * from "./tableDetailsApi"; export * from "./types"; export * from "./jobProfilerApi"; export * from "./txnInsightDetailsApi"; diff --git a/pkg/ui/workspaces/cluster-ui/src/api/tableDetailsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/tableDetailsApi.ts deleted file mode 100644 index 40a71f678b42..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/api/tableDetailsApi.ts +++ /dev/null @@ -1,650 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import moment from "moment-timezone"; - -import { IndexUsageStatistic, recommendDropUnusedIndex } from "../insights"; -import { getLogger, indexUnusedDuration, maybeError } from "../util"; - -import { Format, Identifier, Join, SQL } from "./safesql"; -import { - combineQueryErrors, - executeInternalSql, - formatApiResult, - LARGE_RESULT_SIZE, - LONG_TIMEOUT, - SqlApiQueryResponse, - SqlApiResponse, - SqlExecutionErrorMessage, - SqlExecutionRequest, - SqlStatement, - SqlTxnResult, - txnResultIsEmpty, -} from "./sqlApi"; -import { fromHexString, withTimeout } from "./util"; - -const { ZoneConfig } = cockroach.config.zonepb; -const { ZoneConfigurationLevel } = cockroach.server.serverpb; -type ZoneConfigType = cockroach.config.zonepb.ZoneConfig; -type ZoneConfigLevelType = cockroach.server.serverpb.ZoneConfigurationLevel; - -export function newTableDetailsResponse(): TableDetailsResponse { - return { - idResp: { table_id: "" }, - createStmtResp: { create_statement: "" }, - grantsResp: { grants: [] }, - schemaDetails: { columns: [], indexes: [] }, - zoneConfigResp: { - configure_zone_statement: "", - zone_config: new ZoneConfig({ - inherited_constraints: true, - inherited_lease_preferences: true, - }), - zone_config_level: ZoneConfigurationLevel.CLUSTER, - }, - heuristicsDetails: { stats_last_created_at: null }, - stats: { - spanStats: { - approximate_disk_bytes: 0, - live_bytes: 0, - total_bytes: 0, - range_count: 0, - live_percentage: 0, - }, - replicaData: { - storeIDs: [], - replicaCount: 0, - }, - indexStats: { - has_index_recommendations: false, - }, - }, - }; -} - -export type TableDetailsResponse = { - idResp: SqlApiQueryResponse; - createStmtResp: SqlApiQueryResponse; - grantsResp: SqlApiQueryResponse; - schemaDetails: SqlApiQueryResponse; - zoneConfigResp: SqlApiQueryResponse; - heuristicsDetails: SqlApiQueryResponse; - stats: TableDetailsStats; - error?: SqlExecutionErrorMessage; -}; - -// Table ID. -type TableIdRow = { - table_id: string; -}; - -const getTableId: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: `SELECT $1::regclass::oid as table_id`, - arguments: [escFullTableName.sqlString()], - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - resp.idResp.table_id = txnResult.rows[0].table_id; - } else { - txnResult.error = new Error("fetchTableId: unexpected empty results"); - } - if (txnResult.error) { - resp.idResp.error = txnResult.error; - } - }, -}; - -// Table create statement. -export type TableCreateStatementRow = { create_statement: string }; - -const getTableCreateStatement: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: Format(`SELECT create_statement FROM [SHOW CREATE %1]`, [ - escFullTableName, - ]), - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (txnResult.error) { - resp.createStmtResp.error = txnResult.error; - } - if (!txnResultIsEmpty(txnResult)) { - resp.createStmtResp.create_statement = txnResult.rows[0].create_statement; - } else if (!txnResult.error) { - txnResult.error = new Error( - "getTableCreateStatement: unexpected empty results", - ); - } - }, -}; - -// Table grants. -export type TableGrantsResponse = { - grants: TableGrantsRow[]; -}; - -type TableGrantsRow = { - user: string; - privileges: string[]; -}; - -const getTableGrants: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: Format( - `SELECT grantee as user, array_agg(privilege_type) as privileges - FROM [SHOW GRANTS ON TABLE %1] group by grantee`, - [escFullTableName], - ), - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - resp.grantsResp.grants = txnResult.rows; - } - if (txnResult.error) { - resp.grantsResp.error = txnResult.error; - } - }, -}; - -// Table schema details. -export type TableSchemaDetailsRow = { - columns: string[]; - indexes: string[]; -}; - -const getTableSchemaDetails: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: Format( - `WITH - columns AS (SELECT array_agg(distinct(column_name)) as unique_columns FROM [SHOW COLUMNS FROM %1]), - indexes AS (SELECT array_agg(distinct(index_name)) as unique_indexes FROM [SHOW INDEX FROM %1]) - SELECT - unique_columns as columns, - unique_indexes as indexes - FROM columns CROSS JOIN indexes`, - [escFullTableName], - ), - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - resp.schemaDetails.columns = txnResult.rows[0].columns; - resp.schemaDetails.indexes = txnResult.rows[0].indexes; - } - if (txnResult.error) { - resp.schemaDetails.error = txnResult.error; - } - }, -}; - -// Table zone config. -type TableZoneConfigResponse = { - configure_zone_statement: string; - zone_config: ZoneConfigType; - zone_config_level: ZoneConfigLevelType; -}; - -type TableZoneConfigStatementRow = { raw_config_sql: string }; - -const getTableZoneConfigStmt: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: Format( - `SELECT raw_config_sql FROM [SHOW ZONE CONFIGURATION FOR TABLE %1]`, - [escFullTableName], - ), - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - resp.zoneConfigResp.configure_zone_statement = - txnResult.rows[0].raw_config_sql; - } - if (txnResult.error) { - resp.zoneConfigResp.error = txnResult.error; - } - }, -}; - -type TableZoneConfigRow = { - database_zone_config_hex_string: string; - table_zone_config_hex_string: string; -}; - -const getTableZoneConfig: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: `SELECT - encode(crdb_internal.get_zone_config((SELECT crdb_internal.get_database_id($1))), 'hex') as database_zone_config_hex_string, - encode(crdb_internal.get_zone_config((SELECT $2::regclass::int)), 'hex') as table_zone_config_hex_string`, - arguments: [dbName, escFullTableName.sqlString()], - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - let hexString = ""; - let configLevel = ZoneConfigurationLevel.CLUSTER; - const row = txnResult.rows[0]; - // Check that database_zone_config_bytes is not null - // and not empty. - if ( - row.database_zone_config_hex_string && - row.database_zone_config_hex_string.length !== 0 - ) { - hexString = row.database_zone_config_hex_string; - configLevel = ZoneConfigurationLevel.DATABASE; - } - // Fall back to table_zone_config_bytes if we don't have database_zone_config_bytes. - if ( - hexString === "" && - row.table_zone_config_hex_string && - row.table_zone_config_hex_string.length !== 0 - ) { - hexString = row.table_zone_config_hex_string; - configLevel = ZoneConfigurationLevel.TABLE; - } - - // No zone configuration, return. - if (hexString === "") { - return; - } - - // Try to decode the zone config bytes response. - try { - // Parse the bytes from the hex string. - const zoneConfigBytes = fromHexString(hexString); - // Decode the bytes using ZoneConfig protobuf. - resp.zoneConfigResp.zone_config = ZoneConfig.decode( - new Uint8Array(zoneConfigBytes), - ); - resp.zoneConfigResp.zone_config_level = configLevel; - } catch (e) { - getLogger().error( - `Table Details API - encountered an error decoding zone config string: ${hexString}`, - /* additional context */ undefined, - e, - ); - // Catch and assign the error if we encounter one decoding. - resp.zoneConfigResp.error = maybeError(e); - resp.zoneConfigResp.zone_config_level = ZoneConfigurationLevel.UNKNOWN; - } - } - if (txnResult.error) { - resp.zoneConfigResp.error = txnResult.error; - } - }, -}; - -// Table heuristics details. -export type TableHeuristicDetailsRow = { - stats_last_created_at: moment.Moment; -}; - -const getTableHeuristicsDetails: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: Format( - `SELECT max(created) AS stats_last_created_at FROM [SHOW STATISTICS FOR TABLE %1]`, - [escFullTableName], - ), - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (!txnResultIsEmpty(txnResult)) { - resp.heuristicsDetails.stats_last_created_at = - txnResult.rows[0].stats_last_created_at; - } - if (txnResult.error) { - resp.heuristicsDetails.error = txnResult.error; - } - }, -}; - -// Table details stats. -type TableDetailsStats = { - spanStats: SqlApiQueryResponse; - replicaData: SqlApiQueryResponse; - indexStats: SqlApiQueryResponse; -}; - -// Table span stats. -export type TableSpanStatsRow = { - approximate_disk_bytes: number; - live_bytes: number; - total_bytes: number; - range_count: number; - live_percentage: number; -}; - -const getTableSpanStats: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: `SELECT - range_count, - approximate_disk_bytes, - live_bytes, - total_bytes, - live_percentage - FROM crdb_internal.tenant_span_stats( - (SELECT crdb_internal.get_database_id($1)), - (SELECT $2::regclass::int))`, - arguments: [dbName, escFullTableName.sqlString()], - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (txnResult && txnResult.error) { - resp.stats.spanStats.error = txnResult.error; - } - if (txnResultIsEmpty(txnResult)) { - return; - } - if (txnResult.rows.length === 1) { - resp.stats.spanStats = txnResult.rows[0]; - } else { - resp.stats.spanStats.error = new Error( - `getTableSpanStats: unexpected number of rows (expected 1, got ${txnResult.rows.length})`, - ); - } - }, -}; - -export type TableReplicaData = SqlApiQueryResponse<{ - storeIDs: number[]; - replicaCount: number; -}>; - -// Table replicas. -type TableReplicasRow = { - store_ids: number[]; - replica_count: number; -}; - -// This query is used to get the store ids for which replicas for a table -// are stored. -const getTableReplicaStoreIDs: TableDetailsQuery = { - createStmt: (dbName, tableName) => { - // (xinhaoz) Note that the table name provided here is in an escaped string of the - // form ., and is the result of formatting the retrieved schema and - // table name using QualifiedIdentifier. - // - // Currently we create a QualifiedIdentifier from the table name and schema - // immediately upon receiving the request, extracting the escaped string to use - // throughout the pages - this It can be unclear what format the table name is in - // at any given time and makes formatting less flexible. We should consider either storing - // the schema and table names separately and only creating the QualifiedIdentifier as - // needed, or otherwise indicate through better variable naming the table name format. - const sqlFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - return { - sql: Format( - ` - SELECT count(unnested) AS replica_count, array_agg(DISTINCT unnested) AS store_ids - FROM [SHOW RANGES FROM TABLE %1], unnest(replicas) AS unnested; -`, - [sqlFullTableName], - ), - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - if (txnResult.error) { - resp.stats.replicaData.error = txnResult.error; - // We don't expect to have any rows for this query on error. - return; - } - - // TODO #118957 (xinhaoz) Store IDs and node IDs cannot be used interchangeably. - if (!txnResultIsEmpty(txnResult)) { - resp.stats.replicaData.storeIDs = txnResult?.rows[0]?.store_ids ?? []; - resp.stats.replicaData.replicaCount = - txnResult?.rows[0]?.replica_count ?? 0; - } - }, -}; - -// Table index usage stats. -export type TableIndexUsageStats = { - has_index_recommendations: boolean; -}; - -const getTableIndexUsageStats: TableDetailsQuery = { - createStmt: (dbName, tableName, csIndexUnusedDuration) => { - const escFullTableName = Join( - [new Identifier(dbName), new SQL(tableName)], - new SQL("."), - ); - csIndexUnusedDuration = csIndexUnusedDuration ?? indexUnusedDuration; - return { - sql: Format( - `WITH tableId AS (SELECT $1::regclass::int as table_id) - SELECT * FROM (SELECT - ti.created_at, - us.last_read, - us.total_reads, - '${csIndexUnusedDuration}' as unused_threshold, - '${csIndexUnusedDuration}'::interval as interval_threshold, - now() - COALESCE(us.last_read AT TIME ZONE 'UTC', COALESCE(ti.created_at, '0001-01-01')) as unused_interval - FROM %1.crdb_internal.index_usage_statistics AS us - JOIN tableId ON us.table_id = tableId.table_id - JOIN %1.crdb_internal.table_indexes AS ti ON ( - us.index_id = ti.index_id AND - tableId.table_id = ti.descriptor_id - ) - WHERE $2 != 'system' AND ti.is_unique IS false) - WHERE unused_interval > interval_threshold - ORDER BY total_reads DESC`, - [new Identifier(dbName)], - ), - arguments: [escFullTableName.sqlString(), dbName], - }; - }, - addToTableDetail: ( - txnResult: SqlTxnResult, - resp: TableDetailsResponse, - ) => { - resp.stats.indexStats.has_index_recommendations = txnResult.rows?.some( - row => recommendDropUnusedIndex(row), - ); - if (txnResult.error) { - resp.stats.indexStats.error = txnResult.error; - } - }, -}; - -type TableDetailsQuery = { - createStmt: ( - dbName: string, - tableName: string, - csIndexUnusedDuration: string, - ) => SqlStatement; - addToTableDetail: ( - response: SqlTxnResult, - tableDetail: TableDetailsResponse, - ) => void; -}; - -export type TableDetailsRow = - | TableIdRow - | TableGrantsRow - | TableSchemaDetailsRow - | TableCreateStatementRow - | TableZoneConfigStatementRow - | TableHeuristicDetailsRow - | TableSpanStatsRow - | IndexUsageStatistic - | TableZoneConfigRow - | TableReplicasRow; - -const tableDetailQueries: TableDetailsQuery[] = [ - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableId, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableGrants, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableSchemaDetails, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableCreateStatement, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableZoneConfigStmt, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableHeuristicsDetails, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableSpanStats, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableIndexUsageStats, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableZoneConfig, - // eslint-disable-next-line @typescript-eslint/ban-ts-comment - // @ts-ignore - getTableReplicaStoreIDs, -]; - -export function createTableDetailsReq( - dbName: string, - tableName: string, - csIndexUnusedDuration: string, -): SqlExecutionRequest { - return { - execute: true, - statements: tableDetailQueries.map(query => - query.createStmt(dbName, tableName, csIndexUnusedDuration), - ), - max_result_size: LARGE_RESULT_SIZE, - timeout: LONG_TIMEOUT, - database: dbName, - separate_txns: true, - }; -} - -export type TableDetailsReqParams = { - database: string; - // Note: table name is expected in the following format: "schemaName"."tableName" - table: string; - csIndexUnusedDuration: string; -}; - -export async function getTableDetails( - params: TableDetailsReqParams, - timeout?: moment.Duration, -): Promise> { - return withTimeout( - fetchTableDetails( - params.database, - params.table, - params.csIndexUnusedDuration, - ), - timeout, - ); -} - -async function fetchTableDetails( - databaseName: string, - tableName: string, - csIndexUnusedDuration: string, -): Promise> { - const detailsResponse: TableDetailsResponse = newTableDetailsResponse(); - const req: SqlExecutionRequest = createTableDetailsReq( - databaseName, - tableName, - csIndexUnusedDuration, - ); - const resp = await executeInternalSql(req); - const errs: Error[] = []; - resp.execution.txn_results.forEach(txnResult => { - if (txnResult.error) { - errs.push(txnResult.error); - } - const query = tableDetailQueries[txnResult.statement - 1]; - query.addToTableDetail(txnResult, detailsResponse); - }); - if (resp.error) { - detailsResponse.error = resp.error; - } - - detailsResponse.error = combineQueryErrors(errs, detailsResponse.error); - return formatApiResult( - detailsResponse, - detailsResponse.error, - `retrieving table details information for table '${tableName}'`, - false, - ); -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsConnected.ts b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsConnected.ts deleted file mode 100644 index 1ecfb556a749..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsConnected.ts +++ /dev/null @@ -1,193 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { connect } from "react-redux"; -import { withRouter, RouteComponentProps } from "react-router-dom"; -import { Dispatch } from "redux"; - -import { databaseNameCCAttr } from "src/util/constants"; -import { getMatchParamByName } from "src/util/query"; - -import { deriveTableDetailsMemoized } from "../databases"; -import { Filters } from "../queryFilter"; -import { AppState } from "../store"; -import { actions as analyticsActions } from "../store/analytics"; -import { - selectDropUnusedIndexDuration, - selectIndexRecommendationsEnabled, -} from "../store/clusterSettings/clusterSettings.selectors"; -import { databaseDetailsReducer } from "../store/databaseDetails"; -const databaseDetailsActions = databaseDetailsReducer.actions; -import { - selectDatabaseDetailsGrantsSortSetting, - selectDatabaseDetailsTablesFiltersSetting, - selectDatabaseDetailsTablesSearchSetting, - selectDatabaseDetailsTablesSortSetting, - selectDatabaseDetailsViewModeSetting, -} from "../store/databaseDetails/databaseDetails.selectors"; -import { actions as tableDetailsActions } from "../store/databaseTableDetails"; -import { - actions as localStorageActions, - LocalStorageKeys, -} from "../store/localStorage"; -import { nodeRegionsByIDSelector } from "../store/nodes"; -import { actions as nodesActions } from "../store/nodes/nodes.reducer"; -import { selectIsTenant } from "../store/uiConfig"; - -import { - DatabaseDetailsPage, - DatabaseDetailsPageActions, - DatabaseDetailsPageData, -} from "./databaseDetailsPage"; -import { ViewMode } from "./types"; - -const mapStateToProps = ( - state: AppState, - props: RouteComponentProps, -): DatabaseDetailsPageData => { - const database = getMatchParamByName(props.match, databaseNameCCAttr); - const databaseDetails = state.adminUI?.databaseDetails; - const dbTables = - databaseDetails[database]?.data?.results.tablesResp.tables || []; - const nodeRegions = nodeRegionsByIDSelector(state); - const isTenant = selectIsTenant(state); - const nodeStatuses = state.adminUI?.nodes.data; - return { - loading: !!databaseDetails[database]?.inFlight, - loaded: !!databaseDetails[database]?.valid, - requestError: databaseDetails[database]?.lastError, - queryError: databaseDetails[database]?.data?.results?.error, - name: database, - showNodeRegionsColumn: Object.keys(nodeRegions).length > 1 && !isTenant, - viewMode: selectDatabaseDetailsViewModeSetting(state), - sortSettingTables: selectDatabaseDetailsTablesSortSetting(state), - sortSettingGrants: selectDatabaseDetailsGrantsSortSetting(state), - filters: selectDatabaseDetailsTablesFiltersSetting(state), - search: selectDatabaseDetailsTablesSearchSetting(state), - nodeRegions, - isTenant, - tables: deriveTableDetailsMemoized({ - dbName: database, - tables: dbTables, - tableDetails: state.adminUI?.tableDetails, - nodeRegions, - isTenant, - nodeStatuses, - }), - showIndexRecommendations: selectIndexRecommendationsEnabled(state), - csIndexUnusedDuration: selectDropUnusedIndexDuration(state), - }; -}; - -const mapDispatchToProps = ( - dispatch: Dispatch, -): DatabaseDetailsPageActions => ({ - refreshDatabaseDetails: (database: string, csIndexUnusedDuration: string) => { - dispatch( - databaseDetailsActions.refresh({ database, csIndexUnusedDuration }), - ); - }, - refreshTableDetails: ( - database: string, - table: string, - csIndexUnusedDuration: string, - ) => { - dispatch( - tableDetailsActions.refresh({ database, table, csIndexUnusedDuration }), - ); - }, - onViewModeChange: (viewMode: ViewMode) => { - dispatch( - analyticsActions.track({ - name: "View Mode Clicked", - page: "Database Details", - value: ViewMode[viewMode], - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_DETAILS_VIEW_MODE, - value: viewMode, - }), - ); - }, - onSortingTablesChange: (columnName: string, ascending: boolean) => { - dispatch( - analyticsActions.track({ - name: "Column Sorted", - page: "Database Details", - tableName: "Database Details", - columnName, - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_DETAILS_TABLES_PAGE_SORT, - value: { columnTitle: columnName, ascending: ascending }, - }), - ); - }, - onSortingGrantsChange: (columnName: string, ascending: boolean) => { - dispatch( - analyticsActions.track({ - name: "Column Sorted", - page: "Database Details", - tableName: "Database Details", - columnName, - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_DETAILS_GRANTS_PAGE_SORT, - value: { columnTitle: columnName, ascending: ascending }, - }), - ); - }, - onSearchComplete: (query: string) => { - dispatch( - analyticsActions.track({ - name: "Keyword Searched", - page: "Database Details", - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_DETAILS_TABLES_PAGE_SEARCH, - value: query, - }), - ); - }, - onFilterChange: (filters: Filters) => { - dispatch( - analyticsActions.track({ - name: "Filter Clicked", - page: "Database Details", - filterName: "filters", - value: filters.toString(), - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_DETAILS_TABLES_PAGE_FILTERS, - value: filters, - }), - ); - }, - refreshNodes: () => { - dispatch(nodesActions.refresh()); - }, -}); - -export const ConnectedDatabaseDetailsPage = withRouter( - connect< - DatabaseDetailsPageData, - DatabaseDetailsPageActions, - RouteComponentProps, - AppState - >( - mapStateToProps, - mapDispatchToProps, - )(DatabaseDetailsPage), -); diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.module.scss b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.module.scss deleted file mode 100644 index 87da5a113b28..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.module.scss +++ /dev/null @@ -1,90 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -@import "src/core/index.module"; - -.database-table { - &__col { - &-name { - font-family: $font-family--semi-bold; - } - - &-size, - &-range-count, - &-column-count, - &-index-count, - &-user-count { - width: 8em; - text-align: right; - padding-right: $spacing-x-large; - } - } - - &__no-result { - @include text--body-strong; - } - - &__cell-error { - display: inline-flex; - align-items: center; - gap: 10px; - svg { - fill: $colors--warning; - } - } -} - -.sorted-table { - width: 100%; -} - -.icon { - &__container { - display: inline-flex; - align-items: center; - } - - &--md { - height: 24px; - width: 24px; - margin-right: 12px; - } - - &--s { - height: 16px; - width: 16px; - margin-right: 10px; - } - - &--xxs { - height: 8px; - width: 8px; - } - - &--title { - fill: $colors--title; - } - - &--primary { - fill: $colors--primary-text; - } - - &--warning { - fill: $colors--functional-orange-3; - } -} - -.multiple-lines-info { - margin-bottom: 0px -} - -.bold { - font-weight: $font-weight--extra-bold; - color: $colors--neutral-8; -} - -.schema-name { - color: $colors--neutral-5; -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.spec.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.spec.tsx deleted file mode 100644 index 0a914bba063e..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.spec.tsx +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright 2024 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { expect } from "chai"; -import { shallow } from "enzyme"; -import * as H from "history"; -import React from "react"; - -import { defaultFilters } from "../queryFilter"; -import { indexUnusedDuration } from "../util"; - -import { - DatabaseDetailsPage, - DatabaseDetailsPageProps, -} from "./databaseDetailsPage"; -import { ViewMode } from "./types"; - -describe("DatabaseDetailsPage", () => { - const history = H.createHashHistory(); - const props: DatabaseDetailsPageProps = { - loading: false, - loaded: false, - requestError: undefined, - queryError: undefined, - name: "things", - search: null, - filters: defaultFilters, - nodeRegions: {}, - showNodeRegionsColumn: false, - viewMode: ViewMode.Tables, - sortSettingTables: { ascending: true, columnTitle: "name" }, - sortSettingGrants: { ascending: true, columnTitle: "name" }, - tables: [], - showIndexRecommendations: false, - csIndexUnusedDuration: indexUnusedDuration, - isTenant: false, - refreshTableDetails: () => {}, - refreshNodes: () => {}, - refreshDatabaseDetails: () => {}, - history: history, - location: history.location, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, - }; - it("should call refreshNodes if isTenant is false", () => { - const mockCallback = jest.fn(() => {}); - shallow(); - expect(mockCallback.mock.calls).to.have.length(1); - }); - it("should not call refreshNodes if isTenant is true", () => { - const mockCallback = jest.fn(() => {}); - shallow( - , - ); - expect(mockCallback.mock.calls).to.have.length(0); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx deleted file mode 100644 index 187839e0b3cb..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx +++ /dev/null @@ -1,191 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { storiesOf } from "@storybook/react"; -import * as H from "history"; -import random from "lodash/random"; -import uniq from "lodash/uniq"; -import moment from "moment-timezone"; -import React from "react"; - -import { defaultFilters } from "src/queryFilter"; -import { withBackground, withRouterProvider } from "src/storybook/decorators"; -import { - randomName, - randomRole, - randomTablePrivilege, -} from "src/storybook/fixtures"; -import { indexUnusedDuration } from "src/util/constants"; - -import { - DatabaseDetailsPage, - DatabaseDetailsPageDataTable, - DatabaseDetailsPageProps, -} from "./databaseDetailsPage"; -import { ViewMode } from "./types"; - -const history = H.createHashHistory(); - -const withLoadingIndicator: DatabaseDetailsPageProps = { - loading: true, - loaded: false, - requestError: undefined, - queryError: undefined, - showIndexRecommendations: false, - csIndexUnusedDuration: indexUnusedDuration, - name: randomName(), - tables: [], - viewMode: ViewMode.Tables, - sortSettingTables: { - ascending: false, - columnTitle: "name", - }, - sortSettingGrants: { - ascending: false, - columnTitle: "name", - }, - onSortingTablesChange: () => {}, - onSortingGrantsChange: () => {}, - refreshDatabaseDetails: () => {}, - refreshTableDetails: () => {}, - refreshNodes: () => {}, - search: null, - filters: defaultFilters, - nodeRegions: {}, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, -}; - -const withoutData: DatabaseDetailsPageProps = { - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - showIndexRecommendations: false, - csIndexUnusedDuration: indexUnusedDuration, - name: randomName(), - tables: [], - viewMode: ViewMode.Tables, - sortSettingTables: { - ascending: false, - columnTitle: "name", - }, - sortSettingGrants: { - ascending: false, - columnTitle: "name", - }, - onSortingTablesChange: () => {}, - onSortingGrantsChange: () => {}, - refreshDatabaseDetails: () => {}, - refreshTableDetails: () => {}, - refreshNodes: () => {}, - search: null, - filters: defaultFilters, - nodeRegions: {}, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, -}; - -function createTable(): DatabaseDetailsPageDataTable { - const roles = uniq(new Array(random(1, 3)).map(() => randomRole())); - const privileges = uniq( - new Array(random(1, 5)).map(() => randomTablePrivilege()), - ); - const columns = uniq(new Array(random(1, 5)).map(() => randomName())); - const indexes = uniq(new Array(random(1, 5)).map(() => randomName())); - - return { - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - name: { - qualifiedNameWithSchemaAndTable: "public.table", - schema: "public", - table: "table", - }, - qualifiedDisplayName: "public.table", - details: { - grants: { - roles, - privileges, - }, - schemaDetails: { - columns, - indexes, - }, - statsLastUpdated: { - stats_last_created_at: moment("0001-01-01T00:00:00Z"), - }, - indexStatRecs: { - has_index_recommendations: false, - }, - spanStats: { - live_percentage: random(0, 100), - live_bytes: random(0, 10000), - total_bytes: random(0, 10000), - approximate_disk_bytes: random(0, 10000), - range_count: random(0, 10000), - }, - }, - }; -} - -const withData: DatabaseDetailsPageProps = { - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - showIndexRecommendations: true, - csIndexUnusedDuration: indexUnusedDuration, - name: randomName(), - tables: [createTable()], - viewMode: ViewMode.Tables, - sortSettingTables: { - ascending: false, - columnTitle: "name", - }, - sortSettingGrants: { - ascending: false, - columnTitle: "name", - }, - onSortingTablesChange: () => {}, - onSortingGrantsChange: () => {}, - refreshDatabaseDetails: () => {}, - refreshTableDetails: () => {}, - refreshNodes: () => {}, - search: null, - filters: defaultFilters, - nodeRegions: {}, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, -}; - -storiesOf("Database Details Page", module) - .addDecorator(withRouterProvider) - .addDecorator(withBackground) - .add("with data", () => ) - .add("without data", () => ) - .add("with loading indicator", () => ( - - )); diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx deleted file mode 100644 index c8164c6541d5..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx +++ /dev/null @@ -1,949 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { InlineAlert } from "@cockroachlabs/ui-components"; -import { Tooltip } from "antd"; -import classNames from "classnames/bind"; -import React from "react"; -import { Link, RouteComponentProps } from "react-router-dom"; - -import { Dropdown, DropdownOption } from "src/dropdown"; -import { DatabaseIcon } from "src/icon/databaseIcon"; -import { StackIcon } from "src/icon/stackIcon"; -import { PageConfig, PageConfigItem } from "src/pageConfig"; -import { Pagination } from "src/pagination"; -import { - calculateActiveFilters, - defaultFilters, - Filter, - Filters, -} from "src/queryFilter"; -import { - ColumnDescriptor, - ISortedTablePagination, - SortedTable, - SortSetting, -} from "src/sortedtable"; -import sortableTableStyles from "src/sortedtable/sortedtable.module.scss"; -import { UIConfigState } from "src/store"; -import { TableStatistics } from "src/tableStatistics"; -import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses"; -import { DATE_FORMAT, EncodeDatabaseTableUri } from "src/util/format"; - -import { Anchor } from "../anchor"; -import { - isMaxSizeError, - SqlApiQueryResponse, - SqlExecutionErrorMessage, - TableHeuristicDetailsRow, - TableIndexUsageStats, - TableNameParts, - TableSchemaDetailsRow, - TableSpanStatsRow, -} from "../api"; -import { LoadingCell } from "../databases"; -import { Loading } from "../loading"; -import { Search } from "../search"; -import LoadingError from "../sqlActivity/errorComponent"; -import { Timestamp, Timezone } from "../timestamp"; -import { mvccGarbage, syncHistory, unique } from "../util"; - -import styles from "./databaseDetailsPage.module.scss"; -import { - DbDetailsBreadcrumbs, - DiskSizeCell, - IndexesCell, - MVCCInfoCell, - TableNameCell, -} from "./tableCells"; -import { ViewMode } from "./types"; - -const cx = classNames.bind(styles); -const sortableTableCx = classNames.bind(sortableTableStyles); - -// We break out separate interfaces for some of the nested objects in our data -// both so that they can be available as SortedTable rows and for making -// (typed) test assertions on narrower slices of the data. -// -// The loading and loaded flags help us know when to dispatch the appropriate -// refresh actions. -// -// The overall structure is: -// -// interface DatabaseDetailsPageData { -// loading: boolean; -// loaded: boolean; -// lastError: Error; -// name: string; -// sortSettingTables: SortSetting; -// sortSettingGrants: SortSetting; -// search: string; -// filters: Filters; -// nodeRegions: { [nodeId: string]: string }; -// isTenant: boolean; -// viewMode: ViewMode; -// tables: { // DatabaseDetailsPageDataTable[] -// name: string; -// details: { // DatabaseDetailsPageDataTableDetails -// loading: boolean; -// loaded: boolean; -// lastError: Error; -// columnCount: number; -// indexCount: number; -// userCount: number; -// roles: string[]; -// grants: string[]; -// replicationSizeInBytes: number; -// rangeCount: number; -// nodes: number[]; -// nodesByRegionString: string; -// }; -// }[]; -// } -export interface DatabaseDetailsPageData { - loading: boolean; - loaded: boolean; - // Request error when getting table names. - requestError?: Error; - // Query error when getting table names. - queryError?: SqlExecutionErrorMessage; - name: string; - tables: DatabaseDetailsPageDataTable[]; - sortSettingTables: SortSetting; - sortSettingGrants: SortSetting; - search: string; - filters: Filters; - nodeRegions: { [nodeId: string]: string }; - isTenant?: UIConfigState["isTenant"]; - viewMode: ViewMode; - showNodeRegionsColumn?: boolean; - showIndexRecommendations: boolean; - csIndexUnusedDuration: string; -} - -export interface DatabaseDetailsPageDataTable { - name: TableNameParts; - // Display name containing unquoted, unescaped schema and table name parts. - qualifiedDisplayName: string; - loading: boolean; - loaded: boolean; - // Request error when getting table details. - requestError?: Error; - // Query error when getting table details. - queryError?: SqlExecutionErrorMessage; - details: DatabaseDetailsPageDataTableDetails; -} - -interface GrantsData { - roles: string[]; - privileges: string[]; -} - -export interface DatabaseDetailsPageDataTableDetails { - schemaDetails?: SqlApiQueryResponse; - grants: SqlApiQueryResponse; - statsLastUpdated?: SqlApiQueryResponse; - indexStatRecs?: SqlApiQueryResponse; - spanStats?: SqlApiQueryResponse; - // Array of node IDs used to unambiguously filter by node and region. - nodes?: number[]; - // String of nodes grouped by region in alphabetical order, e.g. - // regionA(n1,n2), regionB(n3). Used for display in the table's - // "Regions/Nodes" column. - nodesByRegionString?: string; -} - -export interface DatabaseDetailsPageActions { - refreshDatabaseDetails: ( - database: string, - csIndexUnusedDuration: string, - ) => void; - refreshTableDetails: ( - database: string, - table: string, - csIndexUnusedDuration: string, - ) => void; - onFilterChange?: (value: Filters) => void; - onSearchComplete?: (query: string) => void; - onSortingTablesChange?: (columnTitle: string, ascending: boolean) => void; - onSortingGrantsChange?: (columnTitle: string, ascending: boolean) => void; - onViewModeChange?: (viewMode: ViewMode) => void; - refreshNodes: () => void; -} - -export type DatabaseDetailsPageProps = DatabaseDetailsPageData & - DatabaseDetailsPageActions & - RouteComponentProps; - -interface DatabaseDetailsPageState { - pagination: ISortedTablePagination; - filters?: Filters; - activeFilters?: number; -} - -const tablePageSize = 20; -const disableTableSortSize = tablePageSize * 2; - -class DatabaseSortedTable extends SortedTable {} - -// filterBySearchQuery returns true if the search query matches the database name. -function filterBySearchQuery( - table: DatabaseDetailsPageDataTable, - search: string, -): boolean { - const matchString = table.qualifiedDisplayName.toLowerCase(); - - if (search.startsWith('"') && search.endsWith('"')) { - search = search.substring(1, search.length - 1); - - return matchString.includes(search); - } - - return search - .toLowerCase() - .split(" ") - .every(val => matchString.includes(val)); -} - -export class DatabaseDetailsPage extends React.Component< - DatabaseDetailsPageProps, - DatabaseDetailsPageState -> { - constructor(props: DatabaseDetailsPageProps) { - super(props); - this.state = { - pagination: { - current: 1, - pageSize: 20, - }, - }; - - const { history } = this.props; - const searchParams = new URLSearchParams(history.location.search); - - // View Mode. - const view = searchParams.get("viewMode") || undefined; - let viewMode = ViewMode.Tables; - if (view === ViewMode.Grants.toString()) { - viewMode = ViewMode.Grants; - } - if ( - this.props.onViewModeChange && - view && - viewMode !== this.props.viewMode - ) { - this.props.onViewModeChange(viewMode); - } - - // Sort Settings. - const ascending = (searchParams.get("ascending") || undefined) === "true"; - const columnTitle = searchParams.get("columnTitle") || undefined; - const sortSetting = - viewMode === ViewMode.Tables - ? this.props.sortSettingTables - : this.props.sortSettingGrants; - const onSortingChange = - viewMode === ViewMode.Tables - ? this.props.onSortingTablesChange - : this.props.onSortingGrantsChange; - - if ( - onSortingChange && - columnTitle && - (sortSetting.columnTitle !== columnTitle || - sortSetting.ascending !== ascending) - ) { - onSortingChange(columnTitle, ascending); - } - } - - componentDidMount(): void { - if (!this.props.isTenant) { - this.props.refreshNodes(); - } - if (!this.props.loaded && !this.props.loading && !this.props.requestError) { - this.props.refreshDatabaseDetails( - this.props.name, - this.props.csIndexUnusedDuration, - ); - } else { - // If the props are already loaded then componentDidUpdate - // will not get called so call refresh to make sure details - // are loaded - this.refresh(); - } - } - - componentDidUpdate( - prevProps: Readonly, - prevState: Readonly, - ): void { - if (this.shouldRefreshTableInformation(prevState, prevProps)) { - this.refresh(); - } - } - - private shouldRefreshTableInformation( - prevState: Readonly, - prevProps: Readonly, - ): boolean { - // No new tables to update - if ( - !this.props.tables || - this.props.tables.length === 0 || - this.props.tables.every(x => x.loaded || x.loading) - ) { - return false; - } - - if (this.state.pagination.current !== prevState.pagination.current) { - return true; - } - - if (prevProps && this.props.search !== prevProps.search) { - return true; - } - - const filteredTables = this.filteredDatabaseTables(); - for ( - let i = 0; - i < filteredTables.length && i < disableTableSortSize; - i++ - ) { - const table = filteredTables[i]; - if (!table.loaded && !table.loading && table.requestError == null) { - return true; - } - } - - return false; - } - - private refresh(): void { - // Load everything by default - let filteredTables = this.props.tables; - - // Loading only the first page. If more than 2 page of tables - // exists the sort will be disabled. - if (this.props.tables.length > disableTableSortSize) { - const startIndex = - this.state.pagination.pageSize * (this.state.pagination.current - 1); - // Result maybe filtered so get db names from filtered results - if (this.props.search && this.props.search.length > 0) { - filteredTables = this.filteredDatabaseTables(); - } - - if (!filteredTables || filteredTables.length === 0) { - return; - } - - // Only load the first page - filteredTables = filteredTables.slice( - startIndex, - startIndex + disableTableSortSize, - ); - } - - filteredTables.forEach(table => { - if (!table.loaded && !table.loading && table.requestError === undefined) { - this.props.refreshTableDetails( - this.props.name, - table.name.qualifiedNameWithSchemaAndTable, - this.props.csIndexUnusedDuration, - ); - } - }); - } - - private changePage(current: number) { - this.setState({ pagination: { ...this.state.pagination, current } }); - } - - changeSortSetting = (ss: SortSetting): void => { - syncHistory( - { - ascending: ss.ascending.toString(), - columnTitle: ss.columnTitle, - }, - this.props.history, - ); - const onSortingChange = - this.props.viewMode === ViewMode.Tables - ? this.props.onSortingTablesChange - : this.props.onSortingGrantsChange; - - if (onSortingChange) { - onSortingChange(ss.columnTitle, ss.ascending); - } - }; - - onClearSearchField = (): void => { - if (this.props.onSearchComplete) { - this.props.onSearchComplete(""); - } - - syncHistory( - { - q: undefined, - }, - this.props.history, - ); - }; - - onClearFilters = (): void => { - if (this.props.onFilterChange) { - this.props.onFilterChange(defaultFilters); - } - - this.setState({ - filters: defaultFilters, - activeFilters: 0, - }); - - this.resetPagination(); - syncHistory( - { - regions: undefined, - nodes: undefined, - }, - this.props.history, - ); - }; - - onSubmitSearchField = (search: string): void => { - if (this.props.onSearchComplete) { - this.props.onSearchComplete(search); - } - - this.resetPagination(); - syncHistory( - { - q: search, - }, - this.props.history, - ); - }; - - onSubmitFilters = (filters: Filters): void => { - if (this.props.onFilterChange) { - this.props.onFilterChange(filters); - } - - this.setState({ - filters: filters, - activeFilters: calculateActiveFilters(filters), - }); - - this.resetPagination(); - syncHistory( - { - regions: filters.regions, - nodes: filters.nodes, - }, - this.props.history, - ); - }; - - resetPagination = (): void => { - this.setState(prevState => { - return { - pagination: { - current: 1, - pageSize: prevState.pagination.pageSize, - }, - }; - }); - }; - - // Returns a list of database tables to the display based on input from the - // search box and the applied filters. - filteredDatabaseTables = (): DatabaseDetailsPageDataTable[] => { - const { search, tables, filters, nodeRegions } = this.props; - - const regionsSelected = - filters.regions?.length > 0 ? filters.regions.split(",") : []; - const nodesSelected = - filters.nodes?.length > 0 ? filters.nodes.split(",") : []; - - // Avoid the loop if no filters/search are applied - if ( - (!search || search.length === 0) && - regionsSelected.length === 0 && - nodesSelected.length === 0 - ) { - return tables; - } - - return tables - .filter(table => (search ? filterBySearchQuery(table, search) : true)) - .filter(table => { - if (regionsSelected.length === 0 && nodesSelected.length === 0) - return true; - - let foundRegion = regionsSelected.length === 0; - let foundNode = nodesSelected.length === 0; - - table.details.nodes?.forEach(node => { - const n = node?.toString() || ""; - if (foundRegion || regionsSelected.includes(nodeRegions[n])) { - foundRegion = true; - } - if (foundNode || nodesSelected.includes("n" + n)) { - foundNode = true; - } - if (foundNode && foundRegion) return true; - }); - - return foundRegion && foundNode; - }); - }; - - private changeViewMode(viewMode: ViewMode) { - syncHistory( - { - viewMode: viewMode.toString(), - }, - this.props.history, - ); - if (this.props.onViewModeChange) { - this.props.onViewModeChange(viewMode); - } - } - - private columns(): ColumnDescriptor[] { - switch (this.props.viewMode) { - case ViewMode.Tables: - return this.columnsForTablesViewMode(); - case ViewMode.Grants: - return this.columnsForGrantsViewMode(); - default: - throw new Error(`Unknown view mode ${this.props.viewMode}`); - } - } - - private columnsForTablesViewMode(): ColumnDescriptor[] { - return ( - [ - { - title: ( - - Tables - - ), - cell: table => , - sort: table => table.name, - className: cx("database-table__col-name"), - name: "name", - }, - { - title: ( - - Replication Size - - ), - cell: table => , - sort: table => table.details.spanStats?.approximate_disk_bytes, - className: cx("database-table__col-size"), - name: "replicationSize", - }, - { - title: ( - - Ranges - - ), - cell: table => ( - - {table.details.spanStats?.range_count} - - ), - sort: table => table.details.spanStats?.range_count, - className: cx("database-table__col-range-count"), - name: "rangeCount", - }, - { - title: ( - - Columns - - ), - cell: table => ( - - {table.details.schemaDetails?.columns?.length} - - ), - sort: table => table.details.schemaDetails?.columns?.length, - className: cx("database-table__col-column-count"), - name: "columnCount", - }, - { - title: ( - - Indexes - - ), - cell: table => ( - - ), - sort: table => table.details.schemaDetails?.indexes?.length, - className: cx("database-table__col-index-count"), - name: "indexCount", - }, - { - title: ( - - Regions - - ), - cell: table => ( - - {table.details.nodesByRegionString && - table.details.nodesByRegionString.length > 0 - ? table.details.nodesByRegionString - : null} - - ), - sort: table => table.details.nodesByRegionString, - className: cx("database-table__col--regions"), - name: "regions", - showByDefault: this.props.showNodeRegionsColumn, - hideIfTenant: true, - }, - { - title: ( - - {"% of total uncompressed logical data that has not been modified (updated or deleted). " + - "A low percentage can cause statements to scan more data ("} - - MVCC values - - {") than required, which can reduce performance."} - - } - > - % of Live Data - - ), - cell: table => ( - - {table.details.spanStats ? ( - - ) : null} - - ), - sort: table => table.details.spanStats?.live_percentage, - name: "livePercentage", - }, - { - title: ( - - Table Stats Last Updated - - ), - cell: table => ( - - - - ), - sort: table => table.details.statsLastUpdated, - className: cx("database-table__col--table-stats"), - name: "tableStatsUpdated", - }, - ] as ColumnDescriptor[] - ).filter(c => c.showByDefault !== false); - } - - private columnsForGrantsViewMode(): ColumnDescriptor[] { - return [ - { - title: ( - - Tables - - ), - cell: table => ( - - - {table.name.schema}. - {table.name.table} - - ), - sort: table => table.qualifiedDisplayName, - className: cx("database-table__col-name"), - name: "name", - }, - { - title: ( - - Users - - ), - cell: table => ( - - {table.details.grants?.roles.length} - - ), - sort: table => table.details.grants?.roles.length, - className: cx("database-table__col-user-count"), - name: "userCount", - }, - { - title: ( - - Roles - - ), - cell: table => ( - - {table.details.grants?.roles.join(", ")} - - ), - sort: table => table.details.grants?.roles.join(", "), - className: cx("database-table__col-roles"), - name: "roles", - }, - { - title: ( - - Grants - - ), - cell: table => ( - - {table.details.grants?.privileges.join(", ")} - - ), - sort: table => table.details.grants?.privileges?.join(", "), - className: cx("database-table__col-grants"), - name: "grants", - }, - ]; - } - - private static viewOptions(): DropdownOption[] { - return [ - { - name: "Tables", - value: ViewMode.Tables, - }, - { - name: "Grants", - value: ViewMode.Grants, - }, - ]; - } - - render(): React.ReactElement { - const { search, filters, isTenant, nodeRegions } = this.props; - - const tablesToDisplay = this.filteredDatabaseTables(); - const activeFilters = calculateActiveFilters(filters); - - const nodes = Object.keys(nodeRegions) - .map(n => Number(n)) - .sort(); - - const regions = unique(Object.values(nodeRegions)); - - const sortSetting = - this.props.viewMode === ViewMode.Tables - ? this.props.sortSettingTables - : this.props.sortSettingGrants; - - const showNodes = !isTenant && nodes.length > 1; - const showRegions = regions.length > 1; - - // Only show the filter component when the viewMode is Tables and if at - // least one of drop-down is shown. - const filterComponent = - this.props.viewMode === ViewMode.Tables && (showNodes || showRegions) ? ( - - "n" + n?.toString())} - activeFilters={activeFilters} - filters={defaultFilters} - onSubmitFilters={this.onSubmitFilters} - showNodes={showNodes} - showRegions={showRegions} - /> - - ) : ( - <> - ); - - return ( -
-
- -

- - {this.props.name} -

-
- - - - - View: {this.props.viewMode} - - - - - - {filterComponent} - - -
- - - LoadingError({ - statsType: "databases", - error: this.props.requestError, - }) - } - > - {isMaxSizeError(this.props.queryError?.message) && ( - - Not all tables are displayed because the maximum number of - tables was reached in the console.  - - } - /> - )} - - - This database has no tables. -
- } - /> - - - - - ); - } -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/index.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/index.tsx deleted file mode 100644 index 826528319c16..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/index.tsx +++ /dev/null @@ -1,8 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -export * from "./databaseDetailsPage"; -export * from "./databaseDetailsConnected"; -export * from "./types"; diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/tableCells.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/tableCells.tsx deleted file mode 100644 index 4b82b825efe8..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/tableCells.tsx +++ /dev/null @@ -1,192 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { Caution } from "@cockroachlabs/icons"; -import { Tooltip } from "antd"; -import classNames from "classnames/bind"; -import React, { useContext } from "react"; -import { Link } from "react-router-dom"; - -import { Breadcrumbs } from "../breadcrumbs"; -import { CockroachCloudContext } from "../contexts"; -import { LoadingCell, getNetworkErrorMessage } from "../databases"; -import { CaretRight } from "../icon/caretRight"; -import { DatabaseIcon } from "../icon/databaseIcon"; -import { - EncodeDatabaseTableUri, - EncodeDatabaseUri, - EncodeUriName, - getMatchParamByName, - schemaNameAttr, -} from "../util"; -import * as format from "../util/format"; - -import { - DatabaseDetailsPageDataTable, - DatabaseDetailsPageDataTableDetails, - DatabaseDetailsPageProps, -} from "./databaseDetailsPage"; -import styles from "./databaseDetailsPage.module.scss"; -import { ViewMode } from "./types"; - -const cx = classNames.bind(styles); - -export const DiskSizeCell = ({ - table, -}: { - table: DatabaseDetailsPageDataTable; -}): JSX.Element => { - return ( - <> - { - - {table.details?.spanStats?.approximate_disk_bytes - ? format.Bytes(table.details?.spanStats?.approximate_disk_bytes) - : null} - - } - - ); -}; - -export const TableNameCell = ({ - table, - dbDetails, -}: { - table: DatabaseDetailsPageDataTable; - dbDetails: DatabaseDetailsPageProps; -}): JSX.Element => { - const isCockroachCloud = useContext(CockroachCloudContext); - let linkURL = ""; - if (isCockroachCloud) { - linkURL = `${location.pathname}/${EncodeUriName( - getMatchParamByName(dbDetails.match, schemaNameAttr), - )}/${EncodeUriName(table.name.qualifiedNameWithSchemaAndTable)}`; - if (dbDetails.viewMode === ViewMode.Grants) { - linkURL += `?viewMode=${ViewMode.Grants}`; - } - } else { - linkURL = EncodeDatabaseTableUri( - dbDetails.name, - table.name.qualifiedNameWithSchemaAndTable, - ); - if (dbDetails.viewMode === ViewMode.Grants) { - linkURL += `?tab=grants`; - } - } - let icon = ; - if (table.requestError || table.queryError) { - icon = ( - - - - ); - } - return ( - - {icon} - {table.name.schema}. - {table.name.table} - - ); -}; - -export const IndexesCell = ({ - table, - showIndexRecommendations, -}: { - table: DatabaseDetailsPageDataTable; - showIndexRecommendations: boolean; -}): JSX.Element => { - const elem = ( - <> - { - - {table.details?.schemaDetails?.indexes?.length} - - } - - ); - // If index recommendations are not enabled or we don't have any index recommendations, - // just return the number of indexes. - if ( - !table.details.indexStatRecs?.has_index_recommendations || - !showIndexRecommendations - ) { - return elem; - } - // Display an icon indicating we have index recommendations next to the number of indexes. - return ( -
- - - - {elem} -
- ); -}; - -export const MVCCInfoCell = ({ - details, -}: { - details: DatabaseDetailsPageDataTableDetails; -}): JSX.Element => { - return ( - <> -

- {format.Percentage(details?.spanStats?.live_percentage, 1, 1)} -

-

- - {format.Bytes(details?.spanStats?.live_bytes)} - {" "} - live data /{" "} - - {format.Bytes(details?.spanStats?.total_bytes)} - - {" total"} -

- - ); -}; - -export const DbDetailsBreadcrumbs = ({ dbName }: { dbName: string }) => { - const isCockroachCloud = useContext(CockroachCloudContext); - return ( - } - /> - ); -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/types.ts b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/types.ts deleted file mode 100644 index d3259279847c..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/types.ts +++ /dev/null @@ -1,9 +0,0 @@ -// Copyright 2024 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -export enum ViewMode { - Tables = "Tables", - Grants = "Grants", -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.module.scss b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.module.scss deleted file mode 100644 index 5ff62dc75504..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.module.scss +++ /dev/null @@ -1,171 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -@import "src/core/index.module"; - -.tab-area { - margin-bottom: $spacing-large; -} - -.database-table-page { - &__indexes { - &--value { - text-align: right; - } - } -} - -.sorted-table { - width: 100%; -} - -.tab-pane { - margin-left: 1px; -} - -.summary-card { - h4 { - @include text--body-strong; - color: $colors--neutral-7; - } - - p { - @include text--body; - overflow-wrap: anywhere; - } - - &__error-cell{ - display: inline-flex; - align-items: center; - gap: 10px; - svg { - fill: $colors--warning; - } - } -} - -.icon { - &__container { - display: inline; - align-items: center; - } - - &--md { - height: 24px; - width: 24px; - margin-right: 12px; - } - - &--s { - height: 16px; - width: 16px; - margin-right: 10px; - } - - &--xxs { - height: 8px; - width: 8px; - } - - &--title { - fill: $colors--title; - } - - &--primary { - fill: $colors--primary-text; - } -} - -.index-stats { - &__summary-card { - width: 75%; - padding-right: 9px; - padding-left: 9px; - margin-right: -9px; - margin-left: 9px; - } - - &__header { - align-items: baseline; - display: flex; - flex-direction: row; - justify-content: space-between; - padding: $spacing-medium $spacing-medium 0; - } - - &__reset-info { - display: flex; - flex-direction: row; - } - - &__last-reset { - color: $colors--neutral-6; - margin-right: $spacing-base; - } - - &__reset-btn { - color: $colors--primary-blue-3; - - &:hover { - color: $colors--primary-blue-3; - text-decoration: underline; - } - } - - &-table { - &__col { - &-indexes { - font-family: $font-family--semi-bold; - width: 30em; - overflow-wrap: anywhere; - } - &-last-used { - width: 25em; - } - } - } -} - -.index-recommendations { - &-icon { - &__exist { - height: 10px; - width: 20px; - fill: $colors--warning; - } - - &__none { - height: 10px; - width: 20px; - fill: $colors--neutral-11; - } - } - - &-text { - &__border { - border-bottom: 1px dashed $colors--disabled; - } - &__tooltip-anchor { - a { - font-size: $font-size--small; - color: $colors--white; - text-decoration: underline; - &:hover { - opacity: 0.8; - color: $colors--white; - } - } - } - } -} - -.bold { - font-weight: $font-weight--extra-bold; - color: $colors--neutral-8; -} - -.row-spaced { - margin-top: 20px; -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.spec.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.spec.tsx deleted file mode 100644 index 339c74bd3b02..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.spec.tsx +++ /dev/null @@ -1,81 +0,0 @@ -// Copyright 2024 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { expect } from "chai"; -import { shallow } from "enzyme"; -import * as H from "history"; -import React from "react"; - -import { util } from "../index"; -import { indexUnusedDuration } from "../util"; - -import { DatabaseTablePage, DatabaseTablePageProps } from "./databaseTablePage"; - -describe("DatabaseTablePage", () => { - const history = H.createHashHistory(); - const props: DatabaseTablePageProps = { - databaseName: "DATABASE", - name: "TABLE", - schemaName: "", - showNodeRegionsSection: false, - details: { - loading: false, - loaded: false, - requestError: undefined, - queryError: undefined, - createStatement: undefined, - replicaData: undefined, - spanStats: undefined, - indexData: undefined, - grants: { - all: [], - error: undefined, - }, - statsLastUpdated: undefined, - nodesByRegionString: "", - }, - automaticStatsCollectionEnabled: true, - indexUsageStatsEnabled: true, - showIndexRecommendations: true, - csIndexUnusedDuration: indexUnusedDuration, - hasAdminRole: false, - indexStats: { - loading: false, - loaded: false, - lastError: undefined, - stats: [], - lastReset: util.minDate, - }, - refreshSettings: () => {}, - refreshUserSQLRoles: () => {}, - refreshTableDetails: () => {}, - isTenant: false, - history: history, - location: history.location, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, - }; - it("should call refreshNodes if isTenant is false", () => { - const mockCallback = jest.fn(() => {}); - shallow(); - expect(mockCallback.mock.calls).to.have.length(1); - }); - it("should not call refreshNodes if isTenant is true", () => { - const mockCallback = jest.fn(() => {}); - - shallow( - , - ); - expect(mockCallback.mock.calls).to.have.length(0); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx deleted file mode 100644 index c5dc57242ef5..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.stories.tsx +++ /dev/null @@ -1,194 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { storiesOf } from "@storybook/react"; -import * as H from "history"; -import random from "lodash/random"; -import uniq from "lodash/uniq"; -import moment from "moment-timezone"; -import React from "react"; - -import { withBackground, withRouterProvider } from "src/storybook/decorators"; -import { - randomName, - randomRole, - randomTablePrivilege, -} from "src/storybook/fixtures"; -import { indexUnusedDuration } from "src/util/constants"; - -import { DatabaseTablePage, DatabaseTablePageProps } from "./databaseTablePage"; -const history = H.createHashHistory(); - -const withLoadingIndicator: DatabaseTablePageProps = { - databaseName: randomName(), - name: randomName(), - automaticStatsCollectionEnabled: true, - schemaName: randomName(), - indexUsageStatsEnabled: false, - showIndexRecommendations: false, - csIndexUnusedDuration: indexUnusedDuration, - details: { - loading: true, - loaded: false, - requestError: null, - queryError: undefined, - createStatement: { create_statement: "" }, - replicaData: { - storeIDs: [], - replicaCount: 0, - }, - indexData: { columns: [], indexes: [] }, - grants: { all: [] }, - statsLastUpdated: { stats_last_created_at: moment("0001-01-01T00:00:00Z") }, - spanStats: { - live_percentage: 0, - live_bytes: 0, - total_bytes: 0, - approximate_disk_bytes: 0, - range_count: 0, - }, - }, - indexStats: { - loading: true, - loaded: false, - lastError: undefined, - stats: [], - lastReset: moment("2021-09-04T13:55:00Z"), - }, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, - refreshTableDetails: () => {}, - refreshIndexStats: () => {}, - resetIndexUsageStats: () => {}, - refreshSettings: () => {}, - refreshUserSQLRoles: () => {}, -}; - -const name = randomName(); - -const withData: DatabaseTablePageProps = { - databaseName: randomName(), - name: name, - automaticStatsCollectionEnabled: true, - schemaName: randomName(), - indexUsageStatsEnabled: true, - showIndexRecommendations: true, - csIndexUnusedDuration: indexUnusedDuration, - details: { - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - createStatement: { - create_statement: ` - CREATE TABLE public.${name} ( - id UUID NOT NULL, - city VARCHAR NOT NULL, - name VARCHAR NULL, - address VARCHAR NULL, - credit_card VARCHAR NULL, - CONSTRAINT "primary" PRIMARY KEY (city ASC, id ASC), - FAMILY "primary" (id, city, name, address, credit_card) - ) - `, - }, - replicaData: { - storeIDs: [1, 2, 3, 4, 5, 6, 7], - replicaCount: 7, - }, - indexData: { - columns: Array(3).map(randomName), - indexes: Array(3).map(randomName), - }, - grants: { - all: [ - { - user: randomRole(), - privileges: uniq( - new Array(random(1, 5)).map(() => randomTablePrivilege()), - ), - }, - ], - }, - statsLastUpdated: { stats_last_created_at: moment("0001-01-01T00:00:00Z") }, - spanStats: { - live_percentage: 2.7, - live_bytes: 12345, - total_bytes: 456789, - approximate_disk_bytes: 44040192, - range_count: 4200, - }, - }, - showNodeRegionsSection: true, - indexStats: { - loading: false, - loaded: true, - lastError: null, - stats: [ - { - totalReads: 0, - lastUsed: moment("2021-10-11T11:29:00Z"), - lastUsedType: "read", - indexName: "primary", - indexRecommendations: [], - }, - { - totalReads: 3, - lastUsed: moment("2021-11-10T16:29:00Z"), - lastUsedType: "read", - indexName: "primary", - indexRecommendations: [], - }, - { - totalReads: 2, - lastUsed: moment("2021-09-04T13:55:00Z"), - lastUsedType: "reset", - indexName: "secondary", - indexRecommendations: [], - }, - { - totalReads: 0, - lastUsed: moment("2022-03-12T14:31:00Z"), - lastUsedType: "created", - indexName: "secondary", - indexRecommendations: [ - { - type: "DROP_UNUSED", - reason: - "This index has not been used and can be removed for better write performance.", - }, - ], - }, - ], - lastReset: moment("2021-09-04T13:55:00Z"), - }, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, - refreshTableDetails: () => {}, - refreshIndexStats: () => {}, - resetIndexUsageStats: () => {}, - refreshSettings: () => {}, - refreshUserSQLRoles: () => {}, -}; - -storiesOf("Database Table Page", module) - .addDecorator(withRouterProvider) - .addDecorator(withBackground) - .add("with data", () => ) - .add("with loading indicator", () => ( - - )); diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx deleted file mode 100644 index 7506ed26fd0f..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx +++ /dev/null @@ -1,715 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { Heading } from "@cockroachlabs/ui-components"; -import { Col, Row, Tabs, Tooltip } from "antd"; -import classNames from "classnames/bind"; -import { Moment } from "moment-timezone"; -import React from "react"; -import { RouteComponentProps } from "react-router-dom"; - -import { Anchor } from "src/anchor"; -import { commonStyles } from "src/common"; -import { StackIcon } from "src/icon/stackIcon"; -import { ColumnDescriptor, SortedTable, SortSetting } from "src/sortedtable"; -import { SqlBox } from "src/sql"; -import { - SummaryCard, - SummaryCardItem, - SummaryCardItemBoolSetting, -} from "src/summaryCard"; -import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses"; -import { - ascendingAttr, - columnTitleAttr, - syncHistory, - tabAttr, - tableStatsClusterSetting, -} from "src/util"; -import * as format from "src/util/format"; -import { DATE_FORMAT_24_TZ } from "src/util/format"; - -import { - SqlApiQueryResponse, - SqlExecutionErrorMessage, - TableCreateStatementRow, - TableHeuristicDetailsRow, - TableReplicaData, - TableSchemaDetailsRow, - TableSpanStatsRow, -} from "../api"; -import { CockroachCloudContext } from "../contexts"; -import { LoadingCell } from "../databases"; -import { RecommendationType } from "../indexDetailsPage"; -import { Loading } from "../loading"; -import booleanSettingStyles from "../settings/booleanSetting.module.scss"; -import LoadingError from "../sqlActivity/errorComponent"; -import { UIConfigState } from "../store"; -import { Timestamp, Timezone } from "../timestamp"; - -import styles from "./databaseTablePage.module.scss"; -import { - ActionCell, - DbTablesBreadcrumbs, - FormatMVCCInfo, - getCreateStmt, - IndexRecCell, - LastReset, - LastUsed, - NameCell, -} from "./helperComponents"; - -const cx = classNames.bind(styles); -const booleanSettingCx = classNames.bind(booleanSettingStyles); - -const { TabPane } = Tabs; - -// We break out separate interfaces for some of the nested objects in our data -// so that we can make (typed) test assertions on narrower slices of the data. -// -// The loading and loaded flags help us know when to dispatch the appropriate -// refresh actions. -// -// The overall structure is: -// -// interface DatabaseTablePageData { -// databaseName: string; -// name: string; -// details: { // DatabaseTablePageDataDetails -// loading: boolean; -// loaded: boolean; -// createStatement: string; -// replicaCount: number; -// indexNames: string[]; -// grants: { -// user: string; -// privilege: string; -// }[]; -// }; -// stats: { // DatabaseTablePageDataStats -// loading: boolean; -// loaded: boolean; -// sizeInBytes: number; -// rangeCount: number; -// nodesByRegionString: string; -// }; -// indexStats: { // DatabaseTablePageIndexStats -// loading: boolean; -// loaded: boolean; -// stats: { -// indexName: string; -// totalReads: number; -// lastUsed: Moment; -// lastUsedType: string; -// }[]; -// lastReset: Moment; -// }; -// } -export interface DatabaseTablePageData { - databaseName: string; - name: string; - schemaName: string; - details: DatabaseTablePageDataDetails; - indexStats: DatabaseTablePageIndexStats; - showNodeRegionsSection?: boolean; - indexUsageStatsEnabled: boolean; - showIndexRecommendations: boolean; - automaticStatsCollectionEnabled?: boolean; - hasAdminRole?: UIConfigState["hasAdminRole"]; - csIndexUnusedDuration: string; - isTenant?: UIConfigState["isTenant"]; -} - -export interface DatabaseTablePageDataDetails { - loading: boolean; - loaded: boolean; - // Request error getting table details - requestError: Error; - // Query error getting table details - queryError: SqlExecutionErrorMessage; - createStatement: SqlApiQueryResponse; - replicaData: SqlApiQueryResponse; - spanStats: SqlApiQueryResponse; - indexData: SqlApiQueryResponse; - grants: SqlApiQueryResponse; - statsLastUpdated?: SqlApiQueryResponse; - nodesByRegionString?: string; -} - -export interface DatabaseTablePageIndexStats { - loading: boolean; - loaded: boolean; - lastError: Error; - stats: IndexStat[]; - lastReset: Moment; -} - -export interface IndexStat { - indexName: string; - totalReads: number; - lastUsed: Moment; - lastUsedType: string; - indexRecommendations: IndexRecommendation[]; -} - -interface IndexRecommendation { - type: RecommendationType; - reason: string; -} - -interface AllGrants { - all: Grant[]; -} - -interface Grant { - user: string; - privileges: string[]; -} - -export interface DatabaseTablePageActions { - refreshTableDetails: ( - database: string, - table: string, - csIndexUnusedDuration: string, - ) => void; - refreshSettings: () => void; - refreshIndexStats?: (database: string, table: string) => void; - resetIndexUsageStats?: (database: string, table: string) => void; - refreshNodes?: () => void; - refreshUserSQLRoles: () => void; -} - -export type DatabaseTablePageProps = DatabaseTablePageData & - DatabaseTablePageActions & - RouteComponentProps; - -interface DatabaseTablePageState { - grantSortSetting: SortSetting; - indexSortSetting: SortSetting; - tab: string; - indexStatsColumns: ColumnDescriptor[]; -} - -const indexTabKey = "overview"; -const grantsTabKey = "grants"; - -class DatabaseTableGrantsTable extends SortedTable {} - -class IndexUsageStatsTable extends SortedTable {} - -export class DatabaseTablePage extends React.Component< - DatabaseTablePageProps, - DatabaseTablePageState -> { - static contextType = CockroachCloudContext; - - constructor(props: DatabaseTablePageProps) { - super(props); - - const { history } = this.props; - const searchParams = new URLSearchParams(history.location.search); - const currentTab = searchParams.get(tabAttr) || indexTabKey; - const indexSort: SortSetting = { - ascending: true, - columnTitle: "last used", - }; - - const grantSort: SortSetting = { - ascending: true, - columnTitle: "username", - }; - - const columnTitle = searchParams.get(columnTitleAttr); - if (columnTitle) { - if (currentTab === grantsTabKey) { - grantSort.columnTitle = columnTitle; - } else { - indexSort.columnTitle = columnTitle; - } - } - - this.state = { - indexSortSetting: indexSort, - grantSortSetting: grantSort, - tab: currentTab, - indexStatsColumns: this.indexStatsColumns(), - }; - } - - onTabChange = (tab: string): void => { - this.setState({ ...this.state, tab }); - - this.updateUrlAttrFromState( - tab === grantsTabKey - ? this.state.grantSortSetting - : this.state.indexSortSetting, - ); - - syncHistory( - { - tab: tab, - }, - this.props.history, - ); - }; - - componentDidMount(): void { - this.refresh(); - if (this.props.refreshSettings != null) { - this.props.refreshSettings(); - } - } - - componentDidUpdate(prevProp: Readonly): void { - this.refresh(); - if ( - prevProp.showIndexRecommendations !== this.props.showIndexRecommendations - ) { - this.setState({ indexStatsColumns: this.indexStatsColumns() }); - } - } - - private refresh() { - this.props.refreshUserSQLRoles(); - if (this.props.refreshNodes != null && !this.props.isTenant) { - this.props.refreshNodes(); - } - - if ( - !this.props.details.loaded && - !this.props.details.loading && - this.props.details.requestError === undefined - ) { - return this.props.refreshTableDetails( - this.props.databaseName, - this.props.name, - this.props.csIndexUnusedDuration, - ); - } - - if ( - !this.props.indexStats.loaded && - !this.props.indexStats.loading && - this.props.indexUsageStatsEnabled - ) { - return this.props.refreshIndexStats( - this.props.databaseName, - this.props.name, - ); - } - } - - private changeIndexSortSetting(sortSetting: SortSetting) { - const stateCopy = { ...this.state }; - stateCopy.indexSortSetting = sortSetting; - this.setState(stateCopy); - this.updateUrlAttrFromState(sortSetting); - } - - private changeGrantSortSetting(sortSetting: SortSetting) { - const stateCopy = { ...this.state }; - stateCopy.grantSortSetting = sortSetting; - this.setState(stateCopy); - this.updateUrlAttrFromState(sortSetting); - } - - private updateUrlAttrFromState(sortSetting: SortSetting) { - const { history } = this.props; - const searchParams = new URLSearchParams(history.location.search); - - searchParams.set(columnTitleAttr, sortSetting.columnTitle); - searchParams.set(ascendingAttr, String(sortSetting.ascending)); - history.location.search = searchParams.toString(); - history.replace(history.location); - } - - private indexStatsColumns(): ColumnDescriptor[] { - const indexStatsColumns: ColumnDescriptor[] = [ - { - name: "indexes", - title: "Indexes", - hideTitleUnderline: true, - className: cx("index-stats-table__col-indexes"), - cell: indexStat => ( - - ), - sort: indexStat => indexStat.indexName, - }, - { - name: "total reads", - title: "Total Reads", - hideTitleUnderline: true, - cell: indexStat => format.Count(indexStat.totalReads), - sort: indexStat => indexStat.totalReads, - }, - { - name: "last used", - title: ( - <> - Last Used - - ), - hideTitleUnderline: true, - className: cx("index-stats-table__col-last-used"), - cell: indexStat => , - sort: indexStat => indexStat.lastUsed, - }, - ]; - if (this.props.showIndexRecommendations) { - indexStatsColumns.push({ - name: "index recommendations", - title: ( - - Index Recommendations - - ), - cell: indexStat => , - sort: indexStat => indexStat.indexRecommendations.length, - }); - const isCockroachCloud = this.context; - if (!isCockroachCloud) { - indexStatsColumns.push({ - name: "action", - title: "", - cell: indexStat => ( - - ), - }); - } - } - return indexStatsColumns; - } - - private grantsColumns: ColumnDescriptor[] = [ - { - name: "username", - title: ( - - User Name - - ), - cell: grant => grant.user, - sort: grant => grant.user, - }, - { - name: "privilege", - title: ( - - Grants - - ), - cell: grant => grant.privileges.join(", "), - sort: grant => grant.privileges.join(", "), - }, - ]; - - render(): React.ReactElement { - const { hasAdminRole } = this.props; - const details: DatabaseTablePageDataDetails = this.props.details; - return ( -
-
- -

- - {this.props.name} -

-
- -
- - - ( - <> - -
- - - - - - - - {details.spanStats?.approximate_disk_bytes - ? format.Bytes( - details.spanStats?.approximate_disk_bytes, - ) - : null} - - } - /> - - {details.replicaData?.replicaCount} - - } - /> - - {details.spanStats?.range_count} - - } - /> - - - - } - /> - {details.statsLastUpdated && ( - - - - } - /> - )} - {this.props.automaticStatsCollectionEnabled != - null && ( - - {" "} - Automatic statistics can help improve query - performance. Learn how to{" "} - - manage statistics collection - - . - - } - /> - )} - - - - - - {this.props.showNodeRegionsSection && ( - - {details.nodesByRegionString && - details.nodesByRegionString?.length - ? details.nodesByRegionString - : null} - - } - /> - )} - - - {details.indexData?.indexes?.join(", ")} - - } - className={cx( - "database-table-page__indexes--value", - )} - /> - - - - {this.props.indexUsageStatsEnabled && ( - - - - - - - )} - - )} - renderError={() => - LoadingError({ - statsType: "databases", - error: details.requestError, - }) - } - /> - - - ( - - )} - renderError={() => - LoadingError({ - statsType: "databases", - error: details.requestError, - }) - } - /> - - - - - ); - } -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePageConnected.ts b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePageConnected.ts deleted file mode 100644 index 121fc07b6c78..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePageConnected.ts +++ /dev/null @@ -1,150 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import { connect } from "react-redux"; -import { RouteComponentProps } from "react-router"; -import { withRouter } from "react-router-dom"; -import { Dispatch } from "redux"; - -import { AppState, uiConfigActions } from "src/store"; -import { actions as analyticsActions } from "src/store/analytics"; -import { actions as clusterSettingsActions } from "src/store/clusterSettings"; -import { - selectAutomaticStatsCollectionEnabled, - selectDropUnusedIndexDuration, - selectIndexRecommendationsEnabled, - selectIndexUsageStatsEnabled, -} from "src/store/clusterSettings/clusterSettings.selectors"; -import { actions as tableDetailsActions } from "src/store/databaseTableDetails"; -import { actions as indexStatsActions } from "src/store/indexStats"; -import { - actions as nodesActions, - nodeRegionsByIDSelector, -} from "src/store/nodes"; -import { selectHasAdminRole, selectIsTenant } from "src/store/uiConfig"; -import { - databaseNameCCAttr, - generateTableID, - getMatchParamByName, - minDate, - schemaNameAttr, - tableNameCCAttr, - TimestampToMoment, -} from "src/util"; - -import { - deriveIndexDetailsMemoized, - deriveTablePageDetailsMemoized, -} from "../databases"; - -import { - DatabaseTablePageActions, - DatabaseTablePageData, - DatabaseTablePage, -} from "./databaseTablePage"; - -export const mapStateToProps = ( - state: AppState, - props: RouteComponentProps, -): DatabaseTablePageData => { - const database = getMatchParamByName(props.match, databaseNameCCAttr); - const table = getMatchParamByName(props.match, tableNameCCAttr); - const schema = getMatchParamByName(props.match, schemaNameAttr); - const tableDetails = state.adminUI?.tableDetails; - const indexUsageStats = state.adminUI?.indexStats; - const details = tableDetails[generateTableID(database, table)]; - const indexStatsState = indexUsageStats[generateTableID(database, table)]; - const lastReset = TimestampToMoment( - indexStatsState?.data?.last_reset, - minDate, - ); - const nodeRegions = nodeRegionsByIDSelector(state); - const isTenant = selectIsTenant(state); - const nodeStatuses = state.adminUI?.nodes.data; - return { - databaseName: database, - name: table, - schemaName: schema, - details: deriveTablePageDetailsMemoized({ - details, - nodeRegions, - isTenant, - nodeStatuses, - }), - showNodeRegionsSection: Object.keys(nodeRegions).length > 1 && !isTenant, - automaticStatsCollectionEnabled: - selectAutomaticStatsCollectionEnabled(state), - indexUsageStatsEnabled: selectIndexUsageStatsEnabled(state), - showIndexRecommendations: selectIndexRecommendationsEnabled(state), - csIndexUnusedDuration: selectDropUnusedIndexDuration(state), - hasAdminRole: selectHasAdminRole(state), - indexStats: { - loading: !!indexStatsState?.inFlight, - loaded: !!indexStatsState?.valid, - lastError: indexStatsState?.lastError, - stats: deriveIndexDetailsMemoized({ database, table, indexUsageStats }), - lastReset, - }, - isTenant, - }; -}; - -export const mapDispatchToProps = ( - dispatch: Dispatch, -): DatabaseTablePageActions => ({ - refreshTableDetails: ( - database: string, - table: string, - csIndexUnusedDuration: string, - ) => { - dispatch( - tableDetailsActions.refresh({ database, table, csIndexUnusedDuration }), - ); - }, - refreshIndexStats: (database: string, table: string) => { - dispatch( - indexStatsActions.refresh( - new cockroach.server.serverpb.TableIndexStatsRequest({ - database, - table, - }), - ), - ); - }, - resetIndexUsageStats: (database: string, table: string) => { - dispatch( - indexStatsActions.reset({ - database, - table, - }), - ); - dispatch( - analyticsActions.track({ - name: "Reset Index Usage", - page: "Index Details", - }), - ); - }, - refreshNodes: () => { - dispatch(nodesActions.refresh()); - }, - refreshSettings: () => { - dispatch(clusterSettingsActions.refresh()); - }, - refreshUserSQLRoles: () => dispatch(uiConfigActions.refreshUserSQLRoles()), -}); - -export const ConnectedDatabaseTablePage = withRouter( - connect< - DatabaseTablePageData, - DatabaseTablePageActions, - RouteComponentProps, - AppState - >( - mapStateToProps, - mapDispatchToProps, - )(DatabaseTablePage), -); diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/helperComponents.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/helperComponents.tsx index 5208d4debfd9..ddc86e71482a 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/helperComponents.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/helperComponents.tsx @@ -3,99 +3,17 @@ // Use of this software is governed by the CockroachDB Software License // included in the /LICENSE file. -import { Search as IndexIcon } from "@cockroachlabs/icons"; -import { Tooltip } from "antd"; -import classNames from "classnames/bind"; import { Moment } from "moment-timezone"; -import React, { useContext } from "react"; -import { Link } from "react-router-dom"; +import React from "react"; -import { Anchor } from "../anchor"; -import { sqlApiErrorMessage } from "../api"; +import { + IndexRecommendation, + IndexRecTypeEnum, +} from "../api/databases/tableIndexesApi"; import { QuoteIdentifier } from "../api/safesql"; -import { Breadcrumbs } from "../breadcrumbs"; -import { CockroachCloudContext } from "../contexts"; -import { CircleFilled } from "../icon"; -import { CaretRight } from "../icon/caretRight"; import IdxRecAction from "../insights/indexActionBtn"; import { Timestamp } from "../timestamp"; -import { - DATE_FORMAT, - DATE_FORMAT_24_TZ, - EncodeDatabaseTableUri, - EncodeDatabaseUri, - EncodeUriName, - minDate, - performanceTuningRecipes, -} from "../util"; -import * as format from "../util/format"; - -import { DatabaseTablePageDataDetails, IndexStat } from "./databaseTablePage"; -import styles from "./databaseTablePage.module.scss"; -const cx = classNames.bind(styles); - -export const NameCell = ({ - indexStat, - showIndexRecommendations, - tableName, -}: { - indexStat: IndexStat; - showIndexRecommendations: boolean; - tableName: string; -}): JSX.Element => { - const isCockroachCloud = useContext(CockroachCloudContext); - if (showIndexRecommendations) { - const linkURL = isCockroachCloud - ? `${location.pathname}/${indexStat.indexName}` - : `${tableName}/index/${EncodeUriName(indexStat.indexName)}`; - return ( - - - {indexStat.indexName} - - ); - } - return ( - <> - - {indexStat.indexName} - - ); -}; - -export const DbTablesBreadcrumbs = ({ - tableName, - schemaName, - databaseName, -}: { - tableName: string; - schemaName: string; - databaseName: string; -}): JSX.Element => { - const isCockroachCloud = useContext(CockroachCloudContext); - return ( - } - /> - ); -}; +import { DATE_FORMAT_24_TZ, minDate } from "../util"; export const LastReset = ({ lastReset, @@ -114,63 +32,14 @@ export const LastReset = ({ ); }; -interface IndexStatProps { - indexStat: IndexStat; +export interface IndexStat { + indexName: string; + totalReads: number; + lastUsed: Moment; + lastUsedType: string; + indexRecommendations: IndexRecommendation[]; } -export const LastUsed = ({ indexStat }: IndexStatProps): JSX.Element => { - // This case only occurs when we have no reads, resets, or creation time on - // the index. - if (indexStat.lastUsed.isSame(minDate)) { - return <>Never; - } - return ( - <> - Last {indexStat.lastUsedType}:{" "} - - - ); -}; - -export const IndexRecCell = ({ indexStat }: IndexStatProps): JSX.Element => { - const classname = - indexStat.indexRecommendations.length > 0 - ? "index-recommendations-icon__exist" - : "index-recommendations-icon__none"; - - if (indexStat.indexRecommendations.length === 0) { - return ( -
- - None -
- ); - } - // Render only the first recommendation for an index. - const recommendation = indexStat.indexRecommendations[0]; - let text: string; - switch (recommendation.type) { - case "DROP_UNUSED": - text = "Drop unused index"; - } - return ( - - {recommendation.reason}{" "} - - Learn more - - - } - > - - {text} - - ); -}; - export const ActionCell = ({ indexStat, tableName, @@ -182,7 +51,7 @@ export const ActionCell = ({ }): JSX.Element => { const query = indexStat.indexRecommendations.map(recommendation => { switch (recommendation.type) { - case "DROP_UNUSED": + case IndexRecTypeEnum.DROP_UNUSED: // Here, `tableName` is a fully qualified name whose identifiers have already been quoted. // See the QuoteIdentifier unit tests for more details. return `DROP INDEX ${tableName}@${QuoteIdentifier( @@ -202,33 +71,3 @@ export const ActionCell = ({ /> ); }; - -export const FormatMVCCInfo = ({ - details, -}: { - details: DatabaseTablePageDataDetails; -}): JSX.Element => { - return ( - <> - {format.Percentage(details.spanStats?.live_percentage, 1, 1)} - {" ("} - - {format.Bytes(details.spanStats?.live_bytes)} - {" "} - live data /{" "} - - {format.Bytes(details.spanStats?.total_bytes)} - - {" total)"} - - ); -}; - -export const getCreateStmt = ({ - createStatement, -}: DatabaseTablePageDataDetails): string => { - return createStatement?.create_statement - ? createStatement?.create_statement - : "(unavailable)\n" + - sqlApiErrorMessage(createStatement?.error?.message || ""); -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/index.ts b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/index.ts deleted file mode 100644 index 3add58291a3f..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/index.ts +++ /dev/null @@ -1,7 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -export * from "./databaseTablePage"; -export * from "./databaseTablePageConnected"; diff --git a/pkg/ui/workspaces/cluster-ui/src/databases/combiners.ts b/pkg/ui/workspaces/cluster-ui/src/databases/combiners.ts deleted file mode 100644 index 3c461e10f30e..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databases/combiners.ts +++ /dev/null @@ -1,317 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import { createSelector } from "@reduxjs/toolkit"; - -import { DatabaseDetailsPageDataTable } from "src/databaseDetailsPage"; - -import { DatabasesListResponse, TableNameParts } from "../api"; -import { DatabasesPageDataDatabase } from "../databasesPage"; -import { DatabaseTablePageDataDetails, IndexStat } from "../databaseTablePage"; -import { RecommendationType as RecType } from "../indexDetailsPage"; -import { - DatabaseDetailsSpanStatsState, - DatabaseDetailsState, -} from "../store/databaseDetails"; -import { TableDetailsState } from "../store/databaseTableDetails"; -import { IndexStatsState } from "../store/indexStats"; -import { generateTableID, longToInt, TimestampToMoment } from "../util"; - -import { - Nodes, - Stores, - buildIndexStatToRecommendationsMap, - getNodeIdsFromStoreIds, - getNodesByRegionString, - normalizePrivileges, - normalizeRoles, -} from "./util"; -type IndexUsageStatistic = - cockroach.server.serverpb.TableIndexStatsResponse.IExtendedCollectedIndexUsageStatistics; -const { RecommendationType } = cockroach.sql.IndexRecommendation; - -interface DerivedDatabaseDetailsParams { - dbListResp: DatabasesListResponse; - databaseDetails: Record; - spanStats: Record; - nodeRegions: Record; - isTenant: boolean; - /** A list of node statuses so that store ids can be mapped to nodes. */ - nodeStatuses: cockroach.server.status.statuspb.INodeStatus[]; -} - -export const deriveDatabaseDetailsMemoized = createSelector( - (params: DerivedDatabaseDetailsParams) => params.dbListResp, - (params: DerivedDatabaseDetailsParams) => params.databaseDetails, - (params: DerivedDatabaseDetailsParams) => params.spanStats, - (params: DerivedDatabaseDetailsParams) => params.nodeRegions, - (params: DerivedDatabaseDetailsParams) => params.isTenant, - (params: DerivedDatabaseDetailsParams) => params.nodeStatuses, - ( - dbListResp, - databaseDetails, - spanStats, - nodeRegions, - isTenant, - nodeStatuses, - ): DatabasesPageDataDatabase[] => { - const databases = dbListResp?.databases ?? []; - return databases.map(dbName => { - const dbDetails = databaseDetails[dbName]; - const spanStatsForDB = spanStats[dbName]; - return deriveDatabaseDetails( - dbName, - dbDetails, - spanStatsForDB, - nodeRegions, - isTenant, - nodeStatuses, - ); - }); - }, -); - -const deriveDatabaseDetails = ( - database: string, - dbDetails: DatabaseDetailsState, - spanStats: DatabaseDetailsSpanStatsState, - nodeRegionsByID: Record, - isTenant: boolean, - nodeStatuses: cockroach.server.status.statuspb.INodeStatus[], -): DatabasesPageDataDatabase => { - const dbStats = dbDetails?.data?.results.stats; - /** List of store IDs for the current cluster. All of the values in the - * `*replicas` columns correspond to store IDs. */ - const stores: Stores = { - kind: "store", - ids: dbStats?.replicaData.storeIDs || [], - }; - /** List of node IDs for the current cluster. */ - const nodes = getNodeIdsFromStoreIds(stores, nodeStatuses); - - const nodesByRegionString = getNodesByRegionString( - nodes, - nodeRegionsByID, - isTenant, - ); - const numIndexRecommendations = - dbStats?.indexStats.num_index_recommendations || 0; - - return { - detailsLoading: !!dbDetails?.inFlight, - detailsLoaded: !!dbDetails?.valid, - spanStatsLoading: !!spanStats?.inFlight, - spanStatsLoaded: !!spanStats?.valid, - detailsRequestError: dbDetails?.lastError, // http request error. - spanStatsRequestError: spanStats?.lastError, // http request error. - detailsQueryError: dbDetails?.data?.results?.error, - spanStatsQueryError: spanStats?.data?.results?.error, - name: database, - spanStats: spanStats?.data?.results.spanStats, - tables: dbDetails?.data?.results.tablesResp, - nodes: nodes.ids, - nodesByRegionString, - numIndexRecommendations, - }; -}; - -interface DerivedTableDetailsParams { - dbName: string; - tables: TableNameParts[]; - tableDetails: Record; - nodeRegions: Record; - isTenant: boolean; - /** A list of node statuses so that store ids can be mapped to nodes. */ - nodeStatuses: cockroach.server.status.statuspb.INodeStatus[]; -} - -export const deriveTableDetailsMemoized = createSelector( - (params: DerivedTableDetailsParams) => params.dbName, - (params: DerivedTableDetailsParams) => params.tables, - (params: DerivedTableDetailsParams) => params.tableDetails, - (params: DerivedTableDetailsParams) => params.nodeRegions, - (params: DerivedTableDetailsParams) => params.isTenant, - (params: DerivedTableDetailsParams) => params.nodeStatuses, - ( - dbName, - tables, - tableDetails, - nodeRegions, - isTenant, - nodeStatuses, - ): DatabaseDetailsPageDataTable[] => { - tables = tables || []; - return tables.map(table => { - const tableID = generateTableID( - dbName, - table.qualifiedNameWithSchemaAndTable, - ); - const details = tableDetails[tableID]; - return deriveDatabaseTableDetails( - table, - details, - nodeRegions, - isTenant, - nodeStatuses, - ); - }); - }, -); - -const deriveDatabaseTableDetails = ( - table: TableNameParts, - details: TableDetailsState, - nodeRegions: Record, - isTenant: boolean, - nodeStatuses: cockroach.server.status.statuspb.INodeStatus[], -): DatabaseDetailsPageDataTable => { - const results = details?.data?.results; - const grants = results?.grantsResp.grants ?? []; - const normalizedRoles = normalizeRoles(grants.map(grant => grant.user)); - const normalizedPrivileges = normalizePrivileges( - [].concat(...grants.map(grant => grant.privileges)), - ); - const stores: Stores = { - kind: "store", - ids: results?.stats.replicaData.storeIDs || [], - }; - const nodes: Nodes = getNodeIdsFromStoreIds(stores, nodeStatuses); - return { - name: table, - qualifiedDisplayName: `${table.schema}.${table.table}`, - loading: !!details?.inFlight, - loaded: !!details?.valid, - requestError: details?.lastError, - queryError: details?.data?.results?.error, - details: { - schemaDetails: results?.schemaDetails, - grants: { - roles: normalizedRoles, - privileges: normalizedPrivileges, - error: results?.grantsResp?.error, - }, - statsLastUpdated: results?.heuristicsDetails, - indexStatRecs: results?.stats.indexStats, - spanStats: results?.stats.spanStats, - nodes: nodes.ids, - nodesByRegionString: getNodesByRegionString(nodes, nodeRegions, isTenant), - }, - }; -}; - -interface DerivedTablePageDetailsParams { - details: TableDetailsState; - nodeRegions: Record; - isTenant: boolean; - /** A list of node statuses so that store ids can be mapped to nodes. */ - nodeStatuses: cockroach.server.status.statuspb.INodeStatus[]; -} - -export const deriveTablePageDetailsMemoized = createSelector( - (params: DerivedTablePageDetailsParams) => params.details, - (params: DerivedTablePageDetailsParams) => params.nodeRegions, - (params: DerivedTablePageDetailsParams) => params.isTenant, - (params: DerivedTablePageDetailsParams) => params.nodeStatuses, - ( - details, - nodeRegions, - isTenant, - nodeStatuses, - ): DatabaseTablePageDataDetails => { - const results = details?.data?.results; - const grants = results?.grantsResp.grants || []; - const normalizedGrants = - grants.map(grant => ({ - user: grant.user, - privileges: normalizePrivileges(grant.privileges), - })) || []; - - const stores: Stores = { - kind: "store", - ids: results?.stats.replicaData.storeIDs || [], - }; - const nodes = getNodeIdsFromStoreIds(stores, nodeStatuses); - - return { - loading: !!details?.inFlight, - loaded: !!details?.valid, - requestError: details?.lastError, - queryError: results?.error, - createStatement: results?.createStmtResp, - replicaData: results?.stats?.replicaData, - indexData: results?.schemaDetails, - grants: { all: normalizedGrants, error: results?.grantsResp?.error }, - statsLastUpdated: results?.heuristicsDetails, - spanStats: results?.stats?.spanStats, - nodesByRegionString: getNodesByRegionString(nodes, nodeRegions, isTenant), - }; - }, -); - -interface DerivedIndexDetailsParams { - database: string; - table: string; - indexUsageStats: Record; -} - -export const deriveIndexDetailsMemoized = createSelector( - (params: DerivedIndexDetailsParams) => params.database, - (params: DerivedIndexDetailsParams) => params.table, - (params: DerivedIndexDetailsParams) => params.indexUsageStats, - (database, table, indexUsageStats): IndexStat[] => { - const indexStats = indexUsageStats[generateTableID(database, table)]; - const lastReset = TimestampToMoment(indexStats?.data?.last_reset); - const stats = indexStats?.data?.statistics || []; - const recommendations = indexStats?.data?.index_recommendations || []; - const recsMap = buildIndexStatToRecommendationsMap(stats, recommendations); - return stats.map(indexStat => { - const indexRecs = recsMap[indexStat?.statistics.key.index_id] || []; - return deriveIndexDetails(indexStat, lastReset, indexRecs); - }); - }, -); - -const deriveIndexDetails = ( - indexStat: IndexUsageStatistic, - lastReset: moment.Moment, - recommendations: cockroach.sql.IIndexRecommendation[], -): IndexStat => { - const lastRead = TimestampToMoment(indexStat.statistics?.stats?.last_read); - let lastUsed, lastUsedType; - if (indexStat.created_at !== null) { - lastUsed = TimestampToMoment(indexStat.created_at); - lastUsedType = "created"; - } else { - lastUsed = lastReset; - lastUsedType = "reset"; - } - if (lastReset.isAfter(lastUsed)) { - lastUsed = lastReset; - lastUsedType = "reset"; - } - if (lastRead.isAfter(lastUsed)) { - lastUsed = lastRead; - lastUsedType = "read"; - } - const indexRecommendations = recommendations.map(indexRec => { - let type: RecType = "Unknown"; - switch (RecommendationType[indexRec.type].toString()) { - case "DROP_UNUSED": - type = "DROP_UNUSED"; - } - return { - type: type, - reason: indexRec.reason, - }; - }); - return { - indexName: indexStat.index_name, - totalReads: longToInt(indexStat.statistics?.stats?.total_read_count), - lastUsed: lastUsed, - lastUsedType: lastUsedType, - indexRecommendations, - }; -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/databases/index.ts b/pkg/ui/workspaces/cluster-ui/src/databases/index.ts deleted file mode 100644 index 968659a82c84..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databases/index.ts +++ /dev/null @@ -1,7 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -export * from "./combiners"; -export * from "./util"; diff --git a/pkg/ui/workspaces/cluster-ui/src/databases/util.spec.tsx b/pkg/ui/workspaces/cluster-ui/src/databases/util.spec.tsx deleted file mode 100644 index b1143b74d419..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databases/util.spec.tsx +++ /dev/null @@ -1,246 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { render } from "@testing-library/react"; -import React from "react"; - -import { INodeStatus } from "../util"; - -import { - Nodes, - Stores, - getNodesByRegionString, - getNodeIdsFromStoreIds, - normalizePrivileges, - normalizeRoles, - LoadingCell, -} from "./util"; - -describe("Getting nodes by region string", () => { - describe("is not tenant", () => { - it("when all nodes different regions", () => { - const nodes: Nodes = { kind: "node", ids: [1, 2, 3] }; - const regions = { - "1": "region1", - "2": "region2", - "3": "region3", - }; - const result = getNodesByRegionString(nodes, regions, false); - expect(result).toEqual(`region1(n1), region2(n2), region3(n3)`); - }); - - it("when all nodes same region", () => { - const nodes: Nodes = { kind: "node", ids: [1, 2, 3] }; - const regions = { - "1": "region1", - "2": "region1", - "3": "region1", - }; - const result = getNodesByRegionString(nodes, regions, false); - expect(result).toEqual(`region1(n1,n2,n3)`); - }); - - it("when some nodes different regions", () => { - const nodes: Nodes = { kind: "node", ids: [1, 2, 3] }; - const regions = { - "1": "region1", - "2": "region1", - "3": "region2", - }; - const result = getNodesByRegionString(nodes, regions, false); - expect(result).toEqual(`region1(n1,n2), region2(n3)`); - }); - - it("when region map is empty", () => { - const nodes: Nodes = { kind: "node", ids: [1, 2, 3] }; - const regions = {}; - const result = getNodesByRegionString(nodes, regions, false); - expect(result).toEqual(""); - }); - - it("when nodes are empty", () => { - const nodes: Nodes = { kind: "node", ids: [] }; - const regions = { - "1": "region1", - "2": "region1", - "3": "region2", - }; - const result = getNodesByRegionString(nodes, regions, false); - expect(result).toEqual(""); - }); - }); -}); - -describe("getNodeIdsFromStoreIds", () => { - it("returns the correct node ids when all nodes have multiple stores", () => { - const stores: Stores = { kind: "store", ids: [1, 3, 6, 2, 4, 5] }; - const nodeStatuses: INodeStatus[] = [ - { - desc: { - node_id: 1, - }, - store_statuses: [{ desc: { store_id: 1 } }, { desc: { store_id: 2 } }], - }, - { - desc: { - node_id: 2, - }, - store_statuses: [{ desc: { store_id: 3 } }, { desc: { store_id: 5 } }], - }, - { - desc: { - node_id: 3, - }, - store_statuses: [{ desc: { store_id: 4 } }, { desc: { store_id: 6 } }], - }, - ]; - const result = getNodeIdsFromStoreIds(stores, nodeStatuses); - expect(result).toEqual({ kind: "node", ids: [1, 2, 3] }); - }); - - it("returns an empty list when no stores ids are provided", () => { - const stores: Stores = { kind: "store", ids: [] }; - const result = getNodeIdsFromStoreIds(stores, []); - expect(result).toEqual({ kind: "node", ids: [] }); - }); - - it("returns the correct node ids when there is one store per node", () => { - const stores: Stores = { kind: "store", ids: [1, 3, 4] }; - const nodeStatuses: INodeStatus[] = [ - { - desc: { - node_id: 1, - }, - store_statuses: [{ desc: { store_id: 1 } }], - }, - { - desc: { - node_id: 2, - }, - store_statuses: [{ desc: { store_id: 3 } }], - }, - { - desc: { - node_id: 3, - }, - store_statuses: [{ desc: { store_id: 4 } }], - }, - ]; - const result = getNodeIdsFromStoreIds(stores, nodeStatuses); - expect(result).toEqual({ kind: "node", ids: [1, 2, 3] }); - }); - it("returns the correct node ids when there is only one node", () => { - const stores: Stores = { kind: "store", ids: [3] }; - const nodeStatuses: INodeStatus[] = [ - { - desc: { - node_id: 1, - }, - store_statuses: [{ desc: { store_id: 3 } }], - }, - ]; - const result = getNodeIdsFromStoreIds(stores, nodeStatuses); - expect(result).toEqual({ kind: "node", ids: [1] }); - }); -}); - -describe("Normalize privileges", () => { - it("sorts correctly when input is disordered", () => { - const privs = ["CREATE", "DELETE", "UPDATE", "ALL", "GRANT"]; - const result = normalizePrivileges(privs); - expect(result).toEqual(["ALL", "CREATE", "GRANT", "UPDATE", "DELETE"]); - }); - - it("removes duplicates", () => { - const privs = ["CREATE", "CREATE", "UPDATE", "ALL", "GRANT"]; - const result = normalizePrivileges(privs); - expect(result).toEqual(["ALL", "CREATE", "GRANT", "UPDATE"]); - }); -}); - -describe("Normalize roles", () => { - it("sorts correctly when input is disordered", () => { - const roles = ["public", "root", "admin"]; - const result = normalizeRoles(roles); - expect(result).toEqual(["root", "admin", "public"]); - }); - - it("removes duplicates", () => { - const roles = ["public", "admin", "admin"]; - const result = normalizeRoles(roles); - expect(result).toEqual(["admin", "public"]); - }); -}); - -describe("LoadingCell", () => { - it("renders empty data", () => { - const { getByText } = render( - - {null} - , - ); - - expect(getByText("No data")).not.toBeNull(); - }); - it("renders with undefined children", () => { - const { getByText } = render( - , - ); - - expect(getByText("No data")).not.toBeNull(); - }); - it("renders skeleton heading when loading", () => { - const { getByRole } = render( - - {null} - , - ); - - expect(getByRole("heading")).not.toBeNull(); - }); - it("renders error name and status icon", () => { - const { getByRole } = render( - - {null} - , - ); - - // TODO(davidh): rendering of antd Tooltip component doesn't work - // here and hence can't be directly tested to contain the error - // name. - expect(getByRole("status")).not.toBeNull(); - }); - it("renders children with no error", () => { - const { getByText } = render( - -
inner data
-
, - ); - - expect(getByText("inner data")).not.toBeNull(); - }); - it("renders children with error together", () => { - const { getByText, getByRole } = render( - -
inner data
-
, - ); - - expect(getByRole("status")).not.toBeNull(); - expect(getByText("inner data")).not.toBeNull(); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/databases/util.tsx b/pkg/ui/workspaces/cluster-ui/src/databases/util.tsx deleted file mode 100644 index e9de0c7fc4e7..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databases/util.tsx +++ /dev/null @@ -1,318 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import { Caution } from "@cockroachlabs/icons"; -import { Skeleton, Tooltip } from "antd"; -import React from "react"; - -import { - isMaxSizeError, - isPrivilegeError, - sqlApiErrorMessage, - SqlExecutionErrorMessage, -} from "../api"; -type IndexUsageStatistic = - cockroach.server.serverpb.TableIndexStatsResponse.IExtendedCollectedIndexUsageStatistics; - -export function combineLoadingErrors( - detailsErr: Error, - isMaxSizeError: boolean, - dbList: string, -): Error { - if (dbList && detailsErr) { - return new GetDatabaseInfoError( - `Failed to load all databases and database details. Partial results are shown. Debug info: ${dbList}, details error: ${detailsErr}`, - ); - } - - if (dbList) { - return new GetDatabaseInfoError( - `Failed to load all databases. Partial results are shown. Debug info: ${dbList}`, - ); - } - - if (detailsErr) { - return detailsErr; - } - - if (isMaxSizeError) { - return new GetDatabaseInfoError( - `Failed to load all databases and database details. Partial results are shown. Debug info: Max size limit reached fetching database details`, - ); - } - - return; -} -export class GetDatabaseInfoError extends Error { - constructor(message: string) { - super(message); - - this.name = this.constructor.name; - } -} - -/** Store ids and node ids are both of type `number[]`. To disambiguate, a - * `kind` field is included in the type. */ -export type Stores = { kind: "store"; ids: number[] }; - -/** Node ids and store IDs are both of type `number[]`. To disambiguate, a - * `kind` field is included in the type. */ -export type Nodes = { kind: "node"; ids: number[] }; - -/** getNodeIdsFromStoreIds converts a list of store IDs to a list of node IDs. - * - * @param stores - Store ids for the cluster. - * @param nodeStatuses - A list of nodes that includes store information. - * @returns A list of node ids for the cluster. - */ -export function getNodeIdsFromStoreIds( - stores: Stores, - nodeStatuses: cockroach.server.status.statuspb.INodeStatus[], -): Nodes { - /** Associates stores with their node. Nodes can have multiple stores: - * `{ store1: node1, store2: node1 }` */ - const nodeByStoreMap: Record = {}; - nodeStatuses?.map(node => - node.store_statuses?.map(store => { - nodeByStoreMap[store.desc.store_id] = node.desc.node_id; - }), - ); - - /** A unique list of node IDs derived from the nodeByStoreMap. */ - const nodeIds = Array.from( - new Set(stores.ids.map(id => nodeByStoreMap[id])), - ).filter(value => value !== undefined); - - return { kind: "node", ids: nodeIds }; -} - -/** getNodesByRegionString converts a list of node IDs to a user-facing string. - * - * @param nodes - Node ids for the cluster. - * @param nodeRegions - A map of node IDs to region IDs. - * @param isTenant - Whether the cluster is a tenant cluster. - * @returns A string of node IDs by region, ordered by region name, e.g. - * `regionA(n1, n2), regionB(n2,n3), ...` - */ -export function getNodesByRegionString( - nodes: Nodes, - nodeRegions: Record, - isTenant: boolean, -): string { - return nodesByRegionMapToString( - createNodesByRegionMap(nodes.ids, nodeRegions), - isTenant, - ); -} - -/** nodesByRegionMapToString converts a map of regions to node ids, - * ordered by region name, e.g. converts: - * `{ regionA: [1, 2], regionB: [2, 3] }` - * to: - * `regionA(n1, n2), regionB(n2,n3), ...` - * - * If the cluster is a tenant cluster, then we redact node info - * and only display the region name, e.g. - * `regionA(n1, n2), regionB(n2,n3), ...` becomes: - * `regionA, regionB, ...` */ -function nodesByRegionMapToString( - nodesByRegion: Record, - isTenant: boolean, -): string { - // Sort the nodes on each key. - const regions = Object.keys(nodesByRegion).sort(); - regions.forEach((region: string) => { - nodesByRegion[region].sort(); - }); - - return regions - .map((region: string) => { - const nodes = nodesByRegion[region]; - return isTenant - ? `${region}` - : `${region}(${nodes.map(id => `n${id}`).join(",")})`; - }) - .join(", "); -} - -function createNodesByRegionMap( - nodes: number[], - nodeRegions: Record, -): Record { - const nodesByRegionMap: Record = {}; - nodes.forEach((node: number) => { - // If the node's region doesn't exist skip it. - if (nodeRegions[node.toString()] == null) { - return; - } - const region: string = nodeRegions[node.toString()]; - if (nodesByRegionMap[region] == null) { - nodesByRegionMap[region] = []; - } - nodesByRegionMap[region].push(node); - }); - return nodesByRegionMap; -} - -// sortByPrecedence sorts a list of strings via a "precedence" mapping. -function sortByPrecedence( - vals: string[], - precedenceMapping: Record, - removeDuplicates?: boolean, -): string[] { - // Sorting function. First compare by precedence. - // If both items have the same precedence level, - // sort alphabetically. - const compareFn = (a: string, b: string) => { - const aPrecedence = precedenceMapping[a]; - const bPrecendence = precedenceMapping[b]; - if (aPrecedence && bPrecendence) { - return precedenceMapping[a] - precedenceMapping[b]; - } - if (aPrecedence) { - return -1; - } - if (bPrecendence) { - return 1; - } - return a.localeCompare(b); - }; - - if (removeDuplicates) { - return [...new Set(vals)].sort(compareFn); - } - return [...vals].sort(compareFn); -} - -export function normalizeRoles(raw: string[]): string[] { - const rolePrecedence: Record = { - root: 1, - admin: 2, - public: 3, - }; - - // Unique roles, sorted by precedence. - return sortByPrecedence(raw, rolePrecedence, true); -} - -// normalizePrivileges sorts privileges by privilege precedence. -export function normalizePrivileges(raw: string[]): string[] { - const privilegePrecedence: Record = { - ALL: 1, - CREATE: 2, - DROP: 3, - GRANT: 4, - SELECT: 5, - INSERT: 6, - UPDATE: 7, - DELETE: 8, - }; - - // Unique privileges, sorted by precedence. - const rawUppers = raw.map(priv => priv.toUpperCase()); - return sortByPrecedence(rawUppers, privilegePrecedence, true); -} - -export function buildIndexStatToRecommendationsMap( - stats: IndexUsageStatistic[], - recs: cockroach.sql.IIndexRecommendation[], -): Record { - const recommendationsMap: Record< - string, - cockroach.sql.IIndexRecommendation[] - > = {}; - stats.forEach(stat => { - const recsForStat = - recs.filter(rec => rec.index_id === stat?.statistics.key.index_id) || []; - if (!recommendationsMap[stat?.statistics.key.index_id]) { - recommendationsMap[stat?.statistics.key.index_id] = recsForStat; - } else { - recommendationsMap[stat?.statistics.key.index_id].push(...recsForStat); - } - }); - return recommendationsMap; -} - -interface LoadingCellProps { - requestError: Error; - queryError?: Error; - loading: boolean; - errorClassName: string; -} - -export const LoadingCell: React.FunctionComponent = ({ - loading, - requestError, - queryError, - errorClassName, - children, -}) => { - if (loading) { - return ( - - ); - } - - let tooltipMsg = ""; - if (requestError) { - tooltipMsg = `Encountered a network error fetching data for this cell: ${requestError.name}`; - } else if (queryError) { - tooltipMsg = getQueryErrorMessage(queryError); - } - - let childrenOrNoData = <>{children}; - if (children == null) { - childrenOrNoData = <>{"No data"}; - } - - // If we encounter an error gathering data for this cell, - // render a warning icon with a tooltip message for the error. - if (tooltipMsg !== "") { - return ( - - - {childrenOrNoData} - - ); - } else { - return childrenOrNoData; - } -}; - -export const getNetworkErrorMessage = (requestError: Error): string => { - return `Encountered a network error: ${requestError.message}`; -}; - -export const getQueryErrorMessage = ( - queryError: SqlExecutionErrorMessage | Error, -): string => { - if (checkPrivilegeError(queryError)) { - return ( - `User has insufficient privileges:\n` + - sqlApiErrorMessage(queryError.message) - ); - } - if (isMaxSizeError(queryError.message)) { - return `Only partial data available, total data size exceeds limit in the console`; - } - // Unexpected error - return the error message. - return sqlApiErrorMessage(queryError.message); -}; - -const checkPrivilegeError = ( - err: SqlExecutionErrorMessage | Error, -): boolean => { - if ("code" in err) { - return isPrivilegeError(err.code); - } - // If the error message includes any mention of privilege, consider it a privilege error. - return err.message.includes("privilege"); -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databaseTableCells.tsx b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databaseTableCells.tsx deleted file mode 100644 index 2bc1fdff40b5..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databaseTableCells.tsx +++ /dev/null @@ -1,108 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { Caution } from "@cockroachlabs/icons"; -import { Tooltip } from "antd"; -import classNames from "classnames/bind"; -import React, { useContext } from "react"; -import { Link } from "react-router-dom"; - -import { CockroachCloudContext } from "../contexts"; -import { - LoadingCell, - getNetworkErrorMessage, - getQueryErrorMessage, -} from "../databases"; -import { CircleFilled } from "../icon"; -import { StackIcon } from "../icon/stackIcon"; -import { EncodeDatabaseUri } from "../util"; -import * as format from "../util/format"; - -import { DatabasesPageDataDatabase } from "./databasesPage"; -import styles from "./databasesPage.module.scss"; - -const cx = classNames.bind(styles); - -interface CellProps { - database: DatabasesPageDataDatabase; -} - -export const DiskSizeCell = ({ database }: CellProps) => ( - - {database.spanStats?.approximate_disk_bytes - ? format.Bytes(database.spanStats?.approximate_disk_bytes) - : null} - -); - -export const IndexRecCell = ({ database }: CellProps): JSX.Element => { - const text = - database.numIndexRecommendations > 0 - ? `${database.numIndexRecommendations} index recommendation(s)` - : "None"; - const classname = - database.numIndexRecommendations > 0 - ? "index-recommendations-icon__exist" - : "index-recommendations-icon__none"; - return ( -
- - {text} -
- ); -}; - -export const DatabaseNameCell = ({ database }: CellProps): JSX.Element => { - const isCockroachCloud = useContext(CockroachCloudContext); - const linkURL = isCockroachCloud - ? `${location.pathname}/${database.name}` - : EncodeDatabaseUri(database.name); - let icon = ; - - const needsWarning = - database.detailsRequestError || - database.spanStatsRequestError || - database.detailsQueryError || - database.spanStatsQueryError; - - if (needsWarning) { - const titleList = []; - if (database.detailsRequestError) { - titleList.push(getNetworkErrorMessage(database.detailsRequestError)); - } - if (database.spanStatsRequestError) { - titleList.push(getNetworkErrorMessage(database.spanStatsRequestError)); - } - if (database.detailsQueryError) { - titleList.push(database.detailsQueryError.message); - } - if (database.spanStatsQueryError) { - titleList.push(getQueryErrorMessage(database.spanStatsQueryError)); - } - - icon = ( - - - - ); - } - return ( - <> - - {icon} - {database.name} - - - ); -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.module.scss b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.module.scss deleted file mode 100644 index edd4ffcf3f0c..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.module.scss +++ /dev/null @@ -1,96 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -@import "src/core/index.module"; - -.databases-table { - &__col { - &-name { - font-family: $font-family--semi-bold; - } - - &-size, - &-table-count, - &-range-count { - width: 10em; - text-align: right; - padding-right: $spacing-x-large; - } - &-idx-rec { - width: 20em; - } - } - - &__no-result { - @include text--body-strong; - } - - &__cell-error { - display: inline-flex; - align-items: center; - gap: 10px; - svg { - fill: $colors--warning; - } - } -} - -.sorted-table { - width: 100%; -} - -.icon { - &__container { - display: inline-flex; - align-items: center; - } - - &--md { - height: 24px; - width: 24px; - margin-right: 12px; - } - - &--s { - height: 16px; - width: 16px; - margin-right: 10px; - } - - &--xxs { - height: 8px; - width: 8px; - } - - &--title { - fill: $colors--title; - } - - &--primary { - fill: $colors--primary-text; - } - - &--warning { - fill: $colors--functional-orange-3; - } -} - -.index-recommendations-icon { - &__exist { - height: 10px; - width: 20px; - fill: $colors--functional-orange-3; - } - - &__none { - height: 10px; - width: 20px; - fill: $colors--neutral-11; - } -} - -.page-config { - margin-bottom: $spacing-smaller; -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.spec.tsx b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.spec.tsx deleted file mode 100644 index 19020ab20719..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.spec.tsx +++ /dev/null @@ -1,92 +0,0 @@ -// Copyright 2024 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { expect } from "chai"; -import { shallow } from "enzyme"; -import * as H from "history"; -import React from "react"; - -import { defaultFilters } from "../queryFilter"; -import { indexUnusedDuration } from "../util"; - -import { DatabasesPage, DatabasesPageProps } from "./databasesPage"; - -describe("DatabasesPage", () => { - const history = H.createHashHistory(); - const props: DatabasesPageProps = { - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - databases: [ - { - detailsLoading: false, - detailsLoaded: false, - spanStatsLoading: false, - spanStatsLoaded: false, - detailsRequestError: undefined, - spanStatsRequestError: undefined, - detailsQueryError: undefined, - spanStatsQueryError: undefined, - name: "system", - nodes: [], - spanStats: undefined, - tables: undefined, - nodesByRegionString: "", - numIndexRecommendations: 0, - }, - { - detailsLoading: false, - detailsLoaded: false, - spanStatsLoading: false, - spanStatsLoaded: false, - detailsRequestError: undefined, - spanStatsRequestError: undefined, - detailsQueryError: undefined, - spanStatsQueryError: undefined, - name: "test", - nodes: [], - spanStats: undefined, - tables: undefined, - nodesByRegionString: "", - numIndexRecommendations: 0, - }, - ], - search: null, - filters: defaultFilters, - nodeRegions: {}, - isTenant: false, - sortSetting: { ascending: true, columnTitle: "name" }, - showNodeRegionsColumn: false, - indexRecommendationsEnabled: true, - csIndexUnusedDuration: indexUnusedDuration, - automaticStatsCollectionEnabled: true, - refreshDatabases: () => {}, - refreshDatabaseDetails: () => {}, - refreshDatabaseSpanStats: () => {}, - refreshSettings: () => {}, - history: history, - location: history.location, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, - }; - it("should call refreshNodes if isTenant is false", () => { - const mockCallback = jest.fn(() => {}); - shallow(); - expect(mockCallback.mock.calls).to.have.length(1); - }); - it("should not call refreshNodes if isTenant is true", () => { - const mockCallback = jest.fn(() => {}); - - shallow( - , - ); - expect(mockCallback.mock.calls).to.have.length(0); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.stories.tsx deleted file mode 100644 index c1ab195b1f02..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.stories.tsx +++ /dev/null @@ -1,141 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { storiesOf } from "@storybook/react"; -import * as H from "history"; -import random from "lodash/random"; -import React from "react"; - -import { defaultFilters } from "src/queryFilter"; -import { withBackground, withRouterProvider } from "src/storybook/decorators"; -import { randomName } from "src/storybook/fixtures"; -import { indexUnusedDuration } from "src/util/constants"; - -import { DatabasesPage, DatabasesPageProps } from "./databasesPage"; -const history = H.createHashHistory(); - -const withLoadingIndicator: DatabasesPageProps = { - loading: true, - loaded: false, - requestError: undefined, - queryError: undefined, - automaticStatsCollectionEnabled: true, - indexRecommendationsEnabled: false, - csIndexUnusedDuration: indexUnusedDuration, - databases: [], - sortSetting: { - ascending: false, - columnTitle: "name", - }, - search: "", - filters: defaultFilters, - nodeRegions: {}, - refreshDatabases: () => {}, - refreshSettings: () => {}, - refreshDatabaseDetails: () => {}, - refreshDatabaseSpanStats: () => {}, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, -}; - -const withoutData: DatabasesPageProps = { - loading: false, - loaded: true, - requestError: undefined, - queryError: undefined, - automaticStatsCollectionEnabled: true, - indexRecommendationsEnabled: false, - csIndexUnusedDuration: indexUnusedDuration, - databases: [], - sortSetting: { - ascending: false, - columnTitle: "name", - }, - search: "", - filters: defaultFilters, - nodeRegions: {}, - onSortingChange: () => {}, - refreshDatabases: () => {}, - refreshSettings: () => {}, - refreshDatabaseDetails: () => {}, - refreshDatabaseSpanStats: () => {}, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, -}; - -const withData: DatabasesPageProps = { - loading: false, - loaded: true, - requestError: undefined, - queryError: undefined, - showNodeRegionsColumn: true, - automaticStatsCollectionEnabled: true, - indexRecommendationsEnabled: true, - csIndexUnusedDuration: indexUnusedDuration, - sortSetting: { - ascending: false, - columnTitle: "name", - }, - search: "", - filters: defaultFilters, - nodeRegions: { - "1": "gcp-us-east1", - "6": "gcp-us-west1", - "8": "gcp-europe-west1", - }, - databases: Array(42).map(() => { - return { - detailsLoading: false, - detailsLoaded: false, - spanStatsLoading: false, - spanStatsLoaded: false, - detailsRequestError: undefined, - detailsQueryError: undefined, - spanStatsRequestError: undefined, - spanStatsQueryError: undefined, - name: randomName(), - sizeInBytes: random(1000.0) * 1024 ** random(1, 2), - tableCount: random(5, 100), - rangeCount: random(50, 500), - nodesByRegionString: - "gcp-europe-west1(n8), gcp-us-east1(n1), gcp-us-west1(n6)", - numIndexRecommendations: 0, - }; - }), - onSortingChange: () => {}, - refreshDatabases: () => {}, - refreshSettings: () => {}, - refreshDatabaseDetails: () => {}, - refreshDatabaseSpanStats: () => {}, - location: history.location, - history, - match: { - url: "", - path: history.location.pathname, - isExact: false, - params: {}, - }, -}; - -storiesOf("Databases Page", module) - .addDecorator(withRouterProvider) - .addDecorator(withBackground) - .add("with data", () => ) - .add("without data", () => ) - .add("with loading indicator", () => ( - - )); diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx deleted file mode 100644 index 96aae47eaee6..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPage.tsx +++ /dev/null @@ -1,765 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { InlineAlert } from "@cockroachlabs/ui-components"; -import { Tooltip } from "antd"; -import classNames from "classnames/bind"; -import merge from "lodash/merge"; -import React from "react"; -import { RouteComponentProps } from "react-router-dom"; - -import { Anchor } from "src/anchor"; -import { StackIcon } from "src/icon/stackIcon"; -import { PageConfig, PageConfigItem } from "src/pageConfig"; -import { Pagination } from "src/pagination"; -import { BooleanSetting } from "src/settings/booleanSetting"; -import { - ColumnDescriptor, - handleSortSettingFromQueryString, - ISortedTablePagination, - SortedTable, - SortSetting, -} from "src/sortedtable"; -import sortableTableStyles from "src/sortedtable/sortedtable.module.scss"; -import { UIConfigState } from "src/store"; -import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses"; -import { syncHistory, tableStatsClusterSetting, unique } from "src/util"; - -import { - DatabaseSpanStatsRow, - DatabaseTablesResponse, - isMaxSizeError, - SqlApiQueryResponse, - SqlExecutionErrorMessage, -} from "../api"; -import { LoadingCell } from "../databases"; -import { Loading } from "../loading"; -import { - calculateActiveFilters, - defaultFilters, - Filter, - Filters, - handleFiltersFromQueryString, -} from "../queryFilter"; -import { Search } from "../search"; -import booleanSettingStyles from "../settings/booleanSetting.module.scss"; -import LoadingError from "../sqlActivity/errorComponent"; -import { TableStatistics } from "../tableStatistics"; - -import styles from "./databasesPage.module.scss"; -import { - DatabaseNameCell, - IndexRecCell, - DiskSizeCell, -} from "./databaseTableCells"; - -const cx = classNames.bind(styles); -const sortableTableCx = classNames.bind(sortableTableStyles); -const booleanSettingCx = classNames.bind(booleanSettingStyles); - -// We break out separate interfaces for some of the nested objects in our data -// both so that they can be available as SortedTable rows and for making -// (typed) test assertions on narrower slices of the data. -// -// The loading and loaded flags help us know when to dispatch the appropriate -// refresh actions. -export interface DatabasesPageData { - loading: boolean; - loaded: boolean; - // Request error when getting database names. - requestError: Error; - // Query error when getting database names. - queryError: SqlExecutionErrorMessage; - databases: DatabasesPageDataDatabase[]; - sortSetting: SortSetting; - search: string; - filters: Filters; - nodeRegions: { [nodeId: string]: string }; - isTenant?: UIConfigState["isTenant"]; - automaticStatsCollectionEnabled?: boolean; - indexRecommendationsEnabled: boolean; - showNodeRegionsColumn?: boolean; - csIndexUnusedDuration: string; -} - -export interface DatabasesPageDataDatabase { - detailsLoading: boolean; - detailsLoaded: boolean; - - spanStatsLoading: boolean; - spanStatsLoaded: boolean; - - // Request error when getting database details. - detailsRequestError: Error; - spanStatsRequestError: Error; - // Query error when getting database details. - detailsQueryError: SqlExecutionErrorMessage; - spanStatsQueryError: SqlExecutionErrorMessage; - name: string; - spanStats?: SqlApiQueryResponse; - tables?: SqlApiQueryResponse; - // Array of node IDs used to unambiguously filter by node and region. - nodes?: number[]; - // String of nodes grouped by region in alphabetical order, e.g. - // regionA(n1,n2), regionB(n3). Used for display in the table's - // "Regions/Nodes" column. - nodesByRegionString?: string; - numIndexRecommendations: number; -} - -export interface DatabasesPageActions { - refreshDatabases: () => void; - refreshDatabaseDetails: ( - database: string, - csIndexUnusedDuration: string, - ) => void; - refreshDatabaseSpanStats: (database: string) => void; - refreshSettings: () => void; - refreshNodes?: () => void; - onFilterChange?: (value: Filters) => void; - onSearchComplete?: (query: string) => void; - onSortingChange?: ( - name: string, - columnTitle: string, - ascending: boolean, - ) => void; -} - -export type DatabasesPageProps = DatabasesPageData & - DatabasesPageActions & - RouteComponentProps; - -interface DatabasesPageState { - pagination: ISortedTablePagination; - filters?: Filters; - activeFilters?: number; - columns: ColumnDescriptor[]; -} - -class DatabasesSortedTable extends SortedTable {} - -// filterBySearchQuery returns true if the search query matches the database name. -function filterBySearchQuery( - database: DatabasesPageDataDatabase, - search: string, -): boolean { - const matchString = database.name.toLowerCase(); - - if (search.startsWith('"') && search.endsWith('"')) { - search = search.substring(1, search.length - 1); - - return matchString.includes(search); - } - - return search - .toLowerCase() - .split(" ") - .every(val => matchString.includes(val)); -} - -const tablePageSize = 20; -const disableTableSortSize = tablePageSize * 2; - -export class DatabasesPage extends React.Component< - DatabasesPageProps, - DatabasesPageState -> { - constructor(props: DatabasesPageProps) { - super(props); - - this.state = { - filters: defaultFilters, - pagination: { - current: 1, - pageSize: tablePageSize, - }, - columns: this.columns(), - }; - - const stateFromHistory = this.getStateFromHistory(); - this.state = merge(this.state, stateFromHistory); - - const { history } = this.props; - const searchParams = new URLSearchParams(history.location.search); - const ascending = (searchParams.get("ascending") || undefined) === "true"; - const columnTitle = searchParams.get("columnTitle") || undefined; - const sortSetting = this.props.sortSetting; - - if ( - this.props.onSortingChange && - columnTitle && - (sortSetting.columnTitle !== columnTitle || - sortSetting.ascending !== ascending) - ) { - this.props.onSortingChange("Databases", columnTitle, ascending); - } - } - - getStateFromHistory = (): Partial => { - const { - history, - search, - sortSetting, - filters, - onFilterChange, - onSearchComplete, - onSortingChange, - } = this.props; - - const searchParams = new URLSearchParams(history.location.search); - - const searchQuery = searchParams.get("q") || undefined; - if (onSearchComplete && searchQuery && search !== searchQuery) { - onSearchComplete(searchQuery); - } - - handleSortSettingFromQueryString( - "Databases", - history.location.search, - sortSetting, - onSortingChange, - ); - - const latestFilter = handleFiltersFromQueryString( - history, - filters, - onFilterChange, - ); - - return { - filters: latestFilter, - activeFilters: calculateActiveFilters(latestFilter), - }; - }; - - componentDidMount(): void { - if (this.props.refreshNodes != null && !this.props.isTenant) { - this.props.refreshNodes(); - } - - if (this.props.refreshSettings != null) { - this.props.refreshSettings(); - } - - if ( - !this.props.loaded && - !this.props.loading && - this.props.requestError === undefined - ) { - return this.props.refreshDatabases(); - } else { - // If the props are already loaded then componentDidUpdate - // will not get called so call refresh to make sure details - // are loaded - this.refresh(); - } - } - - updateQueryParams(): void { - const { history, search } = this.props; - - // Search - const searchParams = new URLSearchParams(history.location.search); - const searchQueryString = searchParams.get("q") || ""; - if (search && search !== searchQueryString) { - syncHistory( - { - q: search, - }, - history, - ); - } - } - - componentDidUpdate( - prevProps: Readonly, - prevState: Readonly, - ): void { - if (this.shouldRefreshDatabaseInformation(prevState, prevProps)) { - this.updateQueryParams(); - this.refresh(); - } - if ( - prevProps.indexRecommendationsEnabled !== - this.props.indexRecommendationsEnabled || - prevProps.showNodeRegionsColumn !== this.props.showNodeRegionsColumn - ) { - this.setState({ columns: this.columns() }); - } - } - - private refresh(): void { - // load everything by default - let filteredDbs: DatabasesPageDataDatabase[] = this.props.databases; - - // Loading only the first page if there are more than - // 40 dbs. If there is more than 40 dbs sort will be disabled. - if (this.props.databases.length > disableTableSortSize) { - const startIndex = - this.state.pagination.pageSize * (this.state.pagination.current - 1); - // Result maybe filtered so get db names from filtered results - if (this.props.search && this.props.search.length > 0) { - filteredDbs = this.filteredDatabasesData(); - } - - if (!filteredDbs || filteredDbs.length === 0) { - return; - } - - // Only load the first page - filteredDbs = filteredDbs.slice( - startIndex, - startIndex + this.state.pagination.pageSize, - ); - } - - filteredDbs.forEach(database => { - if ( - !database.detailsLoaded && - !database.detailsLoading && - database.detailsRequestError == null - ) { - this.props.refreshDatabaseDetails( - database.name, - this.props.csIndexUnusedDuration, - ); - } - if ( - !database.spanStatsLoaded && - !database.spanStatsLoading && - database.spanStatsRequestError == null - ) { - this.props.refreshDatabaseSpanStats(database.name); - } - }); - } - - changePage = (current: number): void => { - this.setState({ pagination: { ...this.state.pagination, current } }); - }; - - changeSortSetting = (ss: SortSetting): void => { - syncHistory( - { - ascending: ss.ascending.toString(), - columnTitle: ss.columnTitle, - }, - this.props.history, - ); - if (this.props.onSortingChange) { - this.props.onSortingChange("Databases", ss.columnTitle, ss.ascending); - } - }; - - resetPagination = (): void => { - this.setState(prevState => { - return { - pagination: { - current: 1, - pageSize: prevState.pagination.pageSize, - }, - }; - }); - }; - - onClearSearchField = (): void => { - if (this.props.onSearchComplete) { - this.props.onSearchComplete(""); - } - - syncHistory( - { - q: undefined, - }, - this.props.history, - ); - }; - - onClearFilters = (): void => { - if (this.props.onFilterChange) { - this.props.onFilterChange(defaultFilters); - } - - this.setState({ - filters: defaultFilters, - activeFilters: 0, - }); - - this.resetPagination(); - syncHistory( - { - regions: undefined, - nodes: undefined, - }, - this.props.history, - ); - }; - - onSubmitSearchField = (search: string): void => { - if (this.props.onSearchComplete) { - this.props.onSearchComplete(search); - } - - this.resetPagination(); - syncHistory( - { - q: search, - }, - this.props.history, - ); - }; - - onSubmitFilters = (filters: Filters): void => { - if (this.props.onFilterChange) { - this.props.onFilterChange(filters); - } - - this.setState({ - filters: filters, - activeFilters: calculateActiveFilters(filters), - }); - - this.resetPagination(); - syncHistory( - { - regions: filters.regions, - nodes: filters.nodes, - }, - this.props.history, - ); - }; - - // Returns a list of databases to the display based on input from the search - // box and the applied filters. - filteredDatabasesData = (): DatabasesPageDataDatabase[] => { - const { search, databases, filters, nodeRegions } = this.props; - - // The regions and nodes selected from the filter dropdown. - const regionsSelected = - filters.regions?.length > 0 ? filters.regions.split(",") : []; - const nodesSelected = - filters.nodes?.length > 0 ? filters.nodes.split(",") : []; - - return databases - .filter(db => (search ? filterBySearchQuery(db, search) : true)) - .filter(db => { - if (regionsSelected.length === 0 && nodesSelected.length === 0) - return true; - - let foundRegion = regionsSelected.length === 0; - let foundNode = nodesSelected.length === 0; - - db.nodes?.forEach(node => { - const n = node?.toString() || ""; - if (foundRegion || regionsSelected.includes(nodeRegions[n])) { - foundRegion = true; - } - if (foundNode || nodesSelected.includes("n" + n)) { - foundNode = true; - } - if (foundNode && foundRegion) return true; - }); - - return foundRegion && foundNode; - }); - }; - - private shouldRefreshDatabaseInformation( - prevState: Readonly, - prevProps: Readonly, - ): boolean { - // No new dbs to update - if ( - !this.props.databases || - this.props.databases.length === 0 || - this.props.databases.every( - x => - x.detailsLoading || - x.detailsLoaded || - x.spanStatsLoaded || - x.spanStatsLoading, - ) - ) { - return false; - } - - if (this.state.pagination.current !== prevState.pagination.current) { - return true; - } - - if (prevProps && this.props.search !== prevProps.search) { - return true; - } - - const filteredDatabases = this.filteredDatabasesData(); - for ( - let i = 0; - i < filteredDatabases.length && i < disableTableSortSize; - i++ - ) { - const db = filteredDatabases[i]; - if ( - db.detailsLoaded || - db.detailsLoading || - db.detailsRequestError != null - ) { - continue; - } - if ( - db.spanStatsLoading || - db.spanStatsLoaded || - db.spanStatsRequestError != null - ) { - continue; - } - // Info is not loaded for a visible database. - return true; - } - - return false; - } - - private columns(): ColumnDescriptor[] { - const columns: ColumnDescriptor[] = [ - { - title: ( - - Databases - - ), - cell: database => , - sort: database => database.name, - className: cx("databases-table__col-name"), - name: "name", - }, - { - title: ( - - Size - - ), - cell: database => , - sort: database => database.spanStats?.approximate_disk_bytes, - className: cx("databases-table__col-size"), - name: "size", - }, - { - title: ( - - Tables - - ), - cell: database => ( - - {database.tables?.tables?.length} - - ), - sort: database => database.tables?.tables.length ?? 0, - className: cx("databases-table__col-table-count"), - name: "tableCount", - }, - { - title: ( - - {this.props.isTenant ? "Regions" : "Regions/Nodes"} - - ), - cell: database => ( - - {database.nodesByRegionString ? database.nodesByRegionString : null} - - ), - sort: database => database.nodesByRegionString, - className: cx("databases-table__col-node-regions"), - name: "nodeRegions", - hideIfTenant: true, - showByDefault: this.props.showNodeRegionsColumn, - }, - ]; - if (this.props.indexRecommendationsEnabled) { - columns.push({ - title: ( - - Index Recommendations - - ), - cell: database => , - sort: database => database.numIndexRecommendations, - className: cx("databases-table__col-idx-rec"), - name: "numIndexRecommendations", - }); - } - return columns; - } - - render(): React.ReactElement { - const displayColumns = this.state.columns.filter( - col => col.showByDefault !== false, - ); - - const { filters, search, nodeRegions, isTenant } = this.props; - const { pagination } = this.state; - - const databasesToDisplay = this.filteredDatabasesData(); - const activeFilters = calculateActiveFilters(filters); - - const nodes = Object.keys(nodeRegions) - .map(n => Number(n)) - .sort(); - - const regions = unique(Object.values(nodeRegions)); - const showNodes = !isTenant && nodes.length > 1; - const showRegions = regions.length > 1; - - // Only show the databases filter if at least one drop-down is shown. - const databasesFilter = - showNodes || showRegions ? ( - - "n" + n?.toString())} - activeFilters={activeFilters} - filters={defaultFilters} - onSubmitFilters={this.onSubmitFilters} - showNodes={showNodes} - showRegions={showRegions} - /> - - ) : ( - <> - ); - - return ( -
-
-

Databases

- {this.props.automaticStatsCollectionEnabled != null && ( - - {" "} - Automatic statistics can help improve query performance. Learn - how to{" "} - - manage statistics collection - - . - - } - /> - )} -
-
- - - - - {databasesFilter} - - - - LoadingError({ - statsType: "databases", - error: isMaxSizeError(this.props.queryError?.message) - ? new Error(this.props.queryError?.message) - : this.props.requestError, - }) - } - > - {isMaxSizeError(this.props.queryError?.message) && ( - - Not all databases are displayed because the maximum number - of databases was reached in the console.  - - } - /> - )} - - - This cluster has no databases. -
- } - /> - - - - - - ); - } -} diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPageConnected.ts b/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPageConnected.ts deleted file mode 100644 index 057a672af692..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/databasesPageConnected.ts +++ /dev/null @@ -1,153 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { connect } from "react-redux"; -import { withRouter } from "react-router-dom"; -import { Dispatch } from "redux"; - -import { deriveDatabaseDetailsMemoized } from "../databases"; -import { Filters } from "../queryFilter"; -import { AppState } from "../store"; -import { actions as analyticsActions } from "../store/analytics"; -import { actions as clusterSettingsActions } from "../store/clusterSettings"; -import { - selectAutomaticStatsCollectionEnabled, - selectDropUnusedIndexDuration, - selectIndexRecommendationsEnabled, -} from "../store/clusterSettings/clusterSettings.selectors"; -import { - databaseDetailsReducer, - databaseDetailsSpanStatsReducer, -} from "../store/databaseDetails"; -import { actions as databasesListActions } from "../store/databasesList"; -import { - databasesListSelector, - selectDatabasesFilters, - selectDatabasesSearch, - selectDatabasesSortSetting, -} from "../store/databasesList/databasesList.selectors"; -import { - actions as localStorageActions, - LocalStorageKeys, -} from "../store/localStorage"; -import { - actions as nodesActions, - nodeRegionsByIDSelector, -} from "../store/nodes"; -import { selectIsTenant } from "../store/uiConfig"; - -import { - DatabasesPage, - DatabasesPageActions, - DatabasesPageData, -} from "./databasesPage"; - -const databaseDetailsActions = databaseDetailsReducer.actions; -const databaseDetailsSpanStatsActions = databaseDetailsSpanStatsReducer.actions; - -const mapStateToProps = (state: AppState): DatabasesPageData => { - const databasesListState = databasesListSelector(state); - const nodeRegions = nodeRegionsByIDSelector(state); - const isTenant = selectIsTenant(state); - return { - loading: !!databasesListState?.inFlight, - loaded: !!databasesListState?.valid, - requestError: databasesListState?.lastError, - queryError: databasesListState?.data?.error, - databases: deriveDatabaseDetailsMemoized({ - dbListResp: databasesListState?.data, - databaseDetails: state.adminUI?.databaseDetails, - spanStats: state.adminUI?.databaseDetailsSpanStats, - nodeRegions, - isTenant, - nodeStatuses: state.adminUI.nodes.data, - }), - sortSetting: selectDatabasesSortSetting(state), - search: selectDatabasesSearch(state), - filters: selectDatabasesFilters(state), - nodeRegions, - isTenant, - automaticStatsCollectionEnabled: - selectAutomaticStatsCollectionEnabled(state), - // Do not show node/regions columns for serverless. - indexRecommendationsEnabled: selectIndexRecommendationsEnabled(state), - showNodeRegionsColumn: Object.keys(nodeRegions).length > 1 && !isTenant, - csIndexUnusedDuration: selectDropUnusedIndexDuration(state), - }; -}; - -const mapDispatchToProps = (dispatch: Dispatch): DatabasesPageActions => ({ - refreshDatabases: () => { - dispatch(databasesListActions.refresh()); - }, - refreshDatabaseDetails: (database: string, csIndexUnusedDuration: string) => { - dispatch( - databaseDetailsActions.refresh({ database, csIndexUnusedDuration }), - ); - }, - refreshDatabaseSpanStats: (database: string) => { - dispatch(databaseDetailsSpanStatsActions.refresh({ database })); - }, - refreshSettings: () => { - dispatch(clusterSettingsActions.refresh()); - }, - refreshNodes: () => { - dispatch(nodesActions.refresh()); - }, - onFilterChange: (filters: Filters) => { - dispatch( - analyticsActions.track({ - name: "Filter Clicked", - page: "Databases", - filterName: "filters", - value: filters.toString(), - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_FILTERS, - value: filters, - }), - ); - }, - onSearchComplete: (query: string) => { - dispatch( - analyticsActions.track({ - name: "Keyword Searched", - page: "Databases", - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_SEARCH, - value: query, - }), - ); - }, - onSortingChange: ( - tableName: string, - columnName: string, - ascending: boolean, - ) => { - dispatch( - analyticsActions.track({ - name: "Column Sorted", - page: "Databases", - tableName, - columnName, - }), - ); - dispatch( - localStorageActions.update({ - key: LocalStorageKeys.DB_SORT, - value: { columnTitle: columnName, ascending: ascending }, - }), - ); - }, -}); - -export const ConnectedDatabasesPage = withRouter( - connect(mapStateToProps, mapDispatchToProps)(DatabasesPage), -); diff --git a/pkg/ui/workspaces/cluster-ui/src/databasesPage/index.ts b/pkg/ui/workspaces/cluster-ui/src/databasesPage/index.ts deleted file mode 100644 index 26e4803bfa70..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/databasesPage/index.ts +++ /dev/null @@ -1,7 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -export * from "./databasesPage"; -export * from "./databasesPageConnected"; diff --git a/pkg/ui/workspaces/cluster-ui/src/index.ts b/pkg/ui/workspaces/cluster-ui/src/index.ts index e3316b99c26c..156d82f07c7c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/index.ts @@ -11,9 +11,6 @@ export * from "./badge"; export * from "./barCharts"; export * from "./button"; export * from "./common"; -export * from "./databaseDetailsPage"; -export * from "./databaseTablePage"; -export * from "./databasesPage"; export * from "./delayed"; export * from "./downloadFile"; export * from "./dropdown"; @@ -50,7 +47,6 @@ export * from "./graphs"; export * from "./selectors"; export * from "./contexts"; export * from "./timestamp"; -export * from "./databases"; export * from "./antdTheme"; export * from "./databasesV2"; export * from "./databaseDetailsV2"; diff --git a/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.stories.tsx index a93a31d9a847..b6d768b70722 100644 --- a/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.stories.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.stories.tsx @@ -40,7 +40,7 @@ const withData: IndexDetailsPageProps = { ], }, breadcrumbItems: [ - { link: "/legacy/databases", name: "Databases" }, + { link: "/databases", name: "Databases" }, { link: `/databases/story_db`, name: "Tables", diff --git a/pkg/ui/workspaces/cluster-ui/src/sql/box.tsx b/pkg/ui/workspaces/cluster-ui/src/sql/box.tsx index ed1698ecc646..708386a4ffd2 100644 --- a/pkg/ui/workspaces/cluster-ui/src/sql/box.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/sql/box.tsx @@ -3,25 +3,38 @@ // Use of this software is governed by the CockroachDB Software License // included in the /LICENSE file. +import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; import classNames from "classnames/bind"; import React from "react"; import { FormatQuery } from "src/util"; -import { api as clusterUiApi } from "../index"; - import { Highlight } from "./highlight"; import styles from "./sqlhighlight.module.scss"; +type ZoneConfigType = cockroach.config.zonepb.ZoneConfig; +type ZoneConfigLevelType = cockroach.server.serverpb.ZoneConfigurationLevel; + export enum SqlBoxSize { SMALL = "small", LARGE = "large", CUSTOM = "custom", } +type DatabaseZoneConfig = { + zone_config: ZoneConfigType; + zone_config_level: ZoneConfigLevelType; +}; + export interface SqlBoxProps { value: string; - zone?: clusterUiApi.DatabaseDetailsResponse; + // (xinhaoz): Came across this while deleting legacy db pages. + // It doesn't seem like there are any usages of this prop today. + // It may have been from a time where we showed the create statement + // for a database. + // Created DatabaseZoneConfig as a replacement until we decide + // whether to bring back create db statement. + zone?: DatabaseZoneConfig; className?: string; size?: SqlBoxSize; format?: boolean; diff --git a/pkg/ui/workspaces/cluster-ui/src/sql/highlight.tsx b/pkg/ui/workspaces/cluster-ui/src/sql/highlight.tsx index 8e2109dd8a8c..d138f9695d44 100644 --- a/pkg/ui/workspaces/cluster-ui/src/sql/highlight.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/sql/highlight.tsx @@ -35,7 +35,7 @@ export class Highlight extends React.Component { renderZone = (): React.ReactElement => { const { zone } = this.props; - const zoneConfig = zone.zoneConfigResp.zone_config; + const zoneConfig = zone.zone_config; return ( CONFIGURE ZONE USING diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.reducer.ts deleted file mode 100644 index f8398da50809..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.reducer.ts +++ /dev/null @@ -1,138 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { createSlice, PayloadAction } from "@reduxjs/toolkit"; - -import { - DatabaseDetailsReqParams, - DatabaseDetailsResponse, - DatabaseDetailsSpanStatsReqParams, - DatabaseDetailsSpanStatsResponse, - ErrorWithKey, - SqlApiResponse, -} from "../../api"; -import { DOMAIN_NAME } from "../utils"; - -type DatabaseDetailsWithKey = { - databaseDetailsResponse: SqlApiResponse; - key: string; // Database name. -}; - -type DatabaseDetailsSpanStatsWithKey = { - response: SqlApiResponse; - key: string; // Database name. -}; - -export type DatabaseDetailsState = { - data?: SqlApiResponse; - // Captures thrown errors. - lastError?: Error; - valid: boolean; - inFlight: boolean; -}; - -export type DatabaseDetailsSpanStatsState = { - data?: SqlApiResponse; - lastError?: Error; - valid: boolean; - inFlight: boolean; -}; - -export type KeyedDatabaseDetailsState = { - [dbName: string]: DatabaseDetailsState; -}; - -export type KeyedDatabaseDetailsSpanStatsState = { - [dbName: string]: DatabaseDetailsSpanStatsState; -}; - -// const initialState: KeyedDatabaseDetailsState = {}; - -export const databaseDetailsReducer = createSlice({ - name: `${DOMAIN_NAME}/databaseDetails`, - initialState: {} as KeyedDatabaseDetailsState, - reducers: { - received: (state, action: PayloadAction) => { - state[action.payload.key] = { - valid: true, - inFlight: false, - data: action.payload.databaseDetailsResponse, - lastError: null, - }; - }, - failed: (state, action: PayloadAction) => { - state[action.payload.key] = { - valid: false, - inFlight: false, - data: null, - lastError: action.payload.err, - }; - }, - refresh: (state, action: PayloadAction) => { - state[action.payload.database] = { - valid: false, - inFlight: true, - data: null, - lastError: null, - }; - }, - request: (state, action: PayloadAction) => { - state[action.payload.database] = { - valid: false, - inFlight: true, - data: null, - lastError: null, - }; - }, - }, -}); - -export const databaseDetailsSpanStatsReducer = createSlice({ - name: `${DOMAIN_NAME}/databaseDetailsSpanStats`, - initialState: {} as KeyedDatabaseDetailsSpanStatsState, - reducers: { - received: ( - state, - action: PayloadAction, - ) => { - state[action.payload.key] = { - valid: true, - inFlight: false, - data: action.payload.response, - lastError: null, - }; - }, - failed: (state, action: PayloadAction) => { - state[action.payload.key] = { - valid: false, - inFlight: false, - data: null, - lastError: action.payload.err, - }; - }, - refresh: ( - state, - action: PayloadAction, - ) => { - state[action.payload.database] = { - valid: false, - inFlight: true, - data: null, - lastError: null, - }; - }, - request: ( - state, - action: PayloadAction, - ) => { - state[action.payload.database] = { - valid: false, - inFlight: true, - data: null, - lastError: null, - }; - }, - }, -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.saga.spec.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.saga.spec.ts deleted file mode 100644 index bd217945f993..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.saga.spec.ts +++ /dev/null @@ -1,151 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import { PayloadAction } from "@reduxjs/toolkit"; -import { expectSaga } from "redux-saga-test-plan"; -import * as matchers from "redux-saga-test-plan/matchers"; -import { - EffectProviders, - StaticProvider, - throwError, -} from "redux-saga-test-plan/providers"; - -import { indexUnusedDuration } from "src/util/constants"; - -import { - DatabaseDetailsReqParams, - DatabaseDetailsResponse, - getDatabaseDetails, - SqlApiResponse, -} from "../../api"; - -import { - KeyedDatabaseDetailsState, - databaseDetailsReducer, -} from "./databaseDetails.reducer"; -import { - refreshDatabaseDetailsSaga, - requestDatabaseDetailsSaga, -} from "./databaseDetails.saga"; - -import ZoneConfig = cockroach.config.zonepb.ZoneConfig; -import ZoneConfigurationLevel = cockroach.server.serverpb.ZoneConfigurationLevel; -const { actions, reducer } = databaseDetailsReducer; - -describe("DatabaseDetails sagas", () => { - const database = "test_db"; - const csIndexUnusedDuration = indexUnusedDuration; - const requestAction: PayloadAction = { - payload: { database, csIndexUnusedDuration }, - type: "request", - }; - const databaseDetailsResponse: SqlApiResponse = { - maxSizeReached: false, - results: { - idResp: { database_id: "mock_id" }, - grantsResp: { - grants: [ - { - user: "test_user", - privileges: ["privilege1", "privilege2", "privilege3"], - }, - { - user: "another_user", - privileges: ["privilege1", "privilege4", "privilege7"], - }, - ], - }, - tablesResp: { - tables: [ - { - schema: "schema", - table: "table", - qualifiedNameWithSchemaAndTable: `"schema"."table"`, - }, - { - schema: "schema2", - table: "table2", - qualifiedNameWithSchemaAndTable: `"schema"."table"`, - }, - { - table: "tabble", - schema: "schema", - qualifiedNameWithSchemaAndTable: `"schema2j"."table2"`, - }, - ], - }, - zoneConfigResp: { - zone_config: new ZoneConfig({ - inherited_constraints: true, - inherited_lease_preferences: true, - }), - zone_config_level: ZoneConfigurationLevel.CLUSTER, - }, - stats: { - replicaData: { - storeIDs: [1, 2, 3], - }, - indexStats: { num_index_recommendations: 4 }, - }, - }, - }; - const databaseDetailsAPIProvider: (EffectProviders | StaticProvider)[] = [ - [matchers.call.fn(getDatabaseDetails), databaseDetailsResponse], - ]; - - describe("refreshDatabaseDetailsSaga", () => { - it("dispatches request DatabaseDetails action", () => { - return expectSaga(refreshDatabaseDetailsSaga, requestAction) - .put(actions.request(requestAction.payload)) - .run(); - }); - }); - - describe("requestDatabaseDetailsSaga", () => { - it("successfully requests database details", () => { - return expectSaga(requestDatabaseDetailsSaga, requestAction) - .provide(databaseDetailsAPIProvider) - .put( - actions.received({ - databaseDetailsResponse, - key: database, - }), - ) - .withReducer(reducer) - .hasFinalState({ - [database]: { - data: databaseDetailsResponse, - lastError: null, - valid: true, - inFlight: false, - }, - }) - .run(); - }); - - it("returns error on failed request", () => { - const error = new Error("Failed request"); - return expectSaga(requestDatabaseDetailsSaga, requestAction) - .provide([[matchers.call.fn(getDatabaseDetails), throwError(error)]]) - .put( - actions.failed({ - err: error, - key: database, - }), - ) - .withReducer(reducer) - .hasFinalState({ - [database]: { - data: null, - lastError: error, - valid: false, - inFlight: false, - }, - }) - .run(); - }); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.saga.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.saga.ts deleted file mode 100644 index 04503d159b36..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.saga.ts +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { PayloadAction } from "@reduxjs/toolkit"; -import moment from "moment"; -import { all, call, put, takeEvery } from "redux-saga/effects"; - -import { - DatabaseDetailsReqParams, - ErrorWithKey, - getDatabaseDetails, -} from "src/api"; - -import { maybeError } from "../../util"; - -import { databaseDetailsReducer } from "./databaseDetails.reducer"; - -const actions = databaseDetailsReducer.actions; -export function* refreshDatabaseDetailsSaga( - action: PayloadAction, -) { - yield put(actions.request(action.payload)); -} - -export function* requestDatabaseDetailsSaga( - action: PayloadAction, -): any { - try { - const result = yield call( - getDatabaseDetails, - action.payload, - moment.duration(10, "m"), - ); - yield put( - actions.received({ - key: action.payload.database, - databaseDetailsResponse: result, - }), - ); - } catch (e) { - const err: ErrorWithKey = { - err: maybeError(e), - key: action.payload.database, - }; - yield put(actions.failed(err)); - } -} - -export function* databaseDetailsSaga() { - yield all([ - takeEvery(actions.refresh, refreshDatabaseDetailsSaga), - takeEvery(actions.request, requestDatabaseDetailsSaga), - ]); -} diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.selectors.ts deleted file mode 100644 index 9930626b2ab5..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetails.selectors.ts +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { LocalStorageKeys } from "../localStorage"; -import { AppState } from "../reducers"; -import { localStorageSelector } from "../utils/selectors"; - -export const selectDatabaseDetailsViewModeSetting = (state: AppState) => { - const localStorage = localStorageSelector(state); - return localStorage[LocalStorageKeys.DB_DETAILS_VIEW_MODE]; -}; - -export const selectDatabaseDetailsTablesSortSetting = (state: AppState) => { - const localStorage = localStorageSelector(state); - return localStorage[LocalStorageKeys.DB_DETAILS_TABLES_PAGE_SORT]; -}; - -export const selectDatabaseDetailsGrantsSortSetting = (state: AppState) => { - const localStorage = localStorageSelector(state); - return localStorage[LocalStorageKeys.DB_DETAILS_GRANTS_PAGE_SORT]; -}; - -export const selectDatabaseDetailsTablesFiltersSetting = (state: AppState) => { - const localStorage = localStorageSelector(state); - return localStorage[LocalStorageKeys.DB_DETAILS_TABLES_PAGE_FILTERS]; -}; - -export const selectDatabaseDetailsTablesSearchSetting = (state: AppState) => { - const localStorage = localStorageSelector(state); - return localStorage[LocalStorageKeys.DB_DETAILS_TABLES_PAGE_SEARCH]; -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetailsSpanStats.saga.spec.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetailsSpanStats.saga.spec.ts deleted file mode 100644 index d935832b1f9f..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetailsSpanStats.saga.spec.ts +++ /dev/null @@ -1,107 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { PayloadAction } from "@reduxjs/toolkit"; -import { expectSaga } from "redux-saga-test-plan"; -import * as matchers from "redux-saga-test-plan/matchers"; -import { - EffectProviders, - StaticProvider, - throwError, -} from "redux-saga-test-plan/providers"; - -import { - DatabaseDetailsSpanStatsReqParams, - DatabaseDetailsSpanStatsResponse, - getDatabaseDetailsSpanStats, - SqlApiResponse, -} from "../../api"; - -import { - databaseDetailsSpanStatsReducer, - KeyedDatabaseDetailsSpanStatsState, -} from "./databaseDetails.reducer"; -import { - refreshDatabaseDetailsSpanStatsSaga, - requestDatabaseDetailsSpanStatsSaga, -} from "./databaseDetailsSpanStats.saga"; -const { actions, reducer } = databaseDetailsSpanStatsReducer; - -describe("DatabaseDetails sagas", () => { - const database = "test_db"; - const requestAction: PayloadAction = { - payload: { database }, - type: "request", - }; - const spanStatsResponse: SqlApiResponse = { - maxSizeReached: false, - results: { - spanStats: { - approximate_disk_bytes: 100, - live_bytes: 200, - total_bytes: 400, - error: undefined, - }, - }, - }; - const provider: (EffectProviders | StaticProvider)[] = [ - [matchers.call.fn(getDatabaseDetailsSpanStats), spanStatsResponse], - ]; - - describe("refreshSpanStatsSaga", () => { - it("dispatches request span stats action", () => { - return expectSaga(refreshDatabaseDetailsSpanStatsSaga, requestAction) - .put(actions.request(requestAction.payload)) - .run(); - }); - }); - - describe("request span stats saga", () => { - it("successfully requests span stats", () => { - return expectSaga(requestDatabaseDetailsSpanStatsSaga, requestAction) - .provide(provider) - .put( - actions.received({ - response: spanStatsResponse, - key: database, - }), - ) - .withReducer(reducer) - .hasFinalState({ - [database]: { - data: spanStatsResponse, - lastError: null, - valid: true, - inFlight: false, - }, - }) - .run(); - }); - - it("returns error on failed request", () => { - const error = new Error("Failed request"); - return expectSaga(requestDatabaseDetailsSpanStatsSaga, requestAction) - .provide([ - [matchers.call.fn(getDatabaseDetailsSpanStats), throwError(error)], - ]) - .put( - actions.failed({ - err: error, - key: database, - }), - ) - .withReducer(reducer) - .hasFinalState({ - [database]: { - data: null, - lastError: error, - valid: false, - inFlight: false, - }, - }) - .run(); - }); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetailsSpanStats.saga.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetailsSpanStats.saga.ts deleted file mode 100644 index 03469f6df3dc..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/databaseDetailsSpanStats.saga.ts +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { PayloadAction } from "@reduxjs/toolkit"; -import { all, call, put, takeEvery } from "redux-saga/effects"; - -import { - DatabaseDetailsSpanStatsReqParams, - ErrorWithKey, - getDatabaseDetailsSpanStats, -} from "src/api"; - -import { maybeError } from "../../util"; - -import { databaseDetailsSpanStatsReducer } from "./databaseDetails.reducer"; - -const actions = databaseDetailsSpanStatsReducer.actions; -export function* refreshDatabaseDetailsSpanStatsSaga( - action: PayloadAction, -) { - yield put(actions.request(action.payload)); -} - -export function* requestDatabaseDetailsSpanStatsSaga( - action: PayloadAction, -): any { - try { - const result = yield call(getDatabaseDetailsSpanStats, action.payload); - yield put( - actions.received({ - key: action.payload.database, - response: result, - }), - ); - } catch (e) { - const err: ErrorWithKey = { - err: maybeError(e), - key: action.payload.database, - }; - yield put(actions.failed(err)); - } -} - -export function* databaseDetailsSpanStatsSaga() { - yield all([ - takeEvery(actions.refresh, refreshDatabaseDetailsSpanStatsSaga), - takeEvery(actions.request, requestDatabaseDetailsSpanStatsSaga), - ]); -} diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/index.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/index.ts deleted file mode 100644 index 3042b78b0d7c..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseDetails/index.ts +++ /dev/null @@ -1,7 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -export * from "./databaseDetails.reducer"; -export * from "./databaseDetails.saga"; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/index.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/index.ts deleted file mode 100644 index fa157ce0db7d..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/index.ts +++ /dev/null @@ -1,7 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -export * from "./tableDetails.reducer"; -export * from "./tableDetails.saga"; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.reducer.ts deleted file mode 100644 index 9f940a8e295e..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.reducer.ts +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { createSlice, PayloadAction } from "@reduxjs/toolkit"; - -import { - ErrorWithKey, - SqlApiResponse, - TableDetailsReqParams, - TableDetailsResponse, -} from "../../api"; -import { generateTableID } from "../../util"; -import { DOMAIN_NAME } from "../utils"; - -type TableDetailsWithKey = { - tableDetailsResponse: SqlApiResponse; - key: string; -}; - -export type TableDetailsState = { - data?: SqlApiResponse; - // Captures thrown errors. - lastError?: Error; - valid: boolean; - inFlight: boolean; -}; - -export type KeyedTableDetailsState = { - [tableID: string]: TableDetailsState; -}; - -const initialState: KeyedTableDetailsState = {}; - -const tableDetailsReducer = createSlice({ - name: `${DOMAIN_NAME}/tableDetails`, - initialState, - reducers: { - received: (state, action: PayloadAction) => { - state[action.payload.key] = { - valid: true, - inFlight: false, - data: action.payload.tableDetailsResponse, - lastError: null, - }; - }, - failed: (state, action: PayloadAction) => { - state[action.payload.key] = { - valid: false, - inFlight: false, - data: null, - lastError: action.payload.err, - }; - }, - refresh: (state, action: PayloadAction) => { - state[generateTableID(action.payload.database, action.payload.table)] = { - valid: false, - inFlight: true, - data: null, - lastError: null, - }; - }, - request: (state, action: PayloadAction) => { - state[generateTableID(action.payload.database, action.payload.table)] = { - valid: false, - inFlight: true, - data: null, - lastError: null, - }; - }, - }, -}); - -export const { reducer, actions } = tableDetailsReducer; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.saga.spec.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.saga.spec.ts deleted file mode 100644 index fe31a8943182..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.saga.spec.ts +++ /dev/null @@ -1,151 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import { PayloadAction } from "@reduxjs/toolkit"; -import moment from "moment"; -import { expectSaga } from "redux-saga-test-plan"; -import * as matchers from "redux-saga-test-plan/matchers"; -import { - EffectProviders, - StaticProvider, - throwError, -} from "redux-saga-test-plan/providers"; - -import { - TableDetailsResponse, - getTableDetails, - SqlApiResponse, - TableDetailsReqParams, -} from "../../api"; -import { generateTableID, indexUnusedDuration } from "../../util"; - -import { - actions, - KeyedTableDetailsState, - reducer, -} from "./tableDetails.reducer"; -import { - refreshTableDetailsSaga, - requestTableDetailsSaga, -} from "./tableDetails.saga"; - -import ZoneConfig = cockroach.config.zonepb.ZoneConfig; -import ZoneConfigurationLevel = cockroach.server.serverpb.ZoneConfigurationLevel; - -describe("TableDetails sagas", () => { - const database = "test_db"; - const table = "test_table"; - const key = generateTableID(database, table); - const requestAction: PayloadAction = { - payload: { database, table, csIndexUnusedDuration: indexUnusedDuration }, - type: "request", - }; - const tableDetailsResponse: SqlApiResponse = { - maxSizeReached: false, - results: { - idResp: { table_id: "mock_table_id" }, - createStmtResp: { create_statement: "CREATE TABLE test_table (num int)" }, - grantsResp: { - grants: [ - { - user: "test_user", - privileges: ["privilege1", "privilege2", "privilege3"], - }, - { - user: "another_user", - privileges: ["privilege1", "privilege4", "privilege7"], - }, - ], - }, - schemaDetails: { - columns: ["col1", "col2", "col3"], - indexes: ["idx1", "idx2", "idx3"], - }, - zoneConfigResp: { - configure_zone_statement: "", - zone_config: new ZoneConfig({ - inherited_constraints: true, - inherited_lease_preferences: true, - }), - zone_config_level: ZoneConfigurationLevel.CLUSTER, - }, - heuristicsDetails: { stats_last_created_at: moment() }, - stats: { - spanStats: { - approximate_disk_bytes: 400, - live_bytes: 30, - total_bytes: 40, - range_count: 50, - live_percentage: 75, - }, - replicaData: { - storeIDs: [1, 2, 3], - replicaCount: 3, - }, - indexStats: { - has_index_recommendations: false, - }, - }, - }, - }; - - const tableDetailsAPIProvider: (EffectProviders | StaticProvider)[] = [ - [matchers.call.fn(getTableDetails), tableDetailsResponse], - ]; - - describe("refreshTableDetailsSaga", () => { - it("dispatches request TableDetails action", () => { - return expectSaga(refreshTableDetailsSaga, requestAction) - .put(actions.request(requestAction.payload)) - .run(); - }); - }); - - describe("requestTableDetailsSaga", () => { - it("successfully requests table details", () => { - return expectSaga(requestTableDetailsSaga, requestAction) - .provide(tableDetailsAPIProvider) - .put( - actions.received({ - tableDetailsResponse, - key, - }), - ) - .withReducer(reducer) - .hasFinalState({ - [key]: { - data: tableDetailsResponse, - lastError: null, - valid: true, - inFlight: false, - }, - }) - .run(); - }); - - it("returns error on failed request", () => { - const error = new Error("Failed request"); - return expectSaga(requestTableDetailsSaga, requestAction) - .provide([[matchers.call.fn(getTableDetails), throwError(error)]]) - .put( - actions.failed({ - err: error, - key, - }), - ) - .withReducer(reducer) - .hasFinalState({ - [key]: { - data: null, - lastError: error, - valid: false, - inFlight: false, - }, - }) - .run(); - }); - }); -}); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.saga.ts b/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.saga.ts deleted file mode 100644 index 05b22afb6247..000000000000 --- a/pkg/ui/workspaces/cluster-ui/src/store/databaseTableDetails/tableDetails.saga.ts +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright 2023 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { PayloadAction } from "@reduxjs/toolkit"; -import moment from "moment"; -import { all, call, put, takeEvery } from "redux-saga/effects"; - -import { ErrorWithKey, getTableDetails, TableDetailsReqParams } from "src/api"; - -import { generateTableID, maybeError } from "../../util"; - -import { actions } from "./tableDetails.reducer"; - -export function* refreshTableDetailsSaga( - action: PayloadAction, -) { - yield put(actions.request(action.payload)); -} - -export function* requestTableDetailsSaga( - action: PayloadAction, -): any { - const key = generateTableID(action.payload.database, action.payload.table); - try { - const result = yield call( - getTableDetails, - action.payload, - moment.duration(10, "m"), - ); - yield put(actions.received({ key, tableDetailsResponse: result })); - } catch (e) { - const err: ErrorWithKey = { - err: maybeError(e), - key, - }; - yield put(actions.failed(err)); - } -} - -export function* tableDetailsSaga() { - yield all([ - takeEvery(actions.refresh, refreshTableDetailsSaga), - takeEvery(actions.request, requestTableDetailsSaga), - ]); -} diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts index c1e336035610..30adaaf8077c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts @@ -5,25 +5,9 @@ import { createSelector } from "reselect"; -import { Filters } from "../../queryFilter"; -import { SortSetting } from "../../sortedtable"; -import { LocalStorageKeys } from "../localStorage"; -import { AppState } from "../reducers"; -import { adminUISelector, localStorageSelector } from "../utils/selectors"; +import { adminUISelector } from "../utils/selectors"; export const databasesListSelector = createSelector( adminUISelector, adminUiState => adminUiState?.databasesList, ); - -export const selectDatabasesSortSetting = (state: AppState): SortSetting => { - return localStorageSelector(state)[LocalStorageKeys.DB_SORT]; -}; - -export const selectDatabasesFilters = (state: AppState): Filters => { - return localStorageSelector(state)[LocalStorageKeys.DB_FILTERS]; -}; - -export const selectDatabasesSearch = (state: AppState): string => { - return localStorageSelector(state)[LocalStorageKeys.DB_SEARCH]; -}; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts index 14eaf54ca5b1..ac4a22a56a3a 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/localStorage/localStorage.reducer.ts @@ -9,7 +9,6 @@ import { SqlStatsSortType, DEFAULT_STATS_REQ_OPTIONS, } from "src/api/statementsApi"; -import { ViewMode } from "src/databaseDetailsPage/types"; import { WorkloadInsightEventFilters } from "src/insights"; import { defaultFilters, Filters } from "src/queryFilter/"; @@ -79,7 +78,6 @@ export type LocalStorageState = { "typeSetting/JobsPage": number; "statusSetting/JobsPage": string; "showSetting/JobsPage": string; - [LocalStorageKeys.DB_DETAILS_VIEW_MODE]: ViewMode; [LocalStorageKeys.ACTIVE_EXECUTIONS_IS_AUTOREFRESH_ENABLED]: boolean; "requestTime/StatementsPage": moment.Moment; "requestTime/TransactionsPage": moment.Moment; @@ -119,8 +117,6 @@ const defaultNameSortSetting: SortSetting = { columnTitle: "name", }; -export const defaultDatabaseDetailsViewMode = ViewMode.Tables; - const defaultFiltersActiveExecutions = { app: "", executionStatus: "", @@ -273,9 +269,6 @@ const initialState: LocalStorageState = { "statusSetting/JobsPage": JSON.parse(localStorage.getItem("statusSetting/JobsPage")) || defaultJobStatusSetting, - [LocalStorageKeys.DB_DETAILS_VIEW_MODE]: - JSON.parse(localStorage.getItem(LocalStorageKeys.DB_DETAILS_VIEW_MODE)) || - defaultDatabaseDetailsViewMode, [LocalStorageKeys.ACTIVE_EXECUTIONS_IS_AUTOREFRESH_ENABLED]: JSON.parse( localStorage.getItem( diff --git a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts index 4c86d6d07653..6967ae1854cf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts @@ -14,20 +14,10 @@ import { ClusterSettingsState, reducer as clusterSettings, } from "./clusterSettings/clusterSettings.reducer"; -import { - KeyedDatabaseDetailsState, - KeyedDatabaseDetailsSpanStatsState, - databaseDetailsReducer, - databaseDetailsSpanStatsReducer, -} from "./databaseDetails"; import { DatabasesListState, reducer as databasesList, } from "./databasesList/databasesList.reducers"; -import { - KeyedTableDetailsState, - reducer as tableDetails, -} from "./databaseTableDetails/tableDetails.reducer"; import { IndexStatsReducerState, reducer as indexStats, @@ -96,9 +86,6 @@ export type AdminUiState = { executionDetailFiles: JobProfilerExecutionDetailFilesState; clusterLocks: ClusterLocksReqState; databasesList: DatabasesListState; - databaseDetails: KeyedDatabaseDetailsState; - databaseDetailsSpanStats: KeyedDatabaseDetailsSpanStatsState; - tableDetails: KeyedTableDetailsState; stmtInsights: StmtInsightsState; txnInsightDetails: TxnInsightDetailsCachedState; txnInsights: TxnInsightsState; @@ -131,9 +118,6 @@ export const reducers = combineReducers({ executionDetailFiles, clusterLocks, databasesList, - databaseDetails: databaseDetailsReducer.reducer, - databaseDetailsSpanStats: databaseDetailsSpanStatsReducer.reducer, - tableDetails, schemaInsights, statementFingerprintInsights, clusterSettings, diff --git a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts index 47a0fe4d88b5..66fc80191554 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts @@ -8,10 +8,7 @@ import { all, fork } from "redux-saga/effects"; import { clusterLocksSaga } from "./clusterLocks/clusterLocks.saga"; import { clusterSettingsSaga } from "./clusterSettings/clusterSettings.saga"; -import { databaseDetailsSaga } from "./databaseDetails"; -import { databaseDetailsSpanStatsSaga } from "./databaseDetails/databaseDetailsSpanStats.saga"; import { databasesListSaga } from "./databasesList"; -import { tableDetailsSaga } from "./databaseTableDetails"; import { indexStatsSaga } from "./indexStats"; import { transactionInsightDetailsSaga } from "./insightDetails/transactionInsightDetails"; import { statementFingerprintInsightsSaga } from "./insights/statementFingerprintInsights"; @@ -44,9 +41,6 @@ export function* sagas(cacheInvalidationPeriod?: number): SagaIterator { fork(jobsSaga), fork(jobSaga), fork(databasesListSaga), - fork(databaseDetailsSaga), - fork(databaseDetailsSpanStatsSaga), - fork(tableDetailsSaga), fork(sessionsSaga), fork(terminateSaga), fork(notifificationsSaga), diff --git a/pkg/ui/workspaces/cluster-ui/src/tableDetailsV2/tableIndexesView.tsx b/pkg/ui/workspaces/cluster-ui/src/tableDetailsV2/tableIndexesView.tsx index c5a2a9dc01e8..9fdec4a9c407 100644 --- a/pkg/ui/workspaces/cluster-ui/src/tableDetailsV2/tableIndexesView.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/tableDetailsV2/tableIndexesView.tsx @@ -90,10 +90,7 @@ const COLUMNS: (TableColumnProps & { hideIfCloud?: boolean })[] = const stat = { indexName: idx.indexName, - indexRecommendations: idx.indexRecs.map(rec => ({ - type: "DROP_UNUSED" as const, - reason: rec.reason, - })), + indexRecommendations: idx.indexRecs, }; // The action button expects an escaped schema qualified table name. return ( diff --git a/pkg/ui/workspaces/cluster-ui/src/util/constants.ts b/pkg/ui/workspaces/cluster-ui/src/util/constants.ts index c1799ad9967e..063baef84c3a 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/constants.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/constants.ts @@ -39,13 +39,6 @@ export const indexUnusedDuration = "168h"; export const REMOTE_DEBUGGING_ERROR_TEXT = "This information is not available due to the current value of the 'server.remote_debugging.mode' setting."; -export const serverToClientErrorMessageMap = new Map([ - [ - "not allowed (due to the 'server.remote_debugging.mode' setting)", - REMOTE_DEBUGGING_ERROR_TEXT, - ], -]); - export const NO_SAMPLES_FOUND = "no samples"; export const INTERNAL_APP_NAME_PREFIX = "$ internal"; diff --git a/pkg/ui/workspaces/db-console/src/app.spec.tsx b/pkg/ui/workspaces/db-console/src/app.spec.tsx index 2b5b22081819..8ce2b9aabf6c 100644 --- a/pkg/ui/workspaces/db-console/src/app.spec.tsx +++ b/pkg/ui/workspaces/db-console/src/app.spec.tsx @@ -16,18 +16,6 @@ stubComponentInModule( ); stubComponentInModule("src/views/cluster/containers/nodeGraphs", "default"); stubComponentInModule("src/views/cluster/containers/events", "EventPage"); -stubComponentInModule( - "src/views/databases/databasesPage", - "DatabasesPageLegacy", -); -stubComponentInModule( - "src/views/databases/databaseDetailsPage", - "DatabaseDetailsPageLegacy", -); -stubComponentInModule( - "src/views/databases/databaseTablePage", - "DatabaseTablePage", -); stubComponentInModule( "src/views/cluster/containers/dataDistribution", "default", @@ -297,15 +285,6 @@ describe("Routing to", () => { }); }); - describe("legacy '/databases/database/:${databaseNameAttr}/table/:${tableNameAttr}' path", () => { - test("redirected to '/database/:${databaseNameAttr}/table/:${tableNameAttr}'", () => { - navigateToPath("/databases/database/some-db-name/table/some-table-name"); - expect(history.location.pathname).toBe( - "/database/some-db-name/table/some-table-name", - ); - }); - }); - describe("'/database' path", () => { test("redirected to '/databases'", () => { navigateToPath("/database"); @@ -313,34 +292,6 @@ describe("Routing to", () => { }); }); - describe("legacy '/legacy/databases' path", () => { - test("routes to component", () => { - navigateToPath("/legacy/databases"); - screen.getByTestId("DatabasesPageLegacy"); - }); - }); - - describe("legacy '/database/:${databaseNameAttr}' path", () => { - test("routes to component", () => { - navigateToPath("/database/some-db-name"); - screen.getByTestId("DatabaseDetailsPageLegacy"); - }); - }); - - describe("'/database/:${databaseNameAttr}/table' path", () => { - test("redirected to '/databases/:${databaseNameAttr}'", () => { - navigateToPath("/database/some-db-name/table"); - expect(history.location.pathname).toBe("/database/some-db-name"); - }); - }); - - describe("'/database/:${databaseNameAttr}/table/:${tableNameAttr}' path", () => { - test("routes to component", () => { - navigateToPath("/database/some-db-name/table/some-table-name"); - screen.getByTestId("DatabaseTablePage"); - }); - }); - { /* data distribution */ } diff --git a/pkg/ui/workspaces/db-console/src/app.tsx b/pkg/ui/workspaces/db-console/src/app.tsx index a336afb7fd7a..1f28dfd658a8 100644 --- a/pkg/ui/workspaces/db-console/src/app.tsx +++ b/pkg/ui/workspaces/db-console/src/app.tsx @@ -55,9 +55,6 @@ import { EventPage } from "src/views/cluster/containers/events"; import NodeGraphs from "src/views/cluster/containers/nodeGraphs"; import NodeLogs from "src/views/cluster/containers/nodeLogs"; import NodeOverview from "src/views/cluster/containers/nodeOverview"; -import { DatabaseDetailsPageLegacy } from "src/views/databases/databaseDetailsPage"; -import { DatabasesPageLegacy } from "src/views/databases/databasesPage"; -import { DatabaseTablePage } from "src/views/databases/databaseTablePage"; import { IndexDetailsPage } from "src/views/databases/indexDetailsPage"; import Raft from "src/views/devtools/containers/raft"; import RaftMessages from "src/views/devtools/containers/raftMessages"; @@ -220,32 +217,17 @@ export const App: React.FC = (props: AppProps) => { path={`/table/:${tableIdAttr}`} component={TableDetailsPageV2} /> - - - { const setAt = moment.utc(); diff --git a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts index c5bf05a83a5d..f6ed45091d8a 100644 --- a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts +++ b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts @@ -108,33 +108,6 @@ const databasesReducerObj = new CachedDataReducer( ); export const refreshDatabases = databasesReducerObj.refresh; -export const databaseRequestPayloadToID = ( - params: clusterUiApi.DatabaseDetailsReqParams, -): string => params.database; - -const databaseDetailsReducerObj = new KeyedCachedDataReducer( - clusterUiApi.getDatabaseDetails, - "databaseDetails", - databaseRequestPayloadToID, - null, - moment.duration(10, "m"), -); - -export const spanStatsRequestPayloadToID = ( - params: clusterUiApi.DatabaseDetailsSpanStatsReqParams, -): string => params.database; - -const databaseDetailsSpanStatsReducerObj = new KeyedCachedDataReducer( - clusterUiApi.getDatabaseDetailsSpanStats, - "databaseDetailsSpanStats", - spanStatsRequestPayloadToID, - null, - moment.duration(10, "m"), -); - -export const refreshDatabaseDetailsSpanStats = - databaseDetailsSpanStatsReducerObj.refresh; - const hotRangesRequestToID = (req: api.HotRangesRequestMessage) => req.page_token; @@ -147,25 +120,10 @@ export const hotRangesReducerObj = new PaginatedCachedDataReducer( moment.duration(30, "minutes"), ); -export const refreshDatabaseDetails = databaseDetailsReducerObj.refresh; - export const refreshHotRanges = hotRangesReducerObj.refresh; -export const tableRequestToID = ( - req: - | api.TableStatsRequestMessage - | api.IndexStatsRequestMessage - | clusterUiApi.TableDetailsReqParams, -): string => generateTableID(req.database, req.table); - -const tableDetailsReducerObj = new KeyedCachedDataReducer( - clusterUiApi.getTableDetails, - "tableDetails", - tableRequestToID, - null, - moment.duration(10, "m"), -); -export const refreshTableDetails = tableDetailsReducerObj.refresh; +export const tableRequestToID = (req: api.IndexStatsRequestMessage): string => + generateTableID(req.database, req.table); const indexStatsReducerObj = new KeyedCachedDataReducer( api.getIndexStats, @@ -583,15 +541,6 @@ export interface APIReducersState { version: CachedDataReducerState; locations: CachedDataReducerState; databases: CachedDataReducerState; - databaseDetails: KeyedCachedDataReducerState< - clusterUiApi.SqlApiResponse - >; - databaseDetailsSpanStats: KeyedCachedDataReducerState< - clusterUiApi.SqlApiResponse - >; - tableDetails: KeyedCachedDataReducerState< - clusterUiApi.SqlApiResponse - >; indexStats: KeyedCachedDataReducerState; nonTableStats: CachedDataReducerState; logs: CachedDataReducerState; @@ -652,11 +601,6 @@ export const apiReducersReducer = combineReducers({ [versionReducerObj.actionNamespace]: versionReducerObj.reducer, [locationsReducerObj.actionNamespace]: locationsReducerObj.reducer, [databasesReducerObj.actionNamespace]: databasesReducerObj.reducer, - [databaseDetailsReducerObj.actionNamespace]: - databaseDetailsReducerObj.reducer, - [databaseDetailsSpanStatsReducerObj.actionNamespace]: - databaseDetailsSpanStatsReducerObj.reducer, - [tableDetailsReducerObj.actionNamespace]: tableDetailsReducerObj.reducer, [indexStatsReducerObj.actionNamespace]: indexStatsReducerObj.reducer, [nonTableStatsReducerObj.actionNamespace]: nonTableStatsReducerObj.reducer, [logsReducerObj.actionNamespace]: logsReducerObj.reducer, diff --git a/pkg/ui/workspaces/db-console/src/util/api.spec.ts b/pkg/ui/workspaces/db-console/src/util/api.spec.ts index 00029c6a3409..7eec74447bc4 100644 --- a/pkg/ui/workspaces/db-console/src/util/api.spec.ts +++ b/pkg/ui/workspaces/db-console/src/util/api.spec.ts @@ -10,524 +10,15 @@ import startsWith from "lodash/startsWith"; import Long from "long"; import moment from "moment-timezone"; -import { - REMOTE_DEBUGGING_ERROR_TEXT, - indexUnusedDuration, -} from "src/util/constants"; -import { mockExecSQLErrors, stubSqlApiCall } from "src/util/fakeApi"; +import { REMOTE_DEBUGGING_ERROR_TEXT } from "src/util/constants"; +import { stubSqlApiCall } from "src/util/fakeApi"; import * as api from "./api"; import fetchMock from "./fetch-mock"; import Severity = protos.cockroach.util.log.Severity; -const cockroach = protos.cockroach; -const { ZoneConfig } = cockroach.config.zonepb; -const { ZoneConfigurationLevel } = cockroach.server.serverpb; - describe("rest api", function () { - describe("databases request", function () { - afterEach(fetchMock.restore); - - it("correctly requests info about all databases", function () { - // Mock out the fetch query to /databases - stubSqlApiCall( - clusterUiApi.databasesRequest, - [ - { - rows: [ - { - database_name: "system", - }, - { - database_name: "test", - }, - ], - }, - ], - ); - - return clusterUiApi.getDatabasesList().then(result => { - expect(fetchMock.calls(clusterUiApi.SQL_API_PATH).length).toBe(1); - expect(result.databases.length).toBe(2); - }); - }); - - it("correctly handles an error", function (done) { - // Mock out the fetch query to /databases, but return a promise that's never resolved to test the timeout - fetchMock.mock({ - matcher: clusterUiApi.SQL_API_PATH, - method: "POST", - response: (_url: string, requestObj: RequestInit) => { - expect(JSON.parse(requestObj.body.toString())).toEqual( - clusterUiApi.databasesRequest, - ); - return { throws: new Error() }; - }, - }); - - clusterUiApi - .getDatabasesList() - .then(_result => { - done(new Error("Request unexpectedly succeeded.")); - }) - .catch(function (e) { - expect(isError(e)).toBeTruthy(); - done(); - }); - }); - - it("correctly times out", function (done) { - // Mock out the fetch query to /databases, but return a promise that's never resolved to test the timeout - fetchMock.mock({ - matcher: clusterUiApi.SQL_API_PATH, - method: "POST", - response: (_url: string, requestObj: RequestInit) => { - expect(JSON.parse(requestObj.body.toString())).toEqual( - clusterUiApi.databasesRequest, - ); - return new Promise(() => {}); - }, - }); - - clusterUiApi - .getDatabasesList(moment.duration(0)) - .then(_result => { - done(new Error("Request unexpectedly succeeded.")); - }) - .catch(function (e) { - expect(startsWith(e.message, "Promise timed out")).toBeTruthy(); - done(); - }); - }); - }); - - describe("database span stats request", () => { - const database = "test"; - afterEach(fetchMock.restore); - beforeEach(fetchMock.restore); - it("correctly requests span stats", () => { - // Mock out the fetch query - stubSqlApiCall( - clusterUiApi.createDatabaseDetailsSpanStatsReq({ - database, - }), - [ - { - rows: [ - { - approximate_disk_bytes: 100, - live_bytes: 200, - total_bytes: 300, - }, - ], - }, - ], - ); - - clusterUiApi.getDatabaseDetailsSpanStats({ database }).then(res => { - expect(res.results.spanStats.approximate_disk_bytes).toEqual(100); - expect(res.results.spanStats.live_bytes).toEqual(200); - expect(res.results.spanStats.total_bytes).toEqual(300); - }); - }); - }); - - describe("database details request", function () { - const database = "test"; - const mockOldDate = new Date(2023, 2, 3); - const mockZoneConfig = new ZoneConfig({ - inherited_constraints: true, - inherited_lease_preferences: true, - null_voter_constraints_is_empty: true, - global_reads: true, - gc: { - ttl_seconds: 100, - }, - }); - const mockZoneConfigBytes: Buffer | Uint8Array = - ZoneConfig.encode(mockZoneConfig).finish(); - const mockZoneConfigHexString = Array.from(mockZoneConfigBytes) - .map(x => x.toString(16).padStart(2, "0")) - .join(""); - - afterEach(fetchMock.restore); - - it("correctly requests details for a specific database", function () { - // Mock out the fetch query - stubSqlApiCall( - clusterUiApi.createDatabaseDetailsReq({ - database, - csIndexUnusedDuration: indexUnusedDuration, - }), - [ - // Database ID query - { rows: [{ database_id: "1" }] }, - // Database grants query - { - rows: [ - { - user: "admin", - privileges: ["ALL"], - }, - { - user: "public", - privileges: ["CONNECT"], - }, - ], - }, - // Database tables query - { - rows: [{ table_schema: "public", table_name: "table1" }], - }, - // Database replicas and regions query - { - rows: [ - { - store_ids: [1, 2, 3], - }, - ], - }, - // Database index usage statistics query - { - rows: [ - { - last_read: mockOldDate.toISOString(), - created_at: mockOldDate.toISOString(), - unused_threshold: "1m", - }, - ], - }, - // Database zone config query - { - rows: [ - { - zone_config_hex_string: mockZoneConfigHexString, - }, - ], - }, - ], - ); - - return clusterUiApi - .getDatabaseDetails({ - database, - csIndexUnusedDuration: indexUnusedDuration, - }) - .then(result => { - expect(fetchMock.calls(clusterUiApi.SQL_API_PATH).length).toBe(1); - expect(result.results.idResp.database_id).toEqual("1"); - expect(result.results.tablesResp.tables.length).toBe(1); - expect(result.results.grantsResp.grants.length).toBe(2); - expect( - result.results.stats.indexStats.num_index_recommendations, - ).toBe(1); - expect(result.results.zoneConfigResp.zone_config).toEqual( - mockZoneConfig, - ); - expect(result.results.zoneConfigResp.zone_config_level).toBe( - ZoneConfigurationLevel.DATABASE, - ); - }); - }); - - it("correctly handles an error", function (done) { - // Mock out the fetch query, but return a 500 status code - const req = clusterUiApi.createDatabaseDetailsReq({ - database, - csIndexUnusedDuration: indexUnusedDuration, - }); - fetchMock.mock({ - matcher: clusterUiApi.SQL_API_PATH, - method: "POST", - response: (_url: string, requestObj: RequestInit) => { - expect(JSON.parse(requestObj.body.toString())).toEqual({ - ...req, - application_name: clusterUiApi.INTERNAL_SQL_API_APP, - database: req.database || clusterUiApi.FALLBACK_DB, - }); - return { throws: new Error() }; - }, - }); - - clusterUiApi - .getDatabaseDetails({ - database, - csIndexUnusedDuration: indexUnusedDuration, - }) - .then(_result => { - done(new Error("Request unexpectedly succeeded.")); - }) - .catch(function (e) { - expect(isError(e)).toBeTruthy(); - done(); - }); - }); - - it("correctly times out", function (done) { - // Mock out the fetch query, but return a promise that's never resolved to test the timeout - const req = clusterUiApi.createDatabaseDetailsReq({ - database, - csIndexUnusedDuration: indexUnusedDuration, - }); - fetchMock.reset(); - fetchMock.mock({ - matcher: clusterUiApi.SQL_API_PATH, - method: "POST", - response: (_url: string, requestObj: RequestInit) => { - expect(JSON.parse(requestObj.body.toString())).toEqual({ - ...req, - application_name: clusterUiApi.INTERNAL_SQL_API_APP, - database: req.database || clusterUiApi.FALLBACK_DB, - }); - return new Promise(() => {}); - }, - }); - - clusterUiApi - .getDatabaseDetails( - { database, csIndexUnusedDuration: indexUnusedDuration }, - moment.duration(0), - ) - .then(_result => { - done(new Error("Request unexpectedly succeeded.")); - }) - .catch(function (e) { - expect(startsWith(e.message, "Promise timed out")).toBeTruthy(); - done(); - }); - }); - - it("should not error when any query fails", async () => { - const req = { database, csIndexUnusedDuration: indexUnusedDuration }; - const mockResults = mockExecSQLErrors(6); - stubSqlApiCall( - clusterUiApi.createDatabaseDetailsReq(req), - mockResults, - ); - - // This call should not throw. - const result = await clusterUiApi.getDatabaseDetails(req); - expect(result.results).toBeDefined(); - expect(result.results.error).toBeDefined(); - }); - }); - - describe("table details request", function () { - const dbName = "testDB"; - const tableName = "testTable"; - const mockOldDate = new Date(2023, 2, 3); - const mockZoneConfig = new ZoneConfig({ - inherited_constraints: true, - inherited_lease_preferences: true, - null_voter_constraints_is_empty: true, - global_reads: true, - gc: { - ttl_seconds: 100, - }, - }); - const mockZoneConfigBytes: Buffer | Uint8Array = - ZoneConfig.encode(mockZoneConfig).finish(); - const mockZoneConfigHexString = Array.from(mockZoneConfigBytes) - .map(x => x.toString(16).padStart(2, "0")) - .join(""); - const mockStatsLastCreatedTimestamp = moment(); - - afterEach(fetchMock.restore); - - it("correctly requests info about a specific table", function () { - // Mock out the fetch query - stubSqlApiCall( - clusterUiApi.createTableDetailsReq( - dbName, - tableName, - indexUnusedDuration, - ), - [ - // Table ID query - { rows: [{ table_id: "1" }] }, - // Table grants query - { - rows: [{ user: "user", privileges: ["ALL", "NONE", "PRIVILEGE"] }], - }, - // Table schema details query - { rows: [{ columns: ["a", "b", "c"], indexes: ["d", "e"] }] }, - // Table create statement query - { rows: [{ create_statement: "mock create stmt" }] }, - // Table zone config statement query - { rows: [{ raw_config_sql: "mock zone config stmt" }] }, - // Table heuristics query - { rows: [{ stats_last_created_at: mockStatsLastCreatedTimestamp }] }, - // Table span stats query - { - rows: [ - { - approximate_disk_bytes: 100, - live_bytes: 200, - total_bytes: 400, - range_count: 400, - live_percentage: 0.5, - }, - ], - }, - // Table index usage statistics query - { - rows: [ - { - last_read: mockOldDate.toISOString(), - created_at: mockOldDate.toISOString(), - unused_threshold: "1m", - }, - ], - }, - // Table zone config query - { - rows: [ - { - database_zone_config_hex_string: mockZoneConfigHexString, - table_zone_config_hex_string: null, - }, - ], - }, - // Table replicas query - { - rows: [{ store_ids: [1, 2, 3], replica_count: 400 }], - }, - ], - ); - - return clusterUiApi - .getTableDetails({ - database: dbName, - table: tableName, - csIndexUnusedDuration: indexUnusedDuration, - }) - .then(resp => { - expect(fetchMock.calls(clusterUiApi.SQL_API_PATH).length).toBe(1); - expect(resp.results.idResp.table_id).toBe("1"); - expect(resp.results.grantsResp.grants.length).toBe(1); - expect(resp.results.schemaDetails.columns.length).toBe(3); - expect(resp.results.schemaDetails.indexes.length).toBe(2); - expect(resp.results.createStmtResp.create_statement).toBe( - "mock create stmt", - ); - expect(resp.results.zoneConfigResp.configure_zone_statement).toBe( - "mock zone config stmt", - ); - expect( - moment(resp.results.heuristicsDetails.stats_last_created_at).isSame( - mockStatsLastCreatedTimestamp, - ), - ).toBe(true); - expect(resp.results.stats.spanStats.approximate_disk_bytes).toBe(100); - expect(resp.results.stats.spanStats.live_bytes).toBe(200); - expect(resp.results.stats.spanStats.total_bytes).toBe(400); - expect(resp.results.stats.spanStats.range_count).toBe(400); - expect(resp.results.stats.spanStats.live_percentage).toBe(0.5); - expect(resp.results.stats.indexStats.has_index_recommendations).toBe( - true, - ); - expect(resp.results.zoneConfigResp.zone_config).toEqual( - mockZoneConfig, - ); - expect(resp.results.zoneConfigResp.zone_config_level).toBe( - ZoneConfigurationLevel.DATABASE, - ); - expect(resp.results.stats.replicaData.storeIDs).toEqual([1, 2, 3]); - }); - }); - - it("correctly handles an error", function (done) { - // Mock out the fetch query, but return a 500 status code - fetchMock.mock({ - matcher: clusterUiApi.SQL_API_PATH, - method: "POST", - response: (_url: string, requestObj: RequestInit) => { - expect(JSON.parse(requestObj.body.toString())).toEqual({ - ...clusterUiApi.createTableDetailsReq( - dbName, - tableName, - indexUnusedDuration, - ), - application_name: clusterUiApi.INTERNAL_SQL_API_APP, - }); - return { throws: new Error() }; - }, - }); - - clusterUiApi - .getTableDetails({ - database: dbName, - table: tableName, - csIndexUnusedDuration: indexUnusedDuration, - }) - .then(_result => { - done(new Error("Request unexpectedly succeeded.")); - }) - .catch(function (e) { - expect(isError(e)).toBeTruthy(); - done(); - }); - }); - - it("correctly times out", function (done) { - // Mock out the fetch query, but return a promise that's never resolved to test the timeout - fetchMock.mock({ - matcher: clusterUiApi.SQL_API_PATH, - method: "POST", - response: (_url: string, requestObj: RequestInit) => { - expect(JSON.parse(requestObj.body.toString())).toEqual({ - ...clusterUiApi.createTableDetailsReq( - dbName, - tableName, - indexUnusedDuration, - ), - application_name: clusterUiApi.INTERNAL_SQL_API_APP, - }); - return new Promise(() => {}); - }, - }); - - clusterUiApi - .getTableDetails( - { - database: dbName, - table: tableName, - csIndexUnusedDuration: indexUnusedDuration, - }, - moment.duration(0), - ) - .then(_result => { - done(new Error("Request unexpectedly succeeded.")); - }) - .catch(function (e) { - expect(startsWith(e.message, "Promise timed out")).toBeTruthy(); - done(); - }); - }); - - it("should not error when any query fails", async () => { - const mockResults = mockExecSQLErrors(10); - stubSqlApiCall( - clusterUiApi.createTableDetailsReq( - dbName, - tableName, - indexUnusedDuration, - ), - mockResults, - ); - - // This call should not throw. - const result = await clusterUiApi.getTableDetails({ - database: dbName, - table: tableName, - csIndexUnusedDuration: indexUnusedDuration, - }); - expect(result.results).toBeDefined(); - expect(result.results.error).toBeDefined(); - }); - }); - describe("events request", function () { afterEach(fetchMock.restore); diff --git a/pkg/ui/workspaces/db-console/src/util/fakeApi.ts b/pkg/ui/workspaces/db-console/src/util/fakeApi.ts index f14ee9434e3b..45ec665ed273 100644 --- a/pkg/ui/workspaces/db-console/src/util/fakeApi.ts +++ b/pkg/ui/workspaces/db-console/src/util/fakeApi.ts @@ -12,12 +12,8 @@ import { cockroach } from "src/js/protos"; import { API_PREFIX, STATUS_PREFIX } from "src/util/api"; import fetchMock from "src/util/fetch-mock"; -const { - SettingsResponse, - TableStatsResponse, - TableIndexStatsResponse, - NodesResponse, -} = cockroach.server.serverpb; +const { SettingsResponse, TableIndexStatsResponse, NodesResponse } = + cockroach.server.serverpb; // These test-time functions provide typesafe wrappers around fetchMock, // stubbing HTTP responses from the admin API. @@ -30,7 +26,7 @@ const { // describe("The thing I'm testing", function() { // it("works like this", function() { // // 1. Set up a fake response from the /databases endpoint. -// fakeApi.stubDatabases({ +// fakeApi.stuatabases({ // databases: ["one", "two", "three"], // }); // @@ -67,20 +63,6 @@ export function stubNodesUI( stubGet(`/nodes_ui`, NodesResponse.encode(response), STATUS_PREFIX); } -export function stubTableStats( - database: string, - table: string, - response: cockroach.server.serverpb.ITableStatsResponse, -) { - stubGet( - `/databases/${encodeURIComponent(database)}/tables/${encodeURIComponent( - table, - )}/stats`, - TableStatsResponse.encode(response), - API_PREFIX, - ); -} - export function stubIndexStats( database: string, table: string, diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/index.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/index.ts deleted file mode 100644 index f21e62c70726..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/index.ts +++ /dev/null @@ -1,18 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { DatabaseDetailsPage } from "@cockroachlabs/cluster-ui"; -import { connect, ReactReduxContext } from "react-redux"; -import { withRouter } from "react-router-dom"; - -import { mapStateToProps, mapDispatchToProps } from "./redux"; - -const connected = withRouter( - connect(mapStateToProps, mapDispatchToProps, null, { - context: ReactReduxContext, - })(DatabaseDetailsPage), -); - -export { connected as DatabaseDetailsPageLegacy }; diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.spec.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.spec.ts deleted file mode 100644 index 91dbdd17b680..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.spec.ts +++ /dev/null @@ -1,600 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { - DatabaseDetailsPageActions, - DatabaseDetailsPageData, - DatabaseDetailsPageDataTable, - defaultFilters, - ViewMode, - api as clusterUiApi, -} from "@cockroachlabs/cluster-ui"; -import { createMemoryHistory } from "history"; -import moment from "moment-timezone"; -import { RouteComponentProps } from "react-router-dom"; -import { bindActionCreators, Store } from "redux"; - -import { AdminUIState, createAdminUIStore } from "src/redux/state"; -import { databaseNameAttr, indexUnusedDuration } from "src/util/constants"; -import * as fakeApi from "src/util/fakeApi"; - -import { mapStateToProps, mapDispatchToProps } from "./redux"; - -function fakeRouteComponentProps( - key: string, - value: string, -): RouteComponentProps { - return { - history: createMemoryHistory(), - location: { - pathname: "", - search: "", - state: {}, - hash: "", - }, - match: { - params: { - [key]: value, - }, - isExact: true, - path: "", - url: "", - }, - }; -} - -class TestDriver { - private readonly actions: DatabaseDetailsPageActions; - private readonly properties: () => DatabaseDetailsPageData; - - constructor( - store: Store, - private readonly database: string, - ) { - this.actions = bindActionCreators( - mapDispatchToProps, - store.dispatch.bind(store), - ); - this.properties = () => - mapStateToProps( - store.getState(), - fakeRouteComponentProps(databaseNameAttr, database), - ); - } - - assertProperties(expected: DatabaseDetailsPageData) { - expect(this.properties()).toEqual(expected); - } - - assertTableDetails(name: string, expected: DatabaseDetailsPageDataTable) { - // We destructure the expected and actual payloads to extract the field - // with Moment type. Moment types cannot be compared using toEqual or toBe, - // we need to use moment's isSame function. - const { - details: { statsLastUpdated, ...restDetails }, - ...table - } = this.findTable(name); - const { - details: { - statsLastUpdated: expectedStatsLastUpdated, - ...expectedRestDetails - }, - ...expectedTable - } = expected; - // Expect table data to be equal (name/loading/loaded/lastError). - expect(table).toEqual(expectedTable); - // Expect remaining details fields to be equal. - expect(restDetails).toEqual(expectedRestDetails); - // Expect Moment type field to be equal. - expect( - // Moments are the same - moment(statsLastUpdated.stats_last_created_at).isSame( - expectedStatsLastUpdated.stats_last_created_at, - ) || - // Moments are null. - (statsLastUpdated.stats_last_created_at === - expectedStatsLastUpdated.stats_last_created_at && - statsLastUpdated.stats_last_created_at === null), - ).toBe(true); - } - - assertTableRoles(name: string, expected: string[]) { - expect(this.findTable(name).details.grants.roles).toEqual(expected); - } - - assertTableGrants(name: string, expected: string[]) { - expect(this.findTable(name).details.grants.privileges).toEqual(expected); - } - - async refreshDatabaseDetails() { - return this.actions.refreshDatabaseDetails( - this.database, - indexUnusedDuration, - ); - } - - async refreshTableDetails(table: string) { - return this.actions.refreshTableDetails( - this.database, - table, - indexUnusedDuration, - ); - } - - async refreshNodes() { - return this.actions.refreshNodes(); - } - - private findTable(name: string) { - return this.properties().tables.find( - t => t.name.qualifiedNameWithSchemaAndTable === name, - ); - } -} - -describe("Database Details Page", function () { - let driver: TestDriver; - - beforeEach(function () { - driver = new TestDriver( - createAdminUIStore(createMemoryHistory()), - "things", - ); - }); - - afterEach(function () { - fakeApi.restore(); - }); - - it("starts in a pre-loading state", function () { - driver.assertProperties({ - loading: false, - loaded: false, - requestError: undefined, - queryError: undefined, - name: "things", - search: null, - filters: defaultFilters, - nodeRegions: {}, - isTenant: false, - showNodeRegionsColumn: false, - viewMode: ViewMode.Tables, - sortSettingTables: { ascending: true, columnTitle: "name" }, - sortSettingGrants: { ascending: true, columnTitle: "name" }, - tables: [], - showIndexRecommendations: false, - csIndexUnusedDuration: indexUnusedDuration, - }); - }); - - it("makes a row for each table", async function () { - fakeApi.stubSqlApiCall( - clusterUiApi.createDatabaseDetailsReq({ - database: "things", - csIndexUnusedDuration: indexUnusedDuration, - }), - [ - // Id - { rows: [] }, - // Grants - { rows: [] }, - // Tables - { - rows: [ - { table_schema: "public", table_name: "foo" }, - { table_schema: "public", table_name: "bar" }, - ], - }, - ], - ); - - await driver.refreshDatabaseDetails(); - driver.assertProperties({ - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - name: "things", - search: null, - filters: defaultFilters, - nodeRegions: {}, - isTenant: false, - showNodeRegionsColumn: false, - showIndexRecommendations: false, - csIndexUnusedDuration: indexUnusedDuration, - viewMode: ViewMode.Tables, - sortSettingTables: { ascending: true, columnTitle: "name" }, - sortSettingGrants: { ascending: true, columnTitle: "name" }, - tables: [ - { - name: { - schema: "public", - table: "foo", - qualifiedNameWithSchemaAndTable: `"public"."foo"`, - }, - qualifiedDisplayName: `public.foo`, - loading: false, - loaded: false, - requestError: undefined, - queryError: undefined, - details: { - schemaDetails: undefined, - grants: { - error: undefined, - roles: [], - privileges: [], - }, - statsLastUpdated: undefined, - indexStatRecs: undefined, - spanStats: undefined, - nodes: [], - nodesByRegionString: "", - }, - }, - { - name: { - schema: "public", - table: "bar", - qualifiedNameWithSchemaAndTable: `"public"."bar"`, - }, - qualifiedDisplayName: `public.bar`, - loading: false, - loaded: false, - requestError: undefined, - queryError: undefined, - details: { - schemaDetails: undefined, - grants: { - error: undefined, - roles: [], - privileges: [], - }, - statsLastUpdated: undefined, - indexStatRecs: undefined, - spanStats: undefined, - nodes: [], - nodesByRegionString: "", - }, - }, - ], - }); - }); - - it("loads table details", async function () { - fakeApi.stubSqlApiCall( - clusterUiApi.createDatabaseDetailsReq({ - database: "things", - csIndexUnusedDuration: indexUnusedDuration, - }), - [ - // Id - { rows: [] }, - // Grants - { rows: [] }, - // Tables - { - rows: [ - { table_schema: "public", table_name: "foo" }, - { table_schema: "public", table_name: "bar" }, - ], - }, - ], - ); - const mockStatsLastCreatedTimestamp = moment(); - - fakeApi.stubSqlApiCall( - clusterUiApi.createTableDetailsReq( - "things", - `"public"."foo"`, - indexUnusedDuration, - ), - [ - // Table ID query - { rows: [{ table_id: "1" }] }, - // Table grants query - { - rows: [ - { user: "admin", privileges: ["CREATE"] }, - { user: "public", privileges: ["SELECT"] }, - ], - }, - // Table schema details query - { rows: [{ columns: ["a", "b", "c"], indexes: ["d", "e"] }] }, - // Table create statement query - {}, - // Table zone config statement query - {}, - // Table heuristics query - { rows: [{ stats_last_created_at: mockStatsLastCreatedTimestamp }] }, - // Table span stats query - { - rows: [ - { - approximate_disk_bytes: 100, - live_bytes: 200, - total_bytes: 400, - range_count: 400, - live_percentage: 0.5, - }, - ], - }, - // Table index usage statistics query - { - rows: [ - { - last_read: new Date().toISOString(), - created_at: new Date().toISOString(), - unused_threshold: "1m", - }, - ], - }, - // Table zone config query - {}, - // Table replicas query - { - rows: [{ store_ids: [1, 2, 3], replica_count: 5 }], - }, - ], - ); - - fakeApi.stubSqlApiCall( - clusterUiApi.createTableDetailsReq( - "things", - `"public"."bar"`, - indexUnusedDuration, - ), - [ - // Table ID query - { rows: [{ table_id: "2" }] }, - // Table grants query - { - rows: [ - { user: "root", privileges: ["ALL"] }, - { user: "app", privileges: ["INSERT"] }, - { user: "data", privileges: ["SELECT"] }, - ], - }, - // Table schema details query - { rows: [{ columns: ["a", "b"], indexes: ["c", "d", "e", "f"] }] }, - // Table create statement query - {}, - // Table zone config statement query - {}, - // Table heuristics query - { rows: [{ stats_last_created_at: null }] }, - // Table span stats query - { - rows: [ - { - approximate_disk_bytes: 10, - live_bytes: 100, - total_bytes: 100, - range_count: 50, - live_percentage: 1, - }, - ], - }, - // Table index usage statistics query - { - rows: [], - }, - // Table zone config query - {}, - // Table replicas query - { - rows: [{ store_ids: [1, 2, 3, 4, 5], replica_count: 5 }], - }, - ], - ); - - fakeApi.stubNodesUI({ - nodes: [...Array(5).keys()].map(node_id => { - return { - desc: { - node_id: node_id + 1, // 1-index offset. - locality: { - tiers: [ - { - key: "region", - value: "gcp-us-east1", - }, - ], - }, - }, - store_statuses: [{ desc: { store_id: node_id + 1 } }], - }; - }), - }); - - await driver.refreshDatabaseDetails(); - await driver.refreshTableDetails(`"public"."foo"`); - await driver.refreshTableDetails(`"public"."bar"`); - await driver.refreshNodes(); - - driver.assertTableDetails(`"public"."foo"`, { - name: { - schema: "public", - table: "foo", - qualifiedNameWithSchemaAndTable: `"public"."foo"`, - }, - qualifiedDisplayName: `public.foo`, - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - details: { - schemaDetails: { - columns: ["a", "b", "c"], - indexes: ["d", "e"], - }, - grants: { - error: undefined, - roles: ["admin", "public"], - privileges: ["CREATE", "SELECT"], - }, - statsLastUpdated: { - stats_last_created_at: mockStatsLastCreatedTimestamp, - }, - indexStatRecs: { has_index_recommendations: true }, - spanStats: { - approximate_disk_bytes: 100, - live_bytes: 200, - total_bytes: 400, - range_count: 400, - live_percentage: 0.5, - }, - nodes: [1, 2, 3], - nodesByRegionString: "gcp-us-east1(n1,n2,n3)", - }, - }); - - driver.assertTableDetails(`"public"."bar"`, { - name: { - schema: "public", - table: "bar", - qualifiedNameWithSchemaAndTable: `"public"."bar"`, - }, - qualifiedDisplayName: `public.bar`, - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - details: { - schemaDetails: { - columns: ["a", "b"], - indexes: ["c", "d", "e", "f"], - }, - grants: { - error: undefined, - roles: ["root", "app", "data"], - privileges: ["ALL", "SELECT", "INSERT"], - }, - statsLastUpdated: { stats_last_created_at: null }, - indexStatRecs: { has_index_recommendations: false }, - spanStats: { - live_percentage: 1, - live_bytes: 100, - total_bytes: 100, - range_count: 50, - approximate_disk_bytes: 10, - }, - nodes: [1, 2, 3, 4, 5], - nodesByRegionString: "gcp-us-east1(n1,n2,n3,n4,n5)", - }, - }); - }); - - it("sorts roles meaningfully", async function () { - fakeApi.stubSqlApiCall( - clusterUiApi.createDatabaseDetailsReq({ - database: "things", - csIndexUnusedDuration: indexUnusedDuration, - }), - [ - // Id - { rows: [] }, - // Grants - { rows: [] }, - // Tables - { - rows: [{ table_schema: "public", table_name: "foo" }], - }, - ], - ); - - fakeApi.stubSqlApiCall( - clusterUiApi.createTableDetailsReq( - "things", - `"public"."foo"`, - indexUnusedDuration, - ), - [ - // Table ID query - {}, - // Table grants query - { - rows: [ - { user: "bzuckercorn", privileges: ["ALL"] }, - { user: "bloblaw", privileges: ["ALL"] }, - { user: "jwweatherman", privileges: ["ALL"] }, - { user: "admin", privileges: ["ALL"] }, - { user: "public", privileges: ["ALL"] }, - { user: "root", privileges: ["ALL"] }, - ], - }, - ], - ); - - await driver.refreshDatabaseDetails(); - await driver.refreshTableDetails(`"public"."foo"`); - - driver.assertTableRoles(`"public"."foo"`, [ - "root", - "admin", - "public", - "bloblaw", - "bzuckercorn", - "jwweatherman", - ]); - }); - - it("sorts grants meaningfully", async function () { - fakeApi.stubSqlApiCall( - clusterUiApi.createDatabaseDetailsReq({ - database: "things", - csIndexUnusedDuration: indexUnusedDuration, - }), - [ - // Id - { rows: [] }, - // Grants - { rows: [] }, - // Tables - { - rows: [{ table_schema: "public", table_name: "foo" }], - }, - ], - ); - - fakeApi.stubSqlApiCall( - clusterUiApi.createTableDetailsReq( - "things", - `"public"."foo"`, - indexUnusedDuration, - ), - [ - // Table ID query - {}, - // Table grants query - { - rows: [ - { - user: "admin", - privileges: ["ALL", "CREATE", "DELETE", "DROP", "GRANT"], - }, - { - user: "public", - privileges: ["DROP", "GRANT", "INSERT", "SELECT", "UPDATE"], - }, - ], - }, - ], - ); - - await driver.refreshDatabaseDetails(); - await driver.refreshTableDetails(`"public"."foo"`); - - driver.assertTableGrants(`"public"."foo"`, [ - "ALL", - "CREATE", - "DROP", - "GRANT", - "SELECT", - "INSERT", - "UPDATE", - "DELETE", - ]); - }); -}); diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.ts deleted file mode 100644 index f5999513526e..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.ts +++ /dev/null @@ -1,137 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { - DatabaseDetailsPageData, - defaultFilters, - Filters, - ViewMode, - deriveTableDetailsMemoized, -} from "@cockroachlabs/cluster-ui"; -import { RouteComponentProps } from "react-router"; - -import { - refreshDatabaseDetails, - refreshNodes, - refreshTableDetails, -} from "src/redux/apiReducers"; -import { - selectDropUnusedIndexDuration, - selectIndexRecommendationsEnabled, -} from "src/redux/clusterSettings"; -import { LocalSetting } from "src/redux/localsettings"; -import { - nodeRegionsByIDSelector, - selectIsMoreThanOneNode, -} from "src/redux/nodes"; -import { AdminUIState } from "src/redux/state"; -import { databaseNameAttr } from "src/util/constants"; -import { getMatchParamByName } from "src/util/query"; - -const sortSettingTablesLocalSetting = new LocalSetting( - "sortSetting/DatabasesDetailsTablesPage", - (state: AdminUIState) => state.localSettings, - { ascending: true, columnTitle: "name" }, -); - -const sortSettingGrantsLocalSetting = new LocalSetting( - "sortSetting/DatabasesDetailsGrantsPage", - (state: AdminUIState) => state.localSettings, - { ascending: true, columnTitle: "name" }, -); - -// Hardcoded isTenant value for db-console. -const isTenant = false; - -const viewModeLocalSetting = new LocalSetting( - "viewMode/DatabasesDetailsPage", - (state: AdminUIState) => state.localSettings, - ViewMode.Tables, -); - -const filtersLocalTablesSetting = new LocalSetting( - "filters/DatabasesDetailsTablesPage", - (state: AdminUIState) => state.localSettings, - defaultFilters, -); - -const searchLocalTablesSetting = new LocalSetting( - "search/DatabasesDetailsTablesPage", - (state: AdminUIState) => state.localSettings, - null, -); - -export const mapStateToProps = ( - state: AdminUIState, - props: RouteComponentProps, -): DatabaseDetailsPageData => { - const database = getMatchParamByName(props.match, databaseNameAttr); - const databaseDetails = state?.cachedData.databaseDetails; - const tableDetails = state?.cachedData.tableDetails; - const dbTables = - databaseDetails[database]?.data?.results.tablesResp.tables || []; - const nodeRegions = nodeRegionsByIDSelector(state); - const nodeStatuses = state?.cachedData.nodes.data; - - return { - loading: !!databaseDetails[database]?.inFlight, - loaded: !!databaseDetails[database]?.valid, - requestError: databaseDetails[database]?.lastError, - queryError: databaseDetails[database]?.data?.results?.error, - name: database, - showNodeRegionsColumn: selectIsMoreThanOneNode(state), - viewMode: viewModeLocalSetting.selector(state), - sortSettingTables: sortSettingTablesLocalSetting.selector(state), - sortSettingGrants: sortSettingGrantsLocalSetting.selector(state), - filters: filtersLocalTablesSetting.selector(state), - search: searchLocalTablesSetting.selector(state), - nodeRegions, - isTenant, - tables: deriveTableDetailsMemoized({ - dbName: database, - tables: dbTables, - tableDetails, - nodeRegions, - isTenant, - nodeStatuses, - }), - showIndexRecommendations: selectIndexRecommendationsEnabled(state), - csIndexUnusedDuration: selectDropUnusedIndexDuration(state), - }; -}; - -export const mapDispatchToProps = { - refreshDatabaseDetails: (database: string, csIndexUnusedDuration: string) => { - return refreshDatabaseDetails({ - database, - csIndexUnusedDuration, - }); - }, - refreshTableDetails: ( - database: string, - table: string, - csIndexUnusedDuration: string, - ) => { - return refreshTableDetails({ - database, - table, - csIndexUnusedDuration, - }); - }, - onViewModeChange: (viewMode: ViewMode) => viewModeLocalSetting.set(viewMode), - onSortingTablesChange: (columnName: string, ascending: boolean) => - sortSettingTablesLocalSetting.set({ - ascending: ascending, - columnTitle: columnName, - }), - onSortingGrantsChange: (columnName: string, ascending: boolean) => - sortSettingGrantsLocalSetting.set({ - ascending: ascending, - columnTitle: columnName, - }), - onSearchComplete: (query: string) => searchLocalTablesSetting.set(query), - onFilterChange: (filters: Filters) => filtersLocalTablesSetting.set(filters), - refreshNodes, -}; diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/index.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/index.ts deleted file mode 100644 index 94df2c321405..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/index.ts +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { DatabaseTablePage } from "@cockroachlabs/cluster-ui"; -import { connect } from "react-redux"; -import { withRouter } from "react-router-dom"; - -import { mapStateToProps, mapDispatchToProps } from "./redux"; - -const connected = withRouter( - connect(mapStateToProps, mapDispatchToProps)(DatabaseTablePage), -); - -export { connected as DatabaseTablePage }; diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/redux.spec.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/redux.spec.ts deleted file mode 100644 index cfbd5a31c174..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/redux.spec.ts +++ /dev/null @@ -1,386 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { - DatabaseTablePageActions, - DatabaseTablePageData, - DatabaseTablePageDataDetails, - DatabaseTablePageIndexStats, - util, - api as clusterUiApi, -} from "@cockroachlabs/cluster-ui"; -import { createMemoryHistory } from "history"; -import Long from "long"; -import moment from "moment-timezone"; -import { RouteComponentProps } from "react-router-dom"; -import { bindActionCreators, Store } from "redux"; - -import { AdminUIState, createAdminUIStore } from "src/redux/state"; -import { - databaseNameAttr, - indexUnusedDuration, - tableNameAttr, -} from "src/util/constants"; -import * as fakeApi from "src/util/fakeApi"; - -import { mapStateToProps, mapDispatchToProps } from "./redux"; - -function fakeRouteComponentProps( - k1: string, - v1: string, - k2: string, - v2: string, -): RouteComponentProps { - return { - history: createMemoryHistory(), - location: { - pathname: "", - search: "", - state: {}, - hash: "", - }, - match: { - params: { - [k1]: v1, - [k2]: v2, - }, - isExact: true, - path: "", - url: "", - }, - }; -} - -class TestDriver { - private readonly actions: DatabaseTablePageActions; - private readonly properties: () => DatabaseTablePageData; - - constructor( - store: Store, - private readonly database: string, - private readonly table: string, - ) { - this.actions = bindActionCreators( - mapDispatchToProps, - store.dispatch.bind(store), - ); - this.properties = () => - mapStateToProps( - store.getState(), - fakeRouteComponentProps( - databaseNameAttr, - database, - tableNameAttr, - table, - ), - ); - } - - assertProperties(expected: DatabaseTablePageData, compareTimestamps = true) { - // Assert moments are equal if not in pre-loading state. - if (compareTimestamps) { - expect(this.properties().indexStats.lastReset).toEqual( - expected.indexStats.lastReset, - ); - } - expect(this.properties()).toEqual(expected); - } - - assertTableDetails(expected: DatabaseTablePageDataDetails) { - // We destructure the expected and actual payloads to extract the field - // with Moment type. Moment types cannot be compared using toEqual or toBe, - // we need to use moment's isSame function. - const { statsLastUpdated, ...rest } = this.properties().details; - const { statsLastUpdated: expectedStatsLastUpdated, ...expectedRest } = - expected; - expect(rest).toEqual(expectedRest); - expect( - // Moments are the same - moment(statsLastUpdated.stats_last_created_at).isSame( - expectedStatsLastUpdated.stats_last_created_at, - ) || - // Moments are null. - (statsLastUpdated.stats_last_created_at === - expectedStatsLastUpdated.stats_last_created_at && - statsLastUpdated.stats_last_created_at === null), - ).toBe(true); - } - - assertIndexStats( - expected: DatabaseTablePageIndexStats, - compareTimestamps = true, - ) { - // Assert moments are equal if not in pre-loading state. - if (compareTimestamps) { - expect( - expected.stats[0].lastUsed.isSame( - this.properties().indexStats.stats[0].lastUsed, - ), - ).toEqual(true); - } - const indexStats = this.properties().indexStats; - expect(expected.lastReset.isSame(indexStats.lastReset)).toEqual(true); - // Remove moment objects. - indexStats.stats.map(stat => { - delete stat.lastUsed; - }); - expected.stats.map(stat => { - delete stat.lastUsed; - }); - delete indexStats.lastReset; - delete expected.lastReset; - // Assert objects without moments are equal. - expect(indexStats).toEqual(expected); - } - - async refreshSettings() { - return this.actions.refreshSettings(); - } - async refreshTableDetails() { - return this.actions.refreshTableDetails( - this.database, - this.table, - indexUnusedDuration, - ); - } - - async refreshIndexStats() { - return this.actions.refreshIndexStats(this.database, this.table); - } -} - -describe("Database Table Page", function () { - let driver: TestDriver; - - beforeEach(function () { - driver = new TestDriver( - createAdminUIStore(createMemoryHistory()), - "DATABASE", - "TABLE", - ); - }); - - afterEach(function () { - fakeApi.restore(); - }); - - it("starts in a pre-loading state", async function () { - fakeApi.stubClusterSettings({ - key_values: { - "sql.stats.automatic_collection.enabled": { value: "true" }, - version: { value: "23.1.0" }, - }, - }); - - await driver.refreshSettings(); - - driver.assertProperties({ - databaseName: "DATABASE", - name: "TABLE", - schemaName: "", - showNodeRegionsSection: false, - details: { - loading: false, - loaded: false, - requestError: undefined, - queryError: undefined, - createStatement: undefined, - replicaData: undefined, - spanStats: undefined, - indexData: undefined, - grants: { - all: [], - error: undefined, - }, - statsLastUpdated: undefined, - nodesByRegionString: "", - }, - automaticStatsCollectionEnabled: true, - indexUsageStatsEnabled: true, - showIndexRecommendations: true, - csIndexUnusedDuration: indexUnusedDuration, - hasAdminRole: false, - indexStats: { - loading: false, - loaded: false, - lastError: undefined, - stats: [], - lastReset: util.minDate, - }, - isTenant: false, - }); - }); - - it("loads table details", async function () { - const mockStatsLastCreatedTimestamp = moment(); - - fakeApi.stubSqlApiCall( - clusterUiApi.createTableDetailsReq( - "DATABASE", - "TABLE", - indexUnusedDuration, - ), - [ - // Table ID query - { rows: [{ table_id: "1" }] }, - // Table grants query - { - rows: [ - { user: "admin", privileges: ["CREATE", "DROP"] }, - { user: "public", privileges: ["SELECT"] }, - ], - }, - // Table schema details query - { - rows: [ - { - columns: ["colA", "colB", "c"], - indexes: ["primary", "anotha", "one"], - }, - ], - }, - // Table create statement query - { rows: [{ create_statement: "CREATE TABLE foo" }] }, - // Table zone config statement query - {}, - // Table heuristics query - { rows: [{ stats_last_created_at: mockStatsLastCreatedTimestamp }] }, - // Table span stats query - { - rows: [ - { - approximate_disk_bytes: 23, - live_bytes: 45, - total_bytes: 45, - range_count: 56, - live_percentage: 1, - }, - ], - }, - // Table index usage statistics query - { - rows: [ - { - last_read: new Date().toISOString(), - created_at: new Date().toISOString(), - unused_threshold: "1m", - }, - ], - }, - // Table zone config query - {}, - // Table replicas query - { - rows: [{ store_ids: [1, 2, 3, 4, 5], replica_count: 10 }], - }, - ], - ); - - await driver.refreshTableDetails(); - - driver.assertTableDetails({ - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - createStatement: { create_statement: "CREATE TABLE foo" }, - replicaData: { storeIDs: [1, 2, 3, 4, 5], replicaCount: 10 }, - spanStats: { - approximate_disk_bytes: 23, - live_bytes: 45, - total_bytes: 45, - range_count: 56, - live_percentage: 1, - }, - indexData: { - columns: ["colA", "colB", "c"], - indexes: ["primary", "anotha", "one"], - }, - grants: { - all: [ - { user: "admin", privileges: ["CREATE", "DROP"] }, - { user: "public", privileges: ["SELECT"] }, - ], - error: undefined, - }, - statsLastUpdated: { - stats_last_created_at: mockStatsLastCreatedTimestamp, - }, - nodesByRegionString: "", - }); - }); - - it("loads index stats", async function () { - fakeApi.stubIndexStats("DATABASE", "TABLE", { - statistics: [ - { - statistics: { - key: { - table_id: 15, - index_id: 2, - }, - stats: { - total_read_count: new Long(2), - last_read: util.stringToTimestamp("2021-11-19T23:01:05.167627Z"), - total_rows_read: new Long(0), - total_write_count: new Long(0), - last_write: util.stringToTimestamp("0001-01-01T00:00:00Z"), - total_rows_written: new Long(0), - }, - }, - index_name: "jobs_status_created_idx", - index_type: "secondary", - }, - { - statistics: { - key: { - table_id: 1, - index_id: 2, - }, - stats: { - total_read_count: new Long(0), - last_read: util.stringToTimestamp("0001-01-01T00:00:00Z"), - total_rows_read: new Long(0), - total_write_count: new Long(0), - last_write: util.stringToTimestamp("0001-01-01T00:00:00Z"), - total_rows_written: new Long(0), - }, - }, - index_name: "index_no_reads_no_resets", - index_type: "secondary", - created_at: util.stringToTimestamp("0001-01-01T00:00:00Z"), - }, - ], - last_reset: util.stringToTimestamp("0001-01-01T00:00:00Z"), - }); - - await driver.refreshIndexStats(); - - driver.assertIndexStats({ - loading: false, - loaded: true, - lastError: null, - stats: [ - { - indexName: "jobs_status_created_idx", - totalReads: 2, - lastUsed: util.TimestampToMoment( - util.stringToTimestamp("2021-11-19T23:01:05.167627Z"), - ), - lastUsedType: "read", - indexRecommendations: [], - }, - { - indexName: "index_no_reads_no_resets", - totalReads: 0, - lastUsed: util.minDate, - lastUsedType: "created", - indexRecommendations: [], - }, - ], - lastReset: util.minDate, - }); - }); -}); diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/redux.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/redux.ts deleted file mode 100644 index 9c46cd7c4ffa..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseTablePage/redux.ts +++ /dev/null @@ -1,107 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { - DatabaseTablePageData, - util, - deriveIndexDetailsMemoized, - deriveTablePageDetailsMemoized, -} from "@cockroachlabs/cluster-ui"; -import { RouteComponentProps } from "react-router"; - -import { cockroach } from "src/js/protos"; -import { - refreshTableDetails, - refreshNodes, - refreshIndexStats, - refreshSettings, - refreshUserSQLRoles, -} from "src/redux/apiReducers"; -import { - selectAutomaticStatsCollectionEnabled, - selectDropUnusedIndexDuration, - selectIndexRecommendationsEnabled, - selectIndexUsageStatsEnabled, -} from "src/redux/clusterSettings"; -import { resetIndexUsageStatsAction } from "src/redux/indexUsageStats"; -import { - nodeRegionsByIDSelector, - selectIsMoreThanOneNode, -} from "src/redux/nodes"; -import { AdminUIState } from "src/redux/state"; -import { selectHasAdminRole } from "src/redux/user"; -import { databaseNameAttr, tableNameAttr } from "src/util/constants"; -import { getMatchParamByName } from "src/util/query"; - -const { TableIndexStatsRequest } = cockroach.server.serverpb; - -// Hardcoded isTenant value for db-console. -const isTenant = false; - -export const mapStateToProps = ( - state: AdminUIState, - props: RouteComponentProps, -): DatabaseTablePageData => { - const database = getMatchParamByName(props.match, databaseNameAttr); - const table = getMatchParamByName(props.match, tableNameAttr); - const tableDetails = state?.cachedData.tableDetails; - const details = tableDetails[util.generateTableID(database, table)]; - const indexUsageStats = state?.cachedData.indexStats; - const indexStats = indexUsageStats[util.generateTableID(database, table)]; - const lastReset = util.TimestampToMoment( - indexStats?.data?.last_reset, - util.minDate, - ); - const nodeRegions = nodeRegionsByIDSelector(state); - const nodeStatuses = state?.cachedData.nodes.data; - - return { - databaseName: database, - name: table, - schemaName: "", - details: deriveTablePageDetailsMemoized({ - details, - nodeRegions, - isTenant, - nodeStatuses, - }), - showNodeRegionsSection: selectIsMoreThanOneNode(state) && !isTenant, - automaticStatsCollectionEnabled: - selectAutomaticStatsCollectionEnabled(state) || false, - hasAdminRole: selectHasAdminRole(state) || false, - showIndexRecommendations: selectIndexRecommendationsEnabled(state), - csIndexUnusedDuration: selectDropUnusedIndexDuration(state), - indexUsageStatsEnabled: selectIndexUsageStatsEnabled(state), - indexStats: { - loading: !!indexStats?.inFlight, - loaded: !!indexStats?.valid, - lastError: indexStats?.lastError, - stats: deriveIndexDetailsMemoized({ database, table, indexUsageStats }), - lastReset: lastReset, - }, - isTenant, - }; -}; - -export const mapDispatchToProps = { - refreshTableDetails: ( - database: string, - table: string, - csIndexUnusedDuration: string, - ) => { - return refreshTableDetails({ - database, - table, - csIndexUnusedDuration, - }); - }, - refreshIndexStats: (database: string, table: string) => { - return refreshIndexStats(new TableIndexStatsRequest({ database, table })); - }, - resetIndexUsageStats: resetIndexUsageStatsAction, - refreshNodes, - refreshSettings, - refreshUserSQLRoles, -}; diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/index.ts b/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/index.ts deleted file mode 100644 index 061f4b38c9ee..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/index.ts +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { - DatabasesPage, - DatabasesPageData, - DatabasesPageActions, -} from "@cockroachlabs/cluster-ui"; -import { connect } from "react-redux"; -import { RouteComponentProps, withRouter } from "react-router-dom"; - -import { AdminUIState } from "src/redux/state"; - -import { mapStateToProps, mapDispatchToProps } from "./redux"; - -const connected = withRouter( - connect< - DatabasesPageData, - DatabasesPageActions, - RouteComponentProps, - AdminUIState - >( - mapStateToProps, - mapDispatchToProps, - )(DatabasesPage), -); - -export { connected as DatabasesPageLegacy }; diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/redux.spec.ts b/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/redux.spec.ts deleted file mode 100644 index 6db95a3912d3..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/redux.spec.ts +++ /dev/null @@ -1,319 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { - DatabasesPageActions, - DatabasesPageData, - DatabasesPageDataDatabase, - defaultFilters, - api as clusterUiApi, -} from "@cockroachlabs/cluster-ui"; -import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -import { createMemoryHistory } from "history"; -import find from "lodash/find"; -import { bindActionCreators, Store } from "redux"; - -import { AdminUIState, createAdminUIStore } from "src/redux/state"; -import { indexUnusedDuration } from "src/util/constants"; -import * as fakeApi from "src/util/fakeApi"; - -import { mapDispatchToProps, mapStateToProps } from "./redux"; - -class TestDriver { - private readonly actions: DatabasesPageActions; - private readonly properties: () => DatabasesPageData; - - constructor(store: Store) { - this.actions = bindActionCreators( - mapDispatchToProps, - store.dispatch.bind(store), - ); - this.properties = () => mapStateToProps(store.getState()); - } - - async refreshDatabases() { - return this.actions.refreshDatabases(); - } - - async refreshDatabaseDetails( - database: string, - csIndexUnusedDuration: string, - ) { - return this.actions.refreshDatabaseDetails(database, csIndexUnusedDuration); - } - - async refreshNodes() { - return this.actions.refreshNodes(); - } - - async refreshSettings() { - return this.actions.refreshSettings(); - } - - assertProperties(expected: DatabasesPageData) { - expect(this.properties()).toEqual(expected); - } - - assertDatabaseProperties( - database: string, - expected: DatabasesPageDataDatabase, - ) { - expect(this.findDatabase(database)).toEqual(expected); - } - - private findDatabase(name: string) { - return find(this.properties().databases, row => row.name === name); - } -} - -describe("Databases Page", function () { - let driver: TestDriver; - - beforeEach(function () { - driver = new TestDriver(createAdminUIStore(createMemoryHistory())); - }); - - afterEach(function () { - fakeApi.restore(); - }); - - it("starts in a pre-loading state", async function () { - fakeApi.stubClusterSettings({ - key_values: { - "sql.stats.automatic_collection.enabled": { value: "true" }, - version: { value: "1000023.1-8" }, - }, - }); - - await driver.refreshSettings(); - - driver.assertProperties({ - loading: false, - loaded: false, - requestError: undefined, - queryError: undefined, - databases: [], - search: null, - filters: defaultFilters, - nodeRegions: {}, - isTenant: false, - sortSetting: { ascending: true, columnTitle: "name" }, - automaticStatsCollectionEnabled: true, - indexRecommendationsEnabled: true, - showNodeRegionsColumn: false, - csIndexUnusedDuration: indexUnusedDuration, - }); - }); - - it("makes a row for each database", async function () { - // Mock out the fetch query to /databases - fakeApi.stubSqlApiCall( - clusterUiApi.databasesRequest, - [ - { - rows: [ - { - database_name: "system", - }, - { - database_name: "test", - }, - ], - }, - ], - ); - fakeApi.stubClusterSettings({ - key_values: { - "sql.stats.automatic_collection.enabled": { value: "true" }, - version: { value: "1000023.1-8" }, - }, - }); - - await driver.refreshDatabases(); - await driver.refreshSettings(); - - driver.assertProperties({ - loading: false, - loaded: true, - requestError: null, - queryError: undefined, - databases: [ - { - detailsLoading: false, - detailsLoaded: false, - spanStatsLoading: false, - spanStatsLoaded: false, - detailsRequestError: undefined, - spanStatsRequestError: undefined, - detailsQueryError: undefined, - spanStatsQueryError: undefined, - name: "system", - nodes: [], - spanStats: undefined, - tables: undefined, - nodesByRegionString: "", - numIndexRecommendations: 0, - }, - { - detailsLoading: false, - detailsLoaded: false, - spanStatsLoading: false, - spanStatsLoaded: false, - detailsRequestError: undefined, - spanStatsRequestError: undefined, - detailsQueryError: undefined, - spanStatsQueryError: undefined, - name: "test", - nodes: [], - spanStats: undefined, - tables: undefined, - nodesByRegionString: "", - numIndexRecommendations: 0, - }, - ], - search: null, - filters: defaultFilters, - nodeRegions: {}, - isTenant: false, - sortSetting: { ascending: true, columnTitle: "name" }, - showNodeRegionsColumn: false, - indexRecommendationsEnabled: true, - csIndexUnusedDuration: indexUnusedDuration, - automaticStatsCollectionEnabled: true, - }); - }); - - it("fills in database details and node/region info", async function () { - const oldDate = new Date(2020, 12, 25, 0, 0, 0, 0); - const regions = [ - "gcp-us-east1", - "gcp-us-east1", - "gcp-europe-west1", - "gcp-us-east1", - "gcp-europe-west2", - "gcp-europe-west1", - ]; - - const nodes: cockroach.server.serverpb.INodeResponse[] = Array.from( - Array(regions.length).keys(), - ).map(node_id => { - return { - desc: { - node_id: node_id + 1, // 1-index offset. - locality: { - tiers: [ - { - key: "region", - value: regions[node_id], - }, - ], - }, - }, - store_statuses: [{ desc: { store_id: node_id + 1 } }], - }; - }); - - fakeApi.stubNodesUI({ - nodes: nodes, - }); - - // Mock out the fetch query to /databases - fakeApi.stubSqlApiCall( - clusterUiApi.databasesRequest, - [ - { - rows: [ - { - database_name: "test", - }, - ], - }, - ], - ); - - fakeApi.stubSqlApiCall( - clusterUiApi.createDatabaseDetailsReq({ - database: "test", - csIndexUnusedDuration: indexUnusedDuration, - }), - [ - // Id - { rows: [] }, - // Grants - { rows: [] }, - // Tables - { - rows: [ - { table_schema: "public", table_name: "foo" }, - { table_schema: "public", table_name: "bar" }, - ], - }, - // Regions and replicas - { - rows: [ - { - store_ids: [1, 2, 3], - }, - { - store_ids: [1, 2, 3], - }, - { - store_ids: [1, 2, 3], - }, - ], - }, - // Index Usage Stats - { - rows: [ - // Generate drop index recommendation - { - last_read: oldDate.toISOString(), - created_at: oldDate.toISOString(), - unused_threshold: "1s", - }, - ], - }, - // Zone Config - { - rows: [], - }, - ], - ); - - await driver.refreshNodes(); - await driver.refreshDatabases(); - await driver.refreshDatabaseDetails("test", indexUnusedDuration); - - driver.assertDatabaseProperties("test", { - detailsLoading: false, - detailsLoaded: true, - spanStatsLoading: false, - spanStatsLoaded: false, - detailsRequestError: null, - spanStatsRequestError: undefined, - detailsQueryError: undefined, - spanStatsQueryError: undefined, - name: "test", - nodes: [1, 2, 3], - spanStats: undefined, - tables: { - tables: [ - { - schema: "public", - table: "foo", - qualifiedNameWithSchemaAndTable: `"public"."foo"`, - }, - { - schema: "public", - table: "bar", - qualifiedNameWithSchemaAndTable: `"public"."bar"`, - }, - ], - }, - nodesByRegionString: "gcp-europe-west1(n3), gcp-us-east1(n1,n2)", - numIndexRecommendations: 1, - }); - }); -}); diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/redux.ts b/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/redux.ts deleted file mode 100644 index d17354c852f3..000000000000 --- a/pkg/ui/workspaces/db-console/src/views/databases/databasesPage/redux.ts +++ /dev/null @@ -1,124 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the CockroachDB Software License -// included in the /LICENSE file. - -import { - DatabasesPageData, - defaultFilters, - Filters, - deriveDatabaseDetailsMemoized, -} from "@cockroachlabs/cluster-ui"; -import { createSelector } from "reselect"; - -import { - refreshDatabases, - refreshDatabaseDetails, - refreshNodes, - refreshSettings, - refreshDatabaseDetailsSpanStats, -} from "src/redux/apiReducers"; -import { - selectAutomaticStatsCollectionEnabled, - selectDropUnusedIndexDuration, - selectIndexRecommendationsEnabled, -} from "src/redux/clusterSettings"; -import { LocalSetting } from "src/redux/localsettings"; -import { - nodeRegionsByIDSelector, - selectIsMoreThanOneNode, -} from "src/redux/nodes"; -import { AdminUIState } from "src/redux/state"; - -const selectLoading = createSelector( - (state: AdminUIState) => state.cachedData.databases, - databases => databases.inFlight, -); - -const selectLoaded = createSelector( - (state: AdminUIState) => state.cachedData.databases, - databases => databases.valid, -); - -const selectLastError = createSelector( - (state: AdminUIState) => state.cachedData.databases, - databases => databases.lastError, -); - -// Hardcoded isTenant value for db-console. -const isTenant = false; - -const sortSettingLocalSetting = new LocalSetting( - "sortSetting/DatabasesPage", - (state: AdminUIState) => state.localSettings, - { ascending: true, columnTitle: "name" }, -); - -const filtersLocalSetting = new LocalSetting( - "filters/DatabasesPage", - (state: AdminUIState) => state.localSettings, - defaultFilters, -); - -const searchLocalSetting = new LocalSetting( - "search/DatabasesPage", - (state: AdminUIState) => state.localSettings, - null, -); - -export const mapStateToProps = (state: AdminUIState): DatabasesPageData => { - const dbListResp = state?.cachedData.databases.data; - const databaseDetails = state?.cachedData.databaseDetails; - const spanStats = state?.cachedData.databaseDetailsSpanStats; - const nodeRegions = nodeRegionsByIDSelector(state); - return { - loading: selectLoading(state), - loaded: selectLoaded(state), - requestError: selectLastError(state), - queryError: dbListResp?.error, - databases: deriveDatabaseDetailsMemoized({ - dbListResp, - databaseDetails, - spanStats, - nodeRegions, - isTenant, - nodeStatuses: state?.cachedData.nodes.data, - }), - sortSetting: sortSettingLocalSetting.selector(state), - filters: filtersLocalSetting.selector(state), - search: searchLocalSetting.selector(state), - nodeRegions, - isTenant, - automaticStatsCollectionEnabled: - selectAutomaticStatsCollectionEnabled(state), - indexRecommendationsEnabled: selectIndexRecommendationsEnabled(state), - showNodeRegionsColumn: selectIsMoreThanOneNode(state), - csIndexUnusedDuration: selectDropUnusedIndexDuration(state), - }; -}; - -export const mapDispatchToProps = { - refreshSettings, - refreshDatabases, - refreshDatabaseDetails: (database: string, csIndexUnusedDuration: string) => { - return refreshDatabaseDetails({ - database, - csIndexUnusedDuration, - }); - }, - refreshDatabaseSpanStats: (database: string) => { - return refreshDatabaseDetailsSpanStats({ database }); - }, - refreshNodes, - onSortingChange: ( - _tableName: string, - columnName: string, - ascending: boolean, - ) => - sortSettingLocalSetting.set({ - ascending: ascending, - columnTitle: columnName, - }), - onSearchComplete: (query: string) => searchLocalSetting.set(query), - onFilterChange: (filters: Filters) => filtersLocalSetting.set(filters), -}; diff --git a/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx b/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx index f46d29389d07..d60934c9cc07 100644 --- a/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx @@ -596,12 +596,6 @@ export default function Debug() { note="_status/hotranges?node_id=[node_id]" /> - - -